You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by tr...@apache.org on 2015/08/31 12:31:37 UTC

[01/10] flink git commit: [FLINK-2291] [runtime] Add ZooKeeper support to elect a leader from a set of JobManager. The leader will then be retrieved from ZooKeeper by the TaskManagers.

Repository: flink
Updated Branches:
  refs/heads/master 0858d9f12 -> b9de4ed37


http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-tests/pom.xml
----------------------------------------------------------------------
diff --git a/flink-tests/pom.xml b/flink-tests/pom.xml
index f410827..1f5bfda 100644
--- a/flink-tests/pom.xml
+++ b/flink-tests/pom.xml
@@ -163,7 +163,6 @@ under the License.
 			<version>${guava.version}</version>
 			<scope>test</scope>
 		</dependency>
-		
 	</dependencies>
 
 	<build>

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorLiveITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorLiveITCase.java b/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorLiveITCase.java
index ccaa486..922fc43 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorLiveITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorLiveITCase.java
@@ -43,6 +43,7 @@ import org.apache.flink.optimizer.plan.OptimizedPlan;
 import org.apache.flink.optimizer.plantranslate.JobGraphGenerator;
 import org.apache.flink.runtime.accumulators.AccumulatorRegistry;
 import org.apache.flink.runtime.akka.AkkaUtils;
+import org.apache.flink.runtime.akka.ListeningBehaviour;
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
 import org.apache.flink.runtime.instance.ActorGateway;
 import org.apache.flink.runtime.instance.AkkaActorGateway;
@@ -119,11 +120,12 @@ public class AccumulatorLiveITCase {
 
 		Configuration config = new Configuration();
 		config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 1);
-		config.setInteger(ConfigConstants.LOCAL_INSTANCE_MANAGER_NUMBER_TASK_MANAGER, 1);
+		config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, 1);
 		config.setString(ConfigConstants.AKKA_ASK_TIMEOUT, TestingUtils.DEFAULT_AKKA_ASK_TIMEOUT());
 		TestingCluster testingCluster = new TestingCluster(config, false, true);
+		testingCluster.start();
 
-		jobManagerGateway = testingCluster.getJobManagerGateway();
+		jobManagerGateway = testingCluster.getLeaderGateway(TestingUtils.TESTING_DURATION());
 		taskManager = testingCluster.getTaskManagersAsJava().get(0);
 
 		// generate test data
@@ -192,7 +194,11 @@ public class AccumulatorLiveITCase {
 
 			// submit job
 
-			jobManagerGateway.tell(new JobManagerMessages.SubmitJob(jobGraph, false), selfGateway);
+			jobManagerGateway.tell(
+					new JobManagerMessages.SubmitJob(
+							jobGraph,
+							ListeningBehaviour.EXECUTION_RESULT),
+					selfGateway);
 			expectMsgClass(TIMEOUT, JobManagerMessages.JobSubmitSuccess.class);
 
 

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-tests/src/test/java/org/apache/flink/test/cancelling/CancellingTestBase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/cancelling/CancellingTestBase.java b/flink-tests/src/test/java/org/apache/flink/test/cancelling/CancellingTestBase.java
index 088ea4d..1a96a1b 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/cancelling/CancellingTestBase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/cancelling/CancellingTestBase.java
@@ -25,6 +25,7 @@ import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.client.JobCancellationException;
 import org.apache.flink.runtime.messages.JobManagerMessages;
+import org.apache.flink.runtime.testingUtils.TestingUtils;
 import org.apache.flink.test.util.ForkableFlinkMiniCluster;
 import org.apache.flink.util.TestLogger;
 import org.junit.Assert;
@@ -115,7 +116,8 @@ public abstract class CancellingTestBase extends TestLogger {
 				public void run() {
 					try {
 						Thread.sleep(msecsTillCanceling);
-						executor.getJobManagerGateway().tell(new JobManagerMessages.CancelJob(jobGraph.getJobID()));
+						executor.getLeaderGateway(TestingUtils.TESTING_DURATION())
+							.tell(new JobManagerMessages.CancelJob(jobGraph.getJobID()));
 					}
 					catch (Throwable t) {
 						error.set(t);
@@ -146,7 +148,7 @@ public abstract class CancellingTestBase extends TestLogger {
 	}
 
 	private JobGraph getJobGraph(final Plan plan) throws Exception {
-		final Optimizer pc = new Optimizer(new DataStatistics(), this.executor.getConfiguration());
+		final Optimizer pc = new Optimizer(new DataStatistics(), this.executor.configuration());
 		final OptimizedPlan op = pc.compile(plan);
 		final JobGraphGenerator jgg = new JobGraphGenerator();
 		return jgg.compileJobGraph(op);

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamFaultToleranceTestBase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamFaultToleranceTestBase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamFaultToleranceTestBase.java
index 59f24b0..20e19a5 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamFaultToleranceTestBase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamFaultToleranceTestBase.java
@@ -47,12 +47,14 @@ public abstract class StreamFaultToleranceTestBase extends TestLogger {
 	public static void startCluster() {
 		try {
 			Configuration config = new Configuration();
-			config.setInteger(ConfigConstants.LOCAL_INSTANCE_MANAGER_NUMBER_TASK_MANAGER, NUM_TASK_MANAGERS);
+			config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, NUM_TASK_MANAGERS);
 			config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, NUM_TASK_SLOTS);
 			config.setString(ConfigConstants.DEFAULT_EXECUTION_RETRY_DELAY_KEY, "0 ms");
 			config.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 12);
 			
 			cluster = new ForkableFlinkMiniCluster(config, false);
+
+			cluster.start();
 		}
 		catch (Exception e) {
 			e.printStackTrace();
@@ -61,9 +63,9 @@ public abstract class StreamFaultToleranceTestBase extends TestLogger {
 	}
 
 	@AfterClass
-	public static void shutdownCluster() {
+	public static void stopCluster() {
 		try {
-			cluster.shutdown();
+			cluster.stop();
 			cluster = null;
 		}
 		catch (Exception e) {
@@ -91,7 +93,7 @@ public abstract class StreamFaultToleranceTestBase extends TestLogger {
 	public void runCheckpointedProgram() {
 		try {
 			StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment(
-					"localhost", cluster.getJobManagerRPCPort());
+					"localhost", cluster.getLeaderRPCPort());
 			env.setParallelism(PARALLELISM);
 			env.enableCheckpointing(500);
 			env.getConfig().disableSysoutLogging();

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-tests/src/test/java/org/apache/flink/test/classloading/ClassLoaderITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/classloading/ClassLoaderITCase.java b/flink-tests/src/test/java/org/apache/flink/test/classloading/ClassLoaderITCase.java
index 5c2f2dc..4480d95 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/classloading/ClassLoaderITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/classloading/ClassLoaderITCase.java
@@ -50,7 +50,7 @@ public class ClassLoaderITCase {
 	public void testJobsWithCustomClassLoader() {
 		try {
 			Configuration config = new Configuration();
-			config.setInteger(ConfigConstants.LOCAL_INSTANCE_MANAGER_NUMBER_TASK_MANAGER, 2);
+			config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, 2);
 			config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 2);
 			config.setString(ConfigConstants.DEFAULT_EXECUTION_RETRY_DELAY_KEY, "0 s");
 
@@ -60,8 +60,10 @@ public class ClassLoaderITCase {
 
 			ForkableFlinkMiniCluster testCluster = new ForkableFlinkMiniCluster(config, false);
 
+			testCluster.start();
+
 			try {
-				int port = testCluster.getJobManagerRPCPort();
+				int port = testCluster.getLeaderRPCPort();
 
 				PackagedProgram inputSplitTestProg = new PackagedProgram(
 						new File(INPUT_SPLITS_PROG_JAR_FILE),

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-tests/src/test/java/org/apache/flink/test/failingPrograms/JobSubmissionFailsITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/failingPrograms/JobSubmissionFailsITCase.java b/flink-tests/src/test/java/org/apache/flink/test/failingPrograms/JobSubmissionFailsITCase.java
index 2bde833..28c2e58 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/failingPrograms/JobSubmissionFailsITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/failingPrograms/JobSubmissionFailsITCase.java
@@ -55,10 +55,12 @@ public class JobSubmissionFailsITCase {
 		try {
 			Configuration config = new Configuration();
 			config.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 4);
-			config.setInteger(ConfigConstants.LOCAL_INSTANCE_MANAGER_NUMBER_TASK_MANAGER, 2);
+			config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, 2);
 			config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, NUM_SLOTS / 2);
 			
 			cluster = new ForkableFlinkMiniCluster(config);
+
+			cluster.start();
 			
 			final JobVertex jobVertex = new JobVertex("Working job vertex.");
 			jobVertex.setInvokableClass(Tasks.NoOpInvokable.class);

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/DistinctITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/DistinctITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/DistinctITCase.java
index d32986d..6e30fac 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/DistinctITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/DistinctITCase.java
@@ -18,11 +18,8 @@
 
 package org.apache.flink.test.javaApiOperators;
 
-import java.util.ArrayList;
 import java.util.List;
 
-import org.apache.avro.generic.GenericData;
-import org.apache.flink.api.common.InvalidProgramException;
 import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.ExecutionEnvironment;
 import org.apache.flink.api.java.functions.KeySelector;

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-tests/src/test/java/org/apache/flink/test/manual/NotSoMiniClusterIterations.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/manual/NotSoMiniClusterIterations.java b/flink-tests/src/test/java/org/apache/flink/test/manual/NotSoMiniClusterIterations.java
index d340f57..f8b0d96 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/manual/NotSoMiniClusterIterations.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/manual/NotSoMiniClusterIterations.java
@@ -49,7 +49,7 @@ public class NotSoMiniClusterIterations {
 
 		try {
 			Configuration config = new Configuration();
-			config.setInteger(ConfigConstants.LOCAL_INSTANCE_MANAGER_NUMBER_TASK_MANAGER, PARALLELISM);
+			config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, PARALLELISM);
 			config.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 8);
 			config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 1);
 			config.setInteger(ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY, 1000);
@@ -60,7 +60,7 @@ public class NotSoMiniClusterIterations {
 
 			cluster = new LocalFlinkMiniCluster(config, false);
 
-			runConnectedComponents(cluster.getJobManagerRPCPort());
+			runConnectedComponents(cluster.getLeaderRPCPort());
 		}
 		catch (Exception e) {
 			e.printStackTrace();

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-tests/src/test/java/org/apache/flink/test/manual/StreamingScalabilityAndLatency.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/manual/StreamingScalabilityAndLatency.java b/flink-tests/src/test/java/org/apache/flink/test/manual/StreamingScalabilityAndLatency.java
index 8d41292..b01a524 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/manual/StreamingScalabilityAndLatency.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/manual/StreamingScalabilityAndLatency.java
@@ -46,7 +46,7 @@ public class StreamingScalabilityAndLatency {
 
 		try {
 			Configuration config = new Configuration();
-			config.setInteger(ConfigConstants.LOCAL_INSTANCE_MANAGER_NUMBER_TASK_MANAGER, TASK_MANAGERS);
+			config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, TASK_MANAGERS);
 			config.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 80);
 			config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, SLOTS_PER_TASK_MANAGER);
 			config.setInteger(ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY, 20000);
@@ -56,7 +56,7 @@ public class StreamingScalabilityAndLatency {
 
 			cluster = new LocalFlinkMiniCluster(config, false, StreamingMode.STREAMING);
 			
-			runPartitioningProgram(cluster.getJobManagerRPCPort(), PARALLELISM);
+			runPartitioningProgram(cluster.getLeaderRPCPort(), PARALLELISM);
 		}
 		catch (Exception e) {
 			e.printStackTrace();

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-tests/src/test/java/org/apache/flink/test/misc/AutoParallelismITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/misc/AutoParallelismITCase.java b/flink-tests/src/test/java/org/apache/flink/test/misc/AutoParallelismITCase.java
index df190d7..7dccb7d 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/misc/AutoParallelismITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/misc/AutoParallelismITCase.java
@@ -54,9 +54,11 @@ public class AutoParallelismITCase {
 	@BeforeClass
 	public static void setupCluster() {
 		Configuration config = new Configuration();
-		config.setInteger(ConfigConstants.LOCAL_INSTANCE_MANAGER_NUMBER_TASK_MANAGER, NUM_TM);
+		config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, NUM_TM);
 		config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, SLOTS_PER_TM);
 		cluster = new ForkableFlinkMiniCluster(config, false);
+
+		cluster.start();
 	}
 
 	@AfterClass
@@ -76,7 +78,7 @@ public class AutoParallelismITCase {
 	public void testProgramWithAutoParallelism() {
 		try {
 			ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment(
-					"localhost", cluster.getJobManagerRPCPort());
+					"localhost", cluster.getLeaderRPCPort());
 
 			env.setParallelism(ExecutionConfig.PARALLELISM_AUTO_MAX);
 			env.getConfig().disableSysoutLogging();

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-tests/src/test/java/org/apache/flink/test/misc/CustomSerializationITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/misc/CustomSerializationITCase.java b/flink-tests/src/test/java/org/apache/flink/test/misc/CustomSerializationITCase.java
index 4e7da83..f30f61f 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/misc/CustomSerializationITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/misc/CustomSerializationITCase.java
@@ -52,6 +52,7 @@ public class CustomSerializationITCase {
 			config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, PARLLELISM);
 			config.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 30);
 			cluster = new ForkableFlinkMiniCluster(config, false);
+			cluster.start();
 		}
 		catch (Exception e) {
 			e.printStackTrace();
@@ -75,7 +76,7 @@ public class CustomSerializationITCase {
 	public void testIncorrectSerializer1() {
 		try {
 			ExecutionEnvironment env =
-					ExecutionEnvironment.createRemoteEnvironment("localhost", cluster.getJobManagerRPCPort());
+					ExecutionEnvironment.createRemoteEnvironment("localhost", cluster.getLeaderRPCPort());
 			
 			env.setParallelism(PARLLELISM);
 			env.getConfig().disableSysoutLogging();
@@ -108,7 +109,7 @@ public class CustomSerializationITCase {
 	public void testIncorrectSerializer2() {
 		try {
 			ExecutionEnvironment env =
-					ExecutionEnvironment.createRemoteEnvironment("localhost", cluster.getJobManagerRPCPort());
+					ExecutionEnvironment.createRemoteEnvironment("localhost", cluster.getLeaderRPCPort());
 
 			env.setParallelism(PARLLELISM);
 			env.getConfig().disableSysoutLogging();
@@ -141,7 +142,7 @@ public class CustomSerializationITCase {
 	public void testIncorrectSerializer3() {
 		try {
 			ExecutionEnvironment env =
-					ExecutionEnvironment.createRemoteEnvironment("localhost", cluster.getJobManagerRPCPort());
+					ExecutionEnvironment.createRemoteEnvironment("localhost", cluster.getLeaderRPCPort());
 
 			env.setParallelism(PARLLELISM);
 			env.getConfig().disableSysoutLogging();
@@ -174,7 +175,7 @@ public class CustomSerializationITCase {
 	public void testIncorrectSerializer4() {
 		try {
 			ExecutionEnvironment env =
-					ExecutionEnvironment.createRemoteEnvironment("localhost", cluster.getJobManagerRPCPort());
+					ExecutionEnvironment.createRemoteEnvironment("localhost", cluster.getLeaderRPCPort());
 
 			env.setParallelism(PARLLELISM);
 			env.getConfig().disableSysoutLogging();

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-tests/src/test/java/org/apache/flink/test/misc/MiscellaneousIssuesITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/misc/MiscellaneousIssuesITCase.java b/flink-tests/src/test/java/org/apache/flink/test/misc/MiscellaneousIssuesITCase.java
index 01e6f62..42419fb 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/misc/MiscellaneousIssuesITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/misc/MiscellaneousIssuesITCase.java
@@ -58,10 +58,12 @@ public class MiscellaneousIssuesITCase {
 	public static void startCluster() {
 		try {
 			Configuration config = new Configuration();
-			config.setInteger(ConfigConstants.LOCAL_INSTANCE_MANAGER_NUMBER_TASK_MANAGER, 2);
+			config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, 2);
 			config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 3);
 			config.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 12);
 			cluster = new ForkableFlinkMiniCluster(config, false);
+
+			cluster.start();
 		}
 		catch (Exception e) {
 			e.printStackTrace();
@@ -85,7 +87,7 @@ public class MiscellaneousIssuesITCase {
 	public void testNullValues() {
 		try {
 			ExecutionEnvironment env =
-					ExecutionEnvironment.createRemoteEnvironment("localhost", cluster.getJobManagerRPCPort());
+					ExecutionEnvironment.createRemoteEnvironment("localhost", cluster.getLeaderRPCPort());
 
 			env.setParallelism(1);
 			env.getConfig().disableSysoutLogging();
@@ -119,7 +121,7 @@ public class MiscellaneousIssuesITCase {
 	public void testDisjointDataflows() {
 		try {
 			ExecutionEnvironment env =
-					ExecutionEnvironment.createRemoteEnvironment("localhost", cluster.getJobManagerRPCPort());
+					ExecutionEnvironment.createRemoteEnvironment("localhost", cluster.getLeaderRPCPort());
 
 			env.setParallelism(5);
 			env.getConfig().disableSysoutLogging();
@@ -142,7 +144,7 @@ public class MiscellaneousIssuesITCase {
 		
 		try {
 			ExecutionEnvironment env =
-					ExecutionEnvironment.createRemoteEnvironment("localhost", cluster.getJobManagerRPCPort());
+					ExecutionEnvironment.createRemoteEnvironment("localhost", cluster.getLeaderRPCPort());
 
 			env.setParallelism(6);
 			env.getConfig().disableSysoutLogging();

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-tests/src/test/java/org/apache/flink/test/misc/SuccessAfterNetworkBuffersFailureITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/misc/SuccessAfterNetworkBuffersFailureITCase.java b/flink-tests/src/test/java/org/apache/flink/test/misc/SuccessAfterNetworkBuffersFailureITCase.java
index 6cb76e6..12b7a68 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/misc/SuccessAfterNetworkBuffersFailureITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/misc/SuccessAfterNetworkBuffersFailureITCase.java
@@ -47,15 +47,17 @@ public class SuccessAfterNetworkBuffersFailureITCase {
 		
 		try {
 			Configuration config = new Configuration();
-			config.setInteger(ConfigConstants.LOCAL_INSTANCE_MANAGER_NUMBER_TASK_MANAGER, 2);
+			config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, 2);
 			config.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 80);
 			config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 8);
 			config.setInteger(ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY, 840);
 			
 			cluster = new ForkableFlinkMiniCluster(config, false);
+
+			cluster.start();
 			
 			try {
-				runConnectedComponents(cluster.getJobManagerRPCPort());
+				runConnectedComponents(cluster.getLeaderRPCPort());
 			}
 			catch (Exception e) {
 				e.printStackTrace();
@@ -63,7 +65,7 @@ public class SuccessAfterNetworkBuffersFailureITCase {
 			}
 	
 			try {
-				runKMeans(cluster.getJobManagerRPCPort());
+				runKMeans(cluster.getLeaderRPCPort());
 				fail("This program execution should have failed.");
 			}
 			catch (ProgramInvocationException e) {
@@ -71,7 +73,7 @@ public class SuccessAfterNetworkBuffersFailureITCase {
 			}
 	
 			try {
-				runConnectedComponents(cluster.getJobManagerRPCPort());
+				runConnectedComponents(cluster.getLeaderRPCPort());
 			}
 			catch (Exception e) {
 				e.printStackTrace();

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-tests/src/test/java/org/apache/flink/test/recovery/SimpleRecoveryITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/SimpleRecoveryITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/SimpleRecoveryITCase.java
index e2f5a71..aa721e9 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/recovery/SimpleRecoveryITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/SimpleRecoveryITCase.java
@@ -49,11 +49,13 @@ public class SimpleRecoveryITCase {
 	@BeforeClass
 	public static void setupCluster() {
 		Configuration config = new Configuration();
-		config.setInteger(ConfigConstants.LOCAL_INSTANCE_MANAGER_NUMBER_TASK_MANAGER, 2);
+		config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, 2);
 		config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 2);
 		config.setString(ConfigConstants.DEFAULT_EXECUTION_RETRY_DELAY_KEY, "100 ms");
 
 		cluster = new ForkableFlinkMiniCluster(config, false);
+
+		cluster.start();
 	}
 
 	@AfterClass
@@ -77,7 +79,7 @@ public class SimpleRecoveryITCase {
 			// attempt 1
 			{
 				ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment(
-						"localhost", cluster.getJobManagerRPCPort());
+						"localhost", cluster.getLeaderRPCPort());
 
 				env.setParallelism(4);
 				env.setNumberOfExecutionRetries(0);
@@ -107,7 +109,7 @@ public class SimpleRecoveryITCase {
 			// attempt 2
 			{
 				ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment(
-						"localhost", cluster.getJobManagerRPCPort());
+						"localhost", cluster.getLeaderRPCPort());
 
 				env.setParallelism(4);
 				env.setNumberOfExecutionRetries(0);
@@ -154,7 +156,7 @@ public class SimpleRecoveryITCase {
 			List<Long> resultCollection = new ArrayList<Long>();
 
 			ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment(
-					"localhost", cluster.getJobManagerRPCPort());
+					"localhost", cluster.getLeaderRPCPort());
 
 			env.setParallelism(4);
 			env.setNumberOfExecutionRetries(1);
@@ -199,7 +201,7 @@ public class SimpleRecoveryITCase {
 			List<Long> resultCollection = new ArrayList<Long>();
 
 			ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment(
-					"localhost", cluster.getJobManagerRPCPort());
+					"localhost", cluster.getLeaderRPCPort());
 
 			env.setParallelism(4);
 			env.setNumberOfExecutionRetries(5);

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerFailureRecoveryITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerFailureRecoveryITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerFailureRecoveryITCase.java
index 877893f..520a0f2 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerFailureRecoveryITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerFailureRecoveryITCase.java
@@ -69,7 +69,7 @@ public class TaskManagerFailureRecoveryITCase {
 
 		try {
 			Configuration config = new Configuration();
-			config.setInteger(ConfigConstants.LOCAL_INSTANCE_MANAGER_NUMBER_TASK_MANAGER, 2);
+			config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, 2);
 			config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, PARALLELISM);
 			config.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 16);
 			
@@ -79,11 +79,13 @@ public class TaskManagerFailureRecoveryITCase {
 
 			cluster = new ForkableFlinkMiniCluster(config, false);
 
+			cluster.start();
+
 			// for the result
 			List<Long> resultCollection = new ArrayList<Long>();
 
 			final ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment(
-					"localhost", cluster.getJobManagerRPCPort());
+					"localhost", cluster.getLeaderRPCPort());
 
 			env.setParallelism(PARALLELISM);
 			env.setNumberOfExecutionRetries(1);

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-tests/src/test/java/org/apache/flink/test/runtime/leaderelection/ZooKeeperLeaderElectionITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/runtime/leaderelection/ZooKeeperLeaderElectionITCase.java b/flink-tests/src/test/java/org/apache/flink/test/runtime/leaderelection/ZooKeeperLeaderElectionITCase.java
new file mode 100644
index 0000000..6035c45
--- /dev/null
+++ b/flink-tests/src/test/java/org/apache/flink/test/runtime/leaderelection/ZooKeeperLeaderElectionITCase.java
@@ -0,0 +1,255 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.test.runtime.leaderelection;
+
+import akka.actor.ActorSystem;
+import akka.actor.Kill;
+import akka.actor.PoisonPill;
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.client.JobClient;
+import org.apache.flink.runtime.client.JobExecutionException;
+import org.apache.flink.runtime.instance.ActorGateway;
+import org.apache.flink.runtime.jobgraph.DistributionPattern;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.jobmanager.Tasks;
+import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalException;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+import org.apache.flink.runtime.messages.JobManagerMessages;
+import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages.WaitForAllVerticesToBeRunningOrFinished;
+import org.apache.flink.runtime.testingUtils.TestingUtils;
+import org.apache.flink.runtime.util.LeaderRetrievalUtils;
+import org.apache.flink.test.util.ForkableFlinkMiniCluster;
+import org.apache.flink.util.TestLogger;
+import org.junit.Test;
+import scala.concurrent.Await;
+import scala.concurrent.Future;
+import scala.concurrent.duration.FiniteDuration;
+
+import static org.junit.Assert.*;
+
+public class ZooKeeperLeaderElectionITCase extends TestLogger {
+
+	private static final FiniteDuration timeout = TestingUtils.TESTING_DURATION();
+
+	/**
+	 * Tests that the TaskManagers successfully register at the new leader once the old leader
+	 * is terminated.
+	 */
+	@Test
+	public void testTaskManagerRegistrationAtReelectedLeader() throws Exception {
+		Configuration configuration = new Configuration();
+
+		int numJMs = 10;
+		int numTMs = 3;
+
+		configuration.setString(ConfigConstants.RECOVERY_MODE, "zookeeper");
+		configuration.setInteger(ConfigConstants.LOCAL_NUMBER_JOB_MANAGER, numJMs);
+		configuration.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, numTMs);
+
+		ForkableFlinkMiniCluster cluster = new ForkableFlinkMiniCluster(configuration);
+
+		try {
+			cluster.start();
+
+			for(int i = 0; i < numJMs; i++) {
+				ActorGateway leadingJM = cluster.getLeaderGateway(timeout);
+
+				cluster.waitForTaskManagersToBeRegisteredAtJobManager(leadingJM.actor());
+
+				Future<Object> registeredTMs = leadingJM.ask(
+						JobManagerMessages.getRequestNumberRegisteredTaskManager(),
+						timeout);
+
+				int numRegisteredTMs = (Integer) Await.result(registeredTMs, timeout);
+
+				assertEquals(numTMs, numRegisteredTMs);
+
+				cluster.clearLeader();
+				leadingJM.tell(PoisonPill.getInstance());
+			}
+		} finally {
+			cluster.stop();
+		}
+	}
+
+	/**
+	 * Tests that a job can be executed after a new leader has been elected. For all except for the
+	 * last leader, the job is blocking. The JobManager will be terminated while executing the
+	 * blocking job. Once only one JobManager is left, it is checked that a non-blocking can be
+	 * successfully executed.
+	 */
+	@Test
+	public void testJobExecutionOnClusterWithLeaderReelection() throws Exception {
+		int numJMs = 10;
+		int numTMs = 3;
+		int numSlotsPerTM = 3;
+		int parallelism = numTMs * numSlotsPerTM;
+
+		Configuration configuration = new Configuration();
+
+		configuration.setString(ConfigConstants.RECOVERY_MODE, "zookeeper");
+		configuration.setInteger(ConfigConstants.LOCAL_NUMBER_JOB_MANAGER, numJMs);
+		configuration.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, numTMs);
+		configuration.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, numSlotsPerTM);
+
+		Tasks.BlockingOnceReceiver$.MODULE$.blocking_$eq(true);
+
+		JobVertex sender = new JobVertex("sender");
+		JobVertex receiver = new JobVertex("receiver");
+
+		sender.setInvokableClass(Tasks.Sender.class);
+		receiver.setInvokableClass(Tasks.BlockingOnceReceiver.class);
+
+		sender.setParallelism(parallelism);
+		receiver.setParallelism(parallelism);
+
+		receiver.connectNewDataSetAsInput(sender, DistributionPattern.POINTWISE);
+
+		SlotSharingGroup slotSharingGroup = new SlotSharingGroup();
+		sender.setSlotSharingGroup(slotSharingGroup);
+		receiver.setSlotSharingGroup(slotSharingGroup);
+
+		final JobGraph graph = new JobGraph("Blocking test job", sender, receiver);
+
+		final ForkableFlinkMiniCluster cluster = new ForkableFlinkMiniCluster(configuration);
+
+		ActorSystem clientActorSystem = null;
+
+		Thread thread = null;
+
+		JobSubmitterRunnable jobSubmission = null;
+
+		try {
+			cluster.start();
+
+			clientActorSystem = cluster.startJobClientActorSystem(graph.getJobID());
+
+			final ActorSystem clientAS = clientActorSystem;
+
+			jobSubmission = new JobSubmitterRunnable(clientAS, cluster, graph);
+
+			thread = new Thread(jobSubmission);
+
+			thread.start();
+
+			// Kill all JobManager except for two
+			for(int i = 0; i < numJMs - 2; i++) {
+				ActorGateway jm = cluster.getLeaderGateway(timeout);
+
+				cluster.waitForTaskManagersToBeRegisteredAtJobManager(jm.actor());
+
+				Future<Object> future = jm.ask(new WaitForAllVerticesToBeRunningOrFinished(graph.getJobID()), timeout);
+
+				Await.ready(future, timeout);
+
+				cluster.clearLeader();
+
+				jm.tell(Kill.getInstance());
+			}
+
+			ActorGateway jm = cluster.getLeaderGateway(timeout);
+
+			cluster.waitForTaskManagersToBeRegisteredAtJobManager(jm.actor());
+
+			Future<Object> future = jm.ask(new WaitForAllVerticesToBeRunningOrFinished(graph.getJobID()), timeout);
+
+			Await.ready(future, timeout);
+
+			cluster.clearLeader();
+
+			// set the BlockinOnceReceiver for the execution on the last JM to non-blocking, so
+			// that it can succeed
+			Tasks.BlockingOnceReceiver$.MODULE$.blocking_$eq(false);
+
+			jm.tell(PoisonPill.getInstance());
+
+			thread.join(timeout.toMillis());
+
+			if(thread.isAlive()) {
+				jobSubmission.finished = true;
+				fail("The job submission thread did not stop (meaning it did not succeeded in" +
+						"executing the test job.");
+			}
+		} finally {
+			if (clientActorSystem != null) {
+				cluster.shutdownJobClientActorSystem(clientActorSystem);
+			}
+
+			if(thread != null && thread.isAlive() && jobSubmission != null) {
+				jobSubmission.finished = true;
+			}
+			cluster.stop();
+		}
+	}
+
+	public static class JobSubmitterRunnable implements Runnable {
+		boolean finished = false;
+
+		final ActorSystem clientActorSystem;
+		final ForkableFlinkMiniCluster cluster;
+		final JobGraph graph;
+
+		public JobSubmitterRunnable(
+				ActorSystem actorSystem,
+				ForkableFlinkMiniCluster cluster,
+				JobGraph graph) {
+			this.clientActorSystem = actorSystem;
+			this.cluster = cluster;
+			this.graph = graph;
+		}
+
+		@Override
+		public void run() {
+			while(!finished) {
+				try {
+					LeaderRetrievalService lrService =
+							LeaderRetrievalUtils.createLeaderRetrievalService(
+									cluster.configuration());
+
+					ActorGateway jobManagerGateway =
+							LeaderRetrievalUtils.retrieveLeaderGateway(
+									lrService,
+									clientActorSystem,
+									timeout);
+
+					JobClient.submitJobAndWait(
+							clientActorSystem,
+							jobManagerGateway,
+							graph,
+							timeout,
+							false,
+							getClass().getClassLoader());
+
+					finished = true;
+				} catch (JobExecutionException e) {
+					// This was expected, so just try again to submit the job
+				} catch (LeaderRetrievalException e) {
+					// This can also happen, so just try again to submit the job
+				} catch (Exception e) {
+					// This was not expected... fail the test case
+					e.printStackTrace();
+					fail("Caught unexpected exception in job submission test case.");
+				}
+			}
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-tests/src/test/java/org/apache/flink/test/runtime/minicluster/LocalFlinkMiniClusterITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/runtime/minicluster/LocalFlinkMiniClusterITCase.java b/flink-tests/src/test/java/org/apache/flink/test/runtime/minicluster/LocalFlinkMiniClusterITCase.java
index c45cfe8..8e87143 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/runtime/minicluster/LocalFlinkMiniClusterITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/runtime/minicluster/LocalFlinkMiniClusterITCase.java
@@ -31,9 +31,6 @@ import org.apache.flink.runtime.testingUtils.TestingUtils;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
-import scala.Option;
-
-import java.util.UUID;
 
 public class LocalFlinkMiniClusterITCase {
 
@@ -59,14 +56,16 @@ public class LocalFlinkMiniClusterITCase {
 
 		try{
 			Configuration config = new Configuration();
-			config.setInteger(ConfigConstants.LOCAL_INSTANCE_MANAGER_NUMBER_TASK_MANAGER, numTMs);
+			config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, numTMs);
 			config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, numSlots);
 			miniCluster = new LocalFlinkMiniCluster(config, true);
 
-			final ActorGateway jmGateway = miniCluster.getJobManagerGateway();
+			miniCluster.start();
+
+			final ActorGateway jmGateway = miniCluster.getLeaderGateway(TestingUtils.TESTING_DURATION());
 
 			new JavaTestKit(system) {{
-				final ActorGateway selfGateway = new AkkaActorGateway(getRef(), Option.<UUID>empty());
+				final ActorGateway selfGateway = new AkkaActorGateway(getRef(), null);
 
 				new Within(TestingUtils.TESTING_DURATION()) {
 

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-tests/src/test/java/org/apache/flink/test/web/WebFrontendITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/web/WebFrontendITCase.java b/flink-tests/src/test/java/org/apache/flink/test/web/WebFrontendITCase.java
index 31f8560..2ca35ed 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/web/WebFrontendITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/web/WebFrontendITCase.java
@@ -20,19 +20,23 @@ package org.apache.flink.test.web;
 
 import org.apache.commons.io.FileUtils;
 import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.runtime.webmonitor.WebMonitor;
 import org.apache.flink.test.util.MultipleProgramsTestBase;
 import org.apache.flink.test.util.TestBaseUtils;
 import org.codehaus.jettison.json.JSONArray;
 import org.codehaus.jettison.json.JSONObject;
 import org.junit.Assert;
 
+import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
+import scala.concurrent.duration.FiniteDuration;
 
 import java.io.File;
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.concurrent.TimeUnit;
 
 @RunWith(Parameterized.class)
 public class WebFrontendITCase extends MultipleProgramsTestBase {
@@ -41,7 +45,16 @@ public class WebFrontendITCase extends MultipleProgramsTestBase {
 	static {
 		startWebServer = true;
 	}
-	
+
+	private static int port = -1;
+
+	@BeforeClass
+	public static void initialize() {
+		WebMonitor webMonitor = cluster.webMonitor().get();
+		port = webMonitor.getServerPort();
+	}
+
+	static final FiniteDuration timeout = new FiniteDuration(10, TimeUnit.SECONDS);
 
 	public WebFrontendITCase(TestExecutionMode m) {
 		super(m);
@@ -57,7 +70,8 @@ public class WebFrontendITCase extends MultipleProgramsTestBase {
 	@Test
 	public void getNumberOfTaskManagers() {
 		try {
-			Assert.assertEquals("{\"taskmanagers\": "+cluster.getTaskManagers().size()+", \"slots\": 4}", TestBaseUtils.getFromHTTP("http://localhost:8081/jobsInfo?get=taskmanagers"));
+			Assert.assertEquals("{\"taskmanagers\": "+cluster.getTaskManagers().size()+", \"slots\": 4}",
+					TestBaseUtils.getFromHTTP("http://localhost:" + port + "/jobsInfo?get=taskmanagers"));
 		}catch(Throwable e) {
 			e.printStackTrace();
 			Assert.fail(e.getMessage());
@@ -67,7 +81,7 @@ public class WebFrontendITCase extends MultipleProgramsTestBase {
 	@Test
 	public void getTaskmanagers() {
 		try {
-			String json = getFromHTTP("http://localhost:8081/setupInfo?get=taskmanagers");
+			String json = getFromHTTP("http://localhost:" + port + "/setupInfo?get=taskmanagers");
 			JSONObject parsed = new JSONObject(json);
 			Object taskManagers = parsed.get("taskmanagers");
 			Assert.assertNotNull(taskManagers);
@@ -102,7 +116,7 @@ public class WebFrontendITCase extends MultipleProgramsTestBase {
 	@Test
 	public void getConfiguration() {
 		try {
-			String config = getFromHTTP("http://localhost:8081/setupInfo?get=globalC");
+			String config = getFromHTTP("http://localhost:" + port + "/setupInfo?get=globalC");
 			JSONObject parsed = new JSONObject(config);
 			Assert.assertEquals(logDir.toString(), parsed.getString("jobmanager.web.logpath"));
 			Assert.assertEquals(cluster.configuration().getString("taskmanager.numberOfTaskSlots", null), parsed.getString("taskmanager.numberOfTaskSlots"));

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-tests/src/test/resources/log4j-test.properties
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/resources/log4j-test.properties b/flink-tests/src/test/resources/log4j-test.properties
index 85897b3..9d29841 100644
--- a/flink-tests/src/test/resources/log4j-test.properties
+++ b/flink-tests/src/test/resources/log4j-test.properties
@@ -27,4 +27,5 @@ log4j.appender.testlogger.layout=org.apache.log4j.PatternLayout
 log4j.appender.testlogger.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n
 
 # suppress the irrelevant (wrong) warnings from the netty channel handler
-log4j.logger.org.jboss.netty.channel.DefaultChannelPipeline=ERROR, testlogger
\ No newline at end of file
+log4j.logger.org.jboss.netty.channel.DefaultChannelPipeline=ERROR, testlogger
+log4j.logger.org.apache.zookeeper=OFF, testlogger
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/jobmanager/JobManagerFailsITCase.scala
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/jobmanager/JobManagerFailsITCase.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/jobmanager/JobManagerFailsITCase.scala
index 577b5c6..b0de0e8 100644
--- a/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/jobmanager/JobManagerFailsITCase.scala
+++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/jobmanager/JobManagerFailsITCase.scala
@@ -28,7 +28,7 @@ import org.scalatest.junit.JUnitRunner
 import org.scalatest.{BeforeAndAfterAll, Matchers, WordSpecLike}
 
 import org.apache.flink.configuration.{ConfigConstants, Configuration}
-import org.apache.flink.runtime.akka.AkkaUtils
+import org.apache.flink.runtime.akka.{ListeningBehaviour, AkkaUtils}
 import org.apache.flink.runtime.jobgraph.{JobVertex, JobGraph}
 import org.apache.flink.runtime.jobmanager.Tasks.{BlockingNoOpInvokable, NoOpInvokable}
 import org.apache.flink.runtime.messages.JobManagerMessages._
@@ -59,28 +59,31 @@ class JobManagerFailsITCase(_system: ActorSystem)
       val cluster = startDeathwatchCluster(num_slots, 1)
 
       val tm = cluster.getTaskManagers(0)
-      val jmGateway = cluster.getJobManagerGateway()
+      val jmGateway = cluster.getLeaderGateway(TestingUtils.TESTING_DURATION)
 
       // disable disconnect message to test death watch
       tm ! DisableDisconnect
 
-      try{
-        jmGateway.tell(RequestNumberRegisteredTaskManager, self)
-        expectMsg(1)
+      try {
+        within(TestingUtils.TESTING_DURATION) {
+          jmGateway.tell(RequestNumberRegisteredTaskManager, self)
+          expectMsg(1)
 
-        tm ! NotifyWhenJobManagerTerminated(jmGateway.actor)
+          tm ! NotifyWhenJobManagerTerminated(jmGateway.actor)
 
-        jmGateway.tell(PoisonPill, self)
+          jmGateway.tell(PoisonPill, self)
 
-        expectMsgClass(classOf[JobManagerTerminated])
+          expectMsgClass(classOf[JobManagerTerminated])
 
-        cluster.restartJobManager()
+          cluster.restartLeadingJobManager()
 
-        cluster.waitForTaskManagersToBeRegistered()
+          cluster.waitForTaskManagersToBeRegistered()
 
-        cluster.getJobManagerGateway().tell(RequestNumberRegisteredTaskManager, self)
+          cluster.getLeaderGateway(TestingUtils.TESTING_DURATION)
+            .tell(RequestNumberRegisteredTaskManager, self)
 
-        expectMsg(1)
+          expectMsg(1)
+        }
       } finally {
         cluster.stop()
       }
@@ -101,12 +104,12 @@ class JobManagerFailsITCase(_system: ActorSystem)
 
       val cluster = startDeathwatchCluster(num_slots / 2, 2)
 
-      var jmGateway = cluster.getJobManagerGateway()
+      var jmGateway = cluster.getLeaderGateway(TestingUtils.TESTING_DURATION)
       val tm = cluster.getTaskManagers(0)
 
       try {
         within(TestingUtils.TESTING_DURATION) {
-          jmGateway.tell(SubmitJob(jobGraph, false), self)
+          jmGateway.tell(SubmitJob(jobGraph, ListeningBehaviour.DETACHED), self)
           expectMsg(JobSubmitSuccess(jobGraph.getJobID))
 
           tm.tell(NotifyWhenJobManagerTerminated(jmGateway.actor()), self)
@@ -115,13 +118,13 @@ class JobManagerFailsITCase(_system: ActorSystem)
 
           expectMsgClass(classOf[JobManagerTerminated])
 
-          cluster.restartJobManager()
-
-          jmGateway = cluster.getJobManagerGateway()
+          cluster.restartLeadingJobManager()
 
           cluster.waitForTaskManagersToBeRegistered()
 
-          jmGateway.tell(SubmitJob(jobGraph2, false), self)
+          jmGateway = cluster.getLeaderGateway(TestingUtils.TESTING_DURATION)
+
+          jmGateway.tell(SubmitJob(jobGraph2, ListeningBehaviour.EXECUTION_RESULT), self)
 
           expectMsg(JobSubmitSuccess(jobGraph2.getJobID()))
 
@@ -138,8 +141,12 @@ class JobManagerFailsITCase(_system: ActorSystem)
   def startDeathwatchCluster(numSlots: Int, numTaskmanagers: Int): ForkableFlinkMiniCluster = {
     val config = new Configuration()
     config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, numSlots)
-    config.setInteger(ConfigConstants.LOCAL_INSTANCE_MANAGER_NUMBER_TASK_MANAGER, numTaskmanagers)
+    config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, numTaskmanagers)
+
+    val cluster = new ForkableFlinkMiniCluster(config, singleActorSystem = false)
+
+    cluster.start()
 
-    new ForkableFlinkMiniCluster(config, singleActorSystem = false)
+    cluster
   }
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/jobmanager/JobManagerLeaderSessionIDITSuite.scala
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/jobmanager/JobManagerLeaderSessionIDITSuite.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/jobmanager/JobManagerLeaderSessionIDITSuite.scala
index 010a086..f1e995f 100644
--- a/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/jobmanager/JobManagerLeaderSessionIDITSuite.scala
+++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/jobmanager/JobManagerLeaderSessionIDITSuite.scala
@@ -23,7 +23,7 @@ import java.util.UUID
 import akka.actor.ActorSystem
 import akka.actor.Status.Success
 import akka.testkit.{ImplicitSender, TestKit}
-import org.apache.flink.runtime.akka.AkkaUtils
+import org.apache.flink.runtime.akka.{ListeningBehaviour, AkkaUtils}
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable
 import org.apache.flink.runtime.jobgraph.{JobGraph, JobVertex}
 import org.apache.flink.runtime.messages.JobManagerMessages.{LeaderSessionMessage, CancelJob,
@@ -66,13 +66,13 @@ class JobManagerLeaderSessionIDITSuite(_system: ActorSystem)
     sender.setInvokableClass(classOf[BlockingUntilSignalNoOpInvokable])
     val jobGraph = new JobGraph("TestJob", sender)
 
-    val oldSessionID = Option(UUID.randomUUID())
+    val oldSessionID = UUID.randomUUID()
 
-    val jmGateway = cluster.getJobManagerGateway()
+    val jmGateway = cluster.getLeaderGateway(TestingUtils.TESTING_DURATION)
     val jm = jmGateway.actor()
 
     within(TestingUtils.TESTING_DURATION) {
-      jmGateway.tell(SubmitJob(jobGraph, false), self)
+      jmGateway.tell(SubmitJob(jobGraph, ListeningBehaviour.EXECUTION_RESULT), self)
 
       expectMsg(Success(jobGraph.getJobID))
 

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/taskmanager/TaskManagerFailsITCase.scala
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/taskmanager/TaskManagerFailsITCase.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/taskmanager/TaskManagerFailsITCase.scala
index a37fae3..869af82 100644
--- a/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/taskmanager/TaskManagerFailsITCase.scala
+++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/taskmanager/TaskManagerFailsITCase.scala
@@ -24,12 +24,12 @@ import akka.testkit.{ImplicitSender, TestKit}
 
 import org.apache.flink.configuration.ConfigConstants
 import org.apache.flink.configuration.Configuration
-import org.apache.flink.runtime.akka.AkkaUtils
+import org.apache.flink.runtime.akka.{ListeningBehaviour, AkkaUtils}
 import org.apache.flink.runtime.client.JobExecutionException
 import org.apache.flink.runtime.jobgraph.{JobVertex, DistributionPattern, JobGraph}
 import org.apache.flink.runtime.jobmanager.Tasks.{NoOpInvokable, BlockingNoOpInvokable, BlockingReceiver, Sender}
 import org.apache.flink.runtime.messages.JobManagerMessages._
-import org.apache.flink.runtime.messages.TaskManagerMessages.{RegisteredAtJobManager, NotifyWhenRegisteredAtJobManager}
+import org.apache.flink.runtime.messages.TaskManagerMessages.{RegisteredAtJobManager, NotifyWhenRegisteredAtAnyJobManager}
 import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages._
 import org.apache.flink.runtime.testingUtils.TestingMessages.DisableDisconnect
 import org.apache.flink.runtime.testingUtils.{ScalaTestingUtils, TestingUtils}
@@ -60,10 +60,12 @@ class TaskManagerFailsITCase(_system: ActorSystem)
     "detect a failing task manager" in {
 
       val num_slots = 11
-      val cluster = startDeathwatchCluster(num_slots, 2)
+      val cluster = createDeathwatchCluster(num_slots, 2)
+
+      cluster.start()
 
       val taskManagers = cluster.getTaskManagers
-      val jmGateway = cluster.getJobManagerGateway()
+      val jmGateway = cluster.getLeaderGateway(TestingUtils.TESTING_DURATION)
 
       jmGateway.tell(DisableDisconnect)
 
@@ -103,11 +105,11 @@ class TaskManagerFailsITCase(_system: ActorSystem)
 
       val cluster = ForkableFlinkMiniCluster.startCluster(num_tasks, 2)
 
-      val jmGateway = cluster.getJobManagerGateway()
+      val jmGateway = cluster.getLeaderGateway(TestingUtils.TESTING_DURATION)
 
       try {
         within(TestingUtils.TESTING_DURATION) {
-          jmGateway.tell(SubmitJob(jobGraph, false), self)
+          jmGateway.tell(SubmitJob(jobGraph, ListeningBehaviour.EXECUTION_RESULT), self)
           expectMsg(JobSubmitSuccess(jobGraph.getJobID()))
 
           jmGateway.tell(WaitForAllVerticesToBeRunningOrFinished(jobID), self)
@@ -156,11 +158,11 @@ class TaskManagerFailsITCase(_system: ActorSystem)
       val cluster = ForkableFlinkMiniCluster.startCluster(num_tasks, 2)
 
       val taskManagers = cluster.getTaskManagers
-      val jmGateway = cluster.getJobManagerGateway()
+      val jmGateway = cluster.getLeaderGateway(TestingUtils.TESTING_DURATION)
 
       try {
         within(TestingUtils.TESTING_DURATION) {
-          jmGateway.tell(SubmitJob(jobGraph, false), self)
+          jmGateway.tell(SubmitJob(jobGraph, ListeningBehaviour.EXECUTION_RESULT), self)
           expectMsg(JobSubmitSuccess(jobGraph.getJobID()))
 
           jmGateway.tell(WaitForAllVerticesToBeRunningOrFinished(jobID), self)
@@ -196,14 +198,16 @@ class TaskManagerFailsITCase(_system: ActorSystem)
       noOp.setInvokableClass(classOf[NoOpInvokable])
       val jobGraph2 = new JobGraph("NoOp Testjob", noOp)
 
-      val cluster = startDeathwatchCluster(num_slots/2, 2)
+      val cluster = createDeathwatchCluster(num_slots/2, 2)
+
+      cluster.start()
 
       var tm = cluster.getTaskManagers(0)
-      val jmGateway = cluster.getJobManagerGateway()
+      val jmGateway = cluster.getLeaderGateway(TestingUtils.TESTING_DURATION)
 
       try{
         within(TestingUtils.TESTING_DURATION){
-          jmGateway.tell(SubmitJob(jobGraph, false), self)
+          jmGateway.tell(SubmitJob(jobGraph, ListeningBehaviour.EXECUTION_RESULT), self)
           expectMsg(JobSubmitSuccess(jobGraph.getJobID))
 
           tm ! PoisonPill
@@ -221,11 +225,11 @@ class TaskManagerFailsITCase(_system: ActorSystem)
 
           tm = cluster.getTaskManagers(0)
 
-          tm ! NotifyWhenRegisteredAtJobManager
+          tm ! NotifyWhenRegisteredAtAnyJobManager
 
           expectMsg(RegisteredAtJobManager)
 
-          jmGateway.tell(SubmitJob(jobGraph2, false), self)
+          jmGateway.tell(SubmitJob(jobGraph2, ListeningBehaviour.EXECUTION_RESULT), self)
 
           expectMsg(JobSubmitSuccess(jobGraph2.getJobID()))
 
@@ -238,10 +242,10 @@ class TaskManagerFailsITCase(_system: ActorSystem)
     }
   }
 
-  def startDeathwatchCluster(numSlots: Int, numTaskmanagers: Int): ForkableFlinkMiniCluster = {
+  def createDeathwatchCluster(numSlots: Int, numTaskmanagers: Int): ForkableFlinkMiniCluster = {
     val config = new Configuration()
     config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, numSlots)
-    config.setInteger(ConfigConstants.LOCAL_INSTANCE_MANAGER_NUMBER_TASK_MANAGER, numTaskmanagers)
+    config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, numTaskmanagers)
 
     new ForkableFlinkMiniCluster(config, singleActorSystem = false)
   }

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-yarn/src/main/java/org/apache/flink/yarn/FlinkYarnCluster.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/FlinkYarnCluster.java b/flink-yarn/src/main/java/org/apache/flink/yarn/FlinkYarnCluster.java
index 56be198..560a584 100644
--- a/flink-yarn/src/main/java/org/apache/flink/yarn/FlinkYarnCluster.java
+++ b/flink-yarn/src/main/java/org/apache/flink/yarn/FlinkYarnCluster.java
@@ -101,9 +101,13 @@ public class FlinkYarnCluster extends AbstractFlinkYarnCluster {
 	 * @throws IOException
 	 * @throws YarnException
 	 */
-	public FlinkYarnCluster(final YarnClient yarnClient, final ApplicationId appId, Configuration hadoopConfig,
-							org.apache.flink.configuration.Configuration flinkConfig,
-							Path sessionFilesDir, boolean detached) throws IOException, YarnException {
+	public FlinkYarnCluster(
+			final YarnClient yarnClient,
+			final ApplicationId appId,
+			Configuration hadoopConfig,
+			org.apache.flink.configuration.Configuration flinkConfig,
+			Path sessionFilesDir,
+			boolean detached) throws IOException, YarnException {
 		this.akkaDuration = AkkaUtils.getTimeout(flinkConfig);
 		this.akkaTimeout = Timeout.durationToTimeout(akkaDuration);
 		this.yarnClient = yarnClient;

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-yarn/src/main/scala/org/apache/flink/yarn/ApplicationClient.scala
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/scala/org/apache/flink/yarn/ApplicationClient.scala b/flink-yarn/src/main/scala/org/apache/flink/yarn/ApplicationClient.scala
index 6930be6..5ae814f 100644
--- a/flink-yarn/src/main/scala/org/apache/flink/yarn/ApplicationClient.scala
+++ b/flink-yarn/src/main/scala/org/apache/flink/yarn/ApplicationClient.scala
@@ -23,7 +23,10 @@ import java.net.InetSocketAddress
 import akka.actor._
 import akka.pattern.ask
 import grizzled.slf4j.Logger
-import org.apache.flink.configuration.Configuration
+import org.apache.flink.configuration.{ConfigConstants, Configuration}
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService
+import org.apache.flink.runtime.util.LeaderRetrievalUtils
+import org.apache.flink.runtime.util.LeaderRetrievalUtils.LeaderGatewayListener
 import org.apache.flink.runtime.{FlinkActor, LogMessages}
 import org.apache.flink.runtime.akka.AkkaUtils
 import org.apache.flink.runtime.jobmanager.JobManager
@@ -74,12 +77,19 @@ class ApplicationClient(flinkConfig: Configuration)
   override def handleMessage: Receive = {
     // ----------------------------- Registration -> Status updates -> shutdown ----------------
     case LocalRegisterClient(address: InetSocketAddress) =>
-      val jmAkkaUrl = JobManager.getRemoteJobManagerAkkaURL(address)
+      flinkConfig.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, address.getHostName());
+      flinkConfig.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, address.getPort());
 
-      val jobManagerFuture = AkkaUtils.getReference(jmAkkaUrl, system, timeout)
+      val leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(flinkConfig)
 
-      jobManagerFuture.onComplete {
-        case Success(jm) => self ! decorateMessage(JobManagerActorRef(jm))
+      val listener = new LeaderGatewayListener(context.system, timeout);
+
+      leaderRetrievalService.start(listener)
+
+      val jobManagerGatewayFuture = listener.getActorGatewayFuture
+
+      jobManagerGatewayFuture.onComplete {
+        case Success(gateway) => self ! decorateMessage(JobManagerActorRef(gateway.actor()))
         case Failure(t) =>
           log.error("Registration at JobManager/ApplicationMaster failed. Shutting " +
             "ApplicationClient down.", t)

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-yarn/src/main/scala/org/apache/flink/yarn/ApplicationMaster.scala
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/scala/org/apache/flink/yarn/ApplicationMaster.scala b/flink-yarn/src/main/scala/org/apache/flink/yarn/ApplicationMaster.scala
index 9e0c976..1d1db7e 100644
--- a/flink-yarn/src/main/scala/org/apache/flink/yarn/ApplicationMaster.scala
+++ b/flink-yarn/src/main/scala/org/apache/flink/yarn/ApplicationMaster.scala
@@ -29,7 +29,7 @@ import org.apache.flink.runtime.akka.AkkaUtils
 import org.apache.flink.runtime.instance.AkkaActorGateway
 import org.apache.flink.runtime.jobmanager.JobManager
 import org.apache.flink.runtime.jobmanager.web.WebInfoServer
-import org.apache.flink.runtime.util.EnvironmentInformation
+import org.apache.flink.runtime.util.{StandaloneUtils, LeaderRetrievalUtils, EnvironmentInformation}
 import org.apache.flink.runtime.webmonitor.WebMonitor
 import org.apache.flink.yarn.Messages.StartYarnSession
 import org.apache.hadoop.security.UserGroupInformation
@@ -112,17 +112,11 @@ object ApplicationMaster {
               streamingMode)
 
           actorSystem = system
-          val extActor = system.asInstanceOf[ExtendedActorSystem]
-          val jobManagerPort = extActor.provider.getDefaultAddress.port.get
+          val address = AkkaUtils.getAddress(actorSystem)
+          val jobManagerPort = address.port.get
 
           if (config.getInteger(ConfigConstants.JOB_MANAGER_WEB_PORT_KEY, 0) != -1) {
             // start the web info server
-            val lookupTimeout = AkkaUtils.getLookupTimeout(config)
-            val jobManagerGateway = JobManager.getJobManagerGateway(jobManager, lookupTimeout)
-            val archiverGateway = new AkkaActorGateway(
-              archiver,
-              jobManagerGateway.leaderSessionID())
-
             LOG.info("Starting Job Manger web frontend.")
             config.setString(ConfigConstants.JOB_MANAGER_WEB_LOG_PATH_KEY, logDirs)
             config.setInteger(ConfigConstants.JOB_MANAGER_WEB_PORT_KEY, 0); // set port to 0.
@@ -130,13 +124,16 @@ object ApplicationMaster {
             config.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, ownHostname)
             config.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, jobManagerPort)
 
+            // TODO: Add support for HA: Make web server work independently from the JM
+            val leaderRetrievalService = StandaloneUtils.createLeaderRetrievalService(config)
+
             webserver = if(
               config.getBoolean(
                 ConfigConstants.JOB_MANAGER_NEW_WEB_FRONTEND_KEY,
                 false)) {
-              JobManager.startWebRuntimeMonitor(config, jobManagerGateway, archiverGateway)
+              JobManager.startWebRuntimeMonitor(config, leaderRetrievalService, actorSystem)
             } else {
-              new WebInfoServer(config, jobManagerGateway, archiverGateway)
+              new WebInfoServer(config, leaderRetrievalService, actorSystem)
             }
 
             webserver.start()
@@ -257,7 +254,8 @@ object ApplicationMaster {
       executionRetries,
       delayBetweenRetries,
       timeout,
-      _) = JobManager.createJobManagerComponents(configuration)
+      _,
+      leaderElectionService) = JobManager.createJobManagerComponents(configuration)
 
     // start the archiver
     val archiver: ActorRef = jobManagerSystem.actorOf(archiveProps, JobManager.ARCHIVE_NAME)
@@ -273,7 +271,8 @@ object ApplicationMaster {
         executionRetries,
         delayBetweenRetries,
         timeout,
-        streamingMode)
+        streamingMode,
+        leaderElectionService)
       with ApplicationMasterActor)
 
     LOG.debug("Starting JobManager actor")

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-yarn/src/main/scala/org/apache/flink/yarn/Messages.scala
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/scala/org/apache/flink/yarn/Messages.scala b/flink-yarn/src/main/scala/org/apache/flink/yarn/Messages.scala
index 99d0345..63c0aa6 100644
--- a/flink-yarn/src/main/scala/org/apache/flink/yarn/Messages.scala
+++ b/flink-yarn/src/main/scala/org/apache/flink/yarn/Messages.scala
@@ -37,9 +37,10 @@ object Messages {
 
   case object JobManagerStopped
 
-  case class StartYarnSession(configuration: Configuration,
-                              actorSystemPort: Int,
-                              webServerport: Int)
+  case class StartYarnSession(
+      configuration: Configuration,
+      actorSystemPort: Int,
+      webServerport: Int)
 
   case class JobManagerActorRef(jobManager: ActorRef)
 

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnTaskManager.scala
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnTaskManager.scala b/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnTaskManager.scala
index 5216030..b95eb86 100644
--- a/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnTaskManager.scala
+++ b/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnTaskManager.scala
@@ -21,6 +21,7 @@ package org.apache.flink.yarn
 import org.apache.flink.runtime.instance.InstanceConnectionInfo
 import org.apache.flink.runtime.io.disk.iomanager.IOManager
 import org.apache.flink.runtime.io.network.NetworkEnvironment
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService
 import org.apache.flink.runtime.memorymanager.DefaultMemoryManager
 import org.apache.flink.runtime.taskmanager.{NetworkEnvironmentConfiguration, TaskManagerConfiguration, TaskManager}
 import org.apache.flink.yarn.Messages.StopYarnSession
@@ -31,19 +32,19 @@ import org.apache.flink.yarn.Messages.StopYarnSession
 class YarnTaskManager(
     config: TaskManagerConfiguration,
     connectionInfo: InstanceConnectionInfo,
-    jobManagerAkkaURL: String,
     memoryManager: DefaultMemoryManager,
     ioManager: IOManager,
     network: NetworkEnvironment,
-    numberOfSlots: Int)
+    numberOfSlots: Int,
+    leaderRetrievalService: LeaderRetrievalService)
   extends TaskManager(
     config,
     connectionInfo,
-    jobManagerAkkaURL,
     memoryManager,
     ioManager,
     network,
-    numberOfSlots) {
+    numberOfSlots,
+    leaderRetrievalService) {
 
   override def handleMessage: Receive = {
     handleYarnMessages orElse super.handleMessage

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index dbba9fc..0e7f995 100644
--- a/pom.xml
+++ b/pom.xml
@@ -87,7 +87,7 @@ under the License.
 		<asm.version>5.0.4</asm.version>
 		<tez.version>0.6.1</tez.version>
 		<zookeeper.version>3.4.6</zookeeper.version>
-		<curatorrecipes.version>2.8.0</curatorrecipes.version>
+		<curator.version>2.8.0</curator.version>
 	</properties>
 
 	<dependencies>
@@ -332,15 +332,22 @@ under the License.
 			</dependency>
 
 			<dependency>
-				<groupId>org.apache.zookeeper</groupId>
-				<artifactId>zookeeper</artifactId>
-				<version>${zookeeper.version}</version>
+				<groupId>org.apache.curator</groupId>
+				<artifactId>curator-recipes</artifactId>
+				<version>${curator.version}</version>
 			</dependency>
 
 			<dependency>
 				<groupId>org.apache.curator</groupId>
-				<artifactId>curator-recipes</artifactId>
-				<version>${curatorrecipes.version}</version>
+				<artifactId>curator-test</artifactId>
+				<version>${curator.version}</version>
+				<scope>test</scope>
+			</dependency>
+
+			<dependency>
+				<groupId>org.apache.zookeeper</groupId>
+				<artifactId>zookeeper</artifactId>
+				<version>${zookeeper.version}</version>
 			</dependency>
 		</dependencies>
 	</dependencyManagement>

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/tools/log4j-travis.properties
----------------------------------------------------------------------
diff --git a/tools/log4j-travis.properties b/tools/log4j-travis.properties
index 69ddcde..34ae971 100644
--- a/tools/log4j-travis.properties
+++ b/tools/log4j-travis.properties
@@ -35,4 +35,5 @@ log4j.appender.file.layout=org.apache.log4j.PatternLayout
 log4j.appender.file.layout.ConversionPattern=%d{HH:mm:ss,SSS} %-5p %-60c %x - %m%n
 
 # suppress the irrelevant (wrong) warnings from the netty channel handler
-log4j.logger.org.jboss.netty.channel.DefaultChannelPipeline=ERROR, file
\ No newline at end of file
+log4j.logger.org.jboss.netty.channel.DefaultChannelPipeline=ERROR, file
+log4j.logger.org.apache.zookeeper=ERROR, file


[03/10] flink git commit: [FLINK-2291] [runtime] Add ZooKeeper support to elect a leader from a set of JobManager. The leader will then be retrieved from ZooKeeper by the TaskManagers.

Posted by tr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingCluster.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingCluster.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingCluster.scala
index 057ffeb..d762ab4 100644
--- a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingCluster.scala
+++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingCluster.scala
@@ -18,16 +18,23 @@
 
 package org.apache.flink.runtime.testingUtils
 
+import java.util.concurrent.TimeoutException
+
+import akka.pattern.ask
 import akka.actor.{ActorRef, Props, ActorSystem}
 import akka.testkit.CallingThreadDispatcher
 import org.apache.flink.configuration.{ConfigConstants, Configuration}
 import org.apache.flink.runtime.StreamingMode
 import org.apache.flink.runtime.jobmanager.JobManager
+import org.apache.flink.runtime.leaderelection.LeaderElectionService
 import org.apache.flink.runtime.minicluster.FlinkMiniCluster
 import org.apache.flink.runtime.net.NetUtils
 import org.apache.flink.runtime.taskmanager.TaskManager
+import org.apache.flink.runtime.testingUtils.TestingMessages.Alive
 import org.apache.flink.runtime.webmonitor.WebMonitor
 
+import scala.concurrent.{Await, Future}
+
 /**
  * Testing cluster which starts the [[JobManager]] and [[TaskManager]] actors with testing support
  * in the same [[ActorSystem]].
@@ -36,13 +43,15 @@ import org.apache.flink.runtime.webmonitor.WebMonitor
  * @param singleActorSystem true if all actors shall be running in the same [[ActorSystem]],
  *                          otherwise false
  */
-class TestingCluster(userConfiguration: Configuration,
-                     singleActorSystem: Boolean,
-                     synchronousDispatcher: Boolean,
-                     streamingMode: StreamingMode)
-  extends FlinkMiniCluster(userConfiguration,
-                           singleActorSystem,
-                           streamingMode) {
+class TestingCluster(
+    userConfiguration: Configuration,
+    singleActorSystem: Boolean,
+    synchronousDispatcher: Boolean,
+    streamingMode: StreamingMode)
+  extends FlinkMiniCluster(
+    userConfiguration,
+    singleActorSystem,
+    streamingMode) {
   
 
   def this(userConfiguration: Configuration,
@@ -68,7 +77,28 @@ class TestingCluster(userConfiguration: Configuration,
     cfg
   }
 
-  override def startJobManager(actorSystem: ActorSystem): (ActorRef, Option[WebMonitor]) = {
+  override def startJobManager(index: Int, actorSystem: ActorSystem): ActorRef = {
+    val config = configuration.clone()
+
+    val jobManagerName = if(singleActorSystem) {
+      JobManager.JOB_MANAGER_NAME + "_" + (index + 1)
+    } else {
+      JobManager.JOB_MANAGER_NAME
+    }
+
+    val archiveName = if(singleActorSystem) {
+      JobManager.ARCHIVE_NAME + "_" + (index + 1)
+    } else {
+      JobManager.ARCHIVE_NAME
+    }
+
+    val jobManagerPort = config.getInteger(
+      ConfigConstants.JOB_MANAGER_IPC_PORT_KEY,
+      ConfigConstants.DEFAULT_JOB_MANAGER_IPC_PORT)
+
+    if(jobManagerPort > 0) {
+      config.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, jobManagerPort + index)
+    }
 
     val (executionContext,
       instanceManager,
@@ -78,11 +108,14 @@ class TestingCluster(userConfiguration: Configuration,
       executionRetries,
       delayBetweenRetries,
       timeout,
-      archiveCount) = JobManager.createJobManagerComponents(configuration)
-    
+      archiveCount,
+      leaderElectionService) = JobManager.createJobManagerComponents(config)
+
     val testArchiveProps = Props(new TestingMemoryArchivist(archiveCount))
-    val archive = actorSystem.actorOf(testArchiveProps, JobManager.ARCHIVE_NAME)
-    
+    val archive = actorSystem.actorOf(testArchiveProps, archiveName)
+
+    val resolvedLeaderElectionService = createLeaderElectionService(leaderElectionService)
+
     val jobManagerProps = Props(
       new TestingJobManager(
         configuration,
@@ -94,7 +127,8 @@ class TestingCluster(userConfiguration: Configuration,
         executionRetries,
         delayBetweenRetries,
         timeout,
-        streamingMode))
+        streamingMode,
+        resolvedLeaderElectionService))
 
     val dispatcherJobManagerProps = if (synchronousDispatcher) {
       // disable asynchronous futures (e.g. accumulator update in Heartbeat)
@@ -103,27 +137,60 @@ class TestingCluster(userConfiguration: Configuration,
       jobManagerProps
     }
 
-    (actorSystem.actorOf(dispatcherJobManagerProps, JobManager.JOB_MANAGER_NAME), None)
+    actorSystem.actorOf(dispatcherJobManagerProps, jobManagerName)
   }
 
   override def startTaskManager(index: Int, system: ActorSystem) = {
 
     val tmActorName = TaskManager.TASK_MANAGER_NAME + "_" + (index + 1)
 
-    val jobManagerPath: Option[String] = if (singleActorSystem) {
-      Some(jobManagerActor.path.toString)
-    } else {
-      None
-    }
-    
     TaskManager.startTaskManagerComponentsAndActor(
       configuration,
       system,
       hostname,
       Some(tmActorName),
-      jobManagerPath,
+      Some(createLeaderRetrievalService),
       numTaskManagers == 1,
       streamingMode,
       classOf[TestingTaskManager])
   }
+
+
+  def createLeaderElectionService(electionService: LeaderElectionService): LeaderElectionService = {
+    electionService
+  }
+
+  @throws(classOf[TimeoutException])
+  @throws(classOf[InterruptedException])
+  def waitForTaskManagersToBeAlive(): Unit = {
+    val aliveFutures = taskManagerActors map {
+      _ map {
+        tm => (tm ? Alive)(timeout)
+      }
+    } getOrElse(Seq())
+
+    val combinedFuture = Future.sequence(aliveFutures)
+
+    Await.ready(combinedFuture, timeout)
+  }
+
+  @throws(classOf[TimeoutException])
+  @throws(classOf[InterruptedException])
+  def waitForActorsToBeAlive(): Unit = {
+    val tmsAliveFutures = taskManagerActors map {
+      _ map {
+        tm => (tm ? Alive)(timeout)
+      }
+    } getOrElse(Seq())
+
+    val jmsAliveFutures = jobManagerActors map {
+      _ map {
+        tm => (tm ? Alive)(timeout)
+      }
+    } getOrElse(Seq())
+
+    val combinedFuture = Future.sequence(tmsAliveFutures ++ jmsAliveFutures)
+
+    Await.ready(combinedFuture, timeout)
+  }
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManager.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManager.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManager.scala
index 987af40..c91a421 100644
--- a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManager.scala
+++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManager.scala
@@ -19,10 +19,13 @@
 package org.apache.flink.runtime.testingUtils
 
 import akka.actor.{Cancellable, Terminated, ActorRef}
-import akka.pattern.{ask, pipe}
+import akka.pattern.pipe
+import akka.pattern.ask
 import org.apache.flink.api.common.JobID
 import org.apache.flink.configuration.Configuration
+import org.apache.flink.runtime.leaderelection.LeaderElectionService
 import org.apache.flink.runtime.{StreamingMode, FlinkActor}
+import org.apache.flink.runtime.StreamingMode
 import org.apache.flink.runtime.execution.ExecutionState
 import org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager
 import org.apache.flink.runtime.instance.InstanceManager
@@ -30,10 +33,12 @@ import org.apache.flink.runtime.jobgraph.JobStatus
 import org.apache.flink.runtime.jobmanager.JobManager
 import org.apache.flink.runtime.jobmanager.scheduler.Scheduler
 import org.apache.flink.runtime.messages.ExecutionGraphMessages.JobStatusChanged
-import org.apache.flink.runtime.messages.Messages.Disconnect
+import org.apache.flink.runtime.messages.JobManagerMessages.GrantLeadership
+import org.apache.flink.runtime.messages.Messages.{Acknowledge, Disconnect}
 import org.apache.flink.runtime.messages.TaskManagerMessages.Heartbeat
 import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages._
-import org.apache.flink.runtime.testingUtils.TestingMessages.DisableDisconnect
+import org.apache.flink.runtime.testingUtils.TestingMessages.{CheckIfJobRemoved, Alive,
+DisableDisconnect}
 import org.apache.flink.runtime.testingUtils.TestingTaskManagerMessages.AccumulatorsChanged
 
 import scala.concurrent.{ExecutionContext, Future}
@@ -64,7 +69,8 @@ class TestingJobManager(
     defaultExecutionRetries: Int,
     delayBetweenRetries: Long,
     timeout: FiniteDuration,
-    mode: StreamingMode)
+    mode: StreamingMode,
+    leaderElectionService: LeaderElectionService)
   extends JobManager(
     flinkConfiguration,
     executionContext,
@@ -75,7 +81,8 @@ class TestingJobManager(
     defaultExecutionRetries,
     delayBetweenRetries,
     timeout,
-    mode) {
+    mode,
+    leaderElectionService) {
 
   import scala.collection.JavaConverters._
   import context._
@@ -93,6 +100,8 @@ class TestingJobManager(
 
   val waitForAccumulatorUpdate = scala.collection.mutable.HashMap[JobID, (Boolean, Set[ActorRef])]()
 
+  val waitForLeader = scala.collection.mutable.HashSet[ActorRef]()
+
   var disconnectDisabled = false
 
   override def handleMessage: Receive = {
@@ -100,6 +109,8 @@ class TestingJobManager(
   }
 
   def handleTestingMessage: Receive = {
+    case Alive => sender() ! Acknowledge
+
     case RequestExecutionGraph(jobID) =>
       currentJobs.get(jobID) match {
         case Some((executionGraph, jobInfo)) => sender ! decorateMessage(
@@ -168,9 +179,23 @@ class TestingJobManager(
         gateway => gateway.ask(NotifyWhenJobRemoved(jobID), timeout).mapTo[Boolean]
       }
 
-      import context.dispatcher
+      val jobRemovedOnJobManager = (self ? CheckIfJobRemoved(jobID))(timeout).mapTo[Boolean]
 
-      Future.fold(responses)(true)(_ & _).map(decorateMessage(_)) pipeTo sender
+      val allFutures = responses ++ Seq(jobRemovedOnJobManager)
+
+      import context.dispatcher
+      Future.fold(allFutures)(true)(_ & _) map(decorateMessage(_)) pipeTo sender
+
+    case CheckIfJobRemoved(jobID) =>
+      if(currentJobs.contains(jobID)) {
+        context.system.scheduler.scheduleOnce(
+          200 milliseconds,
+          self,
+          decorateMessage(CheckIfJobRemoved(jobID))
+        )(context.dispatcher, sender())
+      } else {
+        sender() ! decorateMessage(true)
+      }
 
     case NotifyWhenTaskManagerTerminated(taskManager) =>
       val waiting = waitForTaskManagerToBeTerminated.getOrElse(taskManager.path.name, Set())
@@ -293,6 +318,20 @@ class TestingJobManager(
           }
         }
       }
+
+    case NotifyWhenLeader =>
+      if (leaderElectionService.hasLeadership) {
+        sender() ! true
+      } else {
+        waitForLeader += sender()
+      }
+
+    case msg: GrantLeadership =>
+      super.handleMessage(msg)
+
+      waitForLeader.foreach(_ ! true)
+
+      waitForLeader.clear()
   }
 
   def checkIfAllVerticesRunning(jobID: JobID): Boolean = {

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManagerMessages.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManagerMessages.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManagerMessages.scala
index 17beff0..acade53 100644
--- a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManagerMessages.scala
+++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManagerMessages.scala
@@ -57,12 +57,13 @@ object TestingJobManagerMessages {
   case class NotifyWhenTaskManagerTerminated(taskManager: ActorRef)
   case class TaskManagerTerminated(taskManager: ActorRef)
 
-  /* Registers a listener to receive a message when accumulators changed.
+  /**
+   * Registers a listener to receive a message when accumulators changed.
    * The change must be explicitly triggered by the TestingTaskManager which can receive an
    * [[AccumulatorChanged]] message by a task that changed the accumulators. This message is then
    * forwarded to the JobManager which will send the accumulators in the [[UpdatedAccumulators]]
    * message when the next Heartbeat occurs.
-   * */
+   */
   case class NotifyWhenAccumulatorChange(jobID: JobID)
 
   /**
@@ -71,4 +72,11 @@ object TestingJobManagerMessages {
   case class UpdatedAccumulators(jobID: JobID,
     flinkAccumulators: Map[ExecutionAttemptID, Map[AccumulatorRegistry.Metric, Accumulator[_,_]]],
     userAccumulators: Map[String, Accumulator[_,_]])
+
+  /** Notifies the sender when the [[TestingJobManager]] has been elected as the leader
+    *
+    */
+  case object NotifyWhenLeader
+
+  def getNotifyWhenLeader: AnyRef = NotifyWhenLeader
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingMessages.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingMessages.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingMessages.scala
index 52cc1f3..c840ff0 100644
--- a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingMessages.scala
+++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingMessages.scala
@@ -18,7 +18,17 @@
 
 package org.apache.flink.runtime.testingUtils
 
+import org.apache.flink.api.common.JobID
+
 object TestingMessages {
 
+  case class CheckIfJobRemoved(jobID: JobID)
+
   case object DisableDisconnect
+
+  case object Alive
+
+  def getAlive: AnyRef = Alive
+
+  def getDisableDisconnect: AnyRef = DisableDisconnect
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingTaskManager.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingTaskManager.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingTaskManager.scala
index 8cc1e92..475115e 100644
--- a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingTaskManager.scala
+++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingTaskManager.scala
@@ -19,20 +19,23 @@
 package org.apache.flink.runtime.testingUtils
 
 import akka.actor.{Terminated, ActorRef}
-import org.apache.flink.api.common.JobID
 import org.apache.flink.runtime.execution.ExecutionState
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID
 import org.apache.flink.runtime.instance.InstanceConnectionInfo
 import org.apache.flink.runtime.io.disk.iomanager.IOManager
 import org.apache.flink.runtime.io.network.NetworkEnvironment
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService
 import org.apache.flink.runtime.memorymanager.DefaultMemoryManager
 import org.apache.flink.runtime.messages.JobManagerMessages.{ResponseLeaderSessionID,
 RequestLeaderSessionID}
-import org.apache.flink.runtime.messages.Messages.Disconnect
+import org.apache.flink.runtime.messages.Messages.{Acknowledge, Disconnect}
+import org.apache.flink.runtime.messages.RegistrationMessages.{AlreadyRegistered,
+AcknowledgeRegistration}
 import org.apache.flink.runtime.messages.TaskMessages.{UpdateTaskExecutionState, TaskInFinalState}
 import org.apache.flink.runtime.taskmanager.{TaskManagerConfiguration, TaskManager}
 import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages.NotifyWhenJobRemoved
-import org.apache.flink.runtime.testingUtils.TestingMessages.DisableDisconnect
+import org.apache.flink.runtime.testingUtils.TestingMessages.{CheckIfJobRemoved, Alive,
+DisableDisconnect}
 import org.apache.flink.runtime.testingUtils.TestingTaskManagerMessages._
 
 import scala.concurrent.duration._
@@ -42,7 +45,6 @@ import scala.language.postfixOps
   *
   * @param config
   * @param connectionInfo
-  * @param jobManagerAkkaURL
   * @param memoryManager
   * @param ioManager
   * @param network
@@ -51,25 +53,25 @@ import scala.language.postfixOps
 class TestingTaskManager(
     config: TaskManagerConfiguration,
     connectionInfo: InstanceConnectionInfo,
-    jobManagerAkkaURL: String,
     memoryManager: DefaultMemoryManager,
     ioManager: IOManager,
     network: NetworkEnvironment,
-    numberOfSlots: Int)
+    numberOfSlots: Int,
+    leaderRetrievalService: LeaderRetrievalService)
   extends TaskManager(
     config,
     connectionInfo,
-    jobManagerAkkaURL,
     memoryManager,
     ioManager,
     network,
-    numberOfSlots) {
+    numberOfSlots,
+    leaderRetrievalService) {
 
   import scala.collection.JavaConverters._
 
   val waitForRemoval = scala.collection.mutable.HashMap[ExecutionAttemptID, Set[ActorRef]]()
-  val waitForJobRemoval = scala.collection.mutable.HashMap[JobID, Set[ActorRef]]()
   val waitForJobManagerToBeTerminated = scala.collection.mutable.HashMap[String, Set[ActorRef]]()
+  val waitForRegisteredAtJobManager = scala.collection.mutable.HashMap[ActorRef, Set[ActorRef]]()
   val waitForRunning = scala.collection.mutable.HashMap[ExecutionAttemptID, Set[ActorRef]]()
   val unregisteredTasks = scala.collection.mutable.HashSet[ExecutionAttemptID]()
 
@@ -83,6 +85,8 @@ class TestingTaskManager(
   }
 
   def handleTestingMessage: Receive = {
+    case Alive => sender() ! Acknowledge
+
     case NotifyWhenTaskIsRunning(executionID) => {
       Option(runningTasks.get(executionID)) match {
         case Some(task) if task.getExecutionState == ExecutionState.RUNNING =>
@@ -136,32 +140,28 @@ class TestingTaskManager(
 
     case NotifyWhenJobRemoved(jobID) =>
       if(runningTasks.values.asScala.exists(_.getJobID == jobID)){
-        val set = waitForJobRemoval.getOrElse(jobID, Set())
-        waitForJobRemoval += (jobID -> (set + sender))
-        import context.dispatcher
         context.system.scheduler.scheduleOnce(
           200 milliseconds,
-          this.self,
-          decorateMessage(CheckIfJobRemoved(jobID)))
+          self,
+          decorateMessage(CheckIfJobRemoved(jobID)))(
+          context.dispatcher,
+          sender
+          )
       }else{
-        waitForJobRemoval.get(jobID) match {
-          case Some(listeners) => (listeners + sender) foreach (_ ! decorateMessage(true))
-          case None => sender ! decorateMessage(true)
-        }
+        sender ! decorateMessage(true)
       }
 
     case CheckIfJobRemoved(jobID) =>
       if(runningTasks.values.asScala.forall(_.getJobID != jobID)){
-        waitForJobRemoval.remove(jobID) match {
-          case Some(listeners) => listeners foreach (_ ! decorateMessage(true))
-          case None =>
-        }
+        sender ! decorateMessage(true)
       } else {
-        import context.dispatcher
         context.system.scheduler.scheduleOnce(
           200 milliseconds,
-          this.self,
-          decorateMessage(CheckIfJobRemoved(jobID)))
+          self,
+          decorateMessage(CheckIfJobRemoved(jobID)))(
+          context.dispatcher,
+          sender
+          )
       }
 
     case NotifyWhenJobManagerTerminated(jobManager) =>
@@ -218,6 +218,29 @@ class TestingTaskManager(
       }
 
     case RequestLeaderSessionID =>
-      sender() ! ResponseLeaderSessionID(leaderSessionID)
+      sender() ! ResponseLeaderSessionID(leaderSessionID.orNull)
+
+    case NotifyWhenRegisteredAtJobManager(jobManager: ActorRef) =>
+      if(isConnected && jobManager == currentJobManager.get) {
+        sender() ! true
+      } else {
+        val list = waitForRegisteredAtJobManager.getOrElse(
+          jobManager,
+          Set[ActorRef]())
+
+        waitForRegisteredAtJobManager += jobManager -> (list + sender())
+      }
+
+    case msg @ (_: AcknowledgeRegistration | _: AlreadyRegistered) =>
+      super.handleMessage(msg)
+
+      val jm = sender()
+
+      waitForRegisteredAtJobManager.remove(jm).foreach {
+        listeners => listeners.foreach{
+          listener =>
+            listener ! true
+        }
+      }
   }
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingTaskManagerMessages.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingTaskManagerMessages.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingTaskManagerMessages.scala
index 1c428cc..ca57245 100644
--- a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingTaskManagerMessages.scala
+++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingTaskManagerMessages.scala
@@ -41,8 +41,6 @@ object TestingTaskManagerMessages {
 
   case object RequestNumActiveConnections
   case class ResponseNumActiveConnections(number: Int)
-
-  case class CheckIfJobRemoved(jobID: JobID)
   
   case object RequestRunningTasks
   
@@ -52,6 +50,8 @@ object TestingTaskManagerMessages {
 
   case class JobManagerTerminated(jobManager: ActorRef)
 
+  case class NotifyWhenRegisteredAtJobManager(jobManager: ActorRef)
+
   /**
    * Message to give a hint to the task manager that accumulator values were updated in the task.
    * This message is forwarded to the job manager which knows that it needs to notify listeners

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingUtils.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingUtils.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingUtils.scala
index 914f37c..21939d6 100644
--- a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingUtils.scala
+++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingUtils.scala
@@ -18,15 +18,27 @@
 
 package org.apache.flink.runtime.testingUtils
 
+import java.util.UUID
+
+import akka.actor.{Props, Kill, ActorSystem, ActorRef}
+import akka.pattern.ask
 import com.google.common.util.concurrent.MoreExecutors
 
 import com.typesafe.config.ConfigFactory
+import grizzled.slf4j.Logger
 
 import org.apache.flink.configuration.{ConfigConstants, Configuration}
+import org.apache.flink.runtime.jobmanager.JobManager
+import org.apache.flink.runtime.{LogMessages, LeaderSessionMessageFilter, FlinkActor, StreamingMode}
 import org.apache.flink.runtime.akka.AkkaUtils
+import org.apache.flink.runtime.instance.{AkkaActorGateway, ActorGateway}
+import org.apache.flink.runtime.leaderelection.StandaloneLeaderElectionService
+import org.apache.flink.runtime.leaderretrieval.StandaloneLeaderRetrievalService
+import org.apache.flink.runtime.messages.TaskManagerMessages.NotifyWhenRegisteredAtAnyJobManager
+import org.apache.flink.runtime.taskmanager.TaskManager
 
 import scala.concurrent.duration._
-import scala.concurrent.ExecutionContext
+import scala.concurrent.{Await, ExecutionContext}
 import scala.language.postfixOps
 
 /**
@@ -60,9 +72,14 @@ object TestingUtils {
                           timeout: String = DEFAULT_AKKA_ASK_TIMEOUT): TestingCluster = {
     val config = new Configuration()
     config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, numSlots)
-    config.setInteger(ConfigConstants.LOCAL_INSTANCE_MANAGER_NUMBER_TASK_MANAGER, numTMs)
+    config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, numTMs)
     config.setString(ConfigConstants.AKKA_ASK_TIMEOUT, timeout)
-    new TestingCluster(config)
+
+    val cluster = new TestingCluster(config)
+
+    cluster.start()
+
+    cluster
   }
 
   /** Returns the global [[ExecutionContext]] which is a [[scala.concurrent.forkjoin.ForkJoinPool]]
@@ -132,4 +149,182 @@ object TestingUtils {
       runnables.isEmpty
     }
   }
+
+  def createTaskManager(
+    actorSystem: ActorSystem,
+    jobManager: ActorRef,
+    configuration: Configuration,
+    useLocalCommunication: Boolean,
+    waitForRegistration: Boolean)
+  : ActorGateway = {
+    val jobManagerURL = AkkaUtils.getAkkaURL(actorSystem, jobManager)
+
+    createTaskManager(
+      actorSystem,
+      jobManagerURL,
+      configuration,
+      useLocalCommunication,
+      waitForRegistration
+    )
+  }
+
+  def createTaskManager(
+      actorSystem: ActorSystem,
+      jobManager: ActorGateway,
+      configuration: Configuration,
+      useLocalCommunication: Boolean,
+      waitForRegistration: Boolean)
+    : ActorGateway = {
+    val jobManagerURL = AkkaUtils.getAkkaURL(actorSystem, jobManager.actor)
+
+    createTaskManager(
+      actorSystem,
+      jobManagerURL,
+      configuration,
+      useLocalCommunication,
+      waitForRegistration
+    )
+  }
+
+  /** Creates a local TaskManager in the given ActorSystem. It is given a
+    * [[StandaloneLeaderRetrievalService]] which returns the given jobManagerURL. After creating
+    * the TaskManager, waitForRegistration specifies whether one waits until the TaskManager has
+    * registered at the JobManager. An ActorGateway to the TaskManager is returned.
+    *
+    * @param actorSystem ActorSystem in which the TaskManager shall be started
+    * @param jobManagerURL URL of the JobManager to connect to
+    * @param configuration Configuration
+    * @param useLocalCommunication true if the network stack shall use exclusively local
+    *                              communication
+    * @param waitForRegistration true if the method will wait until the TaskManager has connected to
+    *                            the JobManager
+    * @return ActorGateway of the created TaskManager
+    */
+  def createTaskManager(
+      actorSystem: ActorSystem,
+      jobManagerURL: String,
+      configuration: Configuration,
+      useLocalCommunication: Boolean,
+      waitForRegistration: Boolean)
+    : ActorGateway = {
+
+    val resultingConfiguration = new Configuration()
+
+    resultingConfiguration.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 10)
+
+    resultingConfiguration.addAll(configuration)
+
+    val leaderRetrievalService = Option(new StandaloneLeaderRetrievalService(jobManagerURL))
+
+    val taskManager = TaskManager.startTaskManagerComponentsAndActor(
+      resultingConfiguration,
+      actorSystem,
+      "localhost",
+      None,
+      leaderRetrievalService,
+      useLocalCommunication,
+      StreamingMode.BATCH_ONLY,
+      classOf[TestingTaskManager]
+    )
+
+    if (waitForRegistration) {
+      val notificationResult = (taskManager ? NotifyWhenRegisteredAtAnyJobManager)(TESTING_DURATION)
+
+      Await.ready(notificationResult, TESTING_DURATION)
+    }
+
+    new AkkaActorGateway(taskManager, null)
+  }
+
+  /** Stops the given actor by sending it a Kill message
+    *
+    * @param actor
+    */
+  def stopActor(actor: ActorRef): Unit = {
+    if (actor != null) {
+      actor ! Kill
+    }
+  }
+
+  /** Stops the given actro by sending it a Kill message
+    *
+    * @param actorGateway
+    */
+  def stopActor(actorGateway: ActorGateway): Unit = {
+    if (actorGateway != null) {
+      stopActor(actorGateway.actor())
+    }
+  }
+
+  /** Creates a testing JobManager using the default recovery mode (standalone)
+    *
+    * @param actorSystem
+    * @param configuration
+    * @return
+    */
+  def createJobManager(
+      actorSystem: ActorSystem,
+      configuration: Configuration)
+    : ActorGateway = {
+
+    configuration.setString(ConfigConstants.RECOVERY_MODE, ConfigConstants.DEFAULT_RECOVERY_MODE)
+
+      val (actor, _) = JobManager.startJobManagerActors(
+        configuration,
+        actorSystem,
+        Some(JobManager.JOB_MANAGER_NAME),
+        Some(JobManager.ARCHIVE_NAME),
+        StreamingMode.BATCH_ONLY)
+
+    new AkkaActorGateway(actor, null)
+  }
+
+  /** Creates a forwarding JobManager which sends all received message to the forwarding target.
+    *
+    * @param actorSystem
+    * @param forwardingTarget
+    * @param jobManagerName
+    * @return
+    */
+  def createForwardingJobManager(
+      actorSystem: ActorSystem,
+      forwardingTarget: ActorRef,
+      jobManagerName: Option[String] = None)
+    : ActorGateway = {
+
+    val actor = jobManagerName match {
+      case Some(name) =>
+        actorSystem.actorOf(
+          Props(
+            classOf[ForwardingActor],
+            forwardingTarget,
+            None),
+          name
+        )
+      case None =>
+        actorSystem.actorOf(
+          Props(
+            classOf[ForwardingActor],
+            forwardingTarget,
+            None)
+        )
+    }
+
+    new AkkaActorGateway(actor, null)
+  }
+
+  class ForwardingActor(val target: ActorRef, val leaderSessionID: Option[UUID])
+    extends FlinkActor with LeaderSessionMessageFilter with LogMessages {
+
+    /** Handle incoming messages
+      *
+      * @return
+      */
+    override def handleMessage: Receive = {
+      case msg => target.forward(msg)
+    }
+
+    override val log: Logger = Logger(getClass)
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-shaded-hadoop/pom.xml
----------------------------------------------------------------------
diff --git a/flink-shaded-hadoop/pom.xml b/flink-shaded-hadoop/pom.xml
index d7e3158..71cc895 100644
--- a/flink-shaded-hadoop/pom.xml
+++ b/flink-shaded-hadoop/pom.xml
@@ -101,6 +101,10 @@ under the License.
 									<pattern>org.jboss.netty</pattern>
 									<shadedPattern>org.apache.flink.hadoop.shaded.org.jboss.netty</shadedPattern>
 								</relocation>
+								<relocation>
+									<pattern>org.apache.curator</pattern>
+									<shadedPattern>org.apache.flink.hadoop.shaded.org.apache.curator</shadedPattern>
+								</relocation>
 							</relocations>
 						</configuration>
 					</execution>

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-staging/flink-avro/src/test/java/org/apache/flink/api/avro/AvroExternalJarProgramITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-avro/src/test/java/org/apache/flink/api/avro/AvroExternalJarProgramITCase.java b/flink-staging/flink-avro/src/test/java/org/apache/flink/api/avro/AvroExternalJarProgramITCase.java
index 528e871..f691806 100644
--- a/flink-staging/flink-avro/src/test/java/org/apache/flink/api/avro/AvroExternalJarProgramITCase.java
+++ b/flink-staging/flink-avro/src/test/java/org/apache/flink/api/avro/AvroExternalJarProgramITCase.java
@@ -19,7 +19,6 @@
 package org.apache.flink.api.avro;
 
 import java.io.File;
-import java.net.InetSocketAddress;
 
 import org.apache.flink.client.program.Client;
 import org.apache.flink.client.program.PackagedProgram;
@@ -45,14 +44,21 @@ public class AvroExternalJarProgramITCase {
 			Configuration config = new Configuration();
 			config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 4);
 			testMiniCluster = new ForkableFlinkMiniCluster(config, false);
+			testMiniCluster.start();
 			
 			String jarFile = JAR_FILE;
 			String testData = getClass().getResource(TEST_DATA_FILE).toString();
 			
 			PackagedProgram program = new PackagedProgram(new File(jarFile), new String[] { testData });
+
+			config.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, "localhost");
+			config.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, testMiniCluster.getLeaderRPCPort());
 						
-			Client c = new Client(new InetSocketAddress("localhost", testMiniCluster.getJobManagerRPCPort()),
-					new Configuration(), program.getUserCodeClassLoader(), -1);
+			Client c = new Client(
+					config,
+					program.getUserCodeClassLoader(),
+					-1);
+
 			c.setPrintStatusDuringExecution(false);
 			c.run(program, 4, true);
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-staging/flink-avro/src/test/java/org/apache/flink/api/io/avro/AvroPojoTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-avro/src/test/java/org/apache/flink/api/io/avro/AvroPojoTest.java b/flink-staging/flink-avro/src/test/java/org/apache/flink/api/io/avro/AvroPojoTest.java
index 7cebccb..7f46c7e 100644
--- a/flink-staging/flink-avro/src/test/java/org/apache/flink/api/io/avro/AvroPojoTest.java
+++ b/flink-staging/flink-avro/src/test/java/org/apache/flink/api/io/avro/AvroPojoTest.java
@@ -26,7 +26,6 @@ import org.apache.flink.api.java.ExecutionEnvironment;
 import org.apache.flink.api.java.functions.KeySelector;
 import org.apache.flink.api.java.io.AvroInputFormat;
 import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.typeutils.runtime.PojoSerializer;
 import org.apache.flink.core.fs.Path;
 import org.apache.flink.test.util.MultipleProgramsTestBase;
 import org.apache.flink.util.Collector;

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/DegreesWithExceptionITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/DegreesWithExceptionITCase.java b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/DegreesWithExceptionITCase.java
index b04b24e..955122f 100644
--- a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/DegreesWithExceptionITCase.java
+++ b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/DegreesWithExceptionITCase.java
@@ -47,6 +47,7 @@ public class DegreesWithExceptionITCase {
 			Configuration config = new Configuration();
 			config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, PARALLELISM);
 			cluster = new ForkableFlinkMiniCluster(config, false);
+			cluster.start();
 		}
 		catch (Exception e) {
 			e.printStackTrace();
@@ -72,7 +73,7 @@ public class DegreesWithExceptionITCase {
 	public void testOutDegreesInvalidEdgeSrcId() throws Exception {
 
 		final ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment(
-				"localhost", cluster.getJobManagerRPCPort());
+				"localhost", cluster.getLeaderRPCPort());
 		env.setParallelism(PARALLELISM);
 		env.getConfig().disableSysoutLogging();
 		
@@ -96,7 +97,7 @@ public class DegreesWithExceptionITCase {
 	public void testInDegreesInvalidEdgeTrgId() throws Exception {
 
 		final ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment(
-				"localhost", cluster.getJobManagerRPCPort());
+				"localhost", cluster.getLeaderRPCPort());
 		env.setParallelism(PARALLELISM);
 		env.getConfig().disableSysoutLogging();
 
@@ -120,7 +121,7 @@ public class DegreesWithExceptionITCase {
 	public void testGetDegreesInvalidEdgeTrgId() throws Exception {
 
 		final ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment(
-				"localhost", cluster.getJobManagerRPCPort());
+				"localhost", cluster.getLeaderRPCPort());
 		env.setParallelism(PARALLELISM);
 		env.getConfig().disableSysoutLogging();
 
@@ -144,7 +145,7 @@ public class DegreesWithExceptionITCase {
 	public void testGetDegreesInvalidEdgeSrcId() throws Exception {
 
 		final ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment(
-				"localhost", cluster.getJobManagerRPCPort());
+				"localhost", cluster.getLeaderRPCPort());
 		env.setParallelism(PARALLELISM);
 		env.getConfig().disableSysoutLogging();
 
@@ -168,7 +169,7 @@ public class DegreesWithExceptionITCase {
 	public void testGetDegreesInvalidEdgeSrcTrgId() throws Exception {
 
 		final ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment(
-				"localhost", cluster.getJobManagerRPCPort());
+				"localhost", cluster.getLeaderRPCPort());
 		env.setParallelism(PARALLELISM);
 		env.getConfig().disableSysoutLogging();
 

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnEdgesWithExceptionITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnEdgesWithExceptionITCase.java b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnEdgesWithExceptionITCase.java
index c53227b..ab10947 100644
--- a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnEdgesWithExceptionITCase.java
+++ b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnEdgesWithExceptionITCase.java
@@ -51,6 +51,7 @@ public class ReduceOnEdgesWithExceptionITCase {
 			Configuration config = new Configuration();
 			config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, PARALLELISM);
 			cluster = new ForkableFlinkMiniCluster(config, false);
+			cluster.start();
 		}
 		catch (Exception e) {
 			e.printStackTrace();
@@ -76,7 +77,7 @@ public class ReduceOnEdgesWithExceptionITCase {
 	public void testGroupReduceOnEdgesInvalidEdgeSrcId() throws Exception {
 
 		final ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment(
-				"localhost", cluster.getJobManagerRPCPort());
+				"localhost", cluster.getLeaderRPCPort());
 		env.setParallelism(PARALLELISM);
 		env.getConfig().disableSysoutLogging();
 
@@ -101,7 +102,7 @@ public class ReduceOnEdgesWithExceptionITCase {
 	public void testGroupReduceOnEdgesInvalidEdgeTrgId() throws Exception {
 
 		final ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment(
-				"localhost", cluster.getJobManagerRPCPort());
+				"localhost", cluster.getLeaderRPCPort());
 		env.setParallelism(PARALLELISM);
 		env.getConfig().disableSysoutLogging();
 

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnNeighborsWithExceptionITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnNeighborsWithExceptionITCase.java b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnNeighborsWithExceptionITCase.java
index 21799c7..b32abeb 100644
--- a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnNeighborsWithExceptionITCase.java
+++ b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnNeighborsWithExceptionITCase.java
@@ -52,6 +52,7 @@ public class ReduceOnNeighborsWithExceptionITCase {
 			Configuration config = new Configuration();
 			config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, PARALLELISM);
 			cluster = new ForkableFlinkMiniCluster(config, false);
+			cluster.start();
 		}
 		catch (Exception e) {
 			e.printStackTrace();
@@ -78,7 +79,7 @@ public class ReduceOnNeighborsWithExceptionITCase {
 	public void testGroupReduceOnNeighborsWithVVInvalidEdgeSrcId() throws Exception {
 
 		final ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment(
-				"localhost", cluster.getJobManagerRPCPort());
+				"localhost", cluster.getLeaderRPCPort());
 		env.setParallelism(PARALLELISM);
 		env.getConfig().disableSysoutLogging();
 
@@ -104,7 +105,7 @@ public class ReduceOnNeighborsWithExceptionITCase {
 	public void testGroupReduceOnNeighborsWithVVInvalidEdgeTrgId() throws Exception {
 
 		final ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment(
-				"localhost", cluster.getJobManagerRPCPort());
+				"localhost", cluster.getLeaderRPCPort());
 		env.setParallelism(PARALLELISM);
 		env.getConfig().disableSysoutLogging();
 
@@ -130,7 +131,7 @@ public class ReduceOnNeighborsWithExceptionITCase {
 	public void testGroupReduceOnNeighborsInvalidEdgeSrcId() throws Exception {
 
 		final ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment(
-				"localhost", cluster.getJobManagerRPCPort());
+				"localhost", cluster.getLeaderRPCPort());
 		env.setParallelism(PARALLELISM);
 		env.getConfig().disableSysoutLogging();
 
@@ -156,7 +157,7 @@ public class ReduceOnNeighborsWithExceptionITCase {
 	public void testGroupReduceOnNeighborsInvalidEdgeTrgId() throws Exception {
 
 		final ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment(
-				"localhost", cluster.getJobManagerRPCPort());
+				"localhost", cluster.getLeaderRPCPort());
 		env.setParallelism(PARALLELISM);
 		env.getConfig().disableSysoutLogging();
 

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-staging/flink-hbase/src/test/java/org/apache/flink/addons/hbase/example/HBaseWriteStreamExample.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-hbase/src/test/java/org/apache/flink/addons/hbase/example/HBaseWriteStreamExample.java b/flink-staging/flink-hbase/src/test/java/org/apache/flink/addons/hbase/example/HBaseWriteStreamExample.java
index d1a61d3..a6be1a6 100644
--- a/flink-staging/flink-hbase/src/test/java/org/apache/flink/addons/hbase/example/HBaseWriteStreamExample.java
+++ b/flink-staging/flink-hbase/src/test/java/org/apache/flink/addons/hbase/example/HBaseWriteStreamExample.java
@@ -24,7 +24,6 @@ import org.apache.flink.configuration.Configuration;
 import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.api.functions.source.SourceFunction;
-import org.apache.flink.util.Collector;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.Put;

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-staging/flink-scala-shell/src/main/scala/org.apache.flink/api/scala/FlinkShell.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-scala-shell/src/main/scala/org.apache.flink/api/scala/FlinkShell.scala b/flink-staging/flink-scala-shell/src/main/scala/org.apache.flink/api/scala/FlinkShell.scala
index 7ad1d2f..ad0f655 100644
--- a/flink-staging/flink-scala-shell/src/main/scala/org.apache.flink/api/scala/FlinkShell.scala
+++ b/flink-staging/flink-scala-shell/src/main/scala/org.apache.flink/api/scala/FlinkShell.scala
@@ -67,7 +67,7 @@ object FlinkShell {
     val (host,port) = if (userHost == "none" || userPort == -1 ) {
       println("Creating new local server")
       cluster = new LocalFlinkMiniCluster(new Configuration, false)
-      ("localhost",cluster.getJobManagerRPCPort)
+      ("localhost",cluster.getLeaderRPCPort)
     } else {
       println(s"Connecting to remote server (host: $userHost, port: $userPort).")
       (userHost, userPort)

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-staging/flink-scala-shell/src/test/scala/org/apache/flink/api/scala/ScalaShellITSuite.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-scala-shell/src/test/scala/org/apache/flink/api/scala/ScalaShellITSuite.scala b/flink-staging/flink-scala-shell/src/test/scala/org/apache/flink/api/scala/ScalaShellITSuite.scala
index 01fdaf0..1b04f7f 100644
--- a/flink-staging/flink-scala-shell/src/test/scala/org/apache/flink/api/scala/ScalaShellITSuite.scala
+++ b/flink-staging/flink-scala-shell/src/test/scala/org/apache/flink/api/scala/ScalaShellITSuite.scala
@@ -141,7 +141,7 @@ class ScalaShellITSuite extends FunSuite with Matchers with BeforeAndAfterAll {
     // new local cluster
     val host = "localhost"
     val port = cluster match {
-      case Some(c) => c.getJobManagerRPCPort
+      case Some(c) => c.getLeaderRPCPort
 
       case _ => throw new RuntimeException("Test cluster not initialized.")
     }
@@ -187,7 +187,14 @@ class ScalaShellITSuite extends FunSuite with Matchers with BeforeAndAfterAll {
   val parallelism = 4
 
   override def beforeAll(): Unit = {
-    val cl = TestBaseUtils.startCluster(1, parallelism, StreamingMode.BATCH_ONLY, false, false)
+    val cl = TestBaseUtils.startCluster(
+      1,
+      parallelism,
+      StreamingMode.BATCH_ONLY,
+      false,
+      false,
+      false)
+
     val clusterEnvironment = new TestEnvironment(cl, parallelism)
     clusterEnvironment.setAsContext()
 


[08/10] flink git commit: [FLINK-2291] [runtime] Add ZooKeeper support to elect a leader from a set of JobManager. The leader will then be retrieved from ZooKeeper by the TaskManagers.

Posted by tr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/LeaderElectionService.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/LeaderElectionService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/LeaderElectionService.java
new file mode 100644
index 0000000..b6223ee
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/LeaderElectionService.java
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.leaderelection;
+
+import java.util.UUID;
+
+/**
+ * Interface for a service which allows to elect a leader among a group of contenders.
+ *
+ * Prior to using this service, it has to be started calling the start method. The start method
+ * takes the contender as a parameter. If there are multiple contenders, then each contender has
+ * to instantiate its own leader election service.
+ *
+ * Once a contender has been granted leadership he has to confirm the received leader session ID
+ * by calling the method confirmLeaderSessionID. This will notify the leader election service, that
+ * the contender has received the new leader session ID and that it can now be published for
+ * leader retrieval services.
+ */
+public interface LeaderElectionService {
+
+	/**
+	 * Starts the leader election service. This method can only be called once.
+	 *
+	 * @param contender LeaderContender which applies for the leadership
+	 * @throws Exception
+	 */
+	void start(LeaderContender contender) throws Exception;
+
+	/**
+	 * Stops the leader election service.
+	 * @throws Exception
+	 */
+	void stop() throws Exception;
+
+	/**
+	 * Confirms that the new leader session ID has been successfully received by the new leader.
+	 * This method is usually called by the newly appointed {@link LeaderContender}.
+	 *
+	 * The rational behind this method is to establish an order between setting the new leader
+	 * session ID in the {@link LeaderContender} and publishing the new leader session ID to the
+	 * leader retrieval services.
+	 *
+	 * @param leaderSessionID The new leader session ID
+	 */
+	void confirmLeaderSessionID(UUID leaderSessionID);
+
+	/**
+	 * Returns true if the {@link LeaderContender} with which the service has been started owns
+	 * currently the leadership.
+	 *
+	 * @return true if the associated {@link LeaderContender} is the leader, otherwise false
+	 */
+	boolean hasLeadership();
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/StandaloneLeaderElectionService.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/StandaloneLeaderElectionService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/StandaloneLeaderElectionService.java
new file mode 100644
index 0000000..1a297db
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/StandaloneLeaderElectionService.java
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.leaderelection;
+
+import com.google.common.base.Preconditions;
+
+import java.util.UUID;
+
+/**
+ * Standalone implementation of the {@link LeaderElectionService} interface. The standalone
+ * implementation assumes that there is only a single {@link LeaderContender} and thus directly
+ * grants him the leadership upon start up. Furthermore, there is no communication needed between
+ * multiple standalone leader election services.
+ */
+public class StandaloneLeaderElectionService implements LeaderElectionService {
+
+	private LeaderContender contender = null;
+
+	@Override
+	public void start(LeaderContender newContender) throws Exception {
+		if (contender != null) {
+			// Service was already started
+			throw new IllegalArgumentException("Leader election service cannot be started multiple times.");
+		}
+
+		contender = Preconditions.checkNotNull(newContender);
+
+		// directly grant leadership to the given contender
+		contender.grantLeadership(null);
+	}
+
+	@Override
+	public void stop() {
+		if (contender != null) {
+			contender.revokeLeadership();
+			contender = null;
+		}
+	}
+
+	@Override
+	public void confirmLeaderSessionID(UUID leaderSessionID) {}
+
+	@Override
+	public boolean hasLeadership() {
+		return true;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderElectionService.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderElectionService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderElectionService.java
new file mode 100644
index 0000000..d1fd548
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderElectionService.java
@@ -0,0 +1,265 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.leaderelection;
+
+import com.google.common.base.Preconditions;
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.recipes.cache.ChildData;
+import org.apache.curator.framework.recipes.cache.NodeCache;
+import org.apache.curator.framework.recipes.cache.NodeCacheListener;
+import org.apache.curator.framework.recipes.leader.LeaderLatch;
+import org.apache.curator.framework.recipes.leader.LeaderLatchListener;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.data.Stat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.util.UUID;
+
+/**
+ * Leader election service for multiple JobManager. The leading JobManager is elected using
+ * ZooKeeper. The current leader's address as well as its leader session ID is published via
+ * ZooKeeper as well.
+ */
+public class ZooKeeperLeaderElectionService implements LeaderElectionService, LeaderLatchListener, NodeCacheListener {
+	private static final Logger LOG = LoggerFactory.getLogger(ZooKeeperLeaderElectionService.class);
+
+	/** Client to the ZooKeeper quorum */
+	private final CuratorFramework client;
+
+	/** Curator recipe for leader election */
+	private final LeaderLatch leaderLatch;
+
+	/** Curator recipe to watch a given ZooKeeper node for changes */
+	private final NodeCache cache;
+
+	/** ZooKeeper path of the node which stores the current leader information */
+	private final String leaderPath;
+
+	private UUID issuedLeaderSessionID;
+
+	private UUID confirmedLeaderSessionID;
+
+	/** The leader contender which applies for leadership */
+	private volatile LeaderContender leaderContender;
+
+	private final Object lock = new Object();
+
+	/**
+	 * Creates a ZooKeeperLeaderElectionService object.
+	 *
+	 * @param client Client which is connected to the ZooKeeper quorum
+	 * @param latchPath ZooKeeper node path for the leader election latch
+	 * @param leaderPath ZooKeeper node path for the node which stores the current leader information
+	 */
+	public ZooKeeperLeaderElectionService(CuratorFramework client, String latchPath, String leaderPath) {
+		this.client = client;
+		this.leaderPath = leaderPath;
+
+		leaderLatch = new LeaderLatch(client, latchPath);
+		cache = new NodeCache(client, leaderPath);
+	}
+
+	/**
+	 * Returns the current leader session ID or null, if the contender is not the leader.
+	 *
+	 * @return The last leader session ID or null, if the contender is not the leader
+	 */
+	public UUID getLeaderSessionID() {
+		return confirmedLeaderSessionID;
+	}
+
+	@Override
+	public void start(LeaderContender contender) throws Exception {
+		Preconditions.checkNotNull(contender, "Contender must not be null.");
+		Preconditions.checkState(leaderContender == null, "Contender was already set.");
+
+		LOG.info("Starting ZooKeeperLeaderElectionService.");
+
+		leaderContender = contender;
+
+		leaderLatch.addListener(this);
+		leaderLatch.start();
+
+		cache.getListenable().addListener(this);
+		cache.start();
+	}
+
+	@Override
+	public void stop() throws Exception{
+		LOG.info("Stopping ZooKeeperLeaderElectionService.");
+
+		cache.close();
+		leaderLatch.close();
+		client.close();
+
+		confirmedLeaderSessionID = null;
+		issuedLeaderSessionID = null;
+	}
+
+	@Override
+	public void confirmLeaderSessionID(UUID leaderSessionID) {
+		Preconditions.checkNotNull(leaderSessionID);
+
+		if(leaderLatch.hasLeadership()) {
+			// check if this is an old confirmation call
+			synchronized (lock) {
+				if (leaderSessionID.equals(this.issuedLeaderSessionID)) {
+					confirmedLeaderSessionID = leaderSessionID;
+					writeLeaderInformation(confirmedLeaderSessionID);
+				}
+			}
+		} else {
+			LOG.warn("The leader session ID " + leaderSessionID + " was confirmed even though the" +
+					"corresponding JobManager was not elected as the leader.");
+		}
+	}
+
+	@Override
+	public boolean hasLeadership() {
+		if(leaderLatch.getState().equals(LeaderLatch.State.STARTED)) {
+			return leaderLatch.hasLeadership();
+		} else {
+			return false;
+		}
+	}
+
+	@Override
+	public void isLeader() {
+		synchronized (lock) {
+			issuedLeaderSessionID = UUID.randomUUID();
+			confirmedLeaderSessionID = null;
+			leaderContender.grantLeadership(issuedLeaderSessionID);
+		}
+	}
+
+	@Override
+	public void notLeader() {
+		synchronized (lock) {
+			issuedLeaderSessionID = null;
+			confirmedLeaderSessionID = null;
+			leaderContender.revokeLeadership();
+		}
+	}
+
+	@Override
+	public void nodeChanged() throws Exception {
+		try {
+			// leaderSessionID is null if the leader contender has not yet confirmed the session ID
+			if (leaderLatch.hasLeadership()) {
+				synchronized (lock) {
+					if (confirmedLeaderSessionID != null) {
+						ChildData childData = cache.getCurrentData();
+
+						if (childData == null) {
+							writeLeaderInformation(confirmedLeaderSessionID);
+						} else {
+							byte[] data = childData.getData();
+
+							if (data == null || data.length == 0) {
+								// the data field seems to be empty, rewrite information
+								writeLeaderInformation(confirmedLeaderSessionID);
+							} else {
+								ByteArrayInputStream bais = new ByteArrayInputStream(data);
+								ObjectInputStream ois = new ObjectInputStream(bais);
+
+								String leaderAddress = ois.readUTF();
+								UUID leaderSessionID = (UUID) ois.readObject();
+
+								if (!leaderAddress.equals(this.leaderContender.getAddress()) ||
+										(leaderSessionID == null || !leaderSessionID.equals(confirmedLeaderSessionID))) {
+									// the data field does not correspond to the expected leader information
+									writeLeaderInformation(confirmedLeaderSessionID);
+								}
+							}
+						}
+					}
+				}
+			}
+		} catch (Exception e) {
+			leaderContender.handleError(new Exception("Could not handle node changed event.", e));
+			throw e;
+		}
+	}
+
+	/**
+	 * Writes the current leader's address as well the given leader session ID to ZooKeeper.
+	 *
+	 * @param leaderSessionID Leader session ID which is written to ZooKeeper
+	 */
+	protected void writeLeaderInformation(UUID leaderSessionID) {
+		// this method does not have to be synchronized because the curator framework client
+		// is thread-safe
+		try {
+			ByteArrayOutputStream baos = new ByteArrayOutputStream();
+			ObjectOutputStream oos = new ObjectOutputStream(baos);
+
+			oos.writeUTF(leaderContender.getAddress());
+			oos.writeObject(leaderSessionID);
+
+			oos.close();
+
+			boolean dataWritten = false;
+
+			while(!dataWritten && leaderLatch.hasLeadership()) {
+				Stat stat = client.checkExists().forPath(leaderPath);
+
+				if (stat != null) {
+					long owner = stat.getEphemeralOwner();
+					long sessionID = client.getZookeeperClient().getZooKeeper().getSessionId();
+
+					if(owner == sessionID) {
+						try {
+							client.setData().forPath(leaderPath, baos.toByteArray());
+
+							dataWritten = true;
+						} catch (KeeperException.NoNodeException noNode) {
+							// node was deleted in the meantime
+						}
+					} else {
+						try {
+							client.delete().forPath(leaderPath);
+						} catch (KeeperException.NoNodeException noNode) {
+							// node was deleted in the meantime --> try again
+						}
+					}
+				} else {
+					try {
+						client.create().creatingParentsIfNeeded().withMode(CreateMode.EPHEMERAL).forPath(
+								leaderPath,
+								baos.toByteArray());
+
+						dataWritten = true;
+					} catch (KeeperException.NodeExistsException nodeExists) {
+						// node has been created in the meantime --> try again
+					}
+				}
+			}
+		} catch (Exception e) {
+			leaderContender.handleError(
+					new Exception("Could not write leader address and leader session ID to " +
+							"ZooKeeper.", e));
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/main/java/org/apache/flink/runtime/leaderretrieval/LeaderRetrievalException.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/leaderretrieval/LeaderRetrievalException.java b/flink-runtime/src/main/java/org/apache/flink/runtime/leaderretrieval/LeaderRetrievalException.java
new file mode 100644
index 0000000..8d2a9b5
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/leaderretrieval/LeaderRetrievalException.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.leaderretrieval;
+
+/**
+ * This exception is thrown by the {@link org.apache.flink.runtime.util.LeaderRetrievalUtils} when
+ * the method retrieveLeaderGateway fails to retrieve the current leader's gateway.
+ */
+public class LeaderRetrievalException extends Exception {
+
+	private static final long serialVersionUID = 42;
+
+	public LeaderRetrievalException(String message) {
+		super(message);
+	}
+
+	public LeaderRetrievalException(Throwable cause) {
+		super(cause);
+	}
+
+	public LeaderRetrievalException(String message, Throwable cause) {
+		super(message, cause);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/main/java/org/apache/flink/runtime/leaderretrieval/LeaderRetrievalListener.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/leaderretrieval/LeaderRetrievalListener.java b/flink-runtime/src/main/java/org/apache/flink/runtime/leaderretrieval/LeaderRetrievalListener.java
new file mode 100644
index 0000000..b5ba4e9
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/leaderretrieval/LeaderRetrievalListener.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.leaderretrieval;
+
+import java.util.UUID;
+
+/**
+ * Classes which want to be notified about a changing leader by the {@link LeaderRetrievalService}
+ * have to implement this interface.
+ */
+public interface LeaderRetrievalListener {
+
+	/**
+	 * This method is called by the {@link LeaderRetrievalService} when a new leader is elected.
+	 *
+	 * @param leaderAddress The address of the new leader
+	 * @param leaderSessionID The new leader session ID
+	 */
+	void notifyLeaderAddress(String leaderAddress, UUID leaderSessionID);
+
+	/**
+	 * This method is called by the {@link LeaderRetrievalService} in case of an exception. This
+	 * assures that the {@link LeaderRetrievalListener} is aware of any problems occurring in the
+	 * {@link LeaderRetrievalService} thread.
+	 * @param exception
+	 */
+	void handleError(Exception exception);
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/main/java/org/apache/flink/runtime/leaderretrieval/LeaderRetrievalService.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/leaderretrieval/LeaderRetrievalService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/leaderretrieval/LeaderRetrievalService.java
new file mode 100644
index 0000000..97f8843
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/leaderretrieval/LeaderRetrievalService.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.leaderretrieval;
+
+/**
+ * This interface has to be implemented by a service which retrieves the current leader and notifies
+ * a listener about it.
+ *
+ * Prior to using this service it has to be started by calling the start method. The start method
+ * also takes the {@link LeaderRetrievalListener} as an argument. The service can only be started
+ * once.
+ *
+ * The service should be stopped by calling the stop method.
+ */
+public interface LeaderRetrievalService {
+
+	/**
+	 * Starts the leader retrieval service with the given listener to listen for new leaders. This
+	 * method can only be called once.
+	 *
+	 * @param listener The leader retrieval listener which will be notified about new leaders.
+	 * @throws Exception
+	 */
+	void start(LeaderRetrievalListener listener) throws Exception;
+
+	/**
+	 * Stops the leader retrieval service.
+	 *
+	 * @throws Exception
+	 */
+	void stop() throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/main/java/org/apache/flink/runtime/leaderretrieval/StandaloneLeaderRetrievalService.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/leaderretrieval/StandaloneLeaderRetrievalService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/leaderretrieval/StandaloneLeaderRetrievalService.java
new file mode 100644
index 0000000..dbab41c
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/leaderretrieval/StandaloneLeaderRetrievalService.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.leaderretrieval;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * Standalone implementation of the {@link LeaderRetrievalService}. The standalone implementation
+ * assumes that there is only a single {@link org.apache.flink.runtime.jobmanager.JobManager} whose
+ * address is given to the service when creating it. This address is directly given to the
+ * {@link LeaderRetrievalListener} when the service is started.
+ */
+public class StandaloneLeaderRetrievalService implements LeaderRetrievalService {
+
+	/** Address of the only JobManager */
+	private final String jobManagerAddress;
+
+	/** Listener which wants to be notified about the new leader */
+	private LeaderRetrievalListener leaderListener;
+
+	/**
+	 * Creates a StandaloneLeaderRetrievalService with the given JobManager address.
+	 *
+	 * @param jobManagerAddress The JobManager's address which is returned to the
+	 * 							{@link LeaderRetrievalListener}
+	 */
+	public StandaloneLeaderRetrievalService(String jobManagerAddress) {
+		this.jobManagerAddress = jobManagerAddress;
+	}
+
+	public void start(LeaderRetrievalListener listener) {
+		Preconditions.checkNotNull(listener, "Listener must not be null.");
+		Preconditions.checkState(leaderListener == null, "StandaloneLeaderRetrievalService can " +
+				"only be started once.");
+
+		leaderListener = listener;
+
+		// directly notify the listener, because we already know the leading JobManager's address
+		leaderListener.notifyLeaderAddress(jobManagerAddress, null);
+	}
+
+	public void stop() {}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/main/java/org/apache/flink/runtime/leaderretrieval/ZooKeeperLeaderRetrievalService.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/leaderretrieval/ZooKeeperLeaderRetrievalService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/leaderretrieval/ZooKeeperLeaderRetrievalService.java
new file mode 100644
index 0000000..20ed4d3
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/leaderretrieval/ZooKeeperLeaderRetrievalService.java
@@ -0,0 +1,126 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.leaderretrieval;
+
+import com.google.common.base.Preconditions;
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.recipes.cache.ChildData;
+import org.apache.curator.framework.recipes.cache.NodeCache;
+import org.apache.curator.framework.recipes.cache.NodeCacheListener;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayInputStream;
+import java.io.ObjectInputStream;
+import java.util.Objects;
+import java.util.UUID;
+
+/**
+ * The counterpart to the {@link org.apache.flink.runtime.leaderelection.ZooKeeperLeaderElectionService}.
+ * This implementation of the {@link LeaderRetrievalService} retrieves the current leader which has
+ * been elected by the {@link org.apache.flink.runtime.leaderelection.ZooKeeperLeaderElectionService}.
+ * The leader address as well as the current leader session ID is retrieved from ZooKeeper.
+ */
+public class ZooKeeperLeaderRetrievalService implements LeaderRetrievalService, NodeCacheListener {
+	private static final Logger LOG = LoggerFactory.getLogger(
+		ZooKeeperLeaderRetrievalService.class);
+
+	/** Connection to the used ZooKeeper quorum */
+	private final CuratorFramework client;
+
+	/** Curator recipe to watch changes of a specific ZooKeeper node */
+	private final NodeCache cache;
+
+	/** Listener which will be notified about leader changes */
+	private volatile LeaderRetrievalListener leaderListener;
+
+	private String lastLeaderAddress;
+	private UUID lastLeaderSessionID;
+
+	/**
+	 * Creates a leader retrieval service which uses ZooKeeper to retrieve the leader information.
+	 *
+	 * @param client Client which constitutes the connection to the ZooKeeper quorum
+	 * @param retrievalPath Path of the ZooKeeper node which contains the leader information
+	 */
+	public ZooKeeperLeaderRetrievalService(CuratorFramework client, String retrievalPath) {
+		this.client = client;
+		this.cache = new NodeCache(client, retrievalPath);
+	}
+
+	@Override
+	public void start(LeaderRetrievalListener listener) throws Exception {
+		Preconditions.checkNotNull(listener, "Listener must not be null.");
+		Preconditions.checkState(leaderListener == null, "ZooKeeperLeaderRetrievalService can " +
+				"only be started once.");
+
+		LOG.info("Starting ZooKeeperLeaderRetrievalService.");
+
+		leaderListener = listener;
+
+		cache.getListenable().addListener(this);
+		cache.start();
+	}
+
+	@Override
+	public void stop() throws Exception {
+		LOG.info("Stopping ZooKeeperLeaderRetrievalService.");
+
+		cache.close();
+		client.close();
+	}
+
+	@Override
+	public void nodeChanged() throws Exception {
+		try {
+			ChildData childData = cache.getCurrentData();
+
+			String leaderAddress;
+			UUID leaderSessionID;
+
+			if (childData == null) {
+				leaderAddress = null;
+				leaderSessionID = null;
+			} else {
+				byte[] data = childData.getData();
+
+				if (data == null || data.length == 0) {
+					leaderAddress = null;
+					leaderSessionID = null;
+				} else {
+					ByteArrayInputStream bais = new ByteArrayInputStream(data);
+					ObjectInputStream ois = new ObjectInputStream(bais);
+
+					leaderAddress = ois.readUTF();
+					leaderSessionID = (UUID) ois.readObject();
+				}
+			}
+
+			if(!(Objects.equals(leaderAddress, lastLeaderAddress) &&
+					Objects.equals(leaderSessionID, lastLeaderSessionID))) {
+				lastLeaderAddress = leaderAddress;
+				lastLeaderSessionID = leaderSessionID;
+				leaderListener.notifyLeaderAddress(leaderAddress, leaderSessionID);
+			}
+		} catch (Exception e) {
+			leaderListener.handleError(new Exception("Could not handle node changed event.", e));
+			throw e;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/main/java/org/apache/flink/runtime/messages/LeaderSessionMessageDecorator.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/messages/LeaderSessionMessageDecorator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/messages/LeaderSessionMessageDecorator.java
new file mode 100644
index 0000000..082fdd4
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/messages/LeaderSessionMessageDecorator.java
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.messages;
+
+import java.util.UUID;
+
+/**
+ * Message decorator which wraps message which implement {@link RequiresLeaderSessionID} into
+ * a {@link org.apache.flink.runtime.messages.JobManagerMessages.LeaderSessionMessage}.
+ */
+public class LeaderSessionMessageDecorator implements MessageDecorator {
+
+	/** Leader session ID with which the RequiresLeaderSessionID messages will be decorated */
+	private final UUID leaderSessionID;
+
+	/**
+	 * Sets the leader session ID with which the messages will be decorated.
+	 *
+	 * @param leaderSessionID Leader session ID to be used for decoration
+	 */
+	public LeaderSessionMessageDecorator(UUID leaderSessionID) {
+		this.leaderSessionID = leaderSessionID;
+	}
+
+	@Override
+	public Object decorate(Object message) {
+		if (message instanceof RequiresLeaderSessionID) {
+			return new JobManagerMessages.LeaderSessionMessage(leaderSessionID, message);
+		} else {
+			return message;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/main/java/org/apache/flink/runtime/messages/MessageDecorator.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/messages/MessageDecorator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/messages/MessageDecorator.java
new file mode 100644
index 0000000..25ee47b
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/messages/MessageDecorator.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.messages;
+
+/**
+ * Interface for message decorators
+ */
+public interface MessageDecorator {
+
+	/**
+	 * Decorates a message
+	 *
+	 * @param message Message to decorate
+	 * @return Decorated message
+	 */
+	Object decorate(Object message);
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/main/java/org/apache/flink/runtime/net/NetUtils.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/net/NetUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/net/NetUtils.java
index 94073db..46c07fa 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/net/NetUtils.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/net/NetUtils.java
@@ -27,9 +27,15 @@ import java.net.ServerSocket;
 import java.net.Socket;
 import java.net.SocketAddress;
 import java.util.Enumeration;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
 
+import org.apache.flink.runtime.akka.AkkaUtils;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalException;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import scala.concurrent.duration.FiniteDuration;
 
 /**
  * Utilities to determine the network interface and address that should be used to bind the
@@ -396,4 +402,155 @@ public class NetUtils {
 
 		throw new RuntimeException("Could not find a free permitted port on the machine.");
 	}
+
+	public static class LeaderConnectingAddressListener implements LeaderRetrievalListener {
+
+		private static final FiniteDuration defaultLoggingDelay = new FiniteDuration(400, TimeUnit.MILLISECONDS);
+
+		private enum LeaderRetrievalState {
+			NOT_RETRIEVED,
+			RETRIEVED,
+			NEWLY_RETRIEVED
+		}
+
+		final private Object retrievalLock = new Object();
+
+		private String akkaURL;
+		private LeaderRetrievalState retrievalState = LeaderRetrievalState.NOT_RETRIEVED;
+		private Exception exception;
+
+		public InetAddress findConnectingAddress(
+				FiniteDuration timeout) throws LeaderRetrievalException {
+			return findConnectingAddress(timeout, defaultLoggingDelay);
+		}
+
+		public InetAddress findConnectingAddress(
+				FiniteDuration timeout,
+				FiniteDuration startLoggingAfter)
+			throws LeaderRetrievalException {
+			long startTime = System.currentTimeMillis();
+			long currentSleepTime = MIN_SLEEP_TIME;
+			long elapsedTime = 0;
+			InetSocketAddress targetAddress = null;
+
+			try {
+				while (elapsedTime < timeout.toMillis()) {
+
+					long maxTimeout = timeout.toMillis() - elapsedTime;
+
+					synchronized (retrievalLock) {
+						if (exception != null) {
+							throw exception;
+						}
+
+						if (retrievalState == LeaderRetrievalState.NOT_RETRIEVED) {
+							try {
+								retrievalLock.wait(maxTimeout);
+							} catch (InterruptedException e) {
+								throw new Exception("Finding connecting address was interrupted" +
+										"while waiting for the leader retrieval.");
+							}
+						} else if (retrievalState == LeaderRetrievalState.NEWLY_RETRIEVED) {
+							targetAddress = AkkaUtils.getInetSockeAddressFromAkkaURL(akkaURL);
+
+							LOG.info("Retrieved new target address {}.", targetAddress);
+
+							retrievalState = LeaderRetrievalState.RETRIEVED;
+
+							currentSleepTime = MIN_SLEEP_TIME;
+						} else {
+							currentSleepTime = Math.min(2 * currentSleepTime, MAX_SLEEP_TIME);
+						}
+					}
+
+					if (targetAddress != null) {
+						AddressDetectionState strategy = AddressDetectionState.ADDRESS;
+
+						boolean logging = elapsedTime >= startLoggingAfter.toMillis();
+						if (logging) {
+							LOG.info("Trying to connect to address {}." + targetAddress);
+						}
+
+						do {
+							InetAddress address = NetUtils.findAddressUsingStrategy(strategy, targetAddress, logging);
+							if (address != null) {
+								return address;
+							}
+
+							// pick the next strategy
+							switch (strategy) {
+								case ADDRESS:
+									strategy = AddressDetectionState.FAST_CONNECT;
+									break;
+								case FAST_CONNECT:
+									strategy = AddressDetectionState.SLOW_CONNECT;
+									break;
+								case SLOW_CONNECT:
+									strategy = null;
+									break;
+								default:
+									throw new RuntimeException("Unsupported strategy: " + strategy);
+							}
+						}
+						while (strategy != null);
+					}
+
+					elapsedTime = System.currentTimeMillis() - startTime;
+
+					long timeToWait = Math.min(
+							Math.max(timeout.toMillis() - elapsedTime, 0),
+							currentSleepTime);
+
+					if (timeToWait > 0) {
+						synchronized (retrievalLock) {
+							try {
+								retrievalLock.wait(timeToWait);
+							} catch (InterruptedException e) {
+								throw new Exception("Finding connecting address was interrupted while pausing.");
+							}
+						}
+
+						elapsedTime = System.currentTimeMillis() - startTime;
+					}
+				}
+
+				InetAddress heuristic = null;
+
+				if (targetAddress != null) {
+					LOG.warn("Could not connect to {}. Selecting a local address using heuristics.", targetAddress);
+					heuristic = findAddressUsingStrategy(AddressDetectionState.HEURISTIC, targetAddress, true);
+				}
+
+				if (heuristic != null) {
+					return heuristic;
+				} else {
+					LOG.warn("Could not find any IPv4 address that is not loopback or link-local. Using localhost address.");
+					return InetAddress.getLocalHost();
+				}
+			} catch (Exception e) {
+				throw new LeaderRetrievalException("Could not retrieve the connecting address to the " +
+						"current leader with the akka URL " + akkaURL + ".", e);
+			}
+		}
+
+		@Override
+		public void notifyLeaderAddress(String leaderAddress, UUID leaderSessionID) {
+			if (leaderAddress != null && !leaderAddress.equals("")) {
+				synchronized (retrievalLock) {
+					akkaURL = leaderAddress;
+					retrievalState = LeaderRetrievalState.NEWLY_RETRIEVED;
+
+					retrievalLock.notifyAll();
+				}
+			}
+		}
+
+		@Override
+		public void handleError(Exception exception) {
+			synchronized (retrievalLock) {
+				this.exception = exception;
+				retrievalLock.notifyAll();
+			}
+		}
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java
index c7abce0..a19a57a 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java
@@ -51,8 +51,8 @@ import org.apache.flink.runtime.jobgraph.tasks.CheckpointedOperator;
 import org.apache.flink.runtime.jobgraph.tasks.OperatorStateCarrier;
 import org.apache.flink.runtime.memorymanager.MemoryManager;
 import org.apache.flink.runtime.messages.TaskManagerMessages.FatalError;
-import org.apache.flink.runtime.messages.TaskMessages;
 import org.apache.flink.runtime.messages.TaskMessages.TaskInFinalState;
+import org.apache.flink.runtime.messages.TaskMessages.UpdateTaskExecutionState;
 import org.apache.flink.runtime.state.StateHandle;
 import org.apache.flink.runtime.state.StateUtils;
 import org.apache.flink.util.SerializedValue;
@@ -571,7 +571,7 @@ public class Task implements Runnable {
 			// notify everyone that we switched to running. especially the TaskManager needs
 			// to know this!
 			notifyObservers(ExecutionState.RUNNING, null);
-			taskManager.tell(new TaskMessages.UpdateTaskExecutionState(
+			taskManager.tell(new UpdateTaskExecutionState(
 					new TaskExecutionState(jobId, executionId, ExecutionState.RUNNING)));
 
 			// make sure the user code classloader is accessible thread-locally
@@ -856,8 +856,7 @@ public class Task implements Runnable {
 		}
 
 		TaskExecutionState stateUpdate = new TaskExecutionState(jobId, executionId, newState, error);
-		TaskMessages.UpdateTaskExecutionState actorMessage = new
-				TaskMessages.UpdateTaskExecutionState(stateUpdate);
+		UpdateTaskExecutionState actorMessage = new UpdateTaskExecutionState(stateUpdate);
 
 		for (ActorGateway listener : executionListenerActors) {
 			listener.tell(actorMessage);

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/main/java/org/apache/flink/runtime/util/LeaderConnectionInfo.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/LeaderConnectionInfo.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/LeaderConnectionInfo.java
new file mode 100644
index 0000000..aee023a
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/LeaderConnectionInfo.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.util;
+
+import java.util.UUID;
+
+/**
+ * Wrapper class for a pair of connection address and leader session ID.
+ */
+public class LeaderConnectionInfo {
+
+	private final String address;
+
+	private final UUID leaderSessionID;
+
+	public LeaderConnectionInfo(String address, UUID leaderSessionID) {
+		this.address = address;
+		this.leaderSessionID = leaderSessionID;
+	}
+
+	public String getAddress() {
+		return address;
+	}
+
+	public UUID getLeaderSessionID() {
+		return leaderSessionID;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/main/java/org/apache/flink/runtime/util/LeaderElectionUtils.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/LeaderElectionUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/LeaderElectionUtils.java
new file mode 100644
index 0000000..76657d3
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/LeaderElectionUtils.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.util;
+
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.jobmanager.RecoveryMode;
+import org.apache.flink.runtime.leaderelection.LeaderElectionService;
+import org.apache.flink.runtime.leaderelection.StandaloneLeaderElectionService;
+
+public class LeaderElectionUtils {
+
+	/**
+	 * Creates a {@link LeaderElectionService} based on the provided {@link Configuration} object.
+	 *
+	 * @param configuration Configuration object
+	 * @return {@link LeaderElectionService} which was created based on the provided Configuration
+	 * @throws Exception
+	 */
+	public static LeaderElectionService createLeaderElectionService(Configuration configuration) throws Exception {
+		RecoveryMode recoveryMode = RecoveryMode.valueOf(configuration.getString(
+				ConfigConstants.RECOVERY_MODE,
+				ConfigConstants.DEFAULT_RECOVERY_MODE).toUpperCase()
+		);
+
+		LeaderElectionService leaderElectionService;
+
+		switch(recoveryMode) {
+			case STANDALONE:
+				leaderElectionService = new StandaloneLeaderElectionService();
+				break;
+			case ZOOKEEPER:
+				leaderElectionService = ZooKeeperUtils.createLeaderElectionService(configuration);
+				break;
+			default:
+				throw new Exception("Unknown RecoveryMode " + recoveryMode);
+		}
+
+		return leaderElectionService;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/main/java/org/apache/flink/runtime/util/LeaderRetrievalUtils.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/LeaderRetrievalUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/LeaderRetrievalUtils.java
new file mode 100644
index 0000000..201db03
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/LeaderRetrievalUtils.java
@@ -0,0 +1,244 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.util;
+
+import akka.actor.ActorRef;
+import akka.actor.ActorSystem;
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.akka.AkkaUtils;
+import org.apache.flink.runtime.instance.ActorGateway;
+import org.apache.flink.runtime.instance.AkkaActorGateway;
+import org.apache.flink.runtime.jobmanager.RecoveryMode;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalException;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+import org.apache.flink.runtime.net.NetUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.concurrent.Await;
+import scala.concurrent.Future;
+import scala.concurrent.Promise;
+import scala.concurrent.duration.FiniteDuration;
+
+import java.net.InetAddress;
+import java.util.UUID;
+
+public class LeaderRetrievalUtils {
+
+	private static final Logger LOG = LoggerFactory.getLogger(LeaderRetrievalUtils.class);
+
+	/**
+	 * Creates a {@link LeaderRetrievalService} based on the provided {@link Configuration} object.
+	 *
+	 * @param configuration Configuration containing the settings for the {@link LeaderRetrievalService}
+	 * @return The {@link LeaderRetrievalService} specified in the configuration object
+	 * @throws Exception
+	 */
+	public static LeaderRetrievalService createLeaderRetrievalService(Configuration configuration)
+		throws Exception {
+
+		RecoveryMode recoveryMode = RecoveryMode.valueOf(
+				configuration.getString(
+						ConfigConstants.RECOVERY_MODE,
+						ConfigConstants.DEFAULT_RECOVERY_MODE).toUpperCase());
+
+		switch (recoveryMode) {
+			case STANDALONE:
+				return StandaloneUtils.createLeaderRetrievalService(configuration);
+			case ZOOKEEPER:
+				return ZooKeeperUtils.createLeaderRetrievalService(configuration);
+			default:
+				throw new Exception("Recovery mode " + recoveryMode + " is not supported.");
+		}
+	}
+
+	/**
+	 * Retrieves the current leader gateway using the given {@link LeaderRetrievalService}. If the
+	 * current leader could not be retrieved after the given timeout, then a
+	 * {@link LeaderRetrievalException} is thrown.
+	 *
+	 * @param leaderRetrievalService {@link LeaderRetrievalService} which is used for the leader retrieval
+	 * @param actorSystem ActorSystem which is used for the {@link LeaderRetrievalListener} implementation
+	 * @param timeout Timeout value for the retrieval call
+	 * @return The current leader gateway
+	 * @throws LeaderRetrievalException If the actor gateway could not be retrieved or the timeout has been exceeded
+	 */
+	public static ActorGateway retrieveLeaderGateway(
+			LeaderRetrievalService leaderRetrievalService,
+			ActorSystem actorSystem,
+			FiniteDuration timeout)
+		throws LeaderRetrievalException {
+		LeaderGatewayListener listener = new LeaderGatewayListener(actorSystem, timeout);
+
+		try {
+			leaderRetrievalService.start(listener);
+
+			Future<ActorGateway> actorGatewayFuture = listener.getActorGatewayFuture();
+
+			ActorGateway gateway = Await.result(actorGatewayFuture, timeout);
+
+			return gateway;
+		} catch (Exception e) {
+			throw new LeaderRetrievalException("Could not retrieve the leader gateway", e);
+		} finally {
+			try {
+				leaderRetrievalService.stop();
+			} catch (Exception fe) {
+				LOG.warn("Could not stop the leader retrieval service.", fe);
+			}
+		}
+	}
+
+	/**
+	 * Retrieves the leader akka url and the current leader session ID. The values are stored in a
+	 * {@link LeaderConnectionInfo} instance.
+	 *
+	 * @param leaderRetrievalService Leader retrieval service to retrieve the leader connection
+	 *                               information
+	 * @param timeout Timeout when to give up looking for the leader
+	 * @return LeaderConnectionInfo containing the leader's akka URL and the current leader session
+	 * ID
+	 * @throws LeaderRetrievalException
+	 */
+	public static LeaderConnectionInfo retrieveLeaderConnectionInfo(
+			LeaderRetrievalService leaderRetrievalService,
+			FiniteDuration timeout
+	) throws LeaderRetrievalException {
+		LeaderConnectionInfoListener listener = new LeaderConnectionInfoListener();
+
+		try {
+			leaderRetrievalService.start(listener);
+
+			Future<LeaderConnectionInfo> connectionInfoFuture = listener.getLeaderConnectionInfoFuture();
+
+			LeaderConnectionInfo result = Await.result(connectionInfoFuture, timeout);
+
+			return result;
+		} catch (Exception e) {
+			throw new LeaderRetrievalException("Could not retrieve the leader address and leader " +
+					"session ID.", e);
+		} finally {
+			try {
+				leaderRetrievalService.stop();
+			} catch (Exception fe) {
+				LOG.warn("Could not stop the leader retrieval service.", fe);
+			}
+		}
+	}
+
+	public static InetAddress findConnectingAddress(
+			LeaderRetrievalService leaderRetrievalService,
+			FiniteDuration timeout) throws LeaderRetrievalException {
+		NetUtils.LeaderConnectingAddressListener listener = new NetUtils.LeaderConnectingAddressListener();
+
+		try {
+			leaderRetrievalService.start(listener);
+
+			LOG.info("Trying to select the network interface and address to use " +
+					"by connecting to the leading JobManager.");
+
+			LOG.info("TaskManager will try to connect for " + timeout +
+					" before falling back to heuristics");
+
+			InetAddress result =  listener.findConnectingAddress(timeout);
+
+			return result;
+		} catch (Exception e) {
+			throw new LeaderRetrievalException("Could not find the connecting address by " +
+					"connecting to the current leader.", e);
+		} finally {
+			try {
+				leaderRetrievalService.stop();
+			} catch (Exception fe) {
+				LOG.warn("Could not stop the leader retrieval service.", fe);
+			}
+		}
+	}
+
+	/**
+	 * Helper class which is used by the retrieveLeaderGateway method as the
+	 * {@link LeaderRetrievalListener}.
+	 */
+	public static class LeaderGatewayListener implements LeaderRetrievalListener {
+
+		private final ActorSystem actorSystem;
+		private final FiniteDuration timeout;
+
+		private final Promise<ActorGateway> futureActorGateway = new scala.concurrent.impl.Promise.DefaultPromise<ActorGateway>();
+
+		public LeaderGatewayListener(ActorSystem actorSystem, FiniteDuration timeout) {
+			this.actorSystem = actorSystem;
+			this.timeout = timeout;
+		}
+
+		@Override
+		public void notifyLeaderAddress(String leaderAddress, UUID leaderSessionID) {
+			if(leaderAddress != null && !leaderAddress.equals("") && !futureActorGateway.isCompleted()) {
+				try {
+					ActorRef actorRef = AkkaUtils.getActorRef(leaderAddress, actorSystem, timeout);
+
+					ActorGateway gateway = new AkkaActorGateway(actorRef, leaderSessionID);
+
+					futureActorGateway.success(gateway);
+
+				} catch(Exception e){
+					futureActorGateway.failure(e);
+				}
+			}
+		}
+
+		@Override
+		public void handleError(Exception exception) {
+			if (!futureActorGateway.isCompleted()) {
+				futureActorGateway.failure(exception);
+			}
+		}
+
+		public Future<ActorGateway> getActorGatewayFuture() {
+			return futureActorGateway.future();
+		}
+	}
+
+	/**
+	 * Helper class which is used by the retrieveLeaderConnectionInfo method to retrieve the
+	 * leader's akka URL and the current leader session ID.
+	 */
+	public static class LeaderConnectionInfoListener implements  LeaderRetrievalListener {
+		private final Promise<LeaderConnectionInfo> connectionInfo = new scala.concurrent.impl.Promise.DefaultPromise<>();
+
+		public Future<LeaderConnectionInfo> getLeaderConnectionInfoFuture() {
+			return connectionInfo.future();
+		}
+
+		@Override
+		public void notifyLeaderAddress(String leaderAddress, UUID leaderSessionID) {
+			if(leaderAddress != null && !leaderAddress.equals("") && !connectionInfo.isCompleted()) {
+				connectionInfo.success(new LeaderConnectionInfo(leaderAddress, leaderSessionID));
+			}
+		}
+
+		@Override
+		public void handleError(Exception exception) {
+			if (!connectionInfo.isCompleted()) {
+				connectionInfo.failure(exception);
+			}
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/main/java/org/apache/flink/runtime/util/StandaloneUtils.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/StandaloneUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/StandaloneUtils.java
new file mode 100644
index 0000000..ebb7965
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/StandaloneUtils.java
@@ -0,0 +1,84 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.util;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.jobmanager.JobManager;
+import org.apache.flink.runtime.leaderretrieval.StandaloneLeaderRetrievalService;
+import org.apache.flink.runtime.taskmanager.TaskManager;
+import scala.Option;
+import scala.Tuple2;
+
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.UnknownHostException;
+
+public class StandaloneUtils {
+
+	/**
+	 * Creates a {@link StandaloneLeaderRetrievalService} from the given configuration. The
+	 * host and port for the remote Akka URL are retrieved from the provided configuration.
+	 *
+	 * @param configuration Configuration instance containing the host and port information
+	 * @return StandaloneLeaderRetrievalService
+	 * @throws UnknownHostException
+	 */
+	public static StandaloneLeaderRetrievalService createLeaderRetrievalService(
+			Configuration configuration)
+		throws UnknownHostException {
+		return createLeaderRetrievalService(configuration, null);
+	}
+
+	/**
+	 * Creates a {@link StandaloneLeaderRetrievalService} form the given configuration and the
+	 * JobManager name. The host and port for the remote Akka URL are retrieved from the provided
+	 * configuration. Instead of using the standard JobManager Akka name, the provided one is used
+	 * for the remote Akka URL.
+	 *
+	 * @param configuration Configuration instance containing hte host and port information
+	 * @param jobManagerName Name of the JobManager actor
+	 * @return StandaloneLeaderRetrievalService
+	 * @throws UnknownHostException
+	 */
+	public static StandaloneLeaderRetrievalService createLeaderRetrievalService(
+			Configuration configuration,
+			String jobManagerName)
+		throws UnknownHostException {
+		Tuple2<String, Object> stringIntPair = TaskManager.getAndCheckJobManagerAddress(configuration);
+
+		String jobManagerHostname = stringIntPair._1();
+		int jobManagerPort = (Integer) stringIntPair._2();
+		InetSocketAddress hostPort;
+
+		try {
+			InetAddress inetAddress = InetAddress.getByName(jobManagerHostname);
+			hostPort = new InetSocketAddress(inetAddress, jobManagerPort);
+		}
+		catch (UnknownHostException e) {
+			throw new UnknownHostException("Cannot resolve the JobManager hostname '" + jobManagerHostname
+					+ "' specified in the configuration");
+		}
+
+		String jobManagerAkkaUrl = JobManager.getRemoteJobManagerAkkaURL(
+				hostPort,
+				Option.apply(jobManagerName));
+
+		return new StandaloneLeaderRetrievalService(jobManagerAkkaUrl);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/main/java/org/apache/flink/runtime/util/ZooKeeperUtil.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/ZooKeeperUtil.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/ZooKeeperUtil.java
deleted file mode 100644
index c3d9df4..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/util/ZooKeeperUtil.java
+++ /dev/null
@@ -1,110 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.runtime.util;
-
-import org.apache.curator.framework.CuratorFramework;
-import org.apache.curator.framework.CuratorFrameworkFactory;
-import org.apache.curator.retry.ExponentialBackoffRetry;
-import org.apache.flink.configuration.ConfigConstants;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.configuration.IllegalConfigurationException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * ZooKeeper utilities.
- */
-public class ZooKeeperUtil {
-
-	private static final Logger LOG = LoggerFactory.getLogger(ZooKeeperUtil.class);
-
-	public static CuratorFramework createCuratorFramework(Configuration configuration) throws Exception {
-		String zkQuorum = ZooKeeperUtil.getZooKeeperEnsemble(configuration);
-
-		if (zkQuorum == null || zkQuorum.equals("")) {
-			throw new RuntimeException("No valid ZooKeeper quorum has been specified.");
-		}
-
-		int sessionTimeout = configuration.getInteger(
-				ConfigConstants.ZOOKEEPER_SESSION_TIMEOUT,
-				ConfigConstants.DEFAULT_ZOOKEEPER_SESSION_TIMEOUT);
-
-		int connectionTimeout = configuration.getInteger(
-				ConfigConstants.ZOOKEEPER_CONNECTION_TIMEOUT,
-				ConfigConstants.DEFAULT_ZOOKEEPER_CONNECTION_TIMEOUT);
-
-		int retryWait = configuration.getInteger(
-				ConfigConstants.ZOOKEEPER_RETRY_WAIT,
-				ConfigConstants.DEFAULT_ZOOKEEPER_RETRY_WAIT);
-
-		int maxRetryAttempts = configuration.getInteger(
-				ConfigConstants.ZOOKEEPER_MAX_RETRY_ATTEMPTS,
-				ConfigConstants.DEFAULT_ZOOKEEPER_MAX_RETRY_ATTEMPTS);
-
-		String root = configuration.getString(ConfigConstants.ZOOKEEPER_DIR_KEY,
-				ConfigConstants.DEFAULT_ZOOKEEPER_ZNODE_ROOT);
-
-		LOG.info("Using '{}' as root namespace.", root);
-
-		CuratorFramework cf = CuratorFrameworkFactory.builder()
-				.connectString(zkQuorum)
-				.sessionTimeoutMs(sessionTimeout)
-				.connectionTimeoutMs(connectionTimeout)
-				.retryPolicy(new ExponentialBackoffRetry(retryWait, maxRetryAttempts))
-				// Curator prepends a '/' manually and throws an Exception if the
-				// namespace starts with a '/'.
-				.namespace(root.startsWith("/") ? root.substring(1) : root)
-				.build();
-
-		try {
-			cf.start();
-		}
-		catch (Exception e) {
-			throw new Exception("Could not start CuratorFramework.", e);
-		}
-
-		return cf;
-	}
-
-	/**
-	 * Returns whether high availability is enabled (<=> ZooKeeper quorum configured).
-	 */
-	public static boolean isJobManagerHighAvailabilityEnabled(Configuration flinkConf) {
-		return flinkConf.containsKey(ConfigConstants.ZOOKEEPER_QUORUM_KEY);
-	}
-
-	/**
-	 * Returns the configured ZooKeeper quorum (and removes whitespace, because ZooKeeper does not
-	 * tolerate it).
-	 */
-	public static String getZooKeeperEnsemble(Configuration flinkConf)
-			throws IllegalConfigurationException {
-
-		String zkQuorum = flinkConf.getString(ConfigConstants.ZOOKEEPER_QUORUM_KEY, null);
-
-		if (zkQuorum == null || zkQuorum.equals("")) {
-			throw new IllegalConfigurationException("No ZooKeeper quorum specified in config.");
-		}
-
-		// Remove all whitespace
-		zkQuorum = zkQuorum.replaceAll("\\s+", "");
-
-		return zkQuorum;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/main/java/org/apache/flink/runtime/util/ZooKeeperUtils.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/ZooKeeperUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/ZooKeeperUtils.java
new file mode 100644
index 0000000..14afb7b
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/ZooKeeperUtils.java
@@ -0,0 +1,151 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.util;
+
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.CuratorFrameworkFactory;
+import org.apache.curator.retry.ExponentialBackoffRetry;
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.IllegalConfigurationException;
+import org.apache.flink.runtime.jobmanager.RecoveryMode;
+import org.apache.flink.runtime.leaderelection.ZooKeeperLeaderElectionService;
+import org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalService;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ZooKeeperUtils {
+
+	private static final Logger LOG = LoggerFactory.getLogger(ZooKeeperUtils.class);
+
+	/**
+	 * Starts a {@link CuratorFramework} instance and connects it to the given ZooKeeper
+	 * quorum.
+	 *
+	 * @param configuration {@link Configuration} object containing the configuration values
+	 * @return {@link CuratorFramework} instance
+	 */
+	public static CuratorFramework startCuratorFramework(Configuration configuration) {
+		String zkQuorum = configuration.getString(ConfigConstants.ZOOKEEPER_QUORUM_KEY, "");
+
+		if(zkQuorum == null || zkQuorum.equals("")) {
+			throw new RuntimeException("No valid ZooKeeper quorum has been specified.");
+		}
+
+		int sessionTimeout = configuration.getInteger(
+				ConfigConstants.ZOOKEEPER_SESSION_TIMEOUT,
+				ConfigConstants.DEFAULT_ZOOKEEPER_SESSION_TIMEOUT);
+
+		int connectionTimeout = configuration.getInteger(
+				ConfigConstants.ZOOKEEPER_CONNECTION_TIMEOUT,
+				ConfigConstants.DEFAULT_ZOOKEEPER_CONNECTION_TIMEOUT);
+
+		int retryWait = configuration.getInteger (
+				ConfigConstants.ZOOKEEPER_RETRY_WAIT,
+				ConfigConstants.DEFAULT_ZOOKEEPER_RETRY_WAIT);
+
+		int maxRetryAttempts = configuration.getInteger(
+				ConfigConstants.ZOOKEEPER_MAX_RETRY_ATTEMPTS,
+				ConfigConstants.DEFAULT_ZOOKEEPER_MAX_RETRY_ATTEMPTS);
+
+		String root = configuration.getString(ConfigConstants.ZOOKEEPER_DIR_KEY,
+				ConfigConstants.DEFAULT_ZOOKEEPER_DIR_KEY);
+
+		LOG.info("Using '{}' as root namespace.", root);
+
+		CuratorFramework cf = CuratorFrameworkFactory.builder()
+				.connectString(zkQuorum)
+				.sessionTimeoutMs(sessionTimeout)
+				.connectionTimeoutMs(connectionTimeout)
+				.retryPolicy(new ExponentialBackoffRetry(retryWait, maxRetryAttempts))
+				// Curator prepends a '/' manually and throws an Exception if the
+				// namespace starts with a '/'.
+				.namespace(root.startsWith("/") ? root.substring(1) : root)
+				.build();
+
+		cf.start();
+
+		return cf;
+	}
+
+	/**
+	 * Returns whether high availability is enabled (<=> ZooKeeper quorum configured).
+	 */
+	public static boolean isZooKeeperHighAvailabilityEnabled(Configuration flinkConf) {
+		String recoveryMode = flinkConf.getString(
+				ConfigConstants.RECOVERY_MODE,
+				ConfigConstants.DEFAULT_RECOVERY_MODE).toUpperCase();
+
+		return recoveryMode.equals(RecoveryMode.ZOOKEEPER.name());
+	}
+
+	/**
+	 * Returns the configured ZooKeeper quorum (and removes whitespace, because ZooKeeper does not
+	 * tolerate it).
+	 */
+	public static String getZooKeeperEnsemble(Configuration flinkConf)
+			throws IllegalConfigurationException {
+
+		String zkQuorum = flinkConf.getString(ConfigConstants.ZOOKEEPER_QUORUM_KEY, "");
+
+		if (zkQuorum == null || zkQuorum.equals("")) {
+			throw new IllegalConfigurationException("No ZooKeeper quorum specified in config.");
+		}
+
+		// Remove all whitespace
+		zkQuorum = zkQuorum.replaceAll("\\s+", "");
+
+		return zkQuorum;
+	}
+
+	/**
+	 * Creates a {@link ZooKeeperLeaderRetrievalService} instance.
+	 *
+	 * @param configuration {@link Configuration} object containing the configuration values
+	 * @return {@link ZooKeeperLeaderRetrievalService} instance.
+	 * @throws Exception
+	 */
+	public static ZooKeeperLeaderRetrievalService createLeaderRetrievalService(
+			Configuration configuration) throws Exception{
+		CuratorFramework client = startCuratorFramework(configuration);
+		String leaderPath = configuration.getString(ConfigConstants.ZOOKEEPER_LEADER_PATH,
+				ConfigConstants.DEFAULT_ZOOKEEPER_LEADER_PATH);
+
+		return new ZooKeeperLeaderRetrievalService(client, leaderPath);
+	}
+
+	/**
+	 * Creates a {@link ZooKeeperLeaderElectionService} instance.
+	 *
+	 * @param configuration {@link Configuration} object containing the configuration values
+	 * @return {@link ZooKeeperLeaderElectionService} instance.
+	 * @throws Exception
+	 */
+	public static ZooKeeperLeaderElectionService createLeaderElectionService(
+			Configuration configuration) throws Exception {
+		CuratorFramework client = startCuratorFramework(configuration);
+
+		String latchPath = configuration.getString(ConfigConstants.ZOOKEEPER_LATCH_PATH,
+				ConfigConstants.DEFAULT_ZOOKEEPER_LATCH_PATH);
+		String leaderPath = configuration.getString(ConfigConstants.ZOOKEEPER_LEADER_PATH,
+				ConfigConstants.DEFAULT_ZOOKEEPER_LEADER_PATH);
+
+		return new ZooKeeperLeaderElectionService(client, latchPath, leaderPath);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/main/scala/org/apache/flink/runtime/LeaderSessionMessageDecorator.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/LeaderSessionMessageDecorator.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/LeaderSessionMessageDecorator.scala
deleted file mode 100644
index c6793ed..0000000
--- a/flink-runtime/src/main/scala/org/apache/flink/runtime/LeaderSessionMessageDecorator.scala
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.runtime
-
-import java.util.UUID
-
-import org.apache.flink.runtime.messages.JobManagerMessages.LeaderSessionMessage
-import org.apache.flink.runtime.messages.RequiresLeaderSessionID
-
-/** [[MessageDecorator]] which wraps [[RequiresLeaderSessionID]] messages in a
-  * [[LeaderSessionMessage]] with the given leader session ID.
-  *
-  * @param leaderSessionID Leader session ID which is associated with the
-  *                        [[RequiresLeaderSessionID]] message
-  */
-class LeaderSessionMessageDecorator(val leaderSessionID: Option[UUID]) extends MessageDecorator {
-
-  /** Wraps [[RequiresLeaderSessionID]] messages in a [[LeaderSessionMessage]].
-    *
-    * @param message Message to decorate
-    * @return Decorated message
-    */
-  override def decorate(message: Any): Any = {
-    message match {
-      case msg: RequiresLeaderSessionID =>
-        LeaderSessionMessage(leaderSessionID, msg)
-      case msg => msg
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/main/scala/org/apache/flink/runtime/LeaderSessionMessageFilter.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/LeaderSessionMessageFilter.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/LeaderSessionMessageFilter.scala
new file mode 100644
index 0000000..72db258
--- /dev/null
+++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/LeaderSessionMessageFilter.scala
@@ -0,0 +1,69 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime
+
+import java.util.UUID
+
+import org.apache.flink.runtime.messages.JobManagerMessages.LeaderSessionMessage
+import org.apache.flink.runtime.messages.RequiresLeaderSessionID
+
+/** Mixin to filter out [[LeaderSessionMessage]] which contain an invalid leader session id.
+  * Messages which contain a valid leader session ID are unwrapped and forwarded to the actor.
+  *
+  */
+trait LeaderSessionMessageFilter extends FlinkActor {
+  protected def leaderSessionID: Option[UUID]
+
+  abstract override def receive: Receive = {
+    case leaderMessage@LeaderSessionMessage(msgID, msg) =>
+      if (leaderSessionID.equals(Option(msgID))) {
+        super.receive(msg)
+      } else {
+        handleDiscardedMessage(leaderSessionID, leaderMessage)
+      }
+    case msg: RequiresLeaderSessionID =>
+      throw new Exception(s"Received a message $msg without a leader session ID, even though" +
+        s" the message requires a leader session ID.")
+    case msg =>
+      super.receive(msg)
+  }
+
+  private def handleDiscardedMessage(
+      expectedLeaderSessionID: Option[UUID],
+      msg: LeaderSessionMessage)
+    : Unit = {
+    log.warn(s"Discard message $msg because the expected leader session ID " +
+      s"$expectedLeaderSessionID did not equal the received leader session ID" +
+      s"${msg.leaderSessionID}.")
+  }
+
+  /** Wrap [[RequiresLeaderSessionID]] messages in a [[LeaderSessionMessage]]
+    *
+    * @param message The message to decorate
+    * @return The decorated message
+    */
+  override def decorateMessage(message: Any): Any = {
+    message match {
+      case msg: RequiresLeaderSessionID =>
+        LeaderSessionMessage(leaderSessionID.orNull, super.decorateMessage(msg))
+
+      case msg => super.decorateMessage(msg)
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/main/scala/org/apache/flink/runtime/LeaderSessionMessages.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/LeaderSessionMessages.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/LeaderSessionMessages.scala
deleted file mode 100644
index d54926d..0000000
--- a/flink-runtime/src/main/scala/org/apache/flink/runtime/LeaderSessionMessages.scala
+++ /dev/null
@@ -1,73 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.runtime
-
-import java.util.UUID
-
-import org.apache.flink.runtime.messages.JobManagerMessages.LeaderSessionMessage
-import org.apache.flink.runtime.messages.RequiresLeaderSessionID
-
-/** Mixin to filter out [[LeaderSessionMessage]] which contain an invalid leader session id.
-  * Messages which contain a valid leader session ID are unwrapped and forwarded to the actor.
-  *
-  */
-trait LeaderSessionMessages extends FlinkActor {
-  protected def leaderSessionID: Option[UUID]
-
-  abstract override def receive: Receive = {
-    case LeaderSessionMessage(id, msg) =>
-      // Filter out messages which have not the correct leader session ID
-      (leaderSessionID, id) match {
-        case (Some(currentID), Some(msgID)) =>
-          if(currentID.equals(msgID)) {
-            // correct leader session ID
-            super.receive(msg)
-          } else {
-            // discard message because of incorrect leader session ID
-            handleDiscardedMessage(msg)
-          }
-
-        case _ => handleDiscardedMessage(msg)
-      }
-    case msg: RequiresLeaderSessionID =>
-      throw new Exception(s"Received a message $msg without a leader session ID, even though" +
-        " it requires to have one.")
-    case msg =>
-      // pass the message to the parent's receive method for further processing
-      super.receive(msg)
-  }
-
-  private def handleDiscardedMessage(msg: Any): Unit = {
-    log.debug(s"Discard message $msg because the leader session ID was not correct.")
-  }
-
-  /** Wrap [[RequiresLeaderSessionID]] messages in a [[LeaderSessionMessage]]
-    *
-    * @param message The message to decorate
-    * @return The decorated message
-    */
-  override def decorateMessage(message: Any): Any = {
-    message match {
-      case msg: RequiresLeaderSessionID =>
-        LeaderSessionMessage(leaderSessionID, super.decorateMessage(msg))
-
-      case msg => super.decorateMessage(msg)
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/main/scala/org/apache/flink/runtime/MessageDecorator.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/MessageDecorator.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/MessageDecorator.scala
deleted file mode 100644
index 5b1700f..0000000
--- a/flink-runtime/src/main/scala/org/apache/flink/runtime/MessageDecorator.scala
+++ /dev/null
@@ -1,32 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.runtime
-
-/** Base trait for message decorators
-  *
-  */
-trait MessageDecorator {
-
-  /** Decorates a message
-    *
-    * @param message Message to decorate
-    * @return Decorated message
-    */
-  def decorate(message: Any): Any
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/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 b8cce41..8007ef6 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,7 +18,8 @@
 
 package org.apache.flink.runtime.akka
 
-import java.net.InetAddress
+import java.io.IOException
+import java.net.{InetSocketAddress, InetAddress}
 import java.util.concurrent.{TimeUnit, Callable}
 
 import akka.actor._
@@ -27,7 +28,7 @@ import com.typesafe.config.{Config, ConfigFactory}
 import org.apache.flink.configuration.{ConfigConstants, Configuration}
 import org.jboss.netty.logging.{Slf4JLoggerFactory, InternalLoggerFactory}
 import org.slf4j.LoggerFactory
-import scala.concurrent.{ExecutionContext, Future}
+import scala.concurrent._
 import scala.concurrent.duration._
 import scala.language.postfixOps
 
@@ -324,18 +325,68 @@ object AkkaUtils {
     }
   }
 
-  def getChild(parent: ActorRef, child: String,
-               system: ActorSystem,
-               timeout: FiniteDuration): Future[ActorRef] = {
+  /** Returns a [[Future]] to the [[ActorRef]] of the child of a given actor. The child is specified
+    * by providing its actor name.
+    *
+    * @param parent [[ActorRef]] to the parent of the child to be retrieved
+    * @param child Name of the child actor
+    * @param system [[ActorSystem]] to be used
+    * @param timeout Maximum timeout for the future
+    * @return [[Future]] to the [[ActorRef]] of the child actor
+    */
+  def getChild(
+      parent: ActorRef,
+      child: String,
+      system: ActorSystem,
+      timeout: FiniteDuration)
+    : Future[ActorRef] = {
     system.actorSelection(parent.path / child).resolveOne()(timeout)
   }
 
-  def getReference(path: String, system:
-                   ActorSystem,
-                   timeout: FiniteDuration): Future[ActorRef] = {
+  /** Returns a [[Future]] to the [[ActorRef]] of an actor. The actor is specified by its path.
+    *
+    * @param path Path to the actor to be retrieved
+    * @param system [[ActorSystem]] to be used
+    * @param timeout Maximum timeout for the future
+    * @return [[Future]] to the [[ActorRef]] of the actor
+    */
+  def getActorRefFuture(
+      path: String,
+      system: ActorSystem,
+      timeout: FiniteDuration)
+    : Future[ActorRef] = {
     system.actorSelection(path).resolveOne()(timeout)
   }
 
+  /** Returns an [[ActorRef]] for the actor specified by the path parameter.
+    *
+    * @param path Path to the actor to be retrieved
+    * @param system [[ActorSystem]] to be used
+    * @param timeout Maximum timeout for the future
+    * @throws java.io.IOException
+    * @return [[ActorRef]] of the requested [[Actor]]
+    */
+  @throws(classOf[IOException])
+  def getActorRef(
+      path: String,
+      system: ActorSystem,
+      timeout: FiniteDuration)
+    : ActorRef = {
+    try {
+      val future = AkkaUtils.getActorRefFuture(path, system, timeout)
+      Await.result(future, timeout)
+    }
+    catch {
+      case e @ (_ : ActorNotFound | _ : TimeoutException) =>
+        throw new IOException(
+          s"Actor at $path not reachable. " +
+            "Please make sure that the actor is running and its port is reachable.", e)
+
+      case e: IOException =>
+        throw new IOException(s"Could not connect to the actor at $path", e)
+    }
+  }
+
 
   /**
    * Utility function to construct a future which tries multiple times to execute itself if it
@@ -421,4 +472,64 @@ object AkkaUtils {
     val duration = Duration(ConfigConstants.DEFAULT_AKKA_LOOKUP_TIMEOUT)
     new FiniteDuration(duration.toMillis, TimeUnit.MILLISECONDS)
   }
+
+  /** Returns the address of the given [[ActorSystem]]. The [[Address]] object contains
+    * the port and the host under which the actor system is reachable
+    *
+    * @param system [[ActorSystem]] for which the [[Address]] shall be retrieved
+    * @return [[Address]] of the given [[ActorSystem]]
+    */
+  def getAddress(system: ActorSystem): Address = {
+    RemoteAddressExtension(system).address
+  }
+
+  /** Returns the given [[ActorRef]]'s path string representation with host and port of the
+    * [[ActorSystem]] in which the actor is running.
+    *
+    * @param system [[ActorSystem]] in which the given [[ActorRef]] is running
+    * @param actor [[ActorRef]] of the [[Actor]] for which the URL has to be generated
+    * @return String containing the [[ActorSystem]] independent URL of the [[Actor]]
+    */
+  def getAkkaURL(system: ActorSystem, actor: ActorRef): String = {
+    val address = getAddress(system)
+    actor.path.toStringWithAddress(address)
+  }
+
+  /** Returns the AkkaURL for a given [[ActorSystem]] and a path describing a running [[Actor]] in
+    * the actor system.
+    *
+    * @param system [[ActorSystem]] in which the given [[Actor]] is running
+    * @param path Path describing an [[Actor]] for which the URL has to be generated
+    * @return String containing the [[ActorSystem]] independent URL of an [[Actor]] specified by
+    *         path.
+    */
+  def getAkkaURL(system: ActorSystem, path: String): String = {
+    val address = getAddress(system)
+    address.toString + path
+  }
+
+  /** Extracts the hostname and the port of the remote actor system from the given Akka URL. The
+    * result is an [[InetSocketAddress]] instance containing the extracted hostname and port. If
+    * the Akka URL does not contain the hostname and port information, e.g. a local Akka URL is
+    * provided, then an [[Exception]] is thrown.
+    *
+    * @param akkaURL
+    * @throws java.lang.Exception
+    * @return
+    */
+  @throws(classOf[Exception])
+  def getInetSockeAddressFromAkkaURL(akkaURL: String): InetSocketAddress = {
+    // AkkaURLs have the form schema://systemName@host:port/.... if it's a remote Akka URL
+    val hostPortRegex = """@([^/:]*):(\d*)""".r
+
+    hostPortRegex.findFirstMatchIn(akkaURL) match {
+      case Some(m) =>
+        val host = m.group(1)
+        val port = m.group(2).toInt
+
+        new InetSocketAddress(host, port)
+      case None => throw new Exception("Could not retrieve InetSocketAddress from " +
+        s"Akka URL $akkaURL")
+    }
+  }
 }


[06/10] flink git commit: [FLINK-2291] [runtime] Add ZooKeeper support to elect a leader from a set of JobManager. The leader will then be retrieved from ZooKeeper by the TaskManagers.

Posted by tr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/LocalFlinkMiniCluster.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/LocalFlinkMiniCluster.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/LocalFlinkMiniCluster.scala
index 54c457e..3f8783a 100644
--- a/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/LocalFlinkMiniCluster.scala
+++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/LocalFlinkMiniCluster.scala
@@ -18,32 +18,29 @@
 
 package org.apache.flink.runtime.minicluster
 
-import akka.actor.{ActorRef, ActorSystem, ExtendedActorSystem}
+import akka.actor.{ActorRef, ActorSystem}
 
 import org.apache.flink.api.common.io.FileOutputFormat
 import org.apache.flink.configuration.{ConfigConstants, Configuration}
 import org.apache.flink.runtime.StreamingMode
 import org.apache.flink.runtime.akka.AkkaUtils
-import org.apache.flink.runtime.client.JobClient
-import org.apache.flink.runtime.instance.AkkaActorGateway
 import org.apache.flink.runtime.io.network.netty.NettyConfig
 import org.apache.flink.runtime.jobmanager.JobManager
-import org.apache.flink.runtime.jobmanager.web.WebInfoServer
 import org.apache.flink.runtime.taskmanager.TaskManager
 import org.apache.flink.runtime.util.EnvironmentInformation
-import org.apache.flink.runtime.webmonitor.WebMonitor
 
 import org.slf4j.LoggerFactory
 
 /**
  * Local Flink mini cluster which executes all [[TaskManager]]s and the [[JobManager]] in the same
- * JVM. It extends the [[FlinkMiniCluster]] by providing a [[JobClient]], having convenience
- * functions to setup Flink's configuration and implementations to create [[JobManager]] and
- * [[TaskManager]].
+ * JVM. It extends the [[FlinkMiniCluster]] by having convenience functions to setup Flink's
+ * configuration and implementations to create [[JobManager]] and [[TaskManager]].
  *
  * @param userConfiguration Configuration object with the user provided configuration values
  * @param singleActorSystem true if all actors (JobManager and TaskManager) shall be run in the same
  *                          [[ActorSystem]], otherwise false
+ * @param streamingMode Defines the execution mode of Flink's components (JobManager and
+ *                      TaskManager)
  */
 class LocalFlinkMiniCluster(
     userConfiguration: Configuration,
@@ -51,22 +48,12 @@ class LocalFlinkMiniCluster(
     streamingMode: StreamingMode)
   extends FlinkMiniCluster(userConfiguration, singleActorSystem, streamingMode) {
 
-  
   def this(userConfiguration: Configuration, singleActorSystem: Boolean)
        = this(userConfiguration, singleActorSystem, StreamingMode.BATCH_ONLY)
   
   def this(userConfiguration: Configuration) = this(userConfiguration, true)
 
   // --------------------------------------------------------------------------
-  
-  
-  val jobClientActorSystem = if (singleActorSystem) {
-    jobManagerActorSystem
-  } else {
-    // create an actor system listening on a random port
-    JobClient.startJobClientActorSystem(configuration)
-  }
-
 
   override def generateConfiguration(userConfiguration: Configuration): Configuration = {
     val config = getDefaultConfig
@@ -78,14 +65,28 @@ class LocalFlinkMiniCluster(
     config
   }
 
-  override def startJobManager(system: ActorSystem): (ActorRef, Option[WebMonitor]) = {
+  override def startJobManager(index: Int, system: ActorSystem): ActorRef = {
     val config = configuration.clone()
-       
-    val (jobManager, archiver) = JobManager.startJobManagerActors(config, system, streamingMode)
 
-    val webMonitorOption = startWebServer(config, jobManager, archiver)
+    val jobManagerName = getJobManagerName(index)
+    val archiveName = getArchiveName(index)
+
+    val jobManagerPort = config.getInteger(
+      ConfigConstants.JOB_MANAGER_IPC_PORT_KEY,
+      ConfigConstants.DEFAULT_JOB_MANAGER_IPC_PORT)
+
+    if(jobManagerPort > 0) {
+      config.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, jobManagerPort + index)
+    }
 
-    (jobManager, webMonitorOption)
+    val (jobManager, _) = JobManager.startJobManagerActors(
+      config,
+      system,
+      Some(jobManagerName),
+      Some(archiveName),
+      streamingMode)
+
+    jobManager
   }
 
   override def startTaskManager(index: Int, system: ActorSystem): ActorRef = {
@@ -113,51 +114,32 @@ class LocalFlinkMiniCluster(
     } else {
       TaskManager.TASK_MANAGER_NAME
     }
-
-    val jobManagerPath: Option[String] = if (singleActorSystem) {
-      Some(jobManagerActor.path.toString)
-    } else {
-      None
-    }
     
     TaskManager.startTaskManagerComponentsAndActor(
       config,
       system,
       hostname, // network interface to bind to
       Some(taskManagerActorName), // actor name
-      jobManagerPath, // job manager akka URL
+      Some(createLeaderRetrievalService), // job manager leader retrieval service
       localExecution, // start network stack?
       streamingMode,
       classOf[TaskManager])
   }
 
-  def getJobClientActorSystem: ActorSystem = jobClientActorSystem
-
-  def getJobManagerRPCPort: Int = {
-    if (jobManagerActorSystem.isInstanceOf[ExtendedActorSystem]) {
-      val extActor = jobManagerActorSystem.asInstanceOf[ExtendedActorSystem]
-      extActor.provider.getDefaultAddress.port match {
-        case p: Some[Int] => p.get
-        case _ => -1
-      }
-    } else {
-      -1
-    }
-  }
+  def getLeaderRPCPort: Int = {
+    val index = getLeaderIndex(timeout)
 
-  override def shutdown(): Unit = {
-    super.shutdown()
+    jobManagerActorSystems match {
+      case Some(jmActorSystems) =>
+        AkkaUtils.getAddress(jmActorSystems(index)).port match {
+          case Some(p) => p
+          case None => -1
+        }
 
-    if (!singleActorSystem) {
-      jobClientActorSystem.shutdown()
+      case None => throw new Exception("The JobManager of the LocalFlinkMiniCluster has not been " +
+        "started properly.")
     }
-  }
 
-  override def awaitTermination(): Unit = {
-    if (!singleActorSystem) {
-      jobClientActorSystem.awaitTermination()
-    }
-    super.awaitTermination()
   }
 
   def initializeIOFormatClasses(configuration: Configuration): Unit = {
@@ -178,10 +160,10 @@ class LocalFlinkMiniCluster(
     if (config.getInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, -1) == -1) {
 
       val bufferSizeNew: Int = config.getInteger(
-                                      ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY, -1)
+        ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY, -1)
 
       val bufferSizeOld: Int = config.getInteger(
-                                      ConfigConstants.TASK_MANAGER_NETWORK_BUFFER_SIZE_KEY, -1)
+        ConfigConstants.TASK_MANAGER_NETWORK_BUFFER_SIZE_KEY, -1)
       val bufferSize: Int =
         if (bufferSizeNew != -1) {
           bufferSizeNew
@@ -194,13 +176,15 @@ class LocalFlinkMiniCluster(
           bufferSizeOld
         }
       
-      val bufferMem: Long = config.getLong(ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY,
-          ConfigConstants.DEFAULT_TASK_MANAGER_NETWORK_NUM_BUFFERS) * bufferSize.toLong
+      val bufferMem: Long = config.getLong(
+        ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY,
+        ConfigConstants.DEFAULT_TASK_MANAGER_NETWORK_NUM_BUFFERS) * bufferSize.toLong
 
       val numTaskManager = config.getInteger(
-        ConfigConstants.LOCAL_INSTANCE_MANAGER_NUMBER_TASK_MANAGER, 1)
+        ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, 1)
 
-      val memoryFraction = config.getFloat(ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY,
+      val memoryFraction = config.getFloat(
+        ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY,
         ConfigConstants.DEFAULT_MEMORY_MANAGER_MEMORY_FRACTION)
 
       // full memory size
@@ -218,16 +202,12 @@ class LocalFlinkMiniCluster(
     }
   }
 
-  def getConfiguration: Configuration = {
-    this.userConfiguration
-  }
-
   def getDefaultConfig: Configuration = {
     val config: Configuration = new Configuration()
 
     config.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, hostname)
 
-    config.setInteger(ConfigConstants.LOCAL_INSTANCE_MANAGER_NUMBER_TASK_MANAGER, 1)
+    config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, 1)
 
     // Reduce number of threads for local execution
     config.setInteger(NettyConfig.NUM_THREADS_CLIENT, 1)
@@ -235,8 +215,19 @@ class LocalFlinkMiniCluster(
 
     config
   }
-}
 
-object LocalFlinkMiniCluster {
-//  val LOG = LoggerFactory.getLogger(classOf[LocalFlinkMiniCluster])
+  protected def getJobManagerName(index: Int): String = {
+    if(singleActorSystem) {
+      JobManager.JOB_MANAGER_NAME + "_" + (index + 1)
+    } else {
+      JobManager.JOB_MANAGER_NAME
+    }
+  }
+  protected def getArchiveName(index: Int): String = {
+    if(singleActorSystem) {
+      JobManager.ARCHIVE_NAME + "_" + (index + 1)
+    } else {
+      JobManager.ARCHIVE_NAME
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala
index cc8b8ba..529d3d1 100644
--- a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala
+++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala
@@ -39,8 +39,10 @@ import grizzled.slf4j.Logger
 import org.apache.flink.configuration._
 
 import org.apache.flink.runtime.accumulators.AccumulatorSnapshot
+import org.apache.flink.runtime.leaderretrieval.{LeaderRetrievalListener, LeaderRetrievalService}
+import org.apache.flink.runtime.messages.TaskMessages._
 import org.apache.flink.runtime.messages.checkpoint.{NotifyCheckpointComplete, TriggerCheckpoint, AbstractCheckpointMessage}
-import org.apache.flink.runtime.{FlinkActor, LeaderSessionMessages, LogMessages, StreamingMode}
+import org.apache.flink.runtime.{FlinkActor, LeaderSessionMessageFilter, LogMessages, StreamingMode}
 import org.apache.flink.runtime.akka.AkkaUtils
 import org.apache.flink.runtime.blob.{BlobService, BlobCache}
 import org.apache.flink.runtime.broadcast.BroadcastVariableManager
@@ -55,17 +57,15 @@ import org.apache.flink.runtime.io.disk.iomanager.{IOManager, IOManagerAsync}
 import org.apache.flink.runtime.io.network.NetworkEnvironment
 import org.apache.flink.runtime.io.network.netty.NettyConfig
 import org.apache.flink.runtime.jobgraph.IntermediateDataSetID
-import org.apache.flink.runtime.jobmanager.JobManager
 import org.apache.flink.runtime.memorymanager.{MemoryManager, DefaultMemoryManager}
 import org.apache.flink.runtime.messages.Messages._
 import org.apache.flink.runtime.messages.RegistrationMessages._
 import org.apache.flink.runtime.messages.TaskManagerMessages._
-import org.apache.flink.runtime.messages.TaskMessages._
 import org.apache.flink.runtime.net.NetUtils
 import org.apache.flink.runtime.process.ProcessReaper
 import org.apache.flink.runtime.security.SecurityUtils
 import org.apache.flink.runtime.security.SecurityUtils.FlinkSecuredRunner
-import org.apache.flink.runtime.util.{ZooKeeperUtil, MathUtils, EnvironmentInformation}
+import org.apache.flink.runtime.util.{LeaderRetrievalUtils, MathUtils, EnvironmentInformation}
 
 import scala.concurrent._
 import scala.concurrent.duration._
@@ -120,15 +120,15 @@ import scala.language.postfixOps
 class TaskManager(
     protected val config: TaskManagerConfiguration,
     protected val connectionInfo: InstanceConnectionInfo,
-    protected val jobManagerAkkaURL: String,
     protected val memoryManager: MemoryManager,
     protected val ioManager: IOManager,
     protected val network: NetworkEnvironment,
-    protected val numberOfSlots: Int)
+    protected val numberOfSlots: Int,
+    protected val leaderRetrievalService: LeaderRetrievalService)
   extends FlinkActor
-  with LeaderSessionMessages // Mixin order is important: second we want to filter leader messages
+  with LeaderSessionMessageFilter // Mixin order is important: We want to filter after logging
   with LogMessages // Mixin order is important: first we want to support message logging
-{
+  with LeaderRetrievalListener {
 
   override val log = Logger(getClass)
 
@@ -152,10 +152,12 @@ class TaskManager(
 
   /** Metric serialization */
   private val metricRegistryMapper: ObjectMapper = new ObjectMapper()
-        .registerModule(new MetricsModule(TimeUnit.SECONDS,
-                                          TimeUnit.MILLISECONDS,
-                                          false,
-                                          MetricFilter.ALL))
+    .registerModule(
+      new MetricsModule(
+        TimeUnit.SECONDS,
+        TimeUnit.MILLISECONDS,
+        false,
+        MetricFilter.ALL))
 
   /** Actors which want to be notified once this task manager has been
       registered at the job manager */
@@ -164,19 +166,18 @@ class TaskManager(
   private var blobService: Option[BlobService] = None
   private var libraryCacheManager: Option[LibraryCacheManager] = None
   protected var currentJobManager: Option[ActorRef] = None
-
+  private var jobManagerAkkaURL: Option[String] = None
+ 
   private var instanceID: InstanceID = null
 
   private var heartbeatScheduler: Option[Cancellable] = None
 
-  protected var leaderSessionID: Option[UUID] = None
+  var leaderSessionID: Option[UUID] = None
 
-  private val currentRegistrationSessionID: UUID = UUID.randomUUID()
 
   private val runtimeInfo = new TaskManagerRuntimeInfo(
        connectionInfo.getHostname(),
        new UnmodifiableConfiguration(config.configuration))
-
   // --------------------------------------------------------------------------
   //  Actor messages and life cycle
   // --------------------------------------------------------------------------
@@ -196,18 +197,13 @@ class TaskManager(
       log.info(MemoryLogger.getMemoryUsageStatsAsString(ManagementFactory.getMemoryMXBean))
     }
 
-    // kick off the registration
-    val deadline: Option[Deadline] = config.maxRegistrationDuration.map(_.fromNow)
-
-    self ! decorateMessage(
-      TriggerTaskManagerRegistration(
-        currentRegistrationSessionID,
-        jobManagerAkkaURL,
-        TaskManager.INITIAL_REGISTRATION_TIMEOUT,
-        deadline,
-        1)
-    )
-
+    try {
+      leaderRetrievalService.start(this)
+    } catch {
+      case e: Exception =>
+        log.error("Could not start leader retrieval service.", e)
+        throw new RuntimeException("Could not start leader retrieval service.", e)
+    }
   }
 
   /**
@@ -229,6 +225,12 @@ class TaskManager(
     }
 
     try {
+      leaderRetrievalService.stop()
+    } catch {
+      case e: Exception => log.error("Leader retrieval service did not shut down properly.")
+    }
+
+    try {
       ioManager.shutdown()
     } catch {
       case t: Exception => log.error("I/O manager did not shutdown properly.", t)
@@ -266,6 +268,10 @@ class TaskManager(
     // messages for coordinating checkpoints
     case message: AbstractCheckpointMessage => handleCheckpointingMessage(message)
 
+    case JobManagerLeaderAddress(address, leaderSessionID) => {
+      handleJobManagerLeaderAddress(address, leaderSessionID)
+    }
+
     // registration messages for connecting and disconnecting from / to the JobManager
     case message: RegistrationMessage => handleRegistrationMessage(message)
 
@@ -279,7 +285,7 @@ class TaskManager(
 
     // registers the message sender to be notified once this TaskManager has completed
     // its registration at the JobManager
-    case NotifyWhenRegisteredAtJobManager =>
+    case NotifyWhenRegisteredAtAnyJobManager =>
       if (isConnected) {
         sender ! decorateMessage(RegisteredAtJobManager)
       } else {
@@ -290,6 +296,7 @@ class TaskManager(
     case Terminated(actor: ActorRef) =>
       if (isConnected && actor == currentJobManager.orNull) {
         handleJobManagerDisconnect(sender(), "JobManager is no longer reachable")
+        triggerTaskManagerRegistration()
       }
       else {
         log.warn(s"Received unrecognized disconnect message " +
@@ -298,6 +305,7 @@ class TaskManager(
 
     case Disconnect(msg) =>
       handleJobManagerDisconnect(sender(), "JobManager requested disconnect: " + msg)
+      triggerTaskManagerRegistration()
 
     case FatalError(message, cause) =>
       killTaskManagerFatal(message, cause)
@@ -361,7 +369,7 @@ class TaskManager(
         // was into a terminal state, or in case the JobManager cannot be informed of the
         // state transition
 
-      case updateMsg@UpdateTaskExecutionState(taskExecutionState: TaskExecutionState) =>
+      case updateMsg @ UpdateTaskExecutionState(taskExecutionState: TaskExecutionState) =>
 
           // we receive these from our tasks and forward them to the JobManager
           currentJobManager foreach {
@@ -379,7 +387,7 @@ class TaskManager(
                     FailTask(
                       executionID,
                       new Exception("Task has been cancelled on the JobManager."))
-                  )
+                    )
                   }
 
                 case Failure(t) =>
@@ -416,15 +424,15 @@ class TaskManager(
           val task = runningTasks.get(executionID)
           if (task != null) {
             task.cancelExecution()
-          sender ! decorateMessage(new TaskOperationResult(executionID, true))
+            sender ! decorateMessage(new TaskOperationResult(executionID, true))
           } else {
             log.debug(s"Cannot find task to cancel for execution ${executionID})")
-          sender ! decorateMessage(
-            new TaskOperationResult(
-              executionID,
-              false,
-            "No task with that execution ID was found.")
-          )
+            sender ! decorateMessage(
+              new TaskOperationResult(
+                executionID,
+                false,
+              "No task with that execution ID was found.")
+            )
           }
 
         case PartitionState(taskExecutionId, taskResultId, partitionId, state) =>
@@ -489,148 +497,148 @@ class TaskManager(
    * @param message The registration message.
    */
   private def handleRegistrationMessage(message: RegistrationMessage): Unit = {
-    if(message.registrationSessionID.equals(currentRegistrationSessionID)) {
-      message match {
-        case TriggerTaskManagerRegistration(
-          registrationSessionID,
-          jobManagerURL,
-          timeout,
-          deadline,
-          attempt) =>
+    message match {
+      case TriggerTaskManagerRegistration(
+        jobManagerURL,
+        timeout,
+        deadline,
+        attempt) =>
 
-          if (isConnected) {
-            // this may be the case, if we queue another attempt and
-            // in the meantime, the registration is acknowledged
-            log.debug(
-              "TaskManager was triggered to register at JobManager, but is already registered")
-          }
-          else if (deadline.exists(_.isOverdue())) {
-            // we failed to register in time. that means we should quit
-            log.error("Failed to register at the JobManager withing the defined maximum " +
-              "connect time. Shutting down ...")
+        if (isConnected) {
+          // this may be the case, if we queue another attempt and
+          // in the meantime, the registration is acknowledged
+          log.debug(
+            "TaskManager was triggered to register at JobManager, but is already registered")
+        }
+        else if (deadline.exists(_.isOverdue())) {
+          // we failed to register in time. that means we should quit
+          log.error("Failed to register at the JobManager withing the defined maximum " +
+            "connect time. Shutting down ...")
 
-            // terminate ourselves (hasta la vista)
-            self ! decorateMessage(PoisonPill)
+          // terminate ourselves (hasta la vista)
+          self ! decorateMessage(PoisonPill)
+        }
+        else {
+          if (!jobManagerAkkaURL.equals(Option(jobManagerURL))) {
+            throw new Exception("Invalid internal state: Trying to register at JobManager " +
+              s"${jobManagerURL} even though the current JobManagerAkkaURL is set to " +
+              s"${jobManagerAkkaURL.getOrElse("")}")
           }
-          else {
-            log.info(s"Trying to register at JobManager ${jobManagerURL} " +
-              s"(attempt ${attempt}, timeout: ${timeout})")
-
-            val jobManager = context.actorSelection(jobManagerAkkaURL)
-            jobManager ! decorateMessage(
-              RegisterTaskManager(
-                registrationSessionID,
-                self,
-                connectionInfo,
-                resources,
-                numberOfSlots)
-            )
 
-            // the next timeout computes via exponential backoff with cap
-            val nextTimeout = (timeout * 2).min(TaskManager.MAX_REGISTRATION_TIMEOUT)
+          log.info(s"Trying to register at JobManager ${jobManagerURL} " +
+            s"(attempt ${attempt}, timeout: ${timeout})")
 
-            // schedule (with our timeout s delay) a check triggers a new registration
-            // attempt, if we are not registered by then
-            context.system.scheduler.scheduleOnce(timeout) {
-              if (!isConnected) {
-                self ! decorateMessage(
-                  TriggerTaskManagerRegistration(
-                    registrationSessionID,
-                    jobManagerURL,
-                    nextTimeout,
-                    deadline,
-                    attempt + 1)
-                )
-              }
-            }(context.dispatcher)
-          }
+          val jobManager = context.actorSelection(jobManagerURL)
 
-        // successful registration. associate with the JobManager
-        // we disambiguate duplicate or erroneous messages, to simplify debugging
-        case AcknowledgeRegistration(_, leaderSessionID, jobManager, id, blobPort) =>
-          if (isConnected) {
-            if (jobManager == currentJobManager.orNull) {
-              log.debug("Ignoring duplicate registration acknowledgement.")
-            } else {
-              log.warn(s"Ignoring 'AcknowledgeRegistration' message from ${jobManager.path} , " +
-                s"because the TaskManager is already registered at ${currentJobManager.orNull}")
-            }
+          jobManager ! decorateMessage(
+            RegisterTaskManager(
+              connectionInfo,
+              resources,
+              numberOfSlots)
+          )
+
+          // the next timeout computes via exponential backoff with cap
+          val nextTimeout = (timeout * 2).min(TaskManager.MAX_REGISTRATION_TIMEOUT)
+
+          // schedule (with our timeout s delay) a check triggers a new registration
+          // attempt, if we are not registered by then
+          context.system.scheduler.scheduleOnce(
+            timeout,
+            self,
+            decorateMessage(TriggerTaskManagerRegistration(
+              jobManagerURL,
+              nextTimeout,
+              deadline,
+              attempt + 1)
+            ))(context.dispatcher)
+        }
+
+      // successful registration. associate with the JobManager
+      // we disambiguate duplicate or erroneous messages, to simplify debugging
+      case AcknowledgeRegistration(id, blobPort) =>
+        val jobManager = sender()
+
+        if (isConnected) {
+          if (jobManager == currentJobManager.orNull) {
+            log.debug("Ignoring duplicate registration acknowledgement.")
+          } else {
+            log.warn(s"Ignoring 'AcknowledgeRegistration' message from ${jobManager.path} , " +
+              s"because the TaskManager is already registered at ${currentJobManager.orNull}")
           }
-          else {
-            // not yet connected, so let's associate with that JobManager
-            try {
-              associateWithJobManager(jobManager, id, blobPort, leaderSessionID)
-            } catch {
-              case t: Throwable =>
-                killTaskManagerFatal(
-                  "Unable to start TaskManager components after registering at JobManager", t)
-            }
+        }
+        else {
+          // not yet connected, so let's associate with that JobManager
+          try {
+            associateWithJobManager(jobManager, id, blobPort)
+          } catch {
+            case t: Throwable =>
+              killTaskManagerFatal(
+                "Unable to start TaskManager components after registering at JobManager", t)
           }
+        }
 
-        // we are already registered at that specific JobManager - duplicate answer, rare cases
-        case AlreadyRegistered(_, leaderSesssionID, jobManager, id, blobPort) =>
-          if (isConnected) {
-            if (jobManager == currentJobManager.orNull) {
-              log.debug("Ignoring duplicate registration acknowledgement.")
-            } else {
-              log.warn(s"Received 'AlreadyRegistered' message from " +
-                s"JobManager ${jobManager.path}, even through TaskManager is currently " +
-                s"registered at ${currentJobManager.orNull}")
-            }
-          }
-          else {
-            // not connected, yet, to let's associate
-            log.info("Received 'AlreadyRegistered' message before 'AcknowledgeRegistration'")
+      // we are already registered at that specific JobManager - duplicate answer, rare cases
+      case AlreadyRegistered(id, blobPort) =>
+        val jobManager = sender()
 
-            try {
-              associateWithJobManager(jobManager, id, blobPort, leaderSesssionID)
-            } catch {
-              case t: Throwable =>
-                killTaskManagerFatal(
-                  "Unable to start TaskManager components after registering at JobManager", t)
-            }
+        if (isConnected) {
+          if (jobManager == currentJobManager.orNull) {
+            log.debug("Ignoring duplicate registration acknowledgement.")
+          } else {
+            log.warn(s"Received 'AlreadyRegistered' message from " +
+              s"JobManager ${jobManager.path}, even through TaskManager is currently " +
+              s"registered at ${currentJobManager.orNull}")
           }
+        }
+        else {
+          // not connected, yet, to let's associate
+          log.info("Received 'AlreadyRegistered' message before 'AcknowledgeRegistration'")
+
+          try {
+            associateWithJobManager(jobManager, id, blobPort)
+          } catch {
+            case t: Throwable =>
+              killTaskManagerFatal(
+                "Unable to start TaskManager components after registering at JobManager", t)
+          }
+        }
 
-        case RefuseRegistration(registrationSessionID, reason) =>
-          if (currentJobManager.isEmpty) {
-            log.error(s"The registration at JobManager ${jobManagerAkkaURL} was refused, " +
-              s"because: ${reason}. Retrying later...")
-
-            // try the registration again after some time
+      case RefuseRegistration(reason) =>
+        if (currentJobManager.isEmpty) {
+          log.error(s"The registration at JobManager ${jobManagerAkkaURL} was refused, " +
+            s"because: ${reason}. Retrying later...")
 
-            val delay: FiniteDuration = TaskManager.DELAY_AFTER_REFUSED_REGISTRATION
-            val deadline: Option[Deadline] = config.maxRegistrationDuration.map {
-              timeout => timeout + delay fromNow
-            }
+        if(jobManagerAkkaURL.isDefined) {
+          // try the registration again after some time
+          val delay: FiniteDuration = TaskManager.DELAY_AFTER_REFUSED_REGISTRATION
+          val deadline: Option[Deadline] = config.maxRegistrationDuration.map {
+            timeout => timeout + delay fromNow
+          }
 
-            context.system.scheduler.scheduleOnce(delay) {
-              self ! decorateMessage(
-                TriggerTaskManagerRegistration(
-                  registrationSessionID,
-                  jobManagerAkkaURL,
-                  TaskManager.INITIAL_REGISTRATION_TIMEOUT,
-                  deadline,
-                  1)
-              )
-            }(context.dispatcher)
+          context.system.scheduler.scheduleOnce(delay) {
+            self ! decorateMessage(
+              TriggerTaskManagerRegistration(
+                jobManagerAkkaURL.get,
+                TaskManager.INITIAL_REGISTRATION_TIMEOUT,
+                deadline,
+                1)
+            )
+          }(context.dispatcher)
+        }
+      }
+        else {
+          // ignore RefuseRegistration messages which arrived after AcknowledgeRegistration
+          if (sender() == currentJobManager.orNull) {
+            log.warn(s"Received 'RefuseRegistration' from the JobManager (${sender().path})" +
+              s" even though this TaskManager is already registered there.")
           }
           else {
-            // ignore RefuseRegistration messages which arrived after AcknowledgeRegistration
-            if (sender() == currentJobManager.orNull) {
-              log.warn(s"Received 'RefuseRegistration' from the JobManager (${sender().path})" +
-                s" even though this TaskManager is already registered there.")
-            }
-            else {
-              log.warn(s"Ignoring 'RefuseRegistration' from unrelated " +
-                s"JobManager (${sender().path})")
-            }
+            log.warn(s"Ignoring 'RefuseRegistration' from unrelated " +
+              s"JobManager (${sender().path})")
           }
+        }
 
-        case _ => unhandled(message)
-      }
-    } else {
-      log.debug(s"Discarded registration message ${message}, because the registration session " +
-        "ID was not correct.")
+      case _ => unhandled(message)
     }
   }
 
@@ -644,7 +652,7 @@ class TaskManager(
    *
    * @return True, if the TaskManager is currently connected to a JobManager, false otherwise.
    */
-  private def isConnected : Boolean = currentJobManager.isDefined
+  protected def isConnected : Boolean = currentJobManager.isDefined
 
   /**
    * Associates the TaskManager with the given JobManager. After this
@@ -655,13 +663,11 @@ class TaskManager(
    * @param id The instanceID under which the TaskManager is registered
    *           at the JobManager.
    * @param blobPort The JobManager's port for the BLOB server.
-   * @param newLeaderSessionID Leader session ID of the JobManager
    */
   private def associateWithJobManager(
       jobManager: ActorRef,
       id: InstanceID,
-      blobPort: Int,
-      newLeaderSessionID: UUID)
+      blobPort: Int)
     : Unit = {
 
     if (jobManager == null) {
@@ -673,10 +679,6 @@ class TaskManager(
     if (blobPort <= 0 || blobPort > 65535) {
       throw new IllegalArgumentException("blob port is out of range: " + blobPort)
     }
-    if(newLeaderSessionID == null) {
-      throw new NullPointerException("Leader session ID must not be null.")
-    }
-
 
     // sanity check that we are not currently registered with a different JobManager
     if (isConnected) {
@@ -703,13 +705,12 @@ class TaskManager(
 
     currentJobManager = Some(jobManager)
     instanceID = id
-    leaderSessionID = Some(newLeaderSessionID)
 
     // start the network stack, now that we have the JobManager actor reference
     try {
       network.associateWithTaskManagerAndJobManager(
-        new AkkaActorGateway(jobManager, leaderSessionID),
-        new AkkaActorGateway(self, leaderSessionID)
+        new AkkaActorGateway(jobManager, leaderSessionID.orNull),
+        new AkkaActorGateway(self, leaderSessionID.orNull)
       )
 
 
@@ -791,7 +792,7 @@ class TaskManager(
 
     // de-register from the JobManager (faster detection of disconnect)
     currentJobManager foreach {
-      _ ! decorateMessage(Disconnect(s"TaskManager ${self.path} is shutting down."))
+      _ ! decorateMessage(Disconnect(s"TaskManager ${self.path} is disassociating"))
     }
 
     currentJobManager = None
@@ -812,13 +813,14 @@ class TaskManager(
     network.disassociate()
   }
 
-  private def handleJobManagerDisconnect(jobManager: ActorRef, msg: String): Unit = {
+  protected def handleJobManagerDisconnect(jobManager: ActorRef, msg: String): Unit = {
     if (isConnected && jobManager != null) {
 
       // check if it comes from our JobManager
       if (jobManager == currentJobManager.orNull) {
         try {
-          val message = "Disconnecting from JobManager: " + msg
+          val message = s"TaskManager ${self.path} disconnects from JobManager " +
+            s"${jobManager.path}: " + msg
           log.info(message)
 
           // cancel all our tasks with a proper error message
@@ -826,17 +828,6 @@ class TaskManager(
 
           // reset our state to disassociated
           disassociateFromJobManager()
-
-          // begin attempts to reconnect
-          val deadline: Option[Deadline] = config.maxRegistrationDuration.map(_.fromNow)
-          self ! decorateMessage(
-            TriggerTaskManagerRegistration(
-              currentRegistrationSessionID,
-              jobManagerAkkaURL,
-              TaskManager.INITIAL_REGISTRATION_TIMEOUT,
-              deadline,
-              1)
-          )
         }
         catch {
           // this is pretty bad, it leaves the TaskManager in a state where it cannot
@@ -883,8 +874,8 @@ class TaskManager(
       // create the task. this does not grab any TaskManager resources or download
       // and libraries - the operation does not block
 
-      val jobManagerGateway = new AkkaActorGateway(jobManagerActor, leaderSessionID)
-      val selfGateway = new AkkaActorGateway(self, leaderSessionID)
+      val jobManagerGateway = new AkkaActorGateway(jobManagerActor, leaderSessionID.orNull)
+      val selfGateway = new AkkaActorGateway(self, leaderSessionID.orNull)
 
       val task = new Task(
         tdd,
@@ -1108,6 +1099,58 @@ class TaskManager(
 
     self ! decorateMessage(Kill)
   }
+
+  override def notifyLeaderAddress(leaderAddress: String, leaderSessionID: UUID): Unit = {
+    self ! JobManagerLeaderAddress(leaderAddress, leaderSessionID)
+  }
+
+  /** Handles the notification about a new leader and its address. If the TaskManager is still
+    * connected to another JobManager, it first disconnects from it. If the new JobManager
+    * address is not null, then it starts the registration process.
+    *
+    * @param newJobManagerAkkaURL
+    * @param leaderSessionID
+    */
+  private def handleJobManagerLeaderAddress(
+      newJobManagerAkkaURL: String,
+      leaderSessionID: UUID)
+    : Unit = {
+
+    currentJobManager match {
+      case Some(jm) =>
+        handleJobManagerDisconnect(jm, s"JobManager ${newJobManagerAkkaURL} was elected as leader.")
+      case None =>
+    }
+
+    this.jobManagerAkkaURL = Option(newJobManagerAkkaURL)
+    this.leaderSessionID = Option(leaderSessionID)
+
+    triggerTaskManagerRegistration()
+  }
+
+  /** Starts the TaskManager's registration process to connect to the JobManager.
+    *
+    */
+  def triggerTaskManagerRegistration(): Unit = {
+    if(jobManagerAkkaURL.isDefined) {
+      // begin attempts to reconnect
+      val deadline: Option[Deadline] = config.maxRegistrationDuration.map(_.fromNow)
+
+      self ! decorateMessage(
+        TriggerTaskManagerRegistration(
+          jobManagerAkkaURL.get,
+          TaskManager.INITIAL_REGISTRATION_TIMEOUT,
+          deadline,
+          1)
+      )
+    }
+  }
+
+  override def handleError(exception: Exception): Unit = {
+    log.error("Error in leader retrieval service", exception)
+
+    self ! decorateMessage(PoisonPill)
+  }
 }
 
 /**
@@ -1278,12 +1321,7 @@ object TaskManager {
       taskManagerClass: Class[_ <: TaskManager])
     : Unit = {
 
-    val (jobManagerHostname, jobManagerPort) = getAndCheckJobManagerAddress(configuration)
-
-    val (taskManagerHostname, actorSystemPort) = selectNetworkInterfaceAndPort(
-      configuration,
-      jobManagerHostname,
-      jobManagerPort)
+    val (taskManagerHostname, actorSystemPort) = selectNetworkInterfaceAndPort(configuration)
 
     runTaskManager(
       taskManagerHostname,
@@ -1296,9 +1334,7 @@ object TaskManager {
   @throws(classOf[IOException])
   @throws(classOf[IllegalConfigurationException])
   def selectNetworkInterfaceAndPort(
-      configuration: Configuration,
-      jobManagerHostname: String,
-      jobManagerPort: Int)
+      configuration: Configuration)
     : (String, Int) = {
 
     var taskManagerHostname = configuration.getString(
@@ -1308,25 +1344,12 @@ object TaskManager {
       LOG.info("Using configured hostname/address for TaskManager: " + taskManagerHostname)
     }
     else {
-      // try to find out the hostname of the interface from which the TaskManager
-      // can connect to the JobManager. This involves a reverse name lookup
-      LOG.info("Trying to select the network interface and address to use " +
-        "by connecting to the configured JobManager.")
-
-      LOG.info(s"TaskManager will try to connect for $MAX_STARTUP_CONNECT_TIME milliseconds " +
-        s"before falling back to heuristics")
-
-      val jobManagerAddress = new InetSocketAddress(jobManagerHostname, jobManagerPort)
-      val taskManagerAddress = try {
-        // try to get the address for up to two minutes and start
-        // logging only after ten seconds
-        NetUtils.findConnectingAddress(jobManagerAddress,
-          MAX_STARTUP_CONNECT_TIME, STARTUP_CONNECT_LOG_SUPPRESS)
-      }
-      catch {
-        case t: Throwable => throw new IOException("TaskManager cannot find a network interface " +
-          "that can communicate with the JobManager (" + jobManagerAddress + ")", t)
-      }
+      val leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(configuration)
+      val lookupTimeout = AkkaUtils.getLookupTimeout(configuration)
+
+      val taskManagerAddress = LeaderRetrievalUtils.findConnectingAddress(
+        leaderRetrievalService,
+        lookupTimeout)
 
       taskManagerHostname = taskManagerAddress.getHostName()
       LOG.info(s"TaskManager will use hostname/address '${taskManagerHostname}' " +
@@ -1439,7 +1462,8 @@ object TaskManager {
         taskManagerSystem,
         taskManagerHostname,
         Some(TASK_MANAGER_NAME),
-        None, false,
+        None,
+        false,
         streamingMode,
         taskManagerClass)
 
@@ -1489,9 +1513,9 @@ object TaskManager {
    * @param taskManagerHostname The hostname/address that describes the TaskManager's data location.
    * @param taskManagerActorName Optionally the name of the TaskManager actor. If none is given,
    *                             the actor will use a random name.
-   * @param jobManagerPath Optionally, the JobManager actor path may be provided. If none is
-   *                       provided, the method will construct it automatically from the
-   *                       JobManager hostname an port specified in the configuration.
+   * @param leaderRetrievalServiceOption Optionally, a leader retrieval service can be provided. If
+   *                                     none is given, then a LeaderRetrievalService is
+   *                                     constructed from the configuration.
    * @param localTaskManagerCommunication If true, the TaskManager will not initiate the
    *                                      TCP network stack.
    * @param streamingMode The streaming mode to start the TaskManager in (streaming/batch-only)
@@ -1516,20 +1540,13 @@ object TaskManager {
       actorSystem: ActorSystem,
       taskManagerHostname: String,
       taskManagerActorName: Option[String],
-      jobManagerPath: Option[String],
+      leaderRetrievalServiceOption: Option[LeaderRetrievalService],
       localTaskManagerCommunication: Boolean,
       streamingMode: StreamingMode,
       taskManagerClass: Class[_ <: TaskManager])
     : ActorRef = {
 
-    // get and check the JobManager config
-    val jobManagerAkkaUrl: String = jobManagerPath.getOrElse {
-      val (jobManagerHostname, jobManagerPort) = getAndCheckJobManagerAddress(configuration)
-      val hostPort = new InetSocketAddress(jobManagerHostname, jobManagerPort)
-      JobManager.getRemoteJobManagerAkkaURL(hostPort)
-    }
-
-    val (taskManagerConfig : TaskManagerConfiguration,
+    val (taskManagerConfig : TaskManagerConfiguration,      
       netConfig: NetworkEnvironmentConfiguration,
       connectionInfo: InstanceConnectionInfo
     ) = parseTaskManagerConfiguration(
@@ -1596,9 +1613,9 @@ object TaskManager {
     // start the I/O manager last, it will create some temp directories.
     val ioManager: IOManager = new IOManagerAsync(taskManagerConfig.tmpDirPaths)
 
-    if (ZooKeeperUtil.isJobManagerHighAvailabilityEnabled(configuration)) {
-      // TODO @removeme @tillrohrmann Setup leader retrieval service
-      LOG.info("HA mode.")
+    val leaderRetrievalService = leaderRetrievalServiceOption match {
+      case Some(lrs) => lrs
+      case None => LeaderRetrievalUtils.createLeaderRetrievalService(configuration)
     }
 
     // create the actor properties (which define the actor constructor parameters)
@@ -1606,11 +1623,11 @@ object TaskManager {
       taskManagerClass,
       taskManagerConfig,
       connectionInfo,
-      jobManagerAkkaUrl,
       memoryManager,
       ioManager,
       network,
-      taskManagerConfig.numberOfSlots)
+      taskManagerConfig.numberOfSlots,
+      leaderRetrievalService)
 
     taskManagerActorName match {
       case Some(actorName) => actorSystem.actorOf(tmProps, actorName)
@@ -1639,7 +1656,7 @@ object TaskManager {
       timeout: FiniteDuration)
     : ActorRef = {
     try {
-      val future = AkkaUtils.getReference(taskManagerUrl, system, timeout)
+      val future = AkkaUtils.getActorRefFuture(taskManagerUrl, system, timeout)
       Await.result(future, timeout)
     }
     catch {
@@ -1756,8 +1773,13 @@ object TaskManager {
     val nettyConfig = if (localTaskManagerCommunication) {
       None
     } else {
-      Some(new NettyConfig(
-        connectionInfo.address(), connectionInfo.dataPort(), pageSize, configuration))
+      Some(
+        new NettyConfig(
+          connectionInfo.address(),
+          connectionInfo.dataPort(),
+          pageSize,
+          configuration)
+      )
     }
 
     // Default spill I/O mode for intermediate results
@@ -1822,7 +1844,7 @@ object TaskManager {
    * @param configuration The configuration to read the config values from.
    * @return A 2-tuple (hostname, port).
    */
-  private def getAndCheckJobManagerAddress(configuration: Configuration) : (String, Int) = {
+  def getAndCheckJobManagerAddress(configuration: Configuration) : (String, Int) = {
 
     val hostname = configuration.getString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, null)
 

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/test/java/org/apache/flink/runtime/akka/FlinkUntypedActorTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/akka/FlinkUntypedActorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/akka/FlinkUntypedActorTest.java
index 324b014..3ffc68f 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/akka/FlinkUntypedActorTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/akka/FlinkUntypedActorTest.java
@@ -30,7 +30,6 @@ import org.apache.flink.runtime.testingUtils.TestingUtils;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
-import scala.Option;
 
 import java.util.UUID;
 
@@ -51,10 +50,14 @@ public class FlinkUntypedActorTest {
 		JavaTestKit.shutdownActorSystem(actorSystem);
 	}
 
+	/**
+	 * Tests that LeaderSessionMessage messages with a wrong leader session ID are filtered
+	 * out.
+	 */
 	@Test
 	public void testLeaderSessionMessageFilteringOfFlinkUntypedActor() {
-		final Option<UUID> leaderSessionID = Option.apply(UUID.randomUUID());
-		final Option<UUID> oldSessionID = Option.apply(UUID.randomUUID());
+		final UUID leaderSessionID = UUID.randomUUID();
+		final UUID oldSessionID = UUID.randomUUID();
 
 		TestActorRef<PlainFlinkUntypedActor> actor = null;
 
@@ -76,9 +79,13 @@ public class FlinkUntypedActorTest {
 		}
 	}
 
+	/**
+	 * Tests that an exception is thrown, when the FlinkUntypedActore receives a message which
+	 * extends {@link RequiresLeaderSessionID} and is not wrapped in a LeaderSessionMessage.
+	 */
 	@Test
 	public void testThrowingExceptionWhenReceivingNonWrappedRequiresLeaderSessionIDMessage() {
-		final Option<UUID> leaderSessionID = Option.apply(UUID.randomUUID());
+		final UUID leaderSessionID = UUID.randomUUID();
 
 		TestActorRef<PlainFlinkUntypedActor> actor = null;
 
@@ -95,7 +102,8 @@ public class FlinkUntypedActorTest {
 						"message was sent without being wrapped in LeaderSessionMessage.");
 			} catch (Exception e) {
 				assertEquals("Received a message PlainRequiresLeaderSessionID " +
-						"without a leader session ID, even though it requires to have one.",
+						"without a leader session ID, even though the message requires a " +
+						"leader session ID.",
 						e.getMessage());
 			}
 
@@ -110,14 +118,14 @@ public class FlinkUntypedActorTest {
 		}
 	}
 
-
 	static class PlainFlinkUntypedActor extends FlinkUntypedActor {
 
-		private Option<UUID> leaderSessionID;
+		private UUID leaderSessionID;
 
 		private int messageCounter;
 
-		public PlainFlinkUntypedActor(Option<UUID> leaderSessionID) {
+		public PlainFlinkUntypedActor(UUID leaderSessionID) {
+
 			this.leaderSessionID = leaderSessionID;
 			this.messageCounter = 0;
 		}
@@ -128,7 +136,7 @@ public class FlinkUntypedActorTest {
 		}
 
 		@Override
-		protected Option<UUID> getLeaderSessionID() {
+		protected UUID getLeaderSessionID() {
 			return leaderSessionID;
 		}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CoordinatorShutdownTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CoordinatorShutdownTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CoordinatorShutdownTest.java
index b124304..f6e4ab8 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CoordinatorShutdownTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CoordinatorShutdownTest.java
@@ -20,6 +20,7 @@ package org.apache.flink.runtime.checkpoint;
 
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.akka.ListeningBehaviour;
 import org.apache.flink.runtime.executiongraph.ExecutionGraph;
 import org.apache.flink.runtime.instance.ActorGateway;
 import org.apache.flink.runtime.jobgraph.JobVertex;
@@ -30,6 +31,7 @@ import org.apache.flink.runtime.jobmanager.Tasks;
 
 import org.apache.flink.runtime.messages.JobManagerMessages;
 import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
+import org.apache.flink.runtime.testingUtils.TestingUtils;
 import org.junit.Test;
 
 import scala.concurrent.Await;
@@ -49,8 +51,9 @@ public class CoordinatorShutdownTest {
 		LocalFlinkMiniCluster cluster = null;
 		try {
 			Configuration noTaskManagerConfig = new Configuration();
-			noTaskManagerConfig.setInteger(ConfigConstants.LOCAL_INSTANCE_MANAGER_NUMBER_TASK_MANAGER, 0);
+			noTaskManagerConfig.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, 0);
 			cluster = new LocalFlinkMiniCluster(noTaskManagerConfig, true);
+			cluster.start();
 			
 			// build a test graph with snapshotting enabled
 			JobVertex vertex = new JobVertex("Test Vertex");
@@ -60,17 +63,19 @@ public class CoordinatorShutdownTest {
 			JobGraph testGraph = new JobGraph("test job", vertex);
 			testGraph.setSnapshotSettings(new JobSnapshottingSettings(vertexIdList, vertexIdList, vertexIdList, 5000));
 			
-			ActorGateway jobManager = cluster.getJobManagerGateway();
+			ActorGateway jmGateway = cluster.getLeaderGateway(TestingUtils.TESTING_DURATION());
 
 			FiniteDuration timeout = new FiniteDuration(60, TimeUnit.SECONDS);
-			JobManagerMessages.SubmitJob submitMessage = new JobManagerMessages.SubmitJob(testGraph, false);
+			JobManagerMessages.SubmitJob submitMessage = new JobManagerMessages.SubmitJob(
+					testGraph,
+					ListeningBehaviour.EXECUTION_RESULT);
 			
 			// submit is successful, but then the job dies because no TaskManager / slot is available
-			Future<Object> submitFuture = jobManager.ask(submitMessage, timeout);
+			Future<Object> submitFuture = jmGateway.ask(submitMessage, timeout);
 			Await.result(submitFuture, timeout);
 
 			// get the execution graph and make sure the coordinator is properly shut down
-			Future<Object> jobRequestFuture = jobManager.ask(
+			Future<Object> jobRequestFuture = jmGateway.ask(
 					new JobManagerMessages.RequestJob(testGraph.getJobID()),
 					timeout);
 			
@@ -99,6 +104,7 @@ public class CoordinatorShutdownTest {
 		LocalFlinkMiniCluster cluster = null;
 		try {
 			cluster = new LocalFlinkMiniCluster(new Configuration(), true);
+			cluster.start();
 			
 			// build a test graph with snapshotting enabled
 			JobVertex vertex = new JobVertex("Test Vertex");
@@ -108,17 +114,19 @@ public class CoordinatorShutdownTest {
 			JobGraph testGraph = new JobGraph("test job", vertex);
 			testGraph.setSnapshotSettings(new JobSnapshottingSettings(vertexIdList, vertexIdList, vertexIdList, 5000));
 			
-			ActorGateway jobManager = cluster.getJobManagerGateway();
+			ActorGateway jmGateway = cluster.getLeaderGateway(TestingUtils.TESTING_DURATION());
 
 			FiniteDuration timeout = new FiniteDuration(60, TimeUnit.SECONDS);
-			JobManagerMessages.SubmitJob submitMessage = new JobManagerMessages.SubmitJob(testGraph, false);
+			JobManagerMessages.SubmitJob submitMessage = new JobManagerMessages.SubmitJob(
+					testGraph,
+					ListeningBehaviour.EXECUTION_RESULT);
 
 			// submit is successful, but then the job dies because no TaskManager / slot is available
-			Future<Object> submitFuture = jobManager.ask(submitMessage, timeout);
+			Future<Object> submitFuture = jmGateway.ask(submitMessage, timeout);
 			Await.result(submitFuture, timeout);
 
 			// get the execution graph and make sure the coordinator is properly shut down
-			Future<Object> jobRequestFuture = jobManager.ask(
+			Future<Object> jobRequestFuture = jmGateway.ask(
 					new JobManagerMessages.RequestJob(testGraph.getJobID()),
 					timeout);
 

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexCancelTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexCancelTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexCancelTest.java
index e9b67af..2daa62e 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexCancelTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexCancelTest.java
@@ -35,7 +35,8 @@ import org.apache.flink.api.common.JobID;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.jobmanager.scheduler.Scheduler;
 import org.apache.flink.runtime.messages.Messages;
-import org.apache.flink.runtime.messages.TaskMessages;
+import org.apache.flink.runtime.messages.TaskMessages.SubmitTask;
+import org.apache.flink.runtime.messages.TaskMessages.CancelTask;
 import org.apache.flink.runtime.messages.TaskMessages.TaskOperationResult;
 import org.apache.flink.runtime.testingUtils.TestingUtils;
 
@@ -574,9 +575,9 @@ public class ExecutionVertexCancelTest {
 		@Override
 		public Object handleMessage(Object message) throws Exception {
 			Object result;
-			if(message instanceof TaskMessages.SubmitTask) {
+			if(message instanceof SubmitTask) {
 				result = Messages.getAcknowledge();
-			} else if(message instanceof TaskMessages.CancelTask) {
+			} else if(message instanceof CancelTask) {
 				index++;
 
 				if(index >= results.length){

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/test/java/org/apache/flink/runtime/instance/BaseTestingActorGateway.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/BaseTestingActorGateway.java b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/BaseTestingActorGateway.java
index 2e62781..62bc96b 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/BaseTestingActorGateway.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/BaseTestingActorGateway.java
@@ -20,7 +20,6 @@ package org.apache.flink.runtime.instance;
 
 import akka.actor.ActorRef;
 import akka.dispatch.Futures;
-import scala.Option;
 import scala.concurrent.ExecutionContext;
 import scala.concurrent.Future;
 import scala.concurrent.duration.FiniteDuration;
@@ -114,7 +113,7 @@ abstract public class BaseTestingActorGateway implements ActorGateway {
 	}
 
 	@Override
-	public Option<UUID> leaderSessionID() {
-		return Option.empty();
+	public UUID leaderSessionID() {
+		return null;
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/test/java/org/apache/flink/runtime/instance/DummyActorGateway.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/DummyActorGateway.java b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/DummyActorGateway.java
index 10762f2..3d27611 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/DummyActorGateway.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/DummyActorGateway.java
@@ -19,7 +19,6 @@
 package org.apache.flink.runtime.instance;
 
 import akka.actor.ActorRef;
-import scala.Option;
 import scala.concurrent.ExecutionContext;
 import scala.concurrent.Future;
 import scala.concurrent.duration.FiniteDuration;
@@ -62,7 +61,7 @@ public class DummyActorGateway implements ActorGateway {
 	}
 
 	@Override
-	public Option<UUID> leaderSessionID() {
-		return Option.<UUID>empty();
+	public UUID leaderSessionID() {
+		return null;
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/test/java/org/apache/flink/runtime/instance/InstanceManagerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/InstanceManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/InstanceManagerTest.java
index 2c01f1f..a0166eb 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/InstanceManagerTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/InstanceManagerTest.java
@@ -42,8 +42,6 @@ import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
-import scala.Option;
-import scala.Some;
 
 /**
  * Tests for {@link org.apache.flink.runtime.instance.InstanceManager}.
@@ -52,7 +50,7 @@ public class InstanceManagerTest{
 
 	static ActorSystem system;
 
-	static Option<UUID> leaderSessionID = new Some<UUID>(UUID.randomUUID());
+	static UUID leaderSessionID = UUID.randomUUID();
 
 	@BeforeClass
 	public static void setup(){

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/ServerTransportErrorHandlingTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/ServerTransportErrorHandlingTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/ServerTransportErrorHandlingTest.java
index 6978225..1515f83 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/ServerTransportErrorHandlingTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/ServerTransportErrorHandlingTest.java
@@ -27,7 +27,6 @@ import org.apache.flink.runtime.io.network.buffer.BufferProvider;
 import org.apache.flink.runtime.io.network.buffer.NetworkBufferPool;
 import org.apache.flink.runtime.io.network.netty.CancelPartitionRequestTest.InfiniteSubpartitionView;
 import org.apache.flink.runtime.io.network.netty.NettyTestUtil.NettyServerAndClient;
-import org.apache.flink.runtime.io.network.netty.exception.RemoteTransportException;
 import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
 import org.apache.flink.runtime.io.network.partition.ResultPartitionManager;
 import org.apache.flink.runtime.io.network.partition.consumer.InputChannelID;

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PartialConsumePipelinedResultTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PartialConsumePipelinedResultTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PartialConsumePipelinedResultTest.java
index e9f3a62..fdbffaa 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PartialConsumePipelinedResultTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PartialConsumePipelinedResultTest.java
@@ -51,13 +51,15 @@ public class PartialConsumePipelinedResultTest {
 	@BeforeClass
 	public static void setUp() throws Exception {
 		final Configuration config = new Configuration();
-		config.setInteger(ConfigConstants.LOCAL_INSTANCE_MANAGER_NUMBER_TASK_MANAGER, NUMBER_OF_TMS);
+		config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, NUMBER_OF_TMS);
 		config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, NUMBER_OF_SLOTS_PER_TM);
 		config.setString(ConfigConstants.AKKA_ASK_TIMEOUT, TestingUtils.DEFAULT_AKKA_ASK_TIMEOUT());
 		config.setInteger(ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY, NUMBER_OF_NETWORK_BUFFERS);
 
 		flink = new TestingCluster(config, true);
 
+		flink.start();
+
 		jobClient = JobClient.startJobClientActorSystem(flink.configuration());
 	}
 
@@ -102,7 +104,7 @@ public class PartialConsumePipelinedResultTest {
 
 		JobClient.submitJobAndWait(
 				jobClient,
-				flink.getJobManagerGateway(),
+				flink.getLeaderGateway(TestingUtils.TESTING_DURATION()),
 				jobGraph,
 				TestingUtils.TESTING_DURATION(),
 				false, this.getClass().getClassLoader());

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannelTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannelTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannelTest.java
index 938c2ad..cd56318 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannelTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannelTest.java
@@ -24,13 +24,10 @@ import org.apache.flink.api.common.JobID;
 import org.apache.flink.runtime.execution.CancelTaskException;
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
-import org.apache.flink.runtime.io.network.ConnectionID;
-import org.apache.flink.runtime.io.network.ConnectionManager;
 import org.apache.flink.runtime.io.network.TaskEventDispatcher;
 import org.apache.flink.runtime.io.network.buffer.BufferPool;
 import org.apache.flink.runtime.io.network.buffer.BufferProvider;
 import org.apache.flink.runtime.io.network.buffer.NetworkBufferPool;
-import org.apache.flink.runtime.io.network.netty.PartitionRequestClient;
 import org.apache.flink.runtime.io.network.netty.PartitionStateChecker;
 import org.apache.flink.runtime.io.network.partition.PartitionNotFoundException;
 import org.apache.flink.runtime.io.network.partition.ResultPartition;

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerProcessReapingTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerProcessReapingTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerProcessReapingTest.java
index be73bf5..b2aaab0 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerProcessReapingTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerProcessReapingTest.java
@@ -101,7 +101,7 @@ public class JobManagerProcessReapingTest {
 			Throwable lastError = null;
 			for (int i = 0; i < 40; i++) {
 				try {
-					jobManagerRef = JobManager.getJobManagerRemoteReference(
+					jobManagerRef = JobManager.getJobManagerActorRef(
 							new InetSocketAddress("localhost", jobManagerPort),
 							localSystem, new FiniteDuration(25, TimeUnit.SECONDS));
 					break;

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerTest.java
index 217f46e..db6df75 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerTest.java
@@ -26,6 +26,7 @@ import com.typesafe.config.Config;
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.akka.AkkaUtils;
+import org.apache.flink.runtime.akka.ListeningBehaviour;
 import org.apache.flink.runtime.execution.ExecutionState;
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
 import org.apache.flink.runtime.executiongraph.ExecutionGraph;
@@ -49,16 +50,15 @@ import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages.Execution
 import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages.RequestExecutionGraph;
 import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages.WaitForAllVerticesToBeRunningOrFinished;
 
+import org.apache.flink.runtime.testingUtils.TestingUtils;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
-import scala.Option;
 import scala.Some;
 import scala.Tuple2;
 
 import java.net.InetAddress;
-import java.util.UUID;
 
 import static org.apache.flink.runtime.io.network.partition.ResultPartitionType.PIPELINED;
 import static org.apache.flink.runtime.testingUtils.TestingUtils.DEFAULT_AKKA_ASK_TIMEOUT;
@@ -121,13 +121,18 @@ public class JobManagerTest {
 				final JobGraph jobGraph = new JobGraph("Blocking test job", sender);
 				final JobID jid = jobGraph.getJobID();
 
-				final ActorGateway jobManagerGateway = cluster.getJobManagerGateway();
+				final ActorGateway jobManagerGateway = cluster.getLeaderGateway(
+						TestingUtils.TESTING_DURATION());
 
 				// we can set the leader session ID to None because we don't use this gateway to send messages
-				final ActorGateway testActorGateway = new AkkaActorGateway(getTestActor(), Option.<UUID>empty());
+				final ActorGateway testActorGateway = new AkkaActorGateway(getTestActor(), null);
 
 				// Submit the job and wait for all vertices to be running
-				jobManagerGateway.tell(new SubmitJob(jobGraph, false), testActorGateway);
+				jobManagerGateway.tell(
+						new SubmitJob(
+								jobGraph,
+								ListeningBehaviour.EXECUTION_RESULT),
+						testActorGateway);
 				expectMsgClass(JobManagerMessages.JobSubmitSuccess.class);
 
 				jobManagerGateway.tell(

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobSubmitTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobSubmitTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobSubmitTest.java
index 560fc30..bba1460 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobSubmitTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobSubmitTest.java
@@ -20,16 +20,21 @@ package org.apache.flink.runtime.jobmanager;
 
 import akka.actor.ActorRef;
 import akka.actor.ActorSystem;
+import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.StreamingMode;
 import org.apache.flink.runtime.akka.AkkaUtils;
+import org.apache.flink.runtime.akka.ListeningBehaviour;
 import org.apache.flink.runtime.blob.BlobClient;
 import org.apache.flink.runtime.blob.BlobKey;
 import org.apache.flink.runtime.client.JobExecutionException;
 import org.apache.flink.runtime.instance.ActorGateway;
 import org.apache.flink.runtime.jobgraph.JobVertex;
 import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
 import org.apache.flink.runtime.messages.JobManagerMessages;
+import org.apache.flink.runtime.net.NetUtils;
+import org.apache.flink.runtime.util.LeaderRetrievalUtils;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -54,13 +59,18 @@ public class JobSubmitTest {
 	private static final FiniteDuration timeout = new FiniteDuration(5000, TimeUnit.MILLISECONDS);
 
 	private static ActorSystem jobManagerSystem;
-	private static ActorGateway jobManager;
+	private static ActorGateway jmGateway;
 
 	@BeforeClass
 	public static void setupJobManager() {
 		Configuration config = new Configuration();
 
-		scala.Option<Tuple2<String, Object>> listeningAddress = scala.Option.empty();
+		int port = NetUtils.getAvailablePort();
+
+		config.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, "localhost");
+		config.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, port);
+
+		scala.Option<Tuple2<String, Object>> listeningAddress = scala.Option.apply(new Tuple2<String, Object>("localhost", port));
 		jobManagerSystem = AkkaUtils.createActorSystem(config, listeningAddress);
 		ActorRef jobManagerActorRef = JobManager.startJobManagerActors(
 				config,
@@ -68,12 +78,16 @@ public class JobSubmitTest {
 				StreamingMode.BATCH_ONLY)._1();
 
 		try {
-			jobManager = JobManager.getJobManagerGateway(jobManagerActorRef, timeout);
+			LeaderRetrievalService lrs = LeaderRetrievalUtils.createLeaderRetrievalService(config);
+
+			jmGateway = LeaderRetrievalUtils.retrieveLeaderGateway(
+					lrs,
+					jobManagerSystem,
+					timeout
+			);
 		} catch (Exception e) {
 			fail("Could not retrieve the JobManager gateway. " + e.getMessage());
 		}
-
-
 	}
 
 	@AfterClass
@@ -92,7 +106,7 @@ public class JobSubmitTest {
 			JobGraph jg = new JobGraph("test job", jobVertex);
 
 			// request the blob port from the job manager
-			Future<Object> future = jobManager.ask(JobManagerMessages.getRequestBlobManagerPort(), timeout);
+			Future<Object> future = jmGateway.ask(JobManagerMessages.getRequestBlobManagerPort(), timeout);
 			int blobPort = (Integer) Await.result(future, timeout);
 
 			// upload two dummy bytes and add their keys to the job graph as dependencies
@@ -113,7 +127,11 @@ public class JobSubmitTest {
 			jg.addBlob(key2);
 
 			// submit the job
-			Future<Object> submitFuture = jobManager.ask(new JobManagerMessages.SubmitJob(jg, false), timeout);
+			Future<Object> submitFuture = jmGateway.ask(
+					new JobManagerMessages.SubmitJob(
+							jg,
+							ListeningBehaviour.EXECUTION_RESULT),
+					timeout);
 			try {
 				Await.result(submitFuture, timeout);
 			}
@@ -152,7 +170,11 @@ public class JobSubmitTest {
 			JobGraph jg = new JobGraph("test job", jobVertex);
 
 			// submit the job
-			Future<Object> submitFuture = jobManager.ask(new JobManagerMessages.SubmitJob(jg, false), timeout);
+			Future<Object> submitFuture = jmGateway.ask(
+					new JobManagerMessages.SubmitJob(
+							jg,
+							ListeningBehaviour.EXECUTION_RESULT),
+					timeout);
 			try {
 				Await.result(submitFuture, timeout);
 			}

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/JobManagerLeaderElectionTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/JobManagerLeaderElectionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/JobManagerLeaderElectionTest.java
new file mode 100644
index 0000000..e6067d0
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/JobManagerLeaderElectionTest.java
@@ -0,0 +1,173 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.leaderelection;
+
+import akka.actor.ActorRef;
+import akka.actor.ActorSystem;
+import akka.actor.PoisonPill;
+import akka.actor.Props;
+import akka.pattern.Patterns;
+import akka.testkit.JavaTestKit;
+import akka.util.Timeout;
+import org.apache.curator.test.TestingCluster;
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.StreamingMode;
+import org.apache.flink.runtime.akka.AkkaUtils;
+import org.apache.flink.runtime.blob.BlobServer;
+import org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager;
+import org.apache.flink.runtime.instance.InstanceManager;
+import org.apache.flink.runtime.jobmanager.scheduler.Scheduler;
+import org.apache.flink.runtime.testingUtils.TestingJobManager;
+import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages;
+import org.apache.flink.runtime.testingUtils.TestingUtils;
+import org.apache.flink.runtime.util.LeaderElectionUtils;
+import org.apache.flink.util.TestLogger;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import scala.concurrent.Await;
+import scala.concurrent.Future;
+import scala.concurrent.duration.FiniteDuration;
+
+import java.util.concurrent.TimeUnit;
+
+public class JobManagerLeaderElectionTest extends TestLogger {
+
+	private static ActorSystem actorSystem;
+	private static TestingCluster testingCluster;
+	private static Timeout timeout = new Timeout(TestingUtils.TESTING_DURATION());
+	private static FiniteDuration duration = new FiniteDuration(5, TimeUnit.MINUTES);
+
+	@BeforeClass
+	public static void setup() throws Exception {
+		actorSystem = ActorSystem.create("TestingActorSystem");
+
+		testingCluster = new TestingCluster(3);
+		testingCluster.start();
+	}
+
+	@AfterClass
+	public static void teardown() throws Exception {
+		if (actorSystem != null) {
+			JavaTestKit.shutdownActorSystem(actorSystem);
+		}
+
+		if(testingCluster != null) {
+			testingCluster.stop();
+		}
+	}
+
+	/**
+	 * Tests that a single JobManager is elected as the leader by ZooKeeper.
+	 */
+	@Test
+	public void testLeaderElection() throws Exception {
+		final Configuration configuration = new Configuration();
+
+		configuration.setString(ConfigConstants.RECOVERY_MODE, "zookeeper");
+		configuration.setString(
+				ConfigConstants.ZOOKEEPER_QUORUM_KEY,
+				testingCluster.getConnectString());
+
+		ActorRef jm = null;
+
+		try {
+			Props jmProps = createJobManagerProps(configuration);
+
+			jm = actorSystem.actorOf(jmProps);
+
+			Future<Object> leaderFuture = Patterns.ask(
+					jm,
+					TestingJobManagerMessages.getNotifyWhenLeader(),
+					timeout);
+
+			Await.ready(leaderFuture, duration);
+		} finally {
+			TestingUtils.stopActor(jm);
+		}
+
+	}
+
+	/**
+	 * Tests that a second JobManager is elected as the leader once the previous leader dies.
+	 */
+	@Test
+	public void testLeaderReelection() throws Exception {
+		final Configuration configuration = new Configuration();
+
+		configuration.setString(ConfigConstants.RECOVERY_MODE, "zookeeper");
+		configuration.setString(
+				ConfigConstants.ZOOKEEPER_QUORUM_KEY,
+				testingCluster.getConnectString());
+
+		ActorRef jm;
+		ActorRef jm2 = null;
+
+		try {
+			Props jmProps = createJobManagerProps(configuration);
+
+			jm = actorSystem.actorOf(jmProps);
+
+			Future<Object> leaderFuture = Patterns.ask(
+					jm,
+					TestingJobManagerMessages.getNotifyWhenLeader(),
+					timeout);
+
+			Await.ready(leaderFuture, duration);
+
+			Props jmProps2 = createJobManagerProps(configuration);
+
+			jm2 = actorSystem.actorOf(jmProps2);
+
+			jm.tell(PoisonPill.getInstance(), ActorRef.noSender());
+
+			// now the second JobManager should be elected as the leader
+			Future<Object> leader2Future = Patterns.ask(
+					jm2,
+					TestingJobManagerMessages.getNotifyWhenLeader(),
+					timeout
+			);
+
+			Await.ready(leader2Future, duration);
+		} finally {
+			TestingUtils.stopActor(jm2);
+		}
+	}
+
+	private Props createJobManagerProps(Configuration configuration) throws Exception {
+		LeaderElectionService leaderElectionService = LeaderElectionUtils.
+				createLeaderElectionService(configuration);
+
+		return Props.create(
+				TestingJobManager.class,
+				configuration,
+				TestingUtils.defaultExecutionContext(),
+				new InstanceManager(),
+				new Scheduler(TestingUtils.defaultExecutionContext()),
+				new BlobLibraryCacheManager(new BlobServer(configuration), 10l),
+				ActorRef.noSender(),
+				1,
+				1L,
+				AkkaUtils.getDefaultTimeout(),
+				StreamingMode.BATCH_ONLY,
+				leaderElectionService
+		);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderChangeStateCleanupTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderChangeStateCleanupTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderChangeStateCleanupTest.java
new file mode 100644
index 0000000..8dd380e
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderChangeStateCleanupTest.java
@@ -0,0 +1,273 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.leaderelection;
+
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.StreamingMode;
+import org.apache.flink.runtime.instance.ActorGateway;
+import org.apache.flink.runtime.jobgraph.DistributionPattern;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.jobmanager.Tasks;
+import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup;
+import org.apache.flink.runtime.messages.JobManagerMessages;
+import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages.NotifyWhenJobRemoved;
+import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages.WaitForAllVerticesToBeRunningOrFinished;
+import org.apache.flink.runtime.testingUtils.TestingUtils;
+import org.apache.flink.util.TestLogger;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import scala.concurrent.Await;
+import scala.concurrent.Future;
+import scala.concurrent.duration.FiniteDuration;
+
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+import static org.junit.Assert.*;
+
+public class LeaderChangeStateCleanupTest extends TestLogger {
+
+	private static FiniteDuration timeout = TestingUtils.TESTING_DURATION();
+
+	private int numJMs = 2;
+	private int numTMs = 2;
+	private int numSlotsPerTM = 2;
+	private int parallelism = numTMs * numSlotsPerTM;
+
+	private Configuration configuration;
+	private LeaderElectionRetrievalTestingCluster cluster = null;
+	private JobGraph job = createBlockingJob(parallelism);
+
+	@Before
+	public void before() throws Exception {
+		Tasks.BlockingOnceReceiver$.MODULE$.blocking_$eq(true);
+
+		configuration = new Configuration();
+
+		configuration.setInteger(ConfigConstants.LOCAL_NUMBER_JOB_MANAGER, numJMs);
+		configuration.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, numTMs);
+		configuration.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, numSlotsPerTM);
+
+		cluster = new LeaderElectionRetrievalTestingCluster(configuration, true, false, StreamingMode.BATCH_ONLY);
+		cluster.start(false); // TaskManagers don't have to register at the JobManager
+
+		cluster.waitForActorsToBeAlive(); // we only wait until all actors are alive
+	}
+
+	@After
+	public void after() {
+		if(cluster != null) {
+			cluster.stop();
+		}
+	}
+
+	/**
+	 * Tests that a job is properly canceled in the case of a leader change. In such an event all
+	 * TaskManagers have to disconnect from the previous leader and connect to the newly elected
+	 * leader.
+	 */
+	@Test
+	public void testStateCleanupAfterNewLeaderElectionAndListenerNotification() throws Exception {
+		UUID leaderSessionID1 = UUID.randomUUID();
+		UUID leaderSessionID2 = UUID.randomUUID();
+
+		// first make JM(0) the leader
+		cluster.grantLeadership(0, leaderSessionID1);
+		// notify all listeners
+		cluster.notifyRetrievalListeners(0, leaderSessionID1);
+
+		cluster.waitForTaskManagersToBeRegistered();
+
+		// submit blocking job so that it is not finished when we cancel it
+		cluster.submitJobDetached(job);
+
+		ActorGateway jm = cluster.getLeaderGateway(timeout);
+
+		Future<Object> wait = jm.ask(new WaitForAllVerticesToBeRunningOrFinished(job.getJobID()), timeout);
+
+		Await.ready(wait, timeout);
+
+		Future<Object> jobRemoval = jm.ask(new NotifyWhenJobRemoved(job.getJobID()), timeout);
+
+		// make the JM(1) the new leader
+		cluster.grantLeadership(1, leaderSessionID2);
+		// notify all listeners about the event
+		cluster.notifyRetrievalListeners(1, leaderSessionID2);
+
+		Await.ready(jobRemoval, timeout);
+
+		cluster.waitForTaskManagersToBeRegistered();
+
+		ActorGateway jm2 = cluster.getLeaderGateway(timeout);
+
+		Future<Object> futureNumberSlots = jm2.ask(JobManagerMessages.getRequestTotalNumberOfSlots(), timeout);
+
+		// check that all TMs have registered at the new leader
+		int numberSlots = (Integer)Await.result(futureNumberSlots, timeout);
+
+		assertEquals(parallelism, numberSlots);
+
+		// try to resubmit now the non-blocking job, it should complete successfully
+		Tasks.BlockingOnceReceiver$.MODULE$.blocking_$eq(false);
+		cluster.submitJobAndWait(job, false, timeout);
+	}
+
+	/**
+	 * Tests that a job is properly canceled in the case of a leader change. However, this time only the
+	 * JMs are notified about the leader change and the TMs still believe the old leader to have
+	 * leadership.
+	 */
+	@Test
+	public void testStateCleanupAfterNewLeaderElection() throws Exception {
+		UUID leaderSessionID = UUID.randomUUID();
+		UUID newLeaderSessionID = UUID.randomUUID();
+
+		cluster.grantLeadership(0, leaderSessionID);
+		cluster.notifyRetrievalListeners(0, leaderSessionID);
+
+		cluster.waitForTaskManagersToBeRegistered();
+
+		// submit blocking job so that we can test job clean up
+		cluster.submitJobDetached(job);
+
+		ActorGateway jm = cluster.getLeaderGateway(timeout);
+
+		Future<Object> wait = jm.ask(new WaitForAllVerticesToBeRunningOrFinished(job.getJobID()), timeout);
+
+		Await.ready(wait, timeout);
+
+		Future<Object> jobRemoval = jm.ask(new NotifyWhenJobRemoved(job.getJobID()), timeout);
+
+		// only notify the JMs about the new leader JM(1)
+		cluster.grantLeadership(1, newLeaderSessionID);
+
+		// job should be removed anyway
+		Await.ready(jobRemoval, timeout);
+	}
+
+	/**
+	 * Tests that a job is properly canceled in the event of a leader change. However, this time
+	 * only the TMs are notified about the changing leader. This should be enough to cancel the
+	 * currently running job, though.
+	 */
+	@Test
+	public void testStateCleanupAfterListenerNotification() throws Exception {
+		UUID leaderSessionID = UUID.randomUUID();
+		UUID newLeaderSessionID = UUID.randomUUID();
+
+		cluster.grantLeadership(0, leaderSessionID);
+		cluster.notifyRetrievalListeners(0, leaderSessionID);
+
+		cluster.waitForTaskManagersToBeRegistered();
+
+		// submit blocking job
+		cluster.submitJobDetached(job);
+
+		ActorGateway jm = cluster.getLeaderGateway(timeout);
+
+		Future<Object> wait = jm.ask(new WaitForAllVerticesToBeRunningOrFinished(job.getJobID()), timeout);
+
+		Await.ready(wait, timeout);
+
+		Future<Object> jobRemoval = jm.ask(new NotifyWhenJobRemoved(job.getJobID()), timeout);
+
+		// notify listeners (TMs) about the leader change
+		cluster.notifyRetrievalListeners(1, newLeaderSessionID);
+
+		Await.ready(jobRemoval, timeout);
+	}
+
+	/**
+	 * Tests that the same JobManager can be reelected as the leader. Even though, the same JM
+	 * is elected as the next leader, all currently running jobs should be canceled properly and
+	 * all TMs should disconnect from the leader and then reconnect to it.
+	 */
+	@Test
+	public void testReelectionOfSameJobManager() throws Exception {
+		UUID leaderSessionID = UUID.randomUUID();
+		UUID newLeaderSessionID = UUID.randomUUID();
+
+		FiniteDuration shortTimeout = new FiniteDuration(20, TimeUnit.SECONDS);
+
+		cluster.grantLeadership(0, leaderSessionID);
+		cluster.notifyRetrievalListeners(0, leaderSessionID);
+
+		cluster.waitForTaskManagersToBeRegistered();
+
+		// submit blocking job
+		cluster.submitJobDetached(job);
+
+		ActorGateway jm = cluster.getLeaderGateway(timeout);
+
+		Future<Object> wait = jm.ask(new WaitForAllVerticesToBeRunningOrFinished(job.getJobID()), timeout);
+
+		Await.ready(wait, timeout);
+
+		Future<Object> jobRemoval = jm.ask(new NotifyWhenJobRemoved(job.getJobID()), timeout);
+
+		// make JM(0) again the leader --> this implies first a leadership revokal
+		cluster.grantLeadership(0, newLeaderSessionID);
+
+		Await.ready(jobRemoval, timeout);
+
+		// The TMs should not be able to reconnect since they don't know the current leader
+		// session ID
+		try {
+			cluster.waitForTaskManagersToBeRegistered(shortTimeout);
+			fail("TaskManager should not be able to register at JobManager.");
+		} catch (TimeoutException e) {
+			// expected exception since the TMs have still the old leader session ID
+		}
+
+		// notify the TMs about the new (old) leader
+		cluster.notifyRetrievalListeners(0, newLeaderSessionID);
+
+		cluster.waitForTaskManagersToBeRegistered();
+
+		// try to resubmit now the non-blocking job, it should complete successfully
+		Tasks.BlockingOnceReceiver$.MODULE$.blocking_$eq(false);
+		cluster.submitJobAndWait(job, false, timeout);
+
+	}
+
+	public JobGraph createBlockingJob(int parallelism) {
+		Tasks.BlockingOnceReceiver$.MODULE$.blocking_$eq(true);
+
+		JobVertex sender = new JobVertex("sender");
+		JobVertex receiver = new JobVertex("receiver");
+
+		sender.setInvokableClass(Tasks.Sender.class);
+		receiver.setInvokableClass(Tasks.BlockingOnceReceiver.class);
+
+		sender.setParallelism(parallelism);
+		receiver.setParallelism(parallelism);
+
+		receiver.connectNewDataSetAsInput(sender, DistributionPattern.POINTWISE);
+
+		SlotSharingGroup slotSharingGroup = new SlotSharingGroup();
+		sender.setSlotSharingGroup(slotSharingGroup);
+		receiver.setSlotSharingGroup(slotSharingGroup);
+
+		return new JobGraph("Blocking test job", sender, receiver);
+	}
+}


[02/10] flink git commit: [FLINK-2291] [runtime] Add ZooKeeper support to elect a leader from a set of JobManager. The leader will then be retrieved from ZooKeeper by the TaskManagers.

Posted by tr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java
index 939f564..effb3c8 100644
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java
@@ -113,67 +113,61 @@ public abstract class KafkaConsumerTestBase extends KafkaTestBase {
 	/**
 	 * Test that validates that checkpointing and checkpoint notification works properly
 	 */
-	public void runCheckpointingTest() {
-		try {
-			createTestTopic("testCheckpointing", 1, 1);
+	public void runCheckpointingTest() throws Exception {
+		createTestTopic("testCheckpointing", 1, 1);
 
-			FlinkKafkaConsumer<String> source = getConsumer("testCheckpointing", new JavaDefaultStringSchema(), standardProps);
-			Field pendingCheckpointsField = FlinkKafkaConsumer.class.getDeclaredField("pendingCheckpoints");
-			pendingCheckpointsField.setAccessible(true);
-			LinkedMap pendingCheckpoints = (LinkedMap) pendingCheckpointsField.get(source);
+		FlinkKafkaConsumer<String> source = getConsumer("testCheckpointing", new JavaDefaultStringSchema(), standardProps);
+		Field pendingCheckpointsField = FlinkKafkaConsumer.class.getDeclaredField("pendingCheckpoints");
+		pendingCheckpointsField.setAccessible(true);
+		LinkedMap pendingCheckpoints = (LinkedMap) pendingCheckpointsField.get(source);
 
-			Assert.assertEquals(0, pendingCheckpoints.size());
-			source.setRuntimeContext(new MockRuntimeContext(1, 0));
+		Assert.assertEquals(0, pendingCheckpoints.size());
+		source.setRuntimeContext(new MockRuntimeContext(1, 0));
 
-			final long[] initialOffsets = new long[] { 1337 };
+		final long[] initialOffsets = new long[] { 1337 };
 
-			// first restore
-			source.restoreState(initialOffsets);
+		// first restore
+		source.restoreState(initialOffsets);
 
-			// then open
-			source.open(new Configuration());
-			long[] state1 = source.snapshotState(1, 15);
+		// then open
+		source.open(new Configuration());
+		long[] state1 = source.snapshotState(1, 15);
 
-			assertArrayEquals(initialOffsets, state1);
+		assertArrayEquals(initialOffsets, state1);
 
-			long[] state2 = source.snapshotState(2, 30);
-			Assert.assertArrayEquals(initialOffsets, state2);
-			Assert.assertEquals(2, pendingCheckpoints.size());
+		long[] state2 = source.snapshotState(2, 30);
+		Assert.assertArrayEquals(initialOffsets, state2);
+		Assert.assertEquals(2, pendingCheckpoints.size());
 
-			source.notifyCheckpointComplete(1);
-			Assert.assertEquals(1, pendingCheckpoints.size());
+		source.notifyCheckpointComplete(1);
+		Assert.assertEquals(1, pendingCheckpoints.size());
 
-			source.notifyCheckpointComplete(2);
-			Assert.assertEquals(0, pendingCheckpoints.size());
+		source.notifyCheckpointComplete(2);
+		Assert.assertEquals(0, pendingCheckpoints.size());
 
-			source.notifyCheckpointComplete(666); // invalid checkpoint
-			Assert.assertEquals(0, pendingCheckpoints.size());
+		source.notifyCheckpointComplete(666); // invalid checkpoint
+		Assert.assertEquals(0, pendingCheckpoints.size());
 
-			// create 500 snapshots
-			for (int i = 100; i < 600; i++) {
-				source.snapshotState(i, 15 * i);
-			}
-			Assert.assertEquals(FlinkKafkaConsumer.MAX_NUM_PENDING_CHECKPOINTS, pendingCheckpoints.size());
+		// create 500 snapshots
+		for (int i = 100; i < 600; i++) {
+			source.snapshotState(i, 15 * i);
+		}
+		Assert.assertEquals(FlinkKafkaConsumer.MAX_NUM_PENDING_CHECKPOINTS, pendingCheckpoints.size());
 
-			// commit only the second last
-			source.notifyCheckpointComplete(598);
-			Assert.assertEquals(1, pendingCheckpoints.size());
+		// commit only the second last
+		source.notifyCheckpointComplete(598);
+		Assert.assertEquals(1, pendingCheckpoints.size());
 
-			// access invalid checkpoint
-			source.notifyCheckpointComplete(590);
+		// access invalid checkpoint
+		source.notifyCheckpointComplete(590);
 
-			// and the last
-			source.notifyCheckpointComplete(599);
-			Assert.assertEquals(0, pendingCheckpoints.size());
+		// and the last
+		source.notifyCheckpointComplete(599);
+		Assert.assertEquals(0, pendingCheckpoints.size());
 
-			source.close();
+		source.close();
 
-			deleteTestTopic("testCheckpointing");
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
+		deleteTestTopic("testCheckpointing");
 	}
 
 	/**
@@ -181,70 +175,64 @@ public abstract class KafkaConsumerTestBase extends KafkaTestBase {
 	 *
 	 * This test is only applicable if Teh Flink Kafka Consumer uses the ZooKeeperOffsetHandler.
 	 */
-	public void runOffsetInZookeeperValidationTest() {
-		try {
-			LOG.info("Starting testFlinkKafkaConsumerWithOffsetUpdates()");
+	public void runOffsetInZookeeperValidationTest() throws Exception {
+		LOG.info("Starting testFlinkKafkaConsumerWithOffsetUpdates()");
 
-			final String topicName = "testOffsetHacking";
-			final int parallelism = 3;
-			
-			createTestTopic(topicName, parallelism, 1);
+		final String topicName = "testOffsetHacking";
+		final int parallelism = 3;
 
-			StreamExecutionEnvironment env1 = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
-			env1.getConfig().disableSysoutLogging();
-			env1.enableCheckpointing(50);
-			env1.setNumberOfExecutionRetries(0);
-			env1.setParallelism(parallelism);
+		createTestTopic(topicName, parallelism, 1);
 
-			StreamExecutionEnvironment env2 = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
-			env2.getConfig().disableSysoutLogging();
-			env2.enableCheckpointing(50);
-			env2.setNumberOfExecutionRetries(0);
-			env2.setParallelism(parallelism);
+		StreamExecutionEnvironment env1 = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
+		env1.getConfig().disableSysoutLogging();
+		env1.enableCheckpointing(50);
+		env1.setNumberOfExecutionRetries(0);
+		env1.setParallelism(parallelism);
 
-			StreamExecutionEnvironment env3 = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
-			env3.getConfig().disableSysoutLogging();
-			env3.enableCheckpointing(50);
-			env3.setNumberOfExecutionRetries(0);
-			env3.setParallelism(parallelism);
+		StreamExecutionEnvironment env2 = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
+		env2.getConfig().disableSysoutLogging();
+		env2.enableCheckpointing(50);
+		env2.setNumberOfExecutionRetries(0);
+		env2.setParallelism(parallelism);
 
-			// write a sequence from 0 to 99 to each of the 3 partitions.
-			writeSequence(env1, topicName, 100, parallelism);
+		StreamExecutionEnvironment env3 = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
+		env3.getConfig().disableSysoutLogging();
+		env3.enableCheckpointing(50);
+		env3.setNumberOfExecutionRetries(0);
+		env3.setParallelism(parallelism);
 
-			readSequence(env2, standardProps, parallelism, topicName, 100, 0);
+		// write a sequence from 0 to 99 to each of the 3 partitions.
+		writeSequence(env1, topicName, 100, parallelism);
 
-			ZkClient zkClient = createZookeeperClient();
-			
-			long o1 = ZookeeperOffsetHandler.getOffsetFromZooKeeper(zkClient, standardCC.groupId(), topicName, 0);
-			long o2 = ZookeeperOffsetHandler.getOffsetFromZooKeeper(zkClient, standardCC.groupId(), topicName, 1);
-			long o3 = ZookeeperOffsetHandler.getOffsetFromZooKeeper(zkClient, standardCC.groupId(), topicName, 2);
+		readSequence(env2, standardProps, parallelism, topicName, 100, 0);
 
-			LOG.info("Got final offsets from zookeeper o1={}, o2={}, o3={}", o1, o2, o3);
+		ZkClient zkClient = createZookeeperClient();
 
-			assertTrue(o1 == FlinkKafkaConsumer.OFFSET_NOT_SET || (o1 >= 0 && o1 <= 100));
-			assertTrue(o2 == FlinkKafkaConsumer.OFFSET_NOT_SET || (o1 >= 0 && o1 <= 100));
-			assertTrue(o3 == FlinkKafkaConsumer.OFFSET_NOT_SET || (o1 >= 0 && o1 <= 100));
+		long o1 = ZookeeperOffsetHandler.getOffsetFromZooKeeper(zkClient, standardCC.groupId(), topicName, 0);
+		long o2 = ZookeeperOffsetHandler.getOffsetFromZooKeeper(zkClient, standardCC.groupId(), topicName, 1);
+		long o3 = ZookeeperOffsetHandler.getOffsetFromZooKeeper(zkClient, standardCC.groupId(), topicName, 2);
 
-			LOG.info("Manipulating offsets");
+		LOG.info("Got final offsets from zookeeper o1={}, o2={}, o3={}", o1, o2, o3);
 
-			// set the offset to 50 for the three partitions
-			ZookeeperOffsetHandler.setOffsetInZooKeeper(zkClient, standardCC.groupId(), topicName, 0, 49);
-			ZookeeperOffsetHandler.setOffsetInZooKeeper(zkClient, standardCC.groupId(), topicName, 1, 49);
-			ZookeeperOffsetHandler.setOffsetInZooKeeper(zkClient, standardCC.groupId(), topicName, 2, 49);
+		assertTrue(o1 == FlinkKafkaConsumer.OFFSET_NOT_SET || (o1 >= 0 && o1 <= 100));
+		assertTrue(o2 == FlinkKafkaConsumer.OFFSET_NOT_SET || (o1 >= 0 && o1 <= 100));
+		assertTrue(o3 == FlinkKafkaConsumer.OFFSET_NOT_SET || (o1 >= 0 && o1 <= 100));
 
-			zkClient.close();
-			
-			// create new env
-			readSequence(env3, standardProps, parallelism, topicName, 50, 50);
+		LOG.info("Manipulating offsets");
 
-			deleteTestTopic(topicName);
-			
-			LOG.info("Finished testFlinkKafkaConsumerWithOffsetUpdates()");
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
+		// set the offset to 50 for the three partitions
+		ZookeeperOffsetHandler.setOffsetInZooKeeper(zkClient, standardCC.groupId(), topicName, 0, 49);
+		ZookeeperOffsetHandler.setOffsetInZooKeeper(zkClient, standardCC.groupId(), topicName, 1, 49);
+		ZookeeperOffsetHandler.setOffsetInZooKeeper(zkClient, standardCC.groupId(), topicName, 2, 49);
+
+		zkClient.close();
+
+		// create new env
+		readSequence(env3, standardProps, parallelism, topicName, 50, 50);
+
+		deleteTestTopic(topicName);
+
+		LOG.info("Finished testFlinkKafkaConsumerWithOffsetUpdates()");
 	}
 
 	/**
@@ -255,655 +243,601 @@ public abstract class KafkaConsumerTestBase extends KafkaTestBase {
 	 * (generator source) --> (kafka sink)-[KAFKA-TOPIC]-(kafka source) --> (validating sink)
 	 * </pre>
 	 */
-	public void runSimpleConcurrentProducerConsumerTopology() {
-		try {
-			LOG.info("Starting runSimpleConcurrentProducerConsumerTopology()");
-
-			final String topic = "concurrentProducerConsumerTopic";
-			final int parallelism = 3;
-			final int elementsPerPartition = 100;
-			final int totalElements = parallelism * elementsPerPartition;
+	public void runSimpleConcurrentProducerConsumerTopology() throws Exception {
+		LOG.info("Starting runSimpleConcurrentProducerConsumerTopology()");
 
-			createTestTopic(topic, parallelism, 2);
+		final String topic = "concurrentProducerConsumerTopic";
+		final int parallelism = 3;
+		final int elementsPerPartition = 100;
+		final int totalElements = parallelism * elementsPerPartition;
 
-			final StreamExecutionEnvironment env =
-					StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
-			env.setParallelism(parallelism);
-			env.setNumberOfExecutionRetries(0);
-			env.getConfig().disableSysoutLogging();
+		createTestTopic(topic, parallelism, 2);
 
-			TypeInformation<Tuple2<Long, String>> longStringType = TypeInfoParser.parse("Tuple2<Long, String>");
+		final StreamExecutionEnvironment env =
+				StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
+		env.setParallelism(parallelism);
+		env.setNumberOfExecutionRetries(0);
+		env.getConfig().disableSysoutLogging();
 
-			TypeInformationSerializationSchema<Tuple2<Long, String>> sourceSchema =
-					new TypeInformationSerializationSchema<>(longStringType, env.getConfig());
+		TypeInformation<Tuple2<Long, String>> longStringType = TypeInfoParser.parse("Tuple2<Long, String>");
 
-			TypeInformationSerializationSchema<Tuple2<Long, String>> sinkSchema =
-					new TypeInformationSerializationSchema<>(longStringType, env.getConfig());
+		TypeInformationSerializationSchema<Tuple2<Long, String>> sourceSchema =
+				new TypeInformationSerializationSchema<>(longStringType, env.getConfig());
 
-			// ----------- add producer dataflow ----------
+		TypeInformationSerializationSchema<Tuple2<Long, String>> sinkSchema =
+				new TypeInformationSerializationSchema<>(longStringType, env.getConfig());
 
-			DataStream<Tuple2<Long, String>> stream = env.addSource(new RichParallelSourceFunction<Tuple2<Long,String>>() {
+		// ----------- add producer dataflow ----------
 
-				private boolean running = true;
+		DataStream<Tuple2<Long, String>> stream = env.addSource(new RichParallelSourceFunction<Tuple2<Long,String>>() {
 
-				@Override
-				public void run(SourceContext<Tuple2<Long, String>> ctx) {
-					int cnt = getRuntimeContext().getIndexOfThisSubtask() * elementsPerPartition;
-					int limit = cnt + elementsPerPartition;
+			private boolean running = true;
 
+			@Override
+			public void run(SourceContext<Tuple2<Long, String>> ctx) {
+				int cnt = getRuntimeContext().getIndexOfThisSubtask() * elementsPerPartition;
+				int limit = cnt + elementsPerPartition;
 
-					while (running && cnt < limit) {
-						ctx.collect(new Tuple2<Long, String>(1000L + cnt, "kafka-" + cnt));
-						cnt++;
-					}
-				}
 
-				@Override
-				public void cancel() {
-					running = false;
+				while (running && cnt < limit) {
+					ctx.collect(new Tuple2<Long, String>(1000L + cnt, "kafka-" + cnt));
+					cnt++;
 				}
-			});
-			stream.addSink(new KafkaSink<Tuple2<Long, String>>(brokerConnectionStrings, topic, sinkSchema));
-
-			// ----------- add consumer dataflow ----------
-
-			FlinkKafkaConsumer<Tuple2<Long, String>> source = getConsumer(topic, sourceSchema, standardProps);
+			}
 
-			DataStreamSource<Tuple2<Long, String>> consuming = env.addSource(source).setParallelism(parallelism);
+			@Override
+			public void cancel() {
+				running = false;
+			}
+		});
+		stream.addSink(new KafkaSink<Tuple2<Long, String>>(brokerConnectionStrings, topic, sinkSchema));
 
-			consuming.addSink(new RichSinkFunction<Tuple2<Long, String>>() {
+		// ----------- add consumer dataflow ----------
 
-				private int elCnt = 0;
-				private BitSet validator = new BitSet(totalElements);
+		FlinkKafkaConsumer<Tuple2<Long, String>> source = getConsumer(topic, sourceSchema, standardProps);
 
-				@Override
-				public void invoke(Tuple2<Long, String> value) throws Exception {
-					String[] sp = value.f1.split("-");
-					int v = Integer.parseInt(sp[1]);
+		DataStreamSource<Tuple2<Long, String>> consuming = env.addSource(source).setParallelism(parallelism);
 
-					assertEquals(value.f0 - 1000, (long) v);
+		consuming.addSink(new RichSinkFunction<Tuple2<Long, String>>() {
 
-					assertFalse("Received tuple twice", validator.get(v));
-					validator.set(v);
-					elCnt++;
+			private int elCnt = 0;
+			private BitSet validator = new BitSet(totalElements);
 
-					if (elCnt == totalElements) {
-						// check if everything in the bitset is set to true
-						int nc;
-						if ((nc = validator.nextClearBit(0)) != totalElements) {
-							fail("The bitset was not set to 1 on all elements. Next clear:"
-									+ nc + " Set: " + validator);
-						}
-						throw new SuccessException();
+			@Override
+			public void invoke(Tuple2<Long, String> value) throws Exception {
+				String[] sp = value.f1.split("-");
+				int v = Integer.parseInt(sp[1]);
+
+				assertEquals(value.f0 - 1000, (long) v);
+
+				assertFalse("Received tuple twice", validator.get(v));
+				validator.set(v);
+				elCnt++;
+
+				if (elCnt == totalElements) {
+					// check if everything in the bitset is set to true
+					int nc;
+					if ((nc = validator.nextClearBit(0)) != totalElements) {
+						fail("The bitset was not set to 1 on all elements. Next clear:"
+								+ nc + " Set: " + validator);
 					}
+					throw new SuccessException();
 				}
+			}
 
-				@Override
-				public void close() throws Exception {
-					super.close();
-				}
-			}).setParallelism(1);
+			@Override
+			public void close() throws Exception {
+				super.close();
+			}
+		}).setParallelism(1);
 
-			tryExecute(env, "runSimpleConcurrentProducerConsumerTopology");
+		tryExecute(env, "runSimpleConcurrentProducerConsumerTopology");
 
-			LOG.info("Finished runSimpleConcurrentProducerConsumerTopology()");
+		LOG.info("Finished runSimpleConcurrentProducerConsumerTopology()");
 
-			deleteTestTopic(topic);
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
+		deleteTestTopic(topic);
 	}
 
 	/**
 	 * Tests the proper consumption when having a 1:1 correspondence between kafka partitions and
 	 * Flink sources.
 	 */
-	public void runOneToOneExactlyOnceTest() {
-		try {
-			LOG.info("Starting runOneToOneExactlyOnceTest()");
+	public void runOneToOneExactlyOnceTest() throws Exception {
+		LOG.info("Starting runOneToOneExactlyOnceTest()");
 
-			final String topic = "oneToOneTopic";
-			final int parallelism = 5;
-			final int numElementsPerPartition = 1000;
-			final int totalElements = parallelism * numElementsPerPartition;
-			final int failAfterElements = numElementsPerPartition / 3;
-			
-			createTestTopic(topic, parallelism, 1);
-			
-			DataGenerators.generateRandomizedIntegerSequence(
-					StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort),
-					brokerConnectionStrings, 
-					topic, parallelism, numElementsPerPartition, true);
-			
-			// run the topology that fails and recovers
+		final String topic = "oneToOneTopic";
+		final int parallelism = 5;
+		final int numElementsPerPartition = 1000;
+		final int totalElements = parallelism * numElementsPerPartition;
+		final int failAfterElements = numElementsPerPartition / 3;
 
-			DeserializationSchema<Integer> schema = 
-					new TypeInformationSerializationSchema<>(BasicTypeInfo.INT_TYPE_INFO, new ExecutionConfig());
-			
-			StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
-			env.enableCheckpointing(500);
-			env.setParallelism(parallelism);
-			env.setNumberOfExecutionRetries(3);
-			env.getConfig().disableSysoutLogging();
-			
-			FlinkKafkaConsumer<Integer> kafkaSource = getConsumer(topic, schema, standardProps);
-			
-			env
-					.addSource(kafkaSource)
-					.map(new PartitionValidatingMapper(parallelism, 1))
-					.map(new FailingIdentityMapper<Integer>(failAfterElements))
-					.addSink(new ValidatingExactlyOnceSink(totalElements)).setParallelism(1);
+		createTestTopic(topic, parallelism, 1);
+
+		DataGenerators.generateRandomizedIntegerSequence(
+				StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort),
+				brokerConnectionStrings,
+				topic, parallelism, numElementsPerPartition, true);
+
+		// run the topology that fails and recovers
 
-			FailingIdentityMapper.failedBefore = false;
-			tryExecute(env, "One-to-one exactly once test");
+		DeserializationSchema<Integer> schema =
+				new TypeInformationSerializationSchema<>(BasicTypeInfo.INT_TYPE_INFO, new ExecutionConfig());
 
-			// this cannot be reliably checked, as checkpoints come in time intervals, and
-			// failures after a number of elements
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
+		env.enableCheckpointing(500);
+		env.setParallelism(parallelism);
+		env.setNumberOfExecutionRetries(3);
+		env.getConfig().disableSysoutLogging();
+
+		FlinkKafkaConsumer<Integer> kafkaSource = getConsumer(topic, schema, standardProps);
+
+		env
+				.addSource(kafkaSource)
+				.map(new PartitionValidatingMapper(parallelism, 1))
+				.map(new FailingIdentityMapper<Integer>(failAfterElements))
+				.addSink(new ValidatingExactlyOnceSink(totalElements)).setParallelism(1);
+
+		FailingIdentityMapper.failedBefore = false;
+		tryExecute(env, "One-to-one exactly once test");
+
+		// this cannot be reliably checked, as checkpoints come in time intervals, and
+		// failures after a number of elements
 //			assertTrue("Job did not do a checkpoint before the failure",
 //					FailingIdentityMapper.hasBeenCheckpointedBeforeFailure);
-			
-			deleteTestTopic(topic);
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
+
+		deleteTestTopic(topic);
 	}
 
 	/**
 	 * Tests the proper consumption when having fewer Flink sources than Kafka partitions, so
 	 * one Flink source will read multiple Kafka partitions.
 	 */
-	public void runOneSourceMultiplePartitionsExactlyOnceTest() {
-		try {
-			LOG.info("Starting runOneSourceMultiplePartitionsExactlyOnceTest()");
+	public void runOneSourceMultiplePartitionsExactlyOnceTest() throws Exception {
+		LOG.info("Starting runOneSourceMultiplePartitionsExactlyOnceTest()");
 
-			final String topic = "oneToManyTopic";
-			final int numPartitions = 5;
-			final int numElementsPerPartition = 1000;
-			final int totalElements = numPartitions * numElementsPerPartition;
-			final int failAfterElements = numElementsPerPartition / 3;
-			
-			final int parallelism = 2;
+		final String topic = "oneToManyTopic";
+		final int numPartitions = 5;
+		final int numElementsPerPartition = 1000;
+		final int totalElements = numPartitions * numElementsPerPartition;
+		final int failAfterElements = numElementsPerPartition / 3;
 
-			createTestTopic(topic, numPartitions, 1);
+		final int parallelism = 2;
 
-			DataGenerators.generateRandomizedIntegerSequence(
-					StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort),
-					brokerConnectionStrings,
-					topic, numPartitions, numElementsPerPartition, true);
+		createTestTopic(topic, numPartitions, 1);
 
-			// run the topology that fails and recovers
+		DataGenerators.generateRandomizedIntegerSequence(
+				StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort),
+				brokerConnectionStrings,
+				topic, numPartitions, numElementsPerPartition, true);
 
-			DeserializationSchema<Integer> schema =
-					new TypeInformationSerializationSchema<>(BasicTypeInfo.INT_TYPE_INFO, new ExecutionConfig());
+		// run the topology that fails and recovers
 
-			StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
-			env.enableCheckpointing(500);
-			env.setParallelism(parallelism);
-			env.setNumberOfExecutionRetries(3);
-			env.getConfig().disableSysoutLogging();
+		DeserializationSchema<Integer> schema =
+				new TypeInformationSerializationSchema<>(BasicTypeInfo.INT_TYPE_INFO, new ExecutionConfig());
 
-			FlinkKafkaConsumer<Integer> kafkaSource = getConsumer(topic, schema, standardProps);
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
+		env.enableCheckpointing(500);
+		env.setParallelism(parallelism);
+		env.setNumberOfExecutionRetries(3);
+		env.getConfig().disableSysoutLogging();
 
-			env
-					.addSource(kafkaSource)
-					.map(new PartitionValidatingMapper(numPartitions, 3))
-					.map(new FailingIdentityMapper<Integer>(failAfterElements))
-					.addSink(new ValidatingExactlyOnceSink(totalElements)).setParallelism(1);
+		FlinkKafkaConsumer<Integer> kafkaSource = getConsumer(topic, schema, standardProps);
 
-			FailingIdentityMapper.failedBefore = false;
-			tryExecute(env, "One-source-multi-partitions exactly once test");
+		env
+				.addSource(kafkaSource)
+				.map(new PartitionValidatingMapper(numPartitions, 3))
+				.map(new FailingIdentityMapper<Integer>(failAfterElements))
+				.addSink(new ValidatingExactlyOnceSink(totalElements)).setParallelism(1);
 
-			// this cannot be reliably checked, as checkpoints come in time intervals, and
-			// failures after a number of elements
+		FailingIdentityMapper.failedBefore = false;
+		tryExecute(env, "One-source-multi-partitions exactly once test");
+
+		// this cannot be reliably checked, as checkpoints come in time intervals, and
+		// failures after a number of elements
 //			assertTrue("Job did not do a checkpoint before the failure",
 //					FailingIdentityMapper.hasBeenCheckpointedBeforeFailure);
-			
-			deleteTestTopic(topic);
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
+
+		deleteTestTopic(topic);
 	}
 
 	/**
 	 * Tests the proper consumption when having more Flink sources than Kafka partitions, which means
 	 * that some Flink sources will read no partitions.
 	 */
-	public void runMultipleSourcesOnePartitionExactlyOnceTest() {
-		try {
-			LOG.info("Starting runMultipleSourcesOnePartitionExactlyOnceTest()");
+	public void runMultipleSourcesOnePartitionExactlyOnceTest() throws Exception {
+		LOG.info("Starting runMultipleSourcesOnePartitionExactlyOnceTest()");
 
-			final String topic = "manyToOneTopic";
-			final int numPartitions = 5;
-			final int numElementsPerPartition = 1000;
-			final int totalElements = numPartitions * numElementsPerPartition;
-			final int failAfterElements = numElementsPerPartition / 3;
+		final String topic = "manyToOneTopic";
+		final int numPartitions = 5;
+		final int numElementsPerPartition = 1000;
+		final int totalElements = numPartitions * numElementsPerPartition;
+		final int failAfterElements = numElementsPerPartition / 3;
 
-			final int parallelism = 8;
+		final int parallelism = 8;
 
-			createTestTopic(topic, numPartitions, 1);
+		createTestTopic(topic, numPartitions, 1);
 
-			DataGenerators.generateRandomizedIntegerSequence(
-					StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort),
-					brokerConnectionStrings,
-					topic, numPartitions, numElementsPerPartition, true);
+		DataGenerators.generateRandomizedIntegerSequence(
+				StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort),
+				brokerConnectionStrings,
+				topic, numPartitions, numElementsPerPartition, true);
 
-			// run the topology that fails and recovers
-			
-			DeserializationSchema<Integer> schema =
-					new TypeInformationSerializationSchema<>(BasicTypeInfo.INT_TYPE_INFO, new ExecutionConfig());
+		// run the topology that fails and recovers
 
-			StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
-			env.enableCheckpointing(500);
-			env.setParallelism(parallelism);
-			env.setNumberOfExecutionRetries(3);
-			env.getConfig().disableSysoutLogging();
-			env.setBufferTimeout(0);
+		DeserializationSchema<Integer> schema =
+				new TypeInformationSerializationSchema<>(BasicTypeInfo.INT_TYPE_INFO, new ExecutionConfig());
 
-			FlinkKafkaConsumer<Integer> kafkaSource = getConsumer(topic, schema, standardProps);
-			
-			env
-					.addSource(kafkaSource)
-					.map(new PartitionValidatingMapper(numPartitions, 1))
-					.map(new FailingIdentityMapper<Integer>(failAfterElements))
-					.addSink(new ValidatingExactlyOnceSink(totalElements)).setParallelism(1);
-			
-			FailingIdentityMapper.failedBefore = false;
-			tryExecute(env, "multi-source-one-partitions exactly once test");
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
+		env.enableCheckpointing(500);
+		env.setParallelism(parallelism);
+		env.setNumberOfExecutionRetries(3);
+		env.getConfig().disableSysoutLogging();
+		env.setBufferTimeout(0);
+
+		FlinkKafkaConsumer<Integer> kafkaSource = getConsumer(topic, schema, standardProps);
 
-			// this cannot be reliably checked, as checkpoints come in time intervals, and
-			// failures after a number of elements
+		env
+			.addSource(kafkaSource)
+			.map(new PartitionValidatingMapper(numPartitions, 1))
+			.map(new FailingIdentityMapper<Integer>(failAfterElements))
+			.addSink(new ValidatingExactlyOnceSink(totalElements)).setParallelism(1);
+
+		FailingIdentityMapper.failedBefore = false;
+		tryExecute(env, "multi-source-one-partitions exactly once test");
+
+		// this cannot be reliably checked, as checkpoints come in time intervals, and
+		// failures after a number of elements
 //			assertTrue("Job did not do a checkpoint before the failure",
 //					FailingIdentityMapper.hasBeenCheckpointedBeforeFailure);
-			
-			deleteTestTopic(topic);
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
+
+		deleteTestTopic(topic);
 	}
 	
 	
 	/**
 	 * Tests that the source can be properly canceled when reading full partitions. 
 	 */
-	public void runCancelingOnFullInputTest() {
-		try {
-			final String topic = "cancelingOnFullTopic";
+	public void runCancelingOnFullInputTest() throws Exception {
+		final String topic = "cancelingOnFullTopic";
 
-			final int parallelism = 3;
-			createTestTopic(topic, parallelism, 1);
+		final int parallelism = 3;
+		createTestTopic(topic, parallelism, 1);
 
-			// launch a producer thread
-			DataGenerators.InfiniteStringsGenerator generator =
-					new DataGenerators.InfiniteStringsGenerator(brokerConnectionStrings, topic);
-			generator.start();
+		// launch a producer thread
+		DataGenerators.InfiniteStringsGenerator generator =
+				new DataGenerators.InfiniteStringsGenerator(brokerConnectionStrings, topic);
+		generator.start();
 
-			// launch a consumer asynchronously
+		// launch a consumer asynchronously
 
-			final AtomicReference<Throwable> jobError = new AtomicReference<>();
+		final AtomicReference<Throwable> jobError = new AtomicReference<>();
 
-			final Runnable jobRunner = new Runnable() {
-				@Override
-				public void run() {
-					try {
-						final StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
-						env.setParallelism(parallelism);
-						env.enableCheckpointing(100);
-						env.getConfig().disableSysoutLogging();
+		final Runnable jobRunner = new Runnable() {
+			@Override
+			public void run() {
+				try {
+					final StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
+					env.setParallelism(parallelism);
+					env.enableCheckpointing(100);
+					env.getConfig().disableSysoutLogging();
 
-						FlinkKafkaConsumer<String> source = getConsumer(topic, new JavaDefaultStringSchema(), standardProps);
+					FlinkKafkaConsumer<String> source = getConsumer(topic, new JavaDefaultStringSchema(), standardProps);
 
-						env.addSource(source).addSink(new DiscardingSink<String>());
+					env.addSource(source).addSink(new DiscardingSink<String>());
 
-						env.execute();
-					}
-					catch (Throwable t) {
-						jobError.set(t);
-					}
+					env.execute();
 				}
-			};
-
-			Thread runnerThread = new Thread(jobRunner, "program runner thread");
-			runnerThread.start();
+				catch (Throwable t) {
+					jobError.set(t);
+				}
+			}
+		};
 
-			// wait a bit before canceling
-			Thread.sleep(2000);
+		Thread runnerThread = new Thread(jobRunner, "program runner thread");
+		runnerThread.start();
 
-			// cancel
-			JobManagerCommunicationUtils.cancelCurrentJob(flink.getJobManagerGateway());
+		// wait a bit before canceling
+		Thread.sleep(2000);
 
-			// wait for the program to be done and validate that we failed with the right exception
-			runnerThread.join();
+		// cancel
+		JobManagerCommunicationUtils.cancelCurrentJob(flink.getLeaderGateway(timeout));
 
-			Throwable failueCause = jobError.get();
-			assertNotNull("program did not fail properly due to canceling", failueCause);
-			assertTrue(failueCause.getMessage().contains("Job was cancelled"));
+		// wait for the program to be done and validate that we failed with the right exception
+		runnerThread.join();
 
-			if (generator.isAlive()) {
-				generator.shutdown();
-				generator.join();
-			}
-			else {
-				Throwable t = generator.getError();
-				if (t != null) {
-					t.printStackTrace();
-					fail("Generator failed: " + t.getMessage());
-				} else {
-					fail("Generator failed with no exception");
-				}
-			}
+		Throwable failueCause = jobError.get();
+		assertNotNull("program did not fail properly due to canceling", failueCause);
+		assertTrue(failueCause.getMessage().contains("Job was cancelled"));
 
-			deleteTestTopic(topic);
+		if (generator.isAlive()) {
+			generator.shutdown();
+			generator.join();
 		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
+		else {
+			Throwable t = generator.getError();
+			if (t != null) {
+				t.printStackTrace();
+				fail("Generator failed: " + t.getMessage());
+			} else {
+				fail("Generator failed with no exception");
+			}
 		}
+
+		deleteTestTopic(topic);
 	}
 
 	/**
 	 * Tests that the source can be properly canceled when reading empty partitions. 
 	 */
-	public void runCancelingOnEmptyInputTest() {
-		try {
-			final String topic = "cancelingOnEmptyInputTopic";
+	public void runCancelingOnEmptyInputTest() throws Exception {
+		final String topic = "cancelingOnEmptyInputTopic";
 
-			final int parallelism = 3;
-			createTestTopic(topic, parallelism, 1);
+		final int parallelism = 3;
+		createTestTopic(topic, parallelism, 1);
 
-			final AtomicReference<Throwable> error = new AtomicReference<>();
+		final AtomicReference<Throwable> error = new AtomicReference<>();
 
-			final Runnable jobRunner = new Runnable() {
-				@Override
-				public void run() {
-					try {
-						final StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
-						env.setParallelism(parallelism);
-						env.enableCheckpointing(100);
-						env.getConfig().disableSysoutLogging();
+		final Runnable jobRunner = new Runnable() {
+			@Override
+			public void run() {
+				try {
+					final StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
+					env.setParallelism(parallelism);
+					env.enableCheckpointing(100);
+					env.getConfig().disableSysoutLogging();
 
-						FlinkKafkaConsumer<String> source = getConsumer(topic, new JavaDefaultStringSchema(), standardProps);
+					FlinkKafkaConsumer<String> source = getConsumer(topic, new JavaDefaultStringSchema(), standardProps);
 
-						env.addSource(source).addSink(new DiscardingSink<String>());
+					env.addSource(source).addSink(new DiscardingSink<String>());
 
-						env.execute();
-					}
-					catch (Throwable t) {
-						error.set(t);
-					}
+					env.execute();
+				}
+				catch (Throwable t) {
+					error.set(t);
 				}
-			};
+			}
+		};
 
-			Thread runnerThread = new Thread(jobRunner, "program runner thread");
-			runnerThread.start();
+		Thread runnerThread = new Thread(jobRunner, "program runner thread");
+		runnerThread.start();
 
-			// wait a bit before canceling
-			Thread.sleep(2000);
+		// wait a bit before canceling
+		Thread.sleep(2000);
 
-			// cancel
-			JobManagerCommunicationUtils.cancelCurrentJob(flink.getJobManagerGateway());
+		// cancel
+		JobManagerCommunicationUtils.cancelCurrentJob(flink.getLeaderGateway(timeout));
 
-			// wait for the program to be done and validate that we failed with the right exception
-			runnerThread.join();
+		// wait for the program to be done and validate that we failed with the right exception
+		runnerThread.join();
 
-			Throwable failueCause = error.get();
-			assertNotNull("program did not fail properly due to canceling", failueCause);
-			assertTrue(failueCause.getMessage().contains("Job was cancelled"));
+		Throwable failueCause = error.get();
+		assertNotNull("program did not fail properly due to canceling", failueCause);
+		assertTrue(failueCause.getMessage().contains("Job was cancelled"));
 
-			deleteTestTopic(topic);
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
+		deleteTestTopic(topic);
 	}
 
 	/**
 	 * Tests that the source can be properly canceled when reading full partitions. 
 	 */
-	public void runFailOnDeployTest() {
-		try {
-			final String topic = "failOnDeployTopic";
-			
-			createTestTopic(topic, 2, 1);
+	public void runFailOnDeployTest() throws Exception {
+		final String topic = "failOnDeployTopic";
 
-			DeserializationSchema<Integer> schema =
-					new TypeInformationSerializationSchema<>(BasicTypeInfo.INT_TYPE_INFO, new ExecutionConfig());
+		createTestTopic(topic, 2, 1);
 
-			StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
-			env.setParallelism(12); // needs to be more that the mini cluster has slots
-			env.getConfig().disableSysoutLogging();
+		DeserializationSchema<Integer> schema =
+				new TypeInformationSerializationSchema<>(BasicTypeInfo.INT_TYPE_INFO, new ExecutionConfig());
 
-			FlinkKafkaConsumer<Integer> kafkaSource = getConsumer(topic, schema, standardProps);
-			
-			env
-					.addSource(kafkaSource)
-					.addSink(new DiscardingSink<Integer>());
-			
-			try {
-				env.execute();
-				fail("this test should fail with an exception");
-			}
-			catch (ProgramInvocationException e) {
-				
-				// validate that we failed due to a NoResourceAvailableException
-				Throwable cause = e.getCause();
-				int depth = 0;
-				boolean foundResourceException = false;
-				
-				while (cause != null && depth++ < 20) {
-					if (cause instanceof NoResourceAvailableException) {
-						foundResourceException = true;
-						break;
-					}
-					cause = cause.getCause();
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
+		env.setParallelism(12); // needs to be more that the mini cluster has slots
+		env.getConfig().disableSysoutLogging();
+
+		FlinkKafkaConsumer<Integer> kafkaSource = getConsumer(topic, schema, standardProps);
+
+		env
+				.addSource(kafkaSource)
+				.addSink(new DiscardingSink<Integer>());
+
+		try {
+			env.execute();
+			fail("this test should fail with an exception");
+		}
+		catch (ProgramInvocationException e) {
+
+			// validate that we failed due to a NoResourceAvailableException
+			Throwable cause = e.getCause();
+			int depth = 0;
+			boolean foundResourceException = false;
+
+			while (cause != null && depth++ < 20) {
+				if (cause instanceof NoResourceAvailableException) {
+					foundResourceException = true;
+					break;
 				}
-				
-				assertTrue("Wrong exception", foundResourceException);
+				cause = cause.getCause();
 			}
 
-			deleteTestTopic(topic);
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
+			assertTrue("Wrong exception", foundResourceException);
 		}
+
+		deleteTestTopic(topic);
 	}
 
 	/**
 	 * Test Flink's Kafka integration also with very big records (30MB)
 	 * see http://stackoverflow.com/questions/21020347/kafka-sending-a-15mb-message
 	 */
-	public void runBigRecordTestTopology() {
-		try {
-			LOG.info("Starting runBigRecordTestTopology()");
+	public void runBigRecordTestTopology() throws Exception {
+		LOG.info("Starting runBigRecordTestTopology()");
 
-			final String topic = "bigRecordTestTopic";
-			final int parallelism = 1; // otherwise, the kafka mini clusters may run out of heap space
-			
-			createTestTopic(topic, parallelism, 1);
+		final String topic = "bigRecordTestTopic";
+		final int parallelism = 1; // otherwise, the kafka mini clusters may run out of heap space
 
-			final TypeInformation<Tuple2<Long, byte[]>> longBytesInfo = TypeInfoParser.parse("Tuple2<Long, byte[]>");
+		createTestTopic(topic, parallelism, 1);
 
-			final TypeInformationSerializationSchema<Tuple2<Long, byte[]>> serSchema =
-					new TypeInformationSerializationSchema<>(longBytesInfo, new ExecutionConfig());
+		final TypeInformation<Tuple2<Long, byte[]>> longBytesInfo = TypeInfoParser.parse("Tuple2<Long, byte[]>");
 
-			final TypeInformationSerializationSchema<Tuple2<Long, byte[]>> deserSchema =
-					new TypeInformationSerializationSchema<>(longBytesInfo, new ExecutionConfig());
+		final TypeInformationSerializationSchema<Tuple2<Long, byte[]>> serSchema =
+				new TypeInformationSerializationSchema<>(longBytesInfo, new ExecutionConfig());
 
-			final StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
-			env.setNumberOfExecutionRetries(0);
-			env.getConfig().disableSysoutLogging();
-			env.enableCheckpointing(100);
-			env.setParallelism(parallelism);
+		final TypeInformationSerializationSchema<Tuple2<Long, byte[]>> deserSchema =
+				new TypeInformationSerializationSchema<>(longBytesInfo, new ExecutionConfig());
 
-			// add consuming topology:
-			Properties consumerProps = new Properties();
-			consumerProps.putAll(standardProps);
-			consumerProps.setProperty("fetch.message.max.bytes", Integer.toString(1024 * 1024 * 40));
-			consumerProps.setProperty("max.partition.fetch.bytes", Integer.toString(1024 * 1024 * 40)); // for the new fetcher
-			consumerProps.setProperty("queued.max.message.chunks", "1");
+		final StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
+		env.setNumberOfExecutionRetries(0);
+		env.getConfig().disableSysoutLogging();
+		env.enableCheckpointing(100);
+		env.setParallelism(parallelism);
 
-			FlinkKafkaConsumer<Tuple2<Long, byte[]>> source = getConsumer(topic, serSchema, consumerProps);
-			DataStreamSource<Tuple2<Long, byte[]>> consuming = env.addSource(source);
+		// add consuming topology:
+		Properties consumerProps = new Properties();
+		consumerProps.putAll(standardProps);
+		consumerProps.setProperty("fetch.message.max.bytes", Integer.toString(1024 * 1024 * 40));
+		consumerProps.setProperty("max.partition.fetch.bytes", Integer.toString(1024 * 1024 * 40)); // for the new fetcher
+		consumerProps.setProperty("queued.max.message.chunks", "1");
 
-			consuming.addSink(new SinkFunction<Tuple2<Long, byte[]>>() {
+		FlinkKafkaConsumer<Tuple2<Long, byte[]>> source = getConsumer(topic, serSchema, consumerProps);
+		DataStreamSource<Tuple2<Long, byte[]>> consuming = env.addSource(source);
 
-				private int elCnt = 0;
+		consuming.addSink(new SinkFunction<Tuple2<Long, byte[]>>() {
 
-				@Override
-				public void invoke(Tuple2<Long, byte[]> value) throws Exception {
-					elCnt++;
-					if (value.f0 == -1) {
-						// we should have seen 11 elements now.
-						if(elCnt == 11) {
-							throw new SuccessException();
-						} else {
-							throw new RuntimeException("There have been "+elCnt+" elements");
-						}
-					}
-					if(elCnt > 10) {
-						throw new RuntimeException("More than 10 elements seen: "+elCnt);
+			private int elCnt = 0;
+
+			@Override
+			public void invoke(Tuple2<Long, byte[]> value) throws Exception {
+				elCnt++;
+				if (value.f0 == -1) {
+					// we should have seen 11 elements now.
+					if(elCnt == 11) {
+						throw new SuccessException();
+					} else {
+						throw new RuntimeException("There have been "+elCnt+" elements");
 					}
 				}
-			});
+				if(elCnt > 10) {
+					throw new RuntimeException("More than 10 elements seen: "+elCnt);
+				}
+			}
+		});
 
-			// add producing topology
-			Properties producerProps = new Properties();
-			producerProps.setProperty("max.message.size", Integer.toString(1024 * 1024 * 30));
-			
-			DataStream<Tuple2<Long, byte[]>> stream = env.addSource(new RichSourceFunction<Tuple2<Long, byte[]>>() {
+		// add producing topology
+		Properties producerProps = new Properties();
+		producerProps.setProperty("max.message.size", Integer.toString(1024 * 1024 * 30));
 
-				private boolean running;
+		DataStream<Tuple2<Long, byte[]>> stream = env.addSource(new RichSourceFunction<Tuple2<Long, byte[]>>() {
 
-				@Override
-				public void open(Configuration parameters) throws Exception {
-					super.open(parameters);
-					running = true;
-				}
+			private boolean running;
 
-				@Override
-				public void run(SourceContext<Tuple2<Long, byte[]>> ctx) throws Exception {
-					Random rnd = new Random();
-					long cnt = 0;
-					int fifteenMb = 1024 * 1024 * 15;
+			@Override
+			public void open(Configuration parameters) throws Exception {
+				super.open(parameters);
+				running = true;
+			}
 
-					while (running) {
-						byte[] wl = new byte[fifteenMb + rnd.nextInt(fifteenMb)];
-						ctx.collect(new Tuple2<Long, byte[]>(cnt++, wl));
+			@Override
+			public void run(SourceContext<Tuple2<Long, byte[]>> ctx) throws Exception {
+				Random rnd = new Random();
+				long cnt = 0;
+				int fifteenMb = 1024 * 1024 * 15;
 
-						Thread.sleep(100);
+				while (running) {
+					byte[] wl = new byte[fifteenMb + rnd.nextInt(fifteenMb)];
+					ctx.collect(new Tuple2<Long, byte[]>(cnt++, wl));
 
-						if (cnt == 10) {
-							// signal end
-							ctx.collect(new Tuple2<Long, byte[]>(-1L, new byte[]{1}));
-							break;
-						}
+					Thread.sleep(100);
+
+					if (cnt == 10) {
+						// signal end
+						ctx.collect(new Tuple2<Long, byte[]>(-1L, new byte[]{1}));
+						break;
 					}
 				}
+			}
 
-				@Override
-				public void cancel() {
-					running = false;
-				}
-			});
+			@Override
+			public void cancel() {
+				running = false;
+			}
+		});
 
-			stream.addSink(new KafkaSink<Tuple2<Long, byte[]>>(brokerConnectionStrings, topic,
-					producerProps, deserSchema));
+		stream.addSink(new KafkaSink<Tuple2<Long, byte[]>>(brokerConnectionStrings, topic,
+				producerProps, deserSchema));
 
-			tryExecute(env, "big topology test");
+		tryExecute(env, "big topology test");
 
-			deleteTestTopic(topic);
-			
-			LOG.info("Finished runBigRecordTestTopology()");
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
+		deleteTestTopic(topic);
+
+		LOG.info("Finished runBigRecordTestTopology()");
 	}
 
 	
-	public void runBrokerFailureTest() {
-		try {
-			LOG.info("starting runBrokerFailureTest()");
-			
-			final String topic = "brokerFailureTestTopic";
+	public void runBrokerFailureTest() throws Exception {
+		LOG.info("starting runBrokerFailureTest()");
 
-			final int parallelism = 2;
-			final int numElementsPerPartition = 1000;
-			final int totalElements = parallelism * numElementsPerPartition;
-			final int failAfterElements = numElementsPerPartition / 3;
-			
+		final String topic = "brokerFailureTestTopic";
 
-			createTestTopic(topic, parallelism, 2);
-
-			DataGenerators.generateRandomizedIntegerSequence(
-					StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort),
-					brokerConnectionStrings,
-					topic, parallelism, numElementsPerPartition, true);
-
-			// find leader to shut down
-			ZkClient zkClient = createZookeeperClient();
-			PartitionMetadata firstPart = null;
-			do {
-				if (firstPart != null) {
-					LOG.info("Unable to find leader. error code {}", firstPart.errorCode());
-					// not the first try. Sleep a bit
-					Thread.sleep(150);
-				}
+		final int parallelism = 2;
+		final int numElementsPerPartition = 1000;
+		final int totalElements = parallelism * numElementsPerPartition;
+		final int failAfterElements = numElementsPerPartition / 3;
+
+
+		createTestTopic(topic, parallelism, 2);
+
+		DataGenerators.generateRandomizedIntegerSequence(
+				StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort),
+				brokerConnectionStrings,
+				topic, parallelism, numElementsPerPartition, true);
 
-				Seq<PartitionMetadata> partitionMetadata = AdminUtils.fetchTopicMetadataFromZk(topic, zkClient).partitionsMetadata();
-				firstPart = partitionMetadata.head();
+		// find leader to shut down
+		ZkClient zkClient = createZookeeperClient();
+		PartitionMetadata firstPart = null;
+		do {
+			if (firstPart != null) {
+				LOG.info("Unable to find leader. error code {}", firstPart.errorCode());
+				// not the first try. Sleep a bit
+				Thread.sleep(150);
 			}
-			while (firstPart.errorCode() != 0);
-			zkClient.close();
 
-			final String leaderToShutDown = firstPart.leader().get().connectionString();
-			LOG.info("Leader to shutdown {}", leaderToShutDown);
-			
-			
-			// run the topology that fails and recovers
+			Seq<PartitionMetadata> partitionMetadata = AdminUtils.fetchTopicMetadataFromZk(topic, zkClient).partitionsMetadata();
+			firstPart = partitionMetadata.head();
+		}
+		while (firstPart.errorCode() != 0);
+		zkClient.close();
 
-			DeserializationSchema<Integer> schema =
-					new TypeInformationSerializationSchema<>(BasicTypeInfo.INT_TYPE_INFO, new ExecutionConfig());
+		final String leaderToShutDown = firstPart.leader().get().connectionString();
+		LOG.info("Leader to shutdown {}", leaderToShutDown);
 
-			StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
-			env.setParallelism(parallelism);
-			env.enableCheckpointing(500);
-			env.setNumberOfExecutionRetries(3);
-			env.getConfig().disableSysoutLogging();
-			
 
-			FlinkKafkaConsumer<Integer> kafkaSource = getConsumer(topic, schema, standardProps);
+		// run the topology that fails and recovers
+
+		DeserializationSchema<Integer> schema =
+				new TypeInformationSerializationSchema<>(BasicTypeInfo.INT_TYPE_INFO, new ExecutionConfig());
 
-			env
-					.addSource(kafkaSource)
-					.map(new PartitionValidatingMapper(parallelism, 1))
-					.map(new BrokerKillingMapper<Integer>(leaderToShutDown, failAfterElements))
-					.addSink(new ValidatingExactlyOnceSink(totalElements)).setParallelism(1);
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
+		env.setParallelism(parallelism);
+		env.enableCheckpointing(500);
+		env.setNumberOfExecutionRetries(3);
+		env.getConfig().disableSysoutLogging();
 
-			BrokerKillingMapper.killedLeaderBefore = false;
-			tryExecute(env, "One-to-one exactly once test");
 
-			// this cannot be reliably checked, as checkpoints come in time intervals, and
-			// failures after a number of elements
+		FlinkKafkaConsumer<Integer> kafkaSource = getConsumer(topic, schema, standardProps);
+
+		env
+				.addSource(kafkaSource)
+				.map(new PartitionValidatingMapper(parallelism, 1))
+				.map(new BrokerKillingMapper<Integer>(leaderToShutDown, failAfterElements))
+				.addSink(new ValidatingExactlyOnceSink(totalElements)).setParallelism(1);
+
+		BrokerKillingMapper.killedLeaderBefore = false;
+		tryExecute(env, "One-to-one exactly once test");
+
+		// this cannot be reliably checked, as checkpoints come in time intervals, and
+		// failures after a number of elements
 //			assertTrue("Job did not do a checkpoint before the failure",
 //					BrokerKillingMapper.hasBeenCheckpointedBeforeFailure);
 
-			LOG.info("finished runBrokerFailureTest()");
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
+		LOG.info("finished runBrokerFailureTest()");
 	}
 
 	// ------------------------------------------------------------------------
@@ -961,8 +895,7 @@ public abstract class KafkaConsumerTestBase extends KafkaTestBase {
 		LOG.info("Successfully read sequence for verification");
 	}
 
-	private static void writeSequence(StreamExecutionEnvironment env, String topicName,
-									  final int numElements, int parallelism) throws Exception {
+	private static void writeSequence(StreamExecutionEnvironment env, String topicName, final int numElements, int parallelism) throws Exception {
 
 		TypeInformation<Tuple2<Integer, Integer>> resultType = TypeInfoParser.parse("Tuple2<Integer, Integer>");
 
@@ -1038,8 +971,8 @@ public abstract class KafkaConsumerTestBase extends KafkaTestBase {
 	}
 
 	private static void printTopic(String topicName, ConsumerConfig config,
-								   DeserializationSchema<?> deserializationSchema,
-								   int stopAfter) {
+								DeserializationSchema<?> deserializationSchema,
+								int stopAfter) {
 
 		List<MessageAndMetadata<byte[], byte[]>> contents = readTopicToList(topicName, config, stopAfter);
 		LOG.info("Printing contents of topic {} in consumer grouo {}", topicName, config.groupId());

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaITCase.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaITCase.java
index 0287392..334dfb2 100644
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaITCase.java
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaITCase.java
@@ -36,65 +36,65 @@ public class KafkaITCase extends KafkaConsumerTestBase {
 	// ------------------------------------------------------------------------
 	
 	@Test
-	public void testCheckpointing() {
+	public void testCheckpointing() throws Exception {
 		runCheckpointingTest();
 	}
 
 	@Test
-	public void testOffsetInZookeeper() {
+	public void testOffsetInZookeeper() throws Exception {
 		runOffsetInZookeeperValidationTest();
 	}
 	
 	@Test
-	public void testConcurrentProducerConsumerTopology() {
+	public void testConcurrentProducerConsumerTopology() throws Exception {
 		runSimpleConcurrentProducerConsumerTopology();
 	}
 
 	// --- canceling / failures ---
 	
 	@Test
-	public void testCancelingEmptyTopic() {
+	public void testCancelingEmptyTopic() throws Exception {
 		runCancelingOnEmptyInputTest();
 	}
 
 	@Test
-	public void testCancelingFullTopic() {
+	public void testCancelingFullTopic() throws Exception {
 		runCancelingOnFullInputTest();
 	}
 
 	@Test
-	public void testFailOnDeploy() {
+	public void testFailOnDeploy() throws Exception {
 		runFailOnDeployTest();
 	}
 
 	// --- source to partition mappings and exactly once ---
 	
 	@Test
-	public void testOneToOneSources() {
+	public void testOneToOneSources() throws Exception {
 		runOneToOneExactlyOnceTest();
 	}
 
 	@Test
-	public void testOneSourceMultiplePartitions() {
+	public void testOneSourceMultiplePartitions() throws Exception {
 		runOneSourceMultiplePartitionsExactlyOnceTest();
 	}
 
 	@Test
-	public void testMultipleSourcesOnePartition() {
+	public void testMultipleSourcesOnePartition() throws Exception {
 		runMultipleSourcesOnePartitionExactlyOnceTest();
 	}
 
 	// --- broker failure ---
 
 	@Test
-	public void testBrokerFailure() {
+	public void testBrokerFailure() throws Exception {
 		runBrokerFailureTest();
 	}
 
 	// --- special executions ---
 	
 	@Test
-	public void testBigRecordJob() {
+	public void testBigRecordJob() throws Exception {
 		runBigRecordTestTopology();
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java
index c4438f0..b8afe39 100644
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java
@@ -38,12 +38,14 @@ import org.apache.flink.streaming.connectors.kafka.internals.ZooKeeperStringSeri
 import org.apache.flink.streaming.connectors.kafka.testutils.SuccessException;
 import org.apache.flink.test.util.ForkableFlinkMiniCluster;
 
+import org.apache.flink.util.TestLogger;
 import org.apache.kafka.common.PartitionInfo;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import scala.concurrent.duration.FiniteDuration;
 
 import java.io.File;
 import java.io.IOException;
@@ -51,6 +53,7 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.Properties;
 import java.util.UUID;
+import java.util.concurrent.TimeUnit;
 
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
@@ -69,7 +72,7 @@ import static org.junit.Assert.fail;
  * as per commit <i>bc6b2b2d5f6424d5f377aa6c0871e82a956462ef</i></p>
  */
 @SuppressWarnings("serial")
-public abstract class KafkaTestBase {
+public abstract class KafkaTestBase extends TestLogger {
 
 	protected static final Logger LOG = LoggerFactory.getLogger(KafkaTestBase.class);
 	
@@ -91,6 +94,8 @@ public abstract class KafkaTestBase {
 	protected static ForkableFlinkMiniCluster flink;
 
 	protected static int flinkPort;
+
+	protected static FiniteDuration timeout = new FiniteDuration(10, TimeUnit.SECONDS);
 	
 	
 	
@@ -157,6 +162,7 @@ public abstract class KafkaTestBase {
 		standardProps.setProperty("group.id", "flink-tests");
 		standardProps.setProperty("auto.commit.enable", "false");
 		standardProps.setProperty("zookeeper.session.timeout.ms", "12000"); // 6 seconds is default. Seems to be too small for travis.
+		standardProps.setProperty("zookeeper.connection.timeout.ms", "20000");
 		standardProps.setProperty("auto.offset.reset", "earliest"); // read from the beginning.
 		standardProps.setProperty("fetch.message.max.bytes", "256"); // make a lot of fetches (MESSAGES MUST BE SMALLER!)
 		
@@ -168,13 +174,15 @@ public abstract class KafkaTestBase {
 		// start also a re-usable Flink mini cluster
 		
 		Configuration flinkConfig = new Configuration();
-		flinkConfig.setInteger(ConfigConstants.LOCAL_INSTANCE_MANAGER_NUMBER_TASK_MANAGER, 1);
+		flinkConfig.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, 1);
 		flinkConfig.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 8);
 		flinkConfig.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 16);
 		flinkConfig.setString(ConfigConstants.DEFAULT_EXECUTION_RETRY_DELAY_KEY, "0 s");
 
 		flink = new ForkableFlinkMiniCluster(flinkConfig, false, StreamingMode.STREAMING);
-		flinkPort = flink.getJobManagerRPCPort();
+		flink.start();
+
+		flinkPort = flink.getLeaderRPCPort();
 	}
 
 	@AfterClass

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/RemoteStreamEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/RemoteStreamEnvironment.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/RemoteStreamEnvironment.java
index 2f8938f..ae2c047 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/RemoteStreamEnvironment.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/RemoteStreamEnvironment.java
@@ -19,8 +19,6 @@ package org.apache.flink.streaming.api.environment;
 
 import java.io.File;
 import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.net.UnknownHostException;
 import java.util.ArrayList;
 import java.util.List;
 
@@ -29,6 +27,7 @@ import org.apache.flink.api.common.JobSubmissionResult;
 import org.apache.flink.client.program.Client;
 import org.apache.flink.client.program.JobWithJars;
 import org.apache.flink.client.program.ProgramInvocationException;
+import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.core.fs.Path;
 import org.apache.flink.runtime.jobgraph.JobGraph;
@@ -115,8 +114,11 @@ public class RemoteStreamEnvironment extends StreamExecutionEnvironment {
 		Configuration configuration = jobGraph.getJobConfiguration();
 		ClassLoader usercodeClassLoader = JobWithJars.buildUserCodeClassLoader(jarFiles, getClass().getClassLoader());
 
+		configuration.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, host);
+		configuration.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, port);
+
 		try {
-			Client client = new Client(new InetSocketAddress(host, port), configuration, usercodeClassLoader, -1);
+			Client client = new Client(configuration, usercodeClassLoader, -1);
 			client.setPrintStatusDuringExecution(getConfig().isSysoutLoggingEnabled());
 			
 			JobSubmissionResult result = client.run(jobGraph, true);
@@ -130,9 +132,6 @@ public class RemoteStreamEnvironment extends StreamExecutionEnvironment {
 		catch (ProgramInvocationException e) {
 			throw e;
 		}
-		catch (UnknownHostException e) {
-			throw new ProgramInvocationException(e.getMessage(), e);
-		}
 		catch (Exception e) {
 			String term = e.getMessage() == null ? "." : (": " + e.getMessage());
 			throw new ProgramInvocationException("The program execution failed" + term, e);

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/ClusterUtil.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/ClusterUtil.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/ClusterUtil.java
index c2ab424..4a7eec1 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/ClusterUtil.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/ClusterUtil.java
@@ -63,6 +63,8 @@ public class ClusterUtil {
 
 		try {
 			exec = new LocalFlinkMiniCluster(configuration, true);
+			exec.start();
+			
 			if (detached) {
 				exec.submitJobDetached(jobGraph);
 				return null;

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/timestamp/TimestampITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/timestamp/TimestampITCase.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/timestamp/TimestampITCase.java
index a41cb8c..523d1e3 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/timestamp/TimestampITCase.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/timestamp/TimestampITCase.java
@@ -59,12 +59,14 @@ public class TimestampITCase {
 	public static void startCluster() {
 		try {
 			Configuration config = new Configuration();
-			config.setInteger(ConfigConstants.LOCAL_INSTANCE_MANAGER_NUMBER_TASK_MANAGER, NUM_TASK_MANAGERS);
+			config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, NUM_TASK_MANAGERS);
 			config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, NUM_TASK_SLOTS);
 			config.setString(ConfigConstants.DEFAULT_EXECUTION_RETRY_DELAY_KEY, "0 ms");
 			config.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 12);
 
 			cluster = new ForkableFlinkMiniCluster(config, false);
+
+			cluster.start();
 		}
 		catch (Exception e) {
 			e.printStackTrace();
@@ -100,7 +102,7 @@ public class TimestampITCase {
 		long initialTime = 0L;
 
 		StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment(
-				"localhost", cluster.getJobManagerRPCPort());
+				"localhost", cluster.getLeaderRPCPort());
 		env.setParallelism(PARALLELISM);
 		env.getConfig().disableSysoutLogging();
 		env.getConfig().enableTimestamps();
@@ -139,7 +141,7 @@ public class TimestampITCase {
 
 
 		StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment(
-				"localhost", cluster.getJobManagerRPCPort());
+				"localhost", cluster.getLeaderRPCPort());
 		env.setParallelism(PARALLELISM);
 		env.getConfig().disableSysoutLogging();
 		env.getConfig().enableTimestamps();
@@ -167,7 +169,7 @@ public class TimestampITCase {
 
 
 		StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment(
-				"localhost", cluster.getJobManagerRPCPort());
+				"localhost", cluster.getLeaderRPCPort());
 		env.setParallelism(PARALLELISM);
 		env.getConfig().disableSysoutLogging();
 		Assert.assertEquals("Timestamps are not disabled by default.", false, env.getConfig().areTimestampsEnabled());
@@ -193,7 +195,7 @@ public class TimestampITCase {
 	 */
 	@Test(expected = ProgramInvocationException.class)
 	public void testEventTimeSourceEmitWithoutTimestamp() throws Exception {
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", cluster.getJobManagerRPCPort());
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", cluster.getLeaderRPCPort());
 		env.setParallelism(PARALLELISM);
 		env.getConfig().disableSysoutLogging();
 
@@ -212,7 +214,7 @@ public class TimestampITCase {
 	 */
 	@Test(expected = ProgramInvocationException.class)
 	public void testSourceEmitWithTimestamp() throws Exception {
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", cluster.getJobManagerRPCPort());
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", cluster.getLeaderRPCPort());
 		env.setParallelism(PARALLELISM);
 		env.getConfig().disableSysoutLogging();
 
@@ -231,7 +233,7 @@ public class TimestampITCase {
 	 */
 	@Test(expected = ProgramInvocationException.class)
 	public void testSourceEmitWatermark() throws Exception {
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", cluster.getJobManagerRPCPort());
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", cluster.getLeaderRPCPort());
 		env.setParallelism(PARALLELISM);
 		env.getConfig().disableSysoutLogging();
 

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/StreamingMultipleProgramsTestBase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/StreamingMultipleProgramsTestBase.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/StreamingMultipleProgramsTestBase.java
index 09db1f4..d251a5d 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/StreamingMultipleProgramsTestBase.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/StreamingMultipleProgramsTestBase.java
@@ -76,7 +76,13 @@ public class StreamingMultipleProgramsTestBase extends TestBaseUtils {
 
 	@BeforeClass
 	public static void setup() throws Exception{
-		cluster = TestBaseUtils.startCluster(1, DEFAULT_PARALLELISM, StreamingMode.STREAMING, false, true);
+		cluster = TestBaseUtils.startCluster(
+			1,
+			DEFAULT_PARALLELISM,
+			StreamingMode.STREAMING,
+			false,
+			false,
+			true);
 	}
 
 	@AfterClass

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TestStreamEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TestStreamEnvironment.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TestStreamEnvironment.java
index 8e9e56a..91082d8 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TestStreamEnvironment.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TestStreamEnvironment.java
@@ -70,6 +70,7 @@ public class TestStreamEnvironment extends StreamExecutionEnvironment {
 			configuration.setLong(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, memorySize);
 
 			executor = new ForkableFlinkMiniCluster(configuration);
+			executor.start();
 		}
 		try {
 			sync = true;

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-test-utils/pom.xml
----------------------------------------------------------------------
diff --git a/flink-test-utils/pom.xml b/flink-test-utils/pom.xml
index 1eb5ff2..7c1ac3b 100644
--- a/flink-test-utils/pom.xml
+++ b/flink-test-utils/pom.xml
@@ -89,6 +89,12 @@ under the License.
 			<artifactId>scalatest_${scala.binary.version}</artifactId>
 			<scope>compile</scope>
 		</dependency>
+
+		<dependency>
+			<groupId>org.apache.curator</groupId>
+			<artifactId>curator-test</artifactId>
+			<scope>compile</scope>
+		</dependency>
 	</dependencies>
 
 	<build>

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-test-utils/src/main/java/org/apache/flink/test/util/AbstractTestBase.java
----------------------------------------------------------------------
diff --git a/flink-test-utils/src/main/java/org/apache/flink/test/util/AbstractTestBase.java b/flink-test-utils/src/main/java/org/apache/flink/test/util/AbstractTestBase.java
index a63f6ac..1e85c71 100644
--- a/flink-test-utils/src/main/java/org/apache/flink/test/util/AbstractTestBase.java
+++ b/flink-test-utils/src/main/java/org/apache/flink/test/util/AbstractTestBase.java
@@ -60,7 +60,13 @@ public abstract class AbstractTestBase extends TestBaseUtils {
 	// --------------------------------------------------------------------------------------------
 
 	public void startCluster() throws Exception{
-		this.executor = startCluster(numTaskManagers, taskManagerNumSlots, StreamingMode.BATCH_ONLY, false, true);
+		this.executor = startCluster(
+			numTaskManagers,
+			taskManagerNumSlots,
+			StreamingMode.BATCH_ONLY,
+			false,
+			false,
+			true);
 	}
 
 	public void stopCluster() throws Exception {

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-test-utils/src/main/java/org/apache/flink/test/util/MultipleProgramsTestBase.java
----------------------------------------------------------------------
diff --git a/flink-test-utils/src/main/java/org/apache/flink/test/util/MultipleProgramsTestBase.java b/flink-test-utils/src/main/java/org/apache/flink/test/util/MultipleProgramsTestBase.java
index 97d5db0..2ea61d7 100644
--- a/flink-test-utils/src/main/java/org/apache/flink/test/util/MultipleProgramsTestBase.java
+++ b/flink-test-utils/src/main/java/org/apache/flink/test/util/MultipleProgramsTestBase.java
@@ -101,7 +101,13 @@ public class MultipleProgramsTestBase extends TestBaseUtils {
 
 	@BeforeClass
 	public static void setup() throws Exception{
-		cluster = TestBaseUtils.startCluster(1, DEFAULT_PARALLELISM, StreamingMode.BATCH_ONLY, startWebServer, true);
+		cluster = TestBaseUtils.startCluster(
+			1,
+			DEFAULT_PARALLELISM,
+			StreamingMode.BATCH_ONLY,
+			startWebServer,
+			false,
+			true);
 	}
 
 	@AfterClass

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-test-utils/src/main/java/org/apache/flink/test/util/TestBaseUtils.java
----------------------------------------------------------------------
diff --git a/flink-test-utils/src/main/java/org/apache/flink/test/util/TestBaseUtils.java b/flink-test-utils/src/main/java/org/apache/flink/test/util/TestBaseUtils.java
index 87fab25..a2c7b93 100644
--- a/flink-test-utils/src/main/java/org/apache/flink/test/util/TestBaseUtils.java
+++ b/flink-test-utils/src/main/java/org/apache/flink/test/util/TestBaseUtils.java
@@ -103,34 +103,55 @@ public class TestBaseUtils extends TestLogger {
 	}
 	
 	
-	public static ForkableFlinkMiniCluster startCluster(int numTaskManagers,
-															int taskManagerNumSlots,
-															StreamingMode mode,
-															boolean startWebserver,
-															boolean singleActorSystem) throws Exception {
+	public static ForkableFlinkMiniCluster startCluster(
+		int numTaskManagers,
+		int taskManagerNumSlots,
+		StreamingMode mode,
+		boolean startWebserver,
+		boolean startZooKeeper,
+		boolean singleActorSystem) throws Exception {
 		
+		Configuration config = new Configuration();
+
+		config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, numTaskManagers);
+		config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, taskManagerNumSlots);
+		
+		config.setBoolean(ConfigConstants.LOCAL_START_WEBSERVER, startWebserver);
+
+		if(startZooKeeper) {
+			config.setInteger(ConfigConstants.LOCAL_NUMBER_JOB_MANAGER, 3);
+			config.setString(ConfigConstants.RECOVERY_MODE, "zookeeper");
+		}
+
+		return startCluster(config, mode, singleActorSystem);
+	}
+
+	public static ForkableFlinkMiniCluster startCluster(
+		Configuration config,
+		StreamingMode mode,
+		boolean singleActorSystem) throws Exception {
+
 		logDir = File.createTempFile("TestBaseUtils-logdir", null);
 		Assert.assertTrue("Unable to delete temp file", logDir.delete());
 		Assert.assertTrue("Unable to create temp directory", logDir.mkdir());
-	
-		Configuration config = new Configuration();
-		config.setBoolean(ConfigConstants.FILESYSTEM_DEFAULT_OVERWRITE_KEY, true);
 
-		config.setInteger(ConfigConstants.LOCAL_INSTANCE_MANAGER_NUMBER_TASK_MANAGER, numTaskManagers);
-		
 		config.setLong(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, TASK_MANAGER_MEMORY_SIZE);
-		config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, taskManagerNumSlots);
-		
+		config.setBoolean(ConfigConstants.FILESYSTEM_DEFAULT_OVERWRITE_KEY, true);
+
 		config.setString(ConfigConstants.AKKA_ASK_TIMEOUT, DEFAULT_AKKA_ASK_TIMEOUT + "s");
 		config.setString(ConfigConstants.AKKA_STARTUP_TIMEOUT, DEFAULT_AKKA_STARTUP_TIMEOUT);
-		
-		config.setBoolean(ConfigConstants.LOCAL_INSTANCE_MANAGER_START_WEBSERVER, startWebserver);
+
 		config.setInteger(ConfigConstants.JOB_MANAGER_WEB_PORT_KEY, 8081);
 		config.setString(ConfigConstants.JOB_MANAGER_WEB_LOG_PATH_KEY, logDir.toString());
-		
-		return new ForkableFlinkMiniCluster(config, singleActorSystem, mode);
+
+		ForkableFlinkMiniCluster cluster =  new ForkableFlinkMiniCluster(config, singleActorSystem, mode);
+
+		cluster.start();
+
+		return cluster;
 	}
 
+
 	public static void stopCluster(ForkableFlinkMiniCluster executor, FiniteDuration timeout) throws Exception {
 		if (logDir != null) {
 			FileUtils.deleteDirectory(logDir);

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-test-utils/src/main/java/org/apache/flink/test/util/TestEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-test-utils/src/main/java/org/apache/flink/test/util/TestEnvironment.java b/flink-test-utils/src/main/java/org/apache/flink/test/util/TestEnvironment.java
index 1812422..d0c8e3d 100644
--- a/flink-test-utils/src/main/java/org/apache/flink/test/util/TestEnvironment.java
+++ b/flink-test-utils/src/main/java/org/apache/flink/test/util/TestEnvironment.java
@@ -74,7 +74,7 @@ public class TestEnvironment extends ExecutionEnvironment {
 	private OptimizedPlan compileProgram(String jobName) {
 		Plan p = createProgramPlan(jobName);
 
-		Optimizer pc = new Optimizer(new DataStatistics(), this.executor.getConfiguration());
+		Optimizer pc = new Optimizer(new DataStatistics(), this.executor.configuration());
 		return pc.compile(p);
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-test-utils/src/main/scala/org/apache/flink/test/util/FlinkTestBase.scala
----------------------------------------------------------------------
diff --git a/flink-test-utils/src/main/scala/org/apache/flink/test/util/FlinkTestBase.scala b/flink-test-utils/src/main/scala/org/apache/flink/test/util/FlinkTestBase.scala
index bac1ede..612fdc5 100644
--- a/flink-test-utils/src/main/scala/org/apache/flink/test/util/FlinkTestBase.scala
+++ b/flink-test-utils/src/main/scala/org/apache/flink/test/util/FlinkTestBase.scala
@@ -54,7 +54,14 @@ trait FlinkTestBase extends BeforeAndAfter {
   val parallelism = 4
 
   before {
-    val cl = TestBaseUtils.startCluster(1, parallelism, StreamingMode.BATCH_ONLY, false, true)
+    val cl = TestBaseUtils.startCluster(
+      1,
+      parallelism,
+      StreamingMode.BATCH_ONLY,
+      false,
+      false,
+      true)
+
     val clusterEnvironment = new TestEnvironment(cl, parallelism)
     clusterEnvironment.setAsContext()
 

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-test-utils/src/main/scala/org/apache/flink/test/util/ForkableFlinkMiniCluster.scala
----------------------------------------------------------------------
diff --git a/flink-test-utils/src/main/scala/org/apache/flink/test/util/ForkableFlinkMiniCluster.scala b/flink-test-utils/src/main/scala/org/apache/flink/test/util/ForkableFlinkMiniCluster.scala
index faf8424..69ca6b7 100644
--- a/flink-test-utils/src/main/scala/org/apache/flink/test/util/ForkableFlinkMiniCluster.scala
+++ b/flink-test-utils/src/main/scala/org/apache/flink/test/util/ForkableFlinkMiniCluster.scala
@@ -18,18 +18,25 @@
 
 package org.apache.flink.test.util
 
+import java.util.concurrent.TimeoutException
+
+import akka.pattern.ask
 import akka.actor.{Props, ActorRef, ActorSystem}
 import akka.pattern.Patterns._
+import org.apache.curator.test.TestingCluster
 import org.apache.flink.configuration.{ConfigConstants, Configuration}
 import org.apache.flink.runtime.StreamingMode
-import org.apache.flink.runtime.jobmanager.JobManager
+import org.apache.flink.runtime.akka.AkkaUtils
+import org.apache.flink.runtime.jobmanager.{RecoveryMode, JobManager}
 import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster
 import org.apache.flink.runtime.taskmanager.TaskManager
-import org.apache.flink.runtime.testingUtils.{TestingUtils, TestingJobManager,
-TestingMemoryArchivist, TestingTaskManager}
+import org.apache.flink.runtime.testingUtils.TestingTaskManagerMessages
+.NotifyWhenRegisteredAtJobManager
+import org.apache.flink.runtime.testingUtils.{TestingUtils, TestingTaskManager,
+TestingJobManager, TestingMemoryArchivist}
 import org.apache.flink.runtime.webmonitor.WebMonitor
 
-import scala.concurrent.Await
+import scala.concurrent.{Future, Promise, Await}
 
 /**
  * A forkable mini cluster is a special case of the mini cluster, used for parallel test execution
@@ -45,7 +52,6 @@ class ForkableFlinkMiniCluster(
     singleActorSystem: Boolean,
     streamingMode: StreamingMode)
   extends LocalFlinkMiniCluster(userConfiguration, singleActorSystem, streamingMode) {
-  
 
   def this(userConfiguration: Configuration, singleActorSystem: Boolean) 
        = this(userConfiguration, singleActorSystem, StreamingMode.BATCH_ONLY)
@@ -53,6 +59,8 @@ class ForkableFlinkMiniCluster(
   def this(userConfiguration: Configuration) = this(userConfiguration, true)
   
   // --------------------------------------------------------------------------
+
+  var zookeeperCluster: Option[TestingCluster] = None
   
   override def generateConfiguration(userConfiguration: Configuration): Configuration = {
     val forNumberString = System.getProperty("forkNumber")
@@ -79,7 +87,19 @@ class ForkableFlinkMiniCluster(
     super.generateConfiguration(config)
   }
 
-  override def startJobManager(actorSystem: ActorSystem): (ActorRef, Option[WebMonitor]) = {
+  override def startJobManager(index: Int, actorSystem: ActorSystem): ActorRef = {
+    val config = configuration.clone()
+
+    val jobManagerName = getJobManagerName(index)
+    val archiveName = getArchiveName(index)
+
+    val jobManagerPort = config.getInteger(
+      ConfigConstants.JOB_MANAGER_IPC_PORT_KEY,
+      ConfigConstants.DEFAULT_JOB_MANAGER_IPC_PORT)
+
+    if(jobManagerPort > 0) {
+      config.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, jobManagerPort + index)
+    }
 
     val (executionContext,
       instanceManager,
@@ -89,13 +109,13 @@ class ForkableFlinkMiniCluster(
       executionRetries,
       delayBetweenRetries,
       timeout,
-      archiveCount) = JobManager.createJobManagerComponents(configuration)
+      archiveCount,
+      leaderElectionService) = JobManager.createJobManagerComponents(config)
+      val testArchiveProps = Props(
+        new TestingMemoryArchivist(archiveCount))
 
-    val testArchiveProps = Props(
-      new TestingMemoryArchivist(archiveCount))
+    val archiver = actorSystem.actorOf(testArchiveProps, archiveName)
 
-    val archiver = actorSystem.actorOf(testArchiveProps, JobManager.ARCHIVE_NAME)
-    
     val jobManagerProps = Props(
       new TestingJobManager(
         configuration,
@@ -107,23 +127,24 @@ class ForkableFlinkMiniCluster(
         executionRetries,
         delayBetweenRetries,
         timeout,
-        streamingMode))
-
-    val jobManager = actorSystem.actorOf(jobManagerProps, JobManager.JOB_MANAGER_NAME)
+        streamingMode,
+        leaderElectionService))
 
-    val webMonitorOption = startWebServer(configuration, jobManager, archiver)
+    val jobManager = actorSystem.actorOf(jobManagerProps, jobManagerName)
 
-    (jobManager, webMonitorOption)
+    jobManager
   }
 
   override def startTaskManager(index: Int, system: ActorSystem): ActorRef = {
     val config = configuration.clone()
 
-    val rpcPort = config.getInteger(ConfigConstants.TASK_MANAGER_IPC_PORT_KEY,
-                                    ConfigConstants.DEFAULT_TASK_MANAGER_IPC_PORT)
+    val rpcPort = config.getInteger(
+      ConfigConstants.TASK_MANAGER_IPC_PORT_KEY,
+      ConfigConstants.DEFAULT_TASK_MANAGER_IPC_PORT)
 
-    val dataPort = config.getInteger(ConfigConstants.TASK_MANAGER_DATA_PORT_KEY,
-                                     ConfigConstants.DEFAULT_TASK_MANAGER_DATA_PORT)
+    val dataPort = config.getInteger(
+      ConfigConstants.TASK_MANAGER_DATA_PORT_KEY,
+      ConfigConstants.DEFAULT_TASK_MANAGER_DATA_PORT)
 
     if (rpcPort > 0) {
       config.setInteger(ConfigConstants.TASK_MANAGER_IPC_PORT_KEY, rpcPort + index)
@@ -134,47 +155,130 @@ class ForkableFlinkMiniCluster(
 
     val localExecution = numTaskManagers == 1
 
-    val jobManagerAkkaUrl: Option[String] = if (singleActorSystem) {
-      Some(jobManagerActor.path.toString)
-    } else {
-      None
-    }
+    TaskManager.startTaskManagerComponentsAndActor(
+      config,
+      system,
+      hostname,
+      Some(TaskManager.TASK_MANAGER_NAME + index),
+      Some(createLeaderRetrievalService),
+      localExecution,
+      streamingMode,
+      classOf[TestingTaskManager])
+  }
+
+  def restartLeadingJobManager(): Unit = {
+    this.synchronized {
+      (jobManagerActorSystems, jobManagerActors) match {
+        case (Some(jmActorSystems), Some(jmActors)) =>
+          val leader = getLeaderGateway(AkkaUtils.getTimeout(configuration))
+          val index = getLeaderIndex(AkkaUtils.getTimeout(configuration))
+
+          clearLeader()
+
+          val stopped = gracefulStop(leader.actor(), TestingUtils.TESTING_DURATION)
+          Await.result(stopped, TestingUtils.TESTING_DURATION)
+
+          if(!singleActorSystem) {
+            jmActorSystems(index).shutdown()
+            jmActorSystems(index).awaitTermination()
+          }
+
+          val newJobManagerActorSystem = if(!singleActorSystem) {
+            startJobManagerActorSystem(index)
+          } else {
+            jmActorSystems(0)
+          }
+
+          val newJobManagerActor = startJobManager(index, newJobManagerActorSystem)
+
+          jobManagerActors = Some(jmActors.patch(index, Seq(newJobManagerActor), 1))
+          jobManagerActorSystems = Some(jmActorSystems.patch(
+            index,
+            Seq(newJobManagerActorSystem),
+            1))
+
+          val lrs = createLeaderRetrievalService
+
+          leaderRetrievalService = Some(lrs)
+          lrs.start(this)
 
-    TaskManager.startTaskManagerComponentsAndActor(config, system, hostname,
-        Some(TaskManager.TASK_MANAGER_NAME + index), jobManagerAkkaUrl, localExecution,
-      streamingMode, classOf[TestingTaskManager])
+        case _ => throw new Exception("The JobManager of the ForkableFlinkMiniCluster have not " +
+          "been started properly.")
+      }
+    }
   }
 
-  def restartJobManager(): Unit = {
-    val stopped = gracefulStop(jobManagerActor, TestingUtils.TESTING_DURATION)
-    Await.result(stopped, TestingUtils.TESTING_DURATION)
+  def restartTaskManager(index: Int): Unit = {
+    (taskManagerActorSystems, taskManagerActors) match {
+      case (Some(tmActorSystems), Some(tmActors)) =>
+        val stopped = gracefulStop(tmActors(index), TestingUtils.TESTING_DURATION)
+        Await.result(stopped, TestingUtils.TESTING_DURATION)
+
+        if(!singleActorSystem) {
+          tmActorSystems(index).shutdown()
+          tmActorSystems(index).awaitTermination()
+        }
+
+        val taskManagerActorSystem  = if(!singleActorSystem) {
+          startTaskManagerActorSystem(index)
+        } else {
+          tmActorSystems(0)
+        }
+
+        val taskManagerActor = startTaskManager(index, taskManagerActorSystem)
+
+        taskManagerActors = Some(tmActors.patch(index, Seq(taskManagerActor), 1))
+        taskManagerActorSystems = Some(tmActorSystems.patch(index, Seq(taskManagerActorSystem), 1))
 
-    webMonitor foreach {
-      _.stop()
+      case _ => throw new Exception("The TaskManager of the ForkableFlinkMiniCluster have not " +
+        "been started properly.")
     }
+  }
+
+  override def start(): Unit = {
+    val zookeeperURL = configuration.getString(ConfigConstants.ZOOKEEPER_QUORUM_KEY, "")
+
+    zookeeperCluster = if(recoveryMode == RecoveryMode.ZOOKEEPER && zookeeperURL.equals("")) {
+      LOG.info("Starting ZooKeeper cluster.")
 
-    jobManagerActorSystem.shutdown()
-    jobManagerActorSystem.awaitTermination()
+      val testingCluster = new TestingCluster(numJobManagers)
 
-    jobManagerActorSystem = startJobManagerActorSystem()
-    val (newJobManagerActor, newWebMonitor) = startJobManager(jobManagerActorSystem)
+      configuration.setString(ConfigConstants.ZOOKEEPER_QUORUM_KEY, testingCluster.getConnectString)
 
-    jobManagerActor = newJobManagerActor
-    webMonitor = newWebMonitor
+      testingCluster.start()
+
+      Some(testingCluster)
+    } else {
+      None
+    }
+
+    super.start()
   }
 
-  def restartTaskManager(index: Int): Unit = {
-    val stopped = gracefulStop(taskManagerActors(index), TestingUtils.TESTING_DURATION)
-    Await.result(stopped, TestingUtils.TESTING_DURATION)
+  override def stop(): Unit = {
+    super.stop()
 
-    taskManagerActorSystems(index).shutdown()
-    taskManagerActorSystems(index).awaitTermination()
+    zookeeperCluster.foreach{
+      LOG.info("Stopping ZooKeeper cluster.")
+      _.close()
+    }
+  }
 
-    val taskManagerActorSystem  = startTaskManagerActorSystem(index)
-    val taskManagerActor = startTaskManager(index, taskManagerActorSystem)
+  def waitForTaskManagersToBeRegisteredAtJobManager(jobManager: ActorRef): Unit = {
+    val futures = taskManagerActors.map {
+      _.map {
+        tm => (tm ? NotifyWhenRegisteredAtJobManager(jobManager))(timeout)
+      }
+    }.getOrElse(Seq())
+
+    try {
+      Await.ready(Future.sequence(futures), timeout)
+    } catch {
+      case t: TimeoutException =>
+        throw new Exception("Timeout while waiting for TaskManagers to register at " +
+          s"${jobManager.path}")
+    }
 
-    taskManagerActors = taskManagerActors.patch(index, Seq(taskManagerActor), 1)
-    taskManagerActorSystems = taskManagerActorSystems.patch(index, Seq(taskManagerActorSystem), 1)
   }
 }
 
@@ -182,14 +286,21 @@ object ForkableFlinkMiniCluster {
 
   import org.apache.flink.runtime.testingUtils.TestingUtils.DEFAULT_AKKA_ASK_TIMEOUT
 
-  def startCluster(numSlots: Int,
-                   numTaskManagers: Int,
-                   timeout: String = DEFAULT_AKKA_ASK_TIMEOUT): ForkableFlinkMiniCluster = {
+  def startCluster(
+      numSlots: Int,
+      numTaskManagers: Int,
+      timeout: String = DEFAULT_AKKA_ASK_TIMEOUT)
+    : ForkableFlinkMiniCluster = {
 
     val config = new Configuration()
     config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, numSlots)
-    config.setInteger(ConfigConstants.LOCAL_INSTANCE_MANAGER_NUMBER_TASK_MANAGER, numTaskManagers)
+    config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, numTaskManagers)
     config.setString(ConfigConstants.AKKA_ASK_TIMEOUT, timeout)
-    new ForkableFlinkMiniCluster(config)
+
+    val cluster = new ForkableFlinkMiniCluster(config)
+
+    cluster.start()
+
+    cluster
   }
 }


[09/10] flink git commit: [FLINK-2291] [runtime] Add ZooKeeper support to elect a leader from a set of JobManager. The leader will then be retrieved from ZooKeeper by the TaskManagers.

Posted by tr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/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 355da2d..dc242b2 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
@@ -452,6 +452,31 @@ public final class ConfigConstants {
 	
 	public static final String FLINK_JVM_OPTIONS = "env.java.opts";
 
+  	// --------------------------- Recovery -----------------------------------
+
+	/** Defines recovery mode used for the cluster execution ("standalone", "zookeeper") */
+	public static final String RECOVERY_MODE = "recovery.mode";
+
+  	// --------------------------- ZooKeeper ----------------------------------
+
+	/** ZooKeeper servers. */
+	public static final String ZOOKEEPER_QUORUM_KEY = "ha.zookeeper.quorum";
+
+	/** ZooKeeper root path. */
+	public static final String ZOOKEEPER_DIR_KEY = "ha.zookeeper.dir";
+
+	public static final String ZOOKEEPER_LATCH_PATH = "ha.zookeeper.dir.latch";
+
+	public static final String ZOOKEEPER_LEADER_PATH = "ha.zookeeper.dir.leader";
+
+	public static final String ZOOKEEPER_SESSION_TIMEOUT = "ha.zookeeper.client.session-timeout";
+
+	public static final String ZOOKEEPER_CONNECTION_TIMEOUT = "ha.zookeeper.client.connection-timeout";
+
+	public static final String ZOOKEEPER_RETRY_WAIT = "ha.zookeeper.client.retry-wait";
+
+	public static final String ZOOKEEPER_MAX_RETRY_ATTEMPTS = "ha.zookeeper.client.max-retry-attempts";
+
 	// ------------------------------------------------------------------------
 	//                            Default Values
 	// ------------------------------------------------------------------------
@@ -694,33 +719,23 @@ public final class ConfigConstants {
 	/**
 	 * Sets the number of local task managers
 	 */
-	public static final String LOCAL_INSTANCE_MANAGER_NUMBER_TASK_MANAGER = "localinstancemanager.numtaskmanager";
-
-	public static final String LOCAL_INSTANCE_MANAGER_START_WEBSERVER = "localinstancemanager.start-webserver";
-
-	// --------------------------- ZooKeeper ----------------------------------
-
-	/** ZooKeeper servers. */
-	public static final String ZOOKEEPER_QUORUM_KEY = "ha.zookeeper.quorum";
-
-	/** ZooKeeper root path. */
-	public static final String ZOOKEEPER_DIR_KEY = "ha.zookeeper.dir";
+	public static final String LOCAL_NUMBER_TASK_MANAGER = "local.number-taskmanager";
 
-	public static final String ZOOKEEPER_LATCH_PATH = "ha.zookeeper.dir.latch";
+	public static final int DEFAULT_LOCAL_NUMBER_TASK_MANAGER = 1;
 
-	public static final String ZOOKEEPER_LEADER_PATH = "ha.zookeeper.dir.leader";
+	public static final String LOCAL_NUMBER_JOB_MANAGER = "local.number-jobmanager";
 
-	public static final String ZOOKEEPER_SESSION_TIMEOUT = "ha.zookeeper.client.session-timeout";
+	public static final int DEFAULT_LOCAL_NUMBER_JOB_MANAGER = 1;
 
-	public static final String ZOOKEEPER_CONNECTION_TIMEOUT = "ha.zookeeper.client.connection-timeout";
+	public static final String LOCAL_START_WEBSERVER = "local.start-webserver";
 
-	public static final String ZOOKEEPER_RETRY_WAIT = "ha.zookeeper.client.retry-wait";
+  	// --------------------------- Recovery ---------------------------------
 
-	public static final String ZOOKEEPER_MAX_RETRY_ATTEMPTS = "ha.zookeeper.client.max-retry-attempts";
+	public static String DEFAULT_RECOVERY_MODE = "standalone";
 
-	// - Defaults -------------------------------------------------------------
+	// --------------------------- ZooKeeper ----------------------------------
 
-	public static final String DEFAULT_ZOOKEEPER_ZNODE_ROOT = "/flink";
+	public static final String DEFAULT_ZOOKEEPER_DIR_KEY = "/flink";
 
 	public static final String DEFAULT_ZOOKEEPER_LATCH_PATH = "/leaderlatch";
 

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-core/src/test/java/org/apache/flink/types/parser/ByteParserTest.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/types/parser/ByteParserTest.java b/flink-core/src/test/java/org/apache/flink/types/parser/ByteParserTest.java
index 3676e62..dac5144 100644
--- a/flink-core/src/test/java/org/apache/flink/types/parser/ByteParserTest.java
+++ b/flink-core/src/test/java/org/apache/flink/types/parser/ByteParserTest.java
@@ -22,10 +22,6 @@ package org.apache.flink.types.parser;
 import org.apache.flink.types.parser.ByteParser;
 import org.apache.flink.types.parser.FieldParser;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
 
 public class ByteParserTest extends ParserTestBase<Byte> {
 

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-core/src/test/java/org/apache/flink/util/AbstractIDTest.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/util/AbstractIDTest.java b/flink-core/src/test/java/org/apache/flink/util/AbstractIDTest.java
index 8de3b94..8134a68 100644
--- a/flink-core/src/test/java/org/apache/flink/util/AbstractIDTest.java
+++ b/flink-core/src/test/java/org/apache/flink/util/AbstractIDTest.java
@@ -25,8 +25,6 @@ import static org.junit.Assert.fail;
 import org.apache.flink.core.testutils.CommonTestUtils;
 import org.junit.Test;
 
-import java.nio.ByteBuffer;
-
 /**
  * This class contains tests for the {@link org.apache.flink.util.AbstractID} class.
  */

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-dist/src/main/flink-bin/LICENSE
----------------------------------------------------------------------
diff --git a/flink-dist/src/main/flink-bin/LICENSE b/flink-dist/src/main/flink-bin/LICENSE
index e79ff71..9df6da5 100644
--- a/flink-dist/src/main/flink-bin/LICENSE
+++ b/flink-dist/src/main/flink-bin/LICENSE
@@ -278,6 +278,10 @@ under the Apache License (v 2.0):
  - Jansi (org.fusesource.jansi:jansi:1.4 - https://github.com/fusesource/jansi)
  - Apache Camel Core (org.apache.camel:camel-core:2.10.3 - http://camel.apache.org/camel-core.html)
  - Apache Commons Math (org.apache.commons:commons-math3:3.5 - http://commons.apache.org/proper/commons-math/index.html)
+ - Apache ZooKeeper (org.apache.zookeeper:zookeeper:3.4.6 - https://zookeeper.apache.org/)
+ - Apache Curator (org.apache.curator:curator-recipes:2.8.0 - http://curator.apache.org/)
+ - Apache Curator (org.apache.curator:curator-framework:2.8.0 - http://curator.apache.org/)
+ - Apache Curator (org.apache.curator:curator-client:2.8.0 - http://curator.apache.org/)
 
 
 -----------------------------------------------------------------------
@@ -356,6 +360,7 @@ BSD-style licenses:
  - ASM (org.ow2.asm:asm:5.0.4 - http://asm.ow2.org/) - Copyright (c) 2000-2011 INRIA, France Telecom
  - Grizzled SLF4J (org.clapper:grizzled-slf4j_2.10:1.0.2 - http://software.clapper.org/grizzled-slf4j/) - Copyright (c) 2010 Brian M. Clapper
  - ParaNamer (com.thoughtworks.paranamer:paranamer:2.3 - https://github.com/paul-hammant/paranamer) - Copyright (c) 2006 Paul Hammant & ThoughtWorks Inc
+ - JLine (jline:jline:0.9.94 - http://jline.sourceforge.net/) - Copyright (c) 2002-2006, Marc Prud'hommeaux <mw...@cornell.edu>
 
 
 (Below is the 3-clause BSD license)

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-dist/src/main/flink-bin/NOTICE
----------------------------------------------------------------------
diff --git a/flink-dist/src/main/flink-bin/NOTICE b/flink-dist/src/main/flink-bin/NOTICE
index 7b0fe72..19af75f 100644
--- a/flink-dist/src/main/flink-bin/NOTICE
+++ b/flink-dist/src/main/flink-bin/NOTICE
@@ -52,6 +52,28 @@ available from http://www.digip.org/jansson/.
 
 
 -----------------------------------------------------------------------
+                           Apache ZooKeeper
+-----------------------------------------------------------------------
+
+Apache ZooKeeper
+Copyright 2009-2014 The Apache Software Foundation
+
+This product includes software developed at
+The Apache Software Foundation (http://www.apache.org/).
+
+
+-----------------------------------------------------------------------
+                           Apache Curator
+-----------------------------------------------------------------------
+
+Apache Curator
+Copyright 2013-2014 The Apache Software Foundation
+
+This product includes software developed at
+The Apache Software Foundation (http://www.apache.org/).
+
+
+-----------------------------------------------------------------------
                            Apache Sling
 -----------------------------------------------------------------------
 

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-dist/src/main/flink-bin/bin/config.sh
----------------------------------------------------------------------
diff --git a/flink-dist/src/main/flink-bin/bin/config.sh b/flink-dist/src/main/flink-bin/bin/config.sh
index 1d31493..04afa02 100755
--- a/flink-dist/src/main/flink-bin/bin/config.sh
+++ b/flink-dist/src/main/flink-bin/bin/config.sh
@@ -94,7 +94,7 @@ KEY_ENV_LOG_MAX="env.log.max"
 KEY_ENV_JAVA_HOME="env.java.home"
 KEY_ENV_JAVA_OPTS="env.java.opts"
 KEY_ENV_SSH_OPTS="env.ssh.opts"
-KEY_ZK_QUORUM="ha.zookeeper.quorum"
+KEY_RECOVERY_MODE="recovery.mode"
 KEY_ZK_HEAP_MB="zookeeper.heap.mb"
 
 ########################################################################################################################
@@ -205,8 +205,8 @@ if [ -z "${ZK_HEAP}" ]; then
     ZK_HEAP=$(readFromConfig ${KEY_ZK_HEAP_MB} 0 "${YAML_CONF}")
 fi
 
-if [ -z "${ZK_QUORUM}" ]; then
-    ZK_QUORUM=$(readFromConfig ${KEY_ZK_QUORUM} "" "${YAML_CONF}")
+if [ -z "${RECOVERY_MODE}" ]; then
+    RECOVERY_MODE=$(readFromConfig ${KEY_RECOVERY_MODE} "standalone" "${YAML_CONF}")
 fi
 
 # Arguments for the JVM. Used for job and task manager JVMs.
@@ -268,13 +268,18 @@ readMasters() {
     fi
 
     MASTERS=()
+    WEBUIPORTS=()
 
     GOON=true
     while $GOON; do
         read line || GOON=false
-        HOST=$( extractHostName $line)
-        if [ -n "$HOST" ]; then
+        HOSTWEBUIPORT=$( extractHostName $line)
+
+        if [ -n "$HOSTWEBUIPORT" ]; then
+            HOST=$(echo $HOSTWEBUIPORT | cut -f1 -d:)
+            WEBUIPORT=$(echo $HOSTWEBUIPORT | cut -f2 -d:)
             MASTERS+=(${HOST})
+            WEBUIPORTS+=(${WEBUIPORT})
         fi
     done < "$MASTERS_FILE"
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-dist/src/main/flink-bin/bin/jobmanager.sh
----------------------------------------------------------------------
diff --git a/flink-dist/src/main/flink-bin/bin/jobmanager.sh b/flink-dist/src/main/flink-bin/bin/jobmanager.sh
index 450d36b..c89f53a 100755
--- a/flink-dist/src/main/flink-bin/bin/jobmanager.sh
+++ b/flink-dist/src/main/flink-bin/bin/jobmanager.sh
@@ -18,12 +18,13 @@
 ################################################################################
 
 # Start/stop a Flink JobManager.
-USAGE="Usage: jobmanager.sh (start (local|cluster) [batch|streaming] [host])|stop|stop-all)"
+USAGE="Usage: jobmanager.sh (start (local|cluster) [batch|streaming] [host] [webui-port])|stop|stop-all)"
 
 STARTSTOP=$1
 EXECUTIONMODE=$2
 STREAMINGMODE=$3
 HOST=$4 # optional when starting multiple instances
+WEBUIPORT=$5 # optinal when starting multiple instances
 
 bin=`dirname "$0"`
 bin=`cd "$bin"; pwd`
@@ -65,6 +66,10 @@ if [[ $STARTSTOP == "start" ]]; then
     if [ ! -z $HOST ]; then
         args="${args} --host $HOST"
     fi
+
+    if [ ! -z $WEBUIPORT ]; then
+        args="${args} --webui-port $WEBUIPORT"
+    fi
 fi
 
 ${bin}/flink-daemon.sh $STARTSTOP jobmanager "${args}"

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-dist/src/main/flink-bin/bin/start-cluster.sh
----------------------------------------------------------------------
diff --git a/flink-dist/src/main/flink-bin/bin/start-cluster.sh b/flink-dist/src/main/flink-bin/bin/start-cluster.sh
index 429285e..1fdd885 100755
--- a/flink-dist/src/main/flink-bin/bin/start-cluster.sh
+++ b/flink-dist/src/main/flink-bin/bin/start-cluster.sh
@@ -32,21 +32,26 @@ bin=`cd "$bin"; pwd`
 . "$bin"/config.sh
 
 # Start the JobManager instance(s)
-if [[ -z $ZK_QUORUM ]]; then
-    echo "Starting cluster (${STREAMING_MODE} mode)."
-
-    # Start single JobManager on this machine
-    "$bin"/jobmanager.sh start cluster ${STREAMING_MODE}
-else
+shopt -s nocasematch
+if [[ $RECOVERY_MODE == "zookeeper" ]]; then
     # HA Mode
     readMasters
 
     echo "Starting HA cluster (${STREAMING_MODE} mode) with ${#MASTERS[@]} masters and ${#ZK_QUORUM[@]} peers in ZooKeeper quorum."
 
-    for master in ${MASTERS[@]}; do
-        ssh -n $FLINK_SSH_OPTS $master -- "nohup /bin/bash -l $FLINK_BIN_DIR/jobmanager.sh start cluster ${STREAMING_MODE} ${master} &"
+    for ((i=0;i<${#MASTERS[@]};++i)); do
+        master=${MASTERS[i]}
+        webuiport=${WEBUIPORTS[i]}
+        ssh -n $FLINK_SSH_OPTS $master -- "nohup /bin/bash -l $FLINK_BIN_DIR/jobmanager.sh start cluster ${STREAMING_MODE} ${master} ${webuiport} &"
     done
+
+else
+    echo "Starting cluster (${STREAMING_MODE} mode)."
+
+    # Start single JobManager on this machine
+    "$bin"/jobmanager.sh start cluster ${STREAMING_MODE}
 fi
+shopt -u nocasematch
 
 # Start TaskManager instance(s)
 readSlaves

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-dist/src/main/flink-bin/bin/stop-cluster.sh
----------------------------------------------------------------------
diff --git a/flink-dist/src/main/flink-bin/bin/stop-cluster.sh b/flink-dist/src/main/flink-bin/bin/stop-cluster.sh
index 17a5daf..d6dd572 100755
--- a/flink-dist/src/main/flink-bin/bin/stop-cluster.sh
+++ b/flink-dist/src/main/flink-bin/bin/stop-cluster.sh
@@ -30,13 +30,16 @@ for slave in ${SLAVES[@]}; do
 done
 
 # Stop JobManager instance(s)
-if [[ -z $ZK_QUORUM ]]; then
-    "$bin"/jobmanager.sh stop
-else
-	# HA Mode
+shopt -s nocasematch
+if [[ $RECOVERY_MODE == "zookeeper" ]]; then
+    # HA Mode
     readMasters
 
     for master in ${MASTERS[@]}; do
         ssh -n $FLINK_SSH_OPTS $master -- "nohup /bin/bash -l $bin/jobmanager.sh stop &"
     done
+
+else
+	  "$bin"/jobmanager.sh stop
 fi
+shopt -u nocasematch

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/ExecutionGraphHolder.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/ExecutionGraphHolder.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/ExecutionGraphHolder.java
index a017f3a..4efb7ad 100644
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/ExecutionGraphHolder.java
+++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/ExecutionGraphHolder.java
@@ -23,6 +23,8 @@ import org.apache.flink.runtime.executiongraph.ExecutionGraph;
 import org.apache.flink.runtime.instance.ActorGateway;
 import org.apache.flink.runtime.messages.JobManagerMessages;
 
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 import scala.concurrent.Await;
 import scala.concurrent.Future;
 import scala.concurrent.duration.FiniteDuration;
@@ -30,7 +32,7 @@ import scala.concurrent.duration.FiniteDuration;
 import java.util.WeakHashMap;
 
 /**
- * Gateway to obtaining an {@link ExecutionGraph} from a source, like JobManager or Archiver.
+ * Gateway to obtaining an {@link ExecutionGraph} from a source, like JobManager or Archive.
  * <p>
  * The holder will cache the ExecutionGraph behind a weak reference, which will be cleared
  * at some point once no one else is pointing to the ExecutionGraph.
@@ -38,27 +40,34 @@ import java.util.WeakHashMap;
  * stay valid.
  */
 public class ExecutionGraphHolder {
-	
-	private final ActorGateway source;
+
+	private static final Logger LOG = LoggerFactory.getLogger(ExecutionGraphHolder.class);
+
+	/** Retrieves the current leading JobManager and its corresponding archive */
+	private final JobManagerArchiveRetriever retriever;
 	
 	private final FiniteDuration timeout;
 	
 	private final WeakHashMap<JobID, ExecutionGraph> cache = new WeakHashMap<JobID, ExecutionGraph>();
-	
-	
-	public ExecutionGraphHolder(ActorGateway source) {
-		this(source, WebRuntimeMonitor.DEFAULT_REQUEST_TIMEOUT);
+
+	public ExecutionGraphHolder(JobManagerArchiveRetriever retriever) {
+		this(retriever, WebRuntimeMonitor.DEFAULT_REQUEST_TIMEOUT);
 	}
 
-	public ExecutionGraphHolder(ActorGateway source, FiniteDuration timeout) {
-		if (source == null || timeout == null) {
+	public ExecutionGraphHolder(JobManagerArchiveRetriever retriever, FiniteDuration timeout) {
+		if (retriever == null || timeout == null) {
 			throw new NullPointerException();
 		}
-		this.source = source;
+		this.retriever = retriever;
 		this.timeout = timeout;
 	}
-	
-	
+
+	/**
+	 * Retrieves the execution graph with {@link JobID} jid or null if it cannot be found.
+	 *
+	 * @param jid jobID of the execution graph to be retrieved
+	 * @return the retrieved execution graph or null if it is not retrievable
+	 */
 	public ExecutionGraph getExecutionGraph(JobID jid) {
 		ExecutionGraph cached = cache.get(jid);
 		if (cached != null) {
@@ -66,19 +75,25 @@ public class ExecutionGraphHolder {
 		}
 		
 		try {
-			Future<Object> future = source.ask(new JobManagerMessages.RequestJob(jid), timeout);
-			Object result = Await.result(future, timeout);
-			if (result instanceof JobManagerMessages.JobNotFound) {
+			ActorGateway jobManager = retriever.getJobManagerGateway();
+
+			if (jobManager != null) {
+
+				Future<Object> future = jobManager.ask(new JobManagerMessages.RequestJob(jid), timeout);
+				Object result = Await.result(future, timeout);
+				if (result instanceof JobManagerMessages.JobNotFound) {
+					return null;
+				} else if (result instanceof JobManagerMessages.JobFound) {
+					ExecutionGraph eg = ((JobManagerMessages.JobFound) result).executionGraph();
+					cache.put(jid, eg);
+					return eg;
+				} else {
+					throw new RuntimeException("Unknown response from JobManager / Archive: " + result);
+				}
+			} else {
+				LOG.warn("No connection to the leading JobManager.");
 				return null;
 			}
-			else if (result instanceof JobManagerMessages.JobFound) {
-				ExecutionGraph eg = ((JobManagerMessages.JobFound) result).executionGraph();
-				cache.put(jid, eg);
-				return eg;
-			}
-			else {
-				throw new RuntimeException("Unknown response from JobManager / Archive: " + result);
-			}
 		}
 		catch (Exception e) {
 			throw new RuntimeException("Error requesting execution graph", e);

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/JobManagerArchiveRetriever.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/JobManagerArchiveRetriever.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/JobManagerArchiveRetriever.java
new file mode 100644
index 0000000..91c9ad5
--- /dev/null
+++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/JobManagerArchiveRetriever.java
@@ -0,0 +1,111 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.webmonitor;
+
+import akka.actor.ActorRef;
+import akka.actor.ActorSystem;
+import org.apache.flink.runtime.akka.AkkaUtils;
+import org.apache.flink.runtime.instance.ActorGateway;
+import org.apache.flink.runtime.instance.AkkaActorGateway;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener;
+import org.apache.flink.runtime.messages.JobManagerMessages;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.concurrent.Await;
+import scala.concurrent.Future;
+import scala.concurrent.duration.FiniteDuration;
+
+import java.util.UUID;
+
+/**
+ * Retrieves and stores the actor gateway to the current leading JobManager and its archive. In
+ * case of an error, the {@link WebRuntimeMonitor} to which this instance is associated will be
+ * stopped.
+ */
+public class JobManagerArchiveRetriever implements LeaderRetrievalListener {
+
+	private static final Logger LOG = LoggerFactory.getLogger(JobManagerArchiveRetriever.class);
+
+	private final ActorSystem actorSystem;
+	private final FiniteDuration lookupTimeout;
+	private final FiniteDuration timeout;
+	private final WebMonitor webMonitor;
+
+	/** will be written and read concurrently */
+	private volatile ActorGateway jobManagerGateway;
+	private volatile ActorGateway archiveGateway;
+
+	public JobManagerArchiveRetriever(
+			WebMonitor webMonitor,
+			ActorSystem actorSystem,
+			FiniteDuration lookupTimeout,
+			FiniteDuration timeout) {
+		this.webMonitor = webMonitor;
+		this.actorSystem = actorSystem;
+		this.lookupTimeout = lookupTimeout;
+		this.timeout = timeout;
+	}
+
+	public ActorGateway getJobManagerGateway() {
+		return jobManagerGateway;
+	}
+
+	public ActorGateway getArchiveGateway() {
+		return archiveGateway;
+	}
+
+
+	@Override
+	public void notifyLeaderAddress(String leaderAddress, UUID leaderSessionID) {
+		if (leaderAddress != null && !leaderAddress.equals("")) {
+			try {
+				ActorRef jobManager = AkkaUtils.getActorRef(
+						leaderAddress,
+						actorSystem,
+						lookupTimeout);
+				jobManagerGateway = new AkkaActorGateway(jobManager, leaderSessionID);
+
+				Future<Object> archiveFuture = jobManagerGateway.ask(
+						JobManagerMessages.getRequestArchive(),
+						timeout);
+
+				ActorRef archive = ((JobManagerMessages.ResponseArchive) Await.result(
+						archiveFuture,
+						timeout)
+				).actor();
+
+				archiveGateway = new AkkaActorGateway(archive, leaderSessionID);
+			} catch (Exception e) {
+				handleError(e);
+			}
+		}
+	}
+
+	@Override
+	public void handleError(Exception exception) {
+		LOG.error("Received error from LeaderRetrievalService.", exception);
+
+		try{
+			// stop associated webMonitor
+			webMonitor.stop();
+		} catch (Exception e) {
+			LOG.error("Error while stopping the web server due to a LeaderRetrievalService error.", e);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitor.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitor.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitor.java
index 006d18d..4c38cae 100644
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitor.java
+++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitor.java
@@ -18,6 +18,8 @@
 
 package org.apache.flink.runtime.webmonitor;
 
+import akka.actor.ActorSystem;
+import com.google.common.base.Preconditions;
 import io.netty.bootstrap.ServerBootstrap;
 import io.netty.channel.Channel;
 import io.netty.channel.ChannelInitializer;
@@ -33,7 +35,8 @@ import io.netty.handler.stream.ChunkedWriteHandler;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.IllegalConfigurationException;
-import org.apache.flink.runtime.instance.ActorGateway;
+import org.apache.flink.runtime.akka.AkkaUtils;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
 import org.apache.flink.runtime.webmonitor.files.StaticFileServerHandler;
 import org.apache.flink.runtime.webmonitor.handlers.ExecutionPlanHandler;
 import org.apache.flink.runtime.webmonitor.handlers.JobConfigHandler;
@@ -67,7 +70,7 @@ public class WebRuntimeMonitor implements WebMonitor {
 	public static final FiniteDuration DEFAULT_REQUEST_TIMEOUT = new FiniteDuration(10, TimeUnit.SECONDS);
 	
 	public static final long DEFAULT_REFRESH_INTERVAL = 5000;
-	
+
 	/** Logger for web frontend startup / shutdown messages */
 	private static final Logger LOG = LoggerFactory.getLogger(WebRuntimeMonitor.class);
 	
@@ -77,22 +80,36 @@ public class WebRuntimeMonitor implements WebMonitor {
 	// ------------------------------------------------------------------------
 	
 	private final Object startupShutdownLock = new Object();
-	
-	private final Router router;
+
+	private final LeaderRetrievalService leaderRetrievalService;
+
+	/** LeaderRetrievalListener which stores the currently leading JobManager and its archive */
+	private final JobManagerArchiveRetriever retriever;
 
 	private final int configuredPort;
 
+	private final Router router;
+
 	private ServerBootstrap bootstrap;
 	
 	private Channel serverChannel;
 
-	
-	public WebRuntimeMonitor(Configuration config, ActorGateway jobManager, ActorGateway archive) throws IOException {
+	// ------------------------------------------------------------------------
+
+	public WebRuntimeMonitor(
+			Configuration config,
+			LeaderRetrievalService leaderRetrievalService,
+			ActorSystem actorSystem)
+		throws IOException {
+		Preconditions.checkNotNull(config);
+		this.leaderRetrievalService = Preconditions.checkNotNull(leaderRetrievalService);
+
 		// figure out where our static contents is
 		final String configuredWebRoot = config.getString(ConfigConstants.JOB_MANAGER_WEB_DOC_ROOT_KEY, null);
 		final String flinkRoot = config.getString(ConfigConstants.FLINK_BASE_DIR_PATH_KEY, null);
-		
+
 		final File webRootDir;
+		
 		if (configuredWebRoot != null) {
 			webRootDir = new File(configuredWebRoot);
 		}
@@ -113,22 +130,27 @@ public class WebRuntimeMonitor implements WebMonitor {
 		
 		// port configuration
 		this.configuredPort = config.getInteger(ConfigConstants.JOB_MANAGER_WEB_PORT_KEY,
-												ConfigConstants.DEFAULT_JOB_MANAGER_WEB_FRONTEND_PORT);
+				ConfigConstants.DEFAULT_JOB_MANAGER_WEB_FRONTEND_PORT);
 		if (this.configuredPort < 0) {
 			throw new IllegalArgumentException("Web frontend port is invalid: " + this.configuredPort);
 		}
-		
-		ExecutionGraphHolder currentGraphs = new ExecutionGraphHolder(jobManager);
-		
+
+		FiniteDuration timeout = AkkaUtils.getTimeout(config);
+		FiniteDuration lookupTimeout = AkkaUtils.getTimeout(config);
+
+		retriever = new JobManagerArchiveRetriever(this, actorSystem, lookupTimeout, timeout);
+
+		ExecutionGraphHolder currentGraphs = new ExecutionGraphHolder(retriever);
+
 		router = new Router()
 			// config how to interact with this web server
 			.GET("/config", handler(new RequestConfigHandler(DEFAULT_REFRESH_INTERVAL)))
-			
+
 			// the overview - how many task managers, slots, free slots, ...
-			.GET("/overview", handler(new RequestOverviewHandler(jobManager)))
+			.GET("/overview", handler(new RequestOverviewHandler(retriever)))
 
 			// currently running jobs
-			.GET("/jobs", handler(new RequestJobIdsHandler(jobManager)))
+			.GET("/jobs", handler(new RequestJobIdsHandler(retriever)))
 			.GET("/jobs/:jobid", handler(new JobSummaryHandler(currentGraphs)))
 			.GET("/jobs/:jobid/vertices", handler(new JobVerticesOverviewHandler(currentGraphs)))
 			.GET("/jobs/:jobid/plan", handler(new ExecutionPlanHandler(currentGraphs)))
@@ -137,12 +159,10 @@ public class WebRuntimeMonitor implements WebMonitor {
 //			.GET("/running/:jobid/:jobvertex", handler(new ExecutionPlanHandler(currentGraphs)))
 
 			// the handler for the legacy requests
-			.GET("/jobsInfo", new JobManagerInfoHandler(jobManager, archive, DEFAULT_REQUEST_TIMEOUT))
-					
+			.GET("/jobsInfo", new JobManagerInfoHandler(retriever, DEFAULT_REQUEST_TIMEOUT))
+
 			// this handler serves all the static contents
 			.GET("/:*", new StaticFileServerHandler(webRootDir));
-
-		
 	}
 
 	@Override
@@ -151,13 +171,13 @@ public class WebRuntimeMonitor implements WebMonitor {
 			if (this.bootstrap != null) {
 				throw new IllegalStateException("The server has already been started");
 			}
-			
+
+			final Handler handler = new Handler(router);
+
 			ChannelInitializer<SocketChannel> initializer = new ChannelInitializer<SocketChannel>() {
-	
+
 				@Override
 				protected void initChannel(SocketChannel ch) {
-					Handler handler = new Handler(router);
-					
 					ch.pipeline()
 						.addLast(new HttpServerCodec())
 						.addLast(new HttpObjectAggregator(65536))
@@ -168,7 +188,7 @@ public class WebRuntimeMonitor implements WebMonitor {
 			
 			NioEventLoopGroup bossGroup   = new NioEventLoopGroup(1);
 			NioEventLoopGroup workerGroup = new NioEventLoopGroup();
-	
+
 			this.bootstrap = new ServerBootstrap();
 			this.bootstrap
 					.group(bossGroup, workerGroup)
@@ -177,18 +197,22 @@ public class WebRuntimeMonitor implements WebMonitor {
 	
 			Channel ch = this.bootstrap.bind(configuredPort).sync().channel();
 			this.serverChannel = ch;
-			
+
 			InetSocketAddress bindAddress = (InetSocketAddress) ch.localAddress();
 			String address = bindAddress.getAddress().getHostAddress();
 			int port = bindAddress.getPort();
 			
 			LOG.info("Web frontend listening at " + address + ':' + port);
+
+			leaderRetrievalService.start(retriever);
 		}
 	}
 	
 	@Override
 	public void stop() throws Exception {
 		synchronized (startupShutdownLock) {
+			leaderRetrievalService.stop();
+
 			if (this.serverChannel != null) {
 				this.serverChannel.close().awaitUninterruptibly();
 				this.serverChannel = null;

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/RequestJobIdsHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/RequestJobIdsHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/RequestJobIdsHandler.java
index aa1a39f..8a177f4 100644
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/RequestJobIdsHandler.java
+++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/RequestJobIdsHandler.java
@@ -21,6 +21,7 @@ package org.apache.flink.runtime.webmonitor.handlers;
 import org.apache.flink.runtime.instance.ActorGateway;
 import org.apache.flink.runtime.messages.webmonitor.JobsWithIDsOverview;
 import org.apache.flink.runtime.messages.webmonitor.RequestJobsWithIDsOverview;
+import org.apache.flink.runtime.webmonitor.JobManagerArchiveRetriever;
 import org.apache.flink.runtime.webmonitor.JsonFactory;
 import org.apache.flink.runtime.webmonitor.WebRuntimeMonitor;
 
@@ -37,19 +38,19 @@ import java.util.Map;
  */
 public class RequestJobIdsHandler implements RequestHandler, RequestHandler.JsonResponse {
 	
-	private final ActorGateway target;
+	private final JobManagerArchiveRetriever retriever;
 	
 	private final FiniteDuration timeout;
 	
-	public RequestJobIdsHandler(ActorGateway target) {
-		this(target, WebRuntimeMonitor.DEFAULT_REQUEST_TIMEOUT);
+	public RequestJobIdsHandler(JobManagerArchiveRetriever retriever) {
+		this(retriever, WebRuntimeMonitor.DEFAULT_REQUEST_TIMEOUT);
 	}
 	
-	public RequestJobIdsHandler(ActorGateway target, FiniteDuration timeout) {
-		if (target == null || timeout == null) {
+	public RequestJobIdsHandler(JobManagerArchiveRetriever retriever, FiniteDuration timeout) {
+		if (retriever == null || timeout == null) {
 			throw new NullPointerException();
 		}
-		this.target = target;
+		this.retriever = retriever;
 		this.timeout = timeout;
 	}
 	
@@ -57,9 +58,15 @@ public class RequestJobIdsHandler implements RequestHandler, RequestHandler.Json
 	public String handleRequest(Map<String, String> params) throws Exception {
 		// we need no parameters, get all requests
 		try {
-			Future<Object> future = target.ask(RequestJobsWithIDsOverview.getInstance(), timeout);
-			JobsWithIDsOverview result = (JobsWithIDsOverview) Await.result(future, timeout);
-			return JsonFactory.generateJobsOverviewJSON(result);
+			ActorGateway jobManager = retriever.getJobManagerGateway();
+
+			if (jobManager != null) {
+				Future<Object> future = jobManager.ask(RequestJobsWithIDsOverview.getInstance(), timeout);
+				JobsWithIDsOverview result = (JobsWithIDsOverview) Await.result(future, timeout);
+				return JsonFactory.generateJobsOverviewJSON(result);
+			} else {
+				throw new Exception("No connection to the leading JobManager.");
+			}
 		}
 		catch (Exception e) {
 			throw new RuntimeException("Failed to fetch list of all running jobs: " + e.getMessage(), e);

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/RequestOverviewHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/RequestOverviewHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/RequestOverviewHandler.java
index c2c00c7..ce30122 100644
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/RequestOverviewHandler.java
+++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/RequestOverviewHandler.java
@@ -21,6 +21,7 @@ package org.apache.flink.runtime.webmonitor.handlers;
 import org.apache.flink.runtime.instance.ActorGateway;
 import org.apache.flink.runtime.messages.webmonitor.RequestStatusWithJobIDsOverview;
 import org.apache.flink.runtime.messages.webmonitor.StatusWithJobIDsOverview;
+import org.apache.flink.runtime.webmonitor.JobManagerArchiveRetriever;
 import org.apache.flink.runtime.webmonitor.JsonFactory;
 import org.apache.flink.runtime.webmonitor.WebRuntimeMonitor;
 
@@ -36,29 +37,35 @@ import java.util.Map;
  */
 public class RequestOverviewHandler implements  RequestHandler, RequestHandler.JsonResponse {
 	
-	private final ActorGateway jobManager;
+	private final JobManagerArchiveRetriever retriever;
 	
 	private final FiniteDuration timeout;
 	
 	
-	public RequestOverviewHandler(ActorGateway jobManager) {
-		this(jobManager, WebRuntimeMonitor.DEFAULT_REQUEST_TIMEOUT);
+	public RequestOverviewHandler(JobManagerArchiveRetriever retriever) {
+		this(retriever, WebRuntimeMonitor.DEFAULT_REQUEST_TIMEOUT);
 	}
 	
-	public RequestOverviewHandler(ActorGateway jobManager, FiniteDuration timeout) {
-		if (jobManager == null || timeout == null) {
+	public RequestOverviewHandler(JobManagerArchiveRetriever retriever, FiniteDuration timeout) {
+		if (retriever == null || timeout == null) {
 			throw new NullPointerException();
 		}
-		this.jobManager = jobManager;
+		this.retriever = retriever;
 		this.timeout = timeout;
 	}
 	
 	@Override
 	public String handleRequest(Map<String, String> params) throws Exception {
 		try {
-			Future<Object> future = jobManager.ask(RequestStatusWithJobIDsOverview.getInstance(), timeout);
-			StatusWithJobIDsOverview result = (StatusWithJobIDsOverview) Await.result(future, timeout);
-			return JsonFactory.generateOverviewWithJobIDsJSON(result);
+			ActorGateway jobManager = retriever.getJobManagerGateway();
+
+			if (jobManager != null) {
+				Future<Object> future = jobManager.ask(RequestStatusWithJobIDsOverview.getInstance(), timeout);
+				StatusWithJobIDsOverview result = (StatusWithJobIDsOverview) Await.result(future, timeout);
+				return JsonFactory.generateOverviewWithJobIDsJSON(result);
+			} else {
+				throw new Exception("No connection to the leading job manager.");
+			}
 		}
 		catch (Exception e) {
 			throw new Exception("Failed to fetch the status overview: " + e.getMessage(), e);

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/legacy/JobManagerInfoHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/legacy/JobManagerInfoHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/legacy/JobManagerInfoHandler.java
index 9b52736..3f1842b 100644
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/legacy/JobManagerInfoHandler.java
+++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/legacy/JobManagerInfoHandler.java
@@ -48,6 +48,7 @@ import org.apache.flink.runtime.messages.accumulators.AccumulatorResultsErroneou
 import org.apache.flink.runtime.messages.accumulators.AccumulatorResultsNotFound;
 import org.apache.flink.runtime.messages.accumulators.RequestAccumulatorResultsStringified;
 import org.apache.flink.runtime.util.EnvironmentInformation;
+import org.apache.flink.runtime.webmonitor.JobManagerArchiveRetriever;
 import org.apache.flink.util.ExceptionUtils;
 
 import org.apache.flink.util.StringUtils;
@@ -75,14 +76,15 @@ public class JobManagerInfoHandler extends SimpleChannelInboundHandler<Routed> {
 	private static final Charset ENCODING = Charset.forName("UTF-8");
 
 	/** Underlying JobManager */
-	private final ActorGateway jobmanager;
-	private final ActorGateway archive;
+	private final JobManagerArchiveRetriever retriever;
 	private final FiniteDuration timeout;
 
+	private ActorGateway jobmanager;
+	private ActorGateway archive;
 
-	public JobManagerInfoHandler(ActorGateway jobmanager, ActorGateway archive, FiniteDuration timeout) {
-		this.jobmanager = jobmanager;
-		this.archive = archive;
+
+	public JobManagerInfoHandler(JobManagerArchiveRetriever retriever, FiniteDuration timeout) {
+		this.retriever = retriever;
 		this.timeout = timeout;
 	}
 
@@ -90,6 +92,18 @@ public class JobManagerInfoHandler extends SimpleChannelInboundHandler<Routed> {
 	protected void channelRead0(ChannelHandlerContext ctx, Routed routed) throws Exception {
 		DefaultFullHttpResponse response;
 		try {
+			jobmanager = retriever.getJobManagerGateway();
+
+			if (jobmanager == null) {
+				throw new Exception("No connection to leading JobManager.");
+			}
+
+			archive = retriever.getArchiveGateway();
+
+			if (archive == null) {
+				throw new Exception("No connection to leading JobManager.");
+			}
+
 			String result = handleRequest(routed);
 			byte[] bytes = result.getBytes(ENCODING);
 
@@ -114,6 +128,8 @@ public class JobManagerInfoHandler extends SimpleChannelInboundHandler<Routed> {
 	
 	@SuppressWarnings("unchecked")
 	private String handleRequest(Routed routed) throws Exception {
+
+
 		if ("archive".equals(routed.queryParam("get"))) {
 			Future<Object> response = archive.ask(ArchiveMessages.getRequestArchivedJobs(), timeout);
 

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/runner/TestRunner.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/runner/TestRunner.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/runner/TestRunner.java
index 9a9b6ba..a2978a1 100644
--- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/runner/TestRunner.java
+++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/runner/TestRunner.java
@@ -45,14 +45,15 @@ public class TestRunner {
 
 		// start the cluster with the runtime monitor
 		Configuration configuration = new Configuration();
-		configuration.setBoolean(ConfigConstants.LOCAL_INSTANCE_MANAGER_START_WEBSERVER, true);
+		configuration.setBoolean(ConfigConstants.LOCAL_START_WEBSERVER, true);
 		configuration.setBoolean(ConfigConstants.JOB_MANAGER_NEW_WEB_FRONTEND_KEY, true);
 		configuration.setString(ConfigConstants.JOB_MANAGER_WEB_DOC_ROOT_KEY,
-			"/data/repositories/flink/flink-dist/target/flink-0.10-SNAPSHOT-bin/flink-0.10-SNAPSHOT/resources/web-runtime-monitor");
+			"flink-dist/target/flink-0.10-SNAPSHOT-bin/flink-0.10-SNAPSHOT/resources/web-runtime-monitor");
 		
 		LocalFlinkMiniCluster cluster = new LocalFlinkMiniCluster(configuration, false);
+		cluster.start();
 
-		final int port = cluster.getJobManagerRPCPort();
+		final int port = cluster.getLeaderRPCPort();
 		runWordCount(port);
 		runWebLogAnalysisExample(port);
 		runWordCount(port);

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime-web/src/test/resources/log4j-test.properties
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/test/resources/log4j-test.properties b/flink-runtime-web/src/test/resources/log4j-test.properties
new file mode 100644
index 0000000..2b9292a
--- /dev/null
+++ b/flink-runtime-web/src/test/resources/log4j-test.properties
@@ -0,0 +1,38 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+log4j.rootLogger=OFF, console
+
+# -----------------------------------------------------------------------------
+# Console (use 'console')
+# -----------------------------------------------------------------------------
+log4j.appender.console=org.apache.log4j.ConsoleAppender
+log4j.appender.console.layout=org.apache.flink.util.MavenForkNumberPrefixLayout
+log4j.appender.console.layout.ConversionPattern=%d{HH:mm:ss,SSS} %-5p %-60c %x - %m%n
+
+# -----------------------------------------------------------------------------
+# File (use 'file')
+# -----------------------------------------------------------------------------
+log4j.appender.file=org.apache.log4j.FileAppender
+log4j.appender.file.file=${log.dir}/{$mvn.forkNumber}.log
+log4j.appender.file.append=false
+log4j.appender.file.layout=org.apache.log4j.PatternLayout
+log4j.appender.file.layout.ConversionPattern=%d{HH:mm:ss,SSS} %-5p %-60c %x - %m%n
+
+# suppress the irrelevant (wrong) warnings from the netty channel handler
+log4j.logger.org.jboss.netty.channel.DefaultChannelPipeline=ERROR, console

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime-web/src/test/resources/logback-test.xml
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/test/resources/logback-test.xml b/flink-runtime-web/src/test/resources/logback-test.xml
new file mode 100644
index 0000000..9d4f644
--- /dev/null
+++ b/flink-runtime-web/src/test/resources/logback-test.xml
@@ -0,0 +1,42 @@
+<!--
+  ~ Licensed to the Apache Software Foundation (ASF) under one
+  ~ or more contributor license agreements.  See the NOTICE file
+  ~ distributed with this work for additional information
+  ~ regarding copyright ownership.  The ASF licenses this file
+  ~ to you under the Apache License, Version 2.0 (the
+  ~ "License"); you may not use this file except in compliance
+  ~ with the License.  You may obtain a copy of the License at
+  ~
+  ~     http://www.apache.org/licenses/LICENSE-2.0
+  ~
+  ~ Unless required by applicable law or agreed to in writing, software
+  ~ distributed under the License is distributed on an "AS IS" BASIS,
+  ~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  ~ See the License for the specific language governing permissions and
+  ~ limitations under the License.
+  -->
+
+<configuration>
+    <appender name="STDOUT" class="ch.qos.logback.core.ConsoleAppender">
+        <encoder>
+            <pattern>%d{HH:mm:ss.SSS} [%thread] [%X{sourceThread} - %X{akkaSource}] %-5level %logger{60} - %msg%n</pattern>
+        </encoder>
+    </appender>
+
+    <root level="WARN">
+        <appender-ref ref="STDOUT"/>
+    </root>
+
+    <!-- The following loggers are disabled during tests, because many tests deliberately
+         throw error to test failing scenarios. Logging those would overflow the log. -->
+         <!---->
+    <logger name="org.apache.flink.runtime.operators.DataSinkTask" level="OFF"/>
+    <logger name="org.apache.flink.runtime.operators.RegularPactTask" level="OFF"/>
+    <logger name="org.apache.flink.runtime.client.JobClient" level="OFF"/>
+    <logger name="org.apache.flink.runtime.taskmanager.Task" level="OFF"/>
+    <logger name="org.apache.flink.runtime.jobmanager.JobManager" level="OFF"/>
+    <logger name="org.apache.flink.runtime.testingUtils" level="OFF"/>
+    <logger name="org.apache.flink.runtime.executiongraph.ExecutionGraph" level="OFF"/>
+    <logger name="org.apache.flink.runtime.jobmanager.EventCollector" level="OFF"/>
+    <logger name="org.apache.flink.runtime.instance.InstanceManager" level="OFF"/>
+</configuration>

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/pom.xml
----------------------------------------------------------------------
diff --git a/flink-runtime/pom.xml b/flink-runtime/pom.xml
index 6dc8d42..c5706a1 100644
--- a/flink-runtime/pom.xml
+++ b/flink-runtime/pom.xml
@@ -196,6 +196,12 @@ under the License.
 			<groupId>org.apache.curator</groupId>
 			<artifactId>curator-recipes</artifactId>
 		</dependency>
+		
+		<dependency>
+			<groupId>org.apache.curator</groupId>
+			<artifactId>curator-test</artifactId>
+		</dependency>
+
 	</dependencies>
 
 	<build>

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/main/java/org/apache/flink/runtime/akka/FlinkUntypedActor.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/akka/FlinkUntypedActor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/akka/FlinkUntypedActor.java
index 0623862..7128286 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/akka/FlinkUntypedActor.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/akka/FlinkUntypedActor.java
@@ -23,7 +23,6 @@ import org.apache.flink.runtime.messages.JobManagerMessages.LeaderSessionMessage
 import org.apache.flink.runtime.messages.RequiresLeaderSessionID;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import scala.Option;
 
 import java.util.UUID;
 
@@ -82,28 +81,27 @@ public abstract class FlinkUntypedActor extends UntypedActor {
 	private void handleLeaderSessionID(Object message) throws Exception {
 		if(message instanceof LeaderSessionMessage) {
 			LeaderSessionMessage msg = (LeaderSessionMessage) message;
+			UUID expectedID = getLeaderSessionID();
+			UUID actualID = msg.leaderSessionID();
 
-			if(msg.leaderSessionID().isDefined() && getLeaderSessionID().isDefined()) {
-				if(getLeaderSessionID().equals(msg.leaderSessionID())) {
-					// finally call method to handle message
-					handleMessage(msg.message());
-				} else {
-					handleDiscardedMessage(msg);
-				}
+			if(expectedID == actualID || (expectedID != null && expectedID.equals(actualID))) {
+				handleMessage(msg.message());
 			} else {
-				handleDiscardedMessage(msg);
+				handleDiscardedMessage(expectedID, msg);
 			}
 		} else if (message instanceof RequiresLeaderSessionID) {
 			throw new Exception("Received a message " + message + " without a leader session " +
-					"ID, even though it requires to have one.");
+					"ID, even though the message requires a leader session ID.");
 		} else {
 			// call method to handle message
 			handleMessage(message);
 		}
 	}
 
-	private void handleDiscardedMessage(Object msg) {
-		LOG.debug("Discard message {} because the leader session ID was not correct.", msg);
+	private void handleDiscardedMessage(UUID expectedLeaderSessionID, LeaderSessionMessage msg) {
+		LOG.warn("Discard message {} because the expected leader session ID {} did not " +
+				"equal the received leader session ID {}.", msg, expectedLeaderSessionID,
+				msg.leaderSessionID());
 	}
 
 	/**
@@ -118,7 +116,7 @@ public abstract class FlinkUntypedActor extends UntypedActor {
 	 * Returns the current leader session ID associcated with this actor.
 	 * @return
 	 */
-	protected abstract Option<UUID> getLeaderSessionID();
+	abstract protected UUID getLeaderSessionID();
 
 	/**
 	 * This method should be called for every outgoing message. It wraps messages which require

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/main/java/org/apache/flink/runtime/akka/ListeningBehaviour.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/akka/ListeningBehaviour.java b/flink-runtime/src/main/java/org/apache/flink/runtime/akka/ListeningBehaviour.java
new file mode 100644
index 0000000..59e3f16
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/akka/ListeningBehaviour.java
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.akka;
+
+/**
+ * Defines the listening behaviour of the JobClientActor and thus the messages
+ * which are sent from the JobManger to the JobClientActor.
+ */
+public enum ListeningBehaviour {
+	DETACHED, // only receive the Acknowledge message about the job submission message
+	EXECUTION_RESULT, // receive additionally the SerializedJobExecutionResult
+	EXECUTION_RESULT_AND_STATE_CHANGES // receive additionally the JobStatusChanged messages
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServer.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServer.java
index 626d21f..ef2ef61 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServer.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServer.java
@@ -258,6 +258,10 @@ public class BlobServer extends Thread implements BlobService {
 					LOG.warn("Exception while unregistering BLOB server's cleanup shutdown hook.");
 				}
 			}
+
+			if(LOG.isInfoEnabled()) {
+				LOG.info("Stopped BLOB server at {}:{}", serverSocket.getInetAddress().getHostAddress(), getPort());
+			}
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java
index 3681e9e..8f0b19b 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java
@@ -36,7 +36,6 @@ import org.apache.flink.runtime.messages.checkpoint.NotifyCheckpointComplete;
 import org.apache.flink.runtime.messages.checkpoint.TriggerCheckpoint;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import scala.Option;
 
 import java.util.ArrayDeque;
 import java.util.ArrayList;
@@ -576,7 +575,7 @@ public class CheckpointCoordinator {
 	public ActorGateway createJobStatusListener(
 			ActorSystem actorSystem,
 			long checkpointInterval,
-			Option<UUID> leaderSessionID) {
+			UUID leaderSessionID) {
 		synchronized (lock) {
 			if (shutdown) {
 				throw new IllegalArgumentException("Checkpoint coordinator is shut down");

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorDeActivator.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorDeActivator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorDeActivator.java
index f65be15..7e32b72 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorDeActivator.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorDeActivator.java
@@ -22,7 +22,6 @@ import com.google.common.base.Preconditions;
 import org.apache.flink.runtime.akka.FlinkUntypedActor;
 import org.apache.flink.runtime.jobgraph.JobStatus;
 import org.apache.flink.runtime.messages.ExecutionGraphMessages;
-import scala.Option;
 
 import java.util.UUID;
 
@@ -34,16 +33,17 @@ public class CheckpointCoordinatorDeActivator extends FlinkUntypedActor {
 
 	private final CheckpointCoordinator coordinator;
 	private final long interval;
-	private final Option<UUID> leaderSessionID;
+	private final UUID leaderSessionID;
 	
 	public CheckpointCoordinatorDeActivator(
 			CheckpointCoordinator coordinator,
 			long interval,
-			Option<UUID> leaderSessionID) {
-		Preconditions.checkNotNull(coordinator, "The checkpointCoordinator must not be null.");
-		Preconditions.checkNotNull(leaderSessionID, "The leaderSesssionID must not be null.");
+			UUID leaderSessionID) {
+
+		LOG.info("Create CheckpointCoordinatorDeActivator");
+
+		this.coordinator = Preconditions.checkNotNull(coordinator, "The checkpointCoordinator must not be null.");
 
-		this.coordinator = coordinator;
 		this.interval = interval;
 		this.leaderSessionID = leaderSessionID;
 	}
@@ -67,7 +67,7 @@ public class CheckpointCoordinatorDeActivator extends FlinkUntypedActor {
 	}
 
 	@Override
-	public Option<UUID> getLeaderSessionID() {
+	public UUID getLeaderSessionID() {
 		return leaderSessionID;
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClient.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClient.java b/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClient.java
index 7507643..9d64866 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClient.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClient.java
@@ -31,6 +31,7 @@ import org.apache.flink.api.common.JobID;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.akka.AkkaUtils;
+import org.apache.flink.runtime.akka.ListeningBehaviour;
 import org.apache.flink.runtime.instance.ActorGateway;
 import org.apache.flink.runtime.jobgraph.JobGraph;
 import org.apache.flink.runtime.messages.JobClientMessages;
@@ -120,6 +121,8 @@ public class JobClient {
 	 * @param jobManagerGateway  Gateway to the JobManager that should execute the job.
 	 * @param jobGraph    JobGraph describing the Flink job
 	 * @param timeout     Timeout for futures
+	 * @param sysoutLogUpdates prints log updates to system out if true
+	 * @param userCodeClassloader class loader to be used for deserialization
 	 * @return The job execution result
 	 * @throws org.apache.flink.runtime.client.JobExecutionException Thrown if the job
 	 *                                                               execution fails.
@@ -236,7 +239,10 @@ public class JobClient {
 		Object result;
 		try {
 			Future<Object> future = jobManagerGateway.ask(
-					new JobManagerMessages.SubmitJob(jobGraph, false),
+				new JobManagerMessages.SubmitJob(
+					jobGraph,
+					ListeningBehaviour.DETACHED // only receive the Acknowledge for the job submission message
+				),
 					timeout);
 			
 			result = Await.result(future, timeout);

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClientActor.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClientActor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClientActor.java
index 16c6baf..bf747c4 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClientActor.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClientActor.java
@@ -24,12 +24,12 @@ import akka.actor.Status;
 import akka.actor.Terminated;
 import com.google.common.base.Preconditions;
 import org.apache.flink.runtime.akka.FlinkUntypedActor;
+import org.apache.flink.runtime.akka.ListeningBehaviour;
 import org.apache.flink.runtime.jobgraph.JobGraph;
 import org.apache.flink.runtime.messages.ExecutionGraphMessages;
 import org.apache.flink.runtime.messages.JobClientMessages;
 import org.apache.flink.runtime.messages.JobManagerMessages;
 import org.slf4j.Logger;
-import scala.Option;
 
 import java.util.UUID;
 
@@ -44,18 +44,18 @@ public class JobClientActor extends FlinkUntypedActor {
 	private final boolean sysoutUpdates;
 
 	/** leader session ID of the JobManager when this actor was created */
-	private final Option<UUID> leaderSessionID;
+	private final UUID leaderSessionID;
 
 	/** Actor which submits a job to the JobManager via this actor */
 	private ActorRef submitter;
 
 	public JobClientActor(ActorRef jobManager, Logger logger, boolean sysoutUpdates,
-							Option<UUID> leaderSessionID) {
-		
+			UUID leaderSessionID) {
+
 		this.jobManager = Preconditions.checkNotNull(jobManager, "The JobManager ActorRef must not be null.");
 		this.logger = Preconditions.checkNotNull(logger, "The logger must not be null.");
-		this.leaderSessionID = Preconditions.checkNotNull(leaderSessionID, "The leader session ID option must not be null.");
 
+		this.leaderSessionID = leaderSessionID;
 		this.sysoutUpdates = sysoutUpdates;
 	}
 	
@@ -91,7 +91,11 @@ public class JobClientActor extends FlinkUntypedActor {
 
 					this.submitter = getSender();
 					jobManager.tell(
-							decorateMessage(new JobManagerMessages.SubmitJob(jobGraph, true)), getSelf());
+						decorateMessage(
+							new JobManagerMessages.SubmitJob(
+								jobGraph,
+								ListeningBehaviour.EXECUTION_RESULT_AND_STATE_CHANGES)),
+							getSelf());
 					
 					// make sure we notify the sender when the connection got lost
 					getContext().watch(jobManager);
@@ -102,8 +106,7 @@ public class JobClientActor extends FlinkUntypedActor {
 				String msg = "Received repeated 'SubmitJobAndWait'";
 				logger.error(msg);
 				getSender().tell(
-						decorateMessage(new Status.Failure(new Exception(msg))),
-						ActorRef.noSender());
+					decorateMessage(new Status.Failure(new Exception(msg))), ActorRef.noSender());
 
 				getContext().unwatch(jobManager);
 				getSelf().tell(decorateMessage(PoisonPill.getInstance()), ActorRef.noSender());
@@ -153,7 +156,7 @@ public class JobClientActor extends FlinkUntypedActor {
 	}
 
 	@Override
-	protected Option<UUID> getLeaderSessionID() {
+	protected UUID getLeaderSessionID() {
 		return leaderSessionID;
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java
index 189682b..4cee2f3 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java
@@ -556,7 +556,9 @@ public class Execution implements Serializable {
 							partitionId, partitionLocation);
 
 					final UpdatePartitionInfo updateTaskMessage = new UpdateTaskSinglePartitionInfo(
-							consumer.getAttemptId(), partition.getIntermediateResult().getId(), descriptor);
+						consumer.getAttemptId(),
+						partition.getIntermediateResult().getId(),
+						descriptor);
 
 					sendUpdatePartitionInfoRpcCall(consumerSlot, updateTaskMessage);
 				}

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java
index 3602372..cde1741 100755
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java
@@ -51,7 +51,6 @@ import org.apache.flink.util.InstantiationUtil;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import scala.Option;
 import scala.concurrent.ExecutionContext;
 import scala.concurrent.duration.FiniteDuration;
 
@@ -326,7 +325,7 @@ public class ExecutionGraph implements Serializable {
 			List<ExecutionJobVertex> verticesToWaitFor,
 			List<ExecutionJobVertex> verticesToCommitTo,
 			ActorSystem actorSystem,
-			Option<UUID> leaderSessionID) {
+			UUID leaderSessionID) {
 		// simple sanity checks
 		if (interval < 10 || checkpointTimeout < 10) {
 			throw new IllegalArgumentException();

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/main/java/org/apache/flink/runtime/instance/ActorGateway.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/ActorGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/ActorGateway.java
index fe4a1cd..a82debb 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/ActorGateway.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/ActorGateway.java
@@ -19,11 +19,11 @@
 package org.apache.flink.runtime.instance;
 
 import akka.actor.ActorRef;
-import scala.Option;
 import scala.concurrent.ExecutionContext;
 import scala.concurrent.Future;
 import scala.concurrent.duration.FiniteDuration;
 
+import java.io.Serializable;
 import java.util.UUID;
 
 /**
@@ -31,7 +31,7 @@ import java.util.UUID;
  *
  * It allows to avoid direct interaction with an ActorRef.
  */
-public interface ActorGateway {
+public interface ActorGateway extends Serializable {
 
 	/**
 	 * Sends a message asynchronously and returns its response. The response to the message is
@@ -99,9 +99,9 @@ public interface ActorGateway {
 	ActorRef actor();
 
 	/**
-	 * Returns the leaderSessionID associated with the remote actor or None.
+	 * Returns the leaderSessionID associated with the remote actor or null.
 	 *
-	 * @return Leader session ID if its associated with this gateway, otherwise None
+	 * @return Leader session ID if its associated with this gateway, otherwise null
 	 */
-	Option<UUID> leaderSessionID();
+	UUID leaderSessionID();
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/main/java/org/apache/flink/runtime/instance/AkkaActorGateway.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/AkkaActorGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/AkkaActorGateway.java
index ea55458..c00e7fe 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/AkkaActorGateway.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/AkkaActorGateway.java
@@ -21,31 +21,33 @@ package org.apache.flink.runtime.instance;
 import akka.actor.ActorRef;
 import akka.pattern.Patterns;
 import akka.util.Timeout;
-import org.apache.flink.runtime.LeaderSessionMessageDecorator;
-import org.apache.flink.runtime.MessageDecorator;
 import org.apache.flink.runtime.akka.AkkaUtils;
-import scala.Option;
+import org.apache.flink.runtime.messages.LeaderSessionMessageDecorator;
+import org.apache.flink.runtime.messages.MessageDecorator;
 import scala.concurrent.ExecutionContext;
 import scala.concurrent.Future;
 import scala.concurrent.duration.FiniteDuration;
 
+import java.io.Serializable;
 import java.util.UUID;
 
 /**
  * Concrete {@link ActorGateway} implementation which uses Akka to communicate with remote actors.
  */
-public class AkkaActorGateway implements ActorGateway {
+public class AkkaActorGateway implements ActorGateway, Serializable {
+
+	private static final long serialVersionUID = 42l;
 
 	// ActorRef of the remote instance
 	private final ActorRef actor;
 
 	// Associated leader session ID, which is used for RequiresLeaderSessionID messages
-	private final Option<UUID> leaderSessionID;
+	private final UUID leaderSessionID;
 
 	// Decorator for messages
 	private final MessageDecorator decorator;
 
-	public AkkaActorGateway(ActorRef actor, Option<UUID> leaderSessionID) {
+	public AkkaActorGateway(ActorRef actor, UUID leaderSessionID) {
 		this.actor = actor;
 		this.leaderSessionID = leaderSessionID;
 		// we want to wrap RequiresLeaderSessionID messages in a LeaderSessionMessage
@@ -151,7 +153,7 @@ public class AkkaActorGateway implements ActorGateway {
 	}
 
 	@Override
-	public Option<UUID> leaderSessionID() {
+	public UUID leaderSessionID() {
 		return leaderSessionID;
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceManager.java
index 0a6b4d0..03213e4 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceManager.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceManager.java
@@ -30,7 +30,6 @@ import java.util.UUID;
 import akka.actor.ActorRef;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import scala.Option;
 
 /**
  * Simple manager that keeps track of which TaskManager are available and alive.
@@ -143,7 +142,7 @@ public class InstanceManager {
 			InstanceConnectionInfo connectionInfo,
 			HardwareDescription resources,
 			int numberOfSlots,
-			Option<UUID> leaderSessionID){
+			UUID leaderSessionID){
 		synchronized(this.lock){
 			if (this.isShutdown) {
 				throw new IllegalStateException("InstanceManager is shut down.");
@@ -177,8 +176,14 @@ public class InstanceManager {
 			totalNumberOfAliveTaskSlots += numberOfSlots;
 
 			if (LOG.isInfoEnabled()) {
-				LOG.info(String.format("Registered TaskManager at %s (%s) as %s. Current number of registered hosts is %d.",
-						connectionInfo.getHostname(), taskManager.path(), id, registeredHostsById.size()));
+				LOG.info(String.format("Registered TaskManager at %s (%s) as %s. " +
+								"Current number of registered hosts is %d. " +
+								"Current number of alive task slots is %d.",
+						connectionInfo.getHostname(),
+						taskManager.path(),
+						id,
+						registeredHostsById.size(),
+						totalNumberOfAliveTaskSlots));
 			}
 
 			host.reportHeartBeat();
@@ -190,13 +195,22 @@ public class InstanceManager {
 		}
 	}
 
-	public void unregisterTaskManager(ActorRef taskManager){
+	/**
+	 * Unregisters the TaskManager with the given {@link ActorRef}. Unregistering means to mark
+	 * the given instance as dead and notify {@link InstanceListener} about the dead instance.
+	 *
+	 * @param taskManager TaskManager which is about to be marked dead.
+	 */
+	public void unregisterTaskManager(ActorRef taskManager, boolean terminated){
 		Instance host = registeredHostsByConnection.get(taskManager);
 
 		if(host != null){
 			registeredHostsByConnection.remove(taskManager);
 			registeredHostsById.remove(host.getId());
-			deadHosts.add(taskManager);
+
+			if (terminated) {
+				deadHosts.add(taskManager);
+			}
 
 			host.markDead();
 
@@ -204,12 +218,30 @@ public class InstanceManager {
 
 			notifyDeadInstance(host);
 
-			LOG.info("Unregistered task manager " + taskManager.path().address() + ". Number of " +
+			LOG.info("Unregistered task manager " + taskManager.path() + ". Number of " +
 					"registered task managers " + getNumberOfRegisteredTaskManagers() + ". Number" +
 					" of available slots " + getTotalNumberOfSlots() + ".");
 		}
 	}
 
+	/**
+	 * Unregisters all currently registered TaskManagers from the InstanceManager.
+	 */
+	public void unregisterAllTaskManagers() {
+		for(Instance instance: registeredHostsById.values()) {
+			deadHosts.add(instance.getActorGateway().actor());
+
+			instance.markDead();
+
+			totalNumberOfAliveTaskSlots -= instance.getTotalNumberOfSlots();
+
+			notifyDeadInstance(instance);
+		}
+
+		registeredHostsById.clear();
+		registeredHostsByConnection.clear();
+	}
+
 	public boolean isRegistered(ActorRef taskManager) {
 		return registeredHostsByConnection.containsKey(taskManager);
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NetworkEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NetworkEnvironment.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NetworkEnvironment.java
index 539dbc0..9eb4a9a 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NetworkEnvironment.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NetworkEnvironment.java
@@ -36,6 +36,7 @@ import org.apache.flink.runtime.io.network.partition.ResultPartitionManager;
 import org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate;
 import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
 import org.apache.flink.runtime.messages.JobManagerMessages.RequestPartitionState;
+import org.apache.flink.runtime.messages.TaskMessages.FailTask;
 import org.apache.flink.runtime.taskmanager.NetworkEnvironmentConfiguration;
 import org.apache.flink.runtime.taskmanager.Task;
 import org.apache.flink.runtime.taskmanager.TaskManager;
@@ -51,7 +52,6 @@ import java.io.IOException;
 
 import static com.google.common.base.Preconditions.checkNotNull;
 import static org.apache.flink.runtime.messages.JobManagerMessages.ScheduleOrUpdateConsumers;
-import static org.apache.flink.runtime.messages.TaskMessages.FailTask;
 
 /**
  * Network I/O components of each {@link TaskManager} instance. The network environment contains

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobManagerCliOptions.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobManagerCliOptions.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobManagerCliOptions.java
index edfa87b..1202499 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobManagerCliOptions.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobManagerCliOptions.java
@@ -35,6 +35,8 @@ public class JobManagerCliOptions {
 
 	private String host;
 
+	private int webUIPort = -1;
+
 	// ------------------------------------------------------------------------
 
 	public String getConfigDir() {
@@ -86,4 +88,12 @@ public class JobManagerCliOptions {
 	public void setHost(String host) {
 		this.host = checkNotNull(host);
 	}
+
+	public int getWebUIPort() {
+		return webUIPort;
+	}
+
+	public void setWebUIPort(int webUIPort) {
+		this.webUIPort = webUIPort;
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/RecoveryMode.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/RecoveryMode.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/RecoveryMode.java
new file mode 100644
index 0000000..90a1147
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/RecoveryMode.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.jobmanager;
+
+/**
+ * Recovery mode for Flink's cluster execution. Currently supported modes are:
+ *
+ *   - Standalone: No recovery from JobManager failures
+ *   - ZooKeeper: JobManager high availability via ZooKeeper
+ *     ZooKeeper is used to select a leader among a group of JobManager. This JobManager
+ *     is responsible for the job execution. Upon failure of the leader a new leader is elected
+ *     which will take over the responsibilities of the old leader
+ */
+public enum RecoveryMode {
+	STANDALONE,
+	ZOOKEEPER
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/web/WebInfoServer.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/web/WebInfoServer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/web/WebInfoServer.java
index 4383b65..21a1f51 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/web/WebInfoServer.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/web/WebInfoServer.java
@@ -22,30 +22,41 @@ import java.io.File;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.net.URL;
+import java.util.UUID;
 
+import akka.actor.ActorRef;
+import akka.actor.ActorSystem;
+import com.google.common.base.Preconditions;
 import org.apache.flink.runtime.akka.AkkaUtils;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 
 import org.apache.flink.runtime.instance.ActorGateway;
+import org.apache.flink.runtime.instance.AkkaActorGateway;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+import org.apache.flink.runtime.messages.JobManagerMessages;
 import org.apache.flink.runtime.webmonitor.WebMonitor;
 import org.eclipse.jetty.server.Connector;
+import org.eclipse.jetty.server.Handler;
+import org.eclipse.jetty.server.handler.HandlerCollection;
 import org.eclipse.jetty.server.handler.ResourceHandler;
 import org.eclipse.jetty.server.Server;
-import org.eclipse.jetty.server.handler.HandlerList;
 import org.eclipse.jetty.servlet.ServletContextHandler;
 import org.eclipse.jetty.servlet.ServletHolder;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import scala.concurrent.Await;
+import scala.concurrent.Future;
 import scala.concurrent.duration.FiniteDuration;
 
 /**
  * This class sets up a web-server that contains a web frontend to display information about running jobs.
  * It instantiates and configures an embedded jetty server.
  */
-public class WebInfoServer implements WebMonitor {
+public class WebInfoServer implements WebMonitor, LeaderRetrievalListener {
 
 	/** Web root dir in the jar */
 	private static final String WEB_ROOT_DIR = "web-docs-infoserver";
@@ -56,6 +67,30 @@ public class WebInfoServer implements WebMonitor {
 	/** The jetty server serving all requests. */
 	private final Server server;
 
+	/** Retrieval service for the current leading JobManager */
+	private final LeaderRetrievalService leaderRetrievalService;
+
+	/** ActorSystem used to retrieve the ActorRefs */
+	private final ActorSystem actorSystem;
+
+	/** Collection for the registered jetty handlers */
+	private final HandlerCollection handlers;
+
+	/** Associated configuration */
+	private final Configuration config;
+
+	/** Timeout for the servlets */
+	private final FiniteDuration timeout;
+
+	/** Actor look up timeout */
+	private final FiniteDuration lookupTimeout;
+
+	/** Default jetty handler responsible for serving static content */
+	private final ResourceHandler resourceHandler;
+
+	/** File paths to log dirs */
+	final File[] logDirFiles;
+
 	/** The assigned port where jetty is running. */
 	private int assignedPort = -1;
 
@@ -64,19 +99,23 @@ public class WebInfoServer implements WebMonitor {
 	 * to list all present information concerning the job manager
 	 *
 	 * @param config The Flink configuration.
-	 * @param jobmanager The ActorRef to the JobManager actor
-	 * @param archive The ActorRef to the archive for old jobs
+	 * @param leaderRetrievalService Retrieval service to obtain the current leader
 	 *
 	 * @throws IOException
 	 *         Thrown, if the server setup failed for an I/O related reason.
 	 */
-	public WebInfoServer(Configuration config, ActorGateway jobmanager, ActorGateway archive) throws IOException {
+	public WebInfoServer(
+			Configuration config,
+			LeaderRetrievalService leaderRetrievalService,
+			ActorSystem actorSystem)
+		throws IOException {
 		if (config == null) {
 			throw new IllegalArgumentException("No Configuration has been passed to the web server");
 		}
-		if (jobmanager == null || archive == null) {
-			throw new NullPointerException();
-		}
+
+		this.config = config;
+
+		this.leaderRetrievalService = Preconditions.checkNotNull(leaderRetrievalService);
 
 		// if port == 0, jetty will assign an available port.
 		int port = config.getInteger(ConfigConstants.JOB_MANAGER_WEB_PORT_KEY,
@@ -85,7 +124,10 @@ public class WebInfoServer implements WebMonitor {
 			throw new IllegalArgumentException("Invalid port for the webserver: " + port);
 		}
 
-		final FiniteDuration timeout = AkkaUtils.getTimeout(config);
+		timeout = AkkaUtils.getTimeout(config);
+		lookupTimeout = AkkaUtils.getLookupTimeout(config);
+
+		this.actorSystem = actorSystem;
 
 		// get base path of Flink installation
 		final String basePath = config.getString(ConfigConstants.FLINK_BASE_DIR_PATH_KEY, "");
@@ -99,7 +141,7 @@ public class WebInfoServer implements WebMonitor {
 					"resource " + WEB_ROOT_DIR + " is not included in the jar.");
 		}
 
-		final File[] logDirFiles = new File[logDirPaths.length];
+		logDirFiles = new File[logDirPaths.length];
 		int i = 0;
 		for(String path : logDirPaths) {
 			logDirFiles[i++] = new File(path);
@@ -113,24 +155,16 @@ public class WebInfoServer implements WebMonitor {
 
 		server = new Server(port);
 
-		// ----- the handlers for the servlets -----
-		ServletContextHandler servletContext = new ServletContextHandler(ServletContextHandler.SESSIONS);
-		servletContext.setContextPath("/");
-		servletContext.addServlet(new ServletHolder(new JobManagerInfoServlet(jobmanager, archive, timeout)), "/jobsInfo");
-		servletContext.addServlet(new ServletHolder(new LogfileInfoServlet(logDirFiles)), "/logInfo");
-		servletContext.addServlet(new ServletHolder(new SetupInfoServlet(config, jobmanager, timeout)), "/setupInfo");
-		servletContext.addServlet(new ServletHolder(new MenuServlet()), "/menu");
-
-
 		// ----- the handler serving all the static files -----
-		ResourceHandler resourceHandler = new ResourceHandler();
+		resourceHandler = new ResourceHandler();
 		resourceHandler.setDirectoriesListed(false);
 		resourceHandler.setResourceBase(webRootDir.toExternalForm());
 
 		// ----- add the handlers to the list handler -----
-		HandlerList handlers = new HandlerList();
+
+		// make the HandlerCollection mutable so that we can update it later on
+		handlers = new HandlerCollection(true);
 		handlers.addHandler(resourceHandler);
-		handlers.addHandler(servletContext);
 		server.setHandler(handlers);
 	}
 
@@ -158,12 +192,15 @@ public class WebInfoServer implements WebMonitor {
 		else {
 			LOG.warn("Unable to determine local endpoint of web frontend server");
 		}
+
+		leaderRetrievalService.start(this);
 	}
 
 	/**
 	 * Stop the webserver
 	 */
 	public void stop() throws Exception {
+		leaderRetrievalService.stop();
 		server.stop();
 		assignedPort = -1;
 	}
@@ -171,4 +208,86 @@ public class WebInfoServer implements WebMonitor {
 	public int getServerPort() {
 		return this.assignedPort;
 	}
+
+	@Override
+	public void notifyLeaderAddress(String leaderAddress, UUID leaderSessionID) {
+
+		if(leaderAddress != null && !leaderAddress.equals("")) {
+			try {
+				ActorRef jobManager = AkkaUtils.getActorRef(
+					leaderAddress,
+					actorSystem,
+					lookupTimeout);
+				ActorGateway jobManagerGateway = new AkkaActorGateway(jobManager, leaderSessionID);
+
+				Future<Object> archiveFuture = jobManagerGateway.ask(
+					JobManagerMessages.getRequestArchive(),
+					timeout);
+
+				ActorRef archive = ((JobManagerMessages.ResponseArchive) Await.result(
+					archiveFuture,
+					timeout)).actor();
+
+				ActorGateway archiveGateway = new AkkaActorGateway(archive, leaderSessionID);
+
+				updateHandler(jobManagerGateway, archiveGateway);
+			} catch (Exception e) {
+				handleError(e);
+			}
+		}
+	}
+
+	@Override
+	public void handleError(Exception exception) {
+		LOG.error("Received error from LeaderRetrievalService.", exception);
+
+		try{
+			// stop the whole web server
+			stop();
+		} catch (Exception e) {
+			LOG.error("Error while stopping the web server due to a LeaderRetrievalService error.", e);
+		}
+	}
+
+	/**
+	 * Updates the Flink handlers with the current leading JobManager and archive
+	 *
+	 * @param jobManager ActorGateway to the current JobManager leader
+	 * @param archive ActorGateway to the current archive of the leading JobManager
+	 * @throws Exception
+	 */
+	private void updateHandler(ActorGateway jobManager, ActorGateway archive) throws Exception {
+		// ----- the handlers for the servlets -----
+		ServletContextHandler servletContext = new ServletContextHandler(ServletContextHandler.SESSIONS);
+		servletContext.setContextPath("/");
+		servletContext.addServlet(
+				new ServletHolder(
+						new JobManagerInfoServlet(
+								jobManager,
+								archive,
+								timeout)),
+				"/jobsInfo");
+		servletContext.addServlet(
+				new ServletHolder(
+						new LogfileInfoServlet(
+								logDirFiles)),
+				"/logInfo");
+		servletContext.addServlet(
+				new ServletHolder(
+						new SetupInfoServlet(
+								config,
+								jobManager,
+								timeout)),
+				"/setupInfo");
+		servletContext.addServlet(
+				new ServletHolder(
+						new MenuServlet()),
+				"/menu");
+
+		// replace old handlers with new ones
+		handlers.setHandlers(new Handler[]{resourceHandler, servletContext});
+
+		// start new handler
+		servletContext.start();
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/LeaderContender.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/LeaderContender.java b/flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/LeaderContender.java
new file mode 100644
index 0000000..dcf0a4e
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/LeaderContender.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.leaderelection;
+
+import java.util.UUID;
+
+/**
+ * Interface which has to be implemented to take part in the leader election process of the
+ * {@link LeaderElectionService}.
+ */
+public interface LeaderContender {
+
+	/**
+	 * Callback method which is called by the {@link LeaderElectionService} upon selecting this
+	 * instance as the new leader. The method is called with the new leader session ID.
+	 *
+	 * @param leaderSessionID New leader session ID
+	 */
+	void grantLeadership(UUID leaderSessionID);
+
+	/**
+	 * Callback method which is called by the {@link LeaderElectionService} upon revoking the
+	 * leadership of a former leader. This might happen in case that multiple contenders have
+	 * been granted leadership.
+	 */
+	void revokeLeadership();
+
+	/**
+	 * Returns the address of the {@link LeaderContender} under which other instances can connect
+	 * to it.
+	 *
+	 * @return Address of this contender.
+	 */
+	String getAddress();
+
+	/**
+	 * Callback method which is called by {@link LeaderElectionService} in case of an error in the
+	 * service thread.
+	 *
+	 * @param exception Caught exception
+	 */
+	void handleError(Exception exception);
+}


[05/10] flink git commit: [FLINK-2291] [runtime] Add ZooKeeper support to elect a leader from a set of JobManager. The leader will then be retrieved from ZooKeeper by the TaskManagers.

Posted by tr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderElectionRetrievalTestingCluster.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderElectionRetrievalTestingCluster.java b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderElectionRetrievalTestingCluster.java
new file mode 100644
index 0000000..ff73615
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderElectionRetrievalTestingCluster.java
@@ -0,0 +1,121 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.leaderelection;
+
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.StreamingMode;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+import org.apache.flink.runtime.testingUtils.TestingCluster;
+
+import java.util.UUID;
+
+/**
+ * A testing cluster which allows to manually trigger grantLeadership and notifyRetrievalListener
+ * events. The grantLeadership event assigns the specified JobManager the leadership. The
+ * notifyRetrievalListener notifies all listeners that the specified JobManager (index) has been
+ * granted the leadership.
+ */
+public class LeaderElectionRetrievalTestingCluster extends TestingCluster {
+
+	private final Configuration userConfiguration;
+	private final boolean useSingleActorSystem;
+	private final StreamingMode streamingMode;
+
+	public TestingLeaderElectionService[] leaderElectionServices;
+	public TestingLeaderRetrievalService[] leaderRetrievalServices;
+
+	private int leaderElectionServiceCounter = 0;
+	private int leaderRetrievalServiceCounter = 0;
+
+	private int leaderIndex = -1;
+
+	public LeaderElectionRetrievalTestingCluster(
+			Configuration userConfiguration,
+			boolean singleActorSystem,
+			boolean synchronousDispatcher,
+			StreamingMode streamingMode) {
+		super(userConfiguration, singleActorSystem, synchronousDispatcher, streamingMode);
+
+		this.userConfiguration = userConfiguration;
+		this.useSingleActorSystem = singleActorSystem;
+		this.streamingMode = streamingMode;
+
+		leaderElectionServices = new TestingLeaderElectionService[this.numJobManagers()];
+		leaderRetrievalServices = new TestingLeaderRetrievalService[this.numTaskManagers() + 1];
+	}
+
+	@Override
+	public Configuration userConfiguration() {
+		return this.userConfiguration;
+	}
+
+	@Override
+	public StreamingMode streamingMode() {
+		return streamingMode;
+	}
+
+	@Override
+	public boolean useSingleActorSystem() {
+		return useSingleActorSystem;
+	}
+
+	@Override
+	public LeaderElectionService createLeaderElectionService(LeaderElectionService originalService) {
+		leaderElectionServices[leaderElectionServiceCounter] = new TestingLeaderElectionService();
+
+		LeaderElectionService result = leaderElectionServices[leaderElectionServiceCounter++];
+
+		return result;
+	}
+
+	@Override
+	public LeaderRetrievalService createLeaderRetrievalService() {
+		leaderRetrievalServices[leaderRetrievalServiceCounter] = new TestingLeaderRetrievalService();
+
+		return leaderRetrievalServices[leaderRetrievalServiceCounter++];
+	}
+
+	@Override
+	public int getNumberOfJobManagers() {
+		return this.configuration().getInteger(
+				ConfigConstants.LOCAL_NUMBER_JOB_MANAGER,
+				ConfigConstants.DEFAULT_LOCAL_NUMBER_JOB_MANAGER);
+	}
+
+	public void grantLeadership(int index, UUID leaderSessionID) {
+		if(leaderIndex >= 0) {
+			// first revoke leadership
+			leaderElectionServices[leaderIndex].notLeader();
+		}
+
+		// make the JM with index the new leader
+		leaderElectionServices[index].isLeader(leaderSessionID);
+
+		leaderIndex = index;
+	}
+
+	public void notifyRetrievalListeners(int index, UUID leaderSessionID) {
+		String address = jobManagerActors().get().apply(index).path().toString();
+
+		for(TestingLeaderRetrievalService service: leaderRetrievalServices) {
+			service.notifyListener(address, leaderSessionID);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/StandaloneLeaderElectionTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/StandaloneLeaderElectionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/StandaloneLeaderElectionTest.java
new file mode 100644
index 0000000..86401bc
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/StandaloneLeaderElectionTest.java
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.leaderelection;
+
+import org.apache.flink.runtime.leaderretrieval.StandaloneLeaderRetrievalService;
+import org.apache.flink.util.TestLogger;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class StandaloneLeaderElectionTest extends TestLogger {
+	private static final String TEST_URL = "akka://users/jobmanager";
+
+	/**
+	 * Tests that the standalone leader election and retrieval service return the same leader
+	 * URL.
+	 */
+	@Test
+	public void testStandaloneLeaderElectionRetrieval() throws Exception {
+		StandaloneLeaderElectionService leaderElectionService = new StandaloneLeaderElectionService();
+		StandaloneLeaderRetrievalService leaderRetrievalService = new StandaloneLeaderRetrievalService(TEST_URL);
+		TestingContender contender = new TestingContender(TEST_URL, leaderElectionService);
+		TestingListener testingListener = new TestingListener();
+
+		try {
+			leaderElectionService.start(contender);
+			leaderRetrievalService.start(testingListener);
+
+			contender.waitForLeader(1000l);
+
+			assertTrue(contender.isLeader());
+			assertEquals(null, contender.getLeaderSessionID());
+
+			testingListener.waitForLeader(1000l);
+
+			assertEquals(TEST_URL, testingListener.getAddress());
+			assertEquals(null, testingListener.getLeaderSessionID());
+		} finally {
+			leaderElectionService.stop();
+			leaderRetrievalService.stop();
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingContender.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingContender.java b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingContender.java
new file mode 100644
index 0000000..9b2ab60
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingContender.java
@@ -0,0 +1,146 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.leaderelection;
+
+import java.util.UUID;
+import java.util.concurrent.TimeoutException;
+
+/**
+ * {@link LeaderContender} implementation which provides some convenience functions for testing
+ * purposes.
+ */
+public class TestingContender implements LeaderContender {
+
+	private final String address;
+	private final LeaderElectionService leaderElectionService;
+	private UUID leaderSessionID = null;
+	private boolean leader = false;
+	private Throwable error = null;
+
+	private Object lock = new Object();
+	private Object errorLock = new Object();
+
+	public TestingContender(
+			final String address,
+			final LeaderElectionService leaderElectionService) {
+		this.address = address;
+		this.leaderElectionService = leaderElectionService;
+	}
+
+	/**
+	 * Waits until the contender becomes the leader or until the timeout has been exceeded.
+	 *
+	 * @param timeout
+	 * @throws TimeoutException
+	 */
+	public void waitForLeader(long timeout) throws TimeoutException {
+		long start = System.currentTimeMillis();
+		long curTimeout;
+
+		while (!isLeader() && (curTimeout = timeout - System.currentTimeMillis() + start) > 0) {
+			synchronized (lock) {
+				try {
+					lock.wait(curTimeout);
+				} catch (InterruptedException e) {
+					// we got interrupted so check again for the condition
+				}
+			}
+		}
+
+		if (!isLeader()) {
+			throw new TimeoutException("Contender was not elected as the leader within " +
+					timeout + "ms");
+		}
+	}
+
+	/**
+	 * Waits until an error has been found or until the timeout has been exceeded.
+	 *
+	 * @param timeout
+	 * @throws TimeoutException
+	 */
+	public void waitForError(long timeout) throws TimeoutException {
+		long start = System.currentTimeMillis();
+		long curTimeout;
+
+		while (error == null && (curTimeout = timeout - System.currentTimeMillis() + start) > 0) {
+			synchronized (errorLock) {
+				try {
+					errorLock.wait(curTimeout);
+				} catch (InterruptedException e) {
+					// we got interrupted so check again for the condition
+				}
+			}
+		}
+
+		if (error == null) {
+			throw new TimeoutException("Contender did not see an exception in " +
+					timeout + "ms");
+		}
+	}
+
+	public UUID getLeaderSessionID() {
+		return leaderSessionID;
+	}
+
+	public Throwable getError() {
+		return error;
+	}
+
+	public boolean isLeader() {
+		return leader;
+	}
+
+	@Override
+	public void grantLeadership(UUID leaderSessionID) {
+		synchronized (lock) {
+			this.leaderSessionID = leaderSessionID;
+
+			leaderElectionService.confirmLeaderSessionID(leaderSessionID);
+
+			leader = true;
+
+			lock.notifyAll();
+		}
+	}
+
+	@Override
+	public void revokeLeadership() {
+		synchronized (lock) {
+			leader = false;
+			leaderSessionID = null;
+
+			lock.notifyAll();
+		}
+	}
+
+	@Override
+	public String getAddress() {
+		return address;
+	}
+
+	@Override
+	public void handleError(Exception exception) {
+		synchronized (errorLock) {
+			this.error = exception;
+
+			errorLock.notifyAll();
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingLeaderElectionService.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingLeaderElectionService.java b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingLeaderElectionService.java
new file mode 100644
index 0000000..c4fccd7
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingLeaderElectionService.java
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.leaderelection;
+
+import java.io.Serializable;
+import java.util.UUID;
+
+/**
+ * Test {@link LeaderElectionService} implementation which directly forwards isLeader and notLeader
+ * calls to the contender.
+ */
+public class TestingLeaderElectionService implements LeaderElectionService, Serializable {
+
+	private LeaderContender contender;
+	private boolean hasLeadership = false;
+
+	@Override
+	public void start(LeaderContender contender) throws Exception {
+		this.contender = contender;
+	}
+
+	@Override
+	public void stop() throws Exception {
+
+	}
+
+	@Override
+	public void confirmLeaderSessionID(UUID leaderSessionID) {
+
+	}
+
+	@Override
+	public boolean hasLeadership() {
+		return hasLeadership;
+	}
+
+	public void isLeader(UUID leaderSessionID) {
+		contender.grantLeadership(leaderSessionID);
+	}
+
+	public void notLeader() {
+		contender.revokeLeadership();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingLeaderRetrievalService.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingLeaderRetrievalService.java b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingLeaderRetrievalService.java
new file mode 100644
index 0000000..43902fd
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingLeaderRetrievalService.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.leaderelection;
+
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+
+import java.util.UUID;
+
+/**
+ * Test {@link LeaderRetrievalService} implementation which directly forwards calls of
+ * notifyListener to the listener.
+ */
+public class TestingLeaderRetrievalService implements LeaderRetrievalService {
+
+	private LeaderRetrievalListener listener;
+
+	@Override
+	public void start(LeaderRetrievalListener listener) throws Exception {
+		this.listener = listener;
+	}
+
+	@Override
+	public void stop() throws Exception {
+
+	}
+
+	public void notifyListener(String address, UUID leaderSessionID) {
+		listener.notifyLeaderAddress(address, leaderSessionID);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingListener.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingListener.java b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingListener.java
new file mode 100644
index 0000000..4372414
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingListener.java
@@ -0,0 +1,115 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.leaderelection;
+
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener;
+
+import java.util.UUID;
+import java.util.concurrent.TimeoutException;
+
+/**
+ * Test {@link LeaderRetrievalListener} implementation which offers some convenience functions for
+ * testing purposes.
+ */
+public class TestingListener implements LeaderRetrievalListener {
+
+	private String address;
+	private String oldAddress;
+	private UUID leaderSessionID;
+	private Exception exception;
+
+	private Object lock = new Object();
+
+	public String getAddress() {
+		return address;
+	}
+
+	public UUID getLeaderSessionID() {
+		return leaderSessionID;
+	}
+
+	public void clear() {
+		address = null;
+		leaderSessionID = null;
+	}
+
+	public void waitForLeader(long timeout) throws Exception {
+		long start = System.currentTimeMillis();
+		long curTimeout;
+
+		while (exception == null && address == null && (curTimeout = timeout - System.currentTimeMillis() + start) > 0) {
+			synchronized (lock) {
+				try {
+					lock.wait(curTimeout);
+				} catch (InterruptedException e) {
+					// we got interrupted so check again for the condition
+				}
+			}
+		}
+
+		if (exception != null) {
+			throw exception;
+		} else if (address == null) {
+			throw new TimeoutException("Listener was not notified about a leader within " +
+					timeout + "ms");
+		}
+	}
+
+	public void waitForNewLeader(long timeout) throws Exception {
+		long start = System.currentTimeMillis();
+		long curTimeout;
+
+		while (
+				exception == null &&
+				(address == null || address.equals(oldAddress)) &&
+				(curTimeout = timeout - System.currentTimeMillis() + start) > 0) {
+			synchronized (lock) {
+				try {
+					lock.wait(curTimeout);
+				} catch (InterruptedException e) {
+					// we got interrupted so check again for the condition
+				}
+			}
+		}
+
+		if (exception != null) {
+			throw exception;
+		} else if (address == null || address.equals(oldAddress)) {
+			throw new TimeoutException("Listener was not notified about a leader within " +
+					timeout + "ms");
+		}
+
+		oldAddress = address;
+	}
+
+	@Override
+	public void notifyLeaderAddress(String leaderAddress, UUID leaderSessionID) {
+		synchronized (lock) {
+			this.address = leaderAddress;
+			this.leaderSessionID = leaderSessionID;
+
+			lock.notifyAll();
+		}
+	}
+
+	@Override
+	public void handleError(Exception exception) {
+		this.exception = exception;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderElectionTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderElectionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderElectionTest.java
new file mode 100644
index 0000000..de1f370
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderElectionTest.java
@@ -0,0 +1,550 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.leaderelection;
+
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.api.CreateBuilder;
+import org.apache.curator.framework.api.ProtectACLCreateModePathAndBytesable;
+import org.apache.curator.framework.recipes.cache.ChildData;
+import org.apache.curator.framework.recipes.cache.NodeCache;
+import org.apache.curator.framework.recipes.cache.NodeCacheListener;
+import org.apache.curator.test.TestingCluster;
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalService;
+import org.apache.flink.runtime.util.ZooKeeperUtils;
+import org.apache.flink.util.TestLogger;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Matchers;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+import scala.concurrent.Await;
+import scala.concurrent.Future;
+import scala.concurrent.Promise;
+import scala.concurrent.duration.FiniteDuration;
+
+import java.io.ByteArrayOutputStream;
+import java.io.ObjectOutputStream;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import static org.mockito.Mockito.*;
+import static org.junit.Assert.*;
+
+public class ZooKeeperLeaderElectionTest extends TestLogger {
+	private TestingCluster testingCluster;
+	private static final String TEST_URL = "akka//user/jobmanager";
+	private static final FiniteDuration timeout = new FiniteDuration(20000, TimeUnit.MILLISECONDS);
+
+	@Before
+	public void before() {
+		testingCluster = new TestingCluster(3);
+
+		try {
+			testingCluster.start();
+		} catch (Exception e) {
+			throw new RuntimeException("Could not start ZooKeeper testing cluster.", e);
+		}
+	}
+
+	@After
+	public void after() {
+		try {
+			testingCluster.stop();
+		} catch (Exception e) {
+			throw new RuntimeException("Could not stop ZooKeeper testing cluster.", e);
+		}
+
+		testingCluster = null;
+	}
+
+	/**
+	 * Tests that the ZooKeeperLeaderElection/RetrievalService return both the correct URL.
+	 */
+	@Test
+	public void testZooKeeperLeaderElectionRetrieval() throws Exception {
+		Configuration configuration = new Configuration();
+		configuration.setString(ConfigConstants.ZOOKEEPER_QUORUM_KEY, testingCluster.getConnectString());
+		configuration.setString(ConfigConstants.RECOVERY_MODE, "zookeeper");
+
+		ZooKeeperLeaderElectionService leaderElectionService = null;
+		ZooKeeperLeaderRetrievalService leaderRetrievalService = null;
+
+		try {
+			leaderElectionService = ZooKeeperUtils.createLeaderElectionService(configuration);
+			leaderRetrievalService = ZooKeeperUtils.createLeaderRetrievalService(configuration);
+
+			TestingContender contender = new TestingContender(TEST_URL, leaderElectionService);
+			TestingListener listener = new TestingListener();
+
+			leaderElectionService.start(contender);
+			leaderRetrievalService.start(listener);
+
+			contender.waitForLeader(timeout.toMillis());
+
+			assertTrue(contender.isLeader());
+			assertEquals(leaderElectionService.getLeaderSessionID(), contender.getLeaderSessionID());
+
+			listener.waitForLeader(timeout.toMillis());
+
+			assertEquals(TEST_URL, listener.getAddress());
+			assertEquals(leaderElectionService.getLeaderSessionID(), listener.getLeaderSessionID());
+
+		} finally {
+			if (leaderElectionService != null) {
+				leaderElectionService.stop();
+			}
+
+			if (leaderRetrievalService != null) {
+				leaderRetrievalService.stop();
+			}
+		}
+	}
+
+	/**
+	 * Tests repeatedly the reelection of still available LeaderContender. After a contender has
+	 * been elected as the leader, it is removed. This forces the ZooKeeperLeaderElectionService
+	 * to elect a new leader.
+	 */
+	@Test
+	public void testZooKeeperReelection() throws Exception {
+		Configuration configuration = new Configuration();
+		configuration.setString(ConfigConstants.ZOOKEEPER_QUORUM_KEY, testingCluster.getConnectString());
+		configuration.setString(ConfigConstants.RECOVERY_MODE, "zookeeper");
+
+		int num = 100;
+
+		ZooKeeperLeaderElectionService[] leaderElectionService = new ZooKeeperLeaderElectionService[num];
+		TestingContender[] contenders = new TestingContender[num];
+		ZooKeeperLeaderRetrievalService leaderRetrievalService = null;
+
+		TestingListener listener = new TestingListener();
+
+		try {
+			leaderRetrievalService = ZooKeeperUtils.createLeaderRetrievalService(configuration);
+
+			leaderRetrievalService.start(listener);
+
+			for (int i = 0; i < num; i++) {
+				leaderElectionService[i] = ZooKeeperUtils.createLeaderElectionService(configuration);
+				contenders[i] = new TestingContender(TEST_URL + "_" + i, leaderElectionService[i]);
+
+				leaderElectionService[i].start(contenders[i]);
+			}
+
+			String pattern = TEST_URL + "_" + "(\\d+)";
+			Pattern regex = Pattern.compile(pattern);
+
+			for (int i = 0; i < num; i++) {
+				listener.waitForNewLeader(timeout.toMillis());
+
+				String address = listener.getAddress();
+
+				Matcher m = regex.matcher(address);
+
+				if (m.find()) {
+					int index = Integer.parseInt(m.group(1));
+
+					// check that the leader session ID of the listeners and the leader are equal
+					assertEquals(listener.getLeaderSessionID(), contenders[index].getLeaderSessionID());
+					assertEquals(TEST_URL + "_" + index, listener.getAddress());
+
+					// kill the election service of the leader
+					leaderElectionService[index].stop();
+					leaderElectionService[index] = null;
+				} else {
+					fail("Did not find the leader's index.");
+				}
+			}
+
+		} finally {
+			if (leaderRetrievalService != null) {
+				leaderRetrievalService.stop();
+			}
+
+			for (ZooKeeperLeaderElectionService electionService : leaderElectionService) {
+				if (electionService != null) {
+					electionService.stop();
+				}
+			}
+		}
+	}
+
+	/**
+	 * Tests the repeated reelection of {@link LeaderContender} once the current leader dies.
+	 * Furthermore, it tests that new LeaderElectionServices can be started later on and that they
+	 * successfully register at ZooKeeper and take part in the leader election.
+	 */
+	@Test
+	public void testZooKeeperReelectionWithReplacement() throws Exception {
+		Configuration configuration = new Configuration();
+		configuration.setString(ConfigConstants.ZOOKEEPER_QUORUM_KEY, testingCluster.getConnectString());
+		configuration.setString(ConfigConstants.RECOVERY_MODE, "zookeeper");
+
+		int num = 3;
+		int numTries = 30;
+
+		ZooKeeperLeaderElectionService[] leaderElectionService = new ZooKeeperLeaderElectionService[num];
+		TestingContender[] contenders = new TestingContender[num];
+		ZooKeeperLeaderRetrievalService leaderRetrievalService = null;
+
+		TestingListener listener = new TestingListener();
+
+		try {
+			leaderRetrievalService = ZooKeeperUtils.createLeaderRetrievalService(configuration);
+
+			leaderRetrievalService.start(listener);
+
+			for (int i = 0; i < num; i++) {
+				leaderElectionService[i] = ZooKeeperUtils.createLeaderElectionService(configuration);
+				contenders[i] = new TestingContender(TEST_URL + "_" + i + "_0", leaderElectionService[i]);
+
+				leaderElectionService[i].start(contenders[i]);
+			}
+
+			String pattern = TEST_URL + "_" + "(\\d+)" + "_" + "(\\d+)";
+			Pattern regex = Pattern.compile(pattern);
+
+			for (int i = 0; i < numTries; i++) {
+				listener.waitForLeader(timeout.toMillis());
+
+				String address = listener.getAddress();
+
+				Matcher m = regex.matcher(address);
+
+				if (m.find()) {
+					int index = Integer.parseInt(m.group(1));
+					int lastTry = Integer.parseInt(m.group(2));
+
+					assertEquals(listener.getLeaderSessionID(), contenders[index].getLeaderSessionID());
+
+					// clear the current leader of the listener
+					listener.clear();
+
+					// stop leader election service = revoke leadership
+					leaderElectionService[index].stop();
+					// create new leader election service which takes part in the leader election
+					leaderElectionService[index] = ZooKeeperUtils.createLeaderElectionService(configuration);
+					contenders[index] = new TestingContender(
+							TEST_URL + "_" + index + "_" + (lastTry + 1),
+							leaderElectionService[index]);
+
+					leaderElectionService[index].start(contenders[index]);
+				} else {
+					throw new Exception("Did not find the leader's index.");
+				}
+			}
+
+		} finally {
+			if (leaderRetrievalService != null) {
+				leaderRetrievalService.stop();
+			}
+
+			for (ZooKeeperLeaderElectionService electionService : leaderElectionService) {
+				if (electionService != null) {
+					electionService.stop();
+				}
+			}
+		}
+	}
+
+	/**
+	 * Tests that the current leader is notified when his leader connection information in ZooKeeper
+	 * are overwritten. The leader must re-establish the correct leader connection information in
+	 * ZooKeeper.
+	 */
+	@Test
+	public void testMultipleLeaders() throws Exception {
+		final String FAULTY_CONTENDER_URL = "faultyContender";
+		final String leaderPath = "/leader";
+
+		Configuration configuration = new Configuration();
+		configuration.setString(ConfigConstants.ZOOKEEPER_QUORUM_KEY, testingCluster.getConnectString());
+		configuration.setString(ConfigConstants.RECOVERY_MODE, "zookeeper");
+		configuration.setString(ConfigConstants.ZOOKEEPER_LEADER_PATH, leaderPath);
+
+		ZooKeeperLeaderElectionService leaderElectionService = null;
+		ZooKeeperLeaderRetrievalService leaderRetrievalService = null;
+		TestingListener listener = new TestingListener();
+		TestingContender contender;
+
+		try {
+			leaderElectionService = ZooKeeperUtils.createLeaderElectionService(configuration);
+			leaderRetrievalService = ZooKeeperUtils.createLeaderRetrievalService(configuration);
+
+			contender = new TestingContender(TEST_URL, leaderElectionService);
+
+			leaderElectionService.start(contender);
+			leaderRetrievalService.start(listener);
+
+			listener.waitForLeader(timeout.toMillis());
+
+			assertEquals(listener.getLeaderSessionID(), contender.getLeaderSessionID());
+			assertEquals(TEST_URL, listener.getAddress());
+
+			listener.clear();
+
+			CuratorFramework client = ZooKeeperUtils.startCuratorFramework(configuration);
+
+			ByteArrayOutputStream baos = new ByteArrayOutputStream();
+			ObjectOutputStream oos = new ObjectOutputStream(baos);
+
+			oos.writeUTF(FAULTY_CONTENDER_URL);
+			oos.writeObject(null);
+
+			oos.close();
+
+			// overwrite the current leader address, the leader should notice that and correct it
+			boolean dataWritten = false;
+
+			while(!dataWritten) {
+				client.delete().forPath(leaderPath);
+
+				try {
+					client.create().forPath(leaderPath, baos.toByteArray());
+
+					dataWritten = true;
+				} catch (KeeperException.NodeExistsException e) {
+					// this can happen if the leader election service was faster
+				}
+			}
+
+			listener.waitForLeader(timeout.toMillis());
+
+			if (FAULTY_CONTENDER_URL.equals(listener.getAddress())) {
+				listener.clear();
+				listener.waitForLeader(timeout.toMillis());
+			}
+
+			assertEquals(listener.getLeaderSessionID(), contender.getLeaderSessionID());
+			assertEquals(listener.getAddress(), contender.getAddress());
+
+		} finally {
+			if (leaderElectionService != null) {
+				leaderElectionService.stop();
+			}
+
+			if (leaderRetrievalService != null) {
+				leaderRetrievalService.stop();
+			}
+		}
+	}
+
+	/**
+	 *  Test that errors in the {@link LeaderElectionService} are correctly forwarded to the
+	 *  {@link LeaderContender}.
+	 */
+	@Test
+	public void testExceptionForwarding() throws Exception {
+		Configuration configuration = new Configuration();
+		configuration.setString(ConfigConstants.ZOOKEEPER_QUORUM_KEY, testingCluster.getConnectString());
+		configuration.setString(ConfigConstants.RECOVERY_MODE, "zookeeper");
+
+		ZooKeeperLeaderElectionService leaderElectionService = null;
+		ZooKeeperLeaderRetrievalService leaderRetrievalService = null;
+		TestingListener listener = new TestingListener();
+		TestingContender testingContender;
+
+		CuratorFramework client;
+		final CreateBuilder mockCreateBuilder = mock(CreateBuilder.class);
+		final ProtectACLCreateModePathAndBytesable<String> mockCreateParentsIfNeeded = mock (ProtectACLCreateModePathAndBytesable.class);
+		final Exception testException = new Exception("Test exception");
+
+		try {
+			client = spy(ZooKeeperUtils.startCuratorFramework(configuration));
+
+			Answer<CreateBuilder> answer = new Answer<CreateBuilder>() {
+				private int counter = 0;
+
+				@Override
+				public CreateBuilder answer(InvocationOnMock invocation) throws Throwable {
+					counter++;
+
+					// at first we have to create the leader latch, there it mustn't fail yet
+					if (counter < 2) {
+						return (CreateBuilder) invocation.callRealMethod();
+					} else {
+						return mockCreateBuilder;
+					}
+				}
+			};
+
+			doAnswer(answer).when(client).create();
+
+			when(mockCreateBuilder.creatingParentsIfNeeded()).thenReturn(mockCreateParentsIfNeeded);
+			when(mockCreateParentsIfNeeded.withMode(Matchers.any(CreateMode.class))).thenReturn(mockCreateParentsIfNeeded);
+			when(mockCreateParentsIfNeeded.forPath(Matchers.any(String.class),  Matchers.any(byte[].class))).thenThrow(testException);
+
+			leaderElectionService = new ZooKeeperLeaderElectionService(client, "/latch", "/leader");
+			leaderRetrievalService = ZooKeeperUtils.createLeaderRetrievalService(configuration);
+
+			testingContender = new TestingContender(TEST_URL, leaderElectionService);
+
+			leaderElectionService.start(testingContender);
+			leaderRetrievalService.start(listener);
+
+			testingContender.waitForError(timeout.toMillis());
+
+			assertNotNull(testingContender.getError());
+			assertEquals(testException, testingContender.getError().getCause());
+		} finally {
+			if (leaderElectionService != null) {
+				leaderElectionService.stop();
+			}
+
+			if (leaderRetrievalService != null) {
+				leaderRetrievalService.stop();
+			}
+		}
+	}
+
+	/**
+	 * Tests that there is no information left in the ZooKeeper cluster after all JobManagers
+	 * have terminated. In other words, checks that the ZooKeeperLeaderElection service uses
+	 * ephemeral nodes.
+	 */
+	@Test
+	public void testEphemeralZooKeeperNodes() throws Exception {
+		Configuration configuration = new Configuration();
+		configuration.setString(ConfigConstants.ZOOKEEPER_QUORUM_KEY, testingCluster.getConnectString());
+		configuration.setString(ConfigConstants.RECOVERY_MODE, "zookeeper");
+
+		ZooKeeperLeaderElectionService leaderElectionService;
+		ZooKeeperLeaderRetrievalService leaderRetrievalService = null;
+		TestingContender testingContender;
+		TestingListener listener;
+
+		CuratorFramework client = null;
+		NodeCache cache = null;
+
+		try {
+			leaderElectionService = ZooKeeperUtils.createLeaderElectionService(configuration);
+			leaderRetrievalService = ZooKeeperUtils.createLeaderRetrievalService(configuration);
+			testingContender = new TestingContender(TEST_URL, leaderElectionService);
+			listener = new TestingListener();
+
+			client = ZooKeeperUtils.startCuratorFramework(configuration);
+			final String leaderPath = configuration.getString(ConfigConstants.ZOOKEEPER_LEADER_PATH,
+					ConfigConstants.DEFAULT_ZOOKEEPER_LEADER_PATH);
+			cache = new NodeCache(client, leaderPath);
+
+			ExistsCacheListener existsListener = new ExistsCacheListener(cache);
+			DeletedCacheListener deletedCacheListener = new DeletedCacheListener(cache);
+
+			cache.getListenable().addListener(existsListener);
+			cache.start();
+
+			leaderElectionService.start(testingContender);
+
+			testingContender.waitForLeader(timeout.toMillis());
+
+			Future<Boolean> existsFuture = existsListener.nodeExists();
+
+			Await.result(existsFuture, timeout);
+
+			cache.getListenable().addListener(deletedCacheListener);
+
+			leaderElectionService.stop();
+
+			Future<Boolean> deletedFuture = deletedCacheListener.nodeDeleted();
+
+			// make sure that the leader node has been deleted
+			Await.result(deletedFuture, timeout);
+
+			leaderRetrievalService.start(listener);
+
+			try {
+				listener.waitForLeader(1000);
+
+				fail("TimeoutException was expected because there is no leader registered and " +
+						"thus there shouldn't be any leader information in ZooKeeper.");
+			} catch (TimeoutException e) {
+				//that was expected
+			}
+		} finally {
+			if(leaderRetrievalService != null) {
+				leaderRetrievalService.stop();
+			}
+
+			if (cache != null) {
+				cache.close();
+			}
+
+			if (client != null) {
+				client.close();
+			}
+		}
+	}
+
+	public static class ExistsCacheListener implements NodeCacheListener {
+
+		final Promise<Boolean> existsPromise = new scala.concurrent.impl.Promise.DefaultPromise<>();
+
+		final NodeCache cache;
+
+		public ExistsCacheListener(final NodeCache cache) {
+			this.cache = cache;
+		}
+
+		public Future<Boolean> nodeExists() {
+			return existsPromise.future();
+		}
+
+		@Override
+		public void nodeChanged() throws Exception {
+			ChildData data = cache.getCurrentData();
+
+			if (data != null && !existsPromise.isCompleted()) {
+				existsPromise.success(true);
+				cache.getListenable().removeListener(this);
+			}
+		}
+	}
+
+	public static class DeletedCacheListener implements NodeCacheListener {
+
+		final Promise<Boolean> deletedPromise = new scala.concurrent.impl.Promise.DefaultPromise<>();
+
+		final NodeCache cache;
+
+		public DeletedCacheListener(final NodeCache cache) {
+			this.cache = cache;
+		}
+
+		public Future<Boolean> nodeDeleted() {
+			return deletedPromise.future();
+		}
+
+		@Override
+		public void nodeChanged() throws Exception {
+			ChildData data = cache.getCurrentData();
+
+			if (data == null && !deletedPromise.isCompleted()) {
+				deletedPromise.success(true);
+				cache.getListenable().removeListener(this);
+			}
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderRetrievalTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderRetrievalTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderRetrievalTest.java
new file mode 100644
index 0000000..87161c6
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderRetrievalTest.java
@@ -0,0 +1,212 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.leaderelection;
+
+import org.apache.curator.test.TestingCluster;
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.jobmanager.JobManager;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+import org.apache.flink.runtime.util.LeaderElectionUtils;
+import org.apache.flink.runtime.util.LeaderRetrievalUtils;
+import org.apache.flink.util.TestLogger;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import scala.Option;
+import scala.concurrent.duration.FiniteDuration;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.ServerSocket;
+import java.net.Socket;
+import java.net.SocketAddress;
+import java.net.UnknownHostException;
+import java.util.concurrent.TimeUnit;
+
+import static org.junit.Assert.*;
+
+public class ZooKeeperLeaderRetrievalTest extends TestLogger{
+
+	private TestingCluster testingCluster;
+
+	@Before
+	public void before() {
+		testingCluster = new TestingCluster(3);
+		try {
+			testingCluster.start();
+		} catch (Exception e) {
+			throw new RuntimeException("Could not start ZooKeeper testing cluster.", e);
+		}
+	}
+
+	@After
+	public void after() {
+		if(testingCluster != null) {
+			try {
+				testingCluster.stop();
+			} catch (IOException e) {
+				throw new RuntimeException("Could not stop ZooKeeper testing cluster.", e);
+			}
+			testingCluster = null;
+		}
+	}
+
+	/**
+	 * Tests that LeaderRetrievalUtils.findConnectingAdress finds the correct connecting address
+	 * in case of an old leader address in ZooKeeper and a subsequent election of a new leader.
+	 * The findConnectingAddress should block until the new leader has been elected and his
+	 * address has been written to ZooKeeper.
+	 */
+	@Test
+	public void testConnectingAddressRetrievalWithDelayedLeaderElection() throws Exception {
+		FiniteDuration timeout = new FiniteDuration(1, TimeUnit.MINUTES);
+		Configuration config = new Configuration();
+
+		long sleepingTime = 1000;
+
+		config.setString(ConfigConstants.RECOVERY_MODE, "zookeeper");
+		config.setString(ConfigConstants.ZOOKEEPER_QUORUM_KEY, testingCluster.getConnectString());
+
+		LeaderElectionService leaderElectionService = null;
+		LeaderElectionService faultyLeaderElectionService;
+
+		ServerSocket serverSocket;
+		InetAddress localHost;
+
+		Thread thread;
+
+		try {
+
+			InetSocketAddress wrongInetSocketAddress = new InetSocketAddress(InetAddress.getByName("1.1.1.1"), 1234);
+
+			String wrongAddress = JobManager.getRemoteJobManagerAkkaURL(wrongInetSocketAddress, Option.<String>empty());
+
+			try {
+				localHost = InetAddress.getLocalHost();
+				serverSocket = new ServerSocket(0, 50, localHost);
+			} catch (UnknownHostException e) {
+				// may happen if disconnected. skip test.
+				System.err.println("Skipping 'testNetworkInterfaceSelection' test.");
+				return;
+			}
+			catch (IOException e) {
+				// may happen in certain test setups, skip test.
+				System.err.println("Skipping 'testNetworkInterfaceSelection' test.");
+				return;
+			}
+
+			InetSocketAddress correctInetSocketAddress = new InetSocketAddress(localHost, serverSocket.getLocalPort());
+
+			String correctAddress = JobManager.getRemoteJobManagerAkkaURL(correctInetSocketAddress, Option.<String>empty());
+
+			faultyLeaderElectionService = LeaderElectionUtils.createLeaderElectionService(config);
+			TestingContender wrongLeaderAddressContender = new TestingContender(wrongAddress, faultyLeaderElectionService);
+
+			faultyLeaderElectionService.start(wrongLeaderAddressContender);
+
+			FindConnectingAddress findConnectingAddress = new FindConnectingAddress(config, timeout);
+
+			thread = new Thread(findConnectingAddress);
+
+			thread.start();
+
+			leaderElectionService = LeaderElectionUtils.createLeaderElectionService(config);
+			TestingContender correctLeaderAddressContender = new TestingContender(correctAddress, leaderElectionService);
+
+			Thread.sleep(sleepingTime);
+
+			faultyLeaderElectionService.stop();
+
+			leaderElectionService.start(correctLeaderAddressContender);
+
+			thread.join();
+
+			InetAddress result = findConnectingAddress.getInetAddress();
+
+			// check that we can connect to the localHost
+			Socket socket = new Socket();
+			try {
+				// port 0 = let the OS choose the port
+				SocketAddress bindP = new InetSocketAddress(result, 0);
+				// machine
+				socket.bind(bindP);
+				socket.connect(correctInetSocketAddress, 1000);
+			} finally {
+				socket.close();
+			}
+		} finally {
+			if (leaderElectionService != null) {
+				leaderElectionService.stop();
+			}
+		}
+	}
+
+	/**
+	 * Tests that the LeaderRetrievalUtils.findConnectingAddress stops trying to find the
+	 * connecting address if no leader address has been specified. The call should return
+	 * then InetAddress.getLocalHost().
+	 */
+	@Test
+	public void testTimeoutOfFindConnectingAddress() throws Exception {
+		Configuration config = new Configuration();
+		config.setString(ConfigConstants.RECOVERY_MODE, "zookeeper");
+		config.setString(ConfigConstants.ZOOKEEPER_QUORUM_KEY, testingCluster.getConnectString());
+
+		FiniteDuration timeout = new FiniteDuration(10, TimeUnit.SECONDS);
+
+		LeaderRetrievalService leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(config);
+		InetAddress result = LeaderRetrievalUtils.findConnectingAddress(leaderRetrievalService, timeout);
+
+		assertEquals(InetAddress.getLocalHost(), result);
+	}
+
+	class FindConnectingAddress implements Runnable {
+
+		private final Configuration config;
+		private final FiniteDuration timeout;
+
+		private InetAddress result;
+		private Exception exception;
+
+		public FindConnectingAddress(Configuration config, FiniteDuration timeout) {
+			this.config = config;
+			this.timeout = timeout;
+		}
+
+		@Override
+		public void run() {
+			try {
+				LeaderRetrievalService leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(config);
+				result = LeaderRetrievalUtils.findConnectingAddress(leaderRetrievalService, timeout);
+			} catch (Exception e) {
+				exception = e;
+			}
+		}
+
+		public InetAddress getInetAddress() throws Exception {
+			if (exception != null) {
+				throw exception;
+			} else {
+				return result;
+			}
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/TestData.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/TestData.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/TestData.java
index 8688d4e..2427edd 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/TestData.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/TestData.java
@@ -21,7 +21,6 @@ package org.apache.flink.runtime.operators.testutils;
 import java.util.Comparator;
 import java.util.Random;
 
-import org.apache.flink.api.java.tuple.Tuple;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.types.IntValue;
 import org.apache.flink.types.Record;

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/ForwardingActor.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/ForwardingActor.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/ForwardingActor.java
deleted file mode 100644
index 70e6f22..0000000
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/ForwardingActor.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.runtime.taskmanager;
-
-import akka.actor.UntypedActor;
-
-import java.util.concurrent.BlockingQueue;
-
-/**
- * Actor for testing that simply puts all its messages into a 
- * blocking queue.
- */
-class ForwardingActor extends UntypedActor {
-
-	private final BlockingQueue<Object> queue;
-	
-	public ForwardingActor(BlockingQueue<Object> queue) {
-		this.queue = queue;
-	}
-
-	@Override
-	public void onReceive(Object message) {
-		queue.add(message);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskCancelTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskCancelTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskCancelTest.java
index 1d0b3b1..837b643 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskCancelTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskCancelTest.java
@@ -18,13 +18,9 @@
 
 package org.apache.flink.runtime.taskmanager;
 
-import akka.actor.ActorRef;
-import akka.pattern.Patterns;
-import akka.util.Timeout;
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.client.JobClient;
 import org.apache.flink.runtime.instance.ActorGateway;
 import org.apache.flink.runtime.io.network.api.reader.RecordReader;
 import org.apache.flink.runtime.io.network.api.writer.RecordWriter;
@@ -67,13 +63,14 @@ public class TaskCancelTest {
 		try {
 			// Start a cluster for the given test config
 			final Configuration config = new Configuration();
-			config.setInteger(ConfigConstants.LOCAL_INSTANCE_MANAGER_NUMBER_TASK_MANAGER, 2);
+			config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, 2);
 			config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, sourceParallelism);
 			config.setString(ConfigConstants.AKKA_ASK_TIMEOUT, TestingUtils.DEFAULT_AKKA_ASK_TIMEOUT());
 			config.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY, 4096);
 			config.setInteger(ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY, 2048);
 
 			flink = new TestingCluster(config, false);
+			flink.start();
 
 			// Setup
 			final JobGraph jobGraph = new JobGraph("Cancel Big Union");
@@ -107,23 +104,27 @@ public class TaskCancelTest {
 						ResultPartitionType.PIPELINED);
 			}
 
-			// Run test
-			JobClient.submitJobDetached(
-					flink.getJobManagerGateway(), jobGraph,
-					TestingUtils.TESTING_DURATION(), getClass().getClassLoader());
+			// run the job
+			flink.submitJobDetached(jobGraph);
 
 			// Wait for the job to make some progress and then cancel
 			awaitRunning(
-					flink.jobManagerActor(), jobGraph.getJobID(), TestingUtils.TESTING_DURATION());
+				flink.getLeaderGateway(TestingUtils.TESTING_DURATION()),
+				jobGraph.getJobID(),
+				TestingUtils.TESTING_DURATION());
 
 			Thread.sleep(5000);
 
 			cancelJob(
-					flink.getJobManagerGateway(), jobGraph.getJobID(), TestingUtils.TESTING_DURATION());
+				flink.getLeaderGateway(TestingUtils.TESTING_DURATION()),
+				jobGraph.getJobID(),
+				TestingUtils.TESTING_DURATION());
 
 			// Wait for the job to be cancelled
 			JobStatus status = awaitTermination(
-					flink.jobManagerActor(), jobGraph.getJobID(), TestingUtils.TESTING_DURATION());
+				flink.getLeaderGateway(TestingUtils.TESTING_DURATION()),
+				jobGraph.getJobID(),
+				TestingUtils.TESTING_DURATION());
 
 			if (status == JobStatus.CANCELED) {
 				// Expected :-) All is swell.
@@ -181,7 +182,7 @@ public class TaskCancelTest {
 		}
 	}
 
-	private void awaitRunning(ActorRef jobManager, JobID jobId, FiniteDuration timeout)
+	private void awaitRunning(ActorGateway jobManager, JobID jobId, FiniteDuration timeout)
 			throws Exception {
 
 		checkNotNull(jobManager);
@@ -189,9 +190,9 @@ public class TaskCancelTest {
 		checkNotNull(timeout);
 
 		while (true) {
-			Future<Object> ask = Patterns.ask(jobManager,
+			Future<Object> ask = jobManager.ask(
 					new RequestJobStatus(jobId),
-					new Timeout(timeout));
+					timeout);
 
 			Object result = Await.result(ask, timeout);
 
@@ -223,7 +224,7 @@ public class TaskCancelTest {
 
 	}
 
-	private JobStatus awaitTermination(ActorRef jobManager, JobID jobId, FiniteDuration timeout)
+	private JobStatus awaitTermination(ActorGateway jobManager, JobID jobId, FiniteDuration timeout)
 			throws Exception {
 
 		checkNotNull(jobManager);
@@ -231,9 +232,9 @@ public class TaskCancelTest {
 		checkNotNull(timeout);
 
 		while (true) {
-			Future<Object> ask = Patterns.ask(jobManager,
+			Future<Object> ask = jobManager.ask(
 					new RequestJobStatus(jobId),
-					new Timeout(timeout));
+					timeout);
 
 			Object result = Await.result(ask, timeout);
 

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerComponentsStartupShutdownTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerComponentsStartupShutdownTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerComponentsStartupShutdownTest.java
index 442ddcf..0bb4063 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerComponentsStartupShutdownTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerComponentsStartupShutdownTest.java
@@ -35,6 +35,8 @@ import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
 import org.apache.flink.runtime.io.network.NetworkEnvironment;
 import org.apache.flink.runtime.io.network.netty.NettyConfig;
 import org.apache.flink.runtime.jobmanager.JobManager;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+import org.apache.flink.runtime.leaderretrieval.StandaloneLeaderRetrievalService;
 import org.apache.flink.runtime.memorymanager.DefaultMemoryManager;
 import org.apache.flink.runtime.memorymanager.MemoryManager;
 
@@ -70,8 +72,10 @@ public class TaskManagerComponentsStartupShutdownTest {
 		try {
 			actorSystem = AkkaUtils.createLocalActorSystem(config);
 
-			final ActorRef jobManager = JobManager.startJobManagerActors(config, actorSystem, 
-																			StreamingMode.BATCH_ONLY)._1();
+			final ActorRef jobManager = JobManager.startJobManagerActors(
+				config,
+				actorSystem,
+				StreamingMode.BATCH_ONLY)._1();
 
 			// create the components for the TaskManager manually
 			final TaskManagerConfiguration tmConfig = new TaskManagerConfiguration(
@@ -96,10 +100,18 @@ public class TaskManagerComponentsStartupShutdownTest {
 				netConf);
 			final int numberOfSlots = 1;
 
+			LeaderRetrievalService leaderRetrievalService = new StandaloneLeaderRetrievalService(jobManager.path().toString());
+
 			// create the task manager
-			final Props tmProps = Props.create(TaskManager.class,
-					tmConfig, connectionInfo, jobManager.path().toString(),
-					memManager, ioManager, network, numberOfSlots);
+			final Props tmProps = Props.create(
+					TaskManager.class,
+					tmConfig,
+					connectionInfo,
+					memManager,
+					ioManager,
+					network,
+					numberOfSlots,
+					leaderRetrievalService);
 
 			final ActorRef taskManager = actorSystem.actorOf(tmProps);
 

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerConfigurationTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerConfigurationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerConfigurationTest.java
index d243fbf..430e669 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerConfigurationTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerConfigurationTest.java
@@ -27,6 +27,7 @@ import scala.Tuple2;
 
 import java.io.IOException;
 import java.net.InetAddress;
+import java.net.InetSocketAddress;
 import java.net.ServerSocket;
 import java.net.UnknownHostException;
 
@@ -45,8 +46,10 @@ public class TaskManagerConfigurationTest {
 
 			Configuration config = new Configuration();
 			config.setString(ConfigConstants.TASK_MANAGER_HOSTNAME_KEY, TEST_HOST_NAME);
+			config.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, "localhost");
+			config.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, 7891);
 
-			Tuple2<String, Object> address = TaskManager.selectNetworkInterfaceAndPort(config, "localhost", 7891);
+			Tuple2<String, Object> address = TaskManager.selectNetworkInterfaceAndPort(config);
 
 			// validate the configured test host name
 			assertEquals(TEST_HOST_NAME, address._1());
@@ -63,19 +66,21 @@ public class TaskManagerConfigurationTest {
 			// config with pre-configured hostname to speed up tests (no interface selection)
 			Configuration config = new Configuration();
 			config.setString(ConfigConstants.TASK_MANAGER_HOSTNAME_KEY, "localhost");
+			config.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, "localhost");
+			config.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, 7891);
 
 			// auto port
-			assertEquals(0, TaskManager.selectNetworkInterfaceAndPort(config, "localhost", 7891)._2());
+			assertEquals(0, TaskManager.selectNetworkInterfaceAndPort(config)._2());
 
 			// pre-defined port
 			final int testPort = 22551;
 			config.setInteger(ConfigConstants.TASK_MANAGER_IPC_PORT_KEY, testPort);
-			assertEquals(testPort, TaskManager.selectNetworkInterfaceAndPort(config, "localhost", 7891)._2());
+			assertEquals(testPort, TaskManager.selectNetworkInterfaceAndPort(config)._2());
 
 			// invalid port
 			try {
 				config.setInteger(ConfigConstants.TASK_MANAGER_IPC_PORT_KEY, -1);
-				TaskManager.selectNetworkInterfaceAndPort(config, "localhost", 7891);
+				TaskManager.selectNetworkInterfaceAndPort(config);
 				fail("should fail with an exception");
 			}
 			catch (IllegalConfigurationException e) {
@@ -85,7 +90,7 @@ public class TaskManagerConfigurationTest {
 			// invalid port
 			try {
 				config.setInteger(ConfigConstants.TASK_MANAGER_IPC_PORT_KEY, 100000);
-				TaskManager.selectNetworkInterfaceAndPort(config, "localhost", 7891);
+				TaskManager.selectNetworkInterfaceAndPort(config);
 				fail("should fail with an exception");
 			}
 			catch (IllegalConfigurationException e) {
@@ -122,7 +127,10 @@ public class TaskManagerConfigurationTest {
 			// open a server port to allow the system to connect
 			Configuration config = new Configuration();
 
-			assertNotNull(TaskManager.selectNetworkInterfaceAndPort(config, hostname, server.getLocalPort())._1());
+			config.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, hostname);
+			config.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, server.getLocalPort());
+
+			assertNotNull(TaskManager.selectNetworkInterfaceAndPort(config)._1());
 		}
 		catch (Exception e) {
 			e.printStackTrace();

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerProcessReapingTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerProcessReapingTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerProcessReapingTest.java
index c55a721..5594bfe 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerProcessReapingTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerProcessReapingTest.java
@@ -85,7 +85,10 @@ public class TaskManagerProcessReapingTest {
 			jmActorSystem = AkkaUtils.createActorSystem(
 					new Configuration(), new Some<Tuple2<String, Object>>(localAddress));
 
-			JobManager.startJobManagerActors(new Configuration(), jmActorSystem, StreamingMode.BATCH_ONLY);
+			JobManager.startJobManagerActors(
+				new Configuration(),
+				jmActorSystem,
+				StreamingMode.BATCH_ONLY);
 
 			final int taskManagerPort = NetUtils.getAvailablePort();
 

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerRegistrationTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerRegistrationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerRegistrationTest.java
index df65019..85123a1 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerRegistrationTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerRegistrationTest.java
@@ -21,24 +21,24 @@ package org.apache.flink.runtime.taskmanager;
 import akka.actor.ActorRef;
 import akka.actor.ActorSystem;
 import akka.actor.InvalidActorNameException;
-import akka.actor.Kill;
-import akka.actor.Props;
 import akka.actor.Terminated;
-import akka.actor.UntypedActor;
-import akka.pattern.Patterns;
 import akka.testkit.JavaTestKit;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.StreamingMode;
 import org.apache.flink.runtime.akka.AkkaUtils;
+import org.apache.flink.runtime.instance.ActorGateway;
 import org.apache.flink.runtime.instance.InstanceID;
 import org.apache.flink.runtime.jobmanager.JobManager;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+import org.apache.flink.runtime.leaderretrieval.StandaloneLeaderRetrievalService;
 import org.apache.flink.runtime.messages.JobManagerMessages;
+import org.apache.flink.runtime.messages.JobManagerMessages.LeaderSessionMessage;
 import org.apache.flink.runtime.messages.RegistrationMessages.AcknowledgeRegistration;
 import org.apache.flink.runtime.messages.RegistrationMessages.RegisterTaskManager;
 import org.apache.flink.runtime.messages.RegistrationMessages.RefuseRegistration;
 import org.apache.flink.runtime.messages.TaskManagerMessages;
-import org.apache.flink.runtime.testingUtils.TestingTaskManager;
+import org.apache.flink.runtime.testingUtils.TestingUtils;
 import org.apache.flink.util.TestLogger;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -55,6 +55,10 @@ import java.net.ServerSocket;
 import java.util.UUID;
 import java.util.concurrent.TimeUnit;
 
+import static org.apache.flink.runtime.testingUtils.TestingUtils.createForwardingJobManager;
+import static org.apache.flink.runtime.testingUtils.TestingUtils.stopActor;
+import static org.apache.flink.runtime.testingUtils.TestingUtils.createTaskManager;
+import static org.apache.flink.runtime.testingUtils.TestingUtils.createJobManager;
 import static org.junit.Assert.*;
 
 /**
@@ -69,9 +73,11 @@ public class TaskManagerRegistrationTest extends TestLogger {
 	// use one actor system throughout all tests
 	private static ActorSystem actorSystem;
 
+	private static Configuration config;
+
 	@BeforeClass
 	public static void startActorSystem() {
-		Configuration config = new Configuration();
+		config = new Configuration();
 		config.getString(ConfigConstants.AKKA_ASK_TIMEOUT, "5 s");
 		config.getString(ConfigConstants.AKKA_WATCH_HEARTBEAT_INTERVAL, "200 ms");
 		config.getString(ConfigConstants.AKKA_WATCH_HEARTBEAT_PAUSE, "2 s");
@@ -94,24 +100,38 @@ public class TaskManagerRegistrationTest extends TestLogger {
 	@Test
 	public void testSimpleRegistration() {
 		new JavaTestKit(actorSystem) {{
+
+			ActorGateway jobManager = null;
+			ActorGateway taskManager1 = null;
+			ActorGateway taskManager2 = null;
+
 			try {
 				// a simple JobManager
-				ActorRef jobManager = startJobManager();
+				jobManager = createJobManager(actorSystem, config);
 
 				// start two TaskManagers. it will automatically try to register
-				final ActorRef taskManager1 = startTaskManager(jobManager);
-				final ActorRef taskManager2 = startTaskManager(jobManager);
+				taskManager1 = createTaskManager(
+						actorSystem,
+						jobManager,
+						config,
+						true,
+						false);
+
+				taskManager2 = createTaskManager(
+						actorSystem,
+						jobManager,
+						config,
+						true,
+						false);
 
 				// check that the TaskManagers are registered
-				Future<Object> responseFuture1 = Patterns.ask(
-						taskManager1,
+				Future<Object> responseFuture1 = taskManager1.ask(
 						TaskManagerMessages.getNotifyWhenRegisteredAtJobManagerMessage(),
-						5000);
+						new FiniteDuration(5000, TimeUnit.MILLISECONDS));
 
-				Future<Object> responseFuture2 = Patterns.ask(
-						taskManager2,
+				Future<Object> responseFuture2 = taskManager2.ask(
 						TaskManagerMessages.getNotifyWhenRegisteredAtJobManagerMessage(),
-						5000);
+						new FiniteDuration(5000, TimeUnit.MILLISECONDS));
 
 				Object response1 = Await.result(responseFuture1, new FiniteDuration(5, TimeUnit.SECONDS));
 				Object response2 = Await.result(responseFuture2, new FiniteDuration(5, TimeUnit.SECONDS));
@@ -122,21 +142,20 @@ public class TaskManagerRegistrationTest extends TestLogger {
 				assertTrue(response2 != null && confirmClass.isAssignableFrom(response2.getClass()));
 
 				// check that the JobManager has 2 TaskManagers registered
-				Future<Object> numTaskManagersFuture = Patterns.ask(
-						jobManager,
+				Future<Object> numTaskManagersFuture = jobManager.ask(
 						JobManagerMessages.getRequestNumberRegisteredTaskManager(),
-						1000);
+						new FiniteDuration(1000, TimeUnit.MILLISECONDS));
 
 				Integer count = (Integer) Await.result(numTaskManagersFuture, new FiniteDuration(1, TimeUnit.SECONDS));
 				assertEquals(2, count.intValue());
-
-				stopActor(taskManager1);
-				stopActor(taskManager2);
-				stopActor(jobManager);
 			}
 			catch (Exception e) {
 				e.printStackTrace();
 				fail(e.getMessage());
+			} finally {
+				stopActor(taskManager1);
+				stopActor(taskManager2);
+				stopActor(jobManager);
 			}
 		}};
 	}
@@ -148,23 +167,31 @@ public class TaskManagerRegistrationTest extends TestLogger {
 	@Test
 	public void testDelayedRegistration() {
 		new JavaTestKit(actorSystem) {{
+			ActorGateway jobManager = null;
+			ActorGateway taskManager = null;
+
 			try {
 				// start a TaskManager that tries to register at the JobManager before the JobManager is
 				// available. we give it the regular JobManager akka URL
-				final ActorRef taskManager = startTaskManager(JobManager.getLocalJobManagerAkkaURL(),
-						new Configuration());
+				taskManager = createTaskManager(
+						actorSystem,
+						JobManager.getLocalJobManagerAkkaURL(Option.<String>empty()),
+						new Configuration(),
+						true,
+						false);
+
 				// let it try for a bit
 				Thread.sleep(6000);
 
 				// now start the JobManager, with the regular akka URL
-				final ActorRef jobManager =
-						JobManager.startJobManagerActors(new Configuration(), actorSystem, StreamingMode.BATCH_ONLY)._1();
+				jobManager = createJobManager(
+						actorSystem,
+						new Configuration());
 
 				// check that the TaskManagers are registered
-				Future<Object> responseFuture = Patterns.ask(
-						taskManager,
+				Future<Object> responseFuture = taskManager.ask(
 						TaskManagerMessages.getNotifyWhenRegisteredAtJobManagerMessage(),
-						30000);
+						new FiniteDuration(30000, TimeUnit.MILLISECONDS));
 
 				Object response = Await.result(responseFuture, new FiniteDuration(30, TimeUnit.SECONDS));
 
@@ -172,12 +199,13 @@ public class TaskManagerRegistrationTest extends TestLogger {
 				Class<?> confirmClass = TaskManagerMessages.getRegisteredAtJobManagerMessage().getClass();
 				assertTrue(response != null && confirmClass.isAssignableFrom(response.getClass()));
 
-				stopActor(taskManager);
-				stopActor(jobManager);
 			}
 			catch (Exception e) {
 				e.printStackTrace();
 				fail(e.getMessage());
+			} finally {
+				stopActor(taskManager);
+				stopActor(jobManager);
 			}
 		}};
 	}
@@ -196,29 +224,40 @@ public class TaskManagerRegistrationTest extends TestLogger {
 	@Test
 	public void testShutdownAfterRegistrationDurationExpired() {
 		new JavaTestKit(actorSystem) {{
+
+			ActorGateway taskManager = null;
+
 			try {
 				// registration timeout of 1 second
 				Configuration tmConfig = new Configuration();
 				tmConfig.setString(ConfigConstants.TASK_MANAGER_MAX_REGISTRATION_DURATION, "500 ms");
 
 				// start the taskManager actor
-				final ActorRef taskManager = startTaskManager(JobManager.getLocalJobManagerAkkaURL(), tmConfig);
+				taskManager = createTaskManager(
+						actorSystem,
+						JobManager.getLocalJobManagerAkkaURL(Option.<String>empty()),
+						tmConfig,
+						true,
+						false);
 
 				// make sure it terminates in time, since it cannot register at a JobManager
-				watch(taskManager);
+				watch(taskManager.actor());
+
+				final ActorGateway tm = taskManager;
+
 				new Within(new FiniteDuration(10, TimeUnit.SECONDS)) {
 
 					@Override
 					protected void run() {
-						expectTerminated(taskManager);
+						expectTerminated(tm.actor());
 					}
 				};
-
-				stopActor(taskManager);
 			}
 			catch (Throwable e) {
 				e.printStackTrace();
 				fail(e.getMessage());
+			} finally {
+				stopActor(taskManager);
 			}
 		}};
 	}
@@ -230,10 +269,22 @@ public class TaskManagerRegistrationTest extends TestLogger {
 	@Test
 	public void testTaskManagerResumesConnectAfterRefusedRegistration() {
 		new JavaTestKit(actorSystem) {{
+			ActorGateway jm = null;
+			ActorGateway taskManager =null;
 			try {
+				jm= createForwardingJobManager(actorSystem, getTestActor(), Option.<String>empty());
+				final ActorGateway jmGateway = jm;
+
 				// we make the test actor (the test kit) the JobManager to intercept
 				// the messages
-				final ActorRef taskManager = startTaskManager(getTestActor());
+				taskManager = createTaskManager(
+						actorSystem,
+						jmGateway,
+						config,
+						true,
+						false);
+
+				final ActorGateway taskManagerGateway = taskManager;
 
 				// check and decline initial registration
 				new Within(new FiniteDuration(2, TimeUnit.SECONDS)) {
@@ -244,7 +295,9 @@ public class TaskManagerRegistrationTest extends TestLogger {
 						expectMsgClass(RegisterTaskManager.class);
 
 						// we decline the registration
-						getLastSender().tell(new RefuseRegistration(UUID.randomUUID(), "test reason"), getTestActor());
+						taskManagerGateway.tell(
+								new RefuseRegistration("test reason"),
+								jmGateway);
 					}
 				};
 
@@ -257,12 +310,13 @@ public class TaskManagerRegistrationTest extends TestLogger {
 						expectMsgClass(RegisterTaskManager.class);
 					}
 				};
-
-				stopActor(taskManager);
 			}
 			catch (Throwable e) {
 				e.printStackTrace();
 				fail(e.getMessage());
+			} finally {
+				stopActor(taskManager);
+				stopActor(jm);
 			}
 		}};
 	}
@@ -274,18 +328,29 @@ public class TaskManagerRegistrationTest extends TestLogger {
 	@Test
 	public void testTaskManagerResumesConnectAfterJobManagerFailure() {
 		new JavaTestKit(actorSystem) {{
-			try {
-				final Props fakeJmProps = Props.create(ForwardingActor.class, getTestActor());
-				final String jobManagerName = "FAKE_JOB_MANAGER";
+			ActorGateway fakeJobManager1Gateway = null;
+			ActorGateway fakeJobManager2Gateway = null;
+			ActorGateway taskManagerGateway = null;
 
-				final ActorRef fakeJobManager1 = actorSystem.actorOf(fakeJmProps, jobManagerName);
+			final String JOB_MANAGER_NAME = "ForwardingJobManager";
 
+			try {
+				fakeJobManager1Gateway = createForwardingJobManager(
+						actorSystem,
+						getTestActor(),
+						Option.apply(JOB_MANAGER_NAME));
+				final ActorGateway fakeJM1Gateway = fakeJobManager1Gateway;
 
 				// we make the test actor (the test kit) the JobManager to intercept
 				// the messages
-				final ActorRef taskManager = startTaskManager(fakeJobManager1);
+				taskManagerGateway = createTaskManager(
+						actorSystem,
+						fakeJobManager1Gateway,
+						config,
+						true,
+						false);
 
-				final UUID leaderSessionID = UUID.randomUUID();
+				final ActorGateway tm = taskManagerGateway;
 
 				// validate initial registration
 				new Within(new FiniteDuration(2, TimeUnit.SECONDS)) {
@@ -293,23 +358,22 @@ public class TaskManagerRegistrationTest extends TestLogger {
 					@Override
 					protected void run() {
 						// the TaskManager should try to register
-						RegisterTaskManager message = expectMsgClass(RegisterTaskManager.class);
+						expectMsgClass(RegisterTaskManager.class);
 
 						// we accept the registration
-						taskManager.tell(
+						tm.tell(
 								new AcknowledgeRegistration(
-										message.registrationSessionID(),
-										leaderSessionID,
-										fakeJobManager1,
 										new InstanceID(),
 										45234),
-								fakeJobManager1);
+								fakeJM1Gateway);
 					}
 				};
 
 				// kill the first forwarding JobManager
-				watch(fakeJobManager1);
-				stopActor(fakeJobManager1);
+				watch(fakeJobManager1Gateway.actor());
+				stopActor(fakeJobManager1Gateway.actor());
+
+				final ActorGateway gateway = fakeJobManager1Gateway;
 
 				// wait for the killing to be completed
 				final FiniteDuration timeout = new FiniteDuration(2, TimeUnit.SECONDS);
@@ -327,53 +391,56 @@ public class TaskManagerRegistrationTest extends TestLogger {
 						}
 
 						Terminated terminatedMessage = (Terminated) message;
-						assertEquals(fakeJobManager1, terminatedMessage.actor());
+						assertEquals(gateway.actor(), terminatedMessage.actor());
 					}
 				};
 
+				fakeJobManager1Gateway = null;
+
 				// now start the second fake JobManager and expect that
 				// the TaskManager registers again
 				// the second fake JM needs to have the same actor URL
-				ActorRef fakeJobManager2 = null;
 
 				// since we cannot reliably wait until the actor is unregistered (name is
 				// available again) we loop with multiple tries for 20 seconds
 				long deadline = 20000000000L + System.nanoTime();
 				do {
 					try {
-						fakeJobManager2 = actorSystem.actorOf(fakeJmProps, jobManagerName);
+						fakeJobManager2Gateway = createForwardingJobManager(
+								actorSystem,
+								getTestActor(),
+								Option.apply(JOB_MANAGER_NAME));
 					} catch (InvalidActorNameException e) {
 						// wait and retry
 						Thread.sleep(100);
 					}
-				} while (fakeJobManager2 == null && System.nanoTime() < deadline);
+				} while (fakeJobManager2Gateway == null && System.nanoTime() < deadline);
+
+				final ActorGateway fakeJM2GatewayClosure = fakeJobManager2Gateway;
 
 				// expect the next registration
-				final ActorRef jm2Closure = fakeJobManager2;
 				new Within(new FiniteDuration(10, TimeUnit.SECONDS)) {
 
 					@Override
 					protected void run() {
-						RegisterTaskManager message = expectMsgClass(RegisterTaskManager.class);
+						expectMsgClass(RegisterTaskManager.class);
 
 						// we accept the registration
-						taskManager.tell(
+						tm.tell(
 								new AcknowledgeRegistration(
-										message.registrationSessionID(),
-										leaderSessionID,
-										jm2Closure,
 										new InstanceID(),
 										45234),
-								jm2Closure);
+								fakeJM2GatewayClosure);
 					}
 				};
-
-				stopActor(taskManager);
-				stopActor(fakeJobManager2);
 			}
 			catch (Throwable e) {
 				e.printStackTrace();
 				fail(e.getMessage());
+			} finally {
+				stopActor(taskManagerGateway);
+				stopActor(fakeJobManager1Gateway);
+				stopActor(fakeJobManager2Gateway);
 			}
 		}};
 	}
@@ -383,6 +450,7 @@ public class TaskManagerRegistrationTest extends TestLogger {
 	public void testStartupWhenNetworkStackFailsToInitialize() {
 
 		ServerSocket blocker = null;
+
 		try {
 			blocker = new ServerSocket(0, 50, InetAddress.getByName("localhost"));
 
@@ -392,15 +460,18 @@ public class TaskManagerRegistrationTest extends TestLogger {
 			cfg.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 1);
 
 			new JavaTestKit(actorSystem) {{
+				ActorRef taskManager = null;
+				ActorRef jobManager = null;
+
 				try {
 					// a simple JobManager
-					final ActorRef jobManager = startJobManager();
+					jobManager = startJobManager(config);
 
 					// start a task manager with a configuration that provides a blocked port
-					final ActorRef taskManager = TaskManager.startTaskManagerComponentsAndActor(
+					taskManager = TaskManager.startTaskManagerComponentsAndActor(
 							cfg, actorSystem, "localhost",
 							NONE_STRING, // no actor name -> random
-							new Some<String>(jobManager.path().toString()), // job manager path
+							new Some<LeaderRetrievalService>(new StandaloneLeaderRetrievalService(jobManager.path().toString())),
 							false, // init network stack !!!
 							StreamingMode.BATCH_ONLY,
 							TaskManager.class);
@@ -408,13 +479,13 @@ public class TaskManagerRegistrationTest extends TestLogger {
 					watch(taskManager);
 
 					expectTerminated(new FiniteDuration(20, TimeUnit.SECONDS), taskManager);
-
-					stopActor(taskManager);
-					stopActor(jobManager);
 				}
 				catch (Exception e) {
 					e.printStackTrace();
 					fail(e.getMessage());
+				} finally {
+					stopActor(taskManager);
+					stopActor(jobManager);
 				}
 			}};
 		}
@@ -438,13 +509,23 @@ public class TaskManagerRegistrationTest extends TestLogger {
 	@Test
 	public void testCheckForValidRegistrationSessionIDs() {
 		new JavaTestKit(actorSystem) {{
+
+			ActorGateway taskManagerGateway = null;
+
 			try {
 				// we make the test actor (the test kit) the JobManager to intercept
 				// the messages
-				final ActorRef taskManager = startTaskManager(getTestActor());
+				taskManagerGateway = createTaskManager(
+						actorSystem,
+						getTestActor(),
+						config,
+						true,
+						false);
+
+				final ActorRef taskManager = taskManagerGateway.actor();
 
 				final UUID falseLeaderSessionID = UUID.randomUUID();
-				final UUID trueLeaderSessionID = UUID.randomUUID();
+				final UUID trueLeaderSessionID = null;
 
 				new Within(new FiniteDuration(20, TimeUnit.SECONDS)) {
 
@@ -453,50 +534,51 @@ public class TaskManagerRegistrationTest extends TestLogger {
 						taskManager.tell(TaskManagerMessages.getNotifyWhenRegisteredAtJobManagerMessage(), getTestActor());
 
 						// the TaskManager should try to register
-						RegisterTaskManager registerTaskManager = expectMsgClass(RegisterTaskManager.class);
+
+						LeaderSessionMessage lsm = expectMsgClass(LeaderSessionMessage.class);
+
+						assertTrue(lsm.leaderSessionID() == trueLeaderSessionID);
+						assertTrue(lsm.message() instanceof RegisterTaskManager);
 
 						final ActorRef tm = getLastSender();
 
 						// This AcknowledgeRegistration message should be discarded because the
 						// registration session ID is wrong
-						tm.tell(new AcknowledgeRegistration(
-									UUID.randomUUID(),
-									falseLeaderSessionID,
-									getTestActor(),
-									new InstanceID(),
-									1),
+						tm.tell(
+								new LeaderSessionMessage(
+										falseLeaderSessionID,
+										new AcknowledgeRegistration(
+												new InstanceID(),
+												1)),
 								getTestActor());
 
 						// Valid AcknowledgeRegistration message
-						tm.tell(new AcknowledgeRegistration(
-										registerTaskManager.registrationSessionID(),
+						tm.tell(
+								new LeaderSessionMessage(
 										trueLeaderSessionID,
-										getTestActor(),
-										new InstanceID(),
-										1),
+										new AcknowledgeRegistration(
+												new InstanceID(),
+												1)),
 								getTestActor());
 
 						Object message = null;
 						Object confirmMessageClass = TaskManagerMessages.getRegisteredAtJobManagerMessage().getClass();
 
 						while(message == null || !(message.getClass().equals(confirmMessageClass))) {
-							message = expectMsgAnyClassOf(
-									TaskManagerMessages.getRegisteredAtJobManagerMessage().getClass(),
-									RegisterTaskManager.class,
-									TaskManagerMessages.Heartbeat.class);
+							message = receiveOne(TestingUtils.TESTING_DURATION());
 						}
 
 						tm.tell(JobManagerMessages.getRequestLeaderSessionID(), getTestActor());
 
-						expectMsgEquals(new JobManagerMessages.ResponseLeaderSessionID(Option.apply(trueLeaderSessionID)));
+						expectMsgEquals(new JobManagerMessages.ResponseLeaderSessionID(trueLeaderSessionID));
 					}
 				};
-
-				stopActor(taskManager);
 			}
 			catch (Throwable e) {
 				e.printStackTrace();
 				fail(e.getMessage());
+			} finally {
+				stopActor(taskManagerGateway);
 			}
 		}};
 	}
@@ -505,48 +587,14 @@ public class TaskManagerRegistrationTest extends TestLogger {
 	//  Utility Functions
 	// --------------------------------------------------------------------------------------------
 
-	private static ActorRef startJobManager() throws Exception {
+	private static ActorRef startJobManager(Configuration configuration) throws Exception {
 		// start the actors. don't give names, so they get generated names and we
 		// avoid conflicts with the actor names
-		return JobManager.startJobManagerActors(new Configuration(), actorSystem, 
-												NONE_STRING, NONE_STRING, StreamingMode.BATCH_ONLY)._1();
-	}
-
-	private static ActorRef startTaskManager(ActorRef jobManager) throws Exception {
-		return startTaskManager(jobManager.path().toString(), new Configuration());
-	}
-
-	private static ActorRef startTaskManager(String jobManagerUrl, Configuration config) throws Exception {
-		config.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 1);
-
-		return TaskManager.startTaskManagerComponentsAndActor(
-				config, actorSystem, "localhost",
-				NONE_STRING, // no actor name -> random
-				new Some<String>(jobManagerUrl), // job manager path
-				true, // local network stack only
-				StreamingMode.BATCH_ONLY,
-				TestingTaskManager.class);
-	}
-
-	private static void stopActor(ActorRef actor) {
-		actor.tell(Kill.getInstance(), ActorRef.noSender());
-	}
-
-	// --------------------------------------------------------------------------------------------
-	//  Utility Actor that only forwards messages
-	// --------------------------------------------------------------------------------------------
-
-	public static class ForwardingActor extends UntypedActor {
-
-		private final ActorRef target;
-
-		public ForwardingActor(ActorRef target) {
-			this.target = target;
-		}
-
-		@Override
-		public void onReceive(Object message) throws Exception {
-			target.forward(message, context());
-		}
+		return JobManager.startJobManagerActors(
+			configuration,
+			actorSystem,
+			NONE_STRING,
+			NONE_STRING,
+			StreamingMode.BATCH_ONLY)._1();
 	}
 }


[10/10] flink git commit: [FLINK-2291] [runtime] Add ZooKeeper support to elect a leader from a set of JobManager. The leader will then be retrieved from ZooKeeper by the TaskManagers.

Posted by tr...@apache.org.
[FLINK-2291] [runtime] Add ZooKeeper support to elect a leader from a set of JobManager. The leader will then be retrieved from ZooKeeper by the TaskManagers.

Refactors FlinkMiniCluster to support multiple JobManager

Adds proper remote address resolution for actors

Clean up of LeaderElection and LeaderRetrievalService. Removes synchronization to avoid deadlock.

Adds ZooKeeper start option to TestBaseUtils.startCluster

Removes registration session IDs, using the leader session IDs instead. Sets the leader session ID
 directly in the grantLeadership method. Let the LeaderElectionService select the leader session I
D. Return leader session ID to LeaderRetrievalListeners.

Removes direct ActorRef interaction

Introduces LeaderRetrievalService for the Client and the CliFrontend.

Make ApplicationClient to use the LeaderRetrievalService for JobManager resolution

Adds LeaderElection/Retrieval tests

Added test for exception forwarding from the CuratorFramework to a Contender

Adds test job submission with changing leaders

Adds new test cases for job cleanup after leader election change

Adds new LeaderChangeStateCleanup test case

Adds LeaderElectionRetrievalTestingCluster

Introduces ListeningBehaviour for job submissions

Relocation of org.apache.curator in flink-shaded-hadoop jar

Adds Apache ZooKeeper and Apache Curator to LICENSE and NOTICE files

Increases zookeeper connection timeout to 20000 ms for the KafkaITCase to fix failing tests on Travis

Increased timeouts of ZooKeeperLeaderElectionTest for Travis

Makes the WebInfoServer and the WebRuntimeMonitor to use the LeaderRetrievalService to retrieve the current leading JobManager

Adds proper synchronization to ZooKeeperLeaderElectionService. Fixes StateCheckpointedITCase and PartitionedStateCheckpointingITCase

Adds configuration description for new ZooKeeper configuration values

Fixed port selection of JobManager at startup

Improves logging output

Extends masters file to also specify the webui ports

Adds proper network interface resolution by retrieving the current leader address

Makes the ZooKeeperLeaderElectionService write the leader information in ephemeral nodes so that the information is deleted once the leader has terminated. Fixes a bug in the TaskManager due to call by name semantics of scheduler.scheduleOnce.

Adds jobManagerURL to TriggerTaskManagerRegistration message

Enables findConnectingAddress to use the ZooKeeperLeaderRetrievalService. This allows to test the connection to a possibly changing master node.

Changes startup scripts to respect the recovery mode instead of the ZK_QUORUM

Adjust travis log file to only log zookeeper errors

Updates high availability setup guide

Adds TestLogger to leader election tests

This closes #1016.


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

Branch: refs/heads/master
Commit: b9de4ed37ffa68ef50dc6d6b3819afcc00d1d029
Parents: 0858d9f
Author: Till Rohrmann <tr...@apache.org>
Authored: Fri Jun 26 12:07:39 2015 +0200
Committer: Till Rohrmann <tr...@apache.org>
Committed: Mon Aug 31 11:02:31 2015 +0200

----------------------------------------------------------------------
 docs/setup/config.md                            |   25 +
 docs/setup/jobmanager_high_availability.md      |   28 +-
 flink-clients/pom.xml                           |    8 +
 .../org/apache/flink/client/CliFrontend.java    |  138 ++-
 .../org/apache/flink/client/LocalExecutor.java  |    5 +-
 .../org/apache/flink/client/RemoteExecutor.java |   14 +-
 .../org/apache/flink/client/program/Client.java |  295 +++--
 .../CliFrontendAddressConfigurationTest.java    |   94 +-
 .../flink/client/CliFrontendInfoTest.java       |   17 +-
 .../flink/client/CliFrontendListCancelTest.java |   15 +-
 .../apache/flink/client/CliFrontendRunTest.java |    8 +-
 .../RemoteExecutorHostnameResolutionTest.java   |   12 +-
 .../client/program/ClientConnectionTest.java    |    5 +-
 .../program/ClientHostnameResolutionTest.java   |   98 --
 .../apache/flink/client/program/ClientTest.java |    9 +-
 .../program/ExecutionPlanCreationTest.java      |    8 +-
 ...rRetrievalServiceHostnameResolutionTest.java |  103 ++
 .../stormcompatibility/api/FlinkClient.java     |   17 +-
 .../flink/contrib/streaming/CollectITCase.java  |    4 +-
 .../flink/configuration/ConfigConstants.java    |   53 +-
 .../flink/types/parser/ByteParserTest.java      |    4 -
 .../org/apache/flink/util/AbstractIDTest.java   |    2 -
 flink-dist/src/main/flink-bin/LICENSE           |    5 +
 flink-dist/src/main/flink-bin/NOTICE            |   22 +
 flink-dist/src/main/flink-bin/bin/config.sh     |   15 +-
 flink-dist/src/main/flink-bin/bin/jobmanager.sh |    7 +-
 .../src/main/flink-bin/bin/start-cluster.sh     |   21 +-
 .../src/main/flink-bin/bin/stop-cluster.sh      |   11 +-
 .../webmonitor/ExecutionGraphHolder.java        |   61 +-
 .../webmonitor/JobManagerArchiveRetriever.java  |  111 ++
 .../runtime/webmonitor/WebRuntimeMonitor.java   |   72 +-
 .../handlers/RequestJobIdsHandler.java          |   25 +-
 .../handlers/RequestOverviewHandler.java        |   25 +-
 .../legacy/JobManagerInfoHandler.java           |   26 +-
 .../runtime/webmonitor/runner/TestRunner.java   |    7 +-
 .../src/test/resources/log4j-test.properties    |   38 +
 .../src/test/resources/logback-test.xml         |   42 +
 flink-runtime/pom.xml                           |    6 +
 .../flink/runtime/akka/FlinkUntypedActor.java   |   24 +-
 .../flink/runtime/akka/ListeningBehaviour.java  |   29 +
 .../apache/flink/runtime/blob/BlobServer.java   |    4 +
 .../checkpoint/CheckpointCoordinator.java       |    3 +-
 .../CheckpointCoordinatorDeActivator.java       |   14 +-
 .../apache/flink/runtime/client/JobClient.java  |    8 +-
 .../flink/runtime/client/JobClientActor.java    |   21 +-
 .../flink/runtime/executiongraph/Execution.java |    4 +-
 .../runtime/executiongraph/ExecutionGraph.java  |    3 +-
 .../flink/runtime/instance/ActorGateway.java    |   10 +-
 .../runtime/instance/AkkaActorGateway.java      |   16 +-
 .../flink/runtime/instance/InstanceManager.java |   46 +-
 .../runtime/io/network/NetworkEnvironment.java  |    2 +-
 .../jobmanager/JobManagerCliOptions.java        |   10 +
 .../flink/runtime/jobmanager/RecoveryMode.java  |   33 +
 .../runtime/jobmanager/web/WebInfoServer.java   |  163 ++-
 .../runtime/leaderelection/LeaderContender.java |   59 +
 .../leaderelection/LeaderElectionService.java   |   70 ++
 .../StandaloneLeaderElectionService.java        |   63 +
 .../ZooKeeperLeaderElectionService.java         |  265 +++++
 .../LeaderRetrievalException.java               |   40 +
 .../LeaderRetrievalListener.java                |   44 +
 .../leaderretrieval/LeaderRetrievalService.java |   48 +
 .../StandaloneLeaderRetrievalService.java       |   59 +
 .../ZooKeeperLeaderRetrievalService.java        |  126 ++
 .../messages/LeaderSessionMessageDecorator.java |   49 +
 .../runtime/messages/MessageDecorator.java      |   33 +
 .../org/apache/flink/runtime/net/NetUtils.java  |  157 +++
 .../apache/flink/runtime/taskmanager/Task.java  |    7 +-
 .../runtime/util/LeaderConnectionInfo.java      |   44 +
 .../flink/runtime/util/LeaderElectionUtils.java |   57 +
 .../runtime/util/LeaderRetrievalUtils.java      |  244 ++++
 .../flink/runtime/util/StandaloneUtils.java     |   84 ++
 .../flink/runtime/util/ZooKeeperUtil.java       |  110 --
 .../flink/runtime/util/ZooKeeperUtils.java      |  151 +++
 .../runtime/LeaderSessionMessageDecorator.scala |   46 -
 .../runtime/LeaderSessionMessageFilter.scala    |   69 ++
 .../flink/runtime/LeaderSessionMessages.scala   |   73 --
 .../apache/flink/runtime/MessageDecorator.scala |   32 -
 .../apache/flink/runtime/akka/AkkaUtils.scala   |  127 +-
 .../runtime/akka/RemoteAddressExtension.scala   |   32 +
 .../flink/runtime/jobmanager/JobManager.scala   |  515 +++++----
 .../runtime/jobmanager/MemoryArchivist.scala    |    4 +
 .../messages/ExecutionGraphMessages.scala       |    2 +-
 .../runtime/messages/JobManagerMessages.scala   |   49 +-
 .../runtime/messages/RegistrationMessages.scala |   23 +-
 .../runtime/messages/TaskControlMessages.scala  |  184 +++
 .../runtime/messages/TaskManagerMessages.scala  |   21 +-
 .../flink/runtime/messages/TaskMessages.scala   |  184 ---
 .../runtime/minicluster/FlinkMiniCluster.scala  |  445 +++++--
 .../minicluster/LocalFlinkMiniCluster.scala     |  125 +-
 .../flink/runtime/taskmanager/TaskManager.scala |  500 ++++----
 .../runtime/akka/FlinkUntypedActorTest.java     |   26 +-
 .../checkpoint/CoordinatorShutdownTest.java     |   26 +-
 .../ExecutionVertexCancelTest.java              |    7 +-
 .../instance/BaseTestingActorGateway.java       |    5 +-
 .../runtime/instance/DummyActorGateway.java     |    5 +-
 .../runtime/instance/InstanceManagerTest.java   |    4 +-
 .../netty/ServerTransportErrorHandlingTest.java |    1 -
 .../PartialConsumePipelinedResultTest.java      |    6 +-
 .../consumer/LocalInputChannelTest.java         |    3 -
 .../JobManagerProcessReapingTest.java           |    2 +-
 .../runtime/jobmanager/JobManagerTest.java      |   15 +-
 .../flink/runtime/jobmanager/JobSubmitTest.java |   38 +-
 .../JobManagerLeaderElectionTest.java           |  173 +++
 .../LeaderChangeStateCleanupTest.java           |  273 +++++
 .../LeaderElectionRetrievalTestingCluster.java  |  121 ++
 .../StandaloneLeaderElectionTest.java           |   60 +
 .../leaderelection/TestingContender.java        |  146 +++
 .../TestingLeaderElectionService.java           |   60 +
 .../TestingLeaderRetrievalService.java          |   47 +
 .../runtime/leaderelection/TestingListener.java |  115 ++
 .../ZooKeeperLeaderElectionTest.java            |  550 +++++++++
 .../ZooKeeperLeaderRetrievalTest.java           |  212 ++++
 .../runtime/operators/testutils/TestData.java   |    1 -
 .../runtime/taskmanager/ForwardingActor.java    |   41 -
 .../runtime/taskmanager/TaskCancelTest.java     |   37 +-
 ...askManagerComponentsStartupShutdownTest.java |   22 +-
 .../TaskManagerConfigurationTest.java           |   20 +-
 .../TaskManagerProcessReapingTest.java          |    5 +-
 .../TaskManagerRegistrationTest.java            |  320 +++---
 .../taskmanager/TaskManagerStartupTest.java     |  168 +++
 .../runtime/taskmanager/TaskManagerTest.java    |  410 ++++---
 .../flink/runtime/taskmanager/TaskTest.java     |    2 +-
 .../taskmanager/TestManagerStartupTest.java     |  168 ---
 .../flink/runtime/util/ZooKeeperUtilTest.java   |   17 +-
 .../flink/runtime/akka/AkkaUtilsTest.scala      |   67 ++
 .../flink/runtime/akka/FlinkActorTest.scala     |   16 +-
 .../jobmanager/CoLocationConstraintITCase.scala |    6 +-
 .../jobmanager/JobManagerConnectionTest.scala   |    4 +-
 .../runtime/jobmanager/JobManagerITCase.scala   |   57 +-
 .../jobmanager/JobManagerRegistrationTest.scala |   82 +-
 .../runtime/jobmanager/RecoveryITCase.scala     |   35 +-
 .../runtime/jobmanager/SlotSharingITCase.scala  |   10 +-
 .../TaskManagerFailsWithSlotSharingITCase.scala |   10 +-
 .../testingUtils/ScalaTestingUtils.scala        |    4 +-
 .../runtime/testingUtils/TestingCluster.scala   |  109 +-
 .../testingUtils/TestingJobManager.scala        |   53 +-
 .../TestingJobManagerMessages.scala             |   12 +-
 .../runtime/testingUtils/TestingMessages.scala  |   10 +
 .../testingUtils/TestingTaskManager.scala       |   75 +-
 .../TestingTaskManagerMessages.scala            |    4 +-
 .../runtime/testingUtils/TestingUtils.scala     |  201 +++-
 flink-shaded-hadoop/pom.xml                     |    4 +
 .../api/avro/AvroExternalJarProgramITCase.java  |   12 +-
 .../apache/flink/api/io/avro/AvroPojoTest.java  |    1 -
 .../operations/DegreesWithExceptionITCase.java  |   11 +-
 .../ReduceOnEdgesWithExceptionITCase.java       |    5 +-
 .../ReduceOnNeighborsWithExceptionITCase.java   |    9 +-
 .../hbase/example/HBaseWriteStreamExample.java  |    1 -
 .../org.apache.flink/api/scala/FlinkShell.scala |    2 +-
 .../flink/api/scala/ScalaShellITSuite.scala     |   11 +-
 .../connectors/kafka/KafkaConsumerTestBase.java | 1085 ++++++++----------
 .../streaming/connectors/kafka/KafkaITCase.java |   22 +-
 .../connectors/kafka/KafkaTestBase.java         |   14 +-
 .../environment/RemoteStreamEnvironment.java    |   11 +-
 .../flink/streaming/util/ClusterUtil.java       |    2 +
 .../streaming/timestamp/TimestampITCase.java    |   16 +-
 .../util/StreamingMultipleProgramsTestBase.java |    8 +-
 .../streaming/util/TestStreamEnvironment.java   |    1 +
 flink-test-utils/pom.xml                        |    6 +
 .../flink/test/util/AbstractTestBase.java       |    8 +-
 .../test/util/MultipleProgramsTestBase.java     |    8 +-
 .../apache/flink/test/util/TestBaseUtils.java   |   53 +-
 .../apache/flink/test/util/TestEnvironment.java |    2 +-
 .../apache/flink/test/util/FlinkTestBase.scala  |    9 +-
 .../test/util/ForkableFlinkMiniCluster.scala    |  217 +++-
 flink-tests/pom.xml                             |    1 -
 .../accumulators/AccumulatorLiveITCase.java     |   12 +-
 .../test/cancelling/CancellingTestBase.java     |    6 +-
 .../StreamFaultToleranceTestBase.java           |   10 +-
 .../test/classloading/ClassLoaderITCase.java    |    6 +-
 .../JobSubmissionFailsITCase.java               |    4 +-
 .../test/javaApiOperators/DistinctITCase.java   |    3 -
 .../test/manual/NotSoMiniClusterIterations.java |    4 +-
 .../manual/StreamingScalabilityAndLatency.java  |    4 +-
 .../flink/test/misc/AutoParallelismITCase.java  |    6 +-
 .../test/misc/CustomSerializationITCase.java    |    9 +-
 .../test/misc/MiscellaneousIssuesITCase.java    |   10 +-
 ...SuccessAfterNetworkBuffersFailureITCase.java |   10 +-
 .../test/recovery/SimpleRecoveryITCase.java     |   12 +-
 .../TaskManagerFailureRecoveryITCase.java       |    6 +-
 .../ZooKeeperLeaderElectionITCase.java          |  255 ++++
 .../LocalFlinkMiniClusterITCase.java            |   11 +-
 .../flink/test/web/WebFrontendITCase.java       |   22 +-
 .../src/test/resources/log4j-test.properties    |    3 +-
 .../jobmanager/JobManagerFailsITCase.scala      |   47 +-
 .../JobManagerLeaderSessionIDITSuite.scala      |    8 +-
 .../taskmanager/TaskManagerFailsITCase.scala    |   34 +-
 .../org/apache/flink/yarn/FlinkYarnCluster.java |   10 +-
 .../apache/flink/yarn/ApplicationClient.scala   |   20 +-
 .../apache/flink/yarn/ApplicationMaster.scala   |   25 +-
 .../scala/org/apache/flink/yarn/Messages.scala  |    7 +-
 .../org/apache/flink/yarn/YarnTaskManager.scala |    9 +-
 pom.xml                                         |   19 +-
 tools/log4j-travis.properties                   |    3 +-
 194 files changed, 8626 insertions(+), 3316 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/docs/setup/config.md
----------------------------------------------------------------------
diff --git a/docs/setup/config.md b/docs/setup/config.md
index e2ffda6..35edf7a 100644
--- a/docs/setup/config.md
+++ b/docs/setup/config.md
@@ -357,6 +357,31 @@ so that the Flink client is able to pick those details up. This configuration pa
 changing the default location of that file (for example for environments sharing a Flink 
 installation between users)
 
+## High Availability Mode
+
+- `recovery.mode`: (Default 'standalone') Defines the recovery mode used for the cluster execution. Currently,
+Flink supports the 'standalone' mode where only a single JobManager runs and no JobManager state is checkpointed.
+The high availability mode 'zookeeper' supports the execution of multiple JobManagers and JobManager state checkpointing.
+Among the group of JobManagers, ZooKeeper elects one of them as the leader which is responsible for the cluster execution.
+In case of a JobManager failure, a standby JobManager will be elected as the new leader and is given the last checkpointed JobManager state.
+In order to use the 'zookeeper' mode, it is mandatory to also define the `ha.zookeeper.quorum` configuration value.
+
+- `ha.zookeeper.quorum`: Defines the ZooKeeper quorum URL which is used to connet to the ZooKeeper cluster when the 'zookeeper' recovery mode is selected
+
+- `ha.zookeeper.dir`: (Default '/flink') Defines the root dir under which the ZooKeeper recovery mode will create znodes. 
+
+- `ha.zookeeper.dir.latch`: (Default '/leaderlatch') Defines the znode of the leader latch which is used to elect the leader.
+
+- `ha.zookeeper.dir.leader`: (Default '/leader') Defines the znode of the leader which contains the URL to the leader and the current leader session ID
+
+- `ha.zookeeper.client.session-timeout`: (Default '60000') Defines the session timeout for the ZooKeeper session in ms.
+
+- `ha.zookeeper.client.connection-timeout`: (Default '15000') Defines the connection timeout for ZooKeeper in ms.
+
+- `ha.zookeeper.client.retry-wait`: (Default '5000') Defines the pause between consecutive retries in ms.
+
+- `ha.zookeeper.client.max-retry-attempts`: (Default '3') Defines the number of connection retries before the client gives up.
+
 ## Background
 
 ### Configuring the Network Buffers

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/docs/setup/jobmanager_high_availability.md
----------------------------------------------------------------------
diff --git a/docs/setup/jobmanager_high_availability.md b/docs/setup/jobmanager_high_availability.md
index 8958e17..50379ea 100644
--- a/docs/setup/jobmanager_high_availability.md
+++ b/docs/setup/jobmanager_high_availability.md
@@ -34,11 +34,19 @@ As an example, consider the following setup with three JobManager instances:
 
 ## Configuration
 
-To enable JobManager High Availability you have to configure a **ZooKeeper quorum** and set up a **masters file** with all JobManagers hosts.
+To enable JobManager High Availability you have to set the **recovery mode** to *zookeeper*, configure a **ZooKeeper quorum** and set up a **masters file** with all JobManagers hosts and their web ui ports.
 
 Flink leverages **[ZooKeeper](http://zookeeper.apache.org)** for  *distributed coordination* between all running JobManager instances. ZooKeeper is a separate service from Flink, which provides highly reliable distirbuted coordination via leader election and light-weight consistent state storage. Check out [ZooKeeper's Getting Started Guide](http://zookeeper.apache.org/doc/trunk/zookeeperStarted.html) for more information about ZooKeeper.
 
-Configuring a ZooKeeper quorum in `conf/flink-conf.yaml` *enables* high availability mode and all Flink components try to connect to a JobManager via coordination through ZooKeeper.
+Setting Flink's **recovery mode** to *zookeeper* in `conf/flink-conf.yaml` *enables* high availability mode.
+
+Additionally, you have to configure a **ZooKeeper quorum** in the same configuration file.
+
+In high availabliity mode, all Flink components try to connect to a JobManager via coordination through ZooKeeper.
+
+- **Recovery mode** (required): The *recovery mode* has to be set in `conf/flink-conf.yaml` to *zookeeper* in order to enable high availability mode. 
+  
+  <pre>recovery.mode: zookeeper</pre>
 
 - **ZooKeeper quorum** (required): A *ZooKeeper quorum* is a replicated group of ZooKeeper servers, which provide the distributed coordination service.
   
@@ -55,12 +63,12 @@ Configuring a ZooKeeper quorum in `conf/flink-conf.yaml` *enables* high availabi
 
 In order to start an HA-cluster configure the *masters* file in `conf/masters`:
 
-- **masters file**: The *masters file* contains all hosts, on which JobManagers are started.
+- **masters file**: The *masters file* contains all hosts, on which JobManagers are started, and the ports to which the web user interface binds.
 
   <pre>
-jobManagerAddress1
+jobManagerAddress1:webUIPort1
 [...]
-jobManagerAddressX
+jobManagerAddressX:webUIPortX
   </pre>
 
 After configuring the masters and the ZooKeeper quorum, you can use the provided cluster startup scripts as usual. They will start a HA-cluster. **Keep in mind that the ZooKeeper quorum has to be running when you call the scripts**.
@@ -81,15 +89,17 @@ The script `bin/start-zookeeper-quorum.sh` will start a ZooKeeper server on each
 
 ## Example: Start and stop a local HA-cluster with 2 JobManagers
 
-1. **Configure ZooKeeper quorum** in `conf/flink.yaml`:
+1. **Configure recovery mode and ZooKeeper quorum** in `conf/flink.yaml`:
    
-   <pre>ha.zookeeper.quorum: localhost</pre>
+   <pre>
+recovery.mode: zookeeper
+ha.zookeeper.quorum: localhost</pre>
 
 2. **Configure masters** in `conf/masters`:
 
    <pre>
-localhost
-localhost</pre>
+localhost:8081
+localhost:8082</pre>
 
 3. **Configure ZooKeeper server** in `conf/zoo.cfg` (currently it's only possible to run a single ZooKeeper server per machine):
 

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-clients/pom.xml
----------------------------------------------------------------------
diff --git a/flink-clients/pom.xml b/flink-clients/pom.xml
index e94487f..84264f9 100644
--- a/flink-clients/pom.xml
+++ b/flink-clients/pom.xml
@@ -44,6 +44,14 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-core</artifactId>
+			<version>${project.version}</version>
+			<type>test-jar</type>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
 			<artifactId>flink-runtime</artifactId>
 			<version>${project.version}</version>
 		</dependency>

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java b/flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java
index ea1a6e9..ac8009e 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java
@@ -36,7 +36,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.Properties;
 
-import akka.actor.ActorRef;
 import akka.actor.ActorSystem;
 
 import org.apache.commons.cli.CommandLine;
@@ -64,14 +63,15 @@ import org.apache.flink.optimizer.plan.OptimizedPlan;
 import org.apache.flink.optimizer.plandump.PlanJSONDumpGenerator;
 import org.apache.flink.runtime.client.JobStatusMessage;
 import org.apache.flink.runtime.instance.ActorGateway;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
 import org.apache.flink.runtime.messages.JobManagerMessages;
 import org.apache.flink.runtime.security.SecurityUtils;
 import org.apache.flink.runtime.util.EnvironmentInformation;
+import org.apache.flink.runtime.util.LeaderRetrievalUtils;
 import org.apache.flink.runtime.yarn.AbstractFlinkYarnClient;
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.runtime.jobgraph.JobGraph;
 import org.apache.flink.runtime.jobgraph.JobStatus;
-import org.apache.flink.runtime.jobmanager.JobManager;
 import org.apache.flink.runtime.messages.JobManagerMessages.CancelJob;
 import org.apache.flink.runtime.messages.JobManagerMessages.RunningJobsStatus;
 import org.apache.flink.runtime.yarn.AbstractFlinkYarnCluster;
@@ -129,8 +129,6 @@ public class CliFrontend {
 
 	private final FiniteDuration lookupTimeout;
 
-	private InetSocketAddress jobManagerAddress;
-
 	private ActorSystem actorSystem;
 
 	private AbstractFlinkYarnCluster yarnCluster;
@@ -202,9 +200,12 @@ public class CliFrontend {
 
 			// get the JobManager address from the YARN properties
 			String address = yarnProperties.getProperty(YARN_PROPERTIES_JOBMANAGER_KEY);
+			InetSocketAddress jobManagerAddress;
 			if (address != null) {
 				try {
-					jobManagerAddress = parseJobManagerAddress(address);
+					jobManagerAddress = parseHostPortAddress(address);
+					// store address in config from where it is retrieved by the retrieval service
+					writeJobManagerAddressToConfig(jobManagerAddress);
 				}
 				catch (Exception e) {
 					throw new Exception("YARN properties contain an invalid entry for JobManager address.", e);
@@ -227,6 +228,24 @@ public class CliFrontend {
 
 
 	// --------------------------------------------------------------------------------------------
+	//  Getter & Setter
+	// --------------------------------------------------------------------------------------------
+
+	/**
+	 * Getter which returns a copy of the associated configuration
+	 *
+	 * @return Copy of the associated configuration
+	 */
+	public Configuration getConfiguration() {
+		Configuration copiedConfiguration = new Configuration();
+
+		copiedConfiguration.addAll(config);
+
+		return copiedConfiguration;
+	}
+
+
+	// --------------------------------------------------------------------------------------------
 	//  Execute Actions
 	// --------------------------------------------------------------------------------------------
 
@@ -688,42 +707,26 @@ public class CliFrontend {
 				new PackagedProgram(jarFile, entryPointClass, programArgs);
 	}
 
-	protected InetSocketAddress getJobManagerAddress(CommandLineOptions options) throws Exception {
-
-		// first, check if the address is specified as an option
-		if (options.getJobManagerAddress() != null) {
-			return parseJobManagerAddress(options.getJobManagerAddress());
-		}
-
-		// second, check whether the address was already parsed, or configured through the YARN properties
-		if (jobManagerAddress == null) {
-			// config file must have the address
-			String jobManagerHost = config.getString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, null);
-
-			// verify that there is a jobmanager address and port in the configuration
-			if (jobManagerHost == null) {
-				throw new Exception("Found no configuration in the config directory '" + configDirectory
-						+ "' that specifies the JobManager address.");
-			}
-
-			int jobManagerPort;
-			try {
-				jobManagerPort = config.getInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, -1);
-			}
-			catch (NumberFormatException e) {
-				throw new Exception("Invalid value for the JobManager port (" +
-						ConfigConstants.JOB_MANAGER_IPC_PORT_KEY + ") in the configuration.");
-			}
-
-			if (jobManagerPort == -1) {
-				throw new Exception("Found no configuration in the config directory '" + configDirectory
-						+ "' that specifies the JobManager port.");
-			}
+	/**
+	 * Writes the given job manager address to the associated configuration object
+	 *
+	 * @param address Address to write to the configuration
+	 */
+	protected void writeJobManagerAddressToConfig(InetSocketAddress address) {
+		config.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, address.getHostName());
+		config.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, address.getPort());
+	}
 
-			jobManagerAddress = new InetSocketAddress(jobManagerHost, jobManagerPort);
+	/**
+	 * Updates the associated configuration with the given command line options
+	 *
+	 * @param options Command line options
+	 */
+	protected void updateConfig(CommandLineOptions options) {
+		if(options.getJobManagerAddress() != null){
+			InetSocketAddress jobManagerAddress = parseHostPortAddress(options.getJobManagerAddress());
+			writeJobManagerAddressToConfig(jobManagerAddress);
 		}
-
-		return jobManagerAddress;
 	}
 
 	/**
@@ -735,16 +738,16 @@ public class CliFrontend {
 	 * @throws Exception
 	 */
 	protected ActorGateway getJobManagerGateway(CommandLineOptions options) throws Exception {
-		//TODO: Get ActorRef from YarnCluster if we are in YARN mode.
-
-		InetSocketAddress address = getJobManagerAddress(options);
+		// overwrite config values with given command line options
+		updateConfig(options);
 
 		// start an actor system if needed
 		if (this.actorSystem == null) {
 			LOG.info("Starting actor system to communicate with JobManager");
 			try {
 				scala.Tuple2<String, Object> systemEndpoint = new scala.Tuple2<String, Object>("", 0);
-				this.actorSystem = AkkaUtils.createActorSystem(config,
+				this.actorSystem = AkkaUtils.createActorSystem(
+						config,
 						new Some<scala.Tuple2<String, Object>>(systemEndpoint));
 			}
 			catch (Exception e) {
@@ -754,20 +757,33 @@ public class CliFrontend {
 			LOG.info("Actor system successfully started");
 		}
 
-		LOG.info("Trying to lookup JobManager");
-		ActorRef jmActor = JobManager.getJobManagerRemoteReference(address, actorSystem, lookupTimeout);
-		LOG.info("JobManager is at " + jmActor.path());
+		LOG.info("Trying to lookup the JobManager gateway");
+		// Retrieve the ActorGateway from the LeaderRetrievalService
+		LeaderRetrievalService lrs = LeaderRetrievalUtils.createLeaderRetrievalService(config);
 
-		// Retrieve the ActorGateway from the JobManager's ActorRef
-		return JobManager.getJobManagerGateway(jmActor, lookupTimeout);
+		return LeaderRetrievalUtils.retrieveLeaderGateway(lrs, actorSystem, lookupTimeout);
 	}
 
 	/**
-	 * @param userParallelism The parallelism requested by the user in the CLI frontend.
+	 * Retrieves a {@link Client} object from the given command line options and other parameters.
+	 *
+	 * @param options Command line options which contain JobManager address
+	 * @param classLoader Class loader to use by the Client
+	 * @param programName Program name
+	 * @param userParallelism Given user parallelism
+	 * @return
+	 * @throws Exception
 	 */
-	protected Client getClient(CommandLineOptions options, ClassLoader classLoader, String programName, int userParallelism) throws Exception {
-		InetSocketAddress jobManagerAddress;
+	protected Client getClient(
+			CommandLineOptions options,
+			ClassLoader classLoader,
+			String programName,
+			int userParallelism)
+		throws Exception {
+		InetSocketAddress jobManagerAddress = null;
+
 		int maxSlots = -1;
+
 		if (YARN_DEPLOY_JOBMANAGER.equals(options.getJobManagerAddress())) {
 			logAndSysout("YARN cluster mode detected. Switching Log4j output to console");
 
@@ -830,9 +846,16 @@ public class CliFrontend {
 			}
 		}
 		else {
-			jobManagerAddress = getJobManagerAddress(options);
+			if(options.getJobManagerAddress() != null) {
+				jobManagerAddress = parseHostPortAddress(options.getJobManagerAddress());
+			}
+		}
+
+		if(jobManagerAddress != null) {
+			writeJobManagerAddressToConfig(jobManagerAddress);
 		}
-		return new Client(jobManagerAddress, config, classLoader, maxSlots);
+
+		return new Client(config, classLoader, maxSlots);
 	}
 
 	// --------------------------------------------------------------------------------------------
@@ -992,7 +1015,14 @@ public class CliFrontend {
 	//  Miscellaneous Utilities
 	// --------------------------------------------------------------------------------------------
 
-	private static InetSocketAddress parseJobManagerAddress(String hostAndPort) {
+	/**
+	 * Parses a given host port address of the format URL:PORT and returns an {@link InetSocketAddress}
+	 *
+	 * @param hostAndPort host port string to be parsed
+	 * @return InetSocketAddress object containing the parsed host port information
+	 */
+	private static InetSocketAddress parseHostPortAddress(String hostAndPort) {
+		// code taken from http://stackoverflow.com/questions/2345063/java-common-way-to-validate-and-convert-hostport-to-inetsocketaddress
 		URI uri;
 		try {
 			uri = new URI("my://" + hostAndPort);

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-clients/src/main/java/org/apache/flink/client/LocalExecutor.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/LocalExecutor.java b/flink-clients/src/main/java/org/apache/flink/client/LocalExecutor.java
index b288996..83e2ee4 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/LocalExecutor.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/LocalExecutor.java
@@ -109,6 +109,7 @@ public class LocalExecutor extends PlanExecutor {
 				}
 				// start it up
 				this.flink = new LocalFlinkMiniCluster(configuration, true);
+				this.flink.start();
 			} else {
 				throw new IllegalStateException("The local executor was already started.");
 			}
@@ -168,7 +169,7 @@ public class LocalExecutor extends PlanExecutor {
 			}
 
 			try {
-				Optimizer pc = new Optimizer(new DataStatistics(), this.flink.getConfiguration());
+				Optimizer pc = new Optimizer(new DataStatistics(), this.flink.configuration());
 				OptimizedPlan op = pc.compile(plan);
 				
 				JobGraphGenerator jgg = new JobGraphGenerator();
@@ -251,7 +252,7 @@ public class LocalExecutor extends PlanExecutor {
 		LocalExecutor exec = new LocalExecutor();
 		try {
 			exec.start();
-			Optimizer pc = new Optimizer(new DataStatistics(), exec.flink.getConfiguration());
+			Optimizer pc = new Optimizer(new DataStatistics(), exec.flink.configuration());
 			OptimizedPlan op = pc.compile(plan);
 			PlanJSONDumpGenerator gen = new PlanJSONDumpGenerator();
 

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-clients/src/main/java/org/apache/flink/client/RemoteExecutor.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/RemoteExecutor.java b/flink-clients/src/main/java/org/apache/flink/client/RemoteExecutor.java
index 373d70c..d1be6d2 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/RemoteExecutor.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/RemoteExecutor.java
@@ -32,6 +32,7 @@ import org.apache.flink.api.common.PlanExecutor;
 import org.apache.flink.client.program.Client;
 import org.apache.flink.client.program.JobWithJars;
 import org.apache.flink.client.program.PackagedProgram;
+import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.optimizer.plan.OptimizedPlan;
 import org.apache.flink.optimizer.plandump.PlanJSONDumpGenerator;
 import org.apache.flink.configuration.Configuration;
@@ -53,7 +54,7 @@ public class RemoteExecutor extends PlanExecutor {
 	private static final Logger LOG = LoggerFactory.getLogger(RemoteExecutor.class);
 
 	private final List<String> jarFiles;
-	private final InetSocketAddress address;
+	private final Configuration configuration;
 	
 	public RemoteExecutor(String hostname, int port) {
 		this(hostname, port, Collections.<String>emptyList());
@@ -73,7 +74,10 @@ public class RemoteExecutor extends PlanExecutor {
 
 	public RemoteExecutor(InetSocketAddress inet, List<String> jarFiles) {
 		this.jarFiles = jarFiles;
-		this.address = inet;
+		configuration = new Configuration();
+
+		configuration.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, inet.getHostName());
+		configuration.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, inet.getPort());
 	}
 
 	@Override
@@ -83,7 +87,7 @@ public class RemoteExecutor extends PlanExecutor {
 	}
 	
 	public JobExecutionResult executePlanWithJars(JobWithJars p) throws Exception {
-		Client c = new Client(this.address, new Configuration(), p.getUserCodeClassLoader(), -1);
+		Client c = new Client(configuration, p.getUserCodeClassLoader(), -1);
 		c.setPrintStatusDuringExecution(isPrintingStatusDuringExecution());
 		
 		JobSubmissionResult result = c.run(p, -1, true);
@@ -99,7 +103,7 @@ public class RemoteExecutor extends PlanExecutor {
 		File jarFile = new File(jarPath);
 		PackagedProgram program = new PackagedProgram(jarFile, assemblerClass, args);
 		
-		Client c = new Client(this.address, new Configuration(), program.getUserCodeClassLoader(), -1);
+		Client c = new Client(configuration, program.getUserCodeClassLoader(), -1);
 		c.setPrintStatusDuringExecution(isPrintingStatusDuringExecution());
 		
 		JobSubmissionResult result = c.run(program.getPlanWithJars(), -1, true);
@@ -114,7 +118,7 @@ public class RemoteExecutor extends PlanExecutor {
 	@Override
 	public String getOptimizerPlanAsJSON(Plan plan) throws Exception {
 		JobWithJars p = new JobWithJars(plan, this.jarFiles);
-		Client c = new Client(this.address, new Configuration(), p.getUserCodeClassLoader(), -1);
+		Client c = new Client(configuration, p.getUserCodeClassLoader(), -1);
 		
 		OptimizedPlan op = (OptimizedPlan) c.getOptimizedPlan(p, -1);
 		PlanJSONDumpGenerator jsonGen = new PlanJSONDumpGenerator();

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-clients/src/main/java/org/apache/flink/client/program/Client.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/Client.java b/flink-clients/src/main/java/org/apache/flink/client/program/Client.java
index e90a39c..2e9ba18 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/program/Client.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/program/Client.java
@@ -22,10 +22,6 @@ import java.io.ByteArrayOutputStream;
 import java.io.File;
 import java.io.IOException;
 import java.io.PrintStream;
-import java.net.InetAddress;
-import java.net.InetSocketAddress;
-import java.net.UnknownHostException;
-import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.TimeUnit;
@@ -33,7 +29,6 @@ import java.util.concurrent.TimeUnit;
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.api.common.JobSubmissionResult;
 import org.apache.flink.api.common.accumulators.AccumulatorHelper;
-import org.apache.flink.configuration.IllegalConfigurationException;
 import org.apache.flink.runtime.akka.AkkaUtils;
 import org.apache.flink.api.common.JobExecutionResult;
 import org.apache.flink.api.common.Plan;
@@ -48,15 +43,16 @@ import org.apache.flink.optimizer.plan.OptimizedPlan;
 import org.apache.flink.optimizer.plan.StreamingPlan;
 import org.apache.flink.optimizer.plandump.PlanJSONDumpGenerator;
 import org.apache.flink.optimizer.plantranslate.JobGraphGenerator;
-import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.core.fs.Path;
 import org.apache.flink.runtime.client.JobClient;
 import org.apache.flink.runtime.client.JobExecutionException;
 import org.apache.flink.runtime.instance.ActorGateway;
 import org.apache.flink.runtime.jobgraph.JobGraph;
-import org.apache.flink.runtime.jobmanager.JobManager;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalException;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
 import org.apache.flink.runtime.messages.JobManagerMessages;
+import org.apache.flink.runtime.util.LeaderRetrievalUtils;
 import org.apache.flink.runtime.messages.accumulators.AccumulatorResultsErroneous;
 import org.apache.flink.runtime.messages.accumulators.AccumulatorResultsFound;
 import org.apache.flink.runtime.messages.accumulators.RequestAccumulatorResults;
@@ -67,7 +63,6 @@ import org.slf4j.LoggerFactory;
 import scala.concurrent.Await;
 import scala.concurrent.Future;
 import scala.concurrent.duration.FiniteDuration;
-import akka.actor.ActorRef;
 import akka.actor.ActorSystem;
 
 import com.google.common.base.Preconditions;
@@ -79,18 +74,18 @@ public class Client {
 	
 	private static final Logger LOG = LoggerFactory.getLogger(Client.class);
 
-	/** The configuration to use for the client (optimizer, timeouts, ...) */
+	/**
+	 * The configuration to use for the client (optimizer, timeouts, ...) and to connect to the
+	 * JobManager.
+	 */
 	private final Configuration configuration;
 
-	/** The address of the JobManager to send the program to */
-	private final InetSocketAddress jobManagerAddress;
-
 	/** The optimizer used in the optimization of batch programs */
 	private final Optimizer compiler;
 
 	/** The class loader to use for classes from the user program (e.g., functions and data types) */
 	private final ClassLoader userCodeClassLoader;
-	
+
 	/** Flag indicating whether to sysout print execution updates */
 	private boolean printStatusDuringExecution = true;
 
@@ -98,7 +93,7 @@ public class Client {
 	 * If != -1, this field specifies the total number of available slots on the cluster
 	 * connected to the client.
 	 */
-	private int maxSlots = -1;
+	private int maxSlots;
 
 	/** ID of the last job submitted with this client. */
 	private JobID lastJobId = null;
@@ -107,85 +102,35 @@ public class Client {
 	// ------------------------------------------------------------------------
 	//                            Construction
 	// ------------------------------------------------------------------------
-	
+
 	/**
-	 * Creates a new instance of the class that submits the jobs to a job-manager.
-	 * at the given address using the default port.
-	 * 
-	 * @param jobManagerAddress Address and port of the job-manager.
+	 * Creates a instance that submits the programs to the JobManager defined in the
+	 * configuration. It sets the maximum number of slots to unknown (= -1).
+	 *
+	 * @param config The config used to obtain the JobManager's address.
+	 * @param userCodeClassLoader The class loader to use for loading user code classes.
 	 */
-	public Client(InetSocketAddress jobManagerAddress, Configuration config, 
-							ClassLoader userCodeClassLoader, int maxSlots) throws UnknownHostException
-	{
-		Preconditions.checkNotNull(jobManagerAddress, "JobManager address is null");
-		Preconditions.checkNotNull(config, "Configuration is null");
-		Preconditions.checkNotNull(userCodeClassLoader, "User code ClassLoader is null");
-		
-		this.configuration = config;
-		
-		if (jobManagerAddress.isUnresolved()) {
-			// address is unresolved, resolve it
-			String host = jobManagerAddress.getHostName();
-			if (host == null) {
-				throw new IllegalArgumentException("Host in jobManagerAddress is null");
-			}
-			
-			try {
-				InetAddress address = InetAddress.getByName(host);
-				this.jobManagerAddress = new InetSocketAddress(address, jobManagerAddress.getPort());
-			}
-			catch (UnknownHostException e) {
-				throw new UnknownHostException("Cannot resolve JobManager host name '" + host + "'.");
-			}
-		}
-		else {
-			// address is already resolved, use it as is
-			this.jobManagerAddress = jobManagerAddress;
-		}
-		
-		this.compiler = new Optimizer(new DataStatistics(), new DefaultCostEstimator(), configuration);
-		this.userCodeClassLoader = userCodeClassLoader;
-		this.maxSlots = maxSlots;
+	public Client(Configuration config, ClassLoader userCodeClassLoader) {
+		this(config, userCodeClassLoader, -1);
 	}
 
 	/**
 	 * Creates a instance that submits the programs to the JobManager defined in the
-	 * configuration. This method will try to resolve the JobManager hostname and throw an exception
-	 * if that is not possible.
+	 * configuration.
 	 * 
-	 * @param config The config used to obtain the job-manager's address.
-	 * @param userCodeClassLoader The class loader to use for loading user code classes.   
+	 * @param config The config used to obtain the JobManager's address.
+	 * @param userCodeClassLoader The class loader to use for loading user code classes.
+	 * @param maxSlots The number of maxSlots on the cluster if != -1
 	 */
-	public Client(Configuration config, ClassLoader userCodeClassLoader) throws UnknownHostException {
+	public Client(Configuration config, ClassLoader userCodeClassLoader, int maxSlots) {
 		Preconditions.checkNotNull(config, "Configuration is null");
 		Preconditions.checkNotNull(userCodeClassLoader, "User code ClassLoader is null");
 		
 		this.configuration = config;
 		this.userCodeClassLoader = userCodeClassLoader;
-		
-		// instantiate the address to the job manager
-		final String address = config.getString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, null);
-		if (address == null) {
-			throw new IllegalConfigurationException(
-					"Cannot find address to job manager's RPC service in the global configuration.");
-		}
-		
-		final int port = config.getInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY,
-														ConfigConstants.DEFAULT_JOB_MANAGER_IPC_PORT);
-		if (port < 0) {
-			throw new IllegalConfigurationException("Cannot find port to job manager's RPC service in the global configuration.");
-		}
-		
-		try {
-			InetAddress inetAddress = InetAddress.getByName(address);
-			this.jobManagerAddress = new InetSocketAddress(inetAddress, port);
-		}
-		catch (UnknownHostException e) {
-			throw new UnknownHostException("Cannot resolve the JobManager hostname '" + address
-					+ "' specified in the configuration");
-		}
 
 		this.compiler = new Optimizer(new DataStatistics(), new DefaultCostEstimator(), configuration);
+		this.maxSlots = maxSlots;
 	}
 
 	/**
@@ -377,8 +322,6 @@ public class Client {
 	public JobSubmissionResult run(JobGraph jobGraph, boolean wait) throws ProgramInvocationException {
 		this.lastJobId = jobGraph.getJobID();
 		
-		LOG.info("JobManager actor system address is " + jobManagerAddress);
-		
 		LOG.info("Starting client actor system");
 		final ActorSystem actorSystem;
 		try {
@@ -388,58 +331,59 @@ public class Client {
 			throw new ProgramInvocationException("Could start client actor system.", e);
 		}
 
-		FiniteDuration timeout = AkkaUtils.getTimeout(configuration);
-
-		LOG.info("Looking up JobManager");
-		ActorGateway jobManagerGateway;
-		ActorRef jobManagerActorRef;
 		try {
-			jobManagerActorRef = JobManager.getJobManagerRemoteReference(
-					jobManagerAddress,
-					actorSystem,
-					configuration);
+			FiniteDuration lookupTimeout = AkkaUtils.getLookupTimeout(configuration);
+			FiniteDuration timeout = AkkaUtils.getTimeout(configuration);
 
+			LOG.info("Looking up JobManager");
+			ActorGateway jobManagerGateway;
 
-		} catch (IOException e) {
-			throw new ProgramInvocationException("Failed to resolve JobManager", e);
-		}
+			LeaderRetrievalService leaderRetrievalService;
 
-		try{
-			jobManagerGateway = JobManager.getJobManagerGateway(jobManagerActorRef, timeout);
-		} catch (Exception e) {
-			throw new ProgramInvocationException("Failed to retrieve the JobManager gateway.", e);
-		}
+			try {
+				leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(configuration);
+			} catch (Exception e) {
+				throw new ProgramInvocationException("Could not create the leader retrieval service.", e);
+			}
 
-		LOG.info("JobManager runs at " + jobManagerGateway.path());
+			try {
+				jobManagerGateway = LeaderRetrievalUtils.retrieveLeaderGateway(
+						leaderRetrievalService,
+						actorSystem,
+						lookupTimeout);
+			} catch (LeaderRetrievalException e) {
+				throw new ProgramInvocationException("Failed to retrieve JobManager gateway", e);
+			}
 
-		LOG.info("Communication between client and JobManager will have a timeout of " + timeout);
+			LOG.info("Leading JobManager actor system address is " + jobManagerGateway.path());
 
-		LOG.info("Checking and uploading JAR files");
-		try {
-			JobClient.uploadJarFiles(jobGraph, jobManagerGateway, timeout);
-		}
-		catch (IOException e) {
-			throw new ProgramInvocationException("Could not upload the program's JAR files to the JobManager.", e);
-		}
+			LOG.info("JobManager runs at " + jobManagerGateway.path());
 
-		try{
-			if (wait) {
-				return JobClient.submitJobAndWait(actorSystem,
-						jobManagerGateway, jobGraph, timeout, printStatusDuringExecution, userCodeClassLoader);
+			LOG.info("Communication between client and JobManager will have a timeout of " + timeout);
+
+			LOG.info("Checking and uploading JAR files");
+			try {
+				JobClient.uploadJarFiles(jobGraph, jobManagerGateway, timeout);
+			} catch (IOException e) {
+				throw new ProgramInvocationException("Could not upload the program's JAR files to the JobManager.", e);
 			}
-			else {
-				JobClient.submitJobDetached(jobManagerGateway, jobGraph, timeout, userCodeClassLoader);
-				// return a dummy execution result with the JobId
-				return new JobSubmissionResult(jobGraph.getJobID());
+
+			try {
+				if (wait) {
+					return JobClient.submitJobAndWait(actorSystem,
+						jobManagerGateway, jobGraph, timeout, printStatusDuringExecution, userCodeClassLoader);
+				} else {
+					JobClient.submitJobDetached(jobManagerGateway, jobGraph, timeout, userCodeClassLoader);
+					// return a dummy execution result with the JobId
+					return new JobSubmissionResult(jobGraph.getJobID());
+				}
+			} catch (JobExecutionException e) {
+				throw new ProgramInvocationException("The program execution failed: " + e.getMessage(), e);
+			} catch (Exception e) {
+				throw new ProgramInvocationException("Exception during program execution.", e);
 			}
-		}
-		catch (JobExecutionException e) {
-			throw new ProgramInvocationException("The program execution failed: " + e.getMessage(), e);
-		}
-		catch (Exception e) {
-			throw new ProgramInvocationException("Exception during program execution.", e);
-		}
-		finally {
+		} finally {
+			// shut down started actor system
 			actorSystem.shutdown();
 			
 			// wait at most for 30 seconds, to work around an occasional akka problem
@@ -454,6 +398,7 @@ public class Client {
 	 */
 	public void cancel(JobID jobId) throws Exception {
 		final FiniteDuration timeout = AkkaUtils.getTimeout(configuration);
+		final FiniteDuration lookupTimeout = AkkaUtils.getLookupTimeout(configuration);
 
 		ActorSystem actorSystem;
 		try {
@@ -462,31 +407,48 @@ public class Client {
 			throw new ProgramInvocationException("Could start client actor system.", e);
 		}
 
-		ActorRef jobManager;
 		try {
-			jobManager = JobManager.getJobManagerRemoteReference(jobManagerAddress, actorSystem, timeout);
-		} catch (Exception e) {
-			throw new ProgramInvocationException("Error getting the remote actor reference for the job manager.", e);
-		}
+			ActorGateway jobManagerGateway;
 
-		Future<Object> response;
-		try {
-			ActorGateway jobManagerGateway = JobManager.getJobManagerGateway(jobManager, timeout);
-			response = jobManagerGateway.ask(new JobManagerMessages.CancelJob(jobId), timeout);
-		} catch (Exception e) {
-			throw new ProgramInvocationException("Failed to query the job manager gateway.", e);
-		}
+			LeaderRetrievalService leaderRetrievalService;
+
+			try {
+				leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(configuration);
+			} catch (Exception e) {
+				throw new ProgramInvocationException("Could not create the leader retrieval service.", e);
+			}
 
-		Object result = Await.result(response, timeout);
+			try {
+				jobManagerGateway = LeaderRetrievalUtils.retrieveLeaderGateway(
+						leaderRetrievalService,
+						actorSystem,
+						lookupTimeout);
+			} catch (LeaderRetrievalException e) {
+				throw new ProgramInvocationException("Failed to retrieve JobManager gateway", e);
+			}
 
-		if (result instanceof JobManagerMessages.CancellationSuccess) {
-			LOG.debug("Job cancellation with ID " + jobId + " succeeded.");
-		} else if (result instanceof JobManagerMessages.CancellationFailure) {
-			Throwable t = ((JobManagerMessages.CancellationFailure) result).cause();
-			LOG.debug("Job cancellation with ID " + jobId + " failed.", t);
-			throw new Exception("Failed to cancel the job because of \n" + t.getMessage());
-		} else {
-			throw new Exception("Unknown message received while cancelling.");
+			Future<Object> response;
+			try {
+				response = jobManagerGateway.ask(new JobManagerMessages.CancelJob(jobId), timeout);
+			} catch (Exception e) {
+				throw new ProgramInvocationException("Failed to query the job manager gateway.", e);
+			}
+			
+			Object result = Await.result(response, timeout);
+
+			if (result instanceof JobManagerMessages.CancellationSuccess) {
+				LOG.debug("Job cancellation with ID " + jobId + " succeeded.");
+			} else if (result instanceof JobManagerMessages.CancellationFailure) {
+				Throwable t = ((JobManagerMessages.CancellationFailure) result).cause();
+				LOG.debug("Job cancellation with ID " + jobId + " failed.", t);
+				throw new Exception("Failed to cancel the job because of \n" + t.getMessage());
+			} else {
+				throw new Exception("Unknown message received while cancelling.");
+			}
+		} finally {
+			// shut down started actor system
+			actorSystem.shutdown();
+			actorSystem.awaitTermination();
 		}
 	}
 
@@ -512,6 +474,7 @@ public class Client {
 	public Map<String, Object> getAccumulators(JobID jobID, ClassLoader loader) throws Exception {
 
 		final FiniteDuration timeout = AkkaUtils.getTimeout(configuration);
+		final FiniteDuration lookupTimeout = AkkaUtils.getLookupTimeout(configuration);
 
 		ActorSystem actorSystem;
 		try {
@@ -520,22 +483,32 @@ public class Client {
 			throw new Exception("Could start client actor system.", e);
 		}
 
-		ActorRef jobManager;
 		try {
-			jobManager = JobManager.getJobManagerRemoteReference(jobManagerAddress, actorSystem, timeout);
-		} catch (Exception e) {
-			throw new Exception("Error getting the remote actor reference for the job manager.", e);
-		}
+			ActorGateway jobManagerGateway;
 
-		Future<Object> response;
-		try {
-			ActorGateway jobManagerGateway = JobManager.getJobManagerGateway(jobManager, timeout);
-			response = jobManagerGateway.ask(new RequestAccumulatorResults(jobID), timeout);
-		} catch (Exception e) {
-			throw new Exception("Failed to query the job manager gateway for accumulators.", e);
-		}
+			LeaderRetrievalService leaderRetrievalService;
 
-		try {
+			try {
+				leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(configuration);
+			} catch (Exception e) {
+				throw new ProgramInvocationException("Could not create the leader retrieval service.", e);
+			}
+
+			try {
+				jobManagerGateway = LeaderRetrievalUtils.retrieveLeaderGateway(
+						leaderRetrievalService,
+						actorSystem,
+						lookupTimeout);
+			} catch (LeaderRetrievalException e) {
+				throw new ProgramInvocationException("Failed to retrieve JobManager gateway", e);
+			}
+
+			Future<Object> response;
+			try {
+				response = jobManagerGateway.ask(new RequestAccumulatorResults(jobID), timeout);
+			} catch (Exception e) {
+				throw new Exception("Failed to query the job manager gateway for accumulators.", e);
+			}
 
 			Object result = Await.result(response, timeout);
 
@@ -548,14 +521,12 @@ public class Client {
 			} else if (result instanceof AccumulatorResultsErroneous) {
 				throw ((AccumulatorResultsErroneous) result).cause();
 			} else {
-				LOG.warn("Failed to fetch accumulators for job {}.", jobID);
+				throw new Exception("Failed to fetch accumulators for the job " + jobID + ".");
 			}
-
-		} catch (Exception e) {
-			LOG.error("Error occurred while fetching accumulators for {}", jobID, e);
+		} finally {
+			actorSystem.shutdown();
+			actorSystem.awaitTermination();
 		}
-
-		return Collections.emptyMap();
 	}
 
 

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-clients/src/test/java/org/apache/flink/client/CliFrontendAddressConfigurationTest.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/test/java/org/apache/flink/client/CliFrontendAddressConfigurationTest.java b/flink-clients/src/test/java/org/apache/flink/client/CliFrontendAddressConfigurationTest.java
index 2d41374..9d0b691 100644
--- a/flink-clients/src/test/java/org/apache/flink/client/CliFrontendAddressConfigurationTest.java
+++ b/flink-clients/src/test/java/org/apache/flink/client/CliFrontendAddressConfigurationTest.java
@@ -19,24 +19,25 @@
 package org.apache.flink.client;
 
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.fail;
 
 import static org.mockito.Mockito.*;
 
 import java.io.File;
-import java.net.InetSocketAddress;
 import java.nio.file.Files;
 import java.nio.file.StandardOpenOption;
-
 import org.apache.flink.client.cli.CommandLineOptions;
 
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.Configuration;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.TemporaryFolder;
 
+import java.net.InetSocketAddress;
+
 /**
  * Tests that verify that the CLI client picks up the correct address for the JobManager
  * from configuration and configs.
@@ -62,13 +63,11 @@ public class CliFrontendAddressConfigurationTest {
 			CliFrontend frontend = new CliFrontend(CliFrontendTestUtils.getInvalidConfigDir());
 			CommandLineOptions options = mock(CommandLineOptions.class);
 
-			try {
-				frontend.getJobManagerAddress(options);
-				fail("we expect an exception here because the we have no config");
-			}
-			catch (Exception e) {
-				// expected
-			}
+			frontend.updateConfig(options);
+			Configuration config = frontend.getConfiguration();
+
+			checkJobManagerAddress(config, null, -1);
+
 		}
 		catch (Exception e) {
 			e.printStackTrace();
@@ -84,7 +83,12 @@ public class CliFrontendAddressConfigurationTest {
 			CommandLineOptions options = mock(CommandLineOptions.class);
 			when(options.getJobManagerAddress()).thenReturn("10.221.130.22:7788");
 
-			assertNotNull(frontend.getJobManagerAddress(options));
+			frontend.updateConfig(options);
+			Configuration config = frontend.getConfiguration();
+
+			InetSocketAddress expectedAddress = new InetSocketAddress("10.221.130.22", 7788);
+
+			checkJobManagerAddress(config, expectedAddress.getHostName(), expectedAddress.getPort());
 		}
 		catch (Exception e) {
 			e.printStackTrace();
@@ -98,11 +102,14 @@ public class CliFrontendAddressConfigurationTest {
 			CliFrontend frontend = new CliFrontend(CliFrontendTestUtils.getConfigDir());
 
 			CommandLineOptions options = mock(CommandLineOptions.class);
-			InetSocketAddress address = frontend.getJobManagerAddress(options);
-			
-			assertNotNull(address);
-			assertEquals(CliFrontendTestUtils.TEST_JOB_MANAGER_ADDRESS, address.getAddress().getHostAddress());
-			assertEquals(CliFrontendTestUtils.TEST_JOB_MANAGER_PORT, address.getPort());
+
+			frontend.updateConfig(options);
+			Configuration config = frontend.getConfiguration();
+
+			checkJobManagerAddress(
+					config,
+					CliFrontendTestUtils.TEST_JOB_MANAGER_ADDRESS,
+					CliFrontendTestUtils.TEST_JOB_MANAGER_PORT);
 		}
 		catch (Exception e) {
 			e.printStackTrace();
@@ -135,11 +142,14 @@ public class CliFrontendAddressConfigurationTest {
 			CliFrontend frontend = new CliFrontend(tmpFolder.getAbsolutePath());
 
 			CommandLineOptions options = mock(CommandLineOptions.class);
-			InetSocketAddress address = frontend.getJobManagerAddress(options);
-			
-			assertNotNull(address);
-			assertEquals(CliFrontendTestUtils.TEST_YARN_JOB_MANAGER_ADDRESS, address.getAddress().getHostAddress());
-			assertEquals(CliFrontendTestUtils.TEST_YARN_JOB_MANAGER_PORT, address.getPort());
+
+			frontend.updateConfig(options);
+			Configuration config = frontend.getConfiguration();
+
+			checkJobManagerAddress(
+					config,
+					CliFrontendTestUtils.TEST_YARN_JOB_MANAGER_ADDRESS,
+					CliFrontendTestUtils.TEST_YARN_JOB_MANAGER_PORT);
 		}
 		catch (Exception e) {
 			e.printStackTrace();
@@ -154,10 +164,14 @@ public class CliFrontendAddressConfigurationTest {
 
 			CommandLineOptions options = mock(CommandLineOptions.class);
 
-			InetSocketAddress address = cli.getJobManagerAddress(options);
+			cli.updateConfig(options);
+
+			Configuration config = cli.getConfiguration();
 
-			assertEquals(CliFrontendTestUtils.TEST_JOB_MANAGER_ADDRESS, address.getAddress().getHostAddress());
-			assertEquals(CliFrontendTestUtils.TEST_JOB_MANAGER_PORT, address.getPort());
+			checkJobManagerAddress(
+				config,
+				CliFrontendTestUtils.TEST_JOB_MANAGER_ADDRESS,
+				CliFrontendTestUtils.TEST_JOB_MANAGER_PORT);
 		}
 		catch (Exception e) {
 			e.printStackTrace();
@@ -173,11 +187,13 @@ public class CliFrontendAddressConfigurationTest {
 			CommandLineOptions options = mock(CommandLineOptions.class);
 			when(options.getJobManagerAddress()).thenReturn("10.221.130.22:7788");
 
-			InetSocketAddress address = frontend.getJobManagerAddress(options);
-			
-			assertNotNull(address);
-			assertEquals("10.221.130.22", address.getAddress().getHostAddress());
-			assertEquals(7788, address.getPort());
+			frontend.updateConfig(options);
+
+			Configuration config = frontend.getConfiguration();
+
+			InetSocketAddress expectedAddress = new InetSocketAddress("10.221.130.22", 7788);
+
+			checkJobManagerAddress(config, expectedAddress.getHostName(), expectedAddress.getPort());
 		}
 		catch (Exception e) {
 			e.printStackTrace();
@@ -193,15 +209,25 @@ public class CliFrontendAddressConfigurationTest {
 			CommandLineOptions options = mock(CommandLineOptions.class);
 			when(options.getJobManagerAddress()).thenReturn("10.221.130.22:7788");
 
-			InetSocketAddress address = frontend.getJobManagerAddress(options);
-			
-			assertNotNull(address);
-			assertEquals("10.221.130.22", address.getAddress().getHostAddress());
-			assertEquals(7788, address.getPort());
+			frontend.updateConfig(options);
+
+			Configuration config = frontend.getConfiguration();
+
+			InetSocketAddress expectedAddress = new InetSocketAddress("10.221.130.22", 7788);
+
+			checkJobManagerAddress(config, expectedAddress.getHostName(), expectedAddress.getPort());
 		}
 		catch (Exception e) {
 			e.printStackTrace();
 			fail(e.getMessage());
 		}
 	}
+
+	public void checkJobManagerAddress(Configuration config, String expectedAddress, int expectedPort) {
+		String jobManagerAddress = config.getString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, null);
+		int jobManagerPort = config.getInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, -1);
+
+		assertEquals(expectedAddress, jobManagerAddress);
+		assertEquals(expectedPort, jobManagerPort);
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-clients/src/test/java/org/apache/flink/client/CliFrontendInfoTest.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/test/java/org/apache/flink/client/CliFrontendInfoTest.java b/flink-clients/src/test/java/org/apache/flink/client/CliFrontendInfoTest.java
index cb2585d..751783c 100644
--- a/flink-clients/src/test/java/org/apache/flink/client/CliFrontendInfoTest.java
+++ b/flink-clients/src/test/java/org/apache/flink/client/CliFrontendInfoTest.java
@@ -22,13 +22,13 @@ import org.apache.flink.client.cli.CommandLineOptions;
 import org.apache.flink.client.program.Client;
 import org.apache.flink.client.program.PackagedProgram;
 import org.apache.flink.client.program.ProgramInvocationException;
+import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.optimizer.CompilerException;
 import org.apache.flink.configuration.Configuration;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
 import java.net.InetAddress;
-import java.net.InetSocketAddress;
 
 import static org.junit.Assert.*;
 
@@ -106,9 +106,13 @@ public class CliFrontendInfoTest {
 
 		@Override
 		protected Client getClient(CommandLineOptions options, ClassLoader loader, String programName, int par)
-				throws Exception
-		{
-			return new TestClient(expectedDop);
+				throws Exception {
+			Configuration config = new Configuration();
+
+			config.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, InetAddress.getLocalHost().getHostName());
+			config.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, 6176);
+
+			return new TestClient(config, expectedDop);
 		}
 	}
 	
@@ -116,9 +120,8 @@ public class CliFrontendInfoTest {
 		
 		private final int expectedDop;
 		
-		private TestClient(int expectedDop) throws Exception {
-			super(new InetSocketAddress(InetAddress.getLocalHost(), 6176),
-					new Configuration(), CliFrontendInfoTest.class.getClassLoader(), -1);
+		private TestClient(Configuration config, int expectedDop) throws Exception {
+			super(config, CliFrontendInfoTest.class.getClassLoader(), -1);
 			
 			this.expectedDop = expectedDop;
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-clients/src/test/java/org/apache/flink/client/CliFrontendListCancelTest.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/test/java/org/apache/flink/client/CliFrontendListCancelTest.java b/flink-clients/src/test/java/org/apache/flink/client/CliFrontendListCancelTest.java
index fc64503..736d859 100644
--- a/flink-clients/src/test/java/org/apache/flink/client/CliFrontendListCancelTest.java
+++ b/flink-clients/src/test/java/org/apache/flink/client/CliFrontendListCancelTest.java
@@ -30,7 +30,6 @@ import org.apache.flink.runtime.messages.JobManagerMessages;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
-import scala.Option;
 
 import java.util.UUID;
 
@@ -83,7 +82,7 @@ public class CliFrontendListCancelTest {
 				JobID jid = new JobID();
 				String jidString = jid.toString();
 
-				final Option<UUID> leaderSessionID = Option.<UUID>apply(UUID.randomUUID());
+				final UUID leaderSessionID = UUID.randomUUID();
 
 				final ActorRef jm = actorSystem.actorOf(Props.create(
 								CliJobManager.class,
@@ -106,7 +105,7 @@ public class CliFrontendListCancelTest {
 				JobID jid1 = new JobID();
 				JobID jid2 = new JobID();
 
-				final Option<UUID> leaderSessionID = Option.<UUID>apply(UUID.randomUUID());
+				final UUID leaderSessionID = UUID.randomUUID();
 
 				final ActorRef jm = actorSystem.actorOf(
 						Props.create(
@@ -143,11 +142,11 @@ public class CliFrontendListCancelTest {
 			
 			// test list properly
 			{
-				final Option<UUID> leaderSessionID = Option.<UUID>apply(UUID.randomUUID());
+				final UUID leaderSessionID = UUID.randomUUID();
 				final ActorRef jm = actorSystem.actorOf(
 						Props.create(
 								CliJobManager.class,
-								(Object)null,
+								null,
 								leaderSessionID
 						)
 				);
@@ -183,9 +182,9 @@ public class CliFrontendListCancelTest {
 
 	protected static final class CliJobManager extends FlinkUntypedActor {
 		private final JobID jobID;
-		private final Option<UUID> leaderSessionID;
+		private final UUID leaderSessionID;
 
-		public CliJobManager(final JobID jobID, final Option<UUID> leaderSessionID){
+		public CliJobManager(final JobID jobID, final UUID leaderSessionID){
 			this.jobID = jobID;
 			this.leaderSessionID = leaderSessionID;
 		}
@@ -217,7 +216,7 @@ public class CliFrontendListCancelTest {
 		}
 
 		@Override
-		protected Option<UUID> getLeaderSessionID() {
+		protected UUID getLeaderSessionID() {
 			return leaderSessionID;
 		}
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-clients/src/test/java/org/apache/flink/client/CliFrontendRunTest.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/test/java/org/apache/flink/client/CliFrontendRunTest.java b/flink-clients/src/test/java/org/apache/flink/client/CliFrontendRunTest.java
index 6798806..a7944ce 100644
--- a/flink-clients/src/test/java/org/apache/flink/client/CliFrontendRunTest.java
+++ b/flink-clients/src/test/java/org/apache/flink/client/CliFrontendRunTest.java
@@ -92,18 +92,18 @@ public class CliFrontendRunTest {
 	
 	public static final class RunTestingCliFrontend extends CliFrontend {
 		
-		private final int expectedParallelim;
+		private final int expectedParallelism;
 		private final boolean sysoutLogging;
 		
-		public RunTestingCliFrontend(int expectedParallelim, boolean logging) throws Exception {
+		public RunTestingCliFrontend(int expectedParallelism, boolean logging) throws Exception {
 			super(CliFrontendTestUtils.getConfigDir());
-			this.expectedParallelim = expectedParallelim;
+			this.expectedParallelism = expectedParallelism;
 			this.sysoutLogging = logging;
 		}
 
 		@Override
 		protected int executeProgram(PackagedProgram program, Client client, int parallelism, boolean wait) {
-			assertEquals(this.expectedParallelim, parallelism);
+			assertEquals(this.expectedParallelism, parallelism);
 			assertEquals(client.getPrintStatusDuringExecution(), sysoutLogging);
 			return 0;
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-clients/src/test/java/org/apache/flink/client/RemoteExecutorHostnameResolutionTest.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/test/java/org/apache/flink/client/RemoteExecutorHostnameResolutionTest.java b/flink-clients/src/test/java/org/apache/flink/client/RemoteExecutorHostnameResolutionTest.java
index 9293148..47236af 100644
--- a/flink-clients/src/test/java/org/apache/flink/client/RemoteExecutorHostnameResolutionTest.java
+++ b/flink-clients/src/test/java/org/apache/flink/client/RemoteExecutorHostnameResolutionTest.java
@@ -21,6 +21,7 @@ package org.apache.flink.client;
 import org.apache.flink.api.common.Plan;
 import org.apache.flink.api.java.ExecutionEnvironment;
 import org.apache.flink.api.java.io.DiscardingOutputFormat;
+import org.apache.flink.client.program.ProgramInvocationException;
 import org.junit.Test;
 
 import java.net.InetAddress;
@@ -28,6 +29,7 @@ import java.net.InetSocketAddress;
 import java.net.UnknownHostException;
 import java.util.Collections;
 
+import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 import static org.junit.Assume.assumeTrue;
 
@@ -45,10 +47,11 @@ public class RemoteExecutorHostnameResolutionTest {
 		try {
 			RemoteExecutor exec = new RemoteExecutor(nonExistingHostname, port);
 			exec.executePlan(getProgram());
-			fail("This should fail with an UnknownHostException");
+			fail("This should fail with an ProgramInvocationException");
 		}
-		catch (UnknownHostException e) {
+		catch (ProgramInvocationException e) {
 			// that is what we want!
+			assertTrue(e.getCause() instanceof UnknownHostException);
 		}
 		catch (Exception e) {
 			System.err.println("Wrong exception!");
@@ -66,10 +69,11 @@ public class RemoteExecutorHostnameResolutionTest {
 			InetSocketAddress add = new InetSocketAddress(nonExistingHostname, port);
 			RemoteExecutor exec = new RemoteExecutor(add, Collections.<String>emptyList());
 			exec.executePlan(getProgram());
-			fail("This should fail with an UnknownHostException");
+			fail("This should fail with an ProgramInvocationException");
 		}
-		catch (UnknownHostException e) {
+		catch (ProgramInvocationException e) {
 			// that is what we want!
+			assertTrue(e.getCause() instanceof UnknownHostException);
 		}
 		catch (Exception e) {
 			System.err.println("Wrong exception!");

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-clients/src/test/java/org/apache/flink/client/program/ClientConnectionTest.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/test/java/org/apache/flink/client/program/ClientConnectionTest.java b/flink-clients/src/test/java/org/apache/flink/client/program/ClientConnectionTest.java
index 39b74a3..1b9fd73 100644
--- a/flink-clients/src/test/java/org/apache/flink/client/program/ClientConnectionTest.java
+++ b/flink-clients/src/test/java/org/apache/flink/client/program/ClientConnectionTest.java
@@ -85,13 +85,16 @@ public class ClientConnectionTest {
 		final Configuration config = new Configuration();
 		config.setString(ConfigConstants.AKKA_ASK_TIMEOUT, (ASK_STARTUP_TIMEOUT/1000) + " s");
 		config.setString(ConfigConstants.AKKA_LOOKUP_TIMEOUT, (CONNECT_TIMEOUT/1000) + " s");
+		config.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, unreachableEndpoint.getHostName());
+		config.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, unreachableEndpoint.getPort());
+
 
 		try {
 			JobVertex vertex = new JobVertex("Test Vertex");
 			vertex.setInvokableClass(TestInvokable.class);
 
 			final JobGraph jg = new JobGraph("Test Job", vertex);
-			final Client client = new Client(unreachableEndpoint, config, getClass().getClassLoader(), -1);
+			final Client client = new Client(config, getClass().getClassLoader(), -1);
 
 			final AtomicReference<Throwable> error = new AtomicReference<Throwable>();
 

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-clients/src/test/java/org/apache/flink/client/program/ClientHostnameResolutionTest.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/test/java/org/apache/flink/client/program/ClientHostnameResolutionTest.java b/flink-clients/src/test/java/org/apache/flink/client/program/ClientHostnameResolutionTest.java
deleted file mode 100644
index 41294e6..0000000
--- a/flink-clients/src/test/java/org/apache/flink/client/program/ClientHostnameResolutionTest.java
+++ /dev/null
@@ -1,98 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.client.program;
-
-import org.apache.flink.configuration.ConfigConstants;
-import org.apache.flink.configuration.Configuration;
-import org.junit.Test;
-
-import java.net.InetAddress;
-import java.net.InetSocketAddress;
-import java.net.UnknownHostException;
-
-import static org.junit.Assert.*;
-import static org.junit.Assume.*;
-
-/**
- * Tests that verify that the client correctly handles non-resolvable host names and does not
- * fail with another exception
- */
-public class ClientHostnameResolutionTest {
-	
-	private static final String nonExistingHostname = "foo.bar.com.invalid";
-	
-	@Test
-	public void testUnresolvableHostname1() {
-		
-		checkPreconditions();
-		
-		try {
-			InetSocketAddress addr = new InetSocketAddress(nonExistingHostname, 17234);
-			new Client(addr, new Configuration(), getClass().getClassLoader(), 1);
-			fail("This should fail with an UnknownHostException");
-		}
-		catch (UnknownHostException e) {
-			// that is what we want!
-		}
-		catch (Exception e) {
-			System.err.println("Wrong exception!");
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-
-	@Test
-	public void testUnresolvableHostname2() {
-
-		checkPreconditions();
-		
-		try {
-			Configuration config = new Configuration();
-			config.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, nonExistingHostname);
-			config.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, 17234);
-			
-			new Client(config, getClass().getClassLoader());
-			fail("This should fail with an UnknownHostException");
-		}
-		catch (UnknownHostException e) {
-			// that is what we want!
-		}
-		catch (Exception e) {
-			System.err.println("Wrong exception!");
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-	
-	private static void checkPreconditions() {
-		// the test can only work if the invalid URL cannot be resolves
-		// some internet providers resolve unresolvable URLs to navigational aid servers,
-		// voiding this test.
-		boolean throwsException;
-		try {
-			//noinspection ResultOfMethodCallIgnored
-			InetAddress.getByName(nonExistingHostname);
-			throwsException = false;
-		}
-		catch (UnknownHostException e) {
-			throwsException = true;
-		}
-		assumeTrue(throwsException);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-clients/src/test/java/org/apache/flink/client/program/ClientTest.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/test/java/org/apache/flink/client/program/ClientTest.java b/flink-clients/src/test/java/org/apache/flink/client/program/ClientTest.java
index 46de93d..a4b8acb 100644
--- a/flink-clients/src/test/java/org/apache/flink/client/program/ClientTest.java
+++ b/flink-clients/src/test/java/org/apache/flink/client/program/ClientTest.java
@@ -52,7 +52,6 @@ import org.mockito.stubbing.Answer;
 import org.powermock.core.classloader.annotations.PrepareForTest;
 import org.powermock.modules.junit4.PowerMockRunner;
 
-import scala.Option;
 import scala.Some;
 import scala.Tuple2;
 
@@ -229,7 +228,7 @@ public class ClientTest {
 
 	public static class SuccessReturningActor extends FlinkUntypedActor {
 
-		private final Option<UUID> leaderSessionID = Option.apply(UUID.randomUUID());
+		private UUID leaderSessionID = null;
 
 		@Override
 		public void handleMessage(Object message) {
@@ -252,14 +251,14 @@ public class ClientTest {
 		}
 
 		@Override
-		protected Option<UUID> getLeaderSessionID() {
+		protected UUID getLeaderSessionID() {
 			return leaderSessionID;
 		}
 	}
 
 	public static class FailureReturningActor extends FlinkUntypedActor {
 
-		private Option<UUID> leaderSessionID = Option.apply(UUID.randomUUID());
+		private UUID leaderSessionID = null;
 
 		@Override
 		public void handleMessage(Object message) {
@@ -270,7 +269,7 @@ public class ClientTest {
 		}
 
 		@Override
-		protected Option<UUID> getLeaderSessionID() {
+		protected UUID getLeaderSessionID() {
 			return leaderSessionID;
 		}
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-clients/src/test/java/org/apache/flink/client/program/ExecutionPlanCreationTest.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/test/java/org/apache/flink/client/program/ExecutionPlanCreationTest.java b/flink-clients/src/test/java/org/apache/flink/client/program/ExecutionPlanCreationTest.java
index 67b406d..d1e971f 100644
--- a/flink-clients/src/test/java/org/apache/flink/client/program/ExecutionPlanCreationTest.java
+++ b/flink-clients/src/test/java/org/apache/flink/client/program/ExecutionPlanCreationTest.java
@@ -28,6 +28,7 @@ import org.apache.flink.api.common.functions.MapFunction;
 import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.ExecutionEnvironment;
 import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.optimizer.plan.OptimizedPlan;
 import org.apache.flink.optimizer.plandump.PlanJSONDumpGenerator;
 import org.apache.flink.configuration.Configuration;
@@ -43,8 +44,13 @@ public class ExecutionPlanCreationTest {
 			
 			InetAddress mockAddress = InetAddress.getLocalHost();
 			InetSocketAddress mockJmAddress = new InetSocketAddress(mockAddress, 12345);
+
+			Configuration config = new Configuration();
+
+			config.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, mockJmAddress.getHostName());
+			config.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, mockJmAddress.getPort());
 			
-			Client client = new Client(mockJmAddress, new Configuration(), getClass().getClassLoader(), -1);
+			Client client = new Client(config, getClass().getClassLoader(), -1);
 			OptimizedPlan op = (OptimizedPlan) client.getOptimizedPlan(prg, -1);
 			assertNotNull(op);
 			

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-clients/src/test/java/org/apache/flink/client/program/LeaderRetrievalServiceHostnameResolutionTest.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/test/java/org/apache/flink/client/program/LeaderRetrievalServiceHostnameResolutionTest.java b/flink-clients/src/test/java/org/apache/flink/client/program/LeaderRetrievalServiceHostnameResolutionTest.java
new file mode 100644
index 0000000..ee26145
--- /dev/null
+++ b/flink-clients/src/test/java/org/apache/flink/client/program/LeaderRetrievalServiceHostnameResolutionTest.java
@@ -0,0 +1,103 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.client.program;
+
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.util.LeaderRetrievalUtils;
+import org.apache.flink.util.TestLogger;
+import org.junit.Test;
+
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+
+import static org.junit.Assert.*;
+import static org.junit.Assume.*;
+
+/**
+ * Tests that verify that the LeaderRetrievalSevice correctly handles non-resolvable host names
+ * and does not fail with another exception
+ */
+public class LeaderRetrievalServiceHostnameResolutionTest extends TestLogger {
+	
+	private static final String nonExistingHostname = "foo.bar.com.invalid";
+	
+	@Test
+	public void testUnresolvableHostname1() {
+		
+		checkPreconditions();
+		
+		try {
+			Configuration config = new Configuration();
+
+			config.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, nonExistingHostname);
+			config.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, 17234);
+
+			LeaderRetrievalUtils.createLeaderRetrievalService(config);
+			fail("This should fail with an UnknownHostException");
+		}
+		catch (UnknownHostException e) {
+			// that is what we want!
+		}
+		catch (Exception e) {
+			System.err.println("Wrong exception!");
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+
+	@Test
+	public void testUnresolvableHostname2() {
+
+		checkPreconditions();
+		
+		try {
+			Configuration config = new Configuration();
+			config.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, nonExistingHostname);
+			config.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, 17234);
+
+			LeaderRetrievalUtils.createLeaderRetrievalService(config);
+			fail("This should fail with an UnknownHostException");
+		}
+		catch (UnknownHostException e) {
+			// that is what we want!
+		}
+		catch (Exception e) {
+			System.err.println("Wrong exception!");
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	private static void checkPreconditions() {
+		// the test can only work if the invalid URL cannot be resolves
+		// some internet providers resolve unresolvable URLs to navigational aid servers,
+		// voiding this test.
+		boolean throwsException;
+		try {
+			//noinspection ResultOfMethodCallIgnored
+			InetAddress.getByName(nonExistingHostname);
+			throwsException = false;
+		}
+		catch (UnknownHostException e) {
+			throwsException = true;
+		}
+		assumeTrue(throwsException);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkClient.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkClient.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkClient.java
index 51a4fa1..99e4906 100644
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkClient.java
+++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkClient.java
@@ -55,7 +55,6 @@ import scala.concurrent.duration.FiniteDuration;
 import java.io.File;
 import java.io.IOException;
 import java.net.InetSocketAddress;
-import java.net.UnknownHostException;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
@@ -184,12 +183,14 @@ public class FlinkClient {
 		final Configuration configuration = jobGraph.getJobConfiguration();
 
 		final Client client;
-		try {
-			client = new Client(new InetSocketAddress(this.jobManagerHost, this.jobManagerPort), configuration,
-					JobWithJars.buildUserCodeClassLoader(jarFiles, JobWithJars.class.getClassLoader()), -1);
-		} catch (final UnknownHostException e) {
-			throw new RuntimeException("Cannot execute job due to UnknownHostException", e);
-		}
+
+		configuration.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, jobManagerHost);
+		configuration.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, jobManagerPort);
+
+		client = new Client(
+			configuration,
+			JobWithJars.buildUserCodeClassLoader(jarFiles, JobWithJars.class.getClassLoader()),
+			-1);
 
 		try {
 			client.run(jobGraph, false);
@@ -302,7 +303,7 @@ public class FlinkClient {
 			throw new RuntimeException("Could not start actor system to communicate with JobManager", e);
 		}
 
-		return JobManager.getJobManagerRemoteReference(new InetSocketAddress(this.jobManagerHost, this.jobManagerPort),
+		return JobManager.getJobManagerActorRef(new InetSocketAddress(this.jobManagerHost, this.jobManagerPort),
 				actorSystem, AkkaUtils.getLookupTimeout(configuration));
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-contrib/flink-streaming-contrib/src/test/java/org/apache/flink/contrib/streaming/CollectITCase.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-streaming-contrib/src/test/java/org/apache/flink/contrib/streaming/CollectITCase.java b/flink-contrib/flink-streaming-contrib/src/test/java/org/apache/flink/contrib/streaming/CollectITCase.java
index f21e58c..fab5c9a 100644
--- a/flink-contrib/flink-streaming-contrib/src/test/java/org/apache/flink/contrib/streaming/CollectITCase.java
+++ b/flink-contrib/flink-streaming-contrib/src/test/java/org/apache/flink/contrib/streaming/CollectITCase.java
@@ -22,7 +22,6 @@ import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.test.util.ForkableFlinkMiniCluster;
 import org.junit.Test;
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.contrib.streaming.DataStreamUtils;
 import org.apache.flink.streaming.api.datastream.DataStream;
 import org.junit.Assert;
 
@@ -38,9 +37,10 @@ public class CollectITCase {
 
 		Configuration config = new Configuration();
 		ForkableFlinkMiniCluster cluster = new ForkableFlinkMiniCluster(config, false);
+		cluster.start();
 
 		final StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment(
-				"localhost", cluster.getJobManagerRPCPort());
+				"localhost", cluster.getLeaderRPCPort());
 
 		long N = 10;
 		DataStream<Long> stream = env.generateSequence(1, N);


[04/10] flink git commit: [FLINK-2291] [runtime] Add ZooKeeper support to elect a leader from a set of JobManager. The leader will then be retrieved from ZooKeeper by the TaskManagers.

Posted by tr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerStartupTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerStartupTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerStartupTest.java
new file mode 100644
index 0000000..a3b65f0
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerStartupTest.java
@@ -0,0 +1,168 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.taskmanager;
+
+import static org.junit.Assert.*;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.IllegalConfigurationException;
+import org.apache.flink.runtime.StreamingMode;
+import org.junit.Test;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.ServerSocket;
+import java.util.UUID;
+
+/**
+ * Tests that check how the TaskManager behaves when encountering startup problems.
+ */
+public class TaskManagerStartupTest {
+
+	/**
+	 * Tests that the TaskManager fails synchronously when the actor system port
+	 * is in use.
+	 */
+	@Test
+	public void testStartupWhenTaskmanagerActorPortIsUsed() {
+		ServerSocket blocker = null;
+		try {
+			final String localHostName = "localhost";
+			final InetAddress localAddress = InetAddress.getByName(localHostName);
+
+			// block some port
+			blocker = new ServerSocket(0, 50, localAddress);
+			final int port = blocker.getLocalPort();
+
+			try {
+				TaskManager.runTaskManager(localHostName, port, new Configuration(),
+											StreamingMode.BATCH_ONLY, TaskManager.class);
+				fail("This should fail with an IOException");
+			}
+			catch (IOException e) {
+				// expected. validate the error message
+				assertNotNull(e.getMessage());
+				assertTrue(e.getMessage().contains("Address already in use"));
+			}
+
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+		finally {
+			if (blocker != null) {
+				try {
+					blocker.close();
+				}
+				catch (IOException e) {
+					// no need to log here
+				}
+			}
+		}
+	}
+
+	/**
+	 * Tests that the TaskManager startup fails synchronously when the I/O directories are
+	 * not writable.
+	 */
+	@Test
+	public void testIODirectoryNotWritable() {
+		File tempDir = new File(ConfigConstants.DEFAULT_TASK_MANAGER_TMP_PATH);
+		File nonWritable = new File(tempDir, UUID.randomUUID().toString());
+
+		if (!nonWritable.mkdirs() || !nonWritable.setWritable(false, false)) {
+			System.err.println("Cannot create non-writable temporary file directory. Skipping test.");
+			return;
+		}
+
+		try {
+			Configuration cfg = new Configuration();
+			cfg.setString(ConfigConstants.TASK_MANAGER_TMP_DIR_KEY, nonWritable.getAbsolutePath());
+			cfg.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 4);
+			cfg.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, "localhost");
+			cfg.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, 21656);
+
+			try {
+				TaskManager.runTaskManager("localhost", 0, cfg, StreamingMode.BATCH_ONLY);
+				fail("Should fail synchronously with an exception");
+			}
+			catch (IOException e) {
+				// splendid!
+			}
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+		finally {
+			//noinspection ResultOfMethodCallIgnored
+			nonWritable.setWritable(true, false);
+			try {
+				FileUtils.deleteDirectory(nonWritable);
+			}
+			catch (IOException e) {
+				// best effort
+			}
+		}
+	}
+
+	/**
+	 * Tests that the TaskManager startup fails synchronously when the I/O directories are
+	 * not writable.
+	 */
+	@Test
+	public void testMemoryConfigWrong() {
+		try {
+			Configuration cfg = new Configuration();
+			cfg.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, "localhost");
+			cfg.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, 21656);
+
+			// something invalid
+			cfg.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, -42);
+			try {
+				TaskManager.runTaskManager("localhost", 0, cfg, StreamingMode.BATCH_ONLY);
+				fail("Should fail synchronously with an exception");
+			}
+			catch (IllegalConfigurationException e) {
+				// splendid!
+			}
+
+			// something ridiculously high
+			final long memSize = (((long) Integer.MAX_VALUE - 1) *
+									ConfigConstants.DEFAULT_TASK_MANAGER_NETWORK_BUFFER_SIZE) >> 20;
+			cfg.setLong(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, memSize);
+			try {
+				TaskManager.runTaskManager("localhost", 0, cfg, StreamingMode.BATCH_ONLY);
+				fail("Should fail synchronously with an exception");
+			}
+			catch (Exception e) {
+				// splendid!
+				assertTrue(e.getCause() instanceof OutOfMemoryError);
+			}
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java
index a8a852f..6d811b0 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java
@@ -20,19 +20,13 @@ package org.apache.flink.runtime.taskmanager;
 
 import akka.actor.ActorRef;
 import akka.actor.ActorSystem;
-import akka.actor.Kill;
 import akka.actor.Props;
 import akka.japi.Creator;
-import akka.pattern.Patterns;
 import akka.testkit.JavaTestKit;
-import akka.util.Timeout;
 
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.LeaderSessionMessageDecorator;
-import org.apache.flink.runtime.MessageDecorator;
-import org.apache.flink.runtime.StreamingMode;
 import org.apache.flink.runtime.akka.AkkaUtils;
 import org.apache.flink.runtime.akka.FlinkUntypedActor;
 import org.apache.flink.runtime.blob.BlobKey;
@@ -43,6 +37,8 @@ import org.apache.flink.runtime.deployment.ResultPartitionLocation;
 import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor;
 import org.apache.flink.runtime.execution.ExecutionState;
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.instance.ActorGateway;
+import org.apache.flink.runtime.instance.AkkaActorGateway;
 import org.apache.flink.runtime.instance.InstanceID;
 import org.apache.flink.runtime.io.network.ConnectionID;
 import org.apache.flink.runtime.io.network.partition.PartitionNotFoundException;
@@ -62,9 +58,9 @@ import org.apache.flink.runtime.messages.TaskMessages.PartitionState;
 import org.apache.flink.runtime.messages.TaskMessages.SubmitTask;
 import org.apache.flink.runtime.messages.TaskMessages.TaskOperationResult;
 import org.apache.flink.runtime.net.NetUtils;
-import org.apache.flink.runtime.testingUtils.TestingTaskManager;
 import org.apache.flink.runtime.testingUtils.TestingTaskManagerMessages;
 
+import org.apache.flink.runtime.testingUtils.TestingUtils;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -73,7 +69,6 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import scala.Option;
-import scala.Some;
 import scala.concurrent.Await;
 import scala.concurrent.Future;
 import scala.concurrent.duration.FiniteDuration;
@@ -90,7 +85,6 @@ import java.util.concurrent.TimeUnit;
 
 import static org.apache.flink.runtime.messages.JobManagerMessages.RequestPartitionState;
 import static org.apache.flink.runtime.messages.JobManagerMessages.ScheduleOrUpdateConsumers;
-import static org.apache.flink.runtime.messages.TaskMessages.UpdateTaskExecutionState;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.fail;
@@ -100,16 +94,14 @@ public class TaskManagerTest {
 
 	private static final Logger LOG = LoggerFactory.getLogger(TaskManagerTest.class);
 	
-	private static final Timeout timeout = new Timeout(1, TimeUnit.MINUTES);
+	private static final FiniteDuration timeout = new FiniteDuration(1, TimeUnit.MINUTES);
 
 	private static final FiniteDuration d = new FiniteDuration(20, TimeUnit.SECONDS);
 
 	private static ActorSystem system;
 
-	final static Option<UUID> leaderSessionID = new Some<UUID>(UUID.randomUUID());
+	final static UUID leaderSessionID = null;
 
-	final static MessageDecorator decorator = new LeaderSessionMessageDecorator(leaderSessionID);
-	
 	@BeforeClass
 	public static void setup() {
 		system = AkkaUtils.createLocalActorSystem(new Configuration());
@@ -130,29 +122,35 @@ public class TaskManagerTest {
 		
 		new JavaTestKit(system){{
 
-			ActorRef taskManager = null;
+			ActorGateway taskManager = null;
+			final ActorGateway jobManager = TestingUtils.createForwardingJobManager(
+					system,
+					getTestActor(),
+					Option.<String>empty());
 			
 			try {
-				taskManager = createTaskManager(getTestActor(), false);
-				final ActorRef tm = taskManager;
+				taskManager = TestingUtils.createTaskManager(
+						system,
+						jobManager,
+						new Configuration(),
+						true,
+						false);
+
+				final ActorGateway tm = taskManager;
 				
 				// handle the registration
 				new Within(d) {
 					@Override
 					protected void run() {
-						RegistrationMessages.RegisterTaskManager registrationMessage =
-								expectMsgClass(RegistrationMessages.RegisterTaskManager.class);
+						expectMsgClass(RegistrationMessages.RegisterTaskManager.class);
 						
 						final InstanceID iid = new InstanceID();
-						assertEquals(tm, getLastSender());
+						assertEquals(tm.actor(), getLastSender());
 						tm.tell(
 								new RegistrationMessages.AcknowledgeRegistration(
-										registrationMessage.registrationSessionID(),
-										leaderSessionID.get(),
-										getTestActor(),
 										iid,
 										12345),
-								getTestActor());
+								jobManager);
 					}
 				};
 
@@ -171,7 +169,7 @@ public class TaskManagerTest {
 
 					@Override
 					protected void run() {
-						tm.tell(decorator.decorate(new SubmitTask(tdd)), getRef());
+						tm.tell(new SubmitTask(tdd), jobManager);
 						
 						// TaskManager should acknowledge the submission
 						// heartbeats may be interleaved
@@ -184,14 +182,12 @@ public class TaskManagerTest {
 						} while (System.currentTimeMillis() < deadline);
 
 						// task should have switched to running
-						Object toRunning = decorator.decorate(
-								new TaskMessages.UpdateTaskExecutionState(
-										new TaskExecutionState(jid, eid, ExecutionState.RUNNING)));
+						Object toRunning = new TaskMessages.UpdateTaskExecutionState(
+										new TaskExecutionState(jid, eid, ExecutionState.RUNNING));
 
 						// task should have switched to finished
-						Object toFinished = decorator.decorate(
-								new TaskMessages.UpdateTaskExecutionState(
-										new TaskExecutionState(jid, eid, ExecutionState.FINISHED)));
+						Object toFinished = new TaskMessages.UpdateTaskExecutionState(
+										new TaskExecutionState(jid, eid, ExecutionState.FINISHED));
 						
 						deadline = System.currentTimeMillis() + 10000;
 						do {
@@ -225,9 +221,8 @@ public class TaskManagerTest {
 			}
 			finally {
 				// shut down the actors
-				if (taskManager != null) {
-					taskManager.tell(Kill.getInstance(), ActorRef.noSender());
-				}
+				TestingUtils.stopActor(taskManager);
+				TestingUtils.stopActor(jobManager);
 			}
 		}};
 	}
@@ -241,11 +236,23 @@ public class TaskManagerTest {
 		
 		new JavaTestKit(system){{
 
-			ActorRef jobManager = null;
-			ActorRef taskManager = null;
+			ActorGateway jobManager = null;
+			ActorGateway taskManager = null;
+
+			final ActorGateway testActorGateway = new AkkaActorGateway(
+					getTestActor(),
+					leaderSessionID);
+
 			try {
-				jobManager = system.actorOf(Props.create(SimpleJobManager.class, leaderSessionID.get()));
-				taskManager = createTaskManager(jobManager, true);
+				ActorRef jm = system.actorOf(Props.create(SimpleJobManager.class, leaderSessionID));
+				jobManager = new AkkaActorGateway(jm, leaderSessionID);
+
+				taskManager = TestingUtils.createTaskManager(
+						system,
+						jobManager,
+						new Configuration(),
+						true,
+						true);
 
 				final JobID jid1 = new JobID();
 				final JobID jid2 = new JobID();
@@ -268,24 +275,22 @@ public class TaskManagerTest {
 						Collections.<InputGateDeploymentDescriptor>emptyList(),
 					new ArrayList<BlobKey>(), 0);
 
-				final ActorRef tm = taskManager;
+				final ActorGateway tm = taskManager;
 
 				new Within(d) {
 
 					@Override
 					protected void run() {
 						try {
-							Future<Object> t1Running = Patterns.ask(
-									tm,
+							Future<Object> t1Running = tm.ask(
 									new TestingTaskManagerMessages.NotifyWhenTaskIsRunning(eid1),
 									timeout);
-							Future<Object> t2Running = Patterns.ask(
-									tm,
+							Future<Object> t2Running = tm.ask(
 									new TestingTaskManagerMessages.NotifyWhenTaskIsRunning(eid2),
 									timeout);
 
-							tm.tell(decorator.decorate(new SubmitTask(tdd1)), getRef());
-							tm.tell(decorator.decorate(new SubmitTask(tdd2)), getRef());
+							tm.tell(new SubmitTask(tdd1), testActorGateway);
+							tm.tell(new SubmitTask(tdd2), testActorGateway);
 
 							expectMsgEquals(Messages.getAcknowledge());
 							expectMsgEquals(Messages.getAcknowledge());
@@ -293,7 +298,7 @@ public class TaskManagerTest {
 							Await.ready(t1Running, d);
 							Await.ready(t2Running, d);
 							
-							tm.tell(TestingTaskManagerMessages.getRequestRunningTasksMessage(), getRef());
+							tm.tell(TestingTaskManagerMessages.getRequestRunningTasksMessage(), testActorGateway);
 
 							Map<ExecutionAttemptID, Task> runningTasks = expectMsgClass(TestingTaskManagerMessages
 									.ResponseRunningTasks.class).asJava();
@@ -307,36 +312,36 @@ public class TaskManagerTest {
 							assertEquals(ExecutionState.RUNNING, t1.getExecutionState());
 							assertEquals(ExecutionState.RUNNING, t2.getExecutionState());
 
-							tm.tell(decorator.decorate(new CancelTask(eid1)), getRef());
+							tm.tell(new CancelTask(eid1), testActorGateway);
 
 							expectMsgEquals(new TaskOperationResult(eid1, true));
 
-							Future<Object> response = Patterns.ask(tm, new TestingTaskManagerMessages.NotifyWhenTaskRemoved(eid1),
+							Future<Object> response = tm.ask(new TestingTaskManagerMessages.NotifyWhenTaskRemoved(eid1),
 									timeout);
 							Await.ready(response, d);
 
 							assertEquals(ExecutionState.CANCELED, t1.getExecutionState());
 
-							tm.tell(TestingTaskManagerMessages.getRequestRunningTasksMessage(), getRef());
+							tm.tell(TestingTaskManagerMessages.getRequestRunningTasksMessage(), testActorGateway);
 							runningTasks = expectMsgClass(TestingTaskManagerMessages
 									.ResponseRunningTasks.class).asJava();
 
 							assertEquals(1, runningTasks.size());
 
-							tm.tell(decorator.decorate(new CancelTask(eid1)), getRef());
+							tm.tell(new CancelTask(eid1), testActorGateway);
 							expectMsgEquals(new TaskOperationResult(eid1, false, "No task with that execution ID was " +
 									"found."));
 
-							tm.tell(decorator.decorate(new CancelTask(eid2)), getRef());
+							tm.tell(new CancelTask(eid2), testActorGateway);
 							expectMsgEquals(new TaskOperationResult(eid2, true));
 
-							response = Patterns.ask(tm, new TestingTaskManagerMessages.NotifyWhenTaskRemoved(eid2),
+							response = tm.ask(new TestingTaskManagerMessages.NotifyWhenTaskRemoved(eid2),
 									timeout);
 							Await.ready(response, d);
 
 							assertEquals(ExecutionState.CANCELED, t2.getExecutionState());
 
-							tm.tell(TestingTaskManagerMessages.getRequestRunningTasksMessage(), getRef());
+							tm.tell(TestingTaskManagerMessages.getRequestRunningTasksMessage(), testActorGateway);
 							runningTasks = expectMsgClass(TestingTaskManagerMessages
 									.ResponseRunningTasks.class).asJava();
 
@@ -353,13 +358,8 @@ public class TaskManagerTest {
 				fail(e.getMessage());
 			}
 			finally {
-				// shut down the actors
-				if (taskManager != null) {
-					taskManager.tell(Kill.getInstance(), ActorRef.noSender());
-				}
-				if (jobManager != null) {
-					jobManager.tell(Kill.getInstance(), ActorRef.noSender());
-				}
+				TestingUtils.stopActor(taskManager);
+				TestingUtils.stopActor(jobManager);
 			}
 		}};
 	}
@@ -373,13 +373,25 @@ public class TaskManagerTest {
 		
 		new JavaTestKit(system){{
 
-			ActorRef jobManager = null;
-			ActorRef taskManager = null;
+			ActorGateway jobManager = null;
+			ActorGateway taskManager = null;
+
+			final ActorGateway testActorGateway = new AkkaActorGateway(
+					getTestActor(),
+					leaderSessionID);
+
 			try {
-				jobManager = system.actorOf(Props.create(SimpleJobManager.class, leaderSessionID.get()));
+				ActorRef jm = system.actorOf(Props.create(SimpleJobManager.class, leaderSessionID));
+				jobManager = new AkkaActorGateway(jm, leaderSessionID);
 
-				taskManager = createTaskManager(jobManager, true);
-				final ActorRef tm = taskManager;
+				taskManager = TestingUtils.createTaskManager(
+						system,
+						jobManager,
+						new Configuration(),
+						true,
+						true);
+
+				final ActorGateway tm = taskManager;
 
 				final JobID jid = new JobID();
 
@@ -406,21 +418,21 @@ public class TaskManagerTest {
 					@Override
 					protected void run() {
 						try {
-							tm.tell(decorator.decorate(new SubmitTask(tdd1)), getRef());
-							tm.tell(decorator.decorate(new SubmitTask(tdd2)), getRef());
+							tm.tell(new SubmitTask(tdd1), testActorGateway);
+							tm.tell(new SubmitTask(tdd2), testActorGateway);
 
 							expectMsgEquals(Messages.getAcknowledge());
 							expectMsgEquals(Messages.getAcknowledge());
 
 							tm.tell(new TestingTaskManagerMessages.NotifyWhenTaskRemoved(eid1),
-									getRef());
+									testActorGateway);
 							tm.tell(new TestingTaskManagerMessages.NotifyWhenTaskRemoved(eid2),
-									getRef());
+									testActorGateway);
 
 							expectMsgEquals(true);
 							expectMsgEquals(true);
 
-							tm.tell(TestingTaskManagerMessages.getRequestRunningTasksMessage(), getRef());
+							tm.tell(TestingTaskManagerMessages.getRequestRunningTasksMessage(), testActorGateway);
 							Map<ExecutionAttemptID, Task> tasks = expectMsgClass(TestingTaskManagerMessages
 									.ResponseRunningTasks.class).asJava();
 
@@ -438,12 +450,8 @@ public class TaskManagerTest {
 			}
 			finally {
 				// shut down the actors
-				if (taskManager != null) {
-					taskManager.tell(Kill.getInstance(), ActorRef.noSender());
-				}
-				if (jobManager != null) {
-					jobManager.tell(Kill.getInstance(), ActorRef.noSender());
-				}
+				TestingUtils.stopActor(taskManager);
+				TestingUtils.stopActor(jobManager);
 			}
 		}};
 	}
@@ -457,8 +465,12 @@ public class TaskManagerTest {
 		
 		new JavaTestKit(system){{
 
-			ActorRef jobManager = null;
-			ActorRef taskManager = null;
+			ActorGateway jobManager = null;
+			ActorGateway taskManager = null;
+
+			final ActorGateway testActorGateway = new AkkaActorGateway(
+					getTestActor(),
+					leaderSessionID);
 			try {
 				final JobID jid = new JobID();
 
@@ -468,10 +480,17 @@ public class TaskManagerTest {
 				final ExecutionAttemptID eid1 = new ExecutionAttemptID();
 				final ExecutionAttemptID eid2 = new ExecutionAttemptID();
 
-				jobManager = system.actorOf(Props.create(new SimpleLookupJobManagerCreator(leaderSessionID.get())));
+				ActorRef jm = system.actorOf(Props.create(new SimpleLookupJobManagerCreator(leaderSessionID)));
+				jobManager = new AkkaActorGateway(jm, leaderSessionID);
+
+				taskManager = TestingUtils.createTaskManager(
+						system,
+						jobManager,
+						new Configuration(),
+						true,
+						true);
 
-				taskManager = createTaskManager(jobManager, true);
-				final ActorRef tm = taskManager;
+				final ActorGateway tm = taskManager;
 
 				IntermediateResultPartitionID partitionId = new IntermediateResultPartitionID();
 
@@ -501,30 +520,29 @@ public class TaskManagerTest {
 					@Override
 					protected void run() {
 						try {
-							Future<Object> t1Running = Patterns.ask(
-									tm,
+							Future<Object> t1Running = tm.ask(
 									new TestingTaskManagerMessages.NotifyWhenTaskIsRunning(eid1),
 									timeout);
-							Future<Object> t2Running = Patterns.ask(
-									tm,
+
+							Future<Object> t2Running = tm.ask(
 									new TestingTaskManagerMessages.NotifyWhenTaskIsRunning(eid2),
 									timeout);
 
 							// submit the sender task
-							tm.tell(decorator.decorate(new SubmitTask(tdd1)), getRef());
+							tm.tell(new SubmitTask(tdd1), testActorGateway);
 							expectMsgEquals(Messages.getAcknowledge());
 
 							// wait until the sender task is running
 							Await.ready(t1Running, d);
 
 							// only now (after the sender is running), submit the receiver task
-							tm.tell(decorator.decorate(new SubmitTask(tdd2)), getRef());
+							tm.tell(new SubmitTask(tdd2), testActorGateway);
 							expectMsgEquals(Messages.getAcknowledge());
 							
 							// wait until the receiver task is running
 							Await.ready(t2Running, d);
 
-							tm.tell(TestingTaskManagerMessages.getRequestRunningTasksMessage(), getRef());
+							tm.tell(TestingTaskManagerMessages.getRequestRunningTasksMessage(), testActorGateway);
 							Map<ExecutionAttemptID, Task> tasks = expectMsgClass(TestingTaskManagerMessages.ResponseRunningTasks
 									.class).asJava();
 
@@ -534,19 +552,19 @@ public class TaskManagerTest {
 							// wait until the tasks are done. thread races may cause the tasks to be done before
 							// we get to the check, so we need to guard the check
 							if (t1 != null) {
-								Future<Object> response = Patterns.ask(tm, new TestingTaskManagerMessages.NotifyWhenTaskRemoved(eid1),
+								Future<Object> response = tm.ask(new TestingTaskManagerMessages.NotifyWhenTaskRemoved(eid1),
 										timeout);
 								Await.ready(response, d);
 							}
 
 							if (t2 != null) {
-								Future<Object> response = Patterns.ask(tm, new TestingTaskManagerMessages.NotifyWhenTaskRemoved(eid2),
+								Future<Object> response = tm.ask(new TestingTaskManagerMessages.NotifyWhenTaskRemoved(eid2),
 										timeout);
 								Await.ready(response, d);
 								assertEquals(ExecutionState.FINISHED, t2.getExecutionState());
 							}
 
-							tm.tell(TestingTaskManagerMessages.getRequestRunningTasksMessage(), getRef());
+							tm.tell(TestingTaskManagerMessages.getRequestRunningTasksMessage(), testActorGateway);
 							tasks = expectMsgClass(TestingTaskManagerMessages.ResponseRunningTasks
 									.class).asJava();
 
@@ -565,12 +583,8 @@ public class TaskManagerTest {
 			}
 			finally {
 				// shut down the actors
-				if (taskManager != null) {
-					taskManager.tell(Kill.getInstance(), ActorRef.noSender());
-				}
-				if (jobManager != null) {
-					jobManager.tell(Kill.getInstance(), ActorRef.noSender());
-				}
+				TestingUtils.stopActor(taskManager);
+				TestingUtils.stopActor(jobManager);
 			}
 		}};
 	}
@@ -587,8 +601,12 @@ public class TaskManagerTest {
 		// the second one blocks to be canceled
 		new JavaTestKit(system){{
 
-			ActorRef jobManager = null;
-			ActorRef taskManager = null;
+			ActorGateway jobManager = null;
+			ActorGateway taskManager = null;
+
+			final ActorGateway testActorGateway = new AkkaActorGateway(
+					getTestActor(),
+					leaderSessionID);
 			try {
 				final JobID jid = new JobID();
 
@@ -598,15 +616,24 @@ public class TaskManagerTest {
 				final ExecutionAttemptID eid1 = new ExecutionAttemptID();
 				final ExecutionAttemptID eid2 = new ExecutionAttemptID();
 
-				jobManager = system.actorOf(
+				ActorRef jm = system.actorOf(
 						Props.create(
 								new SimpleLookupFailingUpdateJobManagerCreator(
-										leaderSessionID.get(),
+										leaderSessionID,
 										eid2)
 						)
 				);
-				taskManager = createTaskManager(jobManager, true);
-				final ActorRef tm = taskManager;
+
+				jobManager = new AkkaActorGateway(jm, leaderSessionID);
+
+				taskManager = TestingUtils.createTaskManager(
+						system,
+						jobManager,
+						new Configuration(),
+						true,
+						true);
+
+				final ActorGateway tm = taskManager;
 
 				IntermediateResultPartitionID partitionId = new IntermediateResultPartitionID();
 
@@ -637,17 +664,16 @@ public class TaskManagerTest {
 					@Override
 					protected void run() {
 						try {
-							Future<Object> t1Running = Patterns.ask(
-									tm,
+							Future<Object> t1Running = tm.ask(
 									new TestingTaskManagerMessages.NotifyWhenTaskIsRunning(eid1),
 									timeout);
-							Future<Object> t2Running = Patterns.ask(
-									tm,
+
+							Future<Object> t2Running = tm.ask(
 									new TestingTaskManagerMessages.NotifyWhenTaskIsRunning(eid2),
 									timeout);
 
-							tm.tell(decorator.decorate(new SubmitTask(tdd2)), getRef());
-							tm.tell(decorator.decorate(new SubmitTask(tdd1)), getRef());
+							tm.tell(new SubmitTask(tdd2), testActorGateway);
+							tm.tell(new SubmitTask(tdd1), testActorGateway);
 
 							expectMsgEquals(Messages.getAcknowledge());
 							expectMsgEquals(Messages.getAcknowledge());
@@ -655,35 +681,34 @@ public class TaskManagerTest {
 							Await.ready(t1Running, d);
 							Await.ready(t2Running, d);
 
-							tm.tell(TestingTaskManagerMessages.getRequestRunningTasksMessage(), getRef());
+							tm.tell(TestingTaskManagerMessages.getRequestRunningTasksMessage(), testActorGateway);
 							Map<ExecutionAttemptID, Task> tasks = expectMsgClass(TestingTaskManagerMessages
 									.ResponseRunningTasks.class).asJava();
 
 							Task t1 = tasks.get(eid1);
 							Task t2 = tasks.get(eid2);
 
-							tm.tell(decorator.decorate(new CancelTask(eid2)), getRef());
+							tm.tell(new CancelTask(eid2), testActorGateway);
 							expectMsgEquals(new TaskOperationResult(eid2, true));
 
 							if (t2 != null) {
-								Future<Object> response = Patterns.ask(tm, new TestingTaskManagerMessages.NotifyWhenTaskRemoved(eid2),
+								Future<Object> response = tm.ask(new TestingTaskManagerMessages.NotifyWhenTaskRemoved(eid2),
 										timeout);
 								Await.ready(response, d);
 							}
 
 							if (t1 != null) {
 								if (t1.getExecutionState() == ExecutionState.RUNNING) {
-									tm.tell(decorator.decorate(new CancelTask(eid1)), getRef());
+									tm.tell(new CancelTask(eid1), testActorGateway);
 									expectMsgEquals(new TaskOperationResult(eid1, true));
 								}
-								Future<Object> response = Patterns.ask(
-										tm,
+								Future<Object> response = tm.ask(
 										new TestingTaskManagerMessages.NotifyWhenTaskRemoved(eid1),
 										timeout);
 								Await.ready(response, d);
 							}
 
-							tm.tell(TestingTaskManagerMessages.getRequestRunningTasksMessage(), getRef());
+							tm.tell(TestingTaskManagerMessages.getRequestRunningTasksMessage(), testActorGateway);
 							tasks = expectMsgClass(TestingTaskManagerMessages
 									.ResponseRunningTasks.class).asJava();
 
@@ -702,12 +727,8 @@ public class TaskManagerTest {
 			}
 			finally {
 				// shut down the actors
-				if (taskManager != null) {
-					taskManager.tell(Kill.getInstance(), ActorRef.noSender());
-				}
-				if (jobManager != null) {
-					jobManager.tell(Kill.getInstance(), ActorRef.noSender());
-				}
+				TestingUtils.stopActor(taskManager);
+				TestingUtils.stopActor(jobManager);
 			}
 		}};
 	}
@@ -720,22 +741,36 @@ public class TaskManagerTest {
 
 		new JavaTestKit(system){{
 
-			ActorRef jobManager = null;
-			ActorRef taskManager = null;
+			ActorGateway jobManager = null;
+			ActorGateway taskManager = null;
+
+			final ActorGateway testActorGateway = new AkkaActorGateway(
+					getTestActor(),
+					leaderSessionID);
 
 			try {
 				final IntermediateDataSetID resultId = new IntermediateDataSetID();
 
 				// Create the JM
-				jobManager = system.actorOf(Props.create(
-						new SimplePartitionStateLookupJobManagerCreator(leaderSessionID.get(), getTestActor())));
+				ActorRef jm = system.actorOf(Props.create(
+						new SimplePartitionStateLookupJobManagerCreator(leaderSessionID, getTestActor())));
+
+				jobManager = new AkkaActorGateway(jm, leaderSessionID);
 
 				final int dataPort = NetUtils.getAvailablePort();
-				taskManager = createTaskManager(jobManager, true, false, dataPort);
+				Configuration config = new Configuration();
+				config.setInteger(ConfigConstants.TASK_MANAGER_DATA_PORT_KEY, dataPort);
+
+				taskManager = TestingUtils.createTaskManager(
+						system,
+						jobManager,
+						config,
+						false,
+						true);
 
 				// ---------------------------------------------------------------------------------
 
-				final ActorRef tm = taskManager;
+				final ActorGateway tm = taskManager;
 
 				final JobID jid = new JobID();
 				final JobVertexID vid = new JobVertexID();
@@ -767,16 +802,16 @@ public class TaskManagerTest {
 					@Override
 					protected void run() {
 						// Submit the task
-						tm.tell(decorator.decorate(new SubmitTask(tdd)), getTestActor());
+						tm.tell(new SubmitTask(tdd), testActorGateway);
 						expectMsgClass(Messages.getAcknowledge().getClass());
 
 						// Wait to be notified about the final execution state by the mock JM
 						TaskExecutionState msg = expectMsgClass(TaskExecutionState.class);
 
 						// The task should fail after repeated requests
-						assertEquals(msg.getExecutionState(), ExecutionState.FAILED);
-						assertEquals(msg.getError(ClassLoader.getSystemClassLoader()).getClass(),
-								PartitionNotFoundException.class);
+						assertEquals(ExecutionState.FAILED, msg.getExecutionState());
+						assertEquals(PartitionNotFoundException.class,
+								msg.getError(ClassLoader.getSystemClassLoader()).getClass());
 					}
 				};
 			}
@@ -785,13 +820,8 @@ public class TaskManagerTest {
 				fail(e.getMessage());
 			}
 			finally {
-				if (taskManager != null) {
-					taskManager.tell(Kill.getInstance(), ActorRef.noSender());
-				}
-
-				if (jobManager != null) {
-					jobManager.tell(Kill.getInstance(), ActorRef.noSender());
-				}
+				TestingUtils.stopActor(taskManager);
+				TestingUtils.stopActor(jobManager);
 			}
 		}};
 	}
@@ -804,22 +834,37 @@ public class TaskManagerTest {
 
 		new JavaTestKit(system){{
 
-			ActorRef jobManager = null;
-			ActorRef taskManager = null;
+			ActorGateway jobManager = null;
+			ActorGateway taskManager = null;
+
+			final ActorGateway testActorGateway = new AkkaActorGateway(
+					getTestActor(),
+					leaderSessionID);
 
 			try {
 				final IntermediateDataSetID resultId = new IntermediateDataSetID();
 
 				// Create the JM
-				jobManager = system.actorOf(Props.create(
-						new SimplePartitionStateLookupJobManagerCreator(leaderSessionID.get(), getTestActor())));
+				ActorRef jm = system.actorOf(Props.create(
+						new SimplePartitionStateLookupJobManagerCreator(leaderSessionID, getTestActor())));
+
+				jobManager = new AkkaActorGateway(jm, leaderSessionID);
 
 				final int dataPort = NetUtils.getAvailablePort();
-				taskManager = createTaskManager(jobManager, true, true, dataPort);
+				final Configuration config = new Configuration();
+
+				config.setInteger(ConfigConstants.TASK_MANAGER_DATA_PORT_KEY, dataPort);
+
+				taskManager = TestingUtils.createTaskManager(
+						system,
+						jobManager,
+						config,
+						true,
+						true);
 
 				// ---------------------------------------------------------------------------------
 
-				final ActorRef tm = taskManager;
+				final ActorGateway tm = taskManager;
 
 				final JobID jid = new JobID();
 				final JobVertexID vid = new JobVertexID();
@@ -849,7 +894,7 @@ public class TaskManagerTest {
 					@Override
 					protected void run() {
 						// Submit the task
-						tm.tell(decorator.decorate(new SubmitTask(tdd)), getTestActor());
+						tm.tell(new SubmitTask(tdd), testActorGateway);
 						expectMsgClass(Messages.getAcknowledge().getClass());
 
 						// Wait to be notified about the final execution state by the mock JM
@@ -867,13 +912,8 @@ public class TaskManagerTest {
 				fail(e.getMessage());
 			}
 			finally {
-				if (taskManager != null) {
-					taskManager.tell(Kill.getInstance(), ActorRef.noSender());
-				}
-
-				if (jobManager != null) {
-					jobManager.tell(Kill.getInstance(), ActorRef.noSender());
-				}
+				TestingUtils.stopActor(taskManager);
+				TestingUtils.stopActor(jobManager);
 			}
 		}};
 	}
@@ -882,35 +922,31 @@ public class TaskManagerTest {
 
 	public static class SimpleJobManager extends FlinkUntypedActor {
 
-		private final Option<UUID> leaderSessionID;
+		private final UUID leaderSessionID;
 
 		public SimpleJobManager(UUID leaderSessionID) {
-			this.leaderSessionID = Option.apply(leaderSessionID);
+			this.leaderSessionID = leaderSessionID;
 		}
 
 		public void handleMessage(Object message) throws Exception {
 			if (message instanceof RegistrationMessages.RegisterTaskManager) {
-				final RegistrationMessages.RegisterTaskManager registerTaskManager = (RegistrationMessages.RegisterTaskManager) message;
 				final InstanceID iid = new InstanceID();
 				final ActorRef self = getSelf();
 				getSender().tell(
 						decorateMessage(
 								new RegistrationMessages.AcknowledgeRegistration(
-									registerTaskManager.registrationSessionID(),
-									leaderSessionID.get(),
-									self,
 									iid,
 									12345)
 						),
 						self);
 			}
-			else if(message instanceof UpdateTaskExecutionState){
+			else if(message instanceof TaskMessages.UpdateTaskExecutionState){
 				getSender().tell(true, getSelf());
 			}
 		}
 
 		@Override
-		protected Option<UUID> getLeaderSessionID() {
+		protected UUID getLeaderSessionID() {
 			return leaderSessionID;
 		}
 	}
@@ -945,9 +981,9 @@ public class TaskManagerTest {
 
 		@Override
 		public void handleMessage(Object message) throws Exception{
-			if (message instanceof UpdateTaskExecutionState) {
-				UpdateTaskExecutionState updateMsg =
-						(UpdateTaskExecutionState) message;
+			if (message instanceof TaskMessages.UpdateTaskExecutionState) {
+				TaskMessages.UpdateTaskExecutionState updateMsg =
+						(TaskMessages.UpdateTaskExecutionState) message;
 
 				if(validIDs.contains(updateMsg.taskExecutionState().getID())) {
 					getSender().tell(true, getSelf());
@@ -982,8 +1018,8 @@ public class TaskManagerTest {
 
 				getSender().tell(decorateMessage(resp), getSelf());
 			}
-			else if (message instanceof UpdateTaskExecutionState) {
-				final TaskExecutionState msg = ((UpdateTaskExecutionState) message)
+			else if (message instanceof TaskMessages.UpdateTaskExecutionState) {
+				final TaskExecutionState msg = ((TaskMessages.UpdateTaskExecutionState) message)
 						.taskExecutionState();
 
 				if (msg.getExecutionState().isTerminal()) {
@@ -1048,50 +1084,6 @@ public class TaskManagerTest {
 			return new SimplePartitionStateLookupJobManager(leaderSessionID, testActor);
 		}
 	}
-
-	public static ActorRef createTaskManager(ActorRef jobManager, boolean waitForRegistration) {
-		return createTaskManager(jobManager, waitForRegistration, true, NetUtils.getAvailablePort());
-	}
-
-	public static ActorRef createTaskManager(ActorRef jobManager, boolean waitForRegistration, 
-												boolean useLocalCommunication, int dataPort) {
-		ActorRef taskManager = null;
-		try {
-			Configuration cfg = new Configuration();
-			cfg.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 10);
-			cfg.setInteger(ConfigConstants.TASK_MANAGER_DATA_PORT_KEY, dataPort);
-
-			Option<String> jobMangerUrl = Option.apply(jobManager.path().toString());
-
-			taskManager = TaskManager.startTaskManagerComponentsAndActor(
-					cfg, system, "localhost",
-					Option.<String>empty(),
-					jobMangerUrl,
-					useLocalCommunication,
-					StreamingMode.BATCH_ONLY,
-					TestingTaskManager.class);
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail("Could not create test TaskManager: " + e.getMessage());
-		}
-
-		if (waitForRegistration) {
-			Future<Object> response = Patterns.ask(taskManager, 
-					TaskManagerMessages.getNotifyWhenRegisteredAtJobManagerMessage(), timeout);
-	
-			try {
-				FiniteDuration d = new FiniteDuration(100, TimeUnit.SECONDS);
-				Await.ready(response, d);
-			}
-			catch (Exception e) {
-				e.printStackTrace();
-				fail("Exception while waiting for the task manager registration: " + e.getMessage());
-			}
-		}
-
-		return taskManager;
-	}
 	
 	// --------------------------------------------------------------------------------------------
 	

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java
index f60b0a4..5cfc2a2 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java
@@ -800,7 +800,7 @@ public class TaskTest {
 		try {
 			// we may have to wait for a bit to give the actors time to receive the message
 			// and put it into the queue
-			TaskMessages.UpdateTaskExecutionState message = 
+			TaskMessages.UpdateTaskExecutionState message =
 					(TaskMessages.UpdateTaskExecutionState) listenerMessages.take();
 			assertNotNull("There is no additional listener message", message);
 			

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TestManagerStartupTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TestManagerStartupTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TestManagerStartupTest.java
deleted file mode 100644
index a033eb1..0000000
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TestManagerStartupTest.java
+++ /dev/null
@@ -1,168 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.runtime.taskmanager;
-
-import static org.junit.Assert.*;
-
-import org.apache.commons.io.FileUtils;
-import org.apache.flink.configuration.ConfigConstants;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.configuration.IllegalConfigurationException;
-import org.apache.flink.runtime.StreamingMode;
-import org.junit.Test;
-
-import java.io.File;
-import java.io.IOException;
-import java.net.InetAddress;
-import java.net.ServerSocket;
-import java.util.UUID;
-
-/**
- * Tests that check how the TaskManager behaves when encountering startup problems.
- */
-public class TestManagerStartupTest {
-
-	/**
-	 * Tests that the TaskManager fails synchronously when the actor system port
-	 * is in use.
-	 */
-	@Test
-	public void testStartupWhenTaskmanagerActorPortIsUsed() {
-		ServerSocket blocker = null;
-		try {
-			final String localHostName = "localhost";
-			final InetAddress localAddress = InetAddress.getByName(localHostName);
-
-			// block some port
-			blocker = new ServerSocket(0, 50, localAddress);
-			final int port = blocker.getLocalPort();
-
-			try {
-				TaskManager.runTaskManager(localHostName, port, new Configuration(),
-											StreamingMode.BATCH_ONLY, TaskManager.class);
-				fail("This should fail with an IOException");
-			}
-			catch (IOException e) {
-				// expected. validate the error message
-				assertNotNull(e.getMessage());
-				assertTrue(e.getMessage().contains("Address already in use"));
-			}
-
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-		finally {
-			if (blocker != null) {
-				try {
-					blocker.close();
-				}
-				catch (IOException e) {
-					// no need to log here
-				}
-			}
-		}
-	}
-
-	/**
-	 * Tests that the TaskManager startup fails synchronously when the I/O directories are
-	 * not writable.
-	 */
-	@Test
-	public void testIODirectoryNotWritable() {
-		File tempDir = new File(ConfigConstants.DEFAULT_TASK_MANAGER_TMP_PATH);
-		File nonWritable = new File(tempDir, UUID.randomUUID().toString());
-
-		if (!nonWritable.mkdirs() || !nonWritable.setWritable(false, false)) {
-			System.err.println("Cannot create non-writable temporary file directory. Skipping test.");
-			return;
-		}
-
-		try {
-			Configuration cfg = new Configuration();
-			cfg.setString(ConfigConstants.TASK_MANAGER_TMP_DIR_KEY, nonWritable.getAbsolutePath());
-			cfg.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 4);
-			cfg.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, "localhost");
-			cfg.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, 21656);
-
-			try {
-				TaskManager.runTaskManager("localhost", 0, cfg, StreamingMode.BATCH_ONLY);
-				fail("Should fail synchronously with an exception");
-			}
-			catch (IOException e) {
-				// splendid!
-			}
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-		finally {
-			//noinspection ResultOfMethodCallIgnored
-			nonWritable.setWritable(true, false);
-			try {
-				FileUtils.deleteDirectory(nonWritable);
-			}
-			catch (IOException e) {
-				// best effort
-			}
-		}
-	}
-
-	/**
-	 * Tests that the TaskManager startup fails synchronously when the I/O directories are
-	 * not writable.
-	 */
-	@Test
-	public void testMemoryConfigWrong() {
-		try {
-			Configuration cfg = new Configuration();
-			cfg.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, "localhost");
-			cfg.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, 21656);
-
-			// something invalid
-			cfg.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, -42);
-			try {
-				TaskManager.runTaskManager("localhost", 0, cfg, StreamingMode.BATCH_ONLY);
-				fail("Should fail synchronously with an exception");
-			}
-			catch (IllegalConfigurationException e) {
-				// splendid!
-			}
-
-			// something ridiculously high
-			final long memSize = (((long) Integer.MAX_VALUE - 1) *
-									ConfigConstants.DEFAULT_TASK_MANAGER_NETWORK_BUFFER_SIZE) >> 20;
-			cfg.setLong(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, memSize);
-			try {
-				TaskManager.runTaskManager("localhost", 0, cfg, StreamingMode.BATCH_ONLY);
-				fail("Should fail synchronously with an exception");
-			}
-			catch (Exception e) {
-				// splendid!
-				assertTrue(e.getCause() instanceof OutOfMemoryError);
-			}
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/test/java/org/apache/flink/runtime/util/ZooKeeperUtilTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/util/ZooKeeperUtilTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/ZooKeeperUtilTest.java
index da40e15..0d01f65 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/util/ZooKeeperUtilTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/ZooKeeperUtilTest.java
@@ -20,11 +20,12 @@ package org.apache.flink.runtime.util;
 
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.util.TestLogger;
 import org.junit.Test;
 
 import static org.junit.Assert.assertEquals;
 
-public class ZooKeeperUtilTest {
+public class ZooKeeperUtilTest extends TestLogger {
 
 	@Test
 	public void testZooKeeperEnsembleConnectStringConfiguration() throws Exception {
@@ -36,15 +37,15 @@ public class ZooKeeperUtilTest {
 			expected = "localhost:2891";
 
 			setQuorum(conf, expected);
-			actual = ZooKeeperUtil.getZooKeeperEnsemble(conf);
+			actual = ZooKeeperUtils.getZooKeeperEnsemble(conf);
 			assertEquals(expected, actual);
 
 			setQuorum(conf, " localhost:2891 "); // with leading and trailing whitespace
-			actual = ZooKeeperUtil.getZooKeeperEnsemble(conf);
+			actual = ZooKeeperUtils.getZooKeeperEnsemble(conf);
 			assertEquals(expected, actual);
 
 			setQuorum(conf, "localhost :2891"); // whitespace after port
-			actual = ZooKeeperUtil.getZooKeeperEnsemble(conf);
+			actual = ZooKeeperUtils.getZooKeeperEnsemble(conf);
 			assertEquals(expected, actual);
 		}
 
@@ -52,19 +53,19 @@ public class ZooKeeperUtilTest {
 			expected = "localhost:2891,localhost:2891";
 
 			setQuorum(conf, "localhost:2891,localhost:2891");
-			actual = ZooKeeperUtil.getZooKeeperEnsemble(conf);
+			actual = ZooKeeperUtils.getZooKeeperEnsemble(conf);
 			assertEquals(expected, actual);
 
 			setQuorum(conf, "localhost:2891, localhost:2891");
-			actual = ZooKeeperUtil.getZooKeeperEnsemble(conf);
+			actual = ZooKeeperUtils.getZooKeeperEnsemble(conf);
 			assertEquals(expected, actual);
 
 			setQuorum(conf, "localhost :2891, localhost:2891");
-			actual = ZooKeeperUtil.getZooKeeperEnsemble(conf);
+			actual = ZooKeeperUtils.getZooKeeperEnsemble(conf);
 			assertEquals(expected, actual);
 
 			setQuorum(conf, " localhost:2891, localhost:2891 ");
-			actual = ZooKeeperUtil.getZooKeeperEnsemble(conf);
+			actual = ZooKeeperUtils.getZooKeeperEnsemble(conf);
 			assertEquals(expected, actual);
 		}
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/test/scala/org/apache/flink/runtime/akka/AkkaUtilsTest.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/akka/AkkaUtilsTest.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/akka/AkkaUtilsTest.scala
new file mode 100644
index 0000000..59477db
--- /dev/null
+++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/akka/AkkaUtilsTest.scala
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.akka
+
+import java.net.{InetAddress, InetSocketAddress}
+
+import org.apache.flink.runtime.jobmanager.JobManager
+import org.junit.runner.RunWith
+import org.scalatest.{FunSuite, BeforeAndAfterAll, Matchers}
+import org.scalatest.junit.JUnitRunner
+
+@RunWith(classOf[JUnitRunner])
+class AkkaUtilsTest
+  extends FunSuite
+  with Matchers
+  with BeforeAndAfterAll {
+
+  test("getHostFromAkkaURL should return the correct host from a remote Akka URL") {
+    val host = "127.0.0.1"
+    val port = 1234
+
+    val address = new InetSocketAddress(host, port)
+
+    val remoteAkkaURL = JobManager.getRemoteJobManagerAkkaURL(
+      address,
+      Some("actor"))
+
+    val result = AkkaUtils.getInetSockeAddressFromAkkaURL(remoteAkkaURL)
+
+    result should equal(address)
+  }
+
+  test("getHostFromAkkaURL should throw an exception if the InetSocketAddress cannot be " +
+    "retrieved") {
+    val localAkkaURL = JobManager.getLocalJobManagerAkkaURL(Some("actor"))
+
+    intercept[Exception] {
+      AkkaUtils.getInetSockeAddressFromAkkaURL(localAkkaURL)
+    }
+  }
+
+  test("getHostFromAkkaURL should return host after at sign") {
+    val url = "akka://flink@localhost:1234/user/jobmanager"
+    val expected = new InetSocketAddress("localhost", 1234)
+
+    val result = AkkaUtils.getInetSockeAddressFromAkkaURL(url)
+
+    result should equal(expected)
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/test/scala/org/apache/flink/runtime/akka/FlinkActorTest.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/akka/FlinkActorTest.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/akka/FlinkActorTest.scala
index 78af40e..acd4846 100644
--- a/flink-runtime/src/test/scala/org/apache/flink/runtime/akka/FlinkActorTest.scala
+++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/akka/FlinkActorTest.scala
@@ -25,7 +25,7 @@ import akka.testkit.{TestActorRef, TestKit}
 import grizzled.slf4j.Logger
 import org.apache.flink.runtime.akka.FlinkUntypedActorTest.PlainRequiresLeaderSessionID
 import org.apache.flink.runtime.messages.JobManagerMessages.LeaderSessionMessage
-import org.apache.flink.runtime.{LeaderSessionMessages, FlinkActor}
+import org.apache.flink.runtime.{LeaderSessionMessageFilter, FlinkActor}
 import org.apache.flink.runtime.testingUtils.TestingUtils
 import org.junit.runner.RunWith
 import org.scalatest.junit.JUnitRunner
@@ -45,10 +45,10 @@ class FlinkActorTest(_system: ActorSystem)
   }
 
   test("A Flink actor should only accept LeaderSessionMessages with a valid leader session id") {
-    val leaderSessionID = Some(UUID.randomUUID())
-    val oldSessionID = Some(UUID.randomUUID())
+    val leaderSessionID = UUID.randomUUID()
+    val oldSessionID = UUID.randomUUID()
 
-    val props = Props(classOf[PlainFlinkActor], leaderSessionID)
+    val props = Props(classOf[PlainFlinkActor], Option(leaderSessionID))
 
     val actor = TestActorRef[PlainFlinkActor](props)
 
@@ -63,9 +63,9 @@ class FlinkActorTest(_system: ActorSystem)
 
   test("A Flink actor should throw an exception when receiving an unwrapped " +
     "RequiresLeaderSessionID message") {
-    val leaderSessionID = Some(UUID.randomUUID())
+    val leaderSessionID = UUID.randomUUID()
 
-    val props = Props(classOf[PlainFlinkActor], leaderSessionID)
+    val props = Props(classOf[PlainFlinkActor], Option(leaderSessionID))
     val actor = TestActorRef[PlainFlinkActor](props)
 
     actor.receive(LeaderSessionMessage(leaderSessionID, 1))
@@ -79,7 +79,7 @@ class FlinkActorTest(_system: ActorSystem)
     } catch {
       case e: Exception =>
         e.getMessage should be("Received a message PlainRequiresLeaderSessionID without a " +
-          "leader session ID, even though it requires to have one.")
+          s"leader session ID, even though the message requires a leader session ID.")
     }
   }
 
@@ -91,7 +91,7 @@ class FlinkActorTest(_system: ActorSystem)
 
 class PlainFlinkActor(val leaderSessionID: Option[UUID])
   extends FlinkActor
-  with LeaderSessionMessages {
+  with LeaderSessionMessageFilter {
 
   val log = Logger(getClass)
 

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/CoLocationConstraintITCase.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/CoLocationConstraintITCase.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/CoLocationConstraintITCase.scala
index 94ead78..a0c144a 100644
--- a/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/CoLocationConstraintITCase.scala
+++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/CoLocationConstraintITCase.scala
@@ -21,6 +21,7 @@ package org.apache.flink.runtime.jobmanager
 import akka.actor.ActorSystem
 import akka.actor.Status.Success
 import akka.testkit.{ImplicitSender, TestKit}
+import org.apache.flink.runtime.akka.ListeningBehaviour
 import org.apache.flink.runtime.jobgraph.{JobGraph, DistributionPattern, JobVertex}
 import org.apache.flink.runtime.jobmanager.Tasks.{Receiver, Sender}
 import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup
@@ -29,6 +30,7 @@ import org.apache.flink.runtime.testingUtils.{ScalaTestingUtils, TestingUtils}
 import org.junit.runner.RunWith
 import org.scalatest.junit.JUnitRunner
 import org.scalatest.{BeforeAndAfterAll, Matchers, WordSpecLike}
+import scala.concurrent.duration._
 
 @RunWith(classOf[JUnitRunner])
 class CoLocationConstraintITCase(_system: ActorSystem)
@@ -68,11 +70,11 @@ class CoLocationConstraintITCase(_system: ActorSystem)
       val jobGraph = new JobGraph("Pointwise job", sender, receiver)
 
       val cluster = TestingUtils.startTestingCluster(num_tasks)
-      val gateway = cluster.getJobManagerGateway()
+      val jmGateway = cluster.getLeaderGateway(1 seconds)
 
       try {
         within(TestingUtils.TESTING_DURATION) {
-          gateway.tell(SubmitJob(jobGraph, false), self)
+          jmGateway.tell(SubmitJob(jobGraph, ListeningBehaviour.EXECUTION_RESULT), self)
           expectMsg(JobSubmitSuccess(jobGraph.getJobID))
 
           expectMsgType[JobResultSuccess]

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/JobManagerConnectionTest.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/JobManagerConnectionTest.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/JobManagerConnectionTest.scala
index 8c8ce06..b0fe695 100644
--- a/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/JobManagerConnectionTest.scala
+++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/JobManagerConnectionTest.scala
@@ -59,7 +59,7 @@ class JobManagerConnectionTest {
       mustReturnWithinTimeout(Duration(5*timeout, TimeUnit.MILLISECONDS)) {
         () => {
           try {
-            JobManager.getJobManagerRemoteReference(endpoint, actorSystem, config)
+            JobManager.getJobManagerActorRef(endpoint, actorSystem, config)
             fail("Should fail since the JobManager is not reachable")
           }
           catch {
@@ -95,7 +95,7 @@ class JobManagerConnectionTest {
       mustReturnWithinTimeout(Duration(5*timeout, TimeUnit.MILLISECONDS)) {
         () => {
           try {
-            JobManager.getJobManagerRemoteReference(endpoint, actorSystem, config)
+            JobManager.getJobManagerActorRef(endpoint, actorSystem, config)
             fail("Should fail since the JobManager is not reachable")
           }
           catch {

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/JobManagerITCase.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/JobManagerITCase.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/JobManagerITCase.scala
index ddca3e2..74b7680 100644
--- a/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/JobManagerITCase.scala
+++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/JobManagerITCase.scala
@@ -23,6 +23,7 @@ import akka.actor.ActorSystem
 import akka.actor.Status.{Success, Failure}
 import akka.testkit.{ImplicitSender, TestKit}
 import akka.util.Timeout
+import org.apache.flink.runtime.akka.ListeningBehaviour
 import org.apache.flink.runtime.client.JobExecutionException
 import org.apache.flink.runtime.jobgraph.{JobVertex, DistributionPattern, JobGraph, ScheduleMode}
 import org.apache.flink.runtime.messages.JobManagerMessages._
@@ -66,7 +67,7 @@ class JobManagerITCase(_system: ActorSystem)
       val jobGraph = new JobGraph("Test Job", vertex)
 
       val cluster = TestingUtils.startTestingCluster(1)
-      val jmGateway = cluster.getJobManagerGateway()
+      val jmGateway = cluster.getLeaderGateway(1 seconds)
 
       try {
         val response = jmGateway.ask(RequestTotalNumberOfSlots, timeout.duration).mapTo[Int]
@@ -76,7 +77,7 @@ class JobManagerITCase(_system: ActorSystem)
         availableSlots should equal(1)
 
         within(2 second) {
-          jmGateway.tell(SubmitJob(jobGraph, false), self)
+          jmGateway.tell(SubmitJob(jobGraph, ListeningBehaviour.EXECUTION_RESULT), self)
           expectMsg(JobSubmitSuccess(jobGraph.getJobID()))
         }
 
@@ -108,7 +109,7 @@ class JobManagerITCase(_system: ActorSystem)
       val jobGraph = new JobGraph("Test Job", vertex)
 
       val cluster = TestingUtils.startTestingCluster(num_tasks)
-      val jmGateway = cluster.getJobManagerGateway()
+      val jmGateway = cluster.getLeaderGateway(1 seconds)
 
       try {
         val response = jmGateway.ask(RequestTotalNumberOfSlots, timeout.duration).mapTo[Int]
@@ -118,7 +119,7 @@ class JobManagerITCase(_system: ActorSystem)
         availableSlots should equal(num_tasks)
 
         within(TestingUtils.TESTING_DURATION) {
-          jmGateway.tell(SubmitJob(jobGraph, false), self)
+          jmGateway.tell(SubmitJob(jobGraph, ListeningBehaviour.EXECUTION_RESULT), self)
 
           expectMsg(JobSubmitSuccess(jobGraph.getJobID))
           
@@ -144,11 +145,11 @@ class JobManagerITCase(_system: ActorSystem)
       jobGraph.setAllowQueuedScheduling(true)
 
       val cluster = TestingUtils.startTestingCluster(10)
-      val jmGateway = cluster.getJobManagerGateway()
+      val jmGateway = cluster.getLeaderGateway(1 seconds)
 
       try {
         within(TestingUtils.TESTING_DURATION) {
-          jmGateway.tell(SubmitJob(jobGraph, false), self)
+          jmGateway.tell(SubmitJob(jobGraph, ListeningBehaviour.EXECUTION_RESULT), self)
 
           expectMsg(JobSubmitSuccess(jobGraph.getJobID))
 
@@ -179,11 +180,11 @@ class JobManagerITCase(_system: ActorSystem)
       val jobGraph = new JobGraph("Pointwise Job", sender, receiver)
 
       val cluster = TestingUtils.startTestingCluster(2 * num_tasks)
-      val jmGateway = cluster.getJobManagerGateway()
+      val jmGateway = cluster.getLeaderGateway(1 seconds)
 
       try {
         within(TestingUtils.TESTING_DURATION) {
-          jmGateway.tell(SubmitJob(jobGraph, false), self)
+          jmGateway.tell(SubmitJob(jobGraph, ListeningBehaviour.EXECUTION_RESULT), self)
 
           expectMsg(JobSubmitSuccess(jobGraph.getJobID))
 
@@ -214,11 +215,11 @@ class JobManagerITCase(_system: ActorSystem)
       val jobGraph = new JobGraph("Bipartite Job", sender, receiver)
 
       val cluster = TestingUtils.startTestingCluster(2 * num_tasks)
-      val jmGateway = cluster.getJobManagerGateway()
+      val jmGateway = cluster.getLeaderGateway(1 seconds)
 
       try {
         within(TestingUtils.TESTING_DURATION) {
-          jmGateway.tell(SubmitJob(jobGraph, false), self)
+          jmGateway.tell(SubmitJob(jobGraph, ListeningBehaviour.EXECUTION_RESULT), self)
 
           expectMsg(JobSubmitSuccess(jobGraph.getJobID))
 
@@ -251,11 +252,11 @@ class JobManagerITCase(_system: ActorSystem)
       val jobGraph = new JobGraph("Bipartite Job", sender1, receiver, sender2)
 
       val cluster = TestingUtils.startTestingCluster(6 * num_tasks)
-      val jmGateway = cluster.getJobManagerGateway()
+      val jmGateway = cluster.getLeaderGateway(1 seconds)
 
       try {
         within(TestingUtils.TESTING_DURATION) {
-          jmGateway.tell(SubmitJob(jobGraph, false), self)
+          jmGateway.tell(SubmitJob(jobGraph, ListeningBehaviour.EXECUTION_RESULT), self)
 
           expectMsg(JobSubmitSuccess(jobGraph.getJobID))
           val failure = expectMsgType[JobResultFailure]
@@ -296,11 +297,11 @@ class JobManagerITCase(_system: ActorSystem)
       val jobGraph = new JobGraph("Bipartite Job", sender1, receiver, sender2)
 
       val cluster = TestingUtils.startTestingCluster(6 * num_tasks)
-      val jmGateway = cluster.getJobManagerGateway()
+      val jmGateway = cluster.getLeaderGateway(1 seconds)
 
       try {
         within(TestingUtils.TESTING_DURATION) {
-          jmGateway.tell(SubmitJob(jobGraph, false), self)
+          jmGateway.tell(SubmitJob(jobGraph, ListeningBehaviour.EXECUTION_RESULT), self)
           expectMsg(JobSubmitSuccess(jobGraph.getJobID))
 
           expectMsgType[JobResultSuccess]
@@ -340,11 +341,11 @@ class JobManagerITCase(_system: ActorSystem)
       jobGraph.setScheduleMode(ScheduleMode.ALL)
 
       val cluster = TestingUtils.startTestingCluster(num_tasks, 1)
-      val jmGateway = cluster.getJobManagerGateway()
+      val jmGateway = cluster.getLeaderGateway(1 seconds)
 
       try {
         within(TestingUtils.TESTING_DURATION) {
-          jmGateway.tell(SubmitJob(jobGraph, false), self)
+          jmGateway.tell(SubmitJob(jobGraph, ListeningBehaviour.EXECUTION_RESULT), self)
 
           expectMsg(JobSubmitSuccess(jobGraph.getJobID))
 
@@ -374,7 +375,7 @@ class JobManagerITCase(_system: ActorSystem)
       val jobGraph = new JobGraph("Pointwise Job", sender, receiver)
 
       val cluster = TestingUtils.startTestingCluster(num_tasks)
-      val jmGateway = cluster.getJobManagerGateway()
+      val jmGateway = cluster.getLeaderGateway(1 seconds)
 
       try {
         within(TestingUtils.TESTING_DURATION) {
@@ -383,7 +384,7 @@ class JobManagerITCase(_system: ActorSystem)
         }
 
         within(TestingUtils.TESTING_DURATION) {
-          jmGateway.tell(SubmitJob(jobGraph, false), self)
+          jmGateway.tell(SubmitJob(jobGraph, ListeningBehaviour.EXECUTION_RESULT), self)
           expectMsg(JobSubmitSuccess(jobGraph.getJobID))
 
           val failure = expectMsgType[JobResultFailure]
@@ -422,7 +423,7 @@ class JobManagerITCase(_system: ActorSystem)
       val jobGraph = new JobGraph("Pointwise Job", sender, receiver)
 
       val cluster = TestingUtils.startTestingCluster(num_tasks)
-      val jmGateway = cluster.getJobManagerGateway()
+      val jmGateway = cluster.getLeaderGateway(1 seconds)
 
       try {
         within(TestingUtils.TESTING_DURATION) {
@@ -431,7 +432,7 @@ class JobManagerITCase(_system: ActorSystem)
         }
 
         within(TestingUtils.TESTING_DURATION) {
-          jmGateway.tell(SubmitJob(jobGraph, false), self)
+          jmGateway.tell(SubmitJob(jobGraph, ListeningBehaviour.EXECUTION_RESULT), self)
           expectMsg(JobSubmitSuccess(jobGraph.getJobID))
 
           val failure = expectMsgType[JobResultFailure]
@@ -467,11 +468,11 @@ class JobManagerITCase(_system: ActorSystem)
       val jobGraph = new JobGraph("Pointwise job", sender, receiver)
 
       val cluster = TestingUtils.startTestingCluster(2 * num_tasks)
-      val jmGateway = cluster.getJobManagerGateway()
+      val jmGateway = cluster.getLeaderGateway(1 seconds)
 
       try {
         within(TestingUtils.TESTING_DURATION) {
-          jmGateway.tell(SubmitJob(jobGraph, false), self)
+          jmGateway.tell(SubmitJob(jobGraph, ListeningBehaviour.EXECUTION_RESULT), self)
           expectMsg(JobSubmitSuccess(jobGraph.getJobID))
 
           val failure = expectMsgType[JobResultFailure]
@@ -507,14 +508,14 @@ class JobManagerITCase(_system: ActorSystem)
       val jobGraph = new JobGraph("Pointwise job", sender, receiver)
 
       val cluster = TestingUtils.startTestingCluster(num_tasks)
-      val jmGateway = cluster.getJobManagerGateway()
+      val jmGateway = cluster.getLeaderGateway(1 seconds)
 
       try {
         within(TestingUtils.TESTING_DURATION) {
           jmGateway.tell(RequestTotalNumberOfSlots, self)
           expectMsg(num_tasks)
 
-          jmGateway.tell(SubmitJob(jobGraph, false), self)
+          jmGateway.tell(SubmitJob(jobGraph, ListeningBehaviour.EXECUTION_RESULT), self)
           expectMsg(JobSubmitSuccess(jobGraph.getJobID))
 
           val failure = expectMsgType[JobResultFailure]
@@ -555,14 +556,14 @@ class JobManagerITCase(_system: ActorSystem)
       val jobGraph = new JobGraph("Pointwise job", sender, receiver)
 
       val cluster = TestingUtils.startTestingCluster(num_tasks)
-      val jmGateway = cluster.getJobManagerGateway()
+      val jmGateway = cluster.getLeaderGateway(1 seconds)
 
       try {
         within(TestingUtils.TESTING_DURATION) {
           jmGateway.tell(RequestTotalNumberOfSlots, self)
           expectMsg(num_tasks)
 
-          jmGateway.tell(SubmitJob(jobGraph, false), self)
+          jmGateway.tell(SubmitJob(jobGraph, ListeningBehaviour.EXECUTION_RESULT), self)
           expectMsg(JobSubmitSuccess(jobGraph.getJobID))
 
           val failure = expectMsgType[JobResultFailure]
@@ -598,11 +599,11 @@ class JobManagerITCase(_system: ActorSystem)
       val jobGraph = new JobGraph("SubtaskInFinalStateRaceCondition", source, sink)
 
       val cluster = TestingUtils.startTestingCluster(2*num_tasks)
-      val jmGateway = cluster.getJobManagerGateway()
+      val jmGateway = cluster.getLeaderGateway(1 seconds)
 
       try{
         within(TestingUtils.TESTING_DURATION){
-          jmGateway.tell(SubmitJob(jobGraph, false), self)
+          jmGateway.tell(SubmitJob(jobGraph, ListeningBehaviour.EXECUTION_RESULT), self)
 
           expectMsg(JobSubmitSuccess(jobGraph.getJobID))
           expectMsgType[JobResultSuccess]

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/JobManagerRegistrationTest.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/JobManagerRegistrationTest.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/JobManagerRegistrationTest.scala
index 6c3580d..4368309 100644
--- a/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/JobManagerRegistrationTest.scala
+++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/JobManagerRegistrationTest.scala
@@ -27,6 +27,8 @@ import org.apache.flink.configuration.Configuration
 import org.apache.flink.runtime.StreamingMode
 import org.apache.flink.runtime.akka.AkkaUtils
 import org.apache.flink.runtime.instance.{HardwareDescription, InstanceConnectionInfo, InstanceID}
+import org.apache.flink.runtime.leaderelection.StandaloneLeaderElectionService
+import org.apache.flink.runtime.messages.JobManagerMessages.LeaderSessionMessage
 import org.apache.flink.runtime.messages.RegistrationMessages.{AcknowledgeRegistration, AlreadyRegistered, RegisterTaskManager}
 import org.junit.Assert.{assertNotEquals, assertNotNull}
 import org.scalatest.{BeforeAndAfterAll, Matchers, WordSpecLike}
@@ -62,38 +64,39 @@ ImplicitSender with WordSpecLike with Matchers with BeforeAndAfterAll {
 
         val hardwareDescription = HardwareDescription.extractFromSystem(10)
 
+        val leaderSessionID = UUID.randomUUID()
+
         var id1: InstanceID = null
         var id2: InstanceID = null
-        val registrationSessionID = UUID.randomUUID()
 
         // task manager 1
         within(1 second) {
-          jm ! RegisterTaskManager(
-            registrationSessionID,
-            tmDummy1,
-            connectionInfo1,
-            hardwareDescription,
-            1)
+          jm.tell(
+            RegisterTaskManager(
+              connectionInfo1,
+              hardwareDescription,
+              1),
+            tmDummy1)
 
           val response = receiveOne(1 second)
           response match {
-            case AcknowledgeRegistration(registrationSessionID, _,  _, id, _) => id1 = id
+            case LeaderSessionMessage(leaderSessionID, AcknowledgeRegistration(id, _)) => id1 = id
             case _ => fail("Wrong response message: " + response)
           }
         }
 
         // task manager 2
         within(1 second) {
-          jm ! RegisterTaskManager(
-            registrationSessionID,
-            tmDummy2,
-            connectionInfo2,
-            hardwareDescription,
-            1)
+          jm.tell(
+            RegisterTaskManager(
+              connectionInfo2,
+              hardwareDescription,
+              1),
+            tmDummy2)
 
           val response = receiveOne(1 second)
           response match {
-            case AcknowledgeRegistration(registrationSessionID, _, _, id, _) => id2 = id
+            case LeaderSessionMessage(leaderSessionID, AcknowledgeRegistration(id, _)) => id2 = id
             case _ => fail("Wrong response message: " + response)
           }
         }
@@ -118,27 +121,27 @@ ImplicitSender with WordSpecLike with Matchers with BeforeAndAfterAll {
         val connectionInfo = new InstanceConnectionInfo(InetAddress.getLocalHost,1)
         val hardwareDescription = HardwareDescription.extractFromSystem(10)
 
-        val registrationSessionID = UUID.randomUUID()
-        
         within(1 second) {
-          jm ! RegisterTaskManager(
-            registrationSessionID,
-            tmDummy,
-            connectionInfo,
-            hardwareDescription,
-            1)
-          jm ! RegisterTaskManager(
-            registrationSessionID,
-            tmDummy,
-            connectionInfo,
-            hardwareDescription,
-            1)
-          jm ! RegisterTaskManager(
-            registrationSessionID,
-            tmDummy,
-            connectionInfo,
-            hardwareDescription,
-            1)
+          jm.tell(
+            RegisterTaskManager(
+              connectionInfo,
+              hardwareDescription,
+              1),
+            tmDummy)
+
+          jm.tell(
+            RegisterTaskManager(
+              connectionInfo,
+              hardwareDescription,
+              1),
+            tmDummy)
+
+          jm.tell(
+            RegisterTaskManager(
+              connectionInfo,
+              hardwareDescription,
+              1),
+            tmDummy)
 
           expectMsgType[AcknowledgeRegistration]
           expectMsgType[AlreadyRegistered]
@@ -152,9 +155,12 @@ ImplicitSender with WordSpecLike with Matchers with BeforeAndAfterAll {
   }
 
   private def startTestingJobManager(system: ActorSystem): ActorRef = {
-    val (jm: ActorRef, _) = JobManager.startJobManagerActors(new Configuration(), _system,
-                                                             None, None,
-                                                             StreamingMode.BATCH_ONLY)
+    val (jm: ActorRef, _) = JobManager.startJobManagerActors(
+      new Configuration(),
+      _system,
+      None,
+      None,
+      StreamingMode.BATCH_ONLY)
     jm
   }
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/RecoveryITCase.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/RecoveryITCase.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/RecoveryITCase.scala
index d454e69..850f80f 100644
--- a/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/RecoveryITCase.scala
+++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/RecoveryITCase.scala
@@ -21,6 +21,7 @@ package org.apache.flink.runtime.jobmanager
 import akka.actor.{PoisonPill, ActorSystem}
 import akka.testkit.{ImplicitSender, TestKit}
 import org.apache.flink.configuration.{ConfigConstants, Configuration}
+import org.apache.flink.runtime.akka.ListeningBehaviour
 import org.apache.flink.runtime.jobgraph.{JobStatus, JobGraph, DistributionPattern, JobVertex}
 import org.apache.flink.runtime.jobmanager.Tasks.{BlockingOnceReceiver, FailingOnceReceiver}
 import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup
@@ -30,6 +31,7 @@ import org.apache.flink.runtime.testingUtils.{ScalaTestingUtils, TestingCluster,
 import org.junit.runner.RunWith
 import org.scalatest.{BeforeAndAfterAll, Matchers, WordSpecLike}
 import org.scalatest.junit.JUnitRunner
+import scala.concurrent.duration._
 
 @RunWith(classOf[JUnitRunner])
 class RecoveryITCase(_system: ActorSystem)
@@ -46,12 +48,14 @@ class RecoveryITCase(_system: ActorSystem)
     TestKit.shutdownActorSystem(system)
   }
 
-  def startTestClusterWithHeartbeatTimeout(numSlots: Int,
-                                                numTaskManagers: Int,
-                                                heartbeatTimeout: String): TestingCluster = {
+  def createTestClusterWithHeartbeatTimeout(
+      numSlots: Int,
+      numTaskManagers: Int,
+      heartbeatTimeout: String)
+    : TestingCluster = {
     val config = new Configuration()
     config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, numSlots)
-    config.setInteger(ConfigConstants.LOCAL_INSTANCE_MANAGER_NUMBER_TASK_MANAGER, numTaskManagers)
+    config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, numTaskManagers)
     config.setString(ConfigConstants.AKKA_WATCH_HEARTBEAT_PAUSE, heartbeatTimeout)
     config.setString(ConfigConstants.DEFAULT_EXECUTION_RETRY_DELAY_KEY, heartbeatTimeout)
     new TestingCluster(config)
@@ -77,12 +81,14 @@ class RecoveryITCase(_system: ActorSystem)
       val jobGraph = new JobGraph("Pointwise job", sender, receiver)
       jobGraph.setNumberOfExecutionRetries(1)
 
-      val cluster = startTestClusterWithHeartbeatTimeout(2 * NUM_TASKS, 1, "2 s")
-      val jmGateway = cluster.getJobManagerGateway()
+      val cluster = createTestClusterWithHeartbeatTimeout(2 * NUM_TASKS, 1, "2 s")
+      cluster.start()
+
+      val jmGateway = cluster.getLeaderGateway(1 seconds)
 
       try {
         within(TestingUtils.TESTING_DURATION){
-          jmGateway.tell(SubmitJob(jobGraph, false), self)
+          jmGateway.tell(SubmitJob(jobGraph, ListeningBehaviour.EXECUTION_RESULT), self)
 
           expectMsg(JobSubmitSuccess(jobGraph.getJobID))
 
@@ -120,12 +126,14 @@ class RecoveryITCase(_system: ActorSystem)
       val jobGraph = new JobGraph("Pointwise job", sender, receiver)
       jobGraph.setNumberOfExecutionRetries(1)
 
-      val cluster = startTestClusterWithHeartbeatTimeout(NUM_TASKS, 1, "2 s")
-      val jmGateway = cluster.getJobManagerGateway()
+      val cluster = createTestClusterWithHeartbeatTimeout(NUM_TASKS, 1, "2 s")
+      cluster.start()
+
+      val jmGateway = cluster.getLeaderGateway(1 seconds)
 
       try {
         within(TestingUtils.TESTING_DURATION){
-          jmGateway.tell(SubmitJob(jobGraph, false), self)
+          jmGateway.tell(SubmitJob(jobGraph, ListeningBehaviour.EXECUTION_RESULT), self)
 
           expectMsg(JobSubmitSuccess(jobGraph.getJobID))
 
@@ -163,13 +171,14 @@ class RecoveryITCase(_system: ActorSystem)
       val jobGraph = new JobGraph("Pointwise job", sender, receiver)
       jobGraph.setNumberOfExecutionRetries(1)
 
-      val cluster = startTestClusterWithHeartbeatTimeout(NUM_TASKS, 2, "2 s")
+      val cluster = createTestClusterWithHeartbeatTimeout(NUM_TASKS, 2, "2 s")
+      cluster.start()
 
-      val jmGateway = cluster.getJobManagerGateway()
+      val jmGateway = cluster.getLeaderGateway(1 seconds)
 
       try {
         within(TestingUtils.TESTING_DURATION){
-          jmGateway.tell(SubmitJob(jobGraph, false), self)
+          jmGateway.tell(SubmitJob(jobGraph, ListeningBehaviour.EXECUTION_RESULT), self)
 
           expectMsg(JobSubmitSuccess(jobGraph.getJobID))
 

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/SlotSharingITCase.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/SlotSharingITCase.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/SlotSharingITCase.scala
index 23e8ac1..a6d60dd 100644
--- a/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/SlotSharingITCase.scala
+++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/SlotSharingITCase.scala
@@ -21,6 +21,7 @@ package org.apache.flink.runtime.jobmanager
 import akka.actor.ActorSystem
 import akka.actor.Status.Success
 import akka.testkit.{ImplicitSender, TestKit}
+import org.apache.flink.runtime.akka.ListeningBehaviour
 import org.apache.flink.runtime.jobgraph.{JobVertex, DistributionPattern, JobGraph}
 import org.apache.flink.runtime.jobmanager.Tasks.{Sender, AgnosticBinaryReceiver, Receiver}
 import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup
@@ -29,6 +30,7 @@ import org.apache.flink.runtime.testingUtils.{ScalaTestingUtils, TestingUtils}
 import org.junit.runner.RunWith
 import org.scalatest.junit.JUnitRunner
 import org.scalatest.{BeforeAndAfterAll, Matchers, WordSpecLike}
+import scala.concurrent.duration._
 
 @RunWith(classOf[JUnitRunner])
 class SlotSharingITCase(_system: ActorSystem)
@@ -66,11 +68,11 @@ class SlotSharingITCase(_system: ActorSystem)
       val jobGraph = new JobGraph("Pointwise Job", sender, receiver)
 
       val cluster = TestingUtils.startTestingCluster(num_tasks)
-      val jmGateway = cluster.getJobManagerGateway()
+      val jmGateway = cluster.getLeaderGateway(1 seconds)
 
       try {
         within(TestingUtils.TESTING_DURATION) {
-          jmGateway.tell(SubmitJob(jobGraph, false), self)
+          jmGateway.tell(SubmitJob(jobGraph, ListeningBehaviour.EXECUTION_RESULT), self)
           expectMsg(JobSubmitSuccess(jobGraph.getJobID))
           expectMsgType[JobResultSuccess]
 
@@ -110,11 +112,11 @@ class SlotSharingITCase(_system: ActorSystem)
       val jobGraph = new JobGraph("Bipartite job", sender1, sender2, receiver)
 
       val cluster = TestingUtils.startTestingCluster(num_tasks)
-      val jmGateway = cluster.getJobManagerGateway()
+      val jmGateway = cluster.getLeaderGateway(1 seconds)
 
       try {
         within(TestingUtils.TESTING_DURATION) {
-          jmGateway.tell(SubmitJob(jobGraph, false), self)
+          jmGateway.tell(SubmitJob(jobGraph, ListeningBehaviour.EXECUTION_RESULT), self)
           expectMsg(JobSubmitSuccess(jobGraph.getJobID))
           expectMsgType[JobResultSuccess]
         }

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/TaskManagerFailsWithSlotSharingITCase.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/TaskManagerFailsWithSlotSharingITCase.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/TaskManagerFailsWithSlotSharingITCase.scala
index ae51da2..49a1c95 100644
--- a/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/TaskManagerFailsWithSlotSharingITCase.scala
+++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/TaskManagerFailsWithSlotSharingITCase.scala
@@ -20,6 +20,7 @@ package org.apache.flink.runtime.jobmanager
 
 import akka.actor.{Kill, ActorSystem, PoisonPill}
 import akka.testkit.{ImplicitSender, TestKit}
+import org.apache.flink.runtime.akka.ListeningBehaviour
 import org.apache.flink.runtime.client.JobExecutionException
 import org.apache.flink.runtime.jobgraph.{JobVertex, DistributionPattern, JobGraph}
 import org.apache.flink.runtime.jobmanager.Tasks.{BlockingReceiver, Sender}
@@ -31,6 +32,7 @@ import org.apache.flink.runtime.util.SerializedThrowable
 import org.junit.runner.RunWith
 import org.scalatest.junit.JUnitRunner
 import org.scalatest.{BeforeAndAfterAll, Matchers, WordSpecLike}
+import scala.concurrent.duration._
 
 @RunWith(classOf[JUnitRunner])
 class TaskManagerFailsWithSlotSharingITCase(_system: ActorSystem)
@@ -69,12 +71,12 @@ class TaskManagerFailsWithSlotSharingITCase(_system: ActorSystem)
       val jobID = jobGraph.getJobID
 
       val cluster = TestingUtils.startTestingCluster(num_tasks/2, 2)
-      val jmGateway = cluster.getJobManagerGateway()
+      val jmGateway = cluster.getLeaderGateway(1 seconds)
       val taskManagers = cluster.getTaskManagers
 
       try{
         within(TestingUtils.TESTING_DURATION) {
-          jmGateway.tell(SubmitJob(jobGraph, false), self)
+          jmGateway.tell(SubmitJob(jobGraph, ListeningBehaviour.EXECUTION_RESULT), self)
           expectMsg(JobSubmitSuccess(jobGraph.getJobID))
 
           jmGateway.tell(WaitForAllVerticesToBeRunningOrFinished(jobID), self)
@@ -118,12 +120,12 @@ class TaskManagerFailsWithSlotSharingITCase(_system: ActorSystem)
       val jobID = jobGraph.getJobID
 
       val cluster = TestingUtils.startTestingCluster(num_tasks/2, 2)
-      val jmGateway = cluster.getJobManagerGateway()
+      val jmGateway = cluster.getLeaderGateway(1 seconds)
       val taskManagers = cluster.getTaskManagers
 
       try{
         within(TestingUtils.TESTING_DURATION) {
-          jmGateway.tell(SubmitJob(jobGraph, false), self)
+          jmGateway.tell(SubmitJob(jobGraph, ListeningBehaviour.EXECUTION_RESULT), self)
           expectMsg(JobSubmitSuccess(jobGraph.getJobID))
 
           jmGateway.tell(WaitForAllVerticesToBeRunningOrFinished(jobID), self)

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/ScalaTestingUtils.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/ScalaTestingUtils.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/ScalaTestingUtils.scala
index e63025a..d46dd71 100644
--- a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/ScalaTestingUtils.scala
+++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/ScalaTestingUtils.scala
@@ -26,12 +26,12 @@ import org.apache.flink.runtime.instance.{ActorGateway, AkkaActorGateway}
   */
 trait ScalaTestingUtils {
 
-  /** Converts an [[ActorRef]] into a new [[AkkaActorGateway]] with None leader session ID
+  /** Converts an [[ActorRef]] into a new [[AkkaActorGateway]] with null leader session ID
     *
     * @param actor ActorRef for which the ActorGateway is constructed
     * @return [[ActorGateway]] encapsulating the given [[ActorRef]]
     */
   implicit def actorRef2InstanceGateway(actor: ActorRef): ActorGateway = {
-    new AkkaActorGateway(actor, None)
+    new AkkaActorGateway(actor, null)
   }
 }


[07/10] flink git commit: [FLINK-2291] [runtime] Add ZooKeeper support to elect a leader from a set of JobManager. The leader will then be retrieved from ZooKeeper by the TaskManagers.

Posted by tr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/main/scala/org/apache/flink/runtime/akka/RemoteAddressExtension.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/akka/RemoteAddressExtension.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/akka/RemoteAddressExtension.scala
new file mode 100644
index 0000000..79f63de
--- /dev/null
+++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/akka/RemoteAddressExtension.scala
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.akka
+
+import akka.actor.{Address, ExtensionKey, Extension, ExtendedActorSystem}
+
+/** [[akka.actor.ActorSystem]] [[Extension]] used to obtain the [[Address]] on which the
+  * given ActorSystem is listening.
+  *
+  * @param system
+  */
+class RemoteAddressExtensionImplementation(system: ExtendedActorSystem) extends Extension {
+  def address: Address = system.provider.getDefaultAddress
+}
+
+object RemoteAddressExtension extends ExtensionKey[RemoteAddressExtensionImplementation]{}

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala
index e9a31fb..26bf91b 100644
--- a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala
+++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala
@@ -24,7 +24,8 @@ import java.net.InetSocketAddress
 import java.util.{UUID, Collections}
 
 import akka.actor.Status.Failure
-import akka.actor._
+import akka.actor.{Props, Terminated, PoisonPill, ActorRef, ActorSystem}
+import akka.pattern.ask
 
 import grizzled.slf4j.Logger
 
@@ -36,24 +37,30 @@ import org.apache.flink.runtime.blob.BlobServer
 import org.apache.flink.runtime.client._
 import org.apache.flink.runtime.executiongraph.{ExecutionGraph, ExecutionJobVertex}
 import org.apache.flink.runtime.jobmanager.web.WebInfoServer
+import org.apache.flink.runtime.leaderelection.{LeaderContender, LeaderElectionService}
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService
 import org.apache.flink.runtime.messages.ArchiveMessages.ArchiveExecutionGraph
 import org.apache.flink.runtime.messages.ExecutionGraphMessages.JobStatusChanged
 import org.apache.flink.runtime.messages.Messages.{Acknowledge, Disconnect}
 import org.apache.flink.runtime.messages.TaskMessages.{PartitionState, UpdateTaskExecutionState}
-import org.apache.flink.runtime.messages.accumulators._
-import org.apache.flink.runtime.messages.checkpoint.{AbstractCheckpointMessage, AcknowledgeCheckpoint}
+
+import org.apache.flink.runtime.messages.accumulators.{AccumulatorResultsErroneous,
+AccumulatorResultsFound, RequestAccumulatorResults, AccumulatorMessage,
+AccumulatorResultStringsFound, RequestAccumulatorResultsStringified}
+import org.apache.flink.runtime.messages.checkpoint.{AbstractCheckpointMessage,
+AcknowledgeCheckpoint}
 import org.apache.flink.runtime.messages.webmonitor._
 import org.apache.flink.runtime.process.ProcessReaper
 import org.apache.flink.runtime.security.SecurityUtils
 import org.apache.flink.runtime.security.SecurityUtils.FlinkSecuredRunner
 import org.apache.flink.runtime.taskmanager.TaskManager
-import org.apache.flink.runtime.util.{SerializedThrowable, ZooKeeperUtil, EnvironmentInformation}
+import org.apache.flink.runtime.util._
 import org.apache.flink.runtime.webmonitor.WebMonitor
-import org.apache.flink.runtime.{FlinkActor, StreamingMode, LeaderSessionMessages}
+import org.apache.flink.runtime.{FlinkActor, StreamingMode, LeaderSessionMessageFilter}
 import org.apache.flink.runtime.LogMessages
-import org.apache.flink.runtime.akka.AkkaUtils
+import org.apache.flink.runtime.akka.{ListeningBehaviour, AkkaUtils}
 import org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager
-import org.apache.flink.runtime.instance.{ActorGateway, AkkaActorGateway, InstanceManager}
+import org.apache.flink.runtime.instance.{AkkaActorGateway, InstanceManager}
 import org.apache.flink.runtime.jobgraph.{JobVertexID, JobGraph, JobStatus}
 import org.apache.flink.runtime.jobmanager.scheduler.{Scheduler => FlinkScheduler}
 import org.apache.flink.runtime.messages.JobManagerMessages._
@@ -61,7 +68,6 @@ import org.apache.flink.runtime.messages.RegistrationMessages._
 import org.apache.flink.runtime.messages.TaskManagerMessages.{SendStackTrace, Heartbeat}
 import org.apache.flink.util.{SerializedValue, ExceptionUtils, InstantiationUtil}
 
-import _root_.akka.pattern.ask
 import scala.concurrent._
 import scala.concurrent.duration._
 import scala.concurrent.forkjoin.ForkJoinPool
@@ -105,38 +111,58 @@ class JobManager(
     protected val defaultExecutionRetries: Int,
     protected val delayBetweenRetries: Long,
     protected val timeout: FiniteDuration,
-    protected val mode: StreamingMode)
-  extends FlinkActor
-  with LeaderSessionMessages // order of the mixin is important, we want filtering after logging
-  with LogMessages // order of the mixin is important, we want first logging
-  {
+    protected val mode: StreamingMode,
+    protected val leaderElectionService: LeaderElectionService)
+  extends FlinkActor 
+  with LeaderSessionMessageFilter // mixin oder is important, we want filtering after logging
+  with LogMessages // mixin order is important, we want first logging
+  with LeaderContender {
 
   override val log = Logger(getClass)
 
   /** List of current jobs running jobs */
   protected val currentJobs = scala.collection.mutable.HashMap[JobID, (ExecutionGraph, JobInfo)]()
 
-  override val leaderSessionID = Some(UUID.randomUUID())
+  var leaderSessionID: Option[UUID] = None
 
   /**
    * Run when the job manager is started. Simply logs an informational message.
+   * The method also starts the leader election service.
    */
   override def preStart(): Unit = {
-    log.info(s"Starting JobManager at ${self.path.toSerializationFormat}.")
+    log.info(s"Starting JobManager at ${getAddress}.")
+
+    try {
+      leaderElectionService.start(this)
+    } catch {
+      case e: Exception =>
+        log.error("Could not start the JobManager because the leader election service did not " +
+          "start.", e)
+        throw new RuntimeException("Could not start the leader election service.", e)
+    }
   }
 
   override def postStop(): Unit = {
-    log.info(s"Stopping JobManager ${self.path.toSerializationFormat}.")
+    log.info(s"Stopping JobManager ${getAddress}.")
+
+    cancelAndClearEverything(new Exception("The JobManager is shutting down."))
 
     // disconnect the registered task managers
     instanceManager.getAllRegisteredInstances.asScala.foreach {
-      _.getActorGateway().tell(Disconnect("JobManager is shutting down"))
+      _.getActorGateway().tell(
+        Disconnect("JobManager is shutting down"),
+        new AkkaActorGateway(self, leaderSessionID.orNull))
     }
 
-    archive ! decorateMessage(PoisonPill)
+    try {
+      // revoke leadership and stop leader election service
+      leaderElectionService.stop()
+    } catch {
+      case e: Exception => log.error("Could not properly shutdown the leader election service.")
+    }
 
-    for((e,_) <- currentJobs.values) {
-      e.fail(new Exception("The JobManager is shutting down."))
+    if (archive != ActorRef.noSender) {
+      archive ! decorateMessage(PoisonPill)
     }
 
     instanceManager.shutdown()
@@ -158,13 +184,40 @@ class JobManager(
    */
   override def handleMessage: Receive = {
 
+    case GrantLeadership(newLeaderSessionID) =>
+      log.info(s"JobManager ${getAddress} was granted leadership with leader session ID " +
+        s"${newLeaderSessionID}.")
+
+      leaderSessionID = newLeaderSessionID
+
+      // confirming the leader session ID might be blocking, thus do it in a future
+      future{
+        leaderElectionService.confirmLeaderSessionID(newLeaderSessionID.orNull)
+      }(context.dispatcher)
+
+    case RevokeLeadership =>
+      log.info(s"JobManager ${self.path.toSerializationFormat} was revoked leadership.")
+
+      cancelAndClearEverything(new Exception("JobManager is no longer the leader."))
+
+      // disconnect the registered task managers
+      instanceManager.getAllRegisteredInstances.asScala.foreach {
+        _.getActorGateway().tell(
+          Disconnect("JobManager is no longer the leader"),
+          new AkkaActorGateway(self, leaderSessionID.orNull))
+      }
+
+      instanceManager.unregisterAllTaskManagers()
+
+      leaderSessionID = None
+
     case RegisterTaskManager(
-      registrationSessionID,
-      taskManager,
       connectionInfo,
       hardwareInformation,
       numberOfSlots) =>
 
+      val taskManager = sender()
+
       if (instanceManager.isRegistered(taskManager)) {
         val instanceID = instanceManager.getRegisteredInstance(taskManager).getId
 
@@ -172,9 +225,6 @@ class JobManager(
         //            TaskManager actor, but the ask future!
         sender() ! decorateMessage(
           AlreadyRegistered(
-            registrationSessionID,
-            leaderSessionID.get,
-            self,
             instanceID,
             libraryCacheManager.getBlobServerPort)
         )
@@ -186,15 +236,12 @@ class JobManager(
             connectionInfo,
             hardwareInformation,
             numberOfSlots,
-            leaderSessionID)
+            leaderSessionID.orNull)
 
           // IMPORTANT: Send the response to the "sender", which is not the
           //            TaskManager actor, but the ask future!
           sender() ! decorateMessage(
             AcknowledgeRegistration(
-              registrationSessionID,
-              leaderSessionID.get,
-              self,
               instanceID,
               libraryCacheManager.getBlobServerPort)
           )
@@ -212,7 +259,6 @@ class JobManager(
             //            TaskManager actor, but the ask future!
             sender() ! decorateMessage(
               RefuseRegistration(
-                registrationSessionID,
                 ExceptionUtils.stringifyException(e))
             )
         }
@@ -224,8 +270,8 @@ class JobManager(
     case RequestTotalNumberOfSlots =>
       sender ! decorateMessage(instanceManager.getTotalNumberOfSlots)
 
-    case SubmitJob(jobGraph, listen) =>
-      submitJob(jobGraph, listenToEvents = listen)
+    case SubmitJob(jobGraph, listeningBehaviour) =>
+      submitJob(jobGraph, listeningBehaviour)
 
     case CancelJob(jobID) =>
       log.info(s"Trying to cancel job with ID $jobID.")
@@ -334,22 +380,23 @@ class JobManager(
             jobInfo.end = timeStamp
 
             // is the client waiting for the job result?
-            newJobStatus match {
-              case JobStatus.FINISHED =>
-                val accumulatorResults: java.util.Map[String, SerializedValue[AnyRef]] = try {
+            if(jobInfo.client != ActorRef.noSender) {
+              newJobStatus match {
+                case JobStatus.FINISHED =>
+                  val accumulatorResults: java.util.Map[String, SerializedValue[AnyRef]] = try {
                   executionGraph.getAccumulatorsSerialized()
-                } catch {
-                  case e: Exception =>
-                    log.error(s"Cannot fetch serialized accumulators for job $jobID", e)
-                    Collections.emptyMap()
-                }
+                  } catch {
+                    case e: Exception =>
+                      log.error(s"Cannot fetch serialized accumulators for job $jobID", e)
+                      Collections.emptyMap()
+                  }
                 val result = new SerializedJobExecutionResult(
                   jobID,
                   jobInfo.duration,
                   accumulatorResults)
-                jobInfo.client ! decorateMessage(JobResultSuccess(result))
+                  jobInfo.client ! decorateMessage(JobResultSuccess(result))
 
-              case JobStatus.CANCELED =>
+                case JobStatus.CANCELED =>
                 // the error may be packed as a serialized throwable
                 val unpackedError = SerializedThrowable.get(
                   error, executionGraph.getUserClassLoader())
@@ -358,7 +405,7 @@ class JobManager(
                   new SerializedThrowable(
                     new JobCancellationException(jobID, "Job was cancelled.", unpackedError))))
 
-              case JobStatus.FAILED =>
+                case JobStatus.FAILED =>
                 val unpackedError = SerializedThrowable.get(
                   error, executionGraph.getUserClassLoader())
                 
@@ -366,11 +413,12 @@ class JobManager(
                   new SerializedThrowable(
                     new JobExecutionException(jobID, "Job execution failed.", unpackedError))))
 
-              case x =>
+                case x =>
                 val exception = new JobExecutionException(jobID, s"$x is not a terminal state.")
                 jobInfo.client ! decorateMessage(JobResultFailure(
                   new SerializedThrowable(exception)))
-                throw exception
+                  throw exception
+              }
             }
 
             removeJob(jobID)
@@ -463,6 +511,9 @@ class JobManager(
     case RequestBlobManagerPort =>
       sender ! decorateMessage(libraryCacheManager.getBlobServerPort)
 
+    case RequestArchive =>
+      sender ! decorateMessage(ResponseArchive(archive))
+
     case RequestRegisteredTaskManagers =>
       import scala.collection.JavaConverters._
       sender ! decorateMessage(
@@ -486,13 +537,13 @@ class JobManager(
 
     case RequestStackTrace(instanceID) =>
       val gateway = instanceManager.getRegisteredInstanceById(instanceID).getActorGateway
-      gateway.forward(SendStackTrace, new AkkaActorGateway(sender(), leaderSessionID))
+      gateway.forward(SendStackTrace, new AkkaActorGateway(sender, leaderSessionID.orNull))
 
     case Terminated(taskManager) =>
       if (instanceManager.isRegistered(taskManager)) {
         log.info(s"Task manager ${taskManager.path} terminated.")
 
-        instanceManager.unregisterTaskManager(taskManager)
+        instanceManager.unregisterTaskManager(taskManager, true)
         context.unwatch(taskManager)
       }
 
@@ -505,12 +556,12 @@ class JobManager(
       if (instanceManager.isRegistered(taskManager)) {
         log.info(s"Task manager ${taskManager.path} wants to disconnect, because $msg.")
 
-        instanceManager.unregisterTaskManager(taskManager)
+        instanceManager.unregisterTaskManager(taskManager, false)
         context.unwatch(taskManager)
       }
 
     case RequestLeaderSessionID =>
-      sender() ! ResponseLeaderSessionID(leaderSessionID)
+      sender() ! ResponseLeaderSessionID(leaderSessionID.orNull)
   }
 
   /**
@@ -519,10 +570,9 @@ class JobManager(
    * graph and the execution vertices are queued for scheduling.
    *
    * @param jobGraph representing the Flink job
-   * @param listenToEvents true if the sender wants to listen to job status and execution state
-   *                       change notifications. false if not.
+   * @param listeningBehaviour specifies the listening behaviour of the sender.
    */
-  private def submitJob(jobGraph: JobGraph, listenToEvents: Boolean): Unit = {
+  private def submitJob(jobGraph: JobGraph, listeningBehaviour: ListeningBehaviour): Unit = {
     if (jobGraph == null) {
       sender() ! decorateMessage(JobResultFailure(
         new SerializedThrowable(
@@ -560,6 +610,14 @@ class JobManager(
           throw new JobSubmissionException(jobId, "The given job is empty")
         }
 
+        val client = if(listeningBehaviour == ListeningBehaviour.DETACHED) {
+          // The client does not want to receive the SerializedJobExecutionResult
+          ActorRef.noSender
+        } else {
+          // Send the job execution result back to the sender
+          sender
+        }
+
         // see if there already exists an ExecutionGraph for the corresponding job ID
         executionGraph = currentJobs.getOrElseUpdate(
           jobGraph.getJobID,
@@ -571,7 +629,7 @@ class JobManager(
             timeout,
             jobGraph.getUserJarBlobKeys(),
             userCodeLoader),
-            JobInfo(sender(), System.currentTimeMillis())
+            JobInfo(client, System.currentTimeMillis())
           )
         )._1
 
@@ -603,52 +661,53 @@ class JobManager(
               s"The vertex ${vertex.getID} (${vertex.getName}) has no invokable class.")
           }
 
-          if (vertex.getParallelism() == ExecutionConfig.PARALLELISM_AUTO_MAX) {
-            vertex.setParallelism(numSlots)
-          }
+              if (vertex.getParallelism() == ExecutionConfig.PARALLELISM_AUTO_MAX) {
+                vertex.setParallelism(numSlots)
+              }
 
-          try {
-            vertex.initializeOnMaster(userCodeLoader)
-          }
-          catch {
+              try {
+                vertex.initializeOnMaster(userCodeLoader)
+              }
+              catch {
             case t: Throwable =>
               throw new JobExecutionException(jobId,
                 "Cannot initialize task '" + vertex.getName() + "': " + t.getMessage, t)
-          }
-        }
+              }
+            }
 
-        // topologically sort the job vertices and attach the graph to the existing one
-        val sortedTopology = jobGraph.getVerticesSortedTopologicallyFromSources()
-        if (log.isDebugEnabled) {
-          log.debug(s"Adding ${sortedTopology.size()} vertices from " +
-            s"job graph ${jobId} (${jobName}).")
-        }
-        executionGraph.attachJobGraph(sortedTopology)
+            // topologically sort the job vertices and attach the graph to the existing one
+            val sortedTopology = jobGraph.getVerticesSortedTopologicallyFromSources()
+            if (log.isDebugEnabled) {
+              log.debug(s"Adding ${sortedTopology.size()} vertices from " +
+                s"job graph ${jobId} (${jobName}).")
+            }
+            executionGraph.attachJobGraph(sortedTopology)
 
-        if (log.isDebugEnabled) {
-          log.debug(s"Successfully created execution graph from job graph ${jobId} (${jobName}).")
-        }
+            if (log.isDebugEnabled) {
+              log.debug("Successfully created execution graph from job " +
+                s"graph ${jobId} (${jobName}).")
+            }
 
-        // configure the state checkpointing
-        val snapshotSettings = jobGraph.getSnapshotSettings
-        if (snapshotSettings != null) {
+            // configure the state checkpointing
+            val snapshotSettings = jobGraph.getSnapshotSettings
+            if (snapshotSettings != null) {
 
-          val idToVertex: JobVertexID => ExecutionJobVertex = id => {
-            val vertex = executionGraph.getJobVertex(id)
-            if (vertex == null) {
-              throw new JobSubmissionException(jobId,
-                "The snapshot checkpointing settings refer to non-existent vertex " + id)
-            }
-            vertex
-          }
+              val idToVertex: JobVertexID => ExecutionJobVertex = id => {
+                val vertex = executionGraph.getJobVertex(id)
+                if (vertex == null) {
+                  throw new JobSubmissionException(jobId,
+                    "The snapshot checkpointing settings refer to non-existent vertex " + id)
+                }
+                vertex
+              }
 
-          val triggerVertices: java.util.List[ExecutionJobVertex] =
+              val triggerVertices: java.util.List[ExecutionJobVertex] =
             snapshotSettings.getVerticesToTrigger().asScala.map(idToVertex).asJava
 
-          val ackVertices: java.util.List[ExecutionJobVertex] =
+              val ackVertices: java.util.List[ExecutionJobVertex] =
             snapshotSettings.getVerticesToAcknowledge().asScala.map(idToVertex).asJava
 
-          val confirmVertices: java.util.List[ExecutionJobVertex] =
+              val confirmVertices: java.util.List[ExecutionJobVertex] =
             snapshotSettings.getVerticesToConfirm().asScala.map(idToVertex).asJava
 
           executionGraph.enableSnapshotCheckpointing(
@@ -658,15 +717,16 @@ class JobManager(
             ackVertices,
             confirmVertices,
             context.system,
-            leaderSessionID)
+            leaderSessionID.orNull)
         }
 
         // get notified about job status changes
-        executionGraph.registerJobStatusListener(new AkkaActorGateway(self, leaderSessionID))
+        executionGraph.registerJobStatusListener(
+          new AkkaActorGateway(self, leaderSessionID.orNull))
 
-        if (listenToEvents) {
+        if (listeningBehaviour == ListeningBehaviour.EXECUTION_RESULT_AND_STATE_CHANGES) {
           // the sender wants to be notified about state changes
-          val gateway = new AkkaActorGateway(sender(), leaderSessionID)
+          val gateway = new AkkaActorGateway(sender(), leaderSessionID.orNull)
 
           executionGraph.registerExecutionListener(gateway)
           executionGraph.registerJobStatusListener(gateway)
@@ -929,6 +989,53 @@ class JobManager(
     }
   }
 
+  /** Fails all currently running jobs and empties the list of currently running jobs. If the
+    * [[JobClientActor]] waits for a result, then a [[JobExecutionException]] is sent.
+    *
+    * @param cause Cause for the cancelling.
+    */
+  private def cancelAndClearEverything(cause: Throwable) {
+    for((jobID, (eg, jobInfo)) <- currentJobs) {
+      eg.fail(cause)
+
+      if(jobInfo.client != ActorRef.noSender) {
+        jobInfo.client ! decorateMessage(
+          Failure(
+            new JobExecutionException(
+              jobID,
+              "All jobs are cancelled and cleared.",
+              cause)
+          ))
+      }
+    }
+
+    currentJobs.clear()
+  }
+
+  override def grantLeadership(newLeaderSessionID: UUID): Unit = {
+    self ! decorateMessage(GrantLeadership(Option(newLeaderSessionID)))
+  }
+
+  override def revokeLeadership(): Unit = {
+    leaderSessionID = None
+    self ! decorateMessage(RevokeLeadership)
+  }
+
+  override def getAddress: String = {
+    AkkaUtils.getAkkaURL(context.system, self)
+  }
+
+  /** Handles error occuring in the leader election service
+    *
+    * @param exception
+    */
+  override def handleError(exception: Exception): Unit = {
+    log.error("Received an error from the LeaderElectionService.", exception)
+
+    // terminate JobManager in case of an error
+    self ! decorateMessage(PoisonPill)
+  }
+  
   /**
    * Updates the accumulators reported from a task manager via the Heartbeat message.
    * @param accumulators list of accumulator snapshots
@@ -1001,12 +1108,22 @@ object JobManager {
       System.exit(STARTUP_FAILURE_RETURN_CODE)
     }
 
-    // address and will not be reachable from anyone remote
-    if (listeningPort <= 0 || listeningPort >= 65536) {
-      val message = "Config parameter '" + ConfigConstants.JOB_MANAGER_IPC_PORT_KEY +
-        "' is invalid, it must be great than 0 and less than 65536."
-      LOG.error(message)
-      System.exit(STARTUP_FAILURE_RETURN_CODE)
+    if (ZooKeeperUtils.isZooKeeperHighAvailabilityEnabled(configuration)) {
+      // address and will not be reachable from anyone remote
+      if (listeningPort != 0) {
+        val message = "Config parameter '" + ConfigConstants.JOB_MANAGER_IPC_PORT_KEY +
+          "' is invalid, it must be equal to 0."
+        LOG.error(message)
+        System.exit(STARTUP_FAILURE_RETURN_CODE)
+      }
+    } else {
+      // address and will not be reachable from anyone remote
+      if (listeningPort <= 0 || listeningPort >= 65536) {
+        val message = "Config parameter '" + ConfigConstants.JOB_MANAGER_IPC_PORT_KEY +
+          "' is invalid, it must be greater than 0 and less than 65536."
+        LOG.error(message)
+        System.exit(STARTUP_FAILURE_RETURN_CODE)
+      }
     }
 
     // run the job manager
@@ -1089,7 +1206,7 @@ object JobManager {
     try {
       // bring up the job manager actor
       LOG.info("Starting JobManager actor")
-      val (jobManager, archiver) = startJobManagerActors(
+      val (jobManager, archive) = startJobManagerActors(
         configuration,
         jobManagerSystem,
         streamingMode)
@@ -1114,7 +1231,7 @@ object JobManager {
           jobManagerSystem,
           listeningAddress,
           Some(TaskManager.TASK_MANAGER_NAME),
-          Some(jobManager.path.toString),
+          None,
           true,
           streamingMode,
           classOf[TaskManager])
@@ -1130,9 +1247,15 @@ object JobManager {
       }
 
       if(configuration.getInteger(ConfigConstants.JOB_MANAGER_WEB_PORT_KEY, 0) >= 0) {
-        val lookupTimeout = AkkaUtils.getLookupTimeout(configuration)
-        val jobManagerGateway = JobManager.getJobManagerGateway(jobManager, lookupTimeout)
-        val archiverGateway = new AkkaActorGateway(archiver, jobManagerGateway.leaderSessionID())
+
+        // TODO: Add support for HA. Webserver has to work in dedicated mode. All transferred
+        // information has to be made serializable
+        val address = AkkaUtils.getAddress(jobManagerSystem)
+
+        configuration.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, address.host.get)
+        configuration.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, address.port.get)
+
+        val leaderRetrievalService = StandaloneUtils.createLeaderRetrievalService(configuration)
 
         // start the job manager web frontend
         val webServer = if (
@@ -1143,14 +1266,16 @@ object JobManager {
           LOG.info("Starting NEW JobManger web frontend")
           // start the new web frontend. we need to load this dynamically
           // because it is not in the same project/dependencies
-          startWebRuntimeMonitor(configuration, jobManagerGateway, archiverGateway)
+          startWebRuntimeMonitor(configuration, leaderRetrievalService, jobManagerSystem)
         }
         else {
           LOG.info("Starting JobManger web frontend")
-          new WebInfoServer(configuration, jobManagerGateway, archiverGateway)
+          new WebInfoServer(configuration, leaderRetrievalService, jobManagerSystem)
         }
 
-        webServer.start()
+        if(webServer != null) {
+          webServer.start()
+        }
       }
     }
     catch {
@@ -1208,6 +1333,13 @@ object JobManager {
       } text {
         "Network address for communication with the job manager"
       }
+
+      opt[Int]("webui-port").optional().action { (arg, conf) =>
+        conf.setWebUIPort(arg)
+        conf
+      } text {
+        "Port for the UI web server"
+      }
     }
 
     val config = parser.parse(args, new JobManagerCliOptions()).getOrElse {
@@ -1232,42 +1364,38 @@ object JobManager {
       configuration.setString(ConfigConstants.FLINK_BASE_DIR_PATH_KEY, configDir + "/..")
     }
 
+    if (config.getWebUIPort() >= 0) {
+      configuration.setInteger(ConfigConstants.JOB_MANAGER_WEB_PORT_KEY, config.getWebUIPort())
+    }
+
+    if (config.getHost() != null) {
+      configuration.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, config.getHost())
+    }
+
+    val host = configuration.getString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, null)
+
     // high availability mode
-    val (hostname: String, port: Int ) = 
-      if (ZooKeeperUtil.isJobManagerHighAvailabilityEnabled(configuration)) {
-        // TODO @removeme @tillrohrmann This is the place where the host and random port for JM is
-        // chosen.  For the FlinkMiniCluster you have to choose it on your own.
+    val port: Int =
+      if (ZooKeeperUtils.isZooKeeperHighAvailabilityEnabled(configuration)) {
         LOG.info("Starting JobManager in High-Availability Mode")
-  
-        if (config.getHost() == null) {
-          throw new Exception("Missing parameter '--host'. Parameter is required when " +
-            "running in high-availability mode")
-        }
-  
-        // Let web server listen on random port
-        configuration.setInteger(ConfigConstants.JOB_MANAGER_WEB_PORT_KEY, 0)
-  
-        (config.getHost(), 0)
+
+        configuration.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, 0)
+        0
       }
       else {
         LOG.info("Staring JobManager without high-availability")
-        
-        if (config.getHost() != null) {
-          throw new Exception("Found an explicit address for JobManager communication " +
-            "via the CLI option '--host'.\n" +
-            "This parameter must only be set if the JobManager is started in high-availability " +
-            "mode and connects to a ZooKeeper quorum.\n" +
-            "Please configure ZooKeeper or don't set the '--host' option, so that the JobManager " +
-            "uses the address configured under 'conf/flink-conf.yaml'.")
-        }
   
-        val host = configuration.getString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, null)
-        val port = configuration.getInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY,
+        configuration.getInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY,
             ConfigConstants.DEFAULT_JOB_MANAGER_IPC_PORT)
-        (host, port)
       }
 
-    (configuration, config.getJobManagerMode(), config.getStreamingMode(), hostname, port)
+    val executionMode = config.getJobManagerMode
+    val streamingMode = config.getStreamingMode
+
+    LOG.info(s"Starting JobManager on $host:$port with execution mode $executionMode and " +
+      s"streaming mode $streamingMode")
+
+    (configuration, executionMode, streamingMode, host, port)
   }
 
   /**
@@ -1278,9 +1406,17 @@ object JobManager {
    * @param configuration The configuration from which to parse the config values.
    * @return The members for a default JobManager.
    */
-  def createJobManagerComponents(configuration: Configuration)
-    : (ExecutionContext, InstanceManager, FlinkScheduler, BlobLibraryCacheManager,
-      Props, Int, Long, FiniteDuration, Int) = {
+  def createJobManagerComponents(configuration: Configuration) :
+    (ExecutionContext,
+    InstanceManager,
+    FlinkScheduler,
+    BlobLibraryCacheManager,
+    Props,
+    Int, // execution retries
+    Long, // delay between retries
+    FiniteDuration, // timeout
+    Int, // number of archived jobs
+    LeaderElectionService) = {
 
     val timeout: FiniteDuration = AkkaUtils.getTimeout(configuration)
 
@@ -1346,6 +1482,8 @@ object JobManager {
       }
     }
 
+    val leaderElectionService = LeaderElectionUtils.createLeaderElectionService(configuration)
+
     (executionContext,
       instanceManager,
       scheduler,
@@ -1353,8 +1491,9 @@ object JobManager {
       archiveProps,
       executionRetries,
       delayBetweenRetries,
-      timeout,
-      archiveCount)
+      timeout, 
+      archiveCount, 
+      leaderElectionService)
   }
 
   /**
@@ -1386,7 +1525,7 @@ object JobManager {
    * @param actorSystem The actor system running the JobManager
    * @param jobMangerActorName Optionally the name of the JobManager actor. If none is given,
    *                          the actor will have the name generated by the actor system.
-   * @param archiverActorName Optionally the name of the archive actor. If none is given,
+   * @param archiveActorName Optionally the name of the archive actor. If none is given,
    *                          the actor will have the name generated by the actor system.
    * @param streamingMode The mode to run the system in (streaming vs. batch-only)
    * 
@@ -1396,7 +1535,7 @@ object JobManager {
       configuration: Configuration,
       actorSystem: ActorSystem,
       jobMangerActorName: Option[String],
-      archiverActorName: Option[String],
+      archiveActorName: Option[String],
       streamingMode: StreamingMode)
     : (ActorRef, ActorRef) = {
 
@@ -1408,10 +1547,11 @@ object JobManager {
       executionRetries,
       delayBetweenRetries,
       timeout,
-      _) = createJobManagerComponents(configuration)
+      _,
+      leaderElectionService) = createJobManagerComponents(configuration)
 
     // start the archiver with the given name, or without (avoid name conflicts)
-    val archiver: ActorRef = archiverActorName match {
+    val archive: ActorRef = archiveActorName match {
       case Some(actorName) => actorSystem.actorOf(archiveProps, actorName)
       case None => actorSystem.actorOf(archiveProps)
     }
@@ -1423,18 +1563,19 @@ object JobManager {
       instanceManager,
       scheduler,
       libraryCacheManager,
-      archiver,
+      archive,
       executionRetries,
       delayBetweenRetries,
       timeout,
-      streamingMode)
+      streamingMode,
+      leaderElectionService)
 
     val jobManager: ActorRef = jobMangerActorName match {
       case Some(actorName) => actorSystem.actorOf(jobManagerProps, actorName)
       case None => actorSystem.actorOf(jobManagerProps)
     }
 
-    (jobManager, archiver)
+    (jobManager, archive)
   }
 
   def startActor(props: Props, actorSystem: ActorSystem): ActorRef = {
@@ -1452,9 +1593,13 @@ object JobManager {
    * @param address The address of the JobManager's actor system.
    * @return The akka URL of the JobManager actor.
    */
-  def getRemoteJobManagerAkkaURL(address: InetSocketAddress): String = {
+  def getRemoteJobManagerAkkaURL(
+      address: InetSocketAddress,
+      name: Option[String] = None)
+    : String = {
     val hostPort = address.getAddress().getHostAddress() + ":" + address.getPort()
-    s"akka.tcp://flink@$hostPort/user/$JOB_MANAGER_NAME"
+
+    getJobManagerAkkaURLHelper(s"akka.tcp://flink@$hostPort", name)
   }
 
   /**
@@ -1463,17 +1608,24 @@ object JobManager {
    *
    * @return The local akka URL of the JobManager actor.
    */
-  def getLocalJobManagerAkkaURL: String = {
-    "akka://flink/user/" + JOB_MANAGER_NAME
+  def getLocalJobManagerAkkaURL(name: Option[String] = None): String = {
+    getJobManagerAkkaURLHelper("akka://flink", name)
+  }
+
+  def getJobManagerAkkaURL(system: ActorSystem, name: Option[String] = None): String = {
+    getJobManagerAkkaURLHelper(AkkaUtils.getAddress(system).toString, name)
   }
 
-  def getJobManagerRemoteReferenceFuture(
+  private def getJobManagerAkkaURLHelper(address: String, name: Option[String]): String = {
+    address + "/user/" + name.getOrElse(JOB_MANAGER_NAME)
+  }
+
+  def getJobManagerActorRefFuture(
       address: InetSocketAddress,
       system: ActorSystem,
       timeout: FiniteDuration)
     : Future[ActorRef] = {
-
-    AkkaUtils.getReference(getRemoteJobManagerAkkaURL(address), system, timeout)
+    AkkaUtils.getActorRefFuture(getRemoteJobManagerAkkaURL(address), system, timeout)
   }
 
   /**
@@ -1486,25 +1638,12 @@ object JobManager {
    * @return The ActorRef to the JobManager
    */
   @throws(classOf[IOException])
-  def getJobManagerRemoteReference(
+  def getJobManagerActorRef(
       jobManagerUrl: String,
       system: ActorSystem,
       timeout: FiniteDuration)
     : ActorRef = {
-
-    try {
-      val future = AkkaUtils.getReference(jobManagerUrl, system, timeout)
-      Await.result(future, timeout)
-    }
-    catch {
-      case e @ (_ : ActorNotFound | _ : TimeoutException) =>
-        throw new IOException(
-          s"JobManager at $jobManagerUrl not reachable. " +
-            s"Please make sure that the JobManager is running and its port is reachable.", e)
-
-      case e: IOException =>
-        throw new IOException("Could not connect to JobManager at " + jobManagerUrl, e)
-    }
+    AkkaUtils.getActorRef(jobManagerUrl, system, timeout)
   }
 
   /**
@@ -1517,14 +1656,14 @@ object JobManager {
    * @return The ActorRef to the JobManager
    */
   @throws(classOf[IOException])
-  def getJobManagerRemoteReference(
+  def getJobManagerActorRef(
       address: InetSocketAddress,
       system: ActorSystem,
       timeout: FiniteDuration)
     : ActorRef = {
 
     val jmAddress = getRemoteJobManagerAkkaURL(address)
-    getJobManagerRemoteReference(jmAddress, system, timeout)
+    getJobManagerActorRef(jmAddress, system, timeout)
   }
 
   /**
@@ -1537,37 +1676,16 @@ object JobManager {
    * @return The ActorRef to the JobManager
    */
   @throws(classOf[IOException])
-  def getJobManagerRemoteReference(
+  def getJobManagerActorRef(
       address: InetSocketAddress,
       system: ActorSystem,
       config: Configuration)
     : ActorRef = {
 
     val timeout = AkkaUtils.getLookupTimeout(config)
-    getJobManagerRemoteReference(address, system, timeout)
+    getJobManagerActorRef(address, system, timeout)
   }
 
-  /** Returns the [[ActorGateway]] for the provided JobManager. The function automatically
-    * retrieves the current leader session ID from the JobManager and instantiates the
-    * [[AkkaActorGateway]] with it.
-    *
-    * @param jobManager ActorRef to the [[JobManager]]
-    * @param timeout Timeout for the blocking leader session ID retrieval
-    * @throws java.lang.Exception
-    * @return Gateway to the specified JobManager
-    */
-  @throws(classOf[Exception])
-  def getJobManagerGateway(
-    jobManager: ActorRef,
-    timeout: FiniteDuration
-    ): ActorGateway = {
-    val futureLeaderSessionID = (jobManager ? RequestLeaderSessionID)(timeout)
-      .mapTo[ResponseLeaderSessionID]
-
-    val leaderSessionID = Await.result(futureLeaderSessionID, timeout).leaderSessionID
-
-    new AkkaActorGateway(jobManager, leaderSessionID)
-  }
 
   // --------------------------------------------------------------------------
   //  Utilities
@@ -1581,29 +1699,28 @@ object JobManager {
    * this method does not throw any exceptions, but only logs them.
    * 
    * @param config The configuration for the runtime monitor.
-   * @param jobManager The JobManager actor gateway.
-   * @param archiver The execution graph archive actor.
+   * @param leaderRetrievalService Leader retrieval service to get the leading JobManager
    */
   def startWebRuntimeMonitor(
       config: Configuration,
-      jobManager: ActorGateway,
-      archiver: ActorGateway)
+      leaderRetrievalService: LeaderRetrievalService,
+      actorSystem: ActorSystem)
     : WebMonitor = {
     // try to load and instantiate the class
     try {
       val classname = "org.apache.flink.runtime.webmonitor.WebRuntimeMonitor"
       val clazz: Class[_ <: WebMonitor] = Class.forName(classname)
-                                               .asSubclass(classOf[WebMonitor])
+        .asSubclass(classOf[WebMonitor])
 
       val ctor: Constructor[_ <: WebMonitor] = clazz.getConstructor(classOf[Configuration],
-                                                                    classOf[ActorGateway],
-                                                                    classOf[ActorGateway])
-      ctor.newInstance(config, jobManager, archiver)
+        classOf[LeaderRetrievalService],
+        classOf[ActorSystem])
+      ctor.newInstance(config, leaderRetrievalService, actorSystem)
     }
     catch {
       case e: ClassNotFoundException =>
         LOG.error("Could not load web runtime monitor. " +
-            "Probably reason: flink-runtime-web is not in the classpath")
+          "Probably reason: flink-runtime-web is not in the classpath")
         LOG.debug("Caught exception", e)
         null
       case e: InvocationTargetException =>

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/MemoryArchivist.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/MemoryArchivist.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/MemoryArchivist.scala
index e2891de..702e34b 100644
--- a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/MemoryArchivist.scala
+++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/MemoryArchivist.scala
@@ -71,6 +71,10 @@ class MemoryArchivist(private val max_entries: Int)
   var canceledCnt: Int = 0
   var failedCnt: Int = 0
 
+  override def preStart(): Unit = {
+    log.info(s"Started memory archivist ${self.path}")
+  }
+
   override def handleMessage: Receive = {
     
     /* Receive Execution Graph to archive */

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/ExecutionGraphMessages.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/ExecutionGraphMessages.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/ExecutionGraphMessages.scala
index 83bafaa..2369d3c 100644
--- a/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/ExecutionGraphMessages.scala
+++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/ExecutionGraphMessages.scala
@@ -19,7 +19,7 @@
 package org.apache.flink.runtime.messages
 
 import java.text.SimpleDateFormat
-import java.util.Date
+import java.util.{UUID, Date}
 
 import org.apache.flink.api.common.JobID
 import org.apache.flink.runtime.execution.ExecutionState

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/JobManagerMessages.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/JobManagerMessages.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/JobManagerMessages.scala
index 1c250af..d7bbb8d 100644
--- a/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/JobManagerMessages.scala
+++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/JobManagerMessages.scala
@@ -20,7 +20,9 @@ package org.apache.flink.runtime.messages
 
 import java.util.UUID
 
+import akka.actor.ActorRef
 import org.apache.flink.api.common.JobID
+import org.apache.flink.runtime.akka.ListeningBehaviour
 import org.apache.flink.runtime.client.{SerializedJobExecutionResult, JobStatusMessage}
 import org.apache.flink.runtime.executiongraph.{ExecutionAttemptID, ExecutionGraph}
 import org.apache.flink.runtime.instance.{InstanceID, Instance}
@@ -35,17 +37,32 @@ import scala.collection.JavaConverters._
  */
 object JobManagerMessages {
 
-  case class LeaderSessionMessage(leaderSessionID: Option[UUID], message: Any)
+  /** Wrapper class for leader session messages. Leader session messages implement the
+    * [[RequiresLeaderSessionID]] interface and have to be wrapped in a [[LeaderSessionMessage]],
+    * which also contains the current leader session ID.
+    *
+    * @param leaderSessionID Current leader session ID or null, if no leader session ID was set
+    * @param message [[RequiresLeaderSessionID]] message to be wrapped in a [[LeaderSessionMessage]]
+    */
+  case class LeaderSessionMessage(leaderSessionID: UUID, message: Any)
 
   /**
-   * Submits a job to the job manager. If [[registerForEvents]] is true,
-   * then the sender will be registered as listener for the state change messages.
+   * Submits a job to the job manager. Depending on the [[listeningBehaviour]],
+   * the sender registers for different messages. If [[ListeningBehaviour.DETACHED]], then
+   * it will only be informed whether the submission was successful or not. If
+   * [[ListeningBehaviour.EXECUTION_RESULT]], then it will additionally receive the execution
+   * result. If [[ListeningBehaviour.EXECUTION_RESULT_AND_STATE_CHANGES]], then it will additionally
+   * receive the job status change notifications.
+   *
    * The submission result will be sent back to the sender as a success message.
    *
    * @param jobGraph The job to be submitted to the JobManager
-   * @param registerForEvents if true, then register for state change events
+   * @param listeningBehaviour Specifies to what the sender wants to listen (detached, execution
+   *                           result, execution result and state changes)
    */
-  case class SubmitJob(jobGraph: JobGraph, registerForEvents: Boolean)
+  case class SubmitJob(
+      jobGraph: JobGraph,
+      listeningBehaviour: ListeningBehaviour)
     extends RequiresLeaderSessionID
 
   /**
@@ -161,7 +178,7 @@ object JobManagerMessages {
     *
     * @param leaderSessionID
     */
-  case class ResponseLeaderSessionID(leaderSessionID: Option[UUID])
+  case class ResponseLeaderSessionID(leaderSessionID: UUID)
 
   /**
    * Denotes a successful job submission.
@@ -299,6 +316,22 @@ object JobManagerMessages {
 
   case object JobManagerStatusAlive extends JobManagerStatus
 
+  /** Grants leadership to the receiver. The message contains the new leader session id.
+    *
+     * @param leaderSessionID
+    */
+  case class GrantLeadership(leaderSessionID: Option[UUID])
+
+  /** Revokes leadership of the receiver.
+    */
+  case object RevokeLeadership
+
+  /** Requests the ActorRef of the archiver */
+  case object RequestArchive
+
+  /** Response containing the ActorRef of the archiver */
+  case class ResponseArchive(actor: ActorRef)
+
   // --------------------------------------------------------------------------
   // Utility methods to allow simpler case object access from Java
   // --------------------------------------------------------------------------
@@ -338,4 +371,8 @@ object JobManagerMessages {
   def getRequestLeaderSessionID: AnyRef = {
     RequestLeaderSessionID
   }
+
+  def getRequestArchive: AnyRef = {
+    RequestArchive
+  }
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/RegistrationMessages.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/RegistrationMessages.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/RegistrationMessages.scala
index b435ebc..941d63f 100644
--- a/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/RegistrationMessages.scala
+++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/RegistrationMessages.scala
@@ -18,9 +18,6 @@
 
 package org.apache.flink.runtime.messages
 
-import java.util.UUID
-
-import akka.actor.ActorRef
 import org.apache.flink.runtime.instance.{InstanceConnectionInfo, InstanceID, HardwareDescription}
 
 import scala.concurrent.duration.{Deadline, FiniteDuration}
@@ -34,21 +31,18 @@ object RegistrationMessages {
   /**
    * Marker trait for registration messages.
    */
-  trait RegistrationMessage {
-    def registrationSessionID: UUID
-  }
+  trait RegistrationMessage extends RequiresLeaderSessionID {}
 
   /**
    * Triggers the TaskManager to attempt a registration at the JobManager.
    *
-   * @param jobManagerAkkaURL The actor URL of the JobManager.
+   * @param jobManagerURL Akka URL to the JobManager
    * @param timeout The timeout for the message. The next retry will double this timeout.
    * @param deadline Optional deadline until when the registration must be completed.
    * @param attempt The attempt number, for logging.
    */
   case class TriggerTaskManagerRegistration(
-      registrationSessionID: UUID,
-      jobManagerAkkaURL: String,
+      jobManagerURL: String,
       timeout: FiniteDuration,
       deadline: Option[Deadline],
       attempt: Int)
@@ -58,14 +52,11 @@ object RegistrationMessages {
    * Registers a task manager at the job manager. A successful registration is acknowledged by
    * [[AcknowledgeRegistration]].
    *
-   * @param taskManager The TaskManager actor.
    * @param connectionInfo The TaskManagers connection information.
    * @param resources The TaskManagers resources.
    * @param numberOfSlots The number of processing slots offered by the TaskManager.
    */
   case class RegisterTaskManager(
-      registrationSessionID: UUID,
-      taskManager: ActorRef,
       connectionInfo: InstanceConnectionInfo,
       resources: HardwareDescription,
       numberOfSlots: Int)
@@ -80,9 +71,6 @@ object RegistrationMessages {
    * @param blobPort The server port where the JobManager's BLOB service runs.
    */
   case class AcknowledgeRegistration(
-      registrationSessionID: UUID,
-      leaderSessionID: UUID,
-      jobManager: ActorRef,
       instanceID: InstanceID,
       blobPort: Int)
     extends RegistrationMessage
@@ -94,9 +82,6 @@ object RegistrationMessages {
    * @param blobPort The server port where the JobManager's BLOB service runs.
    */
   case class AlreadyRegistered(
-      registrationSessionID: UUID,
-      leaderSessionID: UUID,
-      jobManager: ActorRef,
       instanceID: InstanceID,
       blobPort: Int)
     extends RegistrationMessage
@@ -107,6 +92,6 @@ object RegistrationMessages {
    *
    * @param reason Reason why the task manager registration was refused
    */
-  case class RefuseRegistration(registrationSessionID: UUID, reason: String)
+  case class RefuseRegistration(reason: String)
     extends RegistrationMessage
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/TaskControlMessages.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/TaskControlMessages.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/TaskControlMessages.scala
new file mode 100644
index 0000000..a80ca99
--- /dev/null
+++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/TaskControlMessages.scala
@@ -0,0 +1,184 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.messages
+
+import org.apache.flink.runtime.deployment.{InputChannelDeploymentDescriptor, TaskDeploymentDescriptor}
+import org.apache.flink.runtime.execution.ExecutionState
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID
+import org.apache.flink.runtime.jobgraph.{IntermediateDataSetID, IntermediateResultPartitionID}
+import org.apache.flink.runtime.messages.JobManagerMessages.RequestPartitionState
+import org.apache.flink.runtime.taskmanager.TaskExecutionState
+
+/**
+ * A set of messages that control the deployment and the state of Tasks executed
+ * on the TaskManager.
+ */
+object TaskMessages {
+
+  /**
+   * Marker trait for task messages.
+   */
+  trait TaskMessage
+
+  // --------------------------------------------------------------------------
+  //  Starting and stopping Tasks
+  // --------------------------------------------------------------------------
+
+  /**
+   * Submits a task to the task manager. The result is to this message is a
+   * [[TaskOperationResult]] message.
+   *
+   * @param tasks Descriptor which contains the information to start the task.
+   */
+  case class SubmitTask(tasks: TaskDeploymentDescriptor)
+    extends TaskMessage with RequiresLeaderSessionID
+
+  /**
+   * Cancels the task associated with [[attemptID]]. The result is sent back to the sender as a
+   * [[TaskOperationResult]] message.
+   *
+   * @param attemptID The task's execution attempt ID.
+   */
+  case class CancelTask(attemptID: ExecutionAttemptID)
+    extends TaskMessage with RequiresLeaderSessionID
+
+  /**
+   * Triggers a fail of specified task from the outside (as opposed to the task throwing
+   * an exception itself) with the given exception as the cause.
+   *
+   * @param executionID The task's execution attempt ID.
+   * @param cause The reason for the external failure.
+   */
+  case class FailTask(executionID: ExecutionAttemptID, cause: Throwable)
+    extends TaskMessage
+
+  /**
+   * Notifies the TaskManager that the task has reached its final state,
+   * either FINISHED, CANCELED, or FAILED.
+   *
+   * @param executionID The task's execution attempt ID.
+   */
+  case class TaskInFinalState(executionID: ExecutionAttemptID)
+    extends TaskMessage
+
+
+  // --------------------------------------------------------------------------
+  //  Updates to Intermediate Results
+  // --------------------------------------------------------------------------
+
+  /**
+   * Answer to a [[RequestPartitionState]] with the state of the respective partition.
+   */
+  case class PartitionState(
+      taskExecutionId: ExecutionAttemptID,
+      taskResultId: IntermediateDataSetID,
+      partitionId: IntermediateResultPartitionID,
+      state: ExecutionState)
+    extends TaskMessage with RequiresLeaderSessionID
+
+  /**
+   * Base class for messages that update the information about location of input partitions
+   */
+  abstract sealed class UpdatePartitionInfo extends TaskMessage with RequiresLeaderSessionID {
+    def executionID: ExecutionAttemptID
+  }
+
+  /**
+   *
+   * @param executionID The task's execution attempt ID.
+   * @param resultId The input reader to update.
+   * @param partitionInfo The partition info update.
+   */
+  case class UpdateTaskSinglePartitionInfo(
+      executionID: ExecutionAttemptID,
+      resultId: IntermediateDataSetID,
+      partitionInfo: InputChannelDeploymentDescriptor)
+    extends UpdatePartitionInfo
+
+  /**
+   *
+   * @param executionID The task's execution attempt ID.
+   * @param partitionInfos List of input gates with channel descriptors to update.
+   */
+  case class UpdateTaskMultiplePartitionInfos(
+      executionID: ExecutionAttemptID,
+      partitionInfos: Seq[(IntermediateDataSetID, InputChannelDeploymentDescriptor)])
+    extends UpdatePartitionInfo
+
+  /**
+   * Fails (and releases) all intermediate result partitions identified by
+   * [[executionID]] from the task manager.
+   *
+   * @param executionID The task's execution attempt ID.
+   */
+  case class FailIntermediateResultPartitions(executionID: ExecutionAttemptID)
+    extends TaskMessage with RequiresLeaderSessionID
+
+
+  // --------------------------------------------------------------------------
+  //  Report Messages
+  // --------------------------------------------------------------------------
+
+  /**
+   * Denotes a state change of a task at the JobManager. The update success is acknowledged by a
+   * boolean value which is sent back to the sender.
+   *
+   * @param taskExecutionState The changed task state
+   */
+  case class UpdateTaskExecutionState(taskExecutionState: TaskExecutionState)
+    extends TaskMessage with RequiresLeaderSessionID
+
+  /**
+   * Response message to updates in the task state. Send for example as a response to
+   *
+   *  - [[SubmitTask]]
+   *  - [[CancelTask]]
+   *
+   * @param executionID identifying the respective task
+   * @param success indicating whether the operation has been successful
+   * @param description Optional description for unsuccessful results.
+   */
+  case class TaskOperationResult(
+      executionID: ExecutionAttemptID,
+      success: Boolean,
+      description: String)
+    extends TaskMessage {
+    def this(executionID: ExecutionAttemptID, success: Boolean) = this(executionID, success, "")
+  }
+
+
+  // --------------------------------------------------------------------------
+  //  Utility Functions
+  // --------------------------------------------------------------------------
+
+  def createUpdateTaskMultiplePartitionInfos(
+      executionID: ExecutionAttemptID,
+      resultIDs: java.util.List[IntermediateDataSetID],
+      partitionInfos: java.util.List[InputChannelDeploymentDescriptor])
+    : UpdateTaskMultiplePartitionInfos = {
+
+    require(resultIDs.size() == partitionInfos.size(),
+      "ResultIDs must have the same length as partitionInfos.")
+
+    import scala.collection.JavaConverters.asScalaBufferConverter
+
+    new UpdateTaskMultiplePartitionInfos(executionID,
+      resultIDs.asScala.zip(partitionInfos.asScala))
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/TaskManagerMessages.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/TaskManagerMessages.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/TaskManagerMessages.scala
index 6cb571c..30c82fe 100644
--- a/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/TaskManagerMessages.scala
+++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/TaskManagerMessages.scala
@@ -18,6 +18,9 @@
 
 package org.apache.flink.runtime.messages
 
+import java.util.UUID
+
+import akka.actor.ActorRef
 import org.apache.flink.runtime.accumulators.AccumulatorSnapshot
 import org.apache.flink.runtime.instance.InstanceID
 
@@ -92,17 +95,25 @@ object TaskManagerMessages {
 
   /**
    * Requests a notification from the task manager as soon as the task manager has been
-   * registered at the job manager. Once the task manager is registered at the job manager a
+   * registered at any job manager. Once the task manager is registered at any job manager a
    * [[RegisteredAtJobManager]] message will be sent to the sender.
    */
-  case object NotifyWhenRegisteredAtJobManager
+  case object NotifyWhenRegisteredAtAnyJobManager
 
   /**
-   * Acknowledges that the task manager has been successfully registered at the job manager. This
-   * message is a response to [[NotifyWhenRegisteredAtJobManager]].
+   * Acknowledges that the task manager has been successfully registered at any job manager. This
+   * message is a response to [[NotifyWhenRegisteredAtAnyJobManager]].
    */
   case object RegisteredAtJobManager
 
+  /** Tells the address of the new leading [[org.apache.flink.runtime.jobmanager.JobManager]]
+    * and the new leader session ID.
+    *
+    * @param jobManagerAddress Address of the new leading JobManager
+    * @param leaderSessionID New leader session ID
+    */
+  case class JobManagerLeaderAddress(jobManagerAddress: String, leaderSessionID: UUID)
+
 
   // --------------------------------------------------------------------------
   //  Utility getters for case objects to simplify access from Java
@@ -113,7 +124,7 @@ object TaskManagerMessages {
    * @return The NotifyWhenRegisteredAtJobManager case object instance.
    */
   def getNotifyWhenRegisteredAtJobManagerMessage:
-            NotifyWhenRegisteredAtJobManager.type = NotifyWhenRegisteredAtJobManager
+            NotifyWhenRegisteredAtAnyJobManager.type = NotifyWhenRegisteredAtAnyJobManager
 
   /**
    * Accessor for the case object instance, to simplify Java interoperability.

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/TaskMessages.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/TaskMessages.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/TaskMessages.scala
deleted file mode 100644
index a80ca99..0000000
--- a/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/TaskMessages.scala
+++ /dev/null
@@ -1,184 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.runtime.messages
-
-import org.apache.flink.runtime.deployment.{InputChannelDeploymentDescriptor, TaskDeploymentDescriptor}
-import org.apache.flink.runtime.execution.ExecutionState
-import org.apache.flink.runtime.executiongraph.ExecutionAttemptID
-import org.apache.flink.runtime.jobgraph.{IntermediateDataSetID, IntermediateResultPartitionID}
-import org.apache.flink.runtime.messages.JobManagerMessages.RequestPartitionState
-import org.apache.flink.runtime.taskmanager.TaskExecutionState
-
-/**
- * A set of messages that control the deployment and the state of Tasks executed
- * on the TaskManager.
- */
-object TaskMessages {
-
-  /**
-   * Marker trait for task messages.
-   */
-  trait TaskMessage
-
-  // --------------------------------------------------------------------------
-  //  Starting and stopping Tasks
-  // --------------------------------------------------------------------------
-
-  /**
-   * Submits a task to the task manager. The result is to this message is a
-   * [[TaskOperationResult]] message.
-   *
-   * @param tasks Descriptor which contains the information to start the task.
-   */
-  case class SubmitTask(tasks: TaskDeploymentDescriptor)
-    extends TaskMessage with RequiresLeaderSessionID
-
-  /**
-   * Cancels the task associated with [[attemptID]]. The result is sent back to the sender as a
-   * [[TaskOperationResult]] message.
-   *
-   * @param attemptID The task's execution attempt ID.
-   */
-  case class CancelTask(attemptID: ExecutionAttemptID)
-    extends TaskMessage with RequiresLeaderSessionID
-
-  /**
-   * Triggers a fail of specified task from the outside (as opposed to the task throwing
-   * an exception itself) with the given exception as the cause.
-   *
-   * @param executionID The task's execution attempt ID.
-   * @param cause The reason for the external failure.
-   */
-  case class FailTask(executionID: ExecutionAttemptID, cause: Throwable)
-    extends TaskMessage
-
-  /**
-   * Notifies the TaskManager that the task has reached its final state,
-   * either FINISHED, CANCELED, or FAILED.
-   *
-   * @param executionID The task's execution attempt ID.
-   */
-  case class TaskInFinalState(executionID: ExecutionAttemptID)
-    extends TaskMessage
-
-
-  // --------------------------------------------------------------------------
-  //  Updates to Intermediate Results
-  // --------------------------------------------------------------------------
-
-  /**
-   * Answer to a [[RequestPartitionState]] with the state of the respective partition.
-   */
-  case class PartitionState(
-      taskExecutionId: ExecutionAttemptID,
-      taskResultId: IntermediateDataSetID,
-      partitionId: IntermediateResultPartitionID,
-      state: ExecutionState)
-    extends TaskMessage with RequiresLeaderSessionID
-
-  /**
-   * Base class for messages that update the information about location of input partitions
-   */
-  abstract sealed class UpdatePartitionInfo extends TaskMessage with RequiresLeaderSessionID {
-    def executionID: ExecutionAttemptID
-  }
-
-  /**
-   *
-   * @param executionID The task's execution attempt ID.
-   * @param resultId The input reader to update.
-   * @param partitionInfo The partition info update.
-   */
-  case class UpdateTaskSinglePartitionInfo(
-      executionID: ExecutionAttemptID,
-      resultId: IntermediateDataSetID,
-      partitionInfo: InputChannelDeploymentDescriptor)
-    extends UpdatePartitionInfo
-
-  /**
-   *
-   * @param executionID The task's execution attempt ID.
-   * @param partitionInfos List of input gates with channel descriptors to update.
-   */
-  case class UpdateTaskMultiplePartitionInfos(
-      executionID: ExecutionAttemptID,
-      partitionInfos: Seq[(IntermediateDataSetID, InputChannelDeploymentDescriptor)])
-    extends UpdatePartitionInfo
-
-  /**
-   * Fails (and releases) all intermediate result partitions identified by
-   * [[executionID]] from the task manager.
-   *
-   * @param executionID The task's execution attempt ID.
-   */
-  case class FailIntermediateResultPartitions(executionID: ExecutionAttemptID)
-    extends TaskMessage with RequiresLeaderSessionID
-
-
-  // --------------------------------------------------------------------------
-  //  Report Messages
-  // --------------------------------------------------------------------------
-
-  /**
-   * Denotes a state change of a task at the JobManager. The update success is acknowledged by a
-   * boolean value which is sent back to the sender.
-   *
-   * @param taskExecutionState The changed task state
-   */
-  case class UpdateTaskExecutionState(taskExecutionState: TaskExecutionState)
-    extends TaskMessage with RequiresLeaderSessionID
-
-  /**
-   * Response message to updates in the task state. Send for example as a response to
-   *
-   *  - [[SubmitTask]]
-   *  - [[CancelTask]]
-   *
-   * @param executionID identifying the respective task
-   * @param success indicating whether the operation has been successful
-   * @param description Optional description for unsuccessful results.
-   */
-  case class TaskOperationResult(
-      executionID: ExecutionAttemptID,
-      success: Boolean,
-      description: String)
-    extends TaskMessage {
-    def this(executionID: ExecutionAttemptID, success: Boolean) = this(executionID, success, "")
-  }
-
-
-  // --------------------------------------------------------------------------
-  //  Utility Functions
-  // --------------------------------------------------------------------------
-
-  def createUpdateTaskMultiplePartitionInfos(
-      executionID: ExecutionAttemptID,
-      resultIDs: java.util.List[IntermediateDataSetID],
-      partitionInfos: java.util.List[InputChannelDeploymentDescriptor])
-    : UpdateTaskMultiplePartitionInfos = {
-
-    require(resultIDs.size() == partitionInfos.size(),
-      "ResultIDs must have the same length as partitionInfos.")
-
-    import scala.collection.JavaConverters.asScalaBufferConverter
-
-    new UpdateTaskMultiplePartitionInfos(executionID,
-      resultIDs.asScala.zip(partitionInfos.asScala))
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/b9de4ed3/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/FlinkMiniCluster.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/FlinkMiniCluster.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/FlinkMiniCluster.scala
index c4c35f8..bbd011a 100644
--- a/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/FlinkMiniCluster.scala
+++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/FlinkMiniCluster.scala
@@ -19,6 +19,7 @@
 package org.apache.flink.runtime.minicluster
 
 import java.net.InetAddress
+import java.util.UUID
 
 import akka.pattern.Patterns.gracefulStop
 import akka.pattern.ask
@@ -26,22 +27,25 @@ import akka.actor.{ActorRef, ActorSystem}
 
 import com.typesafe.config.Config
 
-import org.apache.flink.api.common.{JobExecutionResult, JobSubmissionResult}
+import org.apache.flink.api.common.{JobID, JobExecutionResult, JobSubmissionResult}
 import org.apache.flink.configuration.{ConfigConstants, Configuration}
 import org.apache.flink.runtime.StreamingMode
 import org.apache.flink.runtime.akka.AkkaUtils
 import org.apache.flink.runtime.client.{JobExecutionException, JobClient}
 import org.apache.flink.runtime.instance.{AkkaActorGateway, ActorGateway}
 import org.apache.flink.runtime.jobgraph.JobGraph
-import org.apache.flink.runtime.jobmanager.JobManager
 import org.apache.flink.runtime.jobmanager.web.WebInfoServer
-import org.apache.flink.runtime.messages.TaskManagerMessages.NotifyWhenRegisteredAtJobManager
+import org.apache.flink.runtime.jobmanager.{JobManager, RecoveryMode}
+import org.apache.flink.runtime.leaderretrieval.{LeaderRetrievalService, LeaderRetrievalListener,
+StandaloneLeaderRetrievalService}
+import org.apache.flink.runtime.messages.TaskManagerMessages.NotifyWhenRegisteredAtAnyJobManager
+import org.apache.flink.runtime.util.{StandaloneUtils, ZooKeeperUtils}
 import org.apache.flink.runtime.webmonitor.WebMonitor
 
 import org.slf4j.LoggerFactory
 
 import scala.concurrent.duration.FiniteDuration
-import scala.concurrent.{ExecutionContext, Future, Await}
+import scala.concurrent._
 
 /**
  * Abstract base class for Flink's mini cluster. The mini cluster starts a
@@ -50,15 +54,16 @@ import scala.concurrent.{ExecutionContext, Future, Await}
  * actors can all be run in the same [[ActorSystem]] or each one in its own.
  *
  * @param userConfiguration Configuration object with the user provided configuration values
- * @param singleActorSystem true if all actors (JobManager and TaskManager) shall be run in the same
- *                          [[ActorSystem]], otherwise false
+ * @param useSingleActorSystem true if all actors (JobManager and TaskManager) shall be run in the
+ *                             same [[ActorSystem]], otherwise false
  * @param streamingMode True, if the system should be started in streaming mode, false if
  *                      in pure batch mode.
  */
 abstract class FlinkMiniCluster(
     val userConfiguration: Configuration,
-    val singleActorSystem: Boolean,
-    val streamingMode: StreamingMode) {
+    val useSingleActorSystem: Boolean,
+    val streamingMode: StreamingMode)
+  extends LeaderRetrievalListener {
 
   def this(userConfiguration: Configuration, singleActorSystem: Boolean) 
          = this(userConfiguration, singleActorSystem, StreamingMode.BATCH_ONLY)
@@ -73,55 +78,110 @@ abstract class FlinkMiniCluster(
   // not getLocalHost(), which may be 127.0.1.1
   val hostname = InetAddress.getByName("localhost").getHostAddress()
 
-  val timeout = AkkaUtils.getTimeout(userConfiguration)
-
   val configuration = generateConfiguration(userConfiguration)
 
-  var jobManagerActorSystem = startJobManagerActorSystem()
-  var (jobManagerActor, webMonitor) = startJobManager(jobManagerActorSystem)
+  /** Future to the [[ActorGateway]] of the current leader */
+  var leaderGateway: Promise[ActorGateway] = Promise()
 
-  val numTaskManagers = configuration.getInteger(
-     ConfigConstants.LOCAL_INSTANCE_MANAGER_NUMBER_TASK_MANAGER, 1)
+  /** Future to the index of the current leader */
+  var leaderIndex: Promise[Int] = Promise()
 
-  var (taskManagerActorSystems, taskManagerActors) =
-    (for(i <- 0 until numTaskManagers) yield {
-      val actorSystem = if(singleActorSystem) {
-        jobManagerActorSystem
-      } else {
-        startTaskManagerActorSystem(i)
-      }
+  /** Future lock */
+  val futureLock = new Object()
+
+  implicit val executionContext = ExecutionContext.global
 
-      (actorSystem, startTaskManager(i, actorSystem))
-    }).unzip
+  implicit val timeout = AkkaUtils.getTimeout(userConfiguration)
+
+  val recoveryMode = RecoveryMode.valueOf(configuration.getString(
+    ConfigConstants.RECOVERY_MODE,
+    ConfigConstants.DEFAULT_RECOVERY_MODE).toUpperCase)
+
+  val numJobManagers = getNumberOfJobManagers
+
+  val numTaskManagers = configuration.getInteger(
+    ConfigConstants.LOCAL_NUMBER_TASK_MANAGER,
+    ConfigConstants.DEFAULT_LOCAL_NUMBER_TASK_MANAGER)
 
-  waitForTaskManagersToBeRegistered()
+  var jobManagerActorSystems: Option[Seq[ActorSystem]] = None
+  var jobManagerActors: Option[Seq[ActorRef]] = None
+  var webMonitor: Option[WebMonitor] = None
+  var taskManagerActorSystems: Option[Seq[ActorSystem]] = None
+  var taskManagerActors: Option[Seq[ActorRef]] = None
 
+  protected var leaderRetrievalService: Option[LeaderRetrievalService] = None
 
   // --------------------------------------------------------------------------
-  //                           Construction
+  //                           Abstract Methods
   // --------------------------------------------------------------------------
 
   def generateConfiguration(userConfiguration: Configuration): Configuration
 
-  def startJobManager(system: ActorSystem): (ActorRef, Option[WebMonitor])
+  def startJobManager(index: Int, system: ActorSystem): ActorRef
 
   def startTaskManager(index: Int, system: ActorSystem): ActorRef
 
-  def getJobManagerAkkaConfig: Config = {
-    if (singleActorSystem) {
+  // --------------------------------------------------------------------------
+  //                           Getters/Setters
+  // --------------------------------------------------------------------------
+
+  def getNumberOfJobManagers: Int = {
+    if(recoveryMode == RecoveryMode.STANDALONE) {
+      1
+    } else {
+      configuration.getInteger(
+        ConfigConstants.LOCAL_NUMBER_JOB_MANAGER,
+        ConfigConstants.DEFAULT_LOCAL_NUMBER_JOB_MANAGER
+      )
+    }
+  }
+
+  def getJobManagersAsJava = {
+    import collection.JavaConverters._
+    jobManagerActors.getOrElse(Seq()).asJava
+  }
+
+  def getTaskManagers = {
+    taskManagerActors.getOrElse(Seq())
+  }
+
+  def getTaskManagersAsJava = {
+    import collection.JavaConverters._
+    taskManagerActors.getOrElse(Seq()).asJava
+  }
+
+  def getLeaderGatewayFuture: Future[ActorGateway] = {
+    leaderGateway.future
+  }
+
+  def getLeaderGateway(timeout: FiniteDuration): ActorGateway = {
+    val jmFuture = getLeaderGatewayFuture
+
+    Await.result(jmFuture, timeout)
+  }
+
+  def getLeaderIndexFuture: Future[Int] = {
+    leaderIndex.future
+  }
+
+  def getLeaderIndex(timeout: FiniteDuration): Int = {
+    val indexFuture = getLeaderIndexFuture
+
+    Await.result(indexFuture, timeout)
+  }
+
+  def getJobManagerAkkaConfig(index: Int): Config = {
+    if (useSingleActorSystem) {
       AkkaUtils.getAkkaConfig(configuration, None)
     }
     else {
       val port = configuration.getInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY,
         ConfigConstants.DEFAULT_JOB_MANAGER_IPC_PORT)
 
-      AkkaUtils.getAkkaConfig(configuration, Some((hostname, port)))
-    }
-  }
+      val resolvedPort = if(port != 0) port + index else port
 
-  def startJobManagerActorSystem(): ActorSystem = {
-    val config = getJobManagerAkkaConfig
-    AkkaUtils.createActorSystem(config)
+      AkkaUtils.getAkkaConfig(configuration, Some((hostname, resolvedPort)))
+    }
   }
 
   def getTaskManagerAkkaConfig(index: Int): Config = {
@@ -133,77 +193,95 @@ abstract class FlinkMiniCluster(
     AkkaUtils.getAkkaConfig(configuration, Some((hostname, resolvedPort)))
   }
 
+  // --------------------------------------------------------------------------
+  //                          Start/Stop Methods
+  // --------------------------------------------------------------------------
+
+  def startJobManagerActorSystem(index: Int): ActorSystem = {
+    val config = getJobManagerAkkaConfig(index)
+    AkkaUtils.createActorSystem(config)
+  }
+
   def startTaskManagerActorSystem(index: Int): ActorSystem = {
     val config = getTaskManagerAkkaConfig(index)
 
     AkkaUtils.createActorSystem(config)
   }
 
-  def getJobManagerGateway(): ActorGateway = {
-    // create ActorGateway from the JobManager's ActorRef
-    JobManager.getJobManagerGateway(jobManagerActor, timeout)
+  def startJobClientActorSystem(jobID: JobID): ActorSystem = {
+    if (useSingleActorSystem) {
+      jobManagerActorSystems match {
+        case Some(jmActorSystems) => jmActorSystems(0)
+        case None => throw new JobExecutionException(
+          jobID,
+          "The FlinkMiniCluster has not been started yet.")
+      }
+    } else {
+      JobClient.startJobClientActorSystem(configuration)
+    }
   }
 
-  def getTaskManagers = {
-    taskManagerActors
+  def start(): Unit = {
+    start(true)
   }
 
-  def getTaskManagersAsJava = {
-    import collection.JavaConverters._
-    taskManagerActors.asJava
-  }
+  def start(waitForTaskManagerRegistration: Boolean): Unit = {
+    LOG.info("Starting FlinkMiniCluster.")
 
-  def stop(): Unit = {
-    LOG.info("Stopping FlinkMiniCluster.")
-    shutdown()
-    awaitTermination()
-  }
+    lazy val singleActorSystem = startJobManagerActorSystem(0)
 
-  def shutdown(): Unit = {
-    webMonitor foreach {
-      _.stop()
-    }
+    val (jmActorSystems, jmActors) =
+      (for(i <- 0 until numJobManagers) yield {
+        val actorSystem = if(useSingleActorSystem) {
+          singleActorSystem
+        } else {
+          startJobManagerActorSystem(i)
+        }
+        (actorSystem, startJobManager(i, actorSystem))
+      }).unzip
 
-    val futures = taskManagerActors map {
-        gracefulStop(_, timeout)
-    }
+    jobManagerActorSystems = Some(jmActorSystems)
+    jobManagerActors = Some(jmActors)
 
-    val future = gracefulStop(jobManagerActor, timeout)
+    val lrs = createLeaderRetrievalService();
 
-    implicit val executionContext = ExecutionContext.global
+    leaderRetrievalService = Some(lrs)
+    lrs.start(this)
 
-    Await.ready(Future.sequence(future +: futures), timeout)
+    val (tmActorSystems, tmActors) =
+      (for(i <- 0 until numTaskManagers) yield {
+        val actorSystem = if(useSingleActorSystem) {
+          jmActorSystems(0)
+        } else {
+          startTaskManagerActorSystem(i)
+        }
 
-    if(!singleActorSystem){
-      taskManagerActorSystems foreach {
-        _.shutdown()
-      }
-    }
+        (actorSystem, startTaskManager(i, actorSystem))
+      }).unzip
 
-    jobManagerActorSystem.shutdown()
-  }
+    taskManagerActorSystems = Some(tmActorSystems)
+    taskManagerActors = Some(tmActors)
 
-  def awaitTermination(): Unit = {
-    jobManagerActorSystem.awaitTermination()
+    val jobManagerAkkaURL = AkkaUtils.getAkkaURL(jmActorSystems(0), jmActors(0))
 
-    taskManagerActorSystems foreach {
-      _.awaitTermination()
+    webMonitor = startWebServer(configuration, jmActorSystems(0), jobManagerAkkaURL)
+
+    if(waitForTaskManagerRegistration) {
+      waitForTaskManagersToBeRegistered()
     }
   }
 
   def startWebServer(
       config: Configuration,
-      jobManager: ActorRef,
-      archiver: ActorRef)
+      actorSystem: ActorSystem,
+      jobManagerAkkaURL: String)
     : Option[WebMonitor] = {
     if(
-      config.getBoolean(ConfigConstants.LOCAL_INSTANCE_MANAGER_START_WEBSERVER, false) &&
-      config.getInteger(ConfigConstants.JOB_MANAGER_WEB_PORT_KEY, 0) >= 0) {
-
-      val lookupTimeout = AkkaUtils.getLookupTimeout(config)
+      config.getBoolean(ConfigConstants.LOCAL_START_WEBSERVER, false) &&
+        config.getInteger(ConfigConstants.JOB_MANAGER_WEB_PORT_KEY, 0) >= 0) {
 
-      val jobManagerGateway = JobManager.getJobManagerGateway(jobManager, lookupTimeout)
-      val archiverGateway = new AkkaActorGateway(archiver, jobManagerGateway.leaderSessionID())
+      // TODO: Add support for HA: Make web server work independently from the JM
+      val leaderRetrievalService = new StandaloneLeaderRetrievalService(jobManagerAkkaURL)
 
       // start the job manager web frontend
       val webServer = if (
@@ -214,11 +292,10 @@ abstract class FlinkMiniCluster(
         LOG.info("Starting NEW JobManger web frontend")
         // start the new web frontend. we need to load this dynamically
         // because it is not in the same project/dependencies
-        JobManager.startWebRuntimeMonitor(config, jobManagerGateway, archiverGateway)
-      }
-      else {
+        JobManager.startWebRuntimeMonitor(config, leaderRetrievalService, actorSystem)
+      } else {
         LOG.info("Starting JobManger web frontend")
-        new WebInfoServer(config, jobManagerGateway, archiverGateway)
+        new WebInfoServer(config, leaderRetrievalService, actorSystem)
       }
 
       webServer.start()
@@ -229,18 +306,89 @@ abstract class FlinkMiniCluster(
     }
   }
 
-  def waitForTaskManagersToBeRegistered(): Unit = {
+  def stop(): Unit = {
+    LOG.info("Stopping FlinkMiniCluster.")
+    shutdown()
+    awaitTermination()
+
+    leaderRetrievalService.foreach(_.stop())
+  }
+
+  protected def shutdown(): Unit = {
+    webMonitor foreach {
+      _.stop()
+    }
+    
+    val tmFutures = taskManagerActors map {
+      _.map(gracefulStop(_, timeout))
+    } getOrElse(Seq())
+
+
+    val jmFutures = jobManagerActors map {
+      _.map(gracefulStop(_, timeout))
+    } getOrElse(Seq())
+
     implicit val executionContext = ExecutionContext.global
 
-    val futures = taskManagerActors map {
-      taskManager => (taskManager ? NotifyWhenRegisteredAtJobManager)(timeout)
+    Await.ready(Future.sequence(jmFutures ++ tmFutures), timeout)
+
+    if (!useSingleActorSystem) {
+      taskManagerActorSystems foreach {
+        _ foreach(_.shutdown())
+      }
     }
 
+    jobManagerActorSystems foreach {
+      _ foreach(_.shutdown())
+    }
+  }
+
+  def awaitTermination(): Unit = {
+    jobManagerActorSystems foreach {
+      _ foreach(_.awaitTermination())
+    }
+
+    taskManagerActorSystems foreach {
+      _ foreach(_.awaitTermination())
+    }
+  }
+
+  // --------------------------------------------------------------------------
+  //                          Utility Methods
+  // --------------------------------------------------------------------------
+
+  /** Waits with the default timeout until all task managers have registered at the job manager
+    *
+    * @throws java.util.concurrent.TimeoutException
+    * @throws java.lang.InterruptedException
+    */
+  @throws(classOf[TimeoutException])
+  @throws(classOf[InterruptedException])
+  def waitForTaskManagersToBeRegistered(): Unit = {
+    waitForTaskManagersToBeRegistered(timeout)
+  }
+
+  /** Waits until all task managers have registered at the job manager until the timeout is reached.
+    *
+    * @param timeout
+    * @throws java.util.concurrent.TimeoutException
+    * @throws java.lang.InterruptedException
+    */
+  @throws(classOf[TimeoutException])
+  @throws(classOf[InterruptedException])
+  def waitForTaskManagersToBeRegistered(timeout: FiniteDuration): Unit = {
+    val futures = taskManagerActors map {
+      _ map(taskManager => (taskManager ? NotifyWhenRegisteredAtAnyJobManager)(timeout))
+    } getOrElse(Seq())
+
     Await.ready(Future.sequence(futures), timeout)
   }
 
   @throws(classOf[JobExecutionException])
-  def submitJobAndWait(jobGraph: JobGraph, printUpdates: Boolean): JobExecutionResult = {
+  def submitJobAndWait(
+      jobGraph: JobGraph,
+      printUpdates: Boolean)
+    : JobExecutionResult = {
     submitJobAndWait(jobGraph, printUpdates, timeout)
   }
   
@@ -251,25 +399,126 @@ abstract class FlinkMiniCluster(
       timeout: FiniteDuration)
     : JobExecutionResult = {
 
-    val clientActorSystem = if (singleActorSystem) jobManagerActorSystem
-    else JobClient.startJobClientActorSystem(configuration)
-
-    JobClient.submitJobAndWait(
-      clientActorSystem,
-      getJobManagerGateway(),
-      jobGraph,
-      timeout,
+    val clientActorSystem = startJobClientActorSystem(jobGraph.getJobID)
+
+     try {
+       val jobManagerGateway = try {
+           getLeaderGateway(timeout)
+         } catch {
+           case e: Exception => throw new JobExecutionException(
+             jobGraph.getJobID,
+             "Could not retrieve leading job manager gateway.",
+             e)
+         }
+
+     JobClient.submitJobAndWait(
+       clientActorSystem,
+       jobManagerGateway,
+       jobGraph,
+       timeout,
       printUpdates,
       this.getClass().getClassLoader())
+    } finally {
+       if(!useSingleActorSystem) {
+         // we have to shutdown the just created actor system
+         shutdownJobClientActorSystem(clientActorSystem)
+       }
+     }
   }
 
   @throws(classOf[JobExecutionException])
   def submitJobDetached(jobGraph: JobGraph) : JobSubmissionResult = {
-    JobClient.submitJobDetached(
-      getJobManagerGateway(),
-      jobGraph,
-      timeout,
-      getClass().getClassLoader())
+
+    val jobManagerGateway = try {
+      getLeaderGateway(timeout)
+    } catch {
+      case t: Throwable =>
+        throw new JobExecutionException(
+          jobGraph.getJobID,
+          "Could not retrieve JobManager ActorRef.",
+          t
+        )
+    }
+
+    JobClient.submitJobDetached(jobManagerGateway, jobGraph, timeout, getClass().getClassLoader())
+
     new JobSubmissionResult(jobGraph.getJobID)
   }
+
+  def shutdownJobClientActorSystem(actorSystem: ActorSystem): Unit = {
+    if(!useSingleActorSystem) {
+      actorSystem.shutdown()
+    }
+  }
+
+  protected def createLeaderRetrievalService(): LeaderRetrievalService = {
+    (jobManagerActorSystems, jobManagerActors) match {
+      case (Some(jmActorSystems), Some(jmActors)) =>
+        if (recoveryMode == RecoveryMode.STANDALONE) {
+          new StandaloneLeaderRetrievalService(
+            AkkaUtils.getAkkaURL(jmActorSystems(0), jmActors(0)))
+        } else {
+          ZooKeeperUtils.createLeaderRetrievalService(configuration)
+        }
+
+      case _ => throw new Exception("The FlinkMiniCluster has not been started properly.")
+    }
+  }
+
+  protected def clearLeader(): Unit = {
+    futureLock.synchronized{
+      leaderGateway = Promise()
+      leaderIndex = Promise()
+    }
+  }
+
+  override def notifyLeaderAddress(address: String, leaderSessionID: UUID): Unit = {
+    if (address != null && !address.equals("")) {
+      // only accept leader addresses which are not null and non-empty
+
+      val selectedLeader = (jobManagerActorSystems, jobManagerActors) match {
+        case (Some(systems), Some(actors)) =>
+          val actorPaths = systems.zip(actors).zipWithIndex.map {
+            case ((system, actor), index) => (AkkaUtils.getAkkaURL(system, actor), actor, index)
+          }
+
+          actorPaths.find {
+            case (path, actor, index) => path.equals(address)
+          }.map(x => (x._2, x._3))
+        case _ => None
+      }
+
+      futureLock.synchronized {
+        if (leaderGateway.isCompleted) {
+          // assignments happen atomically and only here
+          leaderGateway = Promise()
+          leaderIndex = Promise()
+        }
+
+        selectedLeader match {
+          case Some((leader, index)) =>
+            leaderGateway.success(new AkkaActorGateway(leader, leaderSessionID))
+            leaderIndex.success(index)
+          case None =>
+            leaderGateway.failure(
+              new Exception(s"Could not find job manager with address ${address}."))
+            leaderIndex.failure(
+              new Exception(s"Could not find job manager index with address ${address}.")
+            )
+        }
+      }
+    }
+  }
+
+  override def handleError(exception: Exception): Unit = {
+    futureLock.synchronized {
+      if(leaderGateway.isCompleted) {
+        leaderGateway = Promise.failed(exception)
+        leaderIndex = Promise.failed(exception)
+      } else{
+        leaderGateway.failure(exception)
+        leaderIndex.failure(exception)
+      }
+    }
+  }
 }