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 2017/05/05 11:48:06 UTC

[01/16] flink git commit: [FLINK-6136] Separate EmbeddedHaServices and StandaloneHaServices

Repository: flink
Updated Branches:
  refs/heads/master 43fa50713 -> d7364fffb


http://git-wip-us.apache.org/repos/asf/flink/blob/a0bb99c7/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNHighAvailabilityITCase.java
----------------------------------------------------------------------
diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNHighAvailabilityITCase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNHighAvailabilityITCase.java
index 546e3d7..bc1af65 100644
--- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNHighAvailabilityITCase.java
+++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNHighAvailabilityITCase.java
@@ -138,7 +138,7 @@ public class YARNHighAvailabilityITCase extends YarnTestBase {
 						@Override
 						protected void run() {
 							try {
-								LeaderRetrievalService lrs = LeaderRetrievalUtils.createLeaderRetrievalService(config);
+								LeaderRetrievalService lrs = LeaderRetrievalUtils.createLeaderRetrievalService(config, false);
 								ActorGateway gateway = LeaderRetrievalUtils.retrieveLeaderGateway(lrs, actorSystem, timeout);
 								ActorGateway selfGateway = new AkkaActorGateway(getRef(), gateway.leaderSessionID());
 
@@ -158,7 +158,7 @@ public class YARNHighAvailabilityITCase extends YarnTestBase {
 					@Override
 					protected void run() {
 						try {
-							LeaderRetrievalService lrs = LeaderRetrievalUtils.createLeaderRetrievalService(config);
+							LeaderRetrievalService lrs = LeaderRetrievalUtils.createLeaderRetrievalService(config, false);
 							ActorGateway gateway2 = LeaderRetrievalUtils.retrieveLeaderGateway(lrs, actorSystem, timeout);
 							ActorGateway selfGateway = new AkkaActorGateway(getRef(), gateway2.leaderSessionID());
 							gateway2.tell(new TestingJobManagerMessages.NotifyWhenAtLeastNumTaskManagerAreRegistered(1), selfGateway);

http://git-wip-us.apache.org/repos/asf/flink/blob/a0bb99c7/flink-yarn/src/main/java/org/apache/flink/yarn/YarnApplicationMasterRunner.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnApplicationMasterRunner.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnApplicationMasterRunner.java
index 492fc0b..ee87cfe 100644
--- a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnApplicationMasterRunner.java
+++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnApplicationMasterRunner.java
@@ -32,6 +32,7 @@ import org.apache.flink.runtime.clusterframework.BootstrapTools;
 import org.apache.flink.runtime.clusterframework.ContaineredTaskManagerParameters;
 import org.apache.flink.runtime.jobmanager.JobManager;
 import org.apache.flink.runtime.jobmanager.MemoryArchivist;
+import org.apache.flink.runtime.jobmaster.JobMaster;
 import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
 import org.apache.flink.runtime.net.SSLUtils;
 import org.apache.flink.runtime.process.ProcessReaper;
@@ -341,7 +342,7 @@ public class YarnApplicationMasterRunner {
 				actorSystem,
 				futureExecutor,
 				ioExecutor,
-				new Some<>(JobManager.JOB_MANAGER_NAME()),
+				new Some<>(JobMaster.JOB_MANAGER_NAME),
 				Option.<String>empty(),
 				getJobManagerClass(),
 				getArchivistClass())._1();

http://git-wip-us.apache.org/repos/asf/flink/blob/a0bb99c7/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterClient.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterClient.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterClient.java
index cd447d7..f044cdd 100644
--- a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterClient.java
+++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterClient.java
@@ -564,7 +564,7 @@ public class YarnClusterClient extends ClusterClient {
 				/* The leader retrieval service for connecting to the cluster and finding the active leader. */
 				LeaderRetrievalService leaderRetrievalService;
 				try {
-					leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(flinkConfig);
+					leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(flinkConfig, true);
 				} catch (Exception e) {
 					throw new RuntimeException("Could not create the leader retrieval service.", e);
 				}

http://git-wip-us.apache.org/repos/asf/flink/blob/a0bb99c7/flink-yarn/src/main/java/org/apache/flink/yarn/YarnFlinkApplicationMasterRunner.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnFlinkApplicationMasterRunner.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnFlinkApplicationMasterRunner.java
index 4e178e6..05b2be8 100644
--- a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnFlinkApplicationMasterRunner.java
+++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnFlinkApplicationMasterRunner.java
@@ -136,12 +136,15 @@ public class YarnFlinkApplicationMasterRunner extends AbstractYarnFlinkApplicati
 
 			synchronized (lock) {
 				LOG.info("Starting High Availability Services");
-				haServices = HighAvailabilityServicesUtils.createAvailableOrEmbeddedServices(config);
+				commonRpcService = createRpcService(config, appMasterHostname, amPortRange);
+
+				haServices = HighAvailabilityServicesUtils.createAvailableOrEmbeddedServices(
+					config,
+					commonRpcService.getExecutor());
 
 				heartbeatServices = HeartbeatServices.fromConfiguration(config);
 				
 				metricRegistry = new MetricRegistry(MetricRegistryConfiguration.fromConfiguration(config));
-				commonRpcService = createRpcService(config, appMasterHostname, amPortRange);
 
 				// ---- (2) init resource manager -------
 				resourceManager = createResourceManager(config);

http://git-wip-us.apache.org/repos/asf/flink/blob/a0bb99c7/flink-yarn/src/main/java/org/apache/flink/yarn/YarnTaskExecutorRunner.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnTaskExecutorRunner.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnTaskExecutorRunner.java
index 2369765..1da52d4 100644
--- a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnTaskExecutorRunner.java
+++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnTaskExecutorRunner.java
@@ -208,13 +208,18 @@ public class YarnTaskExecutorRunner {
 			ResourceID resourceID = new ResourceID(containerId);
 			LOG.info("YARN assigned resource id {} for the task executor.", resourceID.toString());
 
-			haServices = HighAvailabilityServicesUtils.createAvailableOrEmbeddedServices(config);
+			taskExecutorRpcService = TaskManagerRunner.createRpcService(config, haServices);
+
+			haServices = HighAvailabilityServicesUtils.createHighAvailabilityServices(
+				config,
+				taskExecutorRpcService.getExecutor(),
+				HighAvailabilityServicesUtils.AddressResolution.TRY_ADDRESS_RESOLUTION);
+
 			HeartbeatServices heartbeatServices = HeartbeatServices.fromConfiguration(config);
 
 			metricRegistry = new MetricRegistry(MetricRegistryConfiguration.fromConfiguration(config));
 
 			// ---- (2) init task manager runner -------
-			taskExecutorRpcService = TaskManagerRunner.createRpcService(config, haServices);
 			taskManagerRunner = new TaskManagerRunner(
 				config,
 				resourceID,

http://git-wip-us.apache.org/repos/asf/flink/blob/a0bb99c7/flink-yarn/src/main/java/org/apache/flink/yarn/highavailability/YarnHighAvailabilityServices.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/highavailability/YarnHighAvailabilityServices.java b/flink-yarn/src/main/java/org/apache/flink/yarn/highavailability/YarnHighAvailabilityServices.java
index 4c78726..e9c3904 100644
--- a/flink-yarn/src/main/java/org/apache/flink/yarn/highavailability/YarnHighAvailabilityServices.java
+++ b/flink-yarn/src/main/java/org/apache/flink/yarn/highavailability/YarnHighAvailabilityServices.java
@@ -26,6 +26,7 @@ import org.apache.flink.runtime.blob.BlobStore;
 import org.apache.flink.runtime.blob.FileSystemBlobStore;
 import org.apache.flink.runtime.fs.hdfs.HadoopFileSystem;
 import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils;
 import org.apache.flink.runtime.jobmanager.HighAvailabilityMode;
 import org.apache.flink.util.ExceptionUtils;
 import org.apache.flink.util.InstantiationUtil;
@@ -331,7 +332,10 @@ public abstract class YarnHighAvailabilityServices implements HighAvailabilitySe
 		final HighAvailabilityMode mode = HighAvailabilityMode.fromConfig(flinkConfig);
 		switch (mode) {
 			case NONE:
-				return new YarnPreConfiguredMasterNonHaServices(flinkConfig, hadoopConfig);
+				return new YarnPreConfiguredMasterNonHaServices(
+					flinkConfig,
+					hadoopConfig,
+					HighAvailabilityServicesUtils.AddressResolution.TRY_ADDRESS_RESOLUTION);
 
 			case ZOOKEEPER:
 				throw  new UnsupportedOperationException("to be implemented");

http://git-wip-us.apache.org/repos/asf/flink/blob/a0bb99c7/flink-yarn/src/main/java/org/apache/flink/yarn/highavailability/YarnIntraNonHaMasterServices.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/highavailability/YarnIntraNonHaMasterServices.java b/flink-yarn/src/main/java/org/apache/flink/yarn/highavailability/YarnIntraNonHaMasterServices.java
index fd1a45e..abfdb5c 100644
--- a/flink-yarn/src/main/java/org/apache/flink/yarn/highavailability/YarnIntraNonHaMasterServices.java
+++ b/flink-yarn/src/main/java/org/apache/flink/yarn/highavailability/YarnIntraNonHaMasterServices.java
@@ -23,7 +23,7 @@ import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.IllegalConfigurationException;
 import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
 import org.apache.flink.runtime.highavailability.ServicesThreadFactory;
-import org.apache.flink.runtime.highavailability.leaderelection.SingleLeaderElectionService;
+import org.apache.flink.runtime.highavailability.nonha.leaderelection.SingleLeaderElectionService;
 import org.apache.flink.runtime.leaderelection.LeaderElectionService;
 import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/a0bb99c7/flink-yarn/src/main/java/org/apache/flink/yarn/highavailability/YarnPreConfiguredMasterNonHaServices.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/highavailability/YarnPreConfiguredMasterNonHaServices.java b/flink-yarn/src/main/java/org/apache/flink/yarn/highavailability/YarnPreConfiguredMasterNonHaServices.java
index 72d780f..9d05bbe 100644
--- a/flink-yarn/src/main/java/org/apache/flink/yarn/highavailability/YarnPreConfiguredMasterNonHaServices.java
+++ b/flink-yarn/src/main/java/org/apache/flink/yarn/highavailability/YarnPreConfiguredMasterNonHaServices.java
@@ -21,12 +21,13 @@ package org.apache.flink.yarn.highavailability;
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.IllegalConfigurationException;
-import org.apache.flink.runtime.clusterframework.FlinkResourceManager;
 import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils;
 import org.apache.flink.runtime.leaderelection.LeaderElectionService;
 import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
 import org.apache.flink.runtime.leaderretrieval.StandaloneLeaderRetrievalService;
-import org.apache.flink.runtime.rpc.RpcServiceUtils;
+import org.apache.flink.runtime.resourcemanager.ResourceManager;
+import org.apache.flink.runtime.rpc.akka.AkkaRpcServiceUtils;
 import org.apache.flink.yarn.configuration.YarnConfigOptions;
 
 import java.io.IOException;
@@ -82,7 +83,8 @@ public class YarnPreConfiguredMasterNonHaServices extends AbstractYarnNonHaServi
 	 */
 	public YarnPreConfiguredMasterNonHaServices(
 			Configuration config,
-			org.apache.hadoop.conf.Configuration hadoopConf) throws IOException {
+			org.apache.hadoop.conf.Configuration hadoopConf,
+			HighAvailabilityServicesUtils.AddressResolution addressResolution) throws IOException {
 
 		super(config, hadoopConf);
 
@@ -107,8 +109,12 @@ public class YarnPreConfiguredMasterNonHaServices extends AbstractYarnNonHaServi
 						YarnConfigOptions.APP_MASTER_RPC_PORT.key() + "' - port must be in [1, 65535]");
 			}
 
-			this.resourceManagerRpcUrl = RpcServiceUtils.getRpcUrl(
-					rmHost, rmPort, FlinkResourceManager.RESOURCE_MANAGER_NAME, config);
+			this.resourceManagerRpcUrl = AkkaRpcServiceUtils.getRpcUrl(
+				rmHost,
+				rmPort,
+				ResourceManager.RESOURCE_MANAGER_NAME,
+				addressResolution,
+				config);
 
 			// all well!
 			successful = true;

http://git-wip-us.apache.org/repos/asf/flink/blob/a0bb99c7/flink-yarn/src/test/java/org/apache/flink/yarn/highavailability/YarnPreConfiguredMasterHaServicesTest.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/test/java/org/apache/flink/yarn/highavailability/YarnPreConfiguredMasterHaServicesTest.java b/flink-yarn/src/test/java/org/apache/flink/yarn/highavailability/YarnPreConfiguredMasterHaServicesTest.java
index c730102..83ec37d 100644
--- a/flink-yarn/src/test/java/org/apache/flink/yarn/highavailability/YarnPreConfiguredMasterHaServicesTest.java
+++ b/flink-yarn/src/test/java/org/apache/flink/yarn/highavailability/YarnPreConfiguredMasterHaServicesTest.java
@@ -23,6 +23,7 @@ import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.IllegalConfigurationException;
 import org.apache.flink.core.fs.FileSystem;
 import org.apache.flink.core.fs.Path;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils;
 import org.apache.flink.util.TestLogger;
 import org.apache.flink.yarn.configuration.YarnConfigOptions;
 
@@ -93,7 +94,10 @@ public class YarnPreConfiguredMasterHaServicesTest extends TestLogger {
 
 		// missing resource manager address
 		try {
-			new YarnPreConfiguredMasterNonHaServices(flinkConfig, hadoopConfig);
+			new YarnPreConfiguredMasterNonHaServices(
+				flinkConfig,
+				hadoopConfig,
+				HighAvailabilityServicesUtils.AddressResolution.NO_ADDRESS_RESOLUTION);
 			fail();
 		} catch (IllegalConfigurationException e) {
 			// expected
@@ -103,7 +107,10 @@ public class YarnPreConfiguredMasterHaServicesTest extends TestLogger {
 
 		// missing resource manager port
 		try {
-			new YarnPreConfiguredMasterNonHaServices(flinkConfig, hadoopConfig);
+			new YarnPreConfiguredMasterNonHaServices(
+				flinkConfig,
+				hadoopConfig,
+				HighAvailabilityServicesUtils.AddressResolution.NO_ADDRESS_RESOLUTION);
 			fail();
 		} catch (IllegalConfigurationException e) {
 			// expected
@@ -112,7 +119,10 @@ public class YarnPreConfiguredMasterHaServicesTest extends TestLogger {
 		flinkConfig.setInteger(YarnConfigOptions.APP_MASTER_RPC_PORT, 1427);
 
 		// now everything is good ;-)
-		new YarnPreConfiguredMasterNonHaServices(flinkConfig, hadoopConfig).closeAndCleanupAllData();
+		new YarnPreConfiguredMasterNonHaServices(
+			flinkConfig,
+			hadoopConfig,
+			HighAvailabilityServicesUtils.AddressResolution.NO_ADDRESS_RESOLUTION).closeAndCleanupAllData();
 	}
 
 	@Test
@@ -122,7 +132,10 @@ public class YarnPreConfiguredMasterHaServicesTest extends TestLogger {
 		flinkConfig.setInteger(YarnConfigOptions.APP_MASTER_RPC_PORT, 1427);
 
 		// create the services
-		YarnHighAvailabilityServices services = new YarnPreConfiguredMasterNonHaServices(flinkConfig, hadoopConfig);
+		YarnHighAvailabilityServices services = new YarnPreConfiguredMasterNonHaServices(
+			flinkConfig,
+			hadoopConfig,
+			HighAvailabilityServicesUtils.AddressResolution.NO_ADDRESS_RESOLUTION);
 		services.closeAndCleanupAllData();
 
 		final FileSystem fileSystem = HDFS_ROOT_PATH.getFileSystem();
@@ -153,7 +166,10 @@ public class YarnPreConfiguredMasterHaServicesTest extends TestLogger {
 		flinkConfig.setString(YarnConfigOptions.APP_MASTER_RPC_ADDRESS, "localhost");
 		flinkConfig.setInteger(YarnConfigOptions.APP_MASTER_RPC_PORT, 1427);
 
-		YarnHighAvailabilityServices services = new YarnPreConfiguredMasterNonHaServices(flinkConfig, hadoopConfig);
+		YarnHighAvailabilityServices services = new YarnPreConfiguredMasterNonHaServices(
+			flinkConfig,
+			hadoopConfig,
+			HighAvailabilityServicesUtils.AddressResolution.NO_ADDRESS_RESOLUTION);
 
 		// this method is not supported
 		try {


[07/16] flink git commit: [FLINK-6078] Remove CuratorFramework#close calls from ZooKeeper based HA services

Posted by tr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/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 5f35229..92de31a 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
@@ -28,12 +28,14 @@ import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.akka.AkkaUtils;
 import org.apache.flink.runtime.clusterframework.FlinkResourceManager;
 import org.apache.flink.runtime.clusterframework.standalone.StandaloneResourceManager;
+import org.apache.flink.runtime.concurrent.Executors;
 import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
+import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices;
+import org.apache.flink.runtime.highavailability.nonha.embedded.EmbeddedHaServices;
 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.jobmanager.JobManager;
-import org.apache.flink.runtime.jobmanager.MemoryArchivist;
-import org.apache.flink.runtime.jobmaster.JobMaster;
+import org.apache.flink.runtime.leaderelection.TestingLeaderRetrievalService;
 import org.apache.flink.runtime.leaderretrieval.StandaloneLeaderRetrievalService;
 import org.apache.flink.runtime.messages.JobManagerMessages;
 import org.apache.flink.runtime.messages.JobManagerMessages.LeaderSessionMessage;
@@ -43,22 +45,29 @@ import org.apache.flink.runtime.messages.RegistrationMessages.RefuseRegistration
 import org.apache.flink.runtime.messages.TaskManagerMessages;
 import org.apache.flink.runtime.testingUtils.TestingUtils;
 import org.apache.flink.util.TestLogger;
+import org.junit.After;
 import org.junit.AfterClass;
+import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
+import org.mockito.Matchers;
 import scala.Option;
 import scala.concurrent.Await;
 import scala.concurrent.Future;
 import scala.concurrent.duration.Deadline;
 import scala.concurrent.duration.FiniteDuration;
 
+import java.util.Arrays;
 import java.util.UUID;
 import java.util.concurrent.TimeUnit;
 
 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.apache.flink.runtime.testingUtils.TestingUtils.stopActorGatewaysGracefully;
+import static org.apache.flink.runtime.testingUtils.TestingUtils.stopActorGracefully;
 import static org.junit.Assert.*;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
 
 /**
  * The tests in this class verify the behavior of the TaskManager
@@ -67,8 +76,6 @@ import static org.junit.Assert.*;
  */
 public class TaskManagerRegistrationTest extends TestLogger {
 
-	private static final Option<String> NONE_STRING = Option.empty();
-
 	// use one actor system throughout all tests
 	private static ActorSystem actorSystem;
 
@@ -76,6 +83,8 @@ public class TaskManagerRegistrationTest extends TestLogger {
 
 	private static FiniteDuration timeout = new FiniteDuration(20, TimeUnit.SECONDS);
 
+	private TestingHighAvailabilityServices highAvailabilityServices;
+
 	@BeforeClass
 	public static void startActorSystem() {
 		config = new Configuration();
@@ -94,38 +103,58 @@ public class TaskManagerRegistrationTest extends TestLogger {
 		}
 	}
 
+	@Before
+	public void setupTest() {
+		highAvailabilityServices = new TestingHighAvailabilityServices();
+	}
+
+	@After
+	public void tearDownTest() throws Exception {
+		highAvailabilityServices.closeAndCleanupAllData();
+		highAvailabilityServices = null;
+	}
+
 	/**
 	 * A test that verifies that two TaskManagers correctly register at the
 	 * JobManager.
 	 */
 	@Test
-	public void testSimpleRegistration() {
+	public void testSimpleRegistration() throws Exception {
 		new JavaTestKit(actorSystem) {{
 
 			ActorGateway jobManager = null;
 			ActorGateway taskManager1 = null;
 			ActorGateway taskManager2 = null;
+			ActorGateway resourceManager = null;
+
+			EmbeddedHaServices embeddedHaServices = null;
 
 			try {
+				embeddedHaServices = new EmbeddedHaServices(Executors.directExecutor());
+
 				// a simple JobManager
-				jobManager = createJobManager(
+				jobManager = TestingUtils.createJobManager(
 					actorSystem,
 					TestingUtils.defaultExecutor(),
 					TestingUtils.defaultExecutor(),
-					config);
-				startResourceManager(config, jobManager.actor());
+					config,
+					embeddedHaServices);
+
+				resourceManager = new AkkaActorGateway(
+					startResourceManager(config, embeddedHaServices),
+					jobManager.leaderSessionID());
 
 				// start two TaskManagers. it will automatically try to register
 				taskManager1 = createTaskManager(
 						actorSystem,
-						jobManager,
+						embeddedHaServices,
 						config,
 						true,
 						false);
 
 				taskManager2 = createTaskManager(
 						actorSystem,
-						jobManager,
+						embeddedHaServices,
 						config,
 						true,
 						false);
@@ -142,10 +171,8 @@ public class TaskManagerRegistrationTest extends TestLogger {
 				Object response1 = Await.result(responseFuture1, timeout);
 				Object response2 = Await.result(responseFuture2, timeout);
 
-				// this is a hack to work around the way Java can interact with scala case objects
-				Class<?> confirmClass = TaskManagerMessages.getRegisteredAtJobManagerMessage().getClass();
-				assertTrue(response1 != null && confirmClass.isAssignableFrom(response1.getClass()));
-				assertTrue(response2 != null && confirmClass.isAssignableFrom(response2.getClass()));
+				assertTrue(response1 instanceof TaskManagerMessages.RegisteredAtJobManager);
+				assertTrue(response2 instanceof TaskManagerMessages.RegisteredAtJobManager);
 
 				// check that the JobManager has 2 TaskManagers registered
 				Future<Object> numTaskManagersFuture = jobManager.ask(
@@ -159,9 +186,9 @@ public class TaskManagerRegistrationTest extends TestLogger {
 				e.printStackTrace();
 				fail(e.getMessage());
 			} finally {
-				stopActor(taskManager1);
-				stopActor(taskManager2);
-				stopActor(jobManager);
+				stopActorGatewaysGracefully(Arrays.asList(taskManager1, taskManager2, jobManager, resourceManager));
+
+				embeddedHaServices.closeAndCleanupAllData();
 			}
 		}};
 	}
@@ -171,36 +198,35 @@ public class TaskManagerRegistrationTest extends TestLogger {
 	 * JobManager.
 	 */
 	@Test
-	public void testDelayedRegistration() {
+	public void testDelayedRegistration() throws Exception {
 		new JavaTestKit(actorSystem) {{
 			ActorGateway jobManager = null;
 			ActorGateway taskManager = null;
 
-			FiniteDuration delayedTimeout = timeout.$times(3);
+			FiniteDuration delayedTimeout = timeout.$times(3L);
+
+			final EmbeddedHaServices embeddedHaServices = new EmbeddedHaServices(Executors.directExecutor());
 
 			try {
 				// start a TaskManager that tries to register at the JobManager before the JobManager is
 				// available. we give it the regular JobManager akka URL
 				taskManager = createTaskManager(
 						actorSystem,
-						AkkaUtils.getLocalAkkaURL(JobMaster.JOB_MANAGER_NAME),
+						embeddedHaServices,
 						new Configuration(),
 						true,
 						false);
 
 				// let it try for a bit
-				Thread.sleep(6000);
+				Thread.sleep(6000L);
 
 				// now start the JobManager, with the regular akka URL
-				jobManager = createJobManager(
+				jobManager = TestingUtils.createJobManager(
 					actorSystem,
 					TestingUtils.defaultExecutor(),
 					TestingUtils.defaultExecutor(),
-					new Configuration());
-
-				startResourceManager(config, jobManager.actor());
-
-				startResourceManager(config, jobManager.actor());
+					new Configuration(),
+					embeddedHaServices);
 
 				// check that the TaskManagers are registered
 				Future<Object> responseFuture = taskManager.ask(
@@ -209,17 +235,11 @@ public class TaskManagerRegistrationTest extends TestLogger {
 
 				Object response = Await.result(responseFuture, delayedTimeout);
 
-				// this is a hack to work around the way Java can interact with scala case objects
-				Class<?> confirmClass = TaskManagerMessages.getRegisteredAtJobManagerMessage().getClass();
-				assertTrue(response != null && confirmClass.isAssignableFrom(response.getClass()));
-
-			}
-			catch (Exception e) {
-				e.printStackTrace();
-				fail(e.getMessage());
+				assertTrue(response instanceof TaskManagerMessages.RegisteredAtJobManager);
 			} finally {
-				stopActor(taskManager);
-				stopActor(jobManager);
+				stopActorGatewaysGracefully(Arrays.asList(taskManager, jobManager));
+
+				embeddedHaServices.closeAndCleanupAllData();
 			}
 		}};
 	}
@@ -246,10 +266,17 @@ public class TaskManagerRegistrationTest extends TestLogger {
 				Configuration tmConfig = new Configuration();
 				tmConfig.setString(ConfigConstants.TASK_MANAGER_MAX_REGISTRATION_DURATION, "500 ms");
 
+				highAvailabilityServices.setJobMasterLeaderRetriever(
+					HighAvailabilityServices.DEFAULT_JOB_ID,
+					// Give a non-existent job manager address to the task manager
+					new TestingLeaderRetrievalService(
+						"foobar",
+						HighAvailabilityServices.DEFAULT_LEADER_ID));
+
 				// start the taskManager actor
 				taskManager = createTaskManager(
 						actorSystem,
-						AkkaUtils.getLocalAkkaURL(JobMaster.JOB_MANAGER_NAME),
+						highAvailabilityServices,
 						tmConfig,
 						true,
 						false);
@@ -271,7 +298,7 @@ public class TaskManagerRegistrationTest extends TestLogger {
 				e.printStackTrace();
 				fail(e.getMessage());
 			} finally {
-				stopActor(taskManager);
+				stopActorGracefully(taskManager);
 			}
 		}};
 	}
@@ -286,18 +313,28 @@ public class TaskManagerRegistrationTest extends TestLogger {
 			ActorGateway jm = null;
 			ActorGateway taskManager =null;
 			try {
-				jm = TestingUtils.createForwardingActor(actorSystem, getTestActor(), Option.<String>empty());
+				jm = TestingUtils.createForwardingActor(
+					actorSystem,
+					getTestActor(),
+					HighAvailabilityServices.DEFAULT_LEADER_ID,
+					Option.<String>empty());
 				final ActorGateway jmGateway = jm;
 
 				FiniteDuration refusedRegistrationPause = new FiniteDuration(500, TimeUnit.MILLISECONDS);
 				Configuration tmConfig = new Configuration(config);
 				tmConfig.setString(ConfigConstants.TASK_MANAGER_REFUSED_REGISTRATION_PAUSE, refusedRegistrationPause.toString());
 
+				highAvailabilityServices.setJobMasterLeaderRetriever(
+					HighAvailabilityServices.DEFAULT_JOB_ID,
+					new TestingLeaderRetrievalService(
+						jm.path(),
+						HighAvailabilityServices.DEFAULT_LEADER_ID));
+
 				// we make the test actor (the test kit) the JobManager to intercept
 				// the messages
 				taskManager = createTaskManager(
 					actorSystem,
-					jmGateway,
+					highAvailabilityServices,
 					tmConfig,
 					true,
 					false);
@@ -330,13 +367,8 @@ public class TaskManagerRegistrationTest extends TestLogger {
 						expectMsgClass(RegisterTaskManager.class);
 					}
 				};
-			}
-			catch (Throwable e) {
-				e.printStackTrace();
-				fail(e.getMessage());
 			} finally {
-				stopActor(taskManager);
-				stopActor(jm);
+				stopActorGatewaysGracefully(Arrays.asList(taskManager, jm));
 			}
 		}};
 	}
@@ -353,7 +385,18 @@ public class TaskManagerRegistrationTest extends TestLogger {
 			try {
 				FiniteDuration timeout = new FiniteDuration(5, TimeUnit.SECONDS);
 
-				jm = TestingUtils.createForwardingActor(actorSystem, getTestActor(), Option.<String>empty());
+				jm = TestingUtils.createForwardingActor(
+					actorSystem,
+					getTestActor(),
+					HighAvailabilityServices.DEFAULT_LEADER_ID,
+					Option.<String>empty());
+
+				highAvailabilityServices.setJobMasterLeaderRetriever(
+					HighAvailabilityServices.DEFAULT_JOB_ID,
+					new TestingLeaderRetrievalService(
+						jm.path(),
+						HighAvailabilityServices.DEFAULT_LEADER_ID));
+
 				final ActorGateway jmGateway = jm;
 
 				long refusedRegistrationPause = 500;
@@ -368,7 +411,7 @@ public class TaskManagerRegistrationTest extends TestLogger {
 				// the messages
 				taskManager = createTaskManager(
 					actorSystem,
-					jmGateway,
+					highAvailabilityServices,
 					tmConfig,
 					true,
 					false);
@@ -423,8 +466,7 @@ public class TaskManagerRegistrationTest extends TestLogger {
 					+ maxExpectedNumberOfRegisterTaskManagerMessages,
 					registerTaskManagerMessages.length <= maxExpectedNumberOfRegisterTaskManagerMessages);
 			} finally {
-				stopActor(taskManager);
-				stopActor(jm);
+				stopActorGatewaysGracefully(Arrays.asList(taskManager, jm));
 			}
 		}};
 	}
@@ -444,16 +486,25 @@ public class TaskManagerRegistrationTest extends TestLogger {
 
 			try {
 				fakeJobManager1Gateway = TestingUtils.createForwardingActor(
-						actorSystem,
-						getTestActor(),
-						Option.apply(JOB_MANAGER_NAME));
+					actorSystem,
+					getTestActor(),
+					HighAvailabilityServices.DEFAULT_LEADER_ID,
+					Option.apply(JOB_MANAGER_NAME));
 				final ActorGateway fakeJM1Gateway = fakeJobManager1Gateway;
 
+				TestingLeaderRetrievalService testingLeaderRetrievalService = new TestingLeaderRetrievalService(
+					fakeJM1Gateway.path(),
+					HighAvailabilityServices.DEFAULT_LEADER_ID);
+
+				highAvailabilityServices.setJobMasterLeaderRetriever(
+					HighAvailabilityServices.DEFAULT_JOB_ID,
+					testingLeaderRetrievalService);
+
 				// we make the test actor (the test kit) the JobManager to intercept
 				// the messages
 				taskManagerGateway = createTaskManager(
 						actorSystem,
-						fakeJobManager1Gateway,
+						highAvailabilityServices,
 						config,
 						true,
 						false);
@@ -512,9 +563,10 @@ public class TaskManagerRegistrationTest extends TestLogger {
 				do {
 					try {
 						fakeJobManager2Gateway = TestingUtils.createForwardingActor(
-								actorSystem,
-								getTestActor(),
-								Option.apply(JOB_MANAGER_NAME));
+							actorSystem,
+							getTestActor(),
+							HighAvailabilityServices.DEFAULT_LEADER_ID,
+							Option.apply(JOB_MANAGER_NAME));
 					} catch (InvalidActorNameException e) {
 						// wait and retry
 						Thread.sleep(100);
@@ -543,9 +595,7 @@ public class TaskManagerRegistrationTest extends TestLogger {
 				e.printStackTrace();
 				fail(e.getMessage());
 			} finally {
-				stopActor(taskManagerGateway);
-				stopActor(fakeJobManager1Gateway);
-				stopActor(fakeJobManager2Gateway);
+				stopActorGatewaysGracefully(Arrays.asList(taskManagerGateway, fakeJobManager2Gateway));
 			}
 		}};
 	}
@@ -556,21 +606,25 @@ public class TaskManagerRegistrationTest extends TestLogger {
 
 			ActorGateway taskManagerGateway = null;
 
+			final UUID falseLeaderSessionID = UUID.randomUUID();
+			final UUID trueLeaderSessionID = UUID.randomUUID();
+
+			HighAvailabilityServices mockedHighAvailabilityServices = mock(HighAvailabilityServices.class);
+			when(mockedHighAvailabilityServices.getJobManagerLeaderRetriever(Matchers.eq(HighAvailabilityServices.DEFAULT_JOB_ID)))
+				.thenReturn(new StandaloneLeaderRetrievalService(getTestActor().path().toString(), trueLeaderSessionID));
+
 			try {
 				// we make the test actor (the test kit) the JobManager to intercept
 				// the messages
 				taskManagerGateway = createTaskManager(
 						actorSystem,
-						getTestActor(),
+						mockedHighAvailabilityServices,
 						config,
 						true,
 						false);
 
 				final ActorRef taskManager = taskManagerGateway.actor();
 
-				final UUID falseLeaderSessionID = UUID.randomUUID();
-				final UUID trueLeaderSessionID = HighAvailabilityServices.DEFAULT_LEADER_ID;
-
 				new Within(timeout) {
 
 					@Override
@@ -581,7 +635,7 @@ public class TaskManagerRegistrationTest extends TestLogger {
 
 						LeaderSessionMessage lsm = expectMsgClass(LeaderSessionMessage.class);
 
-						assertTrue(lsm.leaderSessionID() == trueLeaderSessionID);
+						assertTrue(lsm.leaderSessionID().equals(trueLeaderSessionID));
 						assertTrue(lsm.message() instanceof RegisterTaskManager);
 
 						final ActorRef tm = getLastSender();
@@ -606,9 +660,8 @@ public class TaskManagerRegistrationTest extends TestLogger {
 								getTestActor());
 
 						Object message = null;
-						Object confirmMessageClass = TaskManagerMessages.getRegisteredAtJobManagerMessage().getClass();
 
-						while(message == null || !(message.getClass().equals(confirmMessageClass))) {
+						while(!(message instanceof TaskManagerMessages.RegisteredAtJobManager)) {
 							message = receiveOne(TestingUtils.TESTING_DURATION());
 						}
 
@@ -617,12 +670,8 @@ public class TaskManagerRegistrationTest extends TestLogger {
 						expectMsgEquals(new JobManagerMessages.ResponseLeaderSessionID(trueLeaderSessionID));
 					}
 				};
-			}
-			catch (Throwable e) {
-				e.printStackTrace();
-				fail(e.getMessage());
 			} finally {
-				stopActor(taskManagerGateway);
+				stopActorGracefully(taskManagerGateway);
 			}
 		}};
 	}
@@ -631,25 +680,11 @@ public class TaskManagerRegistrationTest extends TestLogger {
 	//  Utility Functions
 	// --------------------------------------------------------------------------------------------
 
-	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(
-			configuration,
-			actorSystem,
-			TestingUtils.defaultExecutor(),
-			TestingUtils.defaultExecutor(),
-			NONE_STRING,
-			NONE_STRING,
-			JobManager.class,
-			MemoryArchivist.class)._1();
-	}
-
-	private static ActorRef startResourceManager(Configuration config, ActorRef jobManager) {
+	private static ActorRef startResourceManager(Configuration config, HighAvailabilityServices highAvailabilityServices) {
 		return FlinkResourceManager.startResourceManagerActors(
 			config,
 			actorSystem,
-			new StandaloneLeaderRetrievalService(jobManager.path().toString()),
+			highAvailabilityServices.getJobManagerLeaderRetriever(HighAvailabilityServices.DEFAULT_JOB_ID),
 			StandaloneResourceManager.class);
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/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
index 4df8db3..0e77ddd 100644
--- 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
@@ -26,9 +26,13 @@ import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.IllegalConfigurationException;
 import org.apache.flink.configuration.TaskManagerOptions;
 import org.apache.flink.runtime.clusterframework.types.ResourceID;
-import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
+import org.apache.flink.runtime.highavailability.nonha.embedded.EmbeddedHaServices;
+import org.apache.flink.runtime.testingUtils.TestingUtils;
 import org.apache.flink.runtime.util.StartupUtils;
 import org.apache.flink.util.NetUtils;
+import org.junit.After;
+import org.junit.Before;
 import org.junit.Test;
 import scala.Option;
 
@@ -46,6 +50,21 @@ import java.util.UUID;
  * problems.
  */
 public class TaskManagerStartupTest {
+
+	private HighAvailabilityServices highAvailabilityServices;
+
+	@Before
+	public void setupTest() {
+		highAvailabilityServices = new EmbeddedHaServices(TestingUtils.defaultExecutor());
+	}
+
+	@After
+	public void tearDownTest() throws Exception {
+		if (highAvailabilityServices != null) {
+			highAvailabilityServices.closeAndCleanupAllData();
+			highAvailabilityServices = null;
+		}
+	}
 	
 
 	/**
@@ -55,8 +74,9 @@ public class TaskManagerStartupTest {
 	 * @throws Throwable
 	 */
 	@Test(expected = BindException.class)
-	public void testStartupWhenTaskmanagerActorPortIsUsed() throws BindException {
+	public void testStartupWhenTaskmanagerActorPortIsUsed() throws Exception {
 		ServerSocket blocker = null;
+
 		try {
 			final String localHostName = "localhost";
 			final InetAddress localBindAddress = InetAddress.getByName(NetUtils.getWildcardIPAddress());
@@ -65,8 +85,13 @@ public class TaskManagerStartupTest {
 			blocker = new ServerSocket(0, 50, localBindAddress);
 			final int port = blocker.getLocalPort();
 
-			TaskManager.runTaskManager(localHostName, ResourceID.generate(), port, new Configuration(),
-					TaskManager.class);
+			TaskManager.runTaskManager(
+				localHostName,
+				ResourceID.generate(),
+				port,
+				new Configuration(),
+				highAvailabilityServices,
+				TaskManager.class);
 			fail("This should fail with an IOException");
 
 		}
@@ -92,6 +117,8 @@ public class TaskManagerStartupTest {
 					// no need to log here
 				}
 			}
+
+			highAvailabilityServices.closeAndCleanupAllData();
 		}
 	}
 
@@ -102,7 +129,7 @@ public class TaskManagerStartupTest {
 	 * directories are not writable.
 	 */
 	@Test
-	public void testIODirectoryNotWritable() {
+	public void testIODirectoryNotWritable() throws Exception {
 		File tempDir = new File(ConfigConstants.DEFAULT_TASK_MANAGER_TMP_PATH);
 		File nonWritable = new File(tempDir, UUID.randomUUID().toString());
 
@@ -119,7 +146,12 @@ public class TaskManagerStartupTest {
 			cfg.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, 21656);
 
 			try {
-				TaskManager.runTaskManager("localhost", ResourceID.generate(), 0, cfg);
+				TaskManager.runTaskManager(
+					"localhost",
+					ResourceID.generate(),
+					0,
+					cfg,
+					highAvailabilityServices);
 				fail("Should fail synchronously with an exception");
 			}
 			catch (IOException e) {
@@ -139,6 +171,8 @@ public class TaskManagerStartupTest {
 			catch (IOException e) {
 				// best effort
 			}
+
+			highAvailabilityServices.closeAndCleanupAllData();
 		}
 	}
 
@@ -157,7 +191,12 @@ public class TaskManagerStartupTest {
 			// something invalid
 			cfg.setLong(TaskManagerOptions.MANAGED_MEMORY_SIZE, -42L);
 			try {
-				TaskManager.runTaskManager("localhost", ResourceID.generate(), 0, cfg);
+				TaskManager.runTaskManager(
+					"localhost",
+					ResourceID.generate(),
+					0,
+					cfg,
+					highAvailabilityServices);
 				fail("Should fail synchronously with an exception");
 			}
 			catch (IllegalConfigurationException e) {
@@ -169,7 +208,12 @@ public class TaskManagerStartupTest {
 					TaskManagerOptions.MEMORY_SEGMENT_SIZE.defaultValue()) >> 20;
 			cfg.setLong(TaskManagerOptions.MANAGED_MEMORY_SIZE, memSize);
 			try {
-				TaskManager.runTaskManager("localhost", ResourceID.generate(), 0, cfg);
+				TaskManager.runTaskManager(
+					"localhost",
+					ResourceID.generate(),
+					0,
+					cfg,
+					highAvailabilityServices);
 				fail("Should fail synchronously with an exception");
 			}
 			catch (Exception e) {
@@ -204,9 +248,9 @@ public class TaskManagerStartupTest {
 				cfg,
 				ResourceID.generate(),
 				null,
+				highAvailabilityServices,
 				"localhost",
 				Option.<String>empty(),
-				Option.<LeaderRetrievalService>empty(),
 				false,
 				TaskManager.class);
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/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 4530ade..0f5afc0 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
@@ -47,6 +47,7 @@ import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
 import org.apache.flink.runtime.executiongraph.JobInformation;
 import org.apache.flink.runtime.executiongraph.TaskInformation;
 import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
+import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices;
 import org.apache.flink.runtime.instance.ActorGateway;
 import org.apache.flink.runtime.instance.AkkaActorGateway;
 import org.apache.flink.runtime.instance.InstanceID;
@@ -60,6 +61,8 @@ import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
 import org.apache.flink.runtime.jobmanager.Tasks;
+import org.apache.flink.runtime.leaderelection.TestingLeaderRetrievalService;
+import org.apache.flink.runtime.leaderretrieval.StandaloneLeaderRetrievalService;
 import org.apache.flink.runtime.messages.Acknowledge;
 import org.apache.flink.runtime.messages.RegistrationMessages;
 import org.apache.flink.runtime.messages.StackTraceSampleMessages.TriggerStackTraceSample;
@@ -80,8 +83,10 @@ import org.apache.flink.types.IntValue;
 import org.apache.flink.util.NetUtils;
 import org.apache.flink.util.SerializedValue;
 import org.apache.flink.util.TestLogger;
+import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Assert;
+import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.slf4j.Logger;
@@ -126,7 +131,9 @@ public class TaskManagerTest extends TestLogger {
 
 	private static ActorSystem system;
 
-	final static UUID leaderSessionID = HighAvailabilityServices.DEFAULT_LEADER_ID;
+	final static UUID leaderSessionID = UUID.randomUUID();
+
+	private TestingHighAvailabilityServices highAvailabilityServices;
 
 	@BeforeClass
 	public static void setup() {
@@ -138,20 +145,39 @@ public class TaskManagerTest extends TestLogger {
 		JavaTestKit.shutdownActorSystem(system);
 	}
 
+	@Before
+	public void setupTest() {
+		highAvailabilityServices = new TestingHighAvailabilityServices();
+	}
+
+	@After
+	public void tearDownTest() throws Exception {
+		if (highAvailabilityServices != null) {
+			highAvailabilityServices.closeAndCleanupAllData();
+
+			highAvailabilityServices = null;
+		}
+	}
+
 	@Test
 	public void testSubmitAndExecuteTask() throws IOException {
 		new JavaTestKit(system){{
 
 			ActorGateway taskManager = null;
 			final ActorGateway jobManager = TestingUtils.createForwardingActor(
-					system,
-					getTestActor(),
-					Option.<String>empty());
+				system,
+				getTestActor(),
+				HighAvailabilityServices.DEFAULT_LEADER_ID,
+				Option.<String>empty());
+
+			highAvailabilityServices.setJobMasterLeaderRetriever(
+				HighAvailabilityServices.DEFAULT_JOB_ID,
+				new StandaloneLeaderRetrievalService(jobManager.path(), jobManager.leaderSessionID()));
 
 			try {
 				taskManager = TestingUtils.createTaskManager(
 						system,
-						jobManager,
+						highAvailabilityServices,
 						new Configuration(),
 						true,
 						false);
@@ -261,9 +287,13 @@ public class TaskManagerTest extends TestLogger {
 				ActorRef jm = system.actorOf(Props.create(SimpleJobManager.class, leaderSessionID));
 				jobManager = new AkkaActorGateway(jm, leaderSessionID);
 
+				highAvailabilityServices.setJobMasterLeaderRetriever(
+					HighAvailabilityServices.DEFAULT_JOB_ID,
+					new StandaloneLeaderRetrievalService(jobManager.path(), jobManager.leaderSessionID()));
+
 				taskManager = TestingUtils.createTaskManager(
 						system,
-						jobManager,
+						highAvailabilityServices,
 						new Configuration(),
 						true,
 						true);
@@ -398,9 +428,13 @@ public class TaskManagerTest extends TestLogger {
 				ActorRef jm = system.actorOf(Props.create(SimpleJobManager.class, leaderSessionID));
 				jobManager = new AkkaActorGateway(jm, leaderSessionID);
 
+				highAvailabilityServices.setJobMasterLeaderRetriever(
+					HighAvailabilityServices.DEFAULT_JOB_ID,
+					new StandaloneLeaderRetrievalService(jobManager.path(), jobManager.leaderSessionID()));
+
 				taskManager = TestingUtils.createTaskManager(
 						system,
-						jobManager,
+						highAvailabilityServices,
 						new Configuration(),
 						true,
 						true);
@@ -524,9 +558,13 @@ public class TaskManagerTest extends TestLogger {
 				ActorRef jm = system.actorOf(Props.create(SimpleJobManager.class, leaderSessionID));
 				jobManager = new AkkaActorGateway(jm, leaderSessionID);
 
+				highAvailabilityServices.setJobMasterLeaderRetriever(
+					HighAvailabilityServices.DEFAULT_JOB_ID,
+					new StandaloneLeaderRetrievalService(jobManager.path(), jobManager.leaderSessionID()));
+
 				taskManager = TestingUtils.createTaskManager(
 						system,
-						jobManager,
+						highAvailabilityServices,
 						new Configuration(),
 						true,
 						true);
@@ -624,9 +662,13 @@ public class TaskManagerTest extends TestLogger {
 				ActorRef jm = system.actorOf(Props.create(new SimpleLookupJobManagerCreator(leaderSessionID)));
 				jobManager = new AkkaActorGateway(jm, leaderSessionID);
 
+				highAvailabilityServices.setJobMasterLeaderRetriever(
+					HighAvailabilityServices.DEFAULT_JOB_ID,
+					new StandaloneLeaderRetrievalService(jobManager.path(), jobManager.leaderSessionID()));
+
 				taskManager = TestingUtils.createTaskManager(
 						system,
-						jobManager,
+						highAvailabilityServices,
 						new Configuration(),
 						true,
 						true);
@@ -769,9 +811,13 @@ public class TaskManagerTest extends TestLogger {
 
 				jobManager = new AkkaActorGateway(jm, leaderSessionID);
 
+				highAvailabilityServices.setJobMasterLeaderRetriever(
+					HighAvailabilityServices.DEFAULT_JOB_ID,
+					new StandaloneLeaderRetrievalService(jobManager.path(), jobManager.leaderSessionID()));
+
 				taskManager = TestingUtils.createTaskManager(
 						system,
-						jobManager,
+						highAvailabilityServices,
 						new Configuration(),
 						true,
 						true);
@@ -906,6 +952,10 @@ public class TaskManagerTest extends TestLogger {
 
 				jobManager = new AkkaActorGateway(jm, leaderSessionID);
 
+				highAvailabilityServices.setJobMasterLeaderRetriever(
+					HighAvailabilityServices.DEFAULT_JOB_ID,
+					new StandaloneLeaderRetrievalService(jobManager.path(), jobManager.leaderSessionID()));
+
 				final int dataPort = NetUtils.getAvailablePort();
 				Configuration config = new Configuration();
 				config.setInteger(ConfigConstants.TASK_MANAGER_DATA_PORT_KEY, dataPort);
@@ -914,7 +964,7 @@ public class TaskManagerTest extends TestLogger {
 
 				taskManager = TestingUtils.createTaskManager(
 						system,
-						jobManager,
+						highAvailabilityServices,
 						config,
 						false,
 						true);
@@ -1024,13 +1074,17 @@ public class TaskManagerTest extends TestLogger {
 
 				jobManager = new AkkaActorGateway(jm, leaderSessionID);
 
+				highAvailabilityServices.setJobMasterLeaderRetriever(
+					HighAvailabilityServices.DEFAULT_JOB_ID,
+					new StandaloneLeaderRetrievalService(jobManager.path(), jobManager.leaderSessionID()));
+
 				final Configuration config = new Configuration();
 				config.setInteger(TaskManagerOptions.NETWORK_REQUEST_BACKOFF_INITIAL, 100);
 				config.setInteger(TaskManagerOptions.NETWORK_REQUEST_BACKOFF_MAX, 200);
 
 				taskManager = TestingUtils.createTaskManager(
 						system,
-						jobManager,
+						highAvailabilityServices,
 						config,
 						true,
 						true);
@@ -1122,9 +1176,13 @@ public class TaskManagerTest extends TestLogger {
 				config.setInteger(TaskManagerOptions.NETWORK_REQUEST_BACKOFF_MAX, 200);
 				config.setString(ConfigConstants.TASK_MANAGER_LOG_PATH_KEY, "/i/dont/exist");
 
+				highAvailabilityServices.setJobMasterLeaderRetriever(
+					HighAvailabilityServices.DEFAULT_JOB_ID,
+					new StandaloneLeaderRetrievalService(jobManager.path(), jobManager.leaderSessionID()));
+
 				taskManager = TestingUtils.createTaskManager(
 					system,
-					jobManager,
+					highAvailabilityServices,
 					config,
 					false,
 					true);
@@ -1166,18 +1224,26 @@ public class TaskManagerTest extends TestLogger {
 
 			// We need this to be a JM that answers to update messages for
 			// robustness on Travis (if jobs need to be resubmitted in (4)).
-			ActorRef jm = system.actorOf(Props.create(new SimpleLookupJobManagerCreator(leaderSessionID)));
-			ActorGateway jobManagerActorGateway = new AkkaActorGateway(jm, leaderSessionID);
+			ActorRef jm = system.actorOf(Props.create(new SimpleLookupJobManagerCreator(
+				HighAvailabilityServices.DEFAULT_LEADER_ID)));
+			ActorGateway jobManagerActorGateway = new AkkaActorGateway(
+				jm,
+				HighAvailabilityServices.DEFAULT_LEADER_ID);
 
 			final ActorGateway testActorGateway = new AkkaActorGateway(
 					getTestActor(),
-					leaderSessionID);
+					HighAvailabilityServices.DEFAULT_LEADER_ID);
 
 			try {
 				final ActorGateway jobManager = jobManagerActorGateway;
+
+				highAvailabilityServices.setJobMasterLeaderRetriever(
+					HighAvailabilityServices.DEFAULT_JOB_ID,
+					new StandaloneLeaderRetrievalService(jobManager.path(), jobManager.leaderSessionID()));
+
 				final ActorGateway taskManager = TestingUtils.createTaskManager(
 						system,
-						jobManager,
+						highAvailabilityServices,
 						new Configuration(),
 						true,
 						false);
@@ -1459,11 +1525,15 @@ public class TaskManagerTest extends TestLogger {
 
 	@Test
 	public void testTerminationOnFatalError() {
+		highAvailabilityServices.setJobMasterLeaderRetriever(
+			HighAvailabilityServices.DEFAULT_JOB_ID,
+			new TestingLeaderRetrievalService());
+
 		new JavaTestKit(system){{
 
 			final ActorGateway taskManager = TestingUtils.createTaskManager(
 					system,
-					system.deadLetters(), // no jobmanager
+					highAvailabilityServices, // no jobmanager
 					new Configuration(),
 					true,
 					false);
@@ -1524,9 +1594,13 @@ public class TaskManagerTest extends TestLogger {
 			ActorRef jmActorRef = system.actorOf(Props.create(FailingScheduleOrUpdateConsumersJobManager.class, leaderSessionID), "jobmanager");
 			ActorGateway jobManager = new AkkaActorGateway(jmActorRef, leaderSessionID);
 
+			highAvailabilityServices.setJobMasterLeaderRetriever(
+				HighAvailabilityServices.DEFAULT_JOB_ID,
+				new StandaloneLeaderRetrievalService(jobManager.path(), jobManager.leaderSessionID()));
+
 			final ActorGateway taskManager = TestingUtils.createTaskManager(
 				system,
-				jobManager,
+				highAvailabilityServices,
 				configuration,
 				true,
 				true);
@@ -1562,9 +1636,13 @@ public class TaskManagerTest extends TestLogger {
 			ActorRef jm = system.actorOf(Props.create(SimpleJobManager.class, leaderSessionID));
 			jobManager = new AkkaActorGateway(jm, leaderSessionID);
 
+			highAvailabilityServices.setJobMasterLeaderRetriever(
+				HighAvailabilityServices.DEFAULT_JOB_ID,
+				new StandaloneLeaderRetrievalService(jobManager.path(), jobManager.leaderSessionID()));
+
 			taskManager = TestingUtils.createTaskManager(
 				system,
-				jobManager,
+				highAvailabilityServices,
 				new Configuration(),
 				true,
 				true);
@@ -1619,9 +1697,13 @@ public class TaskManagerTest extends TestLogger {
 			ActorRef jm = system.actorOf(Props.create(SimpleJobManager.class, leaderSessionID));
 			jobManager = new AkkaActorGateway(jm, leaderSessionID);
 
+			highAvailabilityServices.setJobMasterLeaderRetriever(
+				HighAvailabilityServices.DEFAULT_JOB_ID,
+				new StandaloneLeaderRetrievalService(jobManager.path(), jobManager.leaderSessionID()));
+
 			taskManager = TestingUtils.createTaskManager(
 				system,
-				jobManager,
+				highAvailabilityServices,
 				new Configuration(),
 				true,
 				true);
@@ -1679,9 +1761,13 @@ public class TaskManagerTest extends TestLogger {
 			ActorRef jm = system.actorOf(Props.create(SimpleJobManager.class, leaderSessionID));
 			jobManager = new AkkaActorGateway(jm, leaderSessionID);
 
+			highAvailabilityServices.setJobMasterLeaderRetriever(
+				HighAvailabilityServices.DEFAULT_JOB_ID,
+				new StandaloneLeaderRetrievalService(jobManager.path(), jobManager.leaderSessionID()));
+
 			taskManager = TestingUtils.createTaskManager(
 				system,
-				jobManager,
+				highAvailabilityServices,
 				new Configuration(),
 				true,
 				true);
@@ -1724,9 +1810,13 @@ public class TaskManagerTest extends TestLogger {
 			ActorRef jm = system.actorOf(Props.create(SimpleJobManager.class, leaderSessionID));
 			jobManager = new AkkaActorGateway(jm, leaderSessionID);
 
+			highAvailabilityServices.setJobMasterLeaderRetriever(
+				HighAvailabilityServices.DEFAULT_JOB_ID,
+				new StandaloneLeaderRetrievalService(jobManager.path(), jobManager.leaderSessionID()));
+
 			taskManager = TestingUtils.createTaskManager(
 				system,
-				jobManager,
+				highAvailabilityServices,
 				new Configuration(),
 				true,
 				true);

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/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 b2e8005..4be3299 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
@@ -19,7 +19,7 @@
 package org.apache.flink.runtime.jobmanager
 
 import java.net.InetAddress
-import java.util.concurrent.{Executors, ScheduledExecutorService}
+import java.util.concurrent.{Executors, ScheduledExecutorService, TimeUnit}
 
 import akka.actor._
 import akka.testkit.{ImplicitSender, TestKit, TestProbe}
@@ -28,6 +28,7 @@ import org.apache.flink.runtime.akka.AkkaUtils
 import org.apache.flink.runtime.clusterframework.FlinkResourceManager
 import org.apache.flink.runtime.clusterframework.types.ResourceID
 import org.apache.flink.runtime.highavailability.HighAvailabilityServices
+import org.apache.flink.runtime.highavailability.nonha.embedded.EmbeddedHaServices
 import org.apache.flink.runtime.instance._
 import org.apache.flink.runtime.jobmanager.JobManagerRegistrationTest.PlainForwardingActor
 import org.apache.flink.runtime.messages.JobManagerMessages.LeaderSessionMessage
@@ -40,7 +41,7 @@ import org.apache.flink.runtime.util.LeaderRetrievalUtils
 import org.junit.Assert.{assertNotEquals, assertNotNull}
 import org.junit.runner.RunWith
 import org.scalatest.junit.JUnitRunner
-import org.scalatest.{BeforeAndAfterAll, Matchers, WordSpecLike}
+import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, Matchers, WordSpecLike}
 
 import scala.concurrent.Await
 import scala.concurrent.duration._
@@ -51,31 +52,45 @@ import scala.language.postfixOps
  */
 @RunWith(classOf[JUnitRunner])
 class JobManagerRegistrationTest(_system: ActorSystem) extends TestKit(_system) with
-ImplicitSender with WordSpecLike with Matchers with BeforeAndAfterAll {
+ImplicitSender with WordSpecLike with Matchers with BeforeAndAfterAll with BeforeAndAfterEach {
 
   def this() = this(AkkaUtils.createLocalActorSystem(new Configuration()))
 
   val executor: ScheduledExecutorService = Executors.newScheduledThreadPool(2)
+
+  var highAvailabilityServices: HighAvailabilityServices = _
+
+  val timeout = FiniteDuration(30, TimeUnit.SECONDS)
   
   override def afterAll(): Unit = {
     executor.shutdownNow()
     TestKit.shutdownActorSystem(system)
   }
 
+  override def beforeEach(): Unit = {
+    highAvailabilityServices = new EmbeddedHaServices(executor)
+  }
+
+  override def afterEach(): Unit = {
+    if (highAvailabilityServices != null) {
+      highAvailabilityServices.closeAndCleanupAllData()
+    }
+  }
+
   "The JobManager" should {
 
     "assign a TaskManager a unique instance ID" in {
 
       var jmOption: Option[ActorGateway] = None
-      var rmOption: Option[ActorGateway] = None
+      var rmOption: Option[ActorRef] = None
       var tm1Option: Option[ActorRef] = None
       var tm2Option: Option[ActorRef] = None
 
       try {
-        val jm = startTestingJobManager(_system)
+        val jm = startTestingJobManager(_system, highAvailabilityServices)
         jmOption = Some(jm)
 
-        val rm = startTestingResourceManager(_system, jm.actor())
+        val rm = startTestingResourceManager(_system, highAvailabilityServices)
         rmOption = Some(rm)
 
         val probe = TestProbe()
@@ -149,19 +164,19 @@ ImplicitSender with WordSpecLike with Matchers with BeforeAndAfterAll {
     "handle repeated registration calls" in {
 
       var jmOption: Option[ActorGateway] = None
-      var rmOption: Option[ActorGateway] = None
+      var rmOption: Option[ActorRef] = None
 
       try {
         val probe = TestProbe()
 
-        val jm = startTestingJobManager(_system)
+        val jm = startTestingJobManager(_system, highAvailabilityServices)
         jmOption = Some(jm)
-        val rm = startTestingResourceManager(_system, jm.actor())
+        val rm = startTestingResourceManager(_system, highAvailabilityServices)
         rmOption = Some(rm)
 
-        val selfGateway = new AkkaActorGateway(
-          probe.ref,
-          HighAvailabilityServices.DEFAULT_LEADER_ID)
+        val leaderId = jm.leaderSessionID()
+
+        val selfGateway = new AkkaActorGateway(probe.ref, leaderId)
 
         val resourceID = ResourceID.generate()
         val connectionInfo = new TaskManagerLocation(resourceID, InetAddress.getLocalHost, 1)
@@ -199,22 +214,22 @@ ImplicitSender with WordSpecLike with Matchers with BeforeAndAfterAll {
 
           probe.expectMsgType[LeaderSessionMessage] match {
             case LeaderSessionMessage(
-            HighAvailabilityServices.DEFAULT_LEADER_ID,
-            AcknowledgeRegistration(_, _)) =>
+              `leaderId`,
+              AcknowledgeRegistration(_, _)) =>
             case m => fail("Wrong message type: " + m)
           }
 
           probe.expectMsgType[LeaderSessionMessage] match {
             case LeaderSessionMessage(
-            HighAvailabilityServices.DEFAULT_LEADER_ID,
-            AlreadyRegistered(_, _)) =>
+              `leaderId`,
+              AlreadyRegistered(_, _)) =>
             case m => fail("Wrong message type: " + m)
           }
 
           probe.expectMsgType[LeaderSessionMessage] match {
             case LeaderSessionMessage(
-            HighAvailabilityServices.DEFAULT_LEADER_ID,
-            AlreadyRegistered(_, _)) =>
+              `leaderId`,
+              AlreadyRegistered(_, _)) =>
             case m => fail("Wrong message type: " + m)
           }
         }
@@ -225,14 +240,16 @@ ImplicitSender with WordSpecLike with Matchers with BeforeAndAfterAll {
     }
   }
 
-  private def startTestingJobManager(system: ActorSystem): ActorGateway = {
-    val config = new Configuration()
+  private def startTestingJobManager(
+      system: ActorSystem,
+      highAvailabilityServices: HighAvailabilityServices): ActorGateway = {
 
+    val config = new Configuration()
+    
     val components = JobManager.createJobManagerComponents(
       config,
       executor,
-      executor,
-      None)
+      executor)
 
     // Start the JobManager without a MetricRegistry so that we don't start the MetricQueryService.
     // The problem of the MetricQueryService is that it starts an actor with a fixed name. Thus,
@@ -249,25 +266,34 @@ ImplicitSender with WordSpecLike with Matchers with BeforeAndAfterAll {
       ActorRef.noSender,
       components._4,
       components._5,
+      highAvailabilityServices.getJobManagerLeaderElectionService(
+        HighAvailabilityServices.DEFAULT_JOB_ID),
+      highAvailabilityServices.getSubmittedJobGraphStore(),
+      highAvailabilityServices.getCheckpointRecoveryFactory(),
       components._8,
-      components._9,
-      components._10,
-      components._11,
       None)
 
-    val jm = _system.actorOf(props)
+    _system.actorOf(props)
 
-    new AkkaActorGateway(jm, HighAvailabilityServices.DEFAULT_LEADER_ID)
+    LeaderRetrievalUtils.retrieveLeaderGateway(
+      highAvailabilityServices.getJobManagerLeaderRetriever(
+        HighAvailabilityServices.DEFAULT_JOB_ID),
+      system,
+      timeout)
   }
 
-  private def startTestingResourceManager(system: ActorSystem, jm: ActorRef): ActorGateway = {
+  private def startTestingResourceManager(
+      system: ActorSystem,
+      highAvailabilityServices: HighAvailabilityServices)
+    : ActorRef = {
     val config = new Configuration()
-    val rm: ActorRef = FlinkResourceManager.startResourceManagerActors(
+
+    FlinkResourceManager.startResourceManagerActors(
       config,
       _system,
-      LeaderRetrievalUtils.createLeaderRetrievalService(config, jm),
+      highAvailabilityServices.getJobManagerLeaderRetriever(
+        HighAvailabilityServices.DEFAULT_JOB_ID),
       classOf[TestingResourceManager])
-    new AkkaActorGateway(rm, HighAvailabilityServices.DEFAULT_LEADER_ID)
   }
 }
 

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/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 bd4a8fc..7980cdf 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
@@ -34,6 +34,7 @@ import org.apache.flink.runtime.clusterframework.FlinkResourceManager
 import org.apache.flink.runtime.clusterframework.types.ResourceIDRetrievable
 import org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager
 import org.apache.flink.runtime.executiongraph.restart.RestartStrategyFactory
+import org.apache.flink.runtime.highavailability.{HighAvailabilityServices, HighAvailabilityServicesUtils}
 import org.apache.flink.runtime.instance.{ActorGateway, InstanceManager}
 import org.apache.flink.runtime.jobmanager.scheduler.Scheduler
 import org.apache.flink.runtime.jobmanager.{JobManager, MemoryArchivist, SubmittedJobGraphStore}
@@ -48,7 +49,7 @@ import org.apache.flink.runtime.testingUtils.TestingTaskManagerMessages.NotifyWh
 import org.apache.flink.runtime.testutils.TestingResourceManager
 
 import scala.concurrent.duration.FiniteDuration
-import scala.concurrent.{Await, Future}
+import scala.concurrent.{Await, ExecutionContext, Future}
 
 /**
  * Testing cluster which starts the [[JobManager]] and [[TaskManager]] actors with testing support
@@ -60,16 +61,35 @@ import scala.concurrent.{Await, Future}
  */
 class TestingCluster(
     userConfiguration: Configuration,
+    highAvailabilityServices: HighAvailabilityServices,
     singleActorSystem: Boolean,
     synchronousDispatcher: Boolean)
   extends LocalFlinkMiniCluster(
     userConfiguration,
+    highAvailabilityServices,
     singleActorSystem) {
 
-  def this(userConfiguration: Configuration, singleActorSystem: Boolean) =
-    this(userConfiguration, singleActorSystem, false)
+  def this(
+      userConfiguration: Configuration,
+      singleActorSystem: Boolean,
+      synchronousDispatcher: Boolean) = {
+    this(
+      userConfiguration,
+      HighAvailabilityServicesUtils.createAvailableOrEmbeddedServices(
+        userConfiguration,
+        ExecutionContext.global),
+      singleActorSystem,
+      synchronousDispatcher)
+  }
+
+  def this(userConfiguration: Configuration, singleActorSystem: Boolean) = {
+    this(
+      userConfiguration,
+      singleActorSystem,
+      false)
+  }
 
-  def this(userConfiguration: Configuration) = this(userConfiguration, true, false)
+  def this(userConfiguration: Configuration) = this(userConfiguration, true)
 
   // --------------------------------------------------------------------------
 
@@ -224,10 +244,13 @@ class TestingCluster(
             Seq(newJobManagerActorSystem),
             1))
 
-          val lrs = createLeaderRetrievalService()
+          jobManagerLeaderRetrievalService.foreach(_.stop())
+
+          jobManagerLeaderRetrievalService = Option(
+            highAvailabilityServices.getJobManagerLeaderRetriever(
+              HighAvailabilityServices.DEFAULT_JOB_ID))
 
-          jobManagerLeaderRetrievalService = Some(lrs)
-          lrs.start(this)
+          jobManagerLeaderRetrievalService.foreach(_.start(this))
 
         case _ => throw new Exception("The JobManager of the TestingCluster have not " +
                                         "been started properly.")

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/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 d139a3f..c8977f0 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
@@ -27,22 +27,18 @@ import akka.pattern.{Patterns, ask}
 import com.google.common.util.concurrent.MoreExecutors
 import com.typesafe.config.ConfigFactory
 import grizzled.slf4j.Logger
-import org.apache.flink.api.common.JobExecutionResult
 import org.apache.flink.api.common.time.Time
 import org.apache.flink.configuration.{ConfigConstants, Configuration, HighAvailabilityOptions, TaskManagerOptions}
 import org.apache.flink.runtime.akka.AkkaUtils
-import org.apache.flink.runtime.client.JobClient
 import org.apache.flink.runtime.clusterframework.FlinkResourceManager
 import org.apache.flink.runtime.clusterframework.types.ResourceID
 import org.apache.flink.runtime.concurrent.{ScheduledExecutor, ScheduledExecutorServiceAdapter}
-import org.apache.flink.runtime.highavailability.{HighAvailabilityServices, HighAvailabilityServicesUtils}
+import org.apache.flink.runtime.highavailability.HighAvailabilityServices
 import org.apache.flink.runtime.instance.{ActorGateway, AkkaActorGateway}
-import org.apache.flink.runtime.jobgraph.JobGraph
 import org.apache.flink.runtime.jobmanager.{JobManager, MemoryArchivist}
 import org.apache.flink.runtime.jobmaster.JobMaster
-import org.apache.flink.runtime.jobmaster.JobMaster.{ARCHIVE_NAME, JOB_MANAGER_NAME}
 import org.apache.flink.runtime.leaderretrieval.StandaloneLeaderRetrievalService
-import org.apache.flink.runtime.messages.TaskManagerMessages.NotifyWhenRegisteredAtJobManager
+import org.apache.flink.runtime.messages.TaskManagerMessages.{NotifyWhenRegisteredAtJobManager, RegisteredAtJobManager}
 import org.apache.flink.runtime.taskmanager.TaskManager
 import org.apache.flink.runtime.testutils.TestingResourceManager
 import org.apache.flink.runtime.util.LeaderRetrievalUtils
@@ -222,69 +218,13 @@ object TestingUtils {
     }
   }
 
-  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,
-    taskManagerClass: Class[_ <: TaskManager])
-  : ActorGateway = {
-    val jobManagerURL = AkkaUtils.getAkkaURL(actorSystem, jobManager.actor)
-
-    createTaskManager(
-      actorSystem,
-      jobManagerURL,
-      configuration,
-      useLocalCommunication,
-      waitForRegistration,
-      taskManagerClass
-    )
-  }
-
-  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 highAvailabilityServices Service factory for high availability
     * @param configuration Configuration
     * @param useLocalCommunication true if the network stack shall use exclusively local
     *                              communication
@@ -293,15 +233,15 @@ object TestingUtils {
     * @return ActorGateway of the created TaskManager
     */
   def createTaskManager(
-    actorSystem: ActorSystem,
-    jobManagerURL: String,
-    configuration: Configuration,
-    useLocalCommunication: Boolean,
-    waitForRegistration: Boolean)
-  : ActorGateway = {
+      actorSystem: ActorSystem,
+      highAvailabilityServices: HighAvailabilityServices,
+      configuration: Configuration,
+      useLocalCommunication: Boolean,
+      waitForRegistration: Boolean)
+    : ActorGateway = {
     createTaskManager(
       actorSystem,
-      jobManagerURL,
+      highAvailabilityServices,
       configuration,
       useLocalCommunication,
       waitForRegistration,
@@ -311,7 +251,7 @@ object TestingUtils {
 
   def createTaskManager(
       actorSystem: ActorSystem,
-      jobManagerURL: String,
+      highAvailabilityServices: HighAvailabilityServices,
       configuration: Configuration,
       useLocalCommunication: Boolean,
       waitForRegistration: Boolean,
@@ -324,26 +264,27 @@ object TestingUtils {
 
     resultingConfiguration.addAll(configuration)
 
-    val leaderRetrievalService = Option(new StandaloneLeaderRetrievalService(jobManagerURL))
-
     val taskManager = TaskManager.startTaskManagerComponentsAndActor(
       resultingConfiguration,
       ResourceID.generate(),
       actorSystem,
+      highAvailabilityServices,
       "localhost",
       None,
-      leaderRetrievalService,
       useLocalCommunication,
       taskManagerClass
     )
 
-    if (waitForRegistration) {
+    val leaderId = if (waitForRegistration) {
       val notificationResult = (taskManager ? NotifyWhenRegisteredAtJobManager)(TESTING_DURATION)
+        .mapTo[RegisteredAtJobManager]
 
-      Await.ready(notificationResult, TESTING_DURATION)
+      Await.result(notificationResult, TESTING_DURATION).leaderId
+    } else {
+      HighAvailabilityServices.DEFAULT_LEADER_ID
     }
 
-    new AkkaActorGateway(taskManager, HighAvailabilityServices.DEFAULT_LEADER_ID)
+    new AkkaActorGateway(taskManager, leaderId)
   }
 
   /** Stops the given actor by sending it a Kill message
@@ -428,13 +369,15 @@ object TestingUtils {
       actorSystem: ActorSystem,
       futureExecutor: ScheduledExecutorService,
       ioExecutor: Executor,
-      configuration: Configuration)
+      configuration: Configuration,
+      highAvailabilityServices: HighAvailabilityServices)
     : ActorGateway = {
     createJobManager(
       actorSystem,
       futureExecutor,
       ioExecutor,
       configuration,
+      highAvailabilityServices,
       classOf[TestingJobManager],
       ""
     )
@@ -455,6 +398,7 @@ object TestingUtils {
       futureExecutor: ScheduledExecutorService,
       ioExecutor: Executor,
       configuration: Configuration,
+      highAvailabilityServices: HighAvailabilityServices,
       prefix: String)
     : ActorGateway = {
     createJobManager(
@@ -462,6 +406,7 @@ object TestingUtils {
       futureExecutor,
       ioExecutor,
       configuration,
+      highAvailabilityServices,
       classOf[TestingJobManager],
       prefix
     )
@@ -474,6 +419,7 @@ object TestingUtils {
     * @param futureExecutor to run the JobManager's futures
     * @param ioExecutor to run blocking io operations
     * @param configuration Configuration to use
+    * @param highAvailabilityServices Service factory for high availability
     * @param jobManagerClass JobManager class to instantiate
     * @return
     */
@@ -482,10 +428,18 @@ object TestingUtils {
       futureExecutor: ScheduledExecutorService,
       ioExecutor: Executor,
       configuration: Configuration,
+      highAvailabilityServices: HighAvailabilityServices,
       jobManagerClass: Class[_ <: JobManager])
     : ActorGateway = {
 
-    createJobManager(actorSystem, futureExecutor, ioExecutor, configuration, jobManagerClass, "")
+    createJobManager(
+      actorSystem,
+      futureExecutor,
+      ioExecutor,
+      configuration,
+      highAvailabilityServices,
+      jobManagerClass,
+      "")
   }
 
   /**
@@ -496,6 +450,7 @@ object TestingUtils {
     * @param futureExecutor to run the JobManager's futures
     * @param ioExecutor to run blocking io operations
     * @param configuration Configuration to use
+    * @param highAvailabilityServices Service factory for high availability
     * @param jobManagerClass JobManager class to instantiate
     * @param prefix The prefix to use for the Actor names
    * @return
@@ -505,6 +460,7 @@ object TestingUtils {
       futureExecutor: ScheduledExecutorService,
       ioExecutor: Executor,
       configuration: Configuration,
+      highAvailabilityServices: HighAvailabilityServices,
       jobManagerClass: Class[_ <: JobManager],
       prefix: String)
     : ActorGateway = {
@@ -518,24 +474,33 @@ object TestingUtils {
         actorSystem,
         futureExecutor,
         ioExecutor,
+        highAvailabilityServices,
         Some(prefix + JobMaster.JOB_MANAGER_NAME),
         Some(prefix + JobMaster.ARCHIVE_NAME),
         jobManagerClass,
         classOf[MemoryArchivist])
 
-    new AkkaActorGateway(actor, HighAvailabilityServices.DEFAULT_LEADER_ID)
+
+    val leaderId = LeaderRetrievalUtils.retrieveLeaderSessionId(
+        highAvailabilityServices.getJobManagerLeaderRetriever(
+          HighAvailabilityServices.DEFAULT_JOB_ID),
+        TestingUtils.TESTING_TIMEOUT)
+
+    new AkkaActorGateway(actor, leaderId)
   }
 
   /** Creates a forwarding JobManager which sends all received message to the forwarding target.
     *
     * @param actorSystem The actor system to start the actor in.
     * @param forwardingTarget Target to forward to.
+    * @param leaderId leader id for the actor gateway
     * @param actorName Name for forwarding Actor
     * @return
     */
   def createForwardingActor(
       actorSystem: ActorSystem,
       forwardingTarget: ActorRef,
+      leaderId: UUID,
       actorName: Option[String] = None)
     : ActorGateway = {
 
@@ -545,7 +510,7 @@ object TestingUtils {
           Props(
             classOf[ForwardingActor],
             forwardingTarget,
-            Option(HighAvailabilityServices.DEFAULT_LEADER_ID)),
+            Option(leaderId)),
           name
         )
       case None =>
@@ -553,58 +518,39 @@ object TestingUtils {
           Props(
             classOf[ForwardingActor],
             forwardingTarget,
-            Option(HighAvailabilityServices.DEFAULT_LEADER_ID))
+            Option(leaderId))
         )
     }
 
-    new AkkaActorGateway(actor, HighAvailabilityServices.DEFAULT_LEADER_ID)
-  }
-
-  def submitJobAndWait(
-      actorSystem: ActorSystem,
-      jobManager: ActorGateway,
-      jobGraph: JobGraph,
-      config: Configuration)
-    : JobExecutionResult = {
-
-    val jobManagerURL = AkkaUtils.getAkkaURL(actorSystem, jobManager.actor)
-    val leaderRetrievalService = new StandaloneLeaderRetrievalService(jobManagerURL)
-
-    JobClient.submitJobAndWait(
-      actorSystem,
-      config,
-      leaderRetrievalService,
-      jobGraph,
-      TESTING_DURATION,
-      false,
-      Thread.currentThread().getContextClassLoader
-    )
+    new AkkaActorGateway(actor, leaderId)
   }
 
-  /** Creates a testing JobManager using the default recovery mode (standalone)
+  /** Creates a testing JobManager using the given configuration and high availability services.
     *
     * @param actorSystem The actor system to start the actor
-    * @param jobManager The jobManager for the standalone leader service.
     * @param configuration The configuration
+    * @param highAvailabilityServices The high availability services to use
     * @return
     */
   def createResourceManager(
       actorSystem: ActorSystem,
-      jobManager: ActorRef,
-      configuration: Configuration)
+      configuration: Configuration,
+      highAvailabilityServices: HighAvailabilityServices)
   : ActorGateway = {
 
-    configuration.setString(
-      HighAvailabilityOptions.HA_MODE,
-      ConfigConstants.DEFAULT_HA_MODE)
-
-    val actor = FlinkResourceManager.startResourceManagerActors(
+    val resourceManager = FlinkResourceManager.startResourceManagerActors(
       configuration,
       actorSystem,
-      LeaderRetrievalUtils.createLeaderRetrievalService(configuration, jobManager),
+      highAvailabilityServices.getJobManagerLeaderRetriever(
+        HighAvailabilityServices.DEFAULT_JOB_ID),
       classOf[TestingResourceManager])
 
-    new AkkaActorGateway(actor, HighAvailabilityServices.DEFAULT_LEADER_ID)
+    val leaderId = LeaderRetrievalUtils.retrieveLeaderSessionId(
+      highAvailabilityServices.getJobManagerLeaderRetriever(
+        HighAvailabilityServices.DEFAULT_JOB_ID),
+      TestingUtils.TESTING_TIMEOUT)
+
+    new AkkaActorGateway(resourceManager, leaderId)
   }
 
   class ForwardingActor(val target: ActorRef, val leaderSessionID: Option[UUID])

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/flink-scala-shell/src/main/scala/org/apache/flink/api/scala/FlinkShell.scala
----------------------------------------------------------------------
diff --git a/flink-scala-shell/src/main/scala/org/apache/flink/api/scala/FlinkShell.scala b/flink-scala-shell/src/main/scala/org/apache/flink/api/scala/FlinkShell.scala
index 4d3adb2..bb26454 100644
--- a/flink-scala-shell/src/main/scala/org/apache/flink/api/scala/FlinkShell.scala
+++ b/flink-scala-shell/src/main/scala/org/apache/flink/api/scala/FlinkShell.scala
@@ -25,6 +25,7 @@ import org.apache.commons.cli.CommandLine
 import org.apache.flink.client.cli.CliFrontendParser
 import org.apache.flink.client.program.ClusterClient
 import org.apache.flink.client.CliFrontend
+import org.apache.flink.runtime.minicluster.StandaloneMiniCluster
 import org.apache.flink.configuration.{ConfigConstants, GlobalConfiguration}
 import org.apache.flink.runtime.minicluster.{FlinkMiniCluster, LocalFlinkMiniCluster}
 
@@ -140,18 +141,17 @@ object FlinkShell {
 
   def fetchConnectionInfo(
     config: Config
-  ): (String, Int, Option[Either[FlinkMiniCluster, ClusterClient]]) = {
+  ): (String, Int, Option[Either[StandaloneMiniCluster, ClusterClient]]) = {
     config.executionMode match {
       case ExecutionMode.LOCAL => // Local mode
         val config = GlobalConfiguration.loadConfiguration()
         config.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, 0)
 
-        val miniCluster = new LocalFlinkMiniCluster(config, false)
-        miniCluster.start()
+        val miniCluster = new StandaloneMiniCluster(config)
 
         println("\nStarting local Flink cluster (host: localhost, " +
-          s"port: ${miniCluster.getLeaderRPCPort}).\n")
-        ("localhost", miniCluster.getLeaderRPCPort, Some(Left(miniCluster)))
+          s"port: ${miniCluster.getPort}).\n")
+        ("localhost", miniCluster.getPort, Some(Left(miniCluster)))
 
       case ExecutionMode.REMOTE => // Remote mode
         if (config.host.isEmpty || config.port.isEmpty) {
@@ -188,7 +188,7 @@ object FlinkShell {
     val (repl, cluster) = try {
       val (host, port, cluster) = fetchConnectionInfo(config)
       val conf = cluster match {
-        case Some(Left(miniCluster)) => miniCluster.userConfiguration
+        case Some(Left(miniCluster)) => miniCluster.getConfiguration
         case Some(Right(yarnCluster)) => yarnCluster.getFlinkConfiguration
         case None => GlobalConfiguration.loadConfiguration()
       }
@@ -218,7 +218,7 @@ object FlinkShell {
     } finally {
       repl.closeInterpreter()
       cluster match {
-        case Some(Left(miniCluster)) => miniCluster.stop()
+        case Some(Left(miniCluster)) => miniCluster.close()
         case Some(Right(yarnCluster)) => yarnCluster.shutdown()
         case _ =>
       }

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/flink-scala-shell/src/test/scala/org/apache/flink/api/scala/ScalaShellITCase.scala
----------------------------------------------------------------------
diff --git a/flink-scala-shell/src/test/scala/org/apache/flink/api/scala/ScalaShellITCase.scala b/flink-scala-shell/src/test/scala/org/apache/flink/api/scala/ScalaShellITCase.scala
index 00410cc..0e89da3 100644
--- a/flink-scala-shell/src/test/scala/org/apache/flink/api/scala/ScalaShellITCase.scala
+++ b/flink-scala-shell/src/test/scala/org/apache/flink/api/scala/ScalaShellITCase.scala
@@ -19,14 +19,16 @@
 package org.apache.flink.api.scala
 
 import java.io._
-import java.util.concurrent.TimeUnit
 
-import org.apache.flink.configuration.GlobalConfiguration
-import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster
+import akka.actor.ActorRef
+import akka.pattern.Patterns
+import org.apache.flink.runtime.minicluster.StandaloneMiniCluster
+import org.apache.flink.configuration.{ConfigConstants, Configuration, GlobalConfiguration}
 import org.apache.flink.test.util.TestBaseUtils
 import org.apache.flink.util.TestLogger
 import org.junit.{AfterClass, Assert, BeforeClass, Test}
 
+import scala.concurrent.{Await, Future}
 import scala.concurrent.duration.FiniteDuration
 import scala.tools.nsc.Settings
 
@@ -299,18 +301,19 @@ class ScalaShellITCase extends TestLogger {
       .getFile
     val confDir = new File(confFile).getAbsoluteFile.getParent
 
-    val (c, args) = cluster match{
+    val args = cluster match {
       case Some(cl) =>
-        val arg = Array("remote",
-          cl.hostname,
-          Integer.toString(cl.getLeaderRPCPort),
+        Array(
+          "remote",
+          cl.getHostname,
+          Integer.toString(cl.getPort),
           "--configDir",
           confDir)
-        (cl, arg)
-      case None =>
-        throw new AssertionError("Cluster creation failed.")
+      case None => throw new IllegalStateException("Cluster has not been started.")
     }
 
+
+
     //start scala shell with initialized
     // buffered reader for testing
     FlinkShell.bufferedReader = Some(in)
@@ -335,26 +338,24 @@ class ScalaShellITCase extends TestLogger {
 }
 
 object ScalaShellITCase {
-  var cluster: Option[LocalFlinkMiniCluster] = None
+  var cluster: Option[StandaloneMiniCluster] = None
+
   val parallelism = 4
+  val configuration = new Configuration()
 
   @BeforeClass
   def beforeAll(): Unit = {
-    val cl = TestBaseUtils.startCluster(
-      1,
-      parallelism,
-      false,
-      false,
-      false)
-
-    cluster = Some(cl)
+    configuration.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, parallelism)
+
+    cluster = Option(new StandaloneMiniCluster(configuration))
   }
 
   @AfterClass
   def afterAll(): Unit = {
     // The Scala interpreter somehow changes the class loader. Therfore, we have to reset it
     Thread.currentThread().setContextClassLoader(classOf[ScalaShellITCase].getClassLoader)
-    cluster.foreach(c => TestBaseUtils.stopCluster(c, new FiniteDuration(1000, TimeUnit.SECONDS)))
+
+    cluster.foreach(_.close)
   }
 
   /**
@@ -371,47 +372,46 @@ object ScalaShellITCase {
     val oldOut = System.out
     System.setOut(new PrintStream(baos))
 
-    // new local cluster
-    val host = "localhost"
-    val port = cluster match {
-      case Some(c) => c.getLeaderRPCPort
-      case _ => throw new RuntimeException("Test cluster not initialized.")
-    }
-
-    val repl = externalJars match {
-      case Some(ej) => new FlinkILoop(
-        host, port,
-        GlobalConfiguration.loadConfiguration(),
-        Option(Array(ej)),
-        in, new PrintWriter(out))
-
-      case None => new FlinkILoop(
-        host, port,
-        GlobalConfiguration.loadConfiguration(),
-        in, new PrintWriter(out))
-    }
+    cluster match {
+      case Some(cl) =>
+        val repl = externalJars match {
+          case Some(ej) => new FlinkILoop(
+            cl.getHostname,
+            cl.getPort,
+            GlobalConfiguration.loadConfiguration(),
+            Option(Array(ej)),
+            in, new PrintWriter(out))
+
+          case None => new FlinkILoop(
+            cl.getHostname,
+            cl.getPort,
+            GlobalConfiguration.loadConfiguration(),
+            in, new PrintWriter(out))
+        }
 
-    repl.settings = new Settings()
+        repl.settings = new Settings()
 
-    // enable this line to use scala in intellij
-    repl.settings.usejavacp.value = true
+        // enable this line to use scala in intellij
+        repl.settings.usejavacp.value = true
 
-    externalJars match {
-      case Some(ej) => repl.settings.classpath.value = ej
-      case None =>
-    }
+        externalJars match {
+          case Some(ej) => repl.settings.classpath.value = ej
+          case None =>
+        }
 
-    repl.process(repl.settings)
+        repl.process(repl.settings)
 
-    repl.closeInterpreter()
+        repl.closeInterpreter()
 
-    System.setOut(oldOut)
+        System.setOut(oldOut)
 
-    baos.flush()
+        baos.flush()
 
-    val stdout = baos.toString
+        val stdout = baos.toString
 
-    out.toString + stdout
+        out.toString + stdout
+      case _ => throw new IllegalStateException("The cluster has not been started.")
+    }
   }
 
   def findLibraryFolder(paths: String*): File = {

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/RemoteStreamEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/RemoteStreamEnvironment.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/RemoteStreamEnvironment.java
index ff5af87..2ff45ba 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/RemoteStreamEnvironment.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/RemoteStreamEnvironment.java
@@ -219,7 +219,11 @@ public class RemoteStreamEnvironment extends StreamExecutionEnvironment {
 			throw new ProgramInvocationException("The program execution failed" + term, e);
 		}
 		finally {
-			client.shutdown();
+			try {
+				client.shutdown();
+			} catch (Exception e) {
+				LOG.warn("Could not properly shut down the cluster client.", e);
+			}
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/streaming/util/TestStreamEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/streaming/util/TestStreamEnvironment.java b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/streaming/util/TestStreamEnvironment.java
index 64c68dc..5719456 100644
--- a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/streaming/util/TestStreamEnvironment.java
+++ b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/streaming/util/TestStreamEnvironment.java
@@ -19,6 +19,7 @@
 package org.apache.flink.streaming.util;
 
 import org.apache.flink.api.common.JobExecutionResult;
+import org.apache.flink.core.fs.Path;
 import org.apache.flink.runtime.jobgraph.JobGraph;
 import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
@@ -26,43 +27,84 @@ import org.apache.flink.streaming.api.environment.StreamExecutionEnvironmentFact
 import org.apache.flink.streaming.api.graph.StreamGraph;
 import org.apache.flink.util.Preconditions;
 
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+
 /**
- * A StreamExecutionEnvironment that executes its jobs on a test cluster.
+ * A {@link StreamExecutionEnvironment} that executes its jobs on {@link LocalFlinkMiniCluster}.
  */
 public class TestStreamEnvironment extends StreamExecutionEnvironment {
 	
 	/** The mini cluster in which this environment executes its jobs */
-	private LocalFlinkMiniCluster executor;
-	
+	private final LocalFlinkMiniCluster miniCluster;
+
+	private final Collection<Path> jarFiles;
+
+	private final Collection<URL> classPaths;
+
+	public TestStreamEnvironment(
+			LocalFlinkMiniCluster miniCluster,
+			int parallelism,
+			Collection<Path> jarFiles,
+			Collection<URL> classPaths) {
+
+		this.miniCluster = Preconditions.checkNotNull(miniCluster);
+		this.jarFiles = Preconditions.checkNotNull(jarFiles);
+		this.classPaths = Preconditions.checkNotNull(classPaths);
 
-	public TestStreamEnvironment(LocalFlinkMiniCluster executor, int parallelism) {
-		this.executor = Preconditions.checkNotNull(executor);
 		setParallelism(parallelism);
 	}
+
+	public TestStreamEnvironment(
+			LocalFlinkMiniCluster miniCluster,
+			int parallelism) {
+		this(miniCluster, parallelism, Collections.<Path>emptyList(), Collections.<URL>emptyList());
+	}
+
 	
 	@Override
 	public JobExecutionResult execute(String jobName) throws Exception {
 		final StreamGraph streamGraph = getStreamGraph();
 		streamGraph.setJobName(jobName);
 		final JobGraph jobGraph = streamGraph.getJobGraph();
-		return executor.submitJobAndWait(jobGraph, false);
+
+		for (Path jarFile: jarFiles) {
+			jobGraph.addJar(jarFile);
+		}
+
+		jobGraph.setClasspaths(new ArrayList<>(classPaths));
+
+		return miniCluster.submitJobAndWait(jobGraph, false);
 	}
 
 	// ------------------------------------------------------------------------
 
 	/**
 	 * Sets the streaming context environment to a TestStreamEnvironment that runs its programs on
-	 * the given cluster with the given default parallelism.
-	 * 
+	 * the given cluster with the given default parallelism and the specified jar files and class
+	 * paths.
+	 *
 	 * @param cluster The test cluster to run the test program on.
 	 * @param parallelism The default parallelism for the test programs.
+	 * @param jarFiles Additional jar files to execute the job with
+	 * @param classpaths Additional class paths to execute the job with
 	 */
-	public static void setAsContext(final LocalFlinkMiniCluster cluster, final int parallelism) {
-		
+	public static void setAsContext(
+			final LocalFlinkMiniCluster cluster,
+			final int parallelism,
+			final Collection<Path> jarFiles,
+			final Collection<URL> classpaths) {
+
 		StreamExecutionEnvironmentFactory factory = new StreamExecutionEnvironmentFactory() {
 			@Override
 			public StreamExecutionEnvironment createExecutionEnvironment() {
-				return new TestStreamEnvironment(cluster, parallelism);
+				return new TestStreamEnvironment(
+					cluster,
+					parallelism,
+					jarFiles,
+					classpaths);
 			}
 		};
 
@@ -70,6 +112,21 @@ public class TestStreamEnvironment extends StreamExecutionEnvironment {
 	}
 
 	/**
+	 * Sets the streaming context environment to a TestStreamEnvironment that runs its programs on
+	 * the given cluster with the given default parallelism.
+	 * 
+	 * @param cluster The test cluster to run the test program on.
+	 * @param parallelism The default parallelism for the test programs.
+	 */
+	public static void setAsContext(final LocalFlinkMiniCluster cluster, final int parallelism) {
+		setAsContext(
+			cluster,
+			parallelism,
+			Collections.<Path>emptyList(),
+			Collections.<URL>emptyList());
+	}
+
+	/**
 	 * Resets the streaming context environment to null.
 	 */
 	public static void unsetAsContext() {

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/JavaProgramTestBase.java
----------------------------------------------------------------------
diff --git a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/JavaProgramTestBase.java b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/JavaProgramTestBase.java
index 2bcf66d..f06cdb7 100644
--- a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/JavaProgramTestBase.java
+++ b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/JavaProgramTestBase.java
@@ -111,7 +111,7 @@ public abstract class JavaProgramTestBase extends AbstractTestBase {
 			}
 			
 			// prepare the test environment
-			TestEnvironment env = new TestEnvironment(this.executor, this.parallelism);
+			TestEnvironment env = new TestEnvironment(this.executor, this.parallelism, false);
 			env.getConfig().enableObjectReuse();
 			env.setAsContext();
 
@@ -163,7 +163,7 @@ public abstract class JavaProgramTestBase extends AbstractTestBase {
 			}
 
 			// prepare the test environment
-			TestEnvironment env = new TestEnvironment(this.executor, this.parallelism);
+			TestEnvironment env = new TestEnvironment(this.executor, this.parallelism, false);
 			env.getConfig().disableObjectReuse();
 			env.setAsContext();
 

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/MultipleProgramsTestBase.java
----------------------------------------------------------------------
diff --git a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/MultipleProgramsTestBase.java b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/MultipleProgramsTestBase.java
index 2043cd0..ed141d1 100644
--- a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/MultipleProgramsTestBase.java
+++ b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/MultipleProgramsTestBase.java
@@ -86,7 +86,7 @@ public class MultipleProgramsTestBase extends TestBaseUtils {
 		
 		switch(mode){
 			case CLUSTER:
-				new TestEnvironment(cluster, 4).setAsContext();
+				new TestEnvironment(cluster, 4, false).setAsContext();
 				break;
 			case CLUSTER_OBJECT_REUSE:
 				new TestEnvironment(cluster, 4, true).setAsContext();

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/TestEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/TestEnvironment.java b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/TestEnvironment.java
index aea8152..a30db40 100644
--- a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/TestEnvironment.java
+++ b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/TestEnvironment.java
@@ -23,6 +23,7 @@ import org.apache.flink.api.common.Plan;
 import org.apache.flink.api.common.CodeAnalysisMode;
 import org.apache.flink.api.java.ExecutionEnvironment;
 import org.apache.flink.api.java.ExecutionEnvironmentFactory;
+import org.apache.flink.core.fs.Path;
 import org.apache.flink.optimizer.DataStatistics;
 import org.apache.flink.optimizer.Optimizer;
 import org.apache.flink.optimizer.plan.OptimizedPlan;
@@ -30,39 +31,71 @@ import org.apache.flink.optimizer.plandump.PlanJSONDumpGenerator;
 import org.apache.flink.optimizer.plantranslate.JobGraphGenerator;
 import org.apache.flink.runtime.jobgraph.JobGraph;
 import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
+import org.apache.flink.util.Preconditions;
 
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+
+/**
+ * A {@link ExecutionEnvironment} implementation which executes its jobs on a
+ * {@link LocalFlinkMiniCluster}.
+ */
 public class TestEnvironment extends ExecutionEnvironment {
 
-	private final LocalFlinkMiniCluster executor;
+	private final LocalFlinkMiniCluster miniCluster;
 
-	private TestEnvironment lastEnv = null;
+	private final Collection<Path> jarFiles;
 
-	@Override
-	public JobExecutionResult getLastJobExecutionResult() {
-		if (lastEnv == null) {
-			return this.lastJobExecutionResult;
-		}
-		else {
-			return lastEnv.getLastJobExecutionResult();
-		}
-	}
+	private final Collection<URL> classPaths;
+
+	private TestEnvironment lastEnv;
+
+	public TestEnvironment(
+			LocalFlinkMiniCluster miniCluster,
+			int parallelism,
+			boolean isObjectReuseEnabled,
+			Collection<Path> jarFiles,
+			Collection<URL> classPaths) {
+		this.miniCluster = Preconditions.checkNotNull(miniCluster);
+		this.jarFiles = Preconditions.checkNotNull(jarFiles);
+		this.classPaths = Preconditions.checkNotNull(classPaths);
 
-	public TestEnvironment(LocalFlinkMiniCluster executor, int parallelism) {
-		this.executor = executor;
 		setParallelism(parallelism);
 
 		// disabled to improve build time
 		getConfig().setCodeAnalysisMode(CodeAnalysisMode.DISABLE);
-	}
-
-	public TestEnvironment(LocalFlinkMiniCluster executor, int parallelism, boolean isObjectReuseEnabled) {
-		this(executor, parallelism);
 
 		if (isObjectReuseEnabled) {
 			getConfig().enableObjectReuse();
 		} else {
 			getConfig().disableObjectReuse();
 		}
+
+		lastEnv = null;
+	}
+
+	public TestEnvironment(
+			LocalFlinkMiniCluster executor,
+			int parallelism,
+			boolean isObjectReuseEnabled) {
+		this(
+			executor,
+			parallelism,
+			isObjectReuseEnabled,
+			Collections.<Path>emptyList(),
+			Collections.<URL>emptyList());
+	}
+
+	@Override
+	public JobExecutionResult getLastJobExecutionResult() {
+		if (lastEnv == null) {
+			return lastJobExecutionResult;
+		}
+		else {
+			return lastEnv.getLastJobExecutionResult();
+		}
 	}
 
 	@Override
@@ -76,7 +109,13 @@ public class TestEnvironment extends ExecutionEnvironment {
 		JobGraphGenerator jgg = new JobGraphGenerator();
 		JobGraph jobGraph = jgg.compileJobGraph(op);
 
-		this.lastJobExecutionResult = executor.submitJobAndWait(jobGraph, false);
+		for (Path jarFile: jarFiles) {
+			jobGraph.addJar(jarFile);
+		}
+
+		jobGraph.setClasspaths(new ArrayList<>(classPaths));
+
+		this.lastJobExecutionResult = miniCluster.submitJobAndWait(jobGraph, false);
 		return this.lastJobExecutionResult;
 	}
 
@@ -93,7 +132,7 @@ public class TestEnvironment extends ExecutionEnvironment {
 	private OptimizedPlan compileProgram(String jobName) {
 		Plan p = createProgramPlan(jobName);
 
-		Optimizer pc = new Optimizer(new DataStatistics(), this.executor.configuration());
+		Optimizer pc = new Optimizer(new DataStatistics(), this.miniCluster.configuration());
 		return pc.compile(p);
 	}
 
@@ -101,11 +140,65 @@ public class TestEnvironment extends ExecutionEnvironment {
 		ExecutionEnvironmentFactory factory = new ExecutionEnvironmentFactory() {
 			@Override
 			public ExecutionEnvironment createExecutionEnvironment() {
-				lastEnv = new TestEnvironment(executor, getParallelism(), getConfig().isObjectReuseEnabled());
+				lastEnv = new TestEnvironment(miniCluster, getParallelism(), getConfig().isObjectReuseEnabled());
 				return lastEnv;
 			}
 		};
 
 		initializeContextEnvironment(factory);
 	}
+
+	// ---------------------------------------------------------------------------------------------
+
+	/**
+	 * Sets the current {@link ExecutionEnvironment} to be a {@link TestEnvironment}. The test
+	 * environment executes the given jobs on a Flink mini cluster with the given default
+	 * parallelism and the additional jar files and class paths.
+	 *
+	 * @param miniCluster The mini cluster on which to execute the jobs
+	 * @param parallelism The default parallelism
+	 * @param jarFiles Additional jar files to execute the job with
+	 * @param classPaths Additional class paths to execute the job with
+	 */
+	public static void setAsContext(
+		final LocalFlinkMiniCluster miniCluster,
+		final int parallelism,
+		final Collection<Path> jarFiles,
+		final Collection<URL> classPaths) {
+
+		ExecutionEnvironmentFactory factory = new ExecutionEnvironmentFactory() {
+			@Override
+			public ExecutionEnvironment createExecutionEnvironment() {
+				return new TestEnvironment(
+					miniCluster,
+					parallelism,
+					false, jarFiles,
+					classPaths
+				);
+			}
+		};
+
+		initializeContextEnvironment(factory);
+	}
+
+
+	/**
+	 * Sets the current {@link ExecutionEnvironment} to be a {@link TestEnvironment}. The test
+	 * environment executes the given jobs on a Flink mini cluster with the given default
+	 * parallelism and the additional jar files and class paths.
+	 *
+	 * @param miniCluster The mini cluster on which to execute the jobs
+	 * @param parallelism The default parallelism
+	 */
+	public static void setAsContext(final LocalFlinkMiniCluster miniCluster, final int parallelism) {
+		setAsContext(
+			miniCluster,
+			parallelism,
+			Collections.<Path>emptyList(),
+			Collections.<URL>emptyList());
+	}
+
+	public static void unsetAsContext() {
+		resetContextEnvironment();
+	}
 }


[16/16] flink git commit: [FLINK-5974] [mesos] Make mesos service name configurable for dns lookups

Posted by tr...@apache.org.
[FLINK-5974] [mesos] Make mesos service name configurable for dns lookups


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

Branch: refs/heads/master
Commit: d7364fffbf552aed79e537a7aec3af593cb4e159
Parents: 1e53b75
Author: Till Rohrmann <tr...@apache.org>
Authored: Wed May 3 16:48:06 2017 +0200
Committer: Till Rohrmann <tr...@apache.org>
Committed: Fri May 5 11:36:11 2017 +0200

----------------------------------------------------------------------
 docs/setup/mesos.md                             |  4 +++
 .../flink/configuration/ConfigConstants.java    |  4 ---
 .../clusterframework/LaunchableMesosWorker.java | 27 ++++++++++------
 .../MesosApplicationMasterRunner.java           | 17 +++-------
 .../MesosFlinkResourceManager.java              |  9 ++++--
 .../MesosTaskManagerParameters.java             | 34 +++++++++++---------
 6 files changed, 52 insertions(+), 43 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/d7364fff/docs/setup/mesos.md
----------------------------------------------------------------------
diff --git a/docs/setup/mesos.md b/docs/setup/mesos.md
index 4c72fbc..674aa92 100644
--- a/docs/setup/mesos.md
+++ b/docs/setup/mesos.md
@@ -263,3 +263,7 @@ May be set to -1 to disable this feature.
 `mesos.resourcemanager.tasks.container.image.name`: Image name to use for the container (**NO DEFAULT**)
 
 `mesos.resourcemanager.tasks.container.volumes`: A comma seperated list of [host_path:]container_path[:RO|RW]. This allows for mounting additional volumes into your container. (**NO DEFAULT**)
+
+`mesos.resourcemanager.tasks.hostname`: Optional value to define the TaskManager's hostname. The pattern `_TASK_` is replaced by the actual id of the Mesos task. This can be used to configure the TaskManager to use Mesos DNS (e.g. `_TASK_.flink-service.mesos`) for name lookups. (**NO DEFAULT**)
+
+`mesos.resourcemanager.tasks.bootstrap-cmd`: A command which is executed before the TaskManager is started (**NO DEFAULT**).

http://git-wip-us.apache.org/repos/asf/flink/blob/d7364fff/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 cafef2e..975a3d4 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
@@ -1286,10 +1286,6 @@ public final class ConfigConstants {
 
 	public static final String DEFAULT_MESOS_RESOURCEMANAGER_FRAMEWORK_USER = "";
 
-	public static final String MESOS_RESOURCEMANAGER_TASKS_HOSTNAME = "mesos.resourcemanager.tasks.hostname";
-
-	public static final String MESOS_RESOURCEMANAGER_TASKS_BOOTSTRAP_CMD = "mesos.resourcemanager.tasks.cmd-prefix";
-
 	/** Default value to override SSL support for the Artifact Server */
 	public static final boolean DEFAULT_MESOS_ARTIFACT_SERVER_SSL_ENABLED = true;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/d7364fff/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/LaunchableMesosWorker.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/LaunchableMesosWorker.java b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/LaunchableMesosWorker.java
index 3e66a5d..04a406f 100644
--- a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/LaunchableMesosWorker.java
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/LaunchableMesosWorker.java
@@ -34,11 +34,13 @@ import org.apache.flink.util.Preconditions;
 import org.apache.mesos.Protos;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import scala.Option;
 
 import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.atomic.AtomicReference;
+import java.util.regex.Matcher;
 
 import static org.apache.flink.mesos.Utils.variable;
 import static org.apache.flink.mesos.Utils.range;
@@ -200,9 +202,15 @@ public class LaunchableMesosWorker implements LaunchableTask {
 		final StringBuilder jvmArgs = new StringBuilder();
 
 		//configure task manager hostname property if hostname override property is supplied
-		if(params.getTaskManagerHostname().isDefined()) {
-			final String taskManagerHostName = params.getTaskManagerHostname().get().replace("_TASK",taskID.getValue());
-			dynamicProperties.setString(ConfigConstants.TASK_MANAGER_HOSTNAME_KEY, taskManagerHostName);
+		Option<String> taskManagerHostnameOption = params.getTaskManagerHostname();
+
+		if(taskManagerHostnameOption.isDefined()) {
+			// replace the TASK_ID pattern by the actual task id value of the Mesos task
+			final String taskManagerHostname = MesosTaskManagerParameters.TASK_ID_PATTERN
+				.matcher(taskManagerHostnameOption.get())
+				.replaceAll(Matcher.quoteReplacement(taskID.getValue()));
+
+			dynamicProperties.setString(ConfigConstants.TASK_MANAGER_HOSTNAME_KEY, taskManagerHostname);
 		}
 
 		// use the assigned ports for the TM
@@ -251,13 +259,12 @@ public class LaunchableMesosWorker implements LaunchableTask {
 		env.addVariables(variable(MesosConfigKeys.ENV_FRAMEWORK_NAME, mesosConfiguration.frameworkInfo().getName()));
 
 		// build the launch command w/ dynamic application properties
-		StringBuilder launchCommand = new StringBuilder();
-		if(params.bootstrapCommand().isDefined()) {
-			launchCommand.append(params.bootstrapCommand().get()).append(" && ");
-		}
-		launchCommand.append("$FLINK_HOME/bin/mesos-taskmanager.sh ");
-		launchCommand.append(ContainerSpecification.formatSystemProperties(dynamicProperties));
-		cmd.setValue(launchCommand.toString());
+		Option<String> bootstrapCmdOption = params.bootstrapCommand();
+
+		final String bootstrapCommand = bootstrapCmdOption.isDefined() ? bootstrapCmdOption.get() + " && " : "";
+		final String launchCommand = bootstrapCommand + "$FLINK_HOME/bin/mesos-taskmanager.sh " + ContainerSpecification.formatSystemProperties(dynamicProperties);
+
+		cmd.setValue(launchCommand);
 
 		// build the container info
 		Protos.ContainerInfo.Builder containerInfo = Protos.ContainerInfo.newBuilder();

http://git-wip-us.apache.org/repos/asf/flink/blob/d7364fff/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosApplicationMasterRunner.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosApplicationMasterRunner.java b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosApplicationMasterRunner.java
index 1cedcc3..09ef380 100644
--- a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosApplicationMasterRunner.java
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosApplicationMasterRunner.java
@@ -32,6 +32,7 @@ import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.GlobalConfiguration;
 import org.apache.flink.configuration.IllegalConfigurationException;
+import org.apache.flink.configuration.JobManagerOptions;
 import org.apache.flink.core.fs.FileSystem;
 import org.apache.flink.mesos.runtime.clusterframework.services.MesosServices;
 import org.apache.flink.mesos.runtime.clusterframework.services.MesosServicesUtils;
@@ -210,18 +211,10 @@ public class MesosApplicationMasterRunner {
 		try {
 			// ------- (1) load and parse / validate all configurations -------
 
-			final String appMasterHostname;
-			//We will use JM RPC address property if it is supplied through configuration
-			final String jmRpcAddress = config.getString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, null);
-			if(jmRpcAddress != null) {
-				LOG.info("JM RPC address from Flink configuration file: {} ", jmRpcAddress);
-				appMasterHostname = jmRpcAddress;
-			} else {
-				// Note that we use the "appMasterHostname" given by the system, to make sure
-				// we use the hostnames consistently throughout akka.
-				// for akka "localhost" and "localhost.localdomain" are different actors.
-				appMasterHostname = InetAddress.getLocalHost().getHostName();
-			}
+			final String appMasterHostname = config.getString(
+				JobManagerOptions.ADDRESS,
+				InetAddress.getLocalHost().getHostName());
+
 			LOG.info("App Master Hostname to use: {}", appMasterHostname);
 
 			// Mesos configuration

http://git-wip-us.apache.org/repos/asf/flink/blob/d7364fff/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosFlinkResourceManager.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosFlinkResourceManager.java b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosFlinkResourceManager.java
index a9ff6cb..17ffef7 100644
--- a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosFlinkResourceManager.java
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosFlinkResourceManager.java
@@ -669,8 +669,13 @@ public class MesosFlinkResourceManager extends FlinkResourceManager<RegisteredMe
 
 	private LaunchableMesosWorker createLaunchableMesosWorker(Protos.TaskID taskID) {
 		LaunchableMesosWorker launchable =
-			new LaunchableMesosWorker(artifactResolver, taskManagerParameters, taskManagerContainerSpec,
-					taskID, mesosConfig);
+			new LaunchableMesosWorker(
+				artifactResolver,
+				taskManagerParameters,
+				taskManagerContainerSpec,
+				taskID,
+				mesosConfig);
+
 		return launchable;
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/d7364fff/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosTaskManagerParameters.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosTaskManagerParameters.java b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosTaskManagerParameters.java
index b3b8162..4324469 100644
--- a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosTaskManagerParameters.java
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosTaskManagerParameters.java
@@ -33,6 +33,7 @@ import scala.Option;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
+import java.util.regex.Pattern;
 
 import static org.apache.flink.configuration.ConfigOptions.key;
 
@@ -44,33 +45,36 @@ import static org.apache.flink.configuration.ConfigOptions.key;
  */
 public class MesosTaskManagerParameters {
 
+	/** Pattern replaced in the {@link #MESOS_TM_HOSTNAME} by the actual task id of the Mesos task */
+	public static final Pattern TASK_ID_PATTERN = Pattern.compile("_TASK_", Pattern.LITERAL);
+
 	public static final ConfigOption<Integer> MESOS_RM_TASKS_SLOTS =
-			key(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS)
-			.defaultValue(1);
+		key(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS)
+		.defaultValue(1);
 
 	public static final ConfigOption<Integer> MESOS_RM_TASKS_MEMORY_MB =
-			key("mesos.resourcemanager.tasks.mem")
-			.defaultValue(1024);
+		key("mesos.resourcemanager.tasks.mem")
+		.defaultValue(1024);
 
 	public static final ConfigOption<Double> MESOS_RM_TASKS_CPUS =
-			key("mesos.resourcemanager.tasks.cpus")
-			.defaultValue(0.0);
+		key("mesos.resourcemanager.tasks.cpus")
+		.defaultValue(0.0);
 
 	public static final ConfigOption<String> MESOS_RM_CONTAINER_TYPE =
 		key("mesos.resourcemanager.tasks.container.type")
-			.defaultValue("mesos");
+		.defaultValue("mesos");
 
 	public static final ConfigOption<String> MESOS_RM_CONTAINER_IMAGE_NAME =
 		key("mesos.resourcemanager.tasks.container.image.name")
-			.noDefaultValue();
+		.noDefaultValue();
 
 	public static final ConfigOption<String> MESOS_TM_HOSTNAME =
-			key(ConfigConstants.MESOS_RESOURCEMANAGER_TASKS_HOSTNAME)
-			.noDefaultValue();
+		key("mesos.resourcemanager.tasks.hostname")
+		.noDefaultValue();
 
 	public static final ConfigOption<String> MESOS_TM_BOOTSTRAP_CMD =
-			key(ConfigConstants.MESOS_RESOURCEMANAGER_TASKS_BOOTSTRAP_CMD)
-			.noDefaultValue();
+		key("mesos.resourcemanager.tasks.bootstrap-cmd")
+		.noDefaultValue();
 	
 	public static final ConfigOption<String> MESOS_RM_CONTAINER_VOLUMES =
 		key("mesos.resourcemanager.tasks.container.volumes")
@@ -78,7 +82,7 @@ public class MesosTaskManagerParameters {
 	
 	public static final ConfigOption<String> MESOS_CONSTRAINTS_HARD_HOSTATTR =
 		key("mesos.constraints.hard.hostattribute")
-			.noDefaultValue();
+		.noDefaultValue();
 
 	/**
 	 * Value for {@code MESOS_RESOURCEMANAGER_TASKS_CONTAINER_TYPE} setting. Tells to use the Mesos containerizer.
@@ -237,7 +241,7 @@ public class MesosTaskManagerParameters {
 		List<Protos.Volume> containerVolumes = buildVolumes(containerVolOpt);
 
 		//obtain Task Manager Host Name from the configuration
-		Option<String> tmHostname = Option.apply(flinkConfig.getString(MESOS_TM_HOSTNAME));
+		Option<String> taskManagerHostname = Option.apply(flinkConfig.getString(MESOS_TM_HOSTNAME));
 
 		//obtain bootstrap command from the configuration
 		Option<String> tmBootstrapCommand = Option.apply(flinkConfig.getString(MESOS_TM_BOOTSTRAP_CMD));
@@ -250,7 +254,7 @@ public class MesosTaskManagerParameters {
 			containerVolumes,
 			constraints,
 			tmBootstrapCommand,
-			tmHostname);
+			taskManagerHostname);
 	}
 
 	private static List<ConstraintEvaluator> parseConstraints(String mesosConstraints) {


[08/16] flink git commit: [FLINK-6078] Remove CuratorFramework#close calls from ZooKeeper based HA services

Posted by tr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/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 195baa1..c8459e7 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
@@ -18,7 +18,10 @@
 
 package org.apache.flink.runtime.jobmanager;
 
-import akka.actor.*;
+import akka.actor.ActorRef;
+import akka.actor.ActorSystem;
+import akka.actor.PoisonPill;
+import akka.actor.Status;
 import akka.testkit.JavaTestKit;
 import akka.testkit.TestProbe;
 import com.typesafe.config.Config;
@@ -42,6 +45,7 @@ import org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils;
 import org.apache.flink.runtime.executiongraph.ExecutionVertex;
 import org.apache.flink.runtime.executiongraph.IntermediateResultPartition;
 import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
+import org.apache.flink.runtime.highavailability.nonha.embedded.EmbeddedHaServices;
 import org.apache.flink.runtime.instance.ActorGateway;
 import org.apache.flink.runtime.instance.AkkaActorGateway;
 import org.apache.flink.runtime.instance.HardwareDescription;
@@ -55,8 +59,6 @@ import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings;
 import org.apache.flink.runtime.jobgraph.tasks.ExternalizedCheckpointSettings;
 import org.apache.flink.runtime.jobgraph.tasks.JobCheckpointingSettings;
 import org.apache.flink.runtime.jobmanager.JobManagerHARecoveryTest.BlockingStatefulInvokable;
-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.CancelJob;
 import org.apache.flink.runtime.messages.JobManagerMessages.CancellationFailure;
@@ -95,16 +97,15 @@ import org.apache.flink.runtime.testingUtils.TestingUtils;
 import org.apache.flink.runtime.testtasks.BlockingNoOpInvokable;
 import org.apache.flink.runtime.testtasks.NoOpInvokable;
 import org.apache.flink.runtime.testutils.StoppableInvokable;
+import org.apache.flink.runtime.util.LeaderRetrievalUtils;
 import org.apache.flink.util.TestLogger;
-import org.hamcrest.BaseMatcher;
-import org.hamcrest.Description;
-import org.hamcrest.Matcher;
+import org.junit.After;
 import org.junit.AfterClass;
+import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.TemporaryFolder;
-import org.mockito.ArgumentCaptor;
 import scala.Option;
 import scala.Some;
 import scala.Tuple2;
@@ -117,6 +118,7 @@ import scala.reflect.ClassTag$;
 import java.io.File;
 import java.net.InetAddress;
 import java.util.Collections;
+import java.util.UUID;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 
@@ -127,15 +129,14 @@ import static org.apache.flink.runtime.testingUtils.TestingJobManagerMessages.Al
 import static org.apache.flink.runtime.testingUtils.TestingJobManagerMessages.JobStatusIs;
 import static org.apache.flink.runtime.testingUtils.TestingJobManagerMessages.NotifyWhenAtLeastNumTaskManagerAreRegistered;
 import static org.apache.flink.runtime.testingUtils.TestingUtils.DEFAULT_AKKA_ASK_TIMEOUT;
+import static org.apache.flink.runtime.testingUtils.TestingUtils.TESTING_TIMEOUT;
 import static org.apache.flink.runtime.testingUtils.TestingUtils.startTestingCluster;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
-import static org.mockito.Matchers.argThat;
-import static org.mockito.Matchers.eq;
-import static org.mockito.Mockito.*;
+import static org.mockito.Mockito.mock;
 
 public class JobManagerTest extends TestLogger {
 
@@ -144,6 +145,8 @@ public class JobManagerTest extends TestLogger {
 
 	private static ActorSystem system;
 
+	private HighAvailabilityServices highAvailabilityServices;
+
 	@BeforeClass
 	public static void setup() {
 		system = AkkaUtils.createLocalActorSystem(new Configuration());
@@ -154,6 +157,17 @@ public class JobManagerTest extends TestLogger {
 		JavaTestKit.shutdownActorSystem(system);
 	}
 
+	@Before
+	public void setupTest() {
+		highAvailabilityServices = new EmbeddedHaServices(TestingUtils.defaultExecutor());
+	}
+
+	@After
+	public void tearDownTest() throws Exception {
+		highAvailabilityServices.closeAndCleanupAllData();
+		highAvailabilityServices = null;
+	}
+
 	@Test
 	public void testNullHostnameGoesToLocalhost() {
 		try {
@@ -589,32 +603,36 @@ public class JobManagerTest extends TestLogger {
 		Configuration config = new Configuration();
 		config.setString(ConfigConstants.AKKA_ASK_TIMEOUT, "100ms");
 
+		ActorRef jobManagerActor = JobManager.startJobManagerActors(
+			config,
+			system,
+			TestingUtils.defaultExecutor(),
+			TestingUtils.defaultExecutor(),
+			highAvailabilityServices,
+			TestingJobManager.class,
+			MemoryArchivist.class)._1();
+
+		UUID leaderId = LeaderRetrievalUtils.retrieveLeaderSessionId(
+			highAvailabilityServices.getJobManagerLeaderRetriever(HighAvailabilityServices.DEFAULT_JOB_ID),
+			TestingUtils.TESTING_TIMEOUT());
+
 		ActorGateway jobManager = new AkkaActorGateway(
-				JobManager.startJobManagerActors(
-					config,
-					system,
-					TestingUtils.defaultExecutor(),
-					TestingUtils.defaultExecutor(),
-					TestingJobManager.class,
-					MemoryArchivist.class)._1(),
-				HighAvailabilityServices.DEFAULT_LEADER_ID);
-
-		LeaderRetrievalService leaderRetrievalService = new StandaloneLeaderRetrievalService(
-				AkkaUtils.getAkkaURL(system, jobManager.actor()));
+				jobManagerActor,
+				leaderId);
 
 		Configuration tmConfig = new Configuration();
 		tmConfig.setLong(TaskManagerOptions.MANAGED_MEMORY_SIZE, 4L);
 		tmConfig.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 8);
 
 		ActorRef taskManager = TaskManager.startTaskManagerComponentsAndActor(
-				tmConfig,
-				ResourceID.generate(),
-				system,
-				"localhost",
-				scala.Option.<String>empty(),
-				scala.Option.apply(leaderRetrievalService),
-				true,
-				TestingTaskManager.class);
+			tmConfig,
+			ResourceID.generate(),
+			system,
+			highAvailabilityServices,
+			"localhost",
+			scala.Option.<String>empty(),
+			true,
+			TestingTaskManager.class);
 
 		Future<Object> registrationFuture = jobManager
 				.ask(new NotifyWhenAtLeastNumTaskManagerAreRegistered(1), deadline.timeLeft());
@@ -783,6 +801,7 @@ public class JobManagerTest extends TestLogger {
 		// Wait for failure
 		JobStatusIs jobStatus = Await.result(failedFuture, deadline.timeLeft());
 		assertEquals(JobStatus.FAILED, jobStatus.state());
+
 	}
 
 	@Test
@@ -805,25 +824,30 @@ public class JobManagerTest extends TestLogger {
 				actorSystem,
 				TestingUtils.defaultExecutor(),
 				TestingUtils.defaultExecutor(),
+				highAvailabilityServices,
 				Option.apply("jm"),
 				Option.apply("arch"),
 				TestingJobManager.class,
 				TestingMemoryArchivist.class);
 
-			jobManager = new AkkaActorGateway(master._1(), HighAvailabilityServices.DEFAULT_LEADER_ID);
-			archiver = new AkkaActorGateway(master._2(), HighAvailabilityServices.DEFAULT_LEADER_ID);
+			UUID leaderId = LeaderRetrievalUtils.retrieveLeaderSessionId(
+				highAvailabilityServices.getJobManagerLeaderRetriever(HighAvailabilityServices.DEFAULT_JOB_ID),
+				TestingUtils.TESTING_TIMEOUT());
+
+			jobManager = new AkkaActorGateway(master._1(), leaderId);
+			archiver = new AkkaActorGateway(master._2(), leaderId);
 
 			ActorRef taskManagerRef = TaskManager.startTaskManagerComponentsAndActor(
-					config,
-					ResourceID.generate(),
-					actorSystem,
-					"localhost",
-					Option.apply("tm"),
-					Option.<LeaderRetrievalService>apply(new StandaloneLeaderRetrievalService(jobManager.path())),
-					true,
-					TestingTaskManager.class);
+				config,
+				ResourceID.generate(),
+				actorSystem,
+				highAvailabilityServices,
+				"localhost",
+				Option.apply("tm"),
+				true,
+				TestingTaskManager.class);
 
-			taskManager = new AkkaActorGateway(taskManagerRef, HighAvailabilityServices.DEFAULT_LEADER_ID);
+			taskManager = new AkkaActorGateway(taskManagerRef, leaderId);
 
 			// Wait until connected
 			Object msg = new TestingTaskManagerMessages.NotifyWhenRegisteredAtJobManager(jobManager.actor());
@@ -907,6 +931,10 @@ public class JobManagerTest extends TestLogger {
 			if (taskManager != null) {
 				taskManager.actor().tell(PoisonPill.getInstance(), ActorRef.noSender());
 			}
+
+			if (actorSystem != null) {
+				actorSystem.awaitTermination(TESTING_TIMEOUT());
+			}
 		}
 	}
 
@@ -931,25 +959,30 @@ public class JobManagerTest extends TestLogger {
 				actorSystem,
 				TestingUtils.defaultExecutor(),
 				TestingUtils.defaultExecutor(),
+				highAvailabilityServices,
 				Option.apply("jm"),
 				Option.apply("arch"),
 				TestingJobManager.class,
 				TestingMemoryArchivist.class);
 
-			jobManager = new AkkaActorGateway(master._1(), HighAvailabilityServices.DEFAULT_LEADER_ID);
-			archiver = new AkkaActorGateway(master._2(), HighAvailabilityServices.DEFAULT_LEADER_ID);
+			UUID leaderId = LeaderRetrievalUtils.retrieveLeaderSessionId(
+				highAvailabilityServices.getJobManagerLeaderRetriever(HighAvailabilityServices.DEFAULT_JOB_ID),
+				TestingUtils.TESTING_TIMEOUT());
+
+			jobManager = new AkkaActorGateway(master._1(), leaderId);
+			archiver = new AkkaActorGateway(master._2(), leaderId);
 
 			ActorRef taskManagerRef = TaskManager.startTaskManagerComponentsAndActor(
 				config,
 				ResourceID.generate(),
 				actorSystem,
+				highAvailabilityServices,
 				"localhost",
 				Option.apply("tm"),
-				Option.<LeaderRetrievalService>apply(new StandaloneLeaderRetrievalService(jobManager.path())),
 				true,
 				TestingTaskManager.class);
 
-			taskManager = new AkkaActorGateway(taskManagerRef, HighAvailabilityServices.DEFAULT_LEADER_ID);
+			taskManager = new AkkaActorGateway(taskManagerRef, leaderId);
 
 			// Wait until connected
 			Object msg = new TestingTaskManagerMessages.NotifyWhenRegisteredAtJobManager(jobManager.actor());
@@ -1037,25 +1070,30 @@ public class JobManagerTest extends TestLogger {
 				actorSystem,
 				TestingUtils.defaultExecutor(),
 				TestingUtils.defaultExecutor(),
+				highAvailabilityServices,
 				Option.apply("jm"),
 				Option.apply("arch"),
 				TestingJobManager.class,
 				TestingMemoryArchivist.class);
 
-			jobManager = new AkkaActorGateway(master._1(), HighAvailabilityServices.DEFAULT_LEADER_ID);
-			archiver = new AkkaActorGateway(master._2(), HighAvailabilityServices.DEFAULT_LEADER_ID);
+			UUID leaderId = LeaderRetrievalUtils.retrieveLeaderSessionId(
+				highAvailabilityServices.getJobManagerLeaderRetriever(HighAvailabilityServices.DEFAULT_JOB_ID),
+				TestingUtils.TESTING_TIMEOUT());
+
+			jobManager = new AkkaActorGateway(master._1(), leaderId);
+			archiver = new AkkaActorGateway(master._2(), leaderId);
 
 			ActorRef taskManagerRef = TaskManager.startTaskManagerComponentsAndActor(
-					config,
-					ResourceID.generate(),
-					actorSystem,
-					"localhost",
-					Option.apply("tm"),
-					Option.<LeaderRetrievalService>apply(new StandaloneLeaderRetrievalService(jobManager.path())),
-					true,
-					TestingTaskManager.class);
+				config,
+				ResourceID.generate(),
+				actorSystem,
+				highAvailabilityServices,
+				"localhost",
+				Option.apply("tm"),
+				true,
+				TestingTaskManager.class);
 
-			taskManager = new AkkaActorGateway(taskManagerRef, HighAvailabilityServices.DEFAULT_LEADER_ID);
+			taskManager = new AkkaActorGateway(taskManagerRef, leaderId);
 
 			// Wait until connected
 			Object msg = new TestingTaskManagerMessages.NotifyWhenRegisteredAtJobManager(jobManager.actor());
@@ -1115,6 +1153,10 @@ public class JobManagerTest extends TestLogger {
 			if (taskManager != null) {
 				taskManager.actor().tell(PoisonPill.getInstance(), ActorRef.noSender());
 			}
+
+			if (actorSystem != null) {
+				actorSystem.awaitTermination(TestingUtils.TESTING_TIMEOUT());
+			}
 		}
 	}
 
@@ -1137,28 +1179,33 @@ public class JobManagerTest extends TestLogger {
 				actorSystem,
 				TestingUtils.defaultExecutor(),
 				TestingUtils.defaultExecutor(),
+				highAvailabilityServices,
 				Option.apply("jm"),
 				Option.apply("arch"),
 				TestingJobManager.class,
 				TestingMemoryArchivist.class);
 
-			jobManager = new AkkaActorGateway(master._1(), HighAvailabilityServices.DEFAULT_LEADER_ID);
-			archiver = new AkkaActorGateway(master._2(), HighAvailabilityServices.DEFAULT_LEADER_ID);
+			UUID leaderId = LeaderRetrievalUtils.retrieveLeaderSessionId(
+				highAvailabilityServices.getJobManagerLeaderRetriever(HighAvailabilityServices.DEFAULT_JOB_ID),
+				TestingUtils.TESTING_TIMEOUT());
+
+			jobManager = new AkkaActorGateway(master._1(), leaderId);
+			archiver = new AkkaActorGateway(master._2(), leaderId);
 
 			Configuration tmConfig = new Configuration();
 			tmConfig.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 4);
 
 			ActorRef taskManagerRef = TaskManager.startTaskManagerComponentsAndActor(
-					tmConfig,
-					ResourceID.generate(),
-					actorSystem,
-					"localhost",
-					Option.apply("tm"),
-					Option.<LeaderRetrievalService>apply(new StandaloneLeaderRetrievalService(jobManager.path())),
-					true,
-					TestingTaskManager.class);
+				tmConfig,
+				ResourceID.generate(),
+				actorSystem,
+				highAvailabilityServices,
+				"localhost",
+				Option.apply("tm"),
+				true,
+				TestingTaskManager.class);
 
-			taskManager = new AkkaActorGateway(taskManagerRef, HighAvailabilityServices.DEFAULT_LEADER_ID);
+			taskManager = new AkkaActorGateway(taskManagerRef, leaderId);
 
 			// Wait until connected
 			Object msg = new TestingTaskManagerMessages.NotifyWhenRegisteredAtJobManager(jobManager.actor());
@@ -1274,6 +1321,10 @@ public class JobManagerTest extends TestLogger {
 			if (taskManager != null) {
 				taskManager.actor().tell(PoisonPill.getInstance(), ActorRef.noSender());
 			}
+
+			if (actorSystem != null) {
+				actorSystem.awaitTermination(TestingUtils.TESTING_TIMEOUT());
+			}
 		}
 	}
 
@@ -1298,7 +1349,8 @@ public class JobManagerTest extends TestLogger {
 				actorSystem,
 				TestingUtils.defaultExecutor(),
 				TestingUtils.defaultExecutor(),
-				configuration);
+				configuration,
+				highAvailabilityServices);
 
 			final TestProbe probe = TestProbe.apply(actorSystem);
 			final AkkaActorGateway rmGateway = new AkkaActorGateway(probe.ref(), HighAvailabilityServices.DEFAULT_LEADER_ID);
@@ -1311,7 +1363,7 @@ public class JobManagerTest extends TestLogger {
 
 			JobManagerMessages.LeaderSessionMessage leaderSessionMessage = probe.expectMsgClass(JobManagerMessages.LeaderSessionMessage.class);
 
-			assertEquals(HighAvailabilityServices.DEFAULT_LEADER_ID, leaderSessionMessage.leaderSessionID());
+			assertEquals(jmGateway.leaderSessionID(), leaderSessionMessage.leaderSessionID());
 			assertTrue(leaderSessionMessage.message() instanceof RegisterResourceManagerSuccessful);
 
 			jmGateway.tell(

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/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 fcca173..37f503f 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
@@ -26,6 +26,8 @@ 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.highavailability.HighAvailabilityServices;
+import org.apache.flink.runtime.highavailability.nonha.embedded.EmbeddedHaServices;
 import org.apache.flink.runtime.instance.ActorGateway;
 import org.apache.flink.runtime.jobgraph.JobGraph;
 import org.apache.flink.runtime.jobgraph.JobVertex;
@@ -68,6 +70,7 @@ public class JobSubmitTest {
 	private static ActorSystem jobManagerSystem;
 	private static ActorGateway jmGateway;
 	private static Configuration jmConfig;
+	private static HighAvailabilityServices highAvailabilityServices;
 
 	@BeforeClass
 	public static void setupJobManager() {
@@ -81,17 +84,20 @@ public class JobSubmitTest {
 		scala.Option<Tuple2<String, Object>> listeningAddress = scala.Option.apply(new Tuple2<String, Object>("localhost", port));
 		jobManagerSystem = AkkaUtils.createActorSystem(jmConfig, listeningAddress);
 
+		highAvailabilityServices = new EmbeddedHaServices(TestingUtils.defaultExecutor());
+
 		// only start JobManager (no ResourceManager)
 		JobManager.startJobManagerActors(
 			jmConfig,
 			jobManagerSystem,
 			TestingUtils.defaultExecutor(),
 			TestingUtils.defaultExecutor(),
+			highAvailabilityServices,
 			JobManager.class,
 			MemoryArchivist.class)._1();
 
 		try {
-			LeaderRetrievalService lrs = LeaderRetrievalUtils.createLeaderRetrievalService(jmConfig, false);
+			LeaderRetrievalService lrs = highAvailabilityServices.getJobManagerLeaderRetriever(HighAvailabilityServices.DEFAULT_JOB_ID);
 
 			jmGateway = LeaderRetrievalUtils.retrieveLeaderGateway(
 					lrs,
@@ -104,10 +110,15 @@ public class JobSubmitTest {
 	}
 
 	@AfterClass
-	public static void teardownJobmanager() {
+	public static void teardownJobmanager() throws Exception {
 		if (jobManagerSystem != null) {
 			jobManagerSystem.shutdown();
 		}
+
+		if (highAvailabilityServices != null) {
+			highAvailabilityServices.closeAndCleanupAllData();
+			highAvailabilityServices = null;
+		}
 	}
 
 	@Test

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobManagerRunnerMockTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobManagerRunnerMockTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobManagerRunnerMockTest.java
index d2221c5..753c797 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobManagerRunnerMockTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobManagerRunnerMockTest.java
@@ -35,6 +35,7 @@ import org.apache.flink.runtime.metrics.MetricRegistry;
 import org.apache.flink.runtime.metrics.MetricRegistryConfiguration;
 import org.apache.flink.runtime.rpc.FatalErrorHandler;
 import org.apache.flink.runtime.rpc.RpcService;
+import org.apache.flink.util.TestLogger;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Ignore;
@@ -56,7 +57,7 @@ import static org.mockito.Mockito.when;
 
 @RunWith(PowerMockRunner.class)
 @PrepareForTest(JobManagerRunner.class)
-public class JobManagerRunnerMockTest {
+public class JobManagerRunnerMockTest extends TestLogger {
 
 	private JobManagerRunner runner;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderChangeJobRecoveryTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderChangeJobRecoveryTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderChangeJobRecoveryTest.java
index b1bb548..942fcf3 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderChangeJobRecoveryTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderChangeJobRecoveryTest.java
@@ -18,9 +18,13 @@
 
 package org.apache.flink.runtime.leaderelection;
 
+import org.apache.flink.api.common.JobID;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.executiongraph.ExecutionGraph;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
+import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices;
+import org.apache.flink.runtime.highavailability.TestingManualHighAvailabilityServices;
 import org.apache.flink.runtime.instance.ActorGateway;
 import org.apache.flink.runtime.io.network.partition.ResultPartitionType;
 import org.apache.flink.runtime.jobgraph.DistributionPattern;
@@ -28,6 +32,7 @@ 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.testingUtils.TestingCluster;
 import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages;
 import org.apache.flink.util.TestLogger;
 
@@ -52,11 +57,15 @@ public class LeaderChangeJobRecoveryTest extends TestLogger {
 	private int numSlotsPerTM = 1;
 	private int parallelism = numTMs * numSlotsPerTM;
 
-	private LeaderElectionRetrievalTestingCluster cluster = null;
+	private JobID jobId;
+	private TestingCluster cluster = null;
 	private JobGraph job = createBlockingJob(parallelism);
+	private TestingManualHighAvailabilityServices highAvailabilityServices;
 
 	@Before
 	public void before() throws TimeoutException, InterruptedException {
+		jobId = HighAvailabilityServices.DEFAULT_JOB_ID;
+
 		Tasks.BlockingOnceReceiver$.MODULE$.blocking_$eq(true);
 
 		Configuration configuration = new Configuration();
@@ -69,7 +78,13 @@ public class LeaderChangeJobRecoveryTest extends TestLogger {
 		configuration.setInteger(ConfigConstants.RESTART_STRATEGY_FIXED_DELAY_ATTEMPTS, 9999);
 		configuration.setString(ConfigConstants.RESTART_STRATEGY_FIXED_DELAY_DELAY, "100 milli");
 
-		cluster = new LeaderElectionRetrievalTestingCluster(configuration, true, false);
+		highAvailabilityServices = new TestingManualHighAvailabilityServices();
+
+		cluster = new TestingCluster(
+			configuration,
+			highAvailabilityServices,
+			true,
+			false);
 		cluster.start(false);
 
 		// wait for actors to be alive so that they have started their leader retrieval service
@@ -86,8 +101,15 @@ public class LeaderChangeJobRecoveryTest extends TestLogger {
 	public void testNotRestartedWhenLosingLeadership() throws Exception {
 		UUID leaderSessionID = UUID.randomUUID();
 
-		cluster.grantLeadership(0, leaderSessionID);
-		cluster.notifyRetrievalListeners(0, leaderSessionID);
+		highAvailabilityServices.grantLeadership(
+			jobId,
+			0,
+			leaderSessionID);
+
+		highAvailabilityServices.notifyRetrievers(
+			jobId,
+			0,
+			leaderSessionID);
 
 		cluster.waitForTaskManagersToBeRegistered(timeout);
 
@@ -108,7 +130,7 @@ public class LeaderChangeJobRecoveryTest extends TestLogger {
 
 		ExecutionGraph executionGraph = (ExecutionGraph) ((TestingJobManagerMessages.ExecutionGraphFound) responseExecutionGraph).executionGraph();
 
-		cluster.revokeLeadership();
+		highAvailabilityServices.revokeLeadership(jobId);
 
 		executionGraph.getTerminationFuture().get(30, TimeUnit.SECONDS);
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/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
index 7ae9974..bbcbbf0 100644
--- 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
@@ -18,8 +18,11 @@
 
 package org.apache.flink.runtime.leaderelection;
 
+import org.apache.flink.api.common.JobID;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
+import org.apache.flink.runtime.highavailability.TestingManualHighAvailabilityServices;
 import org.apache.flink.runtime.instance.ActorGateway;
 import org.apache.flink.runtime.io.network.partition.ResultPartitionType;
 import org.apache.flink.runtime.jobgraph.DistributionPattern;
@@ -28,6 +31,7 @@ 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.TestingCluster;
 import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages.NotifyWhenJobRemoved;
 import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages.WaitForAllVerticesToBeRunningOrFinished;
 import org.apache.flink.runtime.testingUtils.TestingUtils;
@@ -58,12 +62,16 @@ public class LeaderChangeStateCleanupTest extends TestLogger {
 	private int numSlotsPerTM = 2;
 	private int parallelism = numTMs * numSlotsPerTM;
 
+	private JobID jobId;
 	private Configuration configuration;
-	private LeaderElectionRetrievalTestingCluster cluster = null;
+	private TestingManualHighAvailabilityServices highAvailabilityServices;
+	private TestingCluster cluster = null;
 	private JobGraph job = createBlockingJob(parallelism);
 
 	@Before
 	public void before() throws Exception {
+		jobId = HighAvailabilityServices.DEFAULT_JOB_ID;
+
 		Tasks.BlockingOnceReceiver$.MODULE$.blocking_$eq(true);
 
 		configuration = new Configuration();
@@ -72,7 +80,13 @@ public class LeaderChangeStateCleanupTest extends TestLogger {
 		configuration.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, numTMs);
 		configuration.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, numSlotsPerTM);
 
-		cluster = new LeaderElectionRetrievalTestingCluster(configuration, true, false);
+		highAvailabilityServices = new TestingManualHighAvailabilityServices();
+
+		cluster = new TestingCluster(
+			configuration,
+			highAvailabilityServices,
+			true,
+			false);
 		cluster.start(false); // TaskManagers don't have to register at the JobManager
 
 		cluster.waitForActorsToBeAlive(); // we only wait until all actors are alive
@@ -96,9 +110,9 @@ public class LeaderChangeStateCleanupTest extends TestLogger {
 		UUID leaderSessionID2 = UUID.randomUUID();
 
 		// first make JM(0) the leader
-		cluster.grantLeadership(0, leaderSessionID1);
+		highAvailabilityServices.grantLeadership(jobId, 0, leaderSessionID1);
 		// notify all listeners
-		cluster.notifyRetrievalListeners(0, leaderSessionID1);
+		highAvailabilityServices.notifyRetrievers(jobId, 0, leaderSessionID1);
 
 		cluster.waitForTaskManagersToBeRegistered(timeout);
 
@@ -114,9 +128,9 @@ public class LeaderChangeStateCleanupTest extends TestLogger {
 		Future<Object> jobRemoval = jm.ask(new NotifyWhenJobRemoved(job.getJobID()), timeout);
 
 		// make the JM(1) the new leader
-		cluster.grantLeadership(1, leaderSessionID2);
+		highAvailabilityServices.grantLeadership(jobId, 1, leaderSessionID2);
 		// notify all listeners about the event
-		cluster.notifyRetrievalListeners(1, leaderSessionID2);
+		highAvailabilityServices.notifyRetrievers(jobId, 1, leaderSessionID2);
 
 		Await.ready(jobRemoval, timeout);
 
@@ -133,7 +147,7 @@ public class LeaderChangeStateCleanupTest extends TestLogger {
 
 		// try to resubmit now the non-blocking job, it should complete successfully
 		Tasks.BlockingOnceReceiver$.MODULE$.blocking_$eq(false);
-		cluster.submitJobAndWait(job, false, timeout, new TestingLeaderRetrievalService(jm2.path(), jm2.leaderSessionID()));
+		cluster.submitJobAndWait(job, false, timeout);
 	}
 
 	/**
@@ -146,8 +160,8 @@ public class LeaderChangeStateCleanupTest extends TestLogger {
 		UUID leaderSessionID = UUID.randomUUID();
 		UUID newLeaderSessionID = UUID.randomUUID();
 
-		cluster.grantLeadership(0, leaderSessionID);
-		cluster.notifyRetrievalListeners(0, leaderSessionID);
+		highAvailabilityServices.grantLeadership(jobId, 0, leaderSessionID);
+		highAvailabilityServices.notifyRetrievers(jobId, 0, leaderSessionID);
 
 		cluster.waitForTaskManagersToBeRegistered(timeout);
 
@@ -163,7 +177,7 @@ public class LeaderChangeStateCleanupTest extends TestLogger {
 		Future<Object> jobRemoval = jm.ask(new NotifyWhenJobRemoved(job.getJobID()), timeout);
 
 		// only notify the JMs about the new leader JM(1)
-		cluster.grantLeadership(1, newLeaderSessionID);
+		highAvailabilityServices.grantLeadership(jobId, 1, newLeaderSessionID);
 
 		// job should be removed anyway
 		Await.ready(jobRemoval, timeout);
@@ -179,8 +193,8 @@ public class LeaderChangeStateCleanupTest extends TestLogger {
 		UUID leaderSessionID = UUID.randomUUID();
 		UUID newLeaderSessionID = UUID.randomUUID();
 
-		cluster.grantLeadership(0, leaderSessionID);
-		cluster.notifyRetrievalListeners(0, leaderSessionID);
+		highAvailabilityServices.grantLeadership(jobId, 0, leaderSessionID);
+		highAvailabilityServices.notifyRetrievers(jobId, 0, leaderSessionID);
 
 		cluster.waitForTaskManagersToBeRegistered(timeout);
 
@@ -196,7 +210,7 @@ public class LeaderChangeStateCleanupTest extends TestLogger {
 		Future<Object> jobRemoval = jm.ask(new NotifyWhenJobRemoved(job.getJobID()), timeout);
 
 		// notify listeners (TMs) about the leader change
-		cluster.notifyRetrievalListeners(1, newLeaderSessionID);
+		highAvailabilityServices.notifyRetrievers(jobId, 1, newLeaderSessionID);
 
 		Await.ready(jobRemoval, timeout);
 	}
@@ -213,8 +227,8 @@ public class LeaderChangeStateCleanupTest extends TestLogger {
 
 		FiniteDuration shortTimeout = new FiniteDuration(10, TimeUnit.SECONDS);
 
-		cluster.grantLeadership(0, leaderSessionID);
-		cluster.notifyRetrievalListeners(0, leaderSessionID);
+		highAvailabilityServices.grantLeadership(jobId, 0, leaderSessionID);
+		highAvailabilityServices.notifyRetrievers(jobId, 0, leaderSessionID);
 
 		cluster.waitForTaskManagersToBeRegistered(timeout);
 
@@ -232,7 +246,7 @@ public class LeaderChangeStateCleanupTest extends TestLogger {
 		LOG.info("Make JM(0) again the leader. This should first revoke the leadership.");
 
 		// make JM(0) again the leader --> this implies first a leadership revocation
-		cluster.grantLeadership(0, newLeaderSessionID);
+		highAvailabilityServices.grantLeadership(jobId, 0, newLeaderSessionID);
 
 		Await.ready(jobRemoval, timeout);
 
@@ -250,7 +264,7 @@ public class LeaderChangeStateCleanupTest extends TestLogger {
 		LOG.info("Notify TMs about the new (old) leader.");
 
 		// notify the TMs about the new (old) leader
-		cluster.notifyRetrievalListeners(0, newLeaderSessionID);
+		highAvailabilityServices.notifyRetrievers(jobId,0, newLeaderSessionID);
 
 		cluster.waitForTaskManagersToBeRegistered(timeout);
 
@@ -258,7 +272,7 @@ public class LeaderChangeStateCleanupTest extends TestLogger {
 
 		// try to resubmit now the non-blocking job, it should complete successfully
 		Tasks.BlockingOnceReceiver$.MODULE$.blocking_$eq(false);
-		cluster.submitJobAndWait(job, false, timeout, new TestingLeaderRetrievalService(leaderGateway.path(), leaderGateway.leaderSessionID()));
+		cluster.submitJobAndWait(job, false, timeout);
 	}
 
 	public JobGraph createBlockingJob(int parallelism) {

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/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
deleted file mode 100644
index 1cab0ea..0000000
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderElectionRetrievalTestingCluster.java
+++ /dev/null
@@ -1,121 +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.leaderelection;
-
-import org.apache.flink.configuration.ConfigConstants;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
-import org.apache.flink.runtime.testingUtils.TestingCluster;
-import scala.Option;
-
-import java.util.ArrayList;
-import java.util.List;
-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;
-
-	public List<TestingLeaderElectionService> leaderElectionServices;
-	public List<TestingLeaderRetrievalService> leaderRetrievalServices;
-
-	private int leaderIndex = -1;
-
-	public LeaderElectionRetrievalTestingCluster(
-			Configuration userConfiguration,
-			boolean singleActorSystem,
-			boolean synchronousDispatcher) {
-		super(userConfiguration, singleActorSystem, synchronousDispatcher);
-
-		this.userConfiguration = userConfiguration;
-		this.useSingleActorSystem = singleActorSystem;
-
-		leaderElectionServices = new ArrayList<>();
-		leaderRetrievalServices = new ArrayList<>();
-	}
-
-	@Override
-	public Configuration userConfiguration() {
-		return this.userConfiguration;
-	}
-
-	@Override
-	public boolean useSingleActorSystem() {
-		return useSingleActorSystem;
-	}
-
-	@Override
-	public Option<LeaderElectionService> createLeaderElectionService() {
-		leaderElectionServices.add(new TestingLeaderElectionService());
-
-		LeaderElectionService result = leaderElectionServices.get(leaderElectionServices.size() - 1);
-
-		return Option.apply(result);
-	}
-
-	@Override
-	public LeaderRetrievalService createLeaderRetrievalService() {
-		leaderRetrievalServices.add(new TestingLeaderRetrievalService(
-			null,
-			null));
-
-		return leaderRetrievalServices.get(leaderRetrievalServices.size() - 1);
-	}
-
-	@Override
-	public int getNumberOfJobManagers() {
-		return this.originalConfiguration().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.get(leaderIndex).notLeader();
-		}
-
-		// make the JM with index the new leader
-		leaderElectionServices.get(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);
-		}
-	}
-
-	public void revokeLeadership() {
-		if (leaderIndex >= 0) {
-			leaderElectionServices.get(leaderIndex).notLeader();
-			leaderIndex = -1;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/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
index 16779fa..d456083 100644
--- 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
@@ -63,4 +63,8 @@ public class TestingLeaderElectionService implements LeaderElectionService {
 		contender = null;
 		hasLeadership  = false;
 	}
+
+	public String getAddress() {
+		return contender.getAddress();
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/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
index 887772a..15d3bde 100644
--- 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
@@ -35,6 +35,10 @@ public class TestingLeaderRetrievalService implements LeaderRetrievalService {
 
 	private volatile LeaderRetrievalListener listener;
 
+	public TestingLeaderRetrievalService() {
+		this(null, null);
+	}
+
 	public TestingLeaderRetrievalService(String leaderAddress, UUID leaderSessionID) {
 		this.leaderAddress = leaderAddress;
 		this.leaderSessionID = leaderSessionID;

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/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
index bc8c0b60..6efd270 100644
--- 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
@@ -20,7 +20,6 @@ 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;
@@ -37,6 +36,7 @@ import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 import org.mockito.Matchers;
+import org.mockito.Mockito;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 import org.slf4j.Logger;
@@ -48,6 +48,7 @@ import scala.concurrent.duration.Deadline;
 import scala.concurrent.duration.FiniteDuration;
 
 import java.io.ByteArrayOutputStream;
+import java.io.IOException;
 import java.io.ObjectOutputStream;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
@@ -59,6 +60,11 @@ import static org.junit.Assert.*;
 
 public class ZooKeeperLeaderElectionTest extends TestLogger {
 	private TestingServer testingServer;
+
+	private Configuration configuration;
+
+	private CuratorFramework client;
+
 	private static final String TEST_URL = "akka//user/jobmanager";
 	private static final FiniteDuration timeout = new FiniteDuration(200, TimeUnit.SECONDS);
 
@@ -71,17 +77,26 @@ public class ZooKeeperLeaderElectionTest extends TestLogger {
 		} catch (Exception e) {
 			throw new RuntimeException("Could not start ZooKeeper testing cluster.", e);
 		}
+
+		configuration = new Configuration();
+
+		configuration.setString(HighAvailabilityOptions.HA_ZOOKEEPER_QUORUM, testingServer.getConnectString());
+		configuration.setString(HighAvailabilityOptions.HA_MODE, "zookeeper");
+
+		client = ZooKeeperUtils.startCuratorFramework(configuration);
 	}
 
 	@After
-	public void after() {
-		try {
-			testingServer.stop();
-		} catch (Exception e) {
-			throw new RuntimeException("Could not stop ZooKeeper testing cluster.", e);
+	public void after() throws IOException {
+		if (client != null) {
+			client.close();
+			client = null;
 		}
 
-		testingServer = null;
+		if (testingServer != null) {
+			testingServer.stop();
+			testingServer = null;
+		}
 	}
 
 	/**
@@ -89,16 +104,12 @@ public class ZooKeeperLeaderElectionTest extends TestLogger {
 	 */
 	@Test
 	public void testZooKeeperLeaderElectionRetrieval() throws Exception {
-		Configuration configuration = new Configuration();
-		configuration.setString(HighAvailabilityOptions.HA_ZOOKEEPER_QUORUM, testingServer.getConnectString());
-		configuration.setString(HighAvailabilityOptions.HA_MODE, "zookeeper");
-
 		ZooKeeperLeaderElectionService leaderElectionService = null;
 		ZooKeeperLeaderRetrievalService leaderRetrievalService = null;
 
 		try {
-			leaderElectionService = ZooKeeperUtils.createLeaderElectionService(configuration);
-			leaderRetrievalService = ZooKeeperUtils.createLeaderRetrievalService(configuration);
+			leaderElectionService = ZooKeeperUtils.createLeaderElectionService(client, configuration);
+			leaderRetrievalService = ZooKeeperUtils.createLeaderRetrievalService(client, configuration);
 
 			TestingContender contender = new TestingContender(TEST_URL, leaderElectionService);
 			TestingListener listener = new TestingListener();
@@ -134,10 +145,6 @@ public class ZooKeeperLeaderElectionTest extends TestLogger {
 	 */
 	@Test
 	public void testZooKeeperReelection() throws Exception {
-		Configuration configuration = new Configuration();
-		configuration.setString(HighAvailabilityOptions.HA_ZOOKEEPER_QUORUM, testingServer.getConnectString());
-		configuration.setString(HighAvailabilityOptions.HA_MODE, "zookeeper");
-
 		Deadline deadline = new FiniteDuration(5, TimeUnit.MINUTES).fromNow();
 
 		int num = 10;
@@ -149,14 +156,14 @@ public class ZooKeeperLeaderElectionTest extends TestLogger {
 		TestingListener listener = new TestingListener();
 
 		try {
-			leaderRetrievalService = ZooKeeperUtils.createLeaderRetrievalService(configuration);
+			leaderRetrievalService = ZooKeeperUtils.createLeaderRetrievalService(client, configuration);
 
 			LOG.debug("Start leader retrieval service for the TestingListener.");
 
 			leaderRetrievalService.start(listener);
 
 			for (int i = 0; i < num; i++) {
-				leaderElectionService[i] = ZooKeeperUtils.createLeaderElectionService(configuration);
+				leaderElectionService[i] = ZooKeeperUtils.createLeaderElectionService(client, configuration);
 				contenders[i] = new TestingContender(TEST_URL + "_" + i, leaderElectionService[i]);
 
 				LOG.debug("Start leader election service for contender #{}.", i);
@@ -217,10 +224,6 @@ public class ZooKeeperLeaderElectionTest extends TestLogger {
 	 */
 	@Test
 	public void testZooKeeperReelectionWithReplacement() throws Exception {
-		Configuration configuration = new Configuration();
-		configuration.setString(HighAvailabilityOptions.HA_ZOOKEEPER_QUORUM, testingServer.getConnectString());
-		configuration.setString(HighAvailabilityOptions.HA_MODE, "zookeeper");
-
 		int num = 3;
 		int numTries = 30;
 
@@ -231,12 +234,12 @@ public class ZooKeeperLeaderElectionTest extends TestLogger {
 		TestingListener listener = new TestingListener();
 
 		try {
-			leaderRetrievalService = ZooKeeperUtils.createLeaderRetrievalService(configuration);
+			leaderRetrievalService = ZooKeeperUtils.createLeaderRetrievalService(client, configuration);
 
 			leaderRetrievalService.start(listener);
 
 			for (int i = 0; i < num; i++) {
-				leaderElectionService[i] = ZooKeeperUtils.createLeaderElectionService(configuration);
+				leaderElectionService[i] = ZooKeeperUtils.createLeaderElectionService(client, configuration);
 				contenders[i] = new TestingContender(TEST_URL + "_" + i + "_0", leaderElectionService[i]);
 
 				leaderElectionService[i].start(contenders[i]);
@@ -261,7 +264,7 @@ public class ZooKeeperLeaderElectionTest extends TestLogger {
 					// 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);
+					leaderElectionService[index] = ZooKeeperUtils.createLeaderElectionService(client, configuration);
 					contenders[index] = new TestingContender(
 							TEST_URL + "_" + index + "_" + (lastTry + 1),
 							leaderElectionService[index]);
@@ -295,9 +298,6 @@ public class ZooKeeperLeaderElectionTest extends TestLogger {
 		final String FAULTY_CONTENDER_URL = "faultyContender";
 		final String leaderPath = "/leader";
 
-		Configuration configuration = new Configuration();
-		configuration.setString(HighAvailabilityOptions.HA_ZOOKEEPER_QUORUM, testingServer.getConnectString());
-		configuration.setString(HighAvailabilityOptions.HA_MODE, "zookeeper");
 		configuration.setString(ConfigConstants.HA_ZOOKEEPER_LEADER_PATH, leaderPath);
 
 		ZooKeeperLeaderElectionService leaderElectionService = null;
@@ -308,9 +308,9 @@ public class ZooKeeperLeaderElectionTest extends TestLogger {
 		TestingContender contender;
 
 		try {
-			leaderElectionService = ZooKeeperUtils.createLeaderElectionService(configuration);
-			leaderRetrievalService = ZooKeeperUtils.createLeaderRetrievalService(configuration);
-			leaderRetrievalService2 = ZooKeeperUtils.createLeaderRetrievalService(configuration);
+			leaderElectionService = ZooKeeperUtils.createLeaderElectionService(client, configuration);
+			leaderRetrievalService = ZooKeeperUtils.createLeaderRetrievalService(client, configuration);
+			leaderRetrievalService2 = ZooKeeperUtils.createLeaderRetrievalService(client, configuration);
 
 			contender = new TestingContender(TEST_URL, leaderElectionService);
 
@@ -379,18 +379,13 @@ public class ZooKeeperLeaderElectionTest extends TestLogger {
 	 */
 	@Test
 	public void testExceptionForwarding() throws Exception {
-		Configuration configuration = new Configuration();
-		configuration.setString(HighAvailabilityOptions.HA_ZOOKEEPER_QUORUM, testingServer.getConnectString());
-		configuration.setString(HighAvailabilityOptions.HA_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 CreateBuilder mockCreateBuilder = mock(CreateBuilder.class, Mockito.RETURNS_DEEP_STUBS);
 		final Exception testException = new Exception("Test exception");
 
 		try {
@@ -414,12 +409,14 @@ public class ZooKeeperLeaderElectionTest extends TestLogger {
 
 			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);
+			when(
+				mockCreateBuilder
+				.creatingParentsIfNeeded()
+				.withMode(Matchers.any(CreateMode.class))
+				.forPath(anyString(), any(byte[].class))).thenThrow(testException);
 
 			leaderElectionService = new ZooKeeperLeaderElectionService(client, "/latch", "/leader");
-			leaderRetrievalService = ZooKeeperUtils.createLeaderRetrievalService(configuration);
+			leaderRetrievalService = ZooKeeperUtils.createLeaderRetrievalService(client, configuration);
 
 			testingContender = new TestingContender(TEST_URL, leaderElectionService);
 
@@ -442,34 +439,33 @@ public class ZooKeeperLeaderElectionTest extends TestLogger {
 	}
 
 	/**
-	 * 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
+	 * Tests that there is no information left in the ZooKeeper cluster after the ZooKeeper client
+	 * has terminated. In other words, checks that the ZooKeeperLeaderElection service uses
 	 * ephemeral nodes.
 	 */
 	@Test
 	public void testEphemeralZooKeeperNodes() throws Exception {
-		Configuration configuration = new Configuration();
-		configuration.setString(HighAvailabilityOptions.HA_ZOOKEEPER_QUORUM, testingServer.getConnectString());
-		configuration.setString(HighAvailabilityOptions.HA_MODE, "zookeeper");
-
 		ZooKeeperLeaderElectionService leaderElectionService;
 		ZooKeeperLeaderRetrievalService leaderRetrievalService = null;
 		TestingContender testingContender;
 		TestingListener listener;
 
 		CuratorFramework client = null;
+		CuratorFramework client2 = null;
 		NodeCache cache = null;
 
 		try {
-			leaderElectionService = ZooKeeperUtils.createLeaderElectionService(configuration);
-			leaderRetrievalService = ZooKeeperUtils.createLeaderRetrievalService(configuration);
+			client = ZooKeeperUtils.startCuratorFramework(configuration);
+			client2 = ZooKeeperUtils.startCuratorFramework(configuration);
+
+			leaderElectionService = ZooKeeperUtils.createLeaderElectionService(client, configuration);
+			leaderRetrievalService = ZooKeeperUtils.createLeaderRetrievalService(client2, configuration);
 			testingContender = new TestingContender(TEST_URL, leaderElectionService);
 			listener = new TestingListener();
 
-			client = ZooKeeperUtils.startCuratorFramework(configuration);
 			final String leaderPath = configuration.getString(ConfigConstants.HA_ZOOKEEPER_LEADER_PATH,
 					ConfigConstants.DEFAULT_ZOOKEEPER_LEADER_PATH);
-			cache = new NodeCache(client, leaderPath);
+			cache = new NodeCache(client2, leaderPath);
 
 			ExistsCacheListener existsListener = new ExistsCacheListener(cache);
 			DeletedCacheListener deletedCacheListener = new DeletedCacheListener(cache);
@@ -489,6 +485,9 @@ public class ZooKeeperLeaderElectionTest extends TestLogger {
 
 			leaderElectionService.stop();
 
+			// now stop the underlying client
+			client.close();
+
 			Future<Boolean> deletedFuture = deletedCacheListener.nodeDeleted();
 
 			// make sure that the leader node has been deleted
@@ -497,7 +496,7 @@ public class ZooKeeperLeaderElectionTest extends TestLogger {
 			leaderRetrievalService.start(listener);
 
 			try {
-				listener.waitForNewLeader(1000);
+				listener.waitForNewLeader(1000L);
 
 				fail("TimeoutException was expected because there is no leader registered and " +
 						"thus there shouldn't be any leader information in ZooKeeper.");
@@ -513,8 +512,8 @@ public class ZooKeeperLeaderElectionTest extends TestLogger {
 				cache.close();
 			}
 
-			if (client != null) {
-				client.close();
+			if (client2 != null) {
+				client2.close();
 			}
 		}
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/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
index b79093f..0ea47f2 100644
--- 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
@@ -18,18 +18,17 @@
 
 package org.apache.flink.runtime.leaderelection;
 
-import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.test.TestingServer;
 
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.HighAvailabilityOptions;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
 import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils;
 import org.apache.flink.runtime.jobmaster.JobMaster;
 import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
 import org.apache.flink.runtime.rpc.akka.AkkaRpcServiceUtils;
+import org.apache.flink.runtime.testingUtils.TestingUtils;
 import org.apache.flink.runtime.util.LeaderRetrievalUtils;
-import org.apache.flink.runtime.util.ZooKeeperUtils;
-import org.apache.flink.util.NetUtils;
 import org.apache.flink.util.TestLogger;
 
 import org.junit.After;
@@ -53,29 +52,41 @@ public class ZooKeeperLeaderRetrievalTest extends TestLogger{
 
 	private TestingServer testingServer;
 
+	private Configuration config;
+
+	private HighAvailabilityServices highAvailabilityServices;
+
 	@Before
-	public void before() {
-		try {
-			testingServer = new TestingServer();
-		} catch (Exception e) {
-			throw new RuntimeException("Could not start ZooKeeper testing cluster.", e);
-		}
+	public void before() throws Exception {
+		testingServer = new TestingServer();
+
+		config = new Configuration();
+		config.setString(HighAvailabilityOptions.HA_MODE, "zookeeper");
+		config.setString(HighAvailabilityOptions.HA_ZOOKEEPER_QUORUM, testingServer.getConnectString());
+
+		highAvailabilityServices = HighAvailabilityServicesUtils.createHighAvailabilityServices(
+			config,
+			TestingUtils.defaultExecutor(),
+			HighAvailabilityServicesUtils.AddressResolution.NO_ADDRESS_RESOLUTION);
 	}
 
 	@After
-	public void after() {
+	public void after() throws Exception {
 		if(testingServer != null) {
-			try {
-				testingServer.stop();
-			} catch (IOException e) {
-				throw new RuntimeException("Could not stop ZooKeeper testing cluster.", e);
-			}
+			testingServer.stop();
+
 			testingServer = null;
 		}
+
+		if (highAvailabilityServices != null) {
+			highAvailabilityServices.closeAndCleanupAllData();
+
+			highAvailabilityServices = null;
+		}
 	}
 
 	/**
-	 * Tests that LeaderRetrievalUtils.findConnectingAdress finds the correct connecting address
+	 * Tests that LeaderRetrievalUtils.findConnectingAddress 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.
@@ -83,13 +94,9 @@ public class ZooKeeperLeaderRetrievalTest extends TestLogger{
 	@Test
 	public void testConnectingAddressRetrievalWithDelayedLeaderElection() throws Exception {
 		FiniteDuration timeout = new FiniteDuration(1, TimeUnit.MINUTES);
-		Configuration config = new Configuration();
 
 		long sleepingTime = 1000;
 
-		config.setString(HighAvailabilityOptions.HA_MODE, "zookeeper");
-		config.setString(HighAvailabilityOptions.HA_ZOOKEEPER_QUORUM, testingServer.getConnectString());
-
 		LeaderElectionService leaderElectionService = null;
 		LeaderElectionService faultyLeaderElectionService;
 
@@ -98,12 +105,7 @@ public class ZooKeeperLeaderRetrievalTest extends TestLogger{
 
 		Thread thread;
 
-		CuratorFramework[] client = new CuratorFramework[2];
-
 		try {
-			client[0] = ZooKeeperUtils.startCuratorFramework(config);
-			client[1] = ZooKeeperUtils.startCuratorFramework(config);
-
 			String wrongAddress = AkkaRpcServiceUtils.getRpcUrl(
 				"1.1.1.1",
 				1234,
@@ -126,7 +128,6 @@ public class ZooKeeperLeaderRetrievalTest extends TestLogger{
 			}
 
 			InetSocketAddress correctInetSocketAddress = new InetSocketAddress(localHost, serverSocket.getLocalPort());
-			String hostPort = NetUtils.unresolvedHostAndPortToNormalizedString(localHost.getHostName(), correctInetSocketAddress.getPort());
 
 			String correctAddress = AkkaRpcServiceUtils.getRpcUrl(
 				localHost.getHostName(),
@@ -135,18 +136,21 @@ public class ZooKeeperLeaderRetrievalTest extends TestLogger{
 				HighAvailabilityServicesUtils.AddressResolution.NO_ADDRESS_RESOLUTION,
 				config);
 
-			faultyLeaderElectionService = ZooKeeperUtils.createLeaderElectionService(client[0], config);
+			faultyLeaderElectionService = highAvailabilityServices.getJobManagerLeaderElectionService(
+				HighAvailabilityServices.DEFAULT_JOB_ID);
 			TestingContender wrongLeaderAddressContender = new TestingContender(wrongAddress, faultyLeaderElectionService);
 
 			faultyLeaderElectionService.start(wrongLeaderAddressContender);
 
-			FindConnectingAddress findConnectingAddress = new FindConnectingAddress(config, timeout);
+			FindConnectingAddress findConnectingAddress = new FindConnectingAddress(
+				timeout,
+				highAvailabilityServices.getJobManagerLeaderRetriever(HighAvailabilityServices.DEFAULT_JOB_ID));
 
 			thread = new Thread(findConnectingAddress);
 
 			thread.start();
 
-			leaderElectionService = ZooKeeperUtils.createLeaderElectionService(client[1], config);
+			leaderElectionService = highAvailabilityServices.getJobManagerLeaderElectionService(HighAvailabilityServices.DEFAULT_JOB_ID);
 			TestingContender correctLeaderAddressContender = new TestingContender(correctAddress, leaderElectionService);
 
 			Thread.sleep(sleepingTime);
@@ -174,14 +178,6 @@ public class ZooKeeperLeaderRetrievalTest extends TestLogger{
 			if (leaderElectionService != null) {
 				leaderElectionService.stop();
 			}
-
-			if (client[0] != null) {
-				client[0].close();
-			}
-
-			if (client[1] != null) {
-				client[1].close();
-			}
 		}
 	}
 
@@ -192,13 +188,9 @@ public class ZooKeeperLeaderRetrievalTest extends TestLogger{
 	 */
 	@Test
 	public void testTimeoutOfFindConnectingAddress() throws Exception {
-		Configuration config = new Configuration();
-		config.setString(HighAvailabilityOptions.HA_MODE, "zookeeper");
-		config.setString(HighAvailabilityOptions.HA_ZOOKEEPER_QUORUM, testingServer.getConnectString());
-
-		FiniteDuration timeout = new FiniteDuration(10, TimeUnit.SECONDS);
+		FiniteDuration timeout = new FiniteDuration(10L, TimeUnit.SECONDS);
 
-		LeaderRetrievalService leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(config, false);
+		LeaderRetrievalService leaderRetrievalService = highAvailabilityServices.getJobManagerLeaderRetriever(HighAvailabilityServices.DEFAULT_JOB_ID);
 		InetAddress result = LeaderRetrievalUtils.findConnectingAddress(leaderRetrievalService, timeout);
 
 		assertEquals(InetAddress.getLocalHost(), result);
@@ -206,22 +198,25 @@ public class ZooKeeperLeaderRetrievalTest extends TestLogger{
 
 	static class FindConnectingAddress implements Runnable {
 
-		private final Configuration config;
 		private final FiniteDuration timeout;
+		private final LeaderRetrievalService leaderRetrievalService;
 
 		private InetAddress result;
 		private Exception exception;
 
-		public FindConnectingAddress(Configuration config, FiniteDuration timeout) {
-			this.config = config;
+		public FindConnectingAddress(
+				FiniteDuration timeout,
+				LeaderRetrievalService leaderRetrievalService) {
 			this.timeout = timeout;
+			this.leaderRetrievalService = leaderRetrievalService;
 		}
 
 		@Override
 		public void run() {
 			try {
-				LeaderRetrievalService leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(config, false);
-				result = LeaderRetrievalUtils.findConnectingAddress(leaderRetrievalService, timeout);
+				result = LeaderRetrievalUtils.findConnectingAddress(
+					leaderRetrievalService,
+					timeout);
 			} catch (Exception e) {
 				exception = e;
 			}

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/TaskManagerMetricsTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/TaskManagerMetricsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/TaskManagerMetricsTest.java
index 100c83d..58f2231 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/TaskManagerMetricsTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/TaskManagerMetricsTest.java
@@ -26,10 +26,9 @@ import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.akka.AkkaUtils;
 import org.apache.flink.runtime.clusterframework.types.ResourceID;
 import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
+import org.apache.flink.runtime.highavailability.nonha.embedded.EmbeddedHaServices;
 import org.apache.flink.runtime.jobmanager.JobManager;
 import org.apache.flink.runtime.jobmanager.MemoryArchivist;
-import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
-import org.apache.flink.runtime.leaderretrieval.StandaloneLeaderRetrievalService;
 import org.apache.flink.runtime.messages.TaskManagerMessages;
 import org.apache.flink.runtime.taskexecutor.TaskManagerConfiguration;
 import org.apache.flink.runtime.taskexecutor.TaskManagerServices;
@@ -37,15 +36,17 @@ import org.apache.flink.runtime.taskexecutor.TaskManagerServicesConfiguration;
 import org.apache.flink.runtime.taskmanager.TaskManager;
 
 import org.apache.flink.runtime.testingUtils.TestingUtils;
+import org.apache.flink.util.TestLogger;
 import org.junit.Assert;
 import org.junit.Test;
 
 import scala.concurrent.duration.FiniteDuration;
 
 import java.net.InetAddress;
+import java.util.UUID;
 import java.util.concurrent.TimeUnit;
 
-public class TaskManagerMetricsTest {
+public class TaskManagerMetricsTest extends TestLogger {
 
 	/**
 	 * Tests the metric registry life cycle on JobManager re-connects.
@@ -53,6 +54,9 @@ public class TaskManagerMetricsTest {
 	@Test
 	public void testMetricRegistryLifeCycle() throws Exception {
 		ActorSystem actorSystem = null;
+
+		HighAvailabilityServices highAvailabilityServices = new EmbeddedHaServices(TestingUtils.defaultExecutor());
+
 		try {
 			actorSystem = AkkaUtils.createLocalActorSystem(new Configuration());
 
@@ -64,11 +68,10 @@ public class TaskManagerMetricsTest {
 				actorSystem,
 				TestingUtils.defaultExecutor(),
 				TestingUtils.defaultExecutor(),
+				highAvailabilityServices,
 				JobManager.class,
 				MemoryArchivist.class)._1();
 
-			LeaderRetrievalService leaderRetrievalService = new StandaloneLeaderRetrievalService(jobManager.path().toString());
-
 			// ================================================================
 			// Start TaskManager
 			// ================================================================
@@ -94,7 +97,7 @@ public class TaskManagerMetricsTest {
 				taskManagerServices.getMemoryManager(),
 				taskManagerServices.getIOManager(),
 				taskManagerServices.getNetworkEnvironment(),
-				leaderRetrievalService,
+				highAvailabilityServices.getJobManagerLeaderRetriever(HighAvailabilityServices.DEFAULT_JOB_ID),
 				tmRegistry);
 
 			final ActorRef taskManager = actorSystem.actorOf(tmProps);
@@ -107,20 +110,21 @@ public class TaskManagerMetricsTest {
 							getTestActor());
 
 						// wait for the TM to be registered
-						expectMsgEquals(TaskManagerMessages.getRegisteredAtJobManagerMessage());
+						TaskManagerMessages.RegisteredAtJobManager registeredAtJobManager = expectMsgClass(TaskManagerMessages.RegisteredAtJobManager.class);
+						UUID leaderId = registeredAtJobManager.leaderId();
 
 						// trigger re-registration of TM; this should include a disconnect from the current JM
 						taskManager.tell(
 							new TaskManagerMessages.JobManagerLeaderAddress(
 								jobManager.path().toString(),
-								HighAvailabilityServices.DEFAULT_LEADER_ID),
+								leaderId),
 							jobManager);
 
 						// wait for re-registration to be completed
 						taskManager.tell(TaskManagerMessages.getNotifyWhenRegisteredAtJobManagerMessage(),
 							getTestActor());
 
-						expectMsgEquals(TaskManagerMessages.getRegisteredAtJobManagerMessage());
+						expectMsgClass(TaskManagerMessages.RegisteredAtJobManager.class);
 					}
 				};
 			}};
@@ -135,6 +139,10 @@ public class TaskManagerMetricsTest {
 			if (actorSystem != null) {
 				actorSystem.shutdown();
 			}
+
+			if (highAvailabilityServices != null) {
+				highAvailabilityServices.closeAndCleanupAllData();
+			}
 		}
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/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 cecfe6a..973fddf 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
@@ -34,6 +34,8 @@ import org.apache.flink.runtime.akka.AkkaUtils;
 import org.apache.flink.runtime.clusterframework.FlinkResourceManager;
 import org.apache.flink.runtime.clusterframework.standalone.StandaloneResourceManager;
 import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
+import org.apache.flink.runtime.highavailability.nonha.embedded.EmbeddedHaServices;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
 import org.apache.flink.runtime.io.network.LocalConnectionManager;
@@ -44,7 +46,6 @@ import org.apache.flink.runtime.io.network.partition.ResultPartitionManager;
 import org.apache.flink.runtime.jobmanager.JobManager;
 import org.apache.flink.runtime.jobmanager.MemoryArchivist;
 import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
-import org.apache.flink.runtime.leaderretrieval.StandaloneLeaderRetrievalService;
 import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.runtime.messages.TaskManagerMessages;
 import org.apache.flink.runtime.metrics.MetricRegistry;
@@ -52,23 +53,24 @@ import org.apache.flink.runtime.metrics.MetricRegistryConfiguration;
 import org.apache.flink.runtime.query.KvStateRegistry;
 import org.apache.flink.runtime.taskexecutor.TaskManagerConfiguration;
 import org.apache.flink.runtime.testingUtils.TestingUtils;
-import org.apache.flink.runtime.util.LeaderRetrievalUtils;
 
+import org.apache.flink.util.TestLogger;
 import org.junit.Test;
 
 import scala.concurrent.duration.FiniteDuration;
 
 import java.net.InetAddress;
+import java.util.Arrays;
 import java.util.concurrent.TimeUnit;
 
-public class TaskManagerComponentsStartupShutdownTest {
+public class TaskManagerComponentsStartupShutdownTest extends TestLogger {
 
 	/**
 	 * Makes sure that all components are shut down when the TaskManager
 	 * actor is shut down.
 	 */
 	@Test
-	public void testComponentsStartupShutdown() {
+	public void testComponentsStartupShutdown() throws Exception {
 
 		final String[] TMP_DIR = new String[] { ConfigConstants.DEFAULT_TASK_MANAGER_TMP_PATH };
 		final Time timeout = Time.seconds(100);
@@ -80,21 +82,28 @@ public class TaskManagerComponentsStartupShutdownTest {
 		config.setInteger(ConfigConstants.AKKA_WATCH_THRESHOLD, 1);
 
 		ActorSystem actorSystem = null;
+
+		HighAvailabilityServices highAvailabilityServices = new EmbeddedHaServices(TestingUtils.defaultExecutor());
+
+		ActorRef jobManager = null;
+		ActorRef taskManager = null;
+
 		try {
 			actorSystem = AkkaUtils.createLocalActorSystem(config);
 
-			final ActorRef jobManager = JobManager.startJobManagerActors(
+			jobManager = JobManager.startJobManagerActors(
 				config,
 				actorSystem,
 				TestingUtils.defaultExecutor(),
 				TestingUtils.defaultExecutor(),
+				highAvailabilityServices,
 				JobManager.class,
 				MemoryArchivist.class)._1();
 
 			FlinkResourceManager.startResourceManagerActors(
 				config,
 				actorSystem,
-				LeaderRetrievalUtils.createLeaderRetrievalService(config, jobManager),
+				highAvailabilityServices.getJobManagerLeaderRetriever(HighAvailabilityServices.DEFAULT_JOB_ID),
 				StandaloneResourceManager.class);
 
 			final int numberOfSlots = 1;
@@ -137,7 +146,8 @@ public class TaskManagerComponentsStartupShutdownTest {
 
 			network.start();
 
-			LeaderRetrievalService leaderRetrievalService = new StandaloneLeaderRetrievalService(jobManager.path().toString());
+			LeaderRetrievalService leaderRetrievalService = highAvailabilityServices.getJobManagerLeaderRetriever(
+				HighAvailabilityServices.DEFAULT_JOB_ID);
 
 			MetricRegistryConfiguration metricRegistryConfiguration = MetricRegistryConfiguration.fromConfiguration(config);
 
@@ -154,18 +164,20 @@ public class TaskManagerComponentsStartupShutdownTest {
 				leaderRetrievalService,
 				new MetricRegistry(metricRegistryConfiguration));
 
-			final ActorRef taskManager = actorSystem.actorOf(tmProps);
+			taskManager = actorSystem.actorOf(tmProps);
+
+			final ActorRef finalTaskManager = taskManager;
 
 			new JavaTestKit(actorSystem) {{
 
 				// wait for the TaskManager to be registered
-				new Within(new FiniteDuration(5000, TimeUnit.SECONDS)) {
+				new Within(new FiniteDuration(5000L, TimeUnit.SECONDS)) {
 					@Override
 					protected void run() {
-						taskManager.tell(TaskManagerMessages.getNotifyWhenRegisteredAtJobManagerMessage(),
+						finalTaskManager.tell(TaskManagerMessages.getNotifyWhenRegisteredAtJobManagerMessage(),
 								getTestActor());
 
-						expectMsgEquals(TaskManagerMessages.getRegisteredAtJobManagerMessage());
+						expectMsgClass(TaskManagerMessages.RegisteredAtJobManager.class);
 					}
 				};
 			}};
@@ -184,15 +196,16 @@ public class TaskManagerComponentsStartupShutdownTest {
 			assertTrue(network.isShutdown());
 			assertTrue(ioManager.isProperlyShutDown());
 			assertTrue(memManager.isShutdown());
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-		finally {
+		} finally {
+			TestingUtils.stopActorsGracefully(Arrays.asList(jobManager, taskManager));
+
 			if (actorSystem != null) {
 				actorSystem.shutdown();
+
+				actorSystem.awaitTermination(TestingUtils.TESTING_TIMEOUT());
 			}
+
+			highAvailabilityServices.closeAndCleanupAllData();
 		}
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/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 c0d0455..a760760 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
@@ -24,6 +24,9 @@ import org.apache.flink.configuration.GlobalConfiguration;
 import org.apache.flink.configuration.IllegalConfigurationException;
 import org.apache.flink.core.fs.FileSystem;
 import org.apache.flink.core.testutils.CommonTestUtils;
+import org.apache.flink.runtime.concurrent.Executors;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils;
 import org.junit.Test;
 
 import scala.Tuple2;
@@ -44,16 +47,23 @@ import static org.junit.Assert.*;
 public class TaskManagerConfigurationTest {
 
 	@Test
-	public void testUsePreconfiguredNetworkInterface() {
+	public void testUsePreconfiguredNetworkInterface() throws Exception {
+		final String TEST_HOST_NAME = "testhostname";
+
+		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);
+
+		HighAvailabilityServices highAvailabilityServices = HighAvailabilityServicesUtils.createHighAvailabilityServices(
+			config,
+			Executors.directExecutor(),
+			HighAvailabilityServicesUtils.AddressResolution.NO_ADDRESS_RESOLUTION);
+
 		try {
-			final String TEST_HOST_NAME = "testhostname";
 
-			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);
+			Tuple2<String, Object> address = TaskManager.selectNetworkInterfaceAndPort(config, highAvailabilityServices);
 
 			// validate the configured test host name
 			assertEquals(TEST_HOST_NAME, address._1());
@@ -61,30 +71,37 @@ public class TaskManagerConfigurationTest {
 		catch (Exception e) {
 			e.printStackTrace();
 			fail(e.getMessage());
+		} finally {
+			highAvailabilityServices.closeAndCleanupAllData();
 		}
 	}
 
 	@Test
-	public void testActorSystemPortConfig() {
-		try {
-			// 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);
+	public void testActorSystemPortConfig() throws Exception {
+		// 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);
+
+		HighAvailabilityServices highAvailabilityServices = HighAvailabilityServicesUtils.createHighAvailabilityServices(
+			config,
+			Executors.directExecutor(),
+			HighAvailabilityServicesUtils.AddressResolution.NO_ADDRESS_RESOLUTION);
 
+		try {
 			// auto port
-			assertEquals(0, TaskManager.selectNetworkInterfaceAndPort(config)._2());
+			assertEquals(0, TaskManager.selectNetworkInterfaceAndPort(config, highAvailabilityServices)._2());
 
 			// pre-defined port
 			final int testPort = 22551;
 			config.setInteger(ConfigConstants.TASK_MANAGER_IPC_PORT_KEY, testPort);
-			assertEquals(testPort, TaskManager.selectNetworkInterfaceAndPort(config)._2());
+			assertEquals(testPort, TaskManager.selectNetworkInterfaceAndPort(config, highAvailabilityServices)._2());
 
 			// invalid port
 			try {
 				config.setInteger(ConfigConstants.TASK_MANAGER_IPC_PORT_KEY, -1);
-				TaskManager.selectNetworkInterfaceAndPort(config);
+				TaskManager.selectNetworkInterfaceAndPort(config, highAvailabilityServices);
 				fail("should fail with an exception");
 			}
 			catch (IllegalConfigurationException e) {
@@ -94,7 +111,7 @@ public class TaskManagerConfigurationTest {
 			// invalid port
 			try {
 				config.setInteger(ConfigConstants.TASK_MANAGER_IPC_PORT_KEY, 100000);
-				TaskManager.selectNetworkInterfaceAndPort(config);
+				TaskManager.selectNetworkInterfaceAndPort(config, highAvailabilityServices);
 				fail("should fail with an exception");
 			}
 			catch (IllegalConfigurationException e) {
@@ -104,6 +121,8 @@ public class TaskManagerConfigurationTest {
 		catch (Exception e) {
 			e.printStackTrace();
 			fail(e.getMessage());
+		} finally {
+			highAvailabilityServices.closeAndCleanupAllData();
 		}
 	}
 
@@ -136,7 +155,7 @@ public class TaskManagerConfigurationTest {
 	}
 
 	@Test
-	public void testNetworkInterfaceSelection() {
+	public void testNetworkInterfaceSelection() throws Exception {
 		ServerSocket server;
 		String hostname = "localhost";
 
@@ -149,20 +168,27 @@ public class TaskManagerConfigurationTest {
 			return;
 		}
 
-		try {
-			// open a server port to allow the system to connect
-			Configuration config = new Configuration();
+		// open a server port to allow the system to connect
+		Configuration config = new Configuration();
+
+		config.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, hostname);
+		config.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, server.getLocalPort());
 
-			config.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, hostname);
-			config.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, server.getLocalPort());
+		HighAvailabilityServices highAvailabilityServices = HighAvailabilityServicesUtils.createHighAvailabilityServices(
+			config,
+			Executors.directExecutor(),
+			HighAvailabilityServicesUtils.AddressResolution.NO_ADDRESS_RESOLUTION);
 
-			assertNotNull(TaskManager.selectNetworkInterfaceAndPort(config)._1());
+		try {
+			assertNotNull(TaskManager.selectNetworkInterfaceAndPort(config, highAvailabilityServices)._1());
 		}
 		catch (Exception e) {
 			e.printStackTrace();
 			fail(e.getMessage());
 		}
 		finally {
+			highAvailabilityServices.closeAndCleanupAllData();
+
 			try {
 				server.close();
 			} catch (IOException e) {

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerProcessReapingTestBase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerProcessReapingTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerProcessReapingTestBase.java
index d904004..130610c 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerProcessReapingTestBase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerProcessReapingTestBase.java
@@ -23,17 +23,20 @@ import akka.actor.ActorSystem;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.TaskManagerOptions;
+import org.apache.flink.configuration.JobManagerOptions;
 import org.apache.flink.runtime.akka.AkkaUtils;
 import org.apache.flink.runtime.clusterframework.FlinkResourceManager;
 import org.apache.flink.runtime.clusterframework.standalone.StandaloneResourceManager;
 import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils;
 import org.apache.flink.runtime.jobmanager.JobManager;
 import org.apache.flink.runtime.jobmanager.MemoryArchivist;
-import org.apache.flink.runtime.leaderretrieval.StandaloneLeaderRetrievalService;
 import org.apache.flink.runtime.taskexecutor.TaskExecutor;
 import org.apache.flink.runtime.testingUtils.TestingUtils;
 import org.apache.flink.runtime.testutils.CommonTestUtils;
 import org.apache.flink.util.NetUtils;
+import org.apache.flink.util.TestLogger;
 import org.junit.Test;
 import scala.Some;
 import scala.Tuple2;
@@ -56,7 +59,7 @@ import static org.junit.Assert.fail;
 /**
  * Tests that the TaskManager process properly exits when the TaskManager actor dies.
  */
-public abstract class TaskManagerProcessReapingTestBase {
+public abstract class TaskManagerProcessReapingTestBase extends TestLogger {
 
 	/**
 	 * Called after the task manager has been started up. After calling this
@@ -72,12 +75,25 @@ public abstract class TaskManagerProcessReapingTestBase {
 	}
 
 	@Test
-	public void testReapProcessOnFailure() {
+	public void testReapProcessOnFailure() throws Exception {
 		Process taskManagerProcess = null;
 		ActorSystem jmActorSystem = null;
 
 		final StringWriter processOutput = new StringWriter();
 
+		final Configuration config = new Configuration();
+
+		final String jobManagerHostname = "localhost";
+		final int jobManagerPort = NetUtils.getAvailablePort();
+
+		config.setString(JobManagerOptions.ADDRESS, jobManagerHostname);
+		config.setInteger(JobManagerOptions.PORT, jobManagerPort);
+
+		final HighAvailabilityServices highAvailabilityServices = HighAvailabilityServicesUtils.createHighAvailabilityServices(
+			config,
+			TestingUtils.defaultExecutor(),
+			HighAvailabilityServicesUtils.AddressResolution.NO_ADDRESS_RESOLUTION);
+
 		try {
 			String javaCommand = getJavaCommandPath();
 
@@ -93,29 +109,23 @@ public abstract class TaskManagerProcessReapingTestBase {
 			tempLogFile.deleteOnExit();
 			CommonTestUtils.printLog4jDebugConfig(tempLogFile);
 
-			final int jobManagerPort = NetUtils.getAvailablePort();
-
 			// start a JobManager
-			Tuple2<String, Object> localAddress = new Tuple2<String, Object>("localhost", jobManagerPort);
-			jmActorSystem = AkkaUtils.createActorSystem(
-					new Configuration(), new Some<Tuple2<String, Object>>(localAddress));
+			Tuple2<String, Object> localAddress = new Tuple2<String, Object>(jobManagerHostname, jobManagerPort);
+			jmActorSystem = AkkaUtils.createActorSystem(config, new Some<>(localAddress));
 
 			ActorRef jmActor = JobManager.startJobManagerActors(
-				new Configuration(),
+				config,
 				jmActorSystem,
 				TestingUtils.defaultExecutor(),
 				TestingUtils.defaultExecutor(),
+				highAvailabilityServices,
 				JobManager.class,
 				MemoryArchivist.class)._1;
 
-			// start a ResourceManager
-			StandaloneLeaderRetrievalService standaloneLeaderRetrievalService =
-				new StandaloneLeaderRetrievalService(AkkaUtils.getAkkaURL(jmActorSystem, jmActor));
-
 			FlinkResourceManager.startResourceManagerActors(
 				new Configuration(),
 				jmActorSystem,
-				standaloneLeaderRetrievalService,
+				highAvailabilityServices.getJobManagerLeaderRetriever(HighAvailabilityServices.DEFAULT_JOB_ID),
 				StandaloneResourceManager.class);
 
 			final int taskManagerPort = NetUtils.getAvailablePort();
@@ -205,6 +215,9 @@ public abstract class TaskManagerProcessReapingTestBase {
 			if (jmActorSystem != null) {
 				jmActorSystem.shutdown();
 			}
+			if (highAvailabilityServices != null) {
+				highAvailabilityServices.closeAndCleanupAllData();
+			}
 		}
 	}
 
@@ -222,18 +235,28 @@ public abstract class TaskManagerProcessReapingTestBase {
 
 	public static class TaskManagerTestEntryPoint {
 
-		public static void main(String[] args) {
-			try {
-				int jobManagerPort = Integer.parseInt(args[0]);
-				int taskManagerPort = Integer.parseInt(args[1]);
+		public static void main(String[] args) throws Exception {
+			int jobManagerPort = Integer.parseInt(args[0]);
+			int taskManagerPort = Integer.parseInt(args[1]);
 
-				Configuration cfg = new Configuration();
-				cfg.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, "localhost");
-				cfg.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, jobManagerPort);
-				cfg.setLong(TaskManagerOptions.MANAGED_MEMORY_SIZE, 4L);
-				cfg.setInteger(TaskManagerOptions.NETWORK_NUM_BUFFERS, 256);
+			Configuration cfg = new Configuration();
+			cfg.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, "localhost");
+			cfg.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, jobManagerPort);
+			cfg.setLong(TaskManagerOptions.MANAGED_MEMORY_SIZE, 4L);
+			cfg.setInteger(TaskManagerOptions.NETWORK_NUM_BUFFERS, 256);
 
-				TaskManager.runTaskManager("localhost", ResourceID.generate(), taskManagerPort, cfg);
+			final HighAvailabilityServices highAvailabilityServices = HighAvailabilityServicesUtils.createHighAvailabilityServices(
+				cfg,
+				TestingUtils.defaultExecutor(),
+				HighAvailabilityServicesUtils.AddressResolution.NO_ADDRESS_RESOLUTION);
+
+			try {
+				TaskManager.runTaskManager(
+					"localhost",
+					ResourceID.generate(),
+					taskManagerPort,
+					cfg,
+					highAvailabilityServices);
 
 				// wait forever
 				Object lock = new Object();
@@ -243,6 +266,8 @@ public abstract class TaskManagerProcessReapingTestBase {
 			}
 			catch (Throwable t) {
 				System.exit(1);
+			} finally {
+				highAvailabilityServices.closeAndCleanupAllData();
 			}
 		}
 	}


[02/16] flink git commit: [FLINK-6136] Separate EmbeddedHaServices and StandaloneHaServices

Posted by tr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/a0bb99c7/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
index b22f1a6..05749c4 100644
--- 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
@@ -56,31 +56,20 @@ public class LeaderRetrievalUtils {
 	 * 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 {
-		return createLeaderRetrievalService(configuration, false);
-	}
-
-	/**
-	 * Creates a {@link LeaderRetrievalService} based on the provided {@link Configuration} object.
-	 *
-	 * @param configuration Configuration containing the settings for the {@link LeaderRetrievalService}
 	 * @param resolveInitialHostName If true, resolves the initial hostname
 	 * @return The {@link LeaderRetrievalService} specified in the configuration object
 	 * @throws Exception
 	 */
 	public static LeaderRetrievalService createLeaderRetrievalService(
-			Configuration configuration, boolean resolveInitialHostName)
+			Configuration configuration,
+			boolean resolveInitialHostName)
 		throws Exception {
 
 		HighAvailabilityMode highAvailabilityMode = getRecoveryMode(configuration);
 
 		switch (highAvailabilityMode) {
 			case NONE:
-				return StandaloneUtils.createLeaderRetrievalService(configuration, resolveInitialHostName);
+				return StandaloneUtils.createLeaderRetrievalService(configuration, resolveInitialHostName, null);
 			case ZOOKEEPER:
 				return ZooKeeperUtils.createLeaderRetrievalService(configuration);
 			default:

http://git-wip-us.apache.org/repos/asf/flink/blob/a0bb99c7/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
index 8436ced..1719b38 100644
--- 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
@@ -18,15 +18,15 @@
 
 package org.apache.flink.runtime.util;
 
+import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.jobmanager.JobManager;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils.AddressResolution;
+import org.apache.flink.runtime.jobmaster.JobMaster;
 import org.apache.flink.runtime.leaderretrieval.StandaloneLeaderRetrievalService;
-import org.apache.flink.runtime.taskmanager.TaskManager;
-import org.apache.flink.util.NetUtils;
-import scala.Option;
-import scala.Tuple3;
+import org.apache.flink.runtime.rpc.akka.AkkaRpcServiceUtils;
+import org.apache.flink.util.ConfigurationException;
 
-import java.net.InetAddress;
 import java.net.UnknownHostException;
 
 /**
@@ -40,27 +40,15 @@ public final class StandaloneUtils {
 	 *
 	 * @param configuration Configuration instance containing the host and port information
 	 * @return StandaloneLeaderRetrievalService
+	 * @throws ConfigurationException
 	 * @throws UnknownHostException
 	 */
-	public static StandaloneLeaderRetrievalService createLeaderRetrievalService(
-		Configuration configuration)
-		throws UnknownHostException {
-		return createLeaderRetrievalService(configuration, false);
-	}
-
-	/**
-	 * 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
-	 * @param resolveInitialHostName If true, resolves the hostname of the StandaloneLeaderRetrievalService
-	 * @return StandaloneLeaderRetrievalService
-	 * @throws UnknownHostException
-	 */
-	public static StandaloneLeaderRetrievalService createLeaderRetrievalService(
-			Configuration configuration, boolean resolveInitialHostName)
-		throws UnknownHostException {
-		return createLeaderRetrievalService(configuration, resolveInitialHostName, null);
+	public static StandaloneLeaderRetrievalService createLeaderRetrievalService(Configuration configuration)
+		throws ConfigurationException, UnknownHostException {
+		return createLeaderRetrievalService(
+			configuration,
+			false,
+			null);
 	}
 
 	/**
@@ -73,38 +61,22 @@ public final class StandaloneUtils {
 	 * @param resolveInitialHostName If true, resolves the hostname of the StandaloneLeaderRetrievalService
 	 * @param jobManagerName Name of the JobManager actor
 	 * @return StandaloneLeaderRetrievalService
-	 * @throws UnknownHostException if the host name cannot be resolved into an {@link InetAddress}
+	 * @throws ConfigurationException if the job manager address cannot be retrieved from the configuration
+	 * @throws UnknownHostException if the job manager address cannot be resolved
 	 */
 	public static StandaloneLeaderRetrievalService createLeaderRetrievalService(
 			Configuration configuration,
 			boolean resolveInitialHostName,
 			String jobManagerName)
-		throws UnknownHostException {
-
-		Tuple3<String, String, Object> stringIntPair = TaskManager.getAndCheckJobManagerAddress(configuration);
-
-		String protocol = stringIntPair._1();
-		String jobManagerHostname = stringIntPair._2();
-		int jobManagerPort = (Integer) stringIntPair._3();
-
-		// Do not try to resolve a hostname to prevent resolving to the wrong IP address
-		String hostPort = NetUtils.unresolvedHostAndPortToNormalizedString(jobManagerHostname, jobManagerPort);
-
-		if (resolveInitialHostName) {
-			try {
-				//noinspection ResultOfMethodCallIgnored
-				InetAddress.getByName(jobManagerHostname);
-			}
-			catch (UnknownHostException e) {
-				throw new UnknownHostException("Cannot resolve the JobManager hostname '" + jobManagerHostname
-					+ "' specified in the configuration");
-			}
-		}
+		throws ConfigurationException, UnknownHostException {
+		Tuple2<String, Integer> hostnamePort = HighAvailabilityServicesUtils.getJobManagerAddress(configuration);
 
-		String jobManagerAkkaUrl = JobManager.getRemoteJobManagerAkkaURL(
-				protocol,
-				hostPort,
-				Option.apply(jobManagerName));
+		String jobManagerAkkaUrl = AkkaRpcServiceUtils.getRpcUrl(
+			hostnamePort.f0,
+			hostnamePort.f1,
+			jobManagerName != null ? jobManagerName : JobMaster.JOB_MANAGER_NAME,
+			resolveInitialHostName ? AddressResolution.TRY_ADDRESS_RESOLUTION : AddressResolution.NO_ADDRESS_RESOLUTION,
+			configuration);
 
 		return new StandaloneLeaderRetrievalService(jobManagerAkkaUrl);
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/a0bb99c7/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 d94eb7a..62fa73d 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
@@ -28,7 +28,7 @@ import com.typesafe.config.{Config, ConfigFactory}
 import org.apache.flink.api.common.time.Time
 import org.apache.flink.configuration.{AkkaOptions, ConfigConstants, Configuration}
 import org.apache.flink.runtime.net.SSLUtils
-import org.apache.flink.util.NetUtils
+import org.apache.flink.util.{ConfigurationException, NetUtils, Preconditions}
 import org.jboss.netty.logging.{InternalLoggerFactory, Slf4JLoggerFactory}
 import org.slf4j.LoggerFactory
 
@@ -707,18 +707,15 @@ object AkkaUtils {
       "(d|day)|(h|hour)|(min|minute)|s|sec|second)|(ms|milli|millisecond)|" +
       "(µs|micro|microsecond)|(ns|nano|nanosecond)"
   }
-  
-  /** Returns the protocol field for the URL of the remote actor system given the user configuration
+
+  /**
+    * Returns the local akka url for the given actor name.
     *
-    * @param config instance containing the user provided configuration values
-    * @return the remote url's protocol field
+    * @param actorName Actor name identifying the actor
+    * @return Local Akka URL for the given actor
     */
-  def getAkkaProtocol(config: Configuration): String = {
-    val sslEnabled = config.getBoolean(ConfigConstants.AKKA_SSL_ENABLED,
-        ConfigConstants.DEFAULT_AKKA_SSL_ENABLED) &&
-      SSLUtils.getSSLEnabled(config)
-    if (sslEnabled) "akka.ssl.tcp" else "akka.tcp"
+  def getLocalAkkaURL(actorName: String): String = {
+    "akka://flink/user/" + actorName
   }
-
 }
 

http://git-wip-us.apache.org/repos/asf/flink/blob/a0bb99c7/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 479ec51..68f43da 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
@@ -49,11 +49,15 @@ import org.apache.flink.runtime.execution.SuppressRestartsException
 import org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager
 import org.apache.flink.runtime.executiongraph.restart.RestartStrategyFactory
 import org.apache.flink.runtime.executiongraph._
+import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils
+import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils.AddressResolution
 import org.apache.flink.runtime.instance.{AkkaActorGateway, InstanceID, InstanceManager}
 import org.apache.flink.runtime.jobgraph.{JobGraph, JobStatus}
 import org.apache.flink.runtime.jobmanager.SubmittedJobGraphStore.SubmittedJobGraphListener
 import org.apache.flink.runtime.jobmanager.scheduler.{Scheduler => FlinkScheduler}
 import org.apache.flink.runtime.jobmanager.slots.ActorTaskManagerGateway
+import org.apache.flink.runtime.jobmaster.JobMaster
+import org.apache.flink.runtime.jobmaster.JobMaster.{ARCHIVE_NAME, JOB_MANAGER_NAME}
 import org.apache.flink.runtime.leaderelection.{LeaderContender, LeaderElectionService, StandaloneLeaderElectionService}
 import org.apache.flink.runtime.messages.ArchiveMessages.ArchiveExecutionGraph
 import org.apache.flink.runtime.messages.ExecutionGraphMessages.JobStatusChanged
@@ -72,8 +76,11 @@ import org.apache.flink.runtime.metrics.util.MetricUtils
 import org.apache.flink.runtime.process.ProcessReaper
 import org.apache.flink.runtime.query.KvStateMessage.{LookupKvStateLocation, NotifyKvStateRegistered, NotifyKvStateUnregistered}
 import org.apache.flink.runtime.query.{KvStateMessage, UnknownKvStateLocation}
+import org.apache.flink.runtime.rpc.akka.AkkaRpcServiceUtils
 import org.apache.flink.runtime.security.SecurityUtils
 import org.apache.flink.runtime.security.SecurityUtils.SecurityConfiguration
+import org.apache.flink.runtime.taskexecutor.TaskExecutor
+import org.apache.flink.runtime.taskexecutor.TaskExecutor.TASK_MANAGER_NAME
 import org.apache.flink.runtime.taskmanager.TaskManager
 import org.apache.flink.runtime.util._
 import org.apache.flink.runtime.webmonitor.{WebMonitor, WebMonitorUtils}
@@ -1923,12 +1930,6 @@ object JobManager {
   val STARTUP_FAILURE_RETURN_CODE = 1
   val RUNTIME_FAILURE_RETURN_CODE = 2
 
-  /** Name of the JobManager actor */
-  val JOB_MANAGER_NAME = "jobmanager"
-
-  /** Name of the archive actor */
-  val ARCHIVE_NAME = "archive"
-
 
   /**
    * Entry point (main method) to run the JobManager in a standalone fashion.
@@ -2237,7 +2238,7 @@ object JobManager {
       if (configuration.getInteger(ConfigConstants.JOB_MANAGER_WEB_PORT_KEY, 0) >= 0) {
         LOG.info("Starting JobManager web frontend")
         val leaderRetrievalService = LeaderRetrievalUtils
-          .createLeaderRetrievalService(configuration)
+          .createLeaderRetrievalService(configuration, false)
 
         // start the web frontend. we need to load this dynamically
         // because it is not in the same project/dependencies
@@ -2287,7 +2288,7 @@ object JobManager {
           ResourceID.generate(),
           jobManagerSystem,
           externalHostname,
-          Some(TaskManager.TASK_MANAGER_NAME),
+          Some(TaskExecutor.TASK_MANAGER_NAME),
           None,
           localTaskManagerCommunication = true,
           classOf[TaskManager])
@@ -2305,7 +2306,13 @@ object JobManager {
       // start web monitor
       webMonitor.foreach {
         monitor =>
-          val jobManagerAkkaUrl = JobManager.getRemoteJobManagerAkkaURL(configuration)
+          val hostnamePort = HighAvailabilityServicesUtils.getJobManagerAddress(configuration)
+          val jobManagerAkkaUrl = AkkaRpcServiceUtils.getRpcUrl(
+            hostnamePort.f0,
+            hostnamePort.f1,
+            JobMaster.JOB_MANAGER_NAME,
+            AddressResolution.NO_ADDRESS_RESOLUTION,
+            configuration)
           monitor.start(jobManagerAkkaUrl)
       }
 
@@ -2317,7 +2324,7 @@ object JobManager {
               FlinkResourceManager.startResourceManagerActors(
                 configuration,
                 jobManagerSystem,
-                LeaderRetrievalUtils.createLeaderRetrievalService(configuration),
+                LeaderRetrievalUtils.createLeaderRetrievalService(configuration, false),
                 rmClass))
           case None =>
             LOG.info("Resource Manager class not provided. No resource manager will be started.")
@@ -2631,8 +2638,8 @@ object JobManager {
       actorSystem,
       futureExecutor,
       ioExecutor,
-      Some(JOB_MANAGER_NAME),
-      Some(ARCHIVE_NAME),
+      Some(JobMaster.JOB_MANAGER_NAME),
+      Some(JobMaster.ARCHIVE_NAME),
       jobManagerClass,
       archiveClass)
   }
@@ -2761,121 +2768,4 @@ object JobManager {
       jobRecoveryTimeout,
       metricsRegistry)
   }
-
-  // --------------------------------------------------------------------------
-  //  Resolving the JobManager endpoint
-  // --------------------------------------------------------------------------
-
-  /**
-   * Builds the akka actor path for the JobManager actor, given the socket address
-   * where the JobManager's actor system runs.
-   *
-   * @param protocol The protocol to be used to connect to the remote JobManager's actor system.
-   * @param hostPort The external address of the JobManager's actor system in format host:port
-   * @return The akka URL of the JobManager actor.
-   */
-  def getRemoteJobManagerAkkaURL(
-      protocol: String,
-      hostPort: String,
-      name: Option[String] = None)
-    : String = {
-
-    require(protocol == "akka.tcp" || protocol == "akka.ssl.tcp",
-        "protocol field should be either akka.tcp or akka.ssl.tcp")
-
-    getJobManagerAkkaURLHelper(s"$protocol://flink@$hostPort", name)
-  }
-
-  /**
-   * Returns the JobManager actor's remote Akka URL, given the configured hostname and port.
-   *
-   * @param config The configuration to parse
-   * @return JobManager actor remote Akka URL
-   */
-  def getRemoteJobManagerAkkaURL(config: Configuration) : String = {
-    val (protocol, hostname, port) = TaskManager.getAndCheckJobManagerAddress(config)
-
-    val hostPort = NetUtils.unresolvedHostAndPortToNormalizedString(hostname, port)
-
-    JobManager.getRemoteJobManagerAkkaURL(protocol, hostPort, Option.empty)
-  }
-
-  /**
-   * Builds the akka actor path for the JobManager actor to address the actor within
-   * its own actor system.
-   *
-   * @return The local akka URL of the JobManager actor.
-   */
-  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)
-  }
-
-  private def getJobManagerAkkaURLHelper(address: String, name: Option[String]): String = {
-    address + "/user/" + name.getOrElse(JOB_MANAGER_NAME)
-  }
-
-  /**
-   * Resolves the JobManager actor reference in a blocking fashion.
-   *
-   * @param jobManagerUrl The akka URL of the JobManager.
-   * @param system The local actor system that should perform the lookup.
-   * @param timeout The maximum time to wait until the lookup fails.
-   * @throws java.io.IOException Thrown, if the lookup fails.
-   * @return The ActorRef to the JobManager
-   */
-  @throws(classOf[IOException])
-  def getJobManagerActorRef(
-      jobManagerUrl: String,
-      system: ActorSystem,
-      timeout: FiniteDuration)
-    : ActorRef = {
-    AkkaUtils.getActorRef(jobManagerUrl, system, timeout)
-  }
-
-  /**
-   * Resolves the JobManager actor reference in a blocking fashion.
-   *
-   * @param protocol The protocol to be used to connect to the remote JobManager's actor system.
-   * @param hostPort The external address of the JobManager's actor system in format host:port.
-   * @param system The local actor system that should perform the lookup.
-   * @param timeout The maximum time to wait until the lookup fails.
-   * @throws java.io.IOException Thrown, if the lookup fails.
-   * @return The ActorRef to the JobManager
-   */
-  @throws(classOf[IOException])
-  def getJobManagerActorRef(
-      protocol: String,
-      hostPort: String,
-      system: ActorSystem,
-      timeout: FiniteDuration)
-    : ActorRef = {
-
-    val jmAddress = getRemoteJobManagerAkkaURL(protocol, hostPort)
-    getJobManagerActorRef(jmAddress, system, timeout)
-  }
-
-  /**
-   * Resolves the JobManager actor reference in a blocking fashion.
-   *
-   * @param hostPort The address of the JobManager's actor system in format host:port.
-   * @param system The local actor system that should perform the lookup.
-   * @param config The config describing the maximum time to wait until the lookup fails.
-   * @throws java.io.IOException Thrown, if the lookup fails.
-   * @return The ActorRef to the JobManager
-   */
-  @throws(classOf[IOException])
-  def getJobManagerActorRef(
-      hostPort: String,
-      system: ActorSystem,
-      config: Configuration)
-    : ActorRef = {
-
-    val timeout = AkkaUtils.getLookupTimeout(config)
-    val protocol = AkkaUtils.getAkkaProtocol(config)
-    getJobManagerActorRef(protocol, hostPort, system, timeout)
-  }
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/a0bb99c7/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 3d43da5..2f83548 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
@@ -38,13 +38,14 @@ import org.apache.flink.runtime.io.network.NetworkEnvironment
 import org.apache.flink.runtime.io.network.netty.NettyConfig
 import org.apache.flink.runtime.jobmanager.scheduler.Scheduler
 import org.apache.flink.runtime.jobmanager.{JobManager, MemoryArchivist, SubmittedJobGraphStore}
+import org.apache.flink.runtime.jobmaster.JobMaster
 import org.apache.flink.runtime.leaderelection.LeaderElectionService
 import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService
 import org.apache.flink.runtime.memory.MemoryManager
 import org.apache.flink.runtime.messages.JobManagerMessages
 import org.apache.flink.runtime.messages.JobManagerMessages.{RunningJobsStatus, StoppingFailure, StoppingResponse}
 import org.apache.flink.runtime.metrics.MetricRegistry
-import org.apache.flink.runtime.taskexecutor.{TaskManagerConfiguration, TaskManagerServices, TaskManagerServicesConfiguration}
+import org.apache.flink.runtime.taskexecutor.{TaskExecutor, TaskManagerConfiguration, TaskManagerServices, TaskManagerServicesConfiguration}
 import org.apache.flink.runtime.taskmanager.{TaskManager, TaskManagerLocation}
 import org.apache.flink.runtime.util.EnvironmentInformation
 
@@ -207,9 +208,9 @@ class LocalFlinkMiniCluster(
     val localExecution = numTaskManagers == 1
 
     val taskManagerActorName = if (singleActorSystem) {
-      TaskManager.TASK_MANAGER_NAME + "_" + (index + 1)
+      TaskExecutor.TASK_MANAGER_NAME + "_" + (index + 1)
     } else {
-      TaskManager.TASK_MANAGER_NAME
+      TaskExecutor.TASK_MANAGER_NAME
     }
 
     val resourceID = ResourceID.generate() // generate random resource id
@@ -397,9 +398,9 @@ class LocalFlinkMiniCluster(
 
   protected def getJobManagerName(index: Int): String = {
     if(singleActorSystem) {
-      JobManager.JOB_MANAGER_NAME + "_" + (index + 1)
+      JobMaster.JOB_MANAGER_NAME + "_" + (index + 1)
     } else {
-      JobManager.JOB_MANAGER_NAME
+      JobMaster.JOB_MANAGER_NAME
     }
   }
 
@@ -413,9 +414,9 @@ class LocalFlinkMiniCluster(
 
   protected def getArchiveName(index: Int): String = {
     if(singleActorSystem) {
-      JobManager.ARCHIVE_NAME + "_" + (index + 1)
+      JobMaster.ARCHIVE_NAME + "_" + (index + 1)
     } else {
-      JobManager.ARCHIVE_NAME
+      JobMaster.ARCHIVE_NAME
     }
   }
 

http://git-wip-us.apache.org/repos/asf/flink/blob/a0bb99c7/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 97e55f0..4065660 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
@@ -64,7 +64,7 @@ import org.apache.flink.runtime.metrics.{MetricRegistry => FlinkMetricRegistry}
 import org.apache.flink.runtime.process.ProcessReaper
 import org.apache.flink.runtime.security.SecurityUtils
 import org.apache.flink.runtime.security.SecurityUtils.SecurityConfiguration
-import org.apache.flink.runtime.taskexecutor.{TaskManagerConfiguration, TaskManagerServices, TaskManagerServicesConfiguration}
+import org.apache.flink.runtime.taskexecutor.{TaskExecutor, TaskManagerConfiguration, TaskManagerServices, TaskManagerServicesConfiguration}
 import org.apache.flink.runtime.util._
 import org.apache.flink.runtime.{FlinkActor, LeaderSessionMessageFilter, LogMessages}
 
@@ -1496,9 +1496,6 @@ object TaskManager {
   /** Return code for critical errors during the runtime */
   val RUNTIME_FAILURE_RETURN_CODE = 2
 
-  /** The name of the TaskManager actor */
-  val TASK_MANAGER_NAME = "taskmanager"
-
   /** Maximum time (milli seconds) that the TaskManager will spend searching for a
     * suitable network interface to use for communication */
   val MAX_STARTUP_CONNECT_TIME = 120000L
@@ -1664,7 +1661,9 @@ object TaskManager {
       LOG.info("Using configured hostname/address for TaskManager: " + taskManagerHostname)
     }
     else {
-      val leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(configuration)
+      val leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(
+        configuration,
+        true)
       val lookupTimeout = AkkaUtils.getLookupTimeout(configuration)
 
       val taskManagerAddress = LeaderRetrievalUtils.findConnectingAddress(
@@ -1780,7 +1779,7 @@ object TaskManager {
         resourceID,
         taskManagerSystem,
         taskManagerHostname,
-        Some(TASK_MANAGER_NAME),
+        Some(TaskExecutor.TASK_MANAGER_NAME),
         None,
         localTaskManagerCommunication = false,
         taskManagerClass)
@@ -1969,38 +1968,4 @@ object TaskManager {
         throw new IOException("Could not connect to TaskManager at " + taskManagerUrl, e)
     }
   }
-
-  // --------------------------------------------------------------------------
-  //  Parsing and checking the TaskManager Configuration
-  // --------------------------------------------------------------------------
-
-  /**
-   * Gets the protocol, hostname and port of the JobManager from the configuration. Also checks that
-   * the hostname is not null and the port non-negative.
-   *
-   * @param configuration The configuration to read the config values from.
-   * @return A 3-tuple (protocol, hostname, port).
-   */
-  def getAndCheckJobManagerAddress(configuration: Configuration) : (String, String, Int) = {
-
-    val protocol = AkkaUtils.getAkkaProtocol(configuration)
-
-    val hostname = configuration.getString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, null)
-
-    val port = configuration.getInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY,
-      ConfigConstants.DEFAULT_JOB_MANAGER_IPC_PORT)
-
-    if (hostname == null) {
-      throw new Exception("Config parameter '" + ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY +
-        "' is missing (hostname/address of JobManager to connect to).")
-    }
-
-    if (port <= 0 || port >= 65536) {
-      throw new Exception("Invalid value for '" + ConfigConstants.JOB_MANAGER_IPC_PORT_KEY +
-        "' (port of the JobManager actor system) : " + port +
-        ".  it must be great than 0 and less than 65536.")
-    }
-
-    (protocol, hostname, port)
-  }
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/a0bb99c7/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/TestingHighAvailabilityServices.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/TestingHighAvailabilityServices.java b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/TestingHighAvailabilityServices.java
index 37a86c7..830dbf9 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/TestingHighAvailabilityServices.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/TestingHighAvailabilityServices.java
@@ -22,7 +22,7 @@ import org.apache.flink.api.common.JobID;
 import org.apache.flink.runtime.blob.BlobStore;
 import org.apache.flink.runtime.blob.VoidBlobStore;
 import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory;
-import org.apache.flink.runtime.highavailability.nonha.NonHaRegistry;
+import org.apache.flink.runtime.highavailability.nonha.standalone.StandaloneRunningJobsRegistry;
 import org.apache.flink.runtime.jobmanager.SubmittedJobGraphStore;
 import org.apache.flink.runtime.leaderelection.LeaderElectionService;
 import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
@@ -147,7 +147,7 @@ public class TestingHighAvailabilityServices implements HighAvailabilityServices
 
 	@Override
 	public RunningJobsRegistry getRunningJobsRegistry() {
-		return new NonHaRegistry();
+		return new StandaloneRunningJobsRegistry();
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/a0bb99c7/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/ZooKeeperRegistryTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/ZooKeeperRegistryTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/ZooKeeperRegistryTest.java
deleted file mode 100644
index b1881e5..0000000
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/ZooKeeperRegistryTest.java
+++ /dev/null
@@ -1,82 +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.highavailability;
-
-import org.apache.curator.test.TestingServer;
-import org.apache.flink.api.common.JobID;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.configuration.HighAvailabilityOptions;
-import org.apache.flink.runtime.concurrent.Executors;
-import org.apache.flink.runtime.highavailability.RunningJobsRegistry.JobSchedulingStatus;
-import org.apache.flink.runtime.util.ZooKeeperUtils;
-import org.apache.flink.util.TestLogger;
-
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-import static org.junit.Assert.assertEquals;
-
-
-public class ZooKeeperRegistryTest extends TestLogger {
-
-	private TestingServer testingServer;
-
-	@Before
-	public void before() throws Exception {
-		testingServer = new TestingServer();
-	}
-
-	@After
-	public void after() throws Exception {
-		testingServer.stop();
-		testingServer = null;
-	}
-
-	/**
-	 * Tests that the function of ZookeeperRegistry, setJobRunning(), setJobFinished(), isJobRunning()
-	 */
-	@Test
-	public void testZooKeeperRegistry() throws Exception {
-		Configuration configuration = new Configuration();
-		configuration.setString(HighAvailabilityOptions.HA_ZOOKEEPER_QUORUM, testingServer.getConnectString());
-		configuration.setString(HighAvailabilityOptions.HA_MODE, "zookeeper");
-
-		final HighAvailabilityServices zkHaService = new ZookeeperHaServices(
-				ZooKeeperUtils.startCuratorFramework(configuration), Executors.directExecutor(), configuration);
-
-		final RunningJobsRegistry zkRegistry = zkHaService.getRunningJobsRegistry();
-
-		try {
-			JobID jobID = JobID.generate();
-			assertEquals(JobSchedulingStatus.PENDING, zkRegistry.getJobSchedulingStatus(jobID));
-
-			zkRegistry.setJobRunning(jobID);
-			assertEquals(JobSchedulingStatus.RUNNING, zkRegistry.getJobSchedulingStatus(jobID));
-
-			zkRegistry.setJobFinished(jobID);
-			assertEquals(JobSchedulingStatus.DONE, zkRegistry.getJobSchedulingStatus(jobID));
-
-			zkRegistry.clearJob(jobID);
-			assertEquals(JobSchedulingStatus.PENDING, zkRegistry.getJobSchedulingStatus(jobID));
-		} finally {
-			zkHaService.close();
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/a0bb99c7/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/leaderelection/SingleLeaderElectionServiceTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/leaderelection/SingleLeaderElectionServiceTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/leaderelection/SingleLeaderElectionServiceTest.java
deleted file mode 100644
index a9805a1..0000000
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/leaderelection/SingleLeaderElectionServiceTest.java
+++ /dev/null
@@ -1,226 +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.highavailability.leaderelection;
-
-import org.apache.flink.runtime.concurrent.Executors;
-import org.apache.flink.runtime.leaderelection.LeaderContender;
-import org.apache.flink.runtime.leaderelection.LeaderElectionService;
-import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener;
-import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
-import org.apache.flink.util.StringUtils;
-
-import org.junit.Test;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
-
-import java.util.Random;
-import java.util.UUID;
-import java.util.concurrent.Executor;
-
-import static org.junit.Assert.fail;
-import static org.mockito.Mockito.*;
-
-/**
- * Tests for the {@link SingleLeaderElectionService}.
- */
-public class SingleLeaderElectionServiceTest {
-
-	private static final Random RND = new Random();
-
-	private final Executor executor = Executors.directExecutor();
-
-	// ------------------------------------------------------------------------
-
-	@Test
-	public void testStartStopAssignLeadership() throws Exception {
-		final UUID uuid = UUID.randomUUID();
-		final SingleLeaderElectionService service = new SingleLeaderElectionService(executor, uuid);
-
-		final LeaderContender contender = mockContender(service);
-		final LeaderContender otherContender = mockContender(service);
-
-		service.start(contender);
-		verify(contender, times(1)).grantLeadership(uuid);
-
-		service.stop();
-		verify(contender, times(1)).revokeLeadership();
-
-		// start with a new contender - the old contender must not gain another leadership
-		service.start(otherContender);
-		verify(otherContender, times(1)).grantLeadership(uuid);
-
-		verify(contender, times(1)).grantLeadership(uuid);
-		verify(contender, times(1)).revokeLeadership();
-	}
-
-	@Test
-	public void testStopBeforeConfirmingLeadership() throws Exception {
-		final UUID uuid = UUID.randomUUID();
-		final SingleLeaderElectionService service = new SingleLeaderElectionService(executor, uuid);
-
-		final LeaderContender contender = mock(LeaderContender.class);
-
-		service.start(contender);
-		verify(contender, times(1)).grantLeadership(uuid);
-
-		service.stop();
-
-		// because the leadership was never confirmed, there is no "revoke" call
-		verifyNoMoreInteractions(contender);
-	}
-
-	@Test
-	public void testStartOnlyOnce() throws Exception {
-		final UUID uuid = UUID.randomUUID();
-		final SingleLeaderElectionService service = new SingleLeaderElectionService(executor, uuid);
-
-		final LeaderContender contender = mock(LeaderContender.class);
-		final LeaderContender otherContender = mock(LeaderContender.class);
-
-		service.start(contender);
-		verify(contender, times(1)).grantLeadership(uuid);
-
-		// should not be possible to start again this with another contender
-		try {
-			service.start(otherContender);
-			fail("should fail with an exception");
-		} catch (IllegalStateException e) {
-			// expected
-		}
-
-		// should not be possible to start this again with the same contender
-		try {
-			service.start(contender);
-			fail("should fail with an exception");
-		} catch (IllegalStateException e) {
-			// expected
-		}
-	}
-
-	@Test
-	public void testShutdown() throws Exception {
-		final UUID uuid = UUID.randomUUID();
-		final SingleLeaderElectionService service = new SingleLeaderElectionService(executor, uuid);
-
-		// create a leader contender and let it grab leadership
-		final LeaderContender contender = mockContender(service);
-		service.start(contender);
-		verify(contender, times(1)).grantLeadership(uuid);
-
-		// some leader listeners
-		final LeaderRetrievalListener listener1 = mock(LeaderRetrievalListener.class);
-		final LeaderRetrievalListener listener2 = mock(LeaderRetrievalListener.class);
-
-		LeaderRetrievalService listenerService1 = service.createLeaderRetrievalService();
-		LeaderRetrievalService listenerService2 = service.createLeaderRetrievalService();
-
-		listenerService1.start(listener1);
-		listenerService2.start(listener2);
-
-		// one listener stops
-		listenerService1.stop();
-
-		// shut down the service
-		service.shutdown();
-
-		// the leader contender and running listener should get error notifications
-		verify(contender, times(1)).handleError(any(Exception.class));
-		verify(listener2, times(1)).handleError(any(Exception.class));
-
-		// the stopped listener gets no notification
-		verify(listener1, times(0)).handleError(any(Exception.class));
-
-		// should not be possible to start again after shutdown
-		try {
-			service.start(contender);
-			fail("should fail with an exception");
-		} catch (IllegalStateException e) {
-			// expected
-		}
-
-		// no additional leadership grant
-		verify(contender, times(1)).grantLeadership(any(UUID.class));
-	}
-
-	@Test
-	public void testImmediateShutdown() throws Exception {
-		final UUID uuid = UUID.randomUUID();
-		final SingleLeaderElectionService service = new SingleLeaderElectionService(executor, uuid);
-		service.shutdown();
-
-		final LeaderContender contender = mock(LeaderContender.class);
-		
-		// should not be possible to start
-		try {
-			service.start(contender);
-			fail("should fail with an exception");
-		} catch (IllegalStateException e) {
-			// expected
-		}
-
-		// no additional leadership grant
-		verify(contender, times(0)).grantLeadership(any(UUID.class));
-	}
-
-//	@Test
-//	public void testNotifyListenersWhenLeaderElected() throws Exception {
-//		final UUID uuid = UUID.randomUUID();
-//		final SingleLeaderElectionService service = new SingleLeaderElectionService(executor, uuid);
-//
-//		final LeaderRetrievalListener listener1 = mock(LeaderRetrievalListener.class);
-//		final LeaderRetrievalListener listener2 = mock(LeaderRetrievalListener.class);
-//
-//		LeaderRetrievalService listenerService1 = service.createLeaderRetrievalService();
-//		LeaderRetrievalService listenerService2 = service.createLeaderRetrievalService();
-//
-//		listenerService1.start(listener1);
-//		listenerService1.start(listener2);
-//
-//		final LeaderContender contender = mockContender(service);
-//		service.start(contender);
-//
-//		veri
-//	}
-
-	// ------------------------------------------------------------------------
-	//  utilities
-	// ------------------------------------------------------------------------
-
-	private static LeaderContender mockContender(final LeaderElectionService service) {
-		String address = StringUtils.getRandomString(RND, 5, 10, 'a', 'z');
-		return mockContender(service, address);
-	}
-
-	private static LeaderContender mockContender(final LeaderElectionService service, final String address) {
-		LeaderContender mockContender = mock(LeaderContender.class);
-
-		when(mockContender.getAddress()).thenReturn(address);
-
-		doAnswer(new Answer<Void>() {
-				@Override
-				public Void answer(InvocationOnMock invocation) throws Throwable {
-					final UUID uuid = (UUID) invocation.getArguments()[0];
-					service.confirmLeaderSessionID(uuid);
-					return null;
-				}
-		}).when(mockContender).grantLeadership(any(UUID.class));
-
-		return mockContender;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/a0bb99c7/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/nonha/embedded/EmbeddedHaServicesTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/nonha/embedded/EmbeddedHaServicesTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/nonha/embedded/EmbeddedHaServicesTest.java
new file mode 100644
index 0000000..7bf9364
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/nonha/embedded/EmbeddedHaServicesTest.java
@@ -0,0 +1,167 @@
+/*
+ * 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.highavailability.nonha.embedded;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.concurrent.Executors;
+import org.apache.flink.runtime.leaderelection.LeaderContender;
+import org.apache.flink.runtime.leaderelection.LeaderElectionService;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+import org.apache.flink.util.TestLogger;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.ArgumentCaptor;
+
+import java.util.UUID;
+
+import static junit.framework.TestCase.assertTrue;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.atLeast;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+/**
+ * Tests for the {@link EmbeddedHaServices}.
+ */
+public class EmbeddedHaServicesTest extends TestLogger {
+
+	private EmbeddedHaServices embeddedHaServices;
+
+	@Before
+	public void setupTest() {
+		embeddedHaServices = new EmbeddedHaServices(Executors.directExecutor());
+	}
+
+	@After
+	public void teardownTest() throws Exception {
+		if (embeddedHaServices != null) {
+			embeddedHaServices.closeAndCleanupAllData();
+			embeddedHaServices = null;
+		}
+	}
+
+	/**
+	 * Tests that exactly one JobManager is elected as the leader for a given job id.
+	 */
+	@Test
+	public void testJobManagerLeaderElection() throws Exception {
+		JobID jobId1 = new JobID();
+		JobID jobId2 = new JobID();
+
+		LeaderContender leaderContender1 = mock(LeaderContender.class);
+		LeaderContender leaderContender2 = mock(LeaderContender.class);
+		LeaderContender leaderContenderDifferentJobId = mock(LeaderContender.class);
+
+		LeaderElectionService leaderElectionService1 = embeddedHaServices.getJobManagerLeaderElectionService(jobId1);
+		LeaderElectionService leaderElectionService2 = embeddedHaServices.getJobManagerLeaderElectionService(jobId1);
+		LeaderElectionService leaderElectionServiceDifferentJobId = embeddedHaServices.getJobManagerLeaderElectionService(jobId2);
+
+		leaderElectionService1.start(leaderContender1);
+		leaderElectionService2.start(leaderContender2);
+		leaderElectionServiceDifferentJobId.start(leaderContenderDifferentJobId);
+
+		ArgumentCaptor<UUID> leaderIdArgumentCaptor1 = ArgumentCaptor.forClass(UUID.class);
+		ArgumentCaptor<UUID> leaderIdArgumentCaptor2 = ArgumentCaptor.forClass(UUID.class);
+		verify(leaderContender1, atLeast(0)).grantLeadership(leaderIdArgumentCaptor1.capture());
+		verify(leaderContender2, atLeast(0)).grantLeadership(leaderIdArgumentCaptor2.capture());
+
+		assertTrue(leaderIdArgumentCaptor1.getAllValues().isEmpty() ^ leaderIdArgumentCaptor2.getAllValues().isEmpty());
+
+		verify(leaderContenderDifferentJobId).grantLeadership(any(UUID.class));
+	}
+
+	/**
+	 * Tests that exactly one ResourceManager is elected as the leader.
+	 */
+	@Test
+	public void testResourceManagerLeaderElection() throws Exception {
+		LeaderContender leaderContender1 = mock(LeaderContender.class);
+		LeaderContender leaderContender2 = mock(LeaderContender.class);
+
+		LeaderElectionService leaderElectionService1 = embeddedHaServices.getResourceManagerLeaderElectionService();
+		LeaderElectionService leaderElectionService2 = embeddedHaServices.getResourceManagerLeaderElectionService();
+
+		leaderElectionService1.start(leaderContender1);
+		leaderElectionService2.start(leaderContender2);
+
+		ArgumentCaptor<UUID> leaderIdArgumentCaptor1 = ArgumentCaptor.forClass(UUID.class);
+		ArgumentCaptor<UUID> leaderIdArgumentCaptor2 = ArgumentCaptor.forClass(UUID.class);
+		verify(leaderContender1, atLeast(0)).grantLeadership(leaderIdArgumentCaptor1.capture());
+		verify(leaderContender2, atLeast(0)).grantLeadership(leaderIdArgumentCaptor2.capture());
+
+		assertTrue(leaderIdArgumentCaptor1.getAllValues().isEmpty() ^ leaderIdArgumentCaptor2.getAllValues().isEmpty());
+	}
+
+	/**
+	 * Tests the JobManager leader retrieval for a given job.
+	 */
+	@Test
+	public void testJobManagerLeaderRetrieval() throws Exception {
+		final String address = "foobar";
+		JobID jobId = new JobID();
+		LeaderRetrievalListener leaderRetrievalListener = mock(LeaderRetrievalListener.class);
+		LeaderContender leaderContender = mock(LeaderContender.class);
+		when(leaderContender.getAddress()).thenReturn(address);
+
+		LeaderElectionService leaderElectionService = embeddedHaServices.getJobManagerLeaderElectionService(jobId);
+		LeaderRetrievalService leaderRetrievalService = embeddedHaServices.getJobManagerLeaderRetriever(jobId);
+
+		leaderRetrievalService.start(leaderRetrievalListener);
+		leaderElectionService.start(leaderContender);
+
+		ArgumentCaptor<UUID> leaderIdArgumentCaptor = ArgumentCaptor.forClass(UUID.class);
+		verify(leaderContender).grantLeadership(leaderIdArgumentCaptor.capture());
+
+		final UUID leaderId = leaderIdArgumentCaptor.getValue();
+
+		leaderElectionService.confirmLeaderSessionID(leaderId);
+
+		verify(leaderRetrievalListener).notifyLeaderAddress(eq(address), eq(leaderId));
+	}
+
+	/**
+	 * Tests the ResourceManager leader retrieval for a given job.
+	 */
+	@Test
+	public void testResourceManagerLeaderRetrieval() throws Exception {
+		final String address = "foobar";
+		LeaderRetrievalListener leaderRetrievalListener = mock(LeaderRetrievalListener.class);
+		LeaderContender leaderContender = mock(LeaderContender.class);
+		when(leaderContender.getAddress()).thenReturn(address);
+
+		LeaderElectionService leaderElectionService = embeddedHaServices.getResourceManagerLeaderElectionService();
+		LeaderRetrievalService leaderRetrievalService = embeddedHaServices.getResourceManagerLeaderRetriever();
+
+		leaderRetrievalService.start(leaderRetrievalListener);
+		leaderElectionService.start(leaderContender);
+
+		ArgumentCaptor<UUID> leaderIdArgumentCaptor = ArgumentCaptor.forClass(UUID.class);
+		verify(leaderContender).grantLeadership(leaderIdArgumentCaptor.capture());
+
+		final UUID leaderId = leaderIdArgumentCaptor.getValue();
+
+		leaderElectionService.confirmLeaderSessionID(leaderId);
+
+		verify(leaderRetrievalListener).notifyLeaderAddress(eq(address), eq(leaderId));
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/a0bb99c7/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/nonha/leaderelection/SingleLeaderElectionServiceTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/nonha/leaderelection/SingleLeaderElectionServiceTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/nonha/leaderelection/SingleLeaderElectionServiceTest.java
new file mode 100644
index 0000000..3875c4f
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/nonha/leaderelection/SingleLeaderElectionServiceTest.java
@@ -0,0 +1,226 @@
+/*
+ * 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.highavailability.nonha.leaderelection;
+
+import org.apache.flink.runtime.concurrent.Executors;
+import org.apache.flink.runtime.leaderelection.LeaderContender;
+import org.apache.flink.runtime.leaderelection.LeaderElectionService;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+import org.apache.flink.util.StringUtils;
+
+import org.junit.Test;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+import java.util.Random;
+import java.util.UUID;
+import java.util.concurrent.Executor;
+
+import static org.junit.Assert.fail;
+import static org.mockito.Mockito.*;
+
+/**
+ * Tests for the {@link SingleLeaderElectionService}.
+ */
+public class SingleLeaderElectionServiceTest {
+
+	private static final Random RND = new Random();
+
+	private final Executor executor = Executors.directExecutor();
+
+	// ------------------------------------------------------------------------
+
+	@Test
+	public void testStartStopAssignLeadership() throws Exception {
+		final UUID uuid = UUID.randomUUID();
+		final SingleLeaderElectionService service = new SingleLeaderElectionService(executor, uuid);
+
+		final LeaderContender contender = mockContender(service);
+		final LeaderContender otherContender = mockContender(service);
+
+		service.start(contender);
+		verify(contender, times(1)).grantLeadership(uuid);
+
+		service.stop();
+		verify(contender, times(1)).revokeLeadership();
+
+		// start with a new contender - the old contender must not gain another leadership
+		service.start(otherContender);
+		verify(otherContender, times(1)).grantLeadership(uuid);
+
+		verify(contender, times(1)).grantLeadership(uuid);
+		verify(contender, times(1)).revokeLeadership();
+	}
+
+	@Test
+	public void testStopBeforeConfirmingLeadership() throws Exception {
+		final UUID uuid = UUID.randomUUID();
+		final SingleLeaderElectionService service = new SingleLeaderElectionService(executor, uuid);
+
+		final LeaderContender contender = mock(LeaderContender.class);
+
+		service.start(contender);
+		verify(contender, times(1)).grantLeadership(uuid);
+
+		service.stop();
+
+		// because the leadership was never confirmed, there is no "revoke" call
+		verifyNoMoreInteractions(contender);
+	}
+
+	@Test
+	public void testStartOnlyOnce() throws Exception {
+		final UUID uuid = UUID.randomUUID();
+		final SingleLeaderElectionService service = new SingleLeaderElectionService(executor, uuid);
+
+		final LeaderContender contender = mock(LeaderContender.class);
+		final LeaderContender otherContender = mock(LeaderContender.class);
+
+		service.start(contender);
+		verify(contender, times(1)).grantLeadership(uuid);
+
+		// should not be possible to start again this with another contender
+		try {
+			service.start(otherContender);
+			fail("should fail with an exception");
+		} catch (IllegalStateException e) {
+			// expected
+		}
+
+		// should not be possible to start this again with the same contender
+		try {
+			service.start(contender);
+			fail("should fail with an exception");
+		} catch (IllegalStateException e) {
+			// expected
+		}
+	}
+
+	@Test
+	public void testShutdown() throws Exception {
+		final UUID uuid = UUID.randomUUID();
+		final SingleLeaderElectionService service = new SingleLeaderElectionService(executor, uuid);
+
+		// create a leader contender and let it grab leadership
+		final LeaderContender contender = mockContender(service);
+		service.start(contender);
+		verify(contender, times(1)).grantLeadership(uuid);
+
+		// some leader listeners
+		final LeaderRetrievalListener listener1 = mock(LeaderRetrievalListener.class);
+		final LeaderRetrievalListener listener2 = mock(LeaderRetrievalListener.class);
+
+		LeaderRetrievalService listenerService1 = service.createLeaderRetrievalService();
+		LeaderRetrievalService listenerService2 = service.createLeaderRetrievalService();
+
+		listenerService1.start(listener1);
+		listenerService2.start(listener2);
+
+		// one listener stops
+		listenerService1.stop();
+
+		// shut down the service
+		service.shutdown();
+
+		// the leader contender and running listener should get error notifications
+		verify(contender, times(1)).handleError(any(Exception.class));
+		verify(listener2, times(1)).handleError(any(Exception.class));
+
+		// the stopped listener gets no notification
+		verify(listener1, times(0)).handleError(any(Exception.class));
+
+		// should not be possible to start again after shutdown
+		try {
+			service.start(contender);
+			fail("should fail with an exception");
+		} catch (IllegalStateException e) {
+			// expected
+		}
+
+		// no additional leadership grant
+		verify(contender, times(1)).grantLeadership(any(UUID.class));
+	}
+
+	@Test
+	public void testImmediateShutdown() throws Exception {
+		final UUID uuid = UUID.randomUUID();
+		final SingleLeaderElectionService service = new SingleLeaderElectionService(executor, uuid);
+		service.shutdown();
+
+		final LeaderContender contender = mock(LeaderContender.class);
+		
+		// should not be possible to start
+		try {
+			service.start(contender);
+			fail("should fail with an exception");
+		} catch (IllegalStateException e) {
+			// expected
+		}
+
+		// no additional leadership grant
+		verify(contender, times(0)).grantLeadership(any(UUID.class));
+	}
+
+//	@Test
+//	public void testNotifyListenersWhenLeaderElected() throws Exception {
+//		final UUID uuid = UUID.randomUUID();
+//		final SingleLeaderElectionService service = new SingleLeaderElectionService(executor, uuid);
+//
+//		final LeaderRetrievalListener listener1 = mock(LeaderRetrievalListener.class);
+//		final LeaderRetrievalListener listener2 = mock(LeaderRetrievalListener.class);
+//
+//		LeaderRetrievalService listenerService1 = service.createLeaderRetrievalService();
+//		LeaderRetrievalService listenerService2 = service.createLeaderRetrievalService();
+//
+//		listenerService1.start(listener1);
+//		listenerService1.start(listener2);
+//
+//		final LeaderContender contender = mockContender(service);
+//		service.start(contender);
+//
+//		veri
+//	}
+
+	// ------------------------------------------------------------------------
+	//  utilities
+	// ------------------------------------------------------------------------
+
+	private static LeaderContender mockContender(final LeaderElectionService service) {
+		String address = StringUtils.getRandomString(RND, 5, 10, 'a', 'z');
+		return mockContender(service, address);
+	}
+
+	private static LeaderContender mockContender(final LeaderElectionService service, final String address) {
+		LeaderContender mockContender = mock(LeaderContender.class);
+
+		when(mockContender.getAddress()).thenReturn(address);
+
+		doAnswer(new Answer<Void>() {
+				@Override
+				public Void answer(InvocationOnMock invocation) throws Throwable {
+					final UUID uuid = (UUID) invocation.getArguments()[0];
+					service.confirmLeaderSessionID(uuid);
+					return null;
+				}
+		}).when(mockContender).grantLeadership(any(UUID.class));
+
+		return mockContender;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/a0bb99c7/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/nonha/standalone/StandaloneHaServicesTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/nonha/standalone/StandaloneHaServicesTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/nonha/standalone/StandaloneHaServicesTest.java
new file mode 100644
index 0000000..537ed20
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/nonha/standalone/StandaloneHaServicesTest.java
@@ -0,0 +1,104 @@
+/*
+ * 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.highavailability.nonha.standalone;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
+import org.apache.flink.runtime.leaderelection.LeaderContender;
+import org.apache.flink.runtime.leaderelection.LeaderElectionService;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+import org.apache.flink.util.TestLogger;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+
+/**
+ * Tests for the {@link StandaloneHaServices}.
+ */
+public class StandaloneHaServicesTest extends TestLogger {
+
+	private final String jobManagerAddress = "jobManager";
+	private final String resourceManagerAddress = "resourceManager";
+
+	private StandaloneHaServices standaloneHaServices;
+
+	@Before
+	public void setupTest() {
+
+		standaloneHaServices = new StandaloneHaServices(resourceManagerAddress, jobManagerAddress);
+	}
+
+	@After
+	public void teardownTest() throws Exception {
+		if (standaloneHaServices != null) {
+			standaloneHaServices.closeAndCleanupAllData();
+			standaloneHaServices = null;
+		}
+	}
+
+	/**
+	 * Tests that the standalone leader election services return a fixed address and leader session
+	 * id.
+	 */
+	@Test
+	public void testLeaderElection() throws Exception {
+		JobID jobId = new JobID();
+		LeaderContender jmLeaderContender = mock(LeaderContender.class);
+		LeaderContender rmLeaderContender = mock(LeaderContender.class);
+
+		LeaderElectionService jmLeaderElectionService = standaloneHaServices.getJobManagerLeaderElectionService(jobId);
+		LeaderElectionService rmLeaderElectionService = standaloneHaServices.getResourceManagerLeaderElectionService();
+
+		jmLeaderElectionService.start(jmLeaderContender);
+		rmLeaderElectionService.start(rmLeaderContender);
+
+		verify(jmLeaderContender).grantLeadership(eq(HighAvailabilityServices.DEFAULT_LEADER_ID));
+		verify(rmLeaderContender).grantLeadership(eq(HighAvailabilityServices.DEFAULT_LEADER_ID));
+	}
+
+	/**
+	 * Tests that the standalone leader retrieval services return the specified address and the
+	 * fixed leader session id.
+	 */
+	@Test
+	public void testJobManagerLeaderRetrieval() throws Exception {
+		JobID jobId1 = new JobID();
+		JobID jobId2 = new JobID();
+		LeaderRetrievalListener jmListener1 = mock(LeaderRetrievalListener.class);
+		LeaderRetrievalListener jmListener2 = mock(LeaderRetrievalListener.class);
+		LeaderRetrievalListener rmListener = mock(LeaderRetrievalListener.class);
+
+		LeaderRetrievalService jmLeaderRetrievalService1 = standaloneHaServices.getJobManagerLeaderRetriever(jobId1);
+		LeaderRetrievalService jmLeaderRetrievalService2 = standaloneHaServices.getJobManagerLeaderRetriever(jobId2);
+		LeaderRetrievalService rmLeaderRetrievalService = standaloneHaServices.getResourceManagerLeaderRetriever();
+
+		jmLeaderRetrievalService1.start(jmListener1);
+		jmLeaderRetrievalService2.start(jmListener2);
+		rmLeaderRetrievalService.start(rmListener);
+
+		verify(jmListener1).notifyLeaderAddress(eq(jobManagerAddress), eq(HighAvailabilityServices.DEFAULT_LEADER_ID));
+		verify(jmListener2).notifyLeaderAddress(eq(jobManagerAddress), eq(HighAvailabilityServices.DEFAULT_LEADER_ID));
+		verify(rmListener).notifyLeaderAddress(eq(resourceManagerAddress), eq(HighAvailabilityServices.DEFAULT_LEADER_ID));
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/a0bb99c7/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/zookeeper/ZooKeeperRegistryTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/zookeeper/ZooKeeperRegistryTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/zookeeper/ZooKeeperRegistryTest.java
new file mode 100644
index 0000000..06ffe3c
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/zookeeper/ZooKeeperRegistryTest.java
@@ -0,0 +1,85 @@
+/*
+ * 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.highavailability.zookeeper;
+
+import org.apache.curator.test.TestingServer;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
+import org.apache.flink.runtime.concurrent.Executors;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
+import org.apache.flink.runtime.highavailability.RunningJobsRegistry;
+import org.apache.flink.runtime.highavailability.RunningJobsRegistry.JobSchedulingStatus;
+import org.apache.flink.runtime.highavailability.zookeeper.ZooKeeperHaServices;
+import org.apache.flink.runtime.util.ZooKeeperUtils;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+
+public class ZooKeeperRegistryTest extends TestLogger {
+
+	private TestingServer testingServer;
+
+	@Before
+	public void before() throws Exception {
+		testingServer = new TestingServer();
+	}
+
+	@After
+	public void after() throws Exception {
+		testingServer.stop();
+		testingServer = null;
+	}
+
+	/**
+	 * Tests that the function of ZookeeperRegistry, setJobRunning(), setJobFinished(), isJobRunning()
+	 */
+	@Test
+	public void testZooKeeperRegistry() throws Exception {
+		Configuration configuration = new Configuration();
+		configuration.setString(HighAvailabilityOptions.HA_ZOOKEEPER_QUORUM, testingServer.getConnectString());
+		configuration.setString(HighAvailabilityOptions.HA_MODE, "zookeeper");
+
+		final HighAvailabilityServices zkHaService = new ZooKeeperHaServices(
+				ZooKeeperUtils.startCuratorFramework(configuration), Executors.directExecutor(), configuration);
+
+		final RunningJobsRegistry zkRegistry = zkHaService.getRunningJobsRegistry();
+
+		try {
+			JobID jobID = JobID.generate();
+			assertEquals(JobSchedulingStatus.PENDING, zkRegistry.getJobSchedulingStatus(jobID));
+
+			zkRegistry.setJobRunning(jobID);
+			assertEquals(JobSchedulingStatus.RUNNING, zkRegistry.getJobSchedulingStatus(jobID));
+
+			zkRegistry.setJobFinished(jobID);
+			assertEquals(JobSchedulingStatus.DONE, zkRegistry.getJobSchedulingStatus(jobID));
+
+			zkRegistry.clearJob(jobID);
+			assertEquals(JobSchedulingStatus.PENDING, zkRegistry.getJobSchedulingStatus(jobID));
+		} finally {
+			zkHaService.close();
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/a0bb99c7/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 49c28e6..2ac3ea7 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
@@ -28,8 +28,10 @@ import akka.actor.ActorSystem;
 import akka.actor.PoisonPill;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.runtime.akka.AkkaUtils;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils;
+import org.apache.flink.runtime.jobmaster.JobMaster;
+import org.apache.flink.runtime.rpc.akka.AkkaRpcServiceUtils;
 import org.apache.flink.runtime.testutils.CommonTestUtils;
-import org.apache.flink.util.NetUtils;
 import org.junit.Test;
 
 import org.apache.flink.configuration.Configuration;
@@ -117,10 +119,14 @@ public class JobManagerProcessReapingTest {
 
 			if (jobManagerPort != -1) {
 				try {
-					jobManagerRef = JobManager.getJobManagerActorRef(
-						"akka.tcp",
-						NetUtils.unresolvedHostAndPortToNormalizedString("localhost", jobManagerPort),
-						localSystem, new FiniteDuration(25, TimeUnit.SECONDS));
+					final String jobManagerAkkaUrl = AkkaRpcServiceUtils.getRpcUrl(
+						"localhost",
+						jobManagerPort,
+						JobMaster.JOB_MANAGER_NAME,
+						HighAvailabilityServicesUtils.AddressResolution.NO_ADDRESS_RESOLUTION,
+						AkkaRpcServiceUtils.AkkaProtocol.TCP);
+
+					jobManagerRef = AkkaUtils.getActorRef(jobManagerAkkaUrl, localSystem, new FiniteDuration(25L, TimeUnit.SECONDS));
 				} catch (Throwable t) {
 					// job manager probably not ready yet
 					lastError = t;

http://git-wip-us.apache.org/repos/asf/flink/blob/a0bb99c7/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 bdff401..fcca173 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
@@ -91,7 +91,7 @@ public class JobSubmitTest {
 			MemoryArchivist.class)._1();
 
 		try {
-			LeaderRetrievalService lrs = LeaderRetrievalUtils.createLeaderRetrievalService(jmConfig);
+			LeaderRetrievalService lrs = LeaderRetrievalUtils.createLeaderRetrievalService(jmConfig, false);
 
 			jmGateway = LeaderRetrievalUtils.retrieveLeaderGateway(
 					lrs,

http://git-wip-us.apache.org/repos/asf/flink/blob/a0bb99c7/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
index 1f1eb62..bc8c0b60 100644
--- 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
@@ -140,7 +140,7 @@ public class ZooKeeperLeaderElectionTest extends TestLogger {
 
 		Deadline deadline = new FiniteDuration(5, TimeUnit.MINUTES).fromNow();
 
-		int num = 20;
+		int num = 10;
 
 		ZooKeeperLeaderElectionService[] leaderElectionService = new ZooKeeperLeaderElectionService[num];
 		TestingContender[] contenders = new TestingContender[num];
@@ -194,7 +194,7 @@ public class ZooKeeperLeaderElectionTest extends TestLogger {
 				}
 			}
 
-			assertFalse(deadline.isOverdue());
+			assertFalse("Did not complete the leader reelection in time.", deadline.isOverdue());
 			assertEquals(num, numberSeenLeaders);
 
 		} finally {

http://git-wip-us.apache.org/repos/asf/flink/blob/a0bb99c7/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
index 6a8ff17..b79093f 100644
--- 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
@@ -22,10 +22,11 @@ import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.test.TestingServer;
 
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.akka.AkkaUtils;
 import org.apache.flink.configuration.HighAvailabilityOptions;
-import org.apache.flink.runtime.jobmanager.JobManager;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils;
+import org.apache.flink.runtime.jobmaster.JobMaster;
 import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+import org.apache.flink.runtime.rpc.akka.AkkaRpcServiceUtils;
 import org.apache.flink.runtime.util.LeaderRetrievalUtils;
 import org.apache.flink.runtime.util.ZooKeeperUtils;
 import org.apache.flink.util.NetUtils;
@@ -35,7 +36,6 @@ import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 
-import scala.Option;
 import scala.concurrent.duration.FiniteDuration;
 
 import java.io.IOException;
@@ -104,10 +104,12 @@ public class ZooKeeperLeaderRetrievalTest extends TestLogger{
 			client[0] = ZooKeeperUtils.startCuratorFramework(config);
 			client[1] = ZooKeeperUtils.startCuratorFramework(config);
 
-			String wrongHostPort = NetUtils.unresolvedHostAndPortToNormalizedString("1.1.1.1", 1234);
-
-			String wrongAddress = JobManager.getRemoteJobManagerAkkaURL(AkkaUtils.getAkkaProtocol(config),
-					wrongHostPort, Option.<String>empty());
+			String wrongAddress = AkkaRpcServiceUtils.getRpcUrl(
+				"1.1.1.1",
+				1234,
+				"foobar",
+				HighAvailabilityServicesUtils.AddressResolution.NO_ADDRESS_RESOLUTION,
+				config);
 
 			try {
 				localHost = InetAddress.getLocalHost();
@@ -126,8 +128,12 @@ public class ZooKeeperLeaderRetrievalTest extends TestLogger{
 			InetSocketAddress correctInetSocketAddress = new InetSocketAddress(localHost, serverSocket.getLocalPort());
 			String hostPort = NetUtils.unresolvedHostAndPortToNormalizedString(localHost.getHostName(), correctInetSocketAddress.getPort());
 
-			String correctAddress = JobManager.getRemoteJobManagerAkkaURL(AkkaUtils.getAkkaProtocol(config),
-				hostPort, Option.<String>empty());
+			String correctAddress = AkkaRpcServiceUtils.getRpcUrl(
+				localHost.getHostName(),
+				correctInetSocketAddress.getPort(),
+				JobMaster.JOB_MANAGER_NAME,
+				HighAvailabilityServicesUtils.AddressResolution.NO_ADDRESS_RESOLUTION,
+				config);
 
 			faultyLeaderElectionService = ZooKeeperUtils.createLeaderElectionService(client[0], config);
 			TestingContender wrongLeaderAddressContender = new TestingContender(wrongAddress, faultyLeaderElectionService);
@@ -192,7 +198,7 @@ public class ZooKeeperLeaderRetrievalTest extends TestLogger{
 
 		FiniteDuration timeout = new FiniteDuration(10, TimeUnit.SECONDS);
 
-		LeaderRetrievalService leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(config);
+		LeaderRetrievalService leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(config, false);
 		InetAddress result = LeaderRetrievalUtils.findConnectingAddress(leaderRetrievalService, timeout);
 
 		assertEquals(InetAddress.getLocalHost(), result);
@@ -214,7 +220,7 @@ public class ZooKeeperLeaderRetrievalTest extends TestLogger{
 		@Override
 		public void run() {
 			try {
-				LeaderRetrievalService leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(config);
+				LeaderRetrievalService leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(config, false);
 				result = LeaderRetrievalUtils.findConnectingAddress(leaderRetrievalService, timeout);
 			} catch (Exception e) {
 				exception = e;

http://git-wip-us.apache.org/repos/asf/flink/blob/a0bb99c7/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java
index d3d4d43..cc79c5d 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java
@@ -44,7 +44,7 @@ import org.apache.flink.runtime.heartbeat.HeartbeatListener;
 import org.apache.flink.runtime.heartbeat.HeartbeatManagerImpl;
 import org.apache.flink.runtime.heartbeat.HeartbeatServices;
 import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
-import org.apache.flink.runtime.highavailability.NonHaServices;
+import org.apache.flink.runtime.highavailability.nonha.standalone.StandaloneHaServices;
 import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices;
 import org.apache.flink.runtime.instance.InstanceID;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
@@ -317,6 +317,7 @@ public class TaskExecutorTest extends TestLogger {
 		final ResourceID resourceID = ResourceID.generate();
 		final String resourceManagerAddress = "/resource/manager/address/one";
 		final ResourceID resourceManagerResourceId = new ResourceID(resourceManagerAddress);
+		final String jobManagerAddress = "localhost";
 
 		final TestingSerialRpcService rpc = new TestingSerialRpcService();
 		try {
@@ -335,7 +336,9 @@ public class TaskExecutorTest extends TestLogger {
 			TaskManagerLocation taskManagerLocation = mock(TaskManagerLocation.class);
 			when(taskManagerLocation.getResourceID()).thenReturn(resourceID);
 
-			NonHaServices haServices = new NonHaServices(resourceManagerAddress);
+			StandaloneHaServices haServices = new StandaloneHaServices(
+				resourceManagerAddress,
+				jobManagerAddress);
 
 			final TaskSlotTable taskSlotTable = mock(TaskSlotTable.class);
 			final SlotReport slotReport = new SlotReport();

http://git-wip-us.apache.org/repos/asf/flink/blob/a0bb99c7/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerProcessReapingTestBase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerProcessReapingTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerProcessReapingTestBase.java
index 2fafe5b..d904004 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerProcessReapingTestBase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerProcessReapingTestBase.java
@@ -30,6 +30,7 @@ import org.apache.flink.runtime.clusterframework.types.ResourceID;
 import org.apache.flink.runtime.jobmanager.JobManager;
 import org.apache.flink.runtime.jobmanager.MemoryArchivist;
 import org.apache.flink.runtime.leaderretrieval.StandaloneLeaderRetrievalService;
+import org.apache.flink.runtime.taskexecutor.TaskExecutor;
 import org.apache.flink.runtime.testingUtils.TestingUtils;
 import org.apache.flink.runtime.testutils.CommonTestUtils;
 import org.apache.flink.util.NetUtils;
@@ -138,7 +139,7 @@ public abstract class TaskManagerProcessReapingTestBase {
 			// is started and the TaskManager is up
 			String taskManagerActorName = String.format("akka.tcp://flink@%s/user/%s",
 					"localhost:" + taskManagerPort,
-					TaskManager.TASK_MANAGER_NAME());
+					TaskExecutor.TASK_MANAGER_NAME);
 
 			ActorRef taskManagerRef = null;
 			Throwable lastError = null;

http://git-wip-us.apache.org/repos/asf/flink/blob/a0bb99c7/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 f3b1d4a..5f35229 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
@@ -33,6 +33,7 @@ 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.jobmanager.MemoryArchivist;
+import org.apache.flink.runtime.jobmaster.JobMaster;
 import org.apache.flink.runtime.leaderretrieval.StandaloneLeaderRetrievalService;
 import org.apache.flink.runtime.messages.JobManagerMessages;
 import org.apache.flink.runtime.messages.JobManagerMessages.LeaderSessionMessage;
@@ -182,7 +183,7 @@ public class TaskManagerRegistrationTest extends TestLogger {
 				// available. we give it the regular JobManager akka URL
 				taskManager = createTaskManager(
 						actorSystem,
-						JobManager.getLocalJobManagerAkkaURL(Option.<String>empty()),
+						AkkaUtils.getLocalAkkaURL(JobMaster.JOB_MANAGER_NAME),
 						new Configuration(),
 						true,
 						false);
@@ -248,7 +249,7 @@ public class TaskManagerRegistrationTest extends TestLogger {
 				// start the taskManager actor
 				taskManager = createTaskManager(
 						actorSystem,
-						JobManager.getLocalJobManagerAkkaURL(Option.<String>empty()),
+						AkkaUtils.getLocalAkkaURL(JobMaster.JOB_MANAGER_NAME),
 						tmConfig,
 						true,
 						false);

http://git-wip-us.apache.org/repos/asf/flink/blob/a0bb99c7/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/JobManagerProcess.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/JobManagerProcess.java b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/JobManagerProcess.java
index 48c65c0..6e1fb74 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/JobManagerProcess.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/JobManagerProcess.java
@@ -23,15 +23,17 @@ import akka.actor.ActorSystem;
 import org.apache.flink.api.java.utils.ParameterTool;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.akka.AkkaUtils;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils;
 import org.apache.flink.runtime.jobmanager.JobManager;
 import org.apache.flink.runtime.jobmanager.JobManagerMode;
-import org.apache.flink.util.NetUtils;
+import org.apache.flink.runtime.jobmaster.JobMaster;
+import org.apache.flink.runtime.rpc.akka.AkkaRpcServiceUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import scala.Option;
 import scala.concurrent.duration.Deadline;
 import scala.concurrent.duration.FiniteDuration;
 
+import java.net.UnknownHostException;
 import java.util.ArrayList;
 import java.util.Map;
 import java.util.concurrent.CountDownLatch;
@@ -143,13 +145,15 @@ public class JobManagerProcess extends TestJvmProcess {
 	/**
 	 * Returns the Akka URL of this JobManager.
 	 */
-	public String getJobManagerAkkaURL(FiniteDuration timeout) throws InterruptedException {
+	public String getJobManagerAkkaURL(FiniteDuration timeout) throws InterruptedException, UnknownHostException {
 		int port = getJobManagerPort(timeout);
 
-		return JobManager.getRemoteJobManagerAkkaURL(
-				AkkaUtils.getAkkaProtocol(config),
-				NetUtils.unresolvedHostAndPortToNormalizedString("localhost", port),
-				Option.<String>empty());
+		return AkkaRpcServiceUtils.getRpcUrl(
+			"localhost",
+			port,
+			JobMaster.JOB_MANAGER_NAME,
+			HighAvailabilityServicesUtils.AddressResolution.NO_ADDRESS_RESOLUTION,
+			config);
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/a0bb99c7/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
index 0daac2e..2404fb9 100644
--- 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
@@ -20,7 +20,10 @@ package org.apache.flink.runtime.akka
 
 import java.net.InetSocketAddress
 
+import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils.AddressResolution
 import org.apache.flink.runtime.jobmanager.JobManager
+import org.apache.flink.runtime.rpc.akka.AkkaRpcServiceUtils
+import org.apache.flink.runtime.rpc.akka.AkkaRpcServiceUtils.AkkaProtocol
 import org.apache.flink.util.NetUtils
 import org.junit.runner.RunWith
 import org.scalatest.{BeforeAndAfterAll, FunSuite, Matchers}
@@ -38,19 +41,21 @@ class AkkaUtilsTest
 
     val address = new InetSocketAddress(host, port)
 
-    val remoteAkkaURL = JobManager.getRemoteJobManagerAkkaURL(
-      "akka.tcp",
-      NetUtils.unresolvedHostAndPortToNormalizedString(host, port),
-      Some("actor"))
+    val remoteAkkaUrl = AkkaRpcServiceUtils.getRpcUrl(
+      host,
+      port,
+      "actor",
+      AddressResolution.NO_ADDRESS_RESOLUTION,
+      AkkaProtocol.TCP)
 
-    val result = AkkaUtils.getInetSockeAddressFromAkkaURL(remoteAkkaURL)
+    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"))
+    val localAkkaURL = AkkaUtils.getLocalAkkaURL("actor")
 
     intercept[Exception] {
       AkkaUtils.getInetSockeAddressFromAkkaURL(localAkkaURL)

http://git-wip-us.apache.org/repos/asf/flink/blob/a0bb99c7/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 1489fb2..97a001d 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,10 @@ class JobManagerConnectionTest {
       mustReturnWithinTimeout(Duration(5*timeout, TimeUnit.MILLISECONDS)) {
         () => {
           try {
-            JobManager.getJobManagerActorRef(endpoint, actorSystem, config)
+            AkkaUtils.getActorRef(
+              endpoint,
+              actorSystem,
+              AkkaUtils.getLookupTimeout(config))
             fail("Should fail since the JobManager is not reachable")
           }
           catch {
@@ -95,7 +98,10 @@ class JobManagerConnectionTest {
       mustReturnWithinTimeout(Duration(5*timeout, TimeUnit.MILLISECONDS)) {
         () => {
           try {
-            JobManager.getJobManagerActorRef(endpoint, actorSystem, config)
+            AkkaUtils.getActorRef(
+              endpoint,
+              actorSystem,
+              AkkaUtils.getLookupTimeout(config))
             fail("Should fail since the JobManager is not reachable")
           }
           catch {

http://git-wip-us.apache.org/repos/asf/flink/blob/a0bb99c7/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 876e26b..d139a3f 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
@@ -35,10 +35,12 @@ import org.apache.flink.runtime.client.JobClient
 import org.apache.flink.runtime.clusterframework.FlinkResourceManager
 import org.apache.flink.runtime.clusterframework.types.ResourceID
 import org.apache.flink.runtime.concurrent.{ScheduledExecutor, ScheduledExecutorServiceAdapter}
-import org.apache.flink.runtime.highavailability.HighAvailabilityServices
+import org.apache.flink.runtime.highavailability.{HighAvailabilityServices, HighAvailabilityServicesUtils}
 import org.apache.flink.runtime.instance.{ActorGateway, AkkaActorGateway}
 import org.apache.flink.runtime.jobgraph.JobGraph
 import org.apache.flink.runtime.jobmanager.{JobManager, MemoryArchivist}
+import org.apache.flink.runtime.jobmaster.JobMaster
+import org.apache.flink.runtime.jobmaster.JobMaster.{ARCHIVE_NAME, JOB_MANAGER_NAME}
 import org.apache.flink.runtime.leaderretrieval.StandaloneLeaderRetrievalService
 import org.apache.flink.runtime.messages.TaskManagerMessages.NotifyWhenRegisteredAtJobManager
 import org.apache.flink.runtime.taskmanager.TaskManager
@@ -516,8 +518,8 @@ object TestingUtils {
         actorSystem,
         futureExecutor,
         ioExecutor,
-        Some(prefix + JobManager.JOB_MANAGER_NAME),
-        Some(prefix + JobManager.ARCHIVE_NAME),
+        Some(prefix + JobMaster.JOB_MANAGER_NAME),
+        Some(prefix + JobMaster.ARCHIVE_NAME),
         jobManagerClass,
         classOf[MemoryArchivist])
 
@@ -605,7 +607,6 @@ object TestingUtils {
     new AkkaActorGateway(actor, HighAvailabilityServices.DEFAULT_LEADER_ID)
   }
 
-
   class ForwardingActor(val target: ActorRef, val leaderSessionID: Option[UUID])
     extends FlinkActor with LeaderSessionMessageFilter with LogMessages {
 

http://git-wip-us.apache.org/repos/asf/flink/blob/a0bb99c7/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
index df4f370..5eadba6 100644
--- 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
@@ -275,7 +275,8 @@ public class ZooKeeperLeaderElectionITCase extends TestLogger {
 			try {
 				LeaderRetrievalService lrService =
 						LeaderRetrievalUtils.createLeaderRetrievalService(
-								cluster.configuration());
+							cluster.configuration(),
+							false);
 
 				JobExecutionResult result = JobClient.submitJobAndWait(
 						clientActorSystem,


[10/16] flink git commit: [FLINK-6078] Remove CuratorFramework#close calls from ZooKeeper based HA services

Posted by tr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/flink-libraries/flink-python/src/test/resources/log4j-test.properties
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-python/src/test/resources/log4j-test.properties b/flink-libraries/flink-python/src/test/resources/log4j-test.properties
new file mode 100644
index 0000000..18b51cc
--- /dev/null
+++ b/flink-libraries/flink-python/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.log4j.PatternLayout
+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/ddd6a99a/flink-libraries/flink-python/src/test/resources/logback-test.xml
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-python/src/test/resources/logback-test.xml b/flink-libraries/flink-python/src/test/resources/logback-test.xml
new file mode 100644
index 0000000..1d64d46
--- /dev/null
+++ b/flink-libraries/flink-python/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.BatchTask" 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>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosApplicationMasterRunner.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosApplicationMasterRunner.java b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosApplicationMasterRunner.java
index 0c12745..5513df4 100644
--- a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosApplicationMasterRunner.java
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosApplicationMasterRunner.java
@@ -48,17 +48,17 @@ import org.apache.flink.runtime.clusterframework.overlays.HadoopUserOverlay;
 import org.apache.flink.runtime.clusterframework.overlays.KeytabOverlay;
 import org.apache.flink.runtime.clusterframework.overlays.Krb5ConfOverlay;
 import org.apache.flink.runtime.clusterframework.overlays.SSLStoreOverlay;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils;
 import org.apache.flink.runtime.jobmanager.JobManager;
 import org.apache.flink.runtime.jobmanager.MemoryArchivist;
 import org.apache.flink.runtime.jobmaster.JobMaster;
-import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
 import org.apache.flink.runtime.process.ProcessReaper;
 import org.apache.flink.runtime.security.SecurityUtils;
 import org.apache.flink.runtime.util.EnvironmentInformation;
 import org.apache.flink.runtime.util.ExecutorThreadFactory;
 import org.apache.flink.runtime.util.Hardware;
 import org.apache.flink.runtime.util.JvmShutdownSafeguard;
-import org.apache.flink.runtime.util.LeaderRetrievalUtils;
 import org.apache.flink.runtime.util.SignalHandler;
 import org.apache.flink.runtime.webmonitor.WebMonitor;
 import org.apache.mesos.Protos;
@@ -67,7 +67,6 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import scala.Option;
-import scala.Some;
 import scala.concurrent.duration.Duration;
 import scala.concurrent.duration.FiniteDuration;
 
@@ -206,6 +205,7 @@ public class MesosApplicationMasterRunner {
 		ScheduledExecutorService futureExecutor = null;
 		ExecutorService ioExecutor = null;
 		MesosServices mesosServices = null;
+		HighAvailabilityServices highAvailabilityServices = null;
 
 		try {
 			// ------- (1) load and parse / validate all configurations -------
@@ -295,6 +295,12 @@ public class MesosApplicationMasterRunner {
 			// 3) Resource Master for Mesos
 			// 4) Process reapers for the JobManager and Resource Master
 
+			// 0: Start the JobManager services
+			highAvailabilityServices = HighAvailabilityServicesUtils.createHighAvailabilityServices(
+				config,
+				ioExecutor,
+				HighAvailabilityServicesUtils.AddressResolution.NO_ADDRESS_RESOLUTION);
+
 			// 1: the JobManager
 			LOG.debug("Starting JobManager actor");
 
@@ -304,8 +310,9 @@ public class MesosApplicationMasterRunner {
 				actorSystem,
 				futureExecutor,
 				ioExecutor,
-				new Some<>(JobMaster.JOB_MANAGER_NAME),
-				Option.<String>empty(),
+				highAvailabilityServices,
+				Option.apply(JobMaster.JOB_MANAGER_NAME),
+				Option.apply(JobMaster.ARCHIVE_NAME),
 				getJobManagerClass(),
 				getArchivistClass())._1();
 
@@ -313,7 +320,12 @@ public class MesosApplicationMasterRunner {
 			// 2: the web monitor
 			LOG.debug("Starting Web Frontend");
 
-			webMonitor = BootstrapTools.startWebMonitorIfConfigured(config, actorSystem, jobManager, LOG);
+			webMonitor = BootstrapTools.startWebMonitorIfConfigured(
+				config,
+				highAvailabilityServices,
+				actorSystem,
+				jobManager,
+				LOG);
 			if(webMonitor != null) {
 				final URL webMonitorURL = new URL("http", appMasterHostname, webMonitor.getServerPort(), "/");
 				mesosConfig.frameworkInfo().setWebuiUrl(webMonitorURL.toExternalForm());
@@ -327,17 +339,12 @@ public class MesosApplicationMasterRunner {
 				config,
 				ioExecutor);
 
-			// we need the leader retrieval service here to be informed of new
-			// leader session IDs, even though there can be only one leader ever
-			LeaderRetrievalService leaderRetriever =
-				LeaderRetrievalUtils.createLeaderRetrievalService(config, jobManager);
-
 			Props resourceMasterProps = MesosFlinkResourceManager.createActorProps(
 				getResourceManagerClass(),
 				config,
 				mesosConfig,
 				workerStore,
-				leaderRetriever,
+				highAvailabilityServices.getJobManagerLeaderRetriever(HighAvailabilityServices.DEFAULT_JOB_ID),
 				taskManagerParameters,
 				taskManagerContainerSpec,
 				artifactServer,
@@ -435,6 +442,14 @@ public class MesosApplicationMasterRunner {
 			LOG.error("Failed to stop the artifact server", t);
 		}
 
+		if (highAvailabilityServices != null) {
+			try {
+				highAvailabilityServices.close();
+			} catch (Throwable t) {
+				LOG.error("Could not properly stop the high availability services.");
+			}
+		}
+
 		org.apache.flink.runtime.concurrent.Executors.gracefulShutdown(
 			AkkaUtils.getTimeout(config).toMillis(),
 			TimeUnit.MILLISECONDS,

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/flink-mesos/src/test/java/org/apache/flink/mesos/runtime/clusterframework/MesosFlinkResourceManagerTest.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/test/java/org/apache/flink/mesos/runtime/clusterframework/MesosFlinkResourceManagerTest.java b/flink-mesos/src/test/java/org/apache/flink/mesos/runtime/clusterframework/MesosFlinkResourceManagerTest.java
index 7915e8a..7fe5db5 100644
--- a/flink-mesos/src/test/java/org/apache/flink/mesos/runtime/clusterframework/MesosFlinkResourceManagerTest.java
+++ b/flink-mesos/src/test/java/org/apache/flink/mesos/runtime/clusterframework/MesosFlinkResourceManagerTest.java
@@ -43,11 +43,12 @@ import org.apache.flink.runtime.clusterframework.ContainerSpecification;
 import org.apache.flink.runtime.clusterframework.messages.*;
 import org.apache.flink.runtime.clusterframework.types.ResourceID;
 import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
+import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices;
 import org.apache.flink.runtime.instance.ActorGateway;
 import org.apache.flink.runtime.instance.AkkaActorGateway;
+import org.apache.flink.runtime.leaderelection.TestingLeaderRetrievalService;
 import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
 import org.apache.flink.runtime.testingUtils.TestingUtils;
-import org.apache.flink.runtime.util.LeaderRetrievalUtils;
 import org.apache.flink.util.TestLogger;
 import org.apache.mesos.SchedulerDriver;
 import org.apache.mesos.Protos;
@@ -142,11 +143,10 @@ public class MesosFlinkResourceManagerTest extends TestLogger {
 	/**
 	 * The context fixture.
 	 */
-	static class Context extends JavaTestKit {
+	static class Context extends JavaTestKit implements AutoCloseable {
 
 		// mocks
 		public ActorGateway jobManager;
-		public LeaderRetrievalService retrievalService;
 		public MesosConfiguration mesosConfig;
 		public MesosWorkerStore workerStore;
 		public MesosArtifactResolver artifactResolver;
@@ -163,6 +163,8 @@ public class MesosFlinkResourceManagerTest extends TestLogger {
 		public Protos.TaskID task2 = Protos.TaskID.newBuilder().setValue("taskmanager-00002").build();
 		public Protos.TaskID task3 = Protos.TaskID.newBuilder().setValue("taskmanager-00003").build();
 
+		private final TestingHighAvailabilityServices highAvailabilityServices;
+
 		/**
 		 * Create mock RM dependencies.
 		 */
@@ -170,8 +172,19 @@ public class MesosFlinkResourceManagerTest extends TestLogger {
 			super(system);
 
 			try {
-				jobManager = TestingUtils.createForwardingActor(system, getTestActor(), Option.<String>empty());
-				retrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(config, jobManager.actor());
+				jobManager = TestingUtils.createForwardingActor(
+					system,
+					getTestActor(),
+					HighAvailabilityServices.DEFAULT_LEADER_ID,
+					Option.<String>empty());
+
+				highAvailabilityServices = new TestingHighAvailabilityServices();
+
+				highAvailabilityServices.setJobMasterLeaderRetriever(
+					HighAvailabilityServices.DEFAULT_JOB_ID,
+					new TestingLeaderRetrievalService(
+						jobManager.path(),
+						HighAvailabilityServices.DEFAULT_LEADER_ID));
 
 				// scheduler driver
 				schedulerDriver = mock(SchedulerDriver.class);
@@ -206,7 +219,14 @@ public class MesosFlinkResourceManagerTest extends TestLogger {
 			TestActorRef<TestingMesosFlinkResourceManager> resourceManagerRef =
 				TestActorRef.create(system, MesosFlinkResourceManager.createActorProps(
 					TestingMesosFlinkResourceManager.class,
-					config, mesosConfig, workerStore, retrievalService, tmParams, containerSpecification, artifactResolver, LOG));
+					config,
+					mesosConfig,
+					workerStore,
+					highAvailabilityServices.getJobManagerLeaderRetriever(HighAvailabilityServices.DEFAULT_JOB_ID),
+					tmParams,
+					containerSpecification,
+					artifactResolver,
+					LOG));
 			resourceManagerInstance = resourceManagerRef.underlyingActor();
 			resourceManager = new AkkaActorGateway(resourceManagerRef, HighAvailabilityServices.DEFAULT_LEADER_ID);
 
@@ -235,6 +255,11 @@ public class MesosFlinkResourceManagerTest extends TestLogger {
 				.setLaunch(Protos.Offer.Operation.Launch.newBuilder().addAllTaskInfos(Arrays.asList(taskInfo))
 				).build();
 		}
+
+		@Override
+		public void close() throws Exception {
+			highAvailabilityServices.closeAndCleanupAllData();
+		}
 	}
 
 	/**

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/BackPressureStatsTrackerITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/BackPressureStatsTrackerITCase.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/BackPressureStatsTrackerITCase.java
index e80c509..f31c932 100644
--- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/BackPressureStatsTrackerITCase.java
+++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/BackPressureStatsTrackerITCase.java
@@ -29,6 +29,7 @@ import org.apache.flink.runtime.client.JobClient;
 import org.apache.flink.runtime.executiongraph.ExecutionGraph;
 import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
 import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils;
 import org.apache.flink.runtime.instance.ActorGateway;
 import org.apache.flink.runtime.instance.AkkaActorGateway;
 import org.apache.flink.runtime.io.network.buffer.Buffer;
@@ -103,6 +104,12 @@ public class BackPressureStatsTrackerITCase extends TestLogger {
 
 			jobGraph.addVertex(task);
 
+			final Configuration config = new Configuration();
+
+			final HighAvailabilityServices highAvailabilityServices = HighAvailabilityServicesUtils.createAvailableOrEmbeddedServices(
+				config,
+				TestingUtils.defaultExecutor());
+
 			ActorGateway jobManger = null;
 			ActorGateway taskManager = null;
 
@@ -125,13 +132,17 @@ public class BackPressureStatsTrackerITCase extends TestLogger {
 					testActorSystem,
 					TestingUtils.defaultExecutor(),
 					TestingUtils.defaultExecutor(),
-					new Configuration());
+					config,
+					highAvailabilityServices);
 
-				final Configuration config = new Configuration();
 				config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, parallelism);
 
 				taskManager = TestingUtils.createTaskManager(
-						testActorSystem, jobManger, config, true, true);
+					testActorSystem,
+					highAvailabilityServices,
+					config,
+					true,
+					true);
 
 				final ActorGateway jm = jobManger;
 
@@ -264,6 +275,8 @@ public class BackPressureStatsTrackerITCase extends TestLogger {
 				TestingUtils.stopActor(jobManger);
 				TestingUtils.stopActor(taskManager);
 
+				highAvailabilityServices.closeAndCleanupAllData();
+
 				for (Buffer buf : buffers) {
 					buf.recycle();
 				}

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/StackTraceSampleCoordinatorITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/StackTraceSampleCoordinatorITCase.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/StackTraceSampleCoordinatorITCase.java
index 5463384..b67e735 100644
--- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/StackTraceSampleCoordinatorITCase.java
+++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/StackTraceSampleCoordinatorITCase.java
@@ -29,6 +29,7 @@ import org.apache.flink.runtime.concurrent.Future;
 import org.apache.flink.runtime.executiongraph.ExecutionGraph;
 import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
 import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils;
 import org.apache.flink.runtime.instance.ActorGateway;
 import org.apache.flink.runtime.instance.AkkaActorGateway;
 import org.apache.flink.runtime.jobgraph.JobGraph;
@@ -89,6 +90,12 @@ public class StackTraceSampleCoordinatorITCase extends TestLogger {
 
 			jobGraph.addVertex(task);
 
+			final Configuration config = new Configuration();
+
+			final HighAvailabilityServices highAvailabilityServices = HighAvailabilityServicesUtils.createAvailableOrEmbeddedServices(
+				config,
+				TestingUtils.defaultExecutor());
+
 			ActorGateway jobManger = null;
 			ActorGateway taskManager = null;
 
@@ -97,13 +104,17 @@ public class StackTraceSampleCoordinatorITCase extends TestLogger {
 					testActorSystem,
 					TestingUtils.defaultExecutor(),
 					TestingUtils.defaultExecutor(),
-					new Configuration());
+					config,
+					highAvailabilityServices);
 
-				final Configuration config = new Configuration();
 				config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, parallelism);
 
 				taskManager = TestingUtils.createTaskManager(
-						testActorSystem, jobManger, config, true, true);
+					testActorSystem,
+					highAvailabilityServices,
+					config,
+					true,
+					true);
 
 				final ActorGateway jm = jobManger;
 
@@ -182,6 +193,8 @@ public class StackTraceSampleCoordinatorITCase extends TestLogger {
 			} finally {
 				TestingUtils.stopActor(jobManger);
 				TestingUtils.stopActor(taskManager);
+
+				highAvailabilityServices.closeAndCleanupAllData();
 			}
 		}};
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitorITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitorITCase.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitorITCase.java
index 8af1f46..5ccfe90 100644
--- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitorITCase.java
+++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitorITCase.java
@@ -26,6 +26,8 @@ import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.HighAvailabilityOptions;
 import org.apache.flink.runtime.akka.AkkaUtils;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils;
 import org.apache.flink.runtime.jobmanager.JobManager;
 import org.apache.flink.runtime.jobmanager.MemoryArchivist;
 import org.apache.flink.runtime.leaderelection.TestingListener;
@@ -33,7 +35,6 @@ import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
 import org.apache.flink.runtime.testingUtils.TestingCluster;
 import org.apache.flink.runtime.testingUtils.TestingUtils;
 import org.apache.flink.runtime.testutils.ZooKeeperTestUtils;
-import org.apache.flink.runtime.util.ZooKeeperUtils;
 import org.apache.flink.runtime.webmonitor.files.MimeTypes;
 import org.apache.flink.runtime.webmonitor.testutils.HttpTestClient;
 import org.apache.flink.util.TestLogger;
@@ -49,8 +50,6 @@ import scala.concurrent.duration.FiniteDuration;
 import java.io.File;
 import java.nio.file.Files;
 import java.nio.file.Path;
-import java.util.ArrayList;
-import java.util.List;
 import java.util.Scanner;
 import java.util.UUID;
 import java.util.concurrent.TimeUnit;
@@ -127,7 +126,7 @@ public class WebRuntimeMonitorITCase extends TestLogger {
 
 		ActorSystem[] jobManagerSystem = new ActorSystem[2];
 		WebRuntimeMonitor[] webMonitor = new WebRuntimeMonitor[2];
-		List<LeaderRetrievalService> leaderRetrievalServices = new ArrayList<>();
+		HighAvailabilityServices highAvailabilityServices = null;
 
 		try (TestingServer zooKeeper = new TestingServer()) {
 			final Configuration config = ZooKeeperTestUtils.createZooKeeperHAConfig(
@@ -141,15 +140,20 @@ public class WebRuntimeMonitorITCase extends TestLogger {
 			config.setInteger(ConfigConstants.JOB_MANAGER_WEB_PORT_KEY, 0);
 			config.setString(ConfigConstants.JOB_MANAGER_WEB_LOG_PATH_KEY, logFile.toString());
 
+			highAvailabilityServices = HighAvailabilityServicesUtils.createAvailableOrEmbeddedServices(
+				config,
+				TestingUtils.defaultExecutor());
+
 			for (int i = 0; i < jobManagerSystem.length; i++) {
 				jobManagerSystem[i] = AkkaUtils.createActorSystem(new Configuration(),
 						new Some<>(new Tuple2<String, Object>("localhost", 0)));
 			}
 
 			for (int i = 0; i < webMonitor.length; i++) {
-				LeaderRetrievalService lrs = ZooKeeperUtils.createLeaderRetrievalService(config);
-				leaderRetrievalServices.add(lrs);
-				webMonitor[i] = new WebRuntimeMonitor(config, lrs, jobManagerSystem[i]);
+				webMonitor[i] = new WebRuntimeMonitor(
+					config,
+					highAvailabilityServices.getJobManagerLeaderRetriever(HighAvailabilityServices.DEFAULT_JOB_ID),
+					jobManagerSystem[i]);
 			}
 
 			ActorRef[] jobManager = new ActorRef[2];
@@ -164,6 +168,7 @@ public class WebRuntimeMonitorITCase extends TestLogger {
 					jobManagerSystem[i],
 					TestingUtils.defaultExecutor(),
 					TestingUtils.defaultExecutor(),
+					highAvailabilityServices,
 					JobManager.class,
 					MemoryArchivist.class)._1();
 
@@ -171,8 +176,7 @@ public class WebRuntimeMonitorITCase extends TestLogger {
 				webMonitor[i].start(jobManagerAddress[i]);
 			}
 
-			LeaderRetrievalService lrs = ZooKeeperUtils.createLeaderRetrievalService(config);
-			leaderRetrievalServices.add(lrs);
+			LeaderRetrievalService lrs = highAvailabilityServices.getJobManagerLeaderRetriever(HighAvailabilityServices.DEFAULT_JOB_ID);
 			TestingListener leaderListener = new TestingListener();
 			lrs.start(leaderListener);
 
@@ -247,6 +251,8 @@ public class WebRuntimeMonitorITCase extends TestLogger {
 				assertEquals(response.getType(), MimeTypes.getMimeTypeForExtension("json"));
 				assertTrue(response.getContent().contains("\"taskmanagers\":1") ||
 						response.getContent().contains("\"taskmanagers\":0"));
+			} finally {
+				lrs.stop();
 			}
 		}
 		finally {
@@ -260,8 +266,8 @@ public class WebRuntimeMonitorITCase extends TestLogger {
 				monitor.stop();
 			}
 
-			for (LeaderRetrievalService lrs : leaderRetrievalServices) {
-				lrs.stop();
+			if (highAvailabilityServices != null) {
+				highAvailabilityServices.closeAndCleanupAllData();
 			}
 		}
 	}
@@ -462,7 +468,8 @@ public class WebRuntimeMonitorITCase extends TestLogger {
 
 		WebRuntimeMonitor webMonitor = new WebRuntimeMonitor(
 			config,
-			flink.createLeaderRetrievalService(),
+			flink.highAvailabilityServices().getJobManagerLeaderRetriever(
+				HighAvailabilityServices.DEFAULT_JOB_ID),
 			jmActorSystem);
 
 		webMonitor.start(jobManagerAddress);

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCheckpointRecoveryFactory.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCheckpointRecoveryFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCheckpointRecoveryFactory.java
index 481559b..d60b26f 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCheckpointRecoveryFactory.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCheckpointRecoveryFactory.java
@@ -55,7 +55,7 @@ public class ZooKeeperCheckpointRecoveryFactory implements CheckpointRecoveryFac
 
 	@Override
 	public void stop() {
-		client.close();
+		// Nothing to do
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/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 76d6d86..b570383 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
@@ -34,9 +34,9 @@ import org.apache.flink.runtime.akka.ListeningBehaviour;
 import org.apache.flink.runtime.blob.BlobCache;
 import org.apache.flink.runtime.blob.BlobKey;
 import org.apache.flink.runtime.execution.librarycache.FlinkUserCodeClassLoader;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
 import org.apache.flink.runtime.instance.ActorGateway;
 import org.apache.flink.runtime.jobgraph.JobGraph;
-import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
 import org.apache.flink.runtime.messages.JobClientMessages;
 import org.apache.flink.runtime.messages.JobManagerMessages;
 import org.apache.flink.runtime.util.SerializedThrowable;
@@ -96,14 +96,14 @@ public class JobClient {
 	public static JobListeningContext submitJob(
 			ActorSystem actorSystem,
 			Configuration config,
-			LeaderRetrievalService leaderRetrievalService,
+			HighAvailabilityServices highAvailabilityServices,
 			JobGraph jobGraph,
 			FiniteDuration timeout,
 			boolean sysoutLogUpdates,
 			ClassLoader classLoader) {
 
 		checkNotNull(actorSystem, "The actorSystem must not be null.");
-		checkNotNull(leaderRetrievalService, "The jobManagerGateway must not be null.");
+		checkNotNull(highAvailabilityServices, "The high availability services must not be null.");
 		checkNotNull(jobGraph, "The jobGraph must not be null.");
 		checkNotNull(timeout, "The timeout must not be null.");
 
@@ -112,7 +112,7 @@ public class JobClient {
 		// update messages, watches for disconnect between client and JobManager, ...
 
 		Props jobClientActorProps = JobSubmissionClientActor.createActorProps(
-			leaderRetrievalService,
+			highAvailabilityServices.getJobManagerLeaderRetriever(HighAvailabilityServices.DEFAULT_JOB_ID),
 			timeout,
 			sysoutLogUpdates,
 			config);
@@ -125,11 +125,12 @@ public class JobClient {
 				new Timeout(AkkaUtils.INF_TIMEOUT()));
 
 		return new JobListeningContext(
-				jobGraph.getJobID(),
-				submissionFuture,
-				jobClientActor,
-				timeout,
-				classLoader);
+			jobGraph.getJobID(),
+			submissionFuture,
+			jobClientActor,
+			timeout,
+			classLoader,
+			highAvailabilityServices);
 	}
 
 
@@ -142,7 +143,7 @@ public class JobClient {
 			ActorGateway jobManagerGateWay,
 			Configuration configuration,
 			ActorSystem actorSystem,
-			LeaderRetrievalService leaderRetrievalService,
+			HighAvailabilityServices highAvailabilityServices,
 			FiniteDuration timeout,
 			boolean sysoutLogUpdates) {
 
@@ -150,14 +151,14 @@ public class JobClient {
 		checkNotNull(jobManagerGateWay, "The jobManagerGateWay must not be null.");
 		checkNotNull(configuration, "The configuration must not be null.");
 		checkNotNull(actorSystem, "The actorSystem must not be null.");
-		checkNotNull(leaderRetrievalService, "The jobManagerGateway must not be null.");
+		checkNotNull(highAvailabilityServices, "The high availability services must not be null.");
 		checkNotNull(timeout, "The timeout must not be null.");
 
 		// we create a proxy JobClientActor that deals with all communication with
 		// the JobManager. It forwards the job attachments, checks the success/failure responses, logs
 		// update messages, watches for disconnect between client and JobManager, ...
 		Props jobClientActorProps = JobAttachmentClientActor.createActorProps(
-			leaderRetrievalService,
+			highAvailabilityServices.getJobManagerLeaderRetriever(HighAvailabilityServices.DEFAULT_JOB_ID),
 			timeout,
 			sysoutLogUpdates);
 
@@ -169,12 +170,13 @@ public class JobClient {
 				new Timeout(AkkaUtils.INF_TIMEOUT()));
 
 		return new JobListeningContext(
-				jobID,
-				attachmentFuture,
-				jobClientActor,
-				timeout,
-				actorSystem,
-				configuration);
+			jobID,
+			attachmentFuture,
+			jobClientActor,
+			timeout,
+			actorSystem,
+			configuration,
+			highAvailabilityServices);
 	}
 
 	/**
@@ -357,20 +359,19 @@ public class JobClient {
 	 *
 	 * @param actorSystem The actor system that performs the communication.
 	 * @param config The cluster wide configuration.
-	 * @param leaderRetrievalService Leader retrieval service which used to find the current leading
-	 *                               JobManager
+	 * @param highAvailabilityServices Service factory for high availability services
 	 * @param jobGraph    JobGraph describing the Flink job
 	 * @param timeout     Timeout for futures
 	 * @param sysoutLogUpdates prints log updates to system out if true
 	 * @param classLoader The class loader for deserializing the results
 	 * @return The job execution result
-	 * @throws org.apache.flink.runtime.client.JobExecutionException Thrown if the job
+	 * @throws JobExecutionException Thrown if the job
 	 *                                                               execution fails.
 	 */
 	public static JobExecutionResult submitJobAndWait(
 			ActorSystem actorSystem,
 			Configuration config,
-			LeaderRetrievalService leaderRetrievalService,
+			HighAvailabilityServices highAvailabilityServices,
 			JobGraph jobGraph,
 			FiniteDuration timeout,
 			boolean sysoutLogUpdates,
@@ -379,7 +380,7 @@ public class JobClient {
 		JobListeningContext jobListeningContext = submitJob(
 				actorSystem,
 				config,
-				leaderRetrievalService,
+				highAvailabilityServices,
 				jobGraph,
 				timeout,
 				sysoutLogUpdates,

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/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 a0bf97d..793041f 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
@@ -167,7 +167,8 @@ public abstract class JobClientActor extends FlinkUntypedActor implements Leader
 			JobManagerActorRef msg = (JobManagerActorRef) message;
 			connectToJobManager(msg.jobManager());
 
-			logAndPrintMessage("Connected to JobManager at " + msg.jobManager());
+			logAndPrintMessage("Connected to JobManager at " + msg.jobManager() +
+				" with leader session id " + leaderSessionID + '.');
 
 			connectedToJobManager();
 		}
@@ -326,8 +327,6 @@ public abstract class JobClientActor extends FlinkUntypedActor implements Leader
 			getContext().unwatch(jobManager);
 		}
 
-		LOG.info("Connected to new JobManager {}.", jobManager.path());
-
 		this.jobManager = jobManager;
 		getContext().watch(jobManager);
 

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobListeningContext.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobListeningContext.java b/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobListeningContext.java
index b944ba8..fe8c34c 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobListeningContext.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobListeningContext.java
@@ -22,6 +22,7 @@ import akka.actor.ActorSystem;
 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.highavailability.HighAvailabilityServices;
 import org.apache.flink.runtime.instance.ActorGateway;
 import org.apache.flink.runtime.util.LeaderRetrievalUtils;
 import org.slf4j.Logger;
@@ -47,6 +48,9 @@ public final class JobListeningContext {
 	/** Timeout used Asks */
 	private final FiniteDuration timeout;
 
+	/** Service factory for high availability services */
+	private final HighAvailabilityServices highAvailabilityServices;
+
 	/** ActorSystem for leader retrieval */
 	private ActorSystem actorSystem;
 	/** Flink configuration for initializing the BlobService */
@@ -59,16 +63,18 @@ public final class JobListeningContext {
 	 * Constructor to use when the class loader is available.
 	 */
 	public JobListeningContext(
-		JobID jobID,
-		Future<Object> jobResultFuture,
-		ActorRef jobClientActor,
-		FiniteDuration timeout,
-		ClassLoader classLoader) {
+			JobID jobID,
+			Future<Object> jobResultFuture,
+			ActorRef jobClientActor,
+			FiniteDuration timeout,
+			ClassLoader classLoader,
+			HighAvailabilityServices highAvailabilityServices) {
 		this.jobID = checkNotNull(jobID);
 		this.jobResultFuture = checkNotNull(jobResultFuture);
 		this.jobClientActor = checkNotNull(jobClientActor);
 		this.timeout = checkNotNull(timeout);
 		this.classLoader = checkNotNull(classLoader);
+		this.highAvailabilityServices = checkNotNull(highAvailabilityServices, "highAvailabilityServices");
 	}
 
 	/**
@@ -80,13 +86,15 @@ public final class JobListeningContext {
 		ActorRef jobClientActor,
 		FiniteDuration timeout,
 		ActorSystem actorSystem,
-		Configuration configuration) {
+		Configuration configuration,
+		HighAvailabilityServices highAvailabilityServices) {
 		this.jobID = checkNotNull(jobID);
 		this.jobResultFuture = checkNotNull(jobResultFuture);
 		this.jobClientActor = checkNotNull(jobClientActor);
 		this.timeout = checkNotNull(timeout);
 		this.actorSystem = checkNotNull(actorSystem);
 		this.configuration = checkNotNull(configuration);
+		this.highAvailabilityServices = checkNotNull(highAvailabilityServices, "highAvailabilityServices");
 	}
 
 	/**
@@ -135,7 +143,7 @@ public final class JobListeningContext {
 	private ActorGateway getJobManager() throws JobRetrievalException {
 		try {
 			return LeaderRetrievalUtils.retrieveLeaderGateway(
-				LeaderRetrievalUtils.createLeaderRetrievalService(configuration, true),
+				highAvailabilityServices.getJobManagerLeaderRetriever(HighAvailabilityServices.DEFAULT_JOB_ID),
 				actorSystem,
 				AkkaUtils.getLookupTimeout(configuration));
 		} catch (Exception e) {

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/BootstrapTools.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/BootstrapTools.java b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/BootstrapTools.java
index e356d2b..9bcaa18 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/BootstrapTools.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/BootstrapTools.java
@@ -29,8 +29,8 @@ import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.CoreOptions;
 import org.apache.flink.runtime.akka.AkkaUtils;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
 import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
-import org.apache.flink.runtime.util.LeaderRetrievalUtils;
 import org.apache.flink.runtime.webmonitor.WebMonitor;
 import org.apache.flink.runtime.webmonitor.WebMonitorUtils;
 import org.apache.flink.util.NetUtils;
@@ -165,17 +165,20 @@ public class BootstrapTools {
 
 	/**
 	 * Starts the web frontend.
+	 *
 	 * @param config The Flink config.
+	 * @param highAvailabilityServices Service factory for high availability services
 	 * @param actorSystem The ActorSystem to start the web frontend in.
 	 * @param logger Logger for log output
 	 * @return WebMonitor instance.
 	 * @throws Exception
 	 */
 	public static WebMonitor startWebMonitorIfConfigured(
-				Configuration config,
-				ActorSystem actorSystem,
-				ActorRef jobManager,
-				Logger logger) throws Exception {
+			Configuration config,
+			HighAvailabilityServices highAvailabilityServices,
+			ActorSystem actorSystem,
+			ActorRef jobManager,
+			Logger logger) throws Exception {
 
 
 		// this ensures correct values are present in the web frontend
@@ -186,8 +189,8 @@ public class BootstrapTools {
 		if (config.getInteger(ConfigConstants.JOB_MANAGER_WEB_PORT_KEY, 0) >= 0) {
 			logger.info("Starting JobManager Web Frontend");
 
-			LeaderRetrievalService leaderRetrievalService = 
-				LeaderRetrievalUtils.createLeaderRetrievalService(config, jobManager);
+			LeaderRetrievalService leaderRetrievalService =
+				highAvailabilityServices.getJobManagerLeaderRetriever(HighAvailabilityServices.DEFAULT_JOB_ID);
 
 			// start the web frontend. we need to load this dynamically
 			// because it is not in the same project/dependencies

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServices.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServices.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServices.java
index 3de75d5..69a65dd 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServices.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServices.java
@@ -56,6 +56,13 @@ public interface HighAvailabilityServices extends AutoCloseable {
 	 */
 	UUID DEFAULT_LEADER_ID = new UUID(0, 0);
 
+	/**
+	 * This JobID should be used to identify the old JobManager when using the
+	 * {@link HighAvailabilityServices}. With Flip-6 every JobManager will have a distinct
+	 * JobID assigned.
+	 */
+	JobID DEFAULT_JOB_ID = new JobID(0L, 0L);
+
 	// ------------------------------------------------------------------------
 	//  Services
 	// ------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServicesUtils.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServicesUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServicesUtils.java
index 106be5a..c9e2957 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServicesUtils.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServicesUtils.java
@@ -115,7 +115,7 @@ public class HighAvailabilityServicesUtils {
 		if (port <= 0 || port >= 65536) {
 			throw new ConfigurationException("Invalid value for '" + JobManagerOptions.PORT +
 				"' (port of the JobManager actor system) : " + port +
-				".  it must be great than 0 and less than 65536.");
+				".  it must be greater than 0 and less than 65536.");
 		}
 
 		return Tuple2.of(hostname, port);

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobGraph.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobGraph.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobGraph.java
index 2a8af37..2f5cd25 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobGraph.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobGraph.java
@@ -484,6 +484,15 @@ public class JobGraph implements Serializable {
 	}
 
 	/**
+	 * Gets the list of assigned user jar paths.
+	 *
+	 * @return The list of assigned user jar paths
+	 */
+	public List<Path> getUserJars() {
+		return userJars;
+	}
+
+	/**
 	 * Adds the BLOB referenced by the key to the JobGraph's dependencies.
 	 *
 	 * @param key

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/ZooKeeperSubmittedJobGraphStore.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/ZooKeeperSubmittedJobGraphStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/ZooKeeperSubmittedJobGraphStore.java
index aaafa76..2552088 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/ZooKeeperSubmittedJobGraphStore.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/ZooKeeperSubmittedJobGraphStore.java
@@ -143,11 +143,13 @@ public class ZooKeeperSubmittedJobGraphStore implements SubmittedJobGraphStore {
 			if (isRunning) {
 				jobGraphListener = null;
 
-				pathCache.close();
-
-				client.close();
-
-				isRunning = false;
+				try {
+					pathCache.close();
+				} catch (Exception e) {
+					throw new Exception("Could not properly stop the ZooKeeperSubmittedJobGraphStore.", e);
+				} finally {
+					isRunning = false;
+				}
 			}
 		}
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/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
index 0fa6a9e..21e2ca1 100644
--- 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
@@ -18,6 +18,7 @@
 
 package org.apache.flink.runtime.leaderelection;
 
+import org.apache.flink.util.ExceptionUtils;
 import org.apache.flink.util.Preconditions;
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.framework.recipes.cache.ChildData;
@@ -48,6 +49,8 @@ public class ZooKeeperLeaderElectionService implements LeaderElectionService, Le
 
 	private static final Logger LOG = LoggerFactory.getLogger(ZooKeeperLeaderElectionService.class);
 
+	private final Object lock = new Object();
+
 	/** Client to the ZooKeeper quorum */
 	private final CuratorFramework client;
 
@@ -62,12 +65,12 @@ public class ZooKeeperLeaderElectionService implements LeaderElectionService, Le
 
 	private UUID issuedLeaderSessionID;
 
-	private UUID confirmedLeaderSessionID;
+	private volatile UUID confirmedLeaderSessionID;
 
 	/** The leader contender which applies for leadership */
 	private volatile LeaderContender leaderContender;
 
-	private final Object lock = new Object();
+	private volatile boolean running;
 
 	private final ConnectionStateListener listener = new ConnectionStateListener() {
 		@Override
@@ -84,11 +87,17 @@ public class ZooKeeperLeaderElectionService implements LeaderElectionService, Le
 	 * @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;
+		this.client = Preconditions.checkNotNull(client, "CuratorFramework client");
+		this.leaderPath = Preconditions.checkNotNull(leaderPath, "leaderPath");
 
 		leaderLatch = new LeaderLatch(client, latchPath);
 		cache = new NodeCache(client, leaderPath);
+
+		issuedLeaderSessionID = null;
+		confirmedLeaderSessionID = null;
+		leaderContender = null;
+
+		running = false;
 	}
 
 	/**
@@ -105,32 +114,56 @@ public class ZooKeeperLeaderElectionService implements LeaderElectionService, Le
 		Preconditions.checkNotNull(contender, "Contender must not be null.");
 		Preconditions.checkState(leaderContender == null, "Contender was already set.");
 
-		LOG.info("Starting ZooKeeperLeaderElectionService.");
+		LOG.info("Starting ZooKeeperLeaderElectionService {}.", this);
+
+		synchronized (lock) {
 
-		leaderContender = contender;
+			leaderContender = contender;
 
-		leaderLatch.addListener(this);
-		leaderLatch.start();
+			leaderLatch.addListener(this);
+			leaderLatch.start();
 
-		cache.getListenable().addListener(this);
-		cache.start();
+			cache.getListenable().addListener(this);
+			cache.start();
 
-		client.getConnectionStateListenable().addListener(listener);
+			client.getConnectionStateListenable().addListener(listener);
+
+			running = true;
+		}
 	}
 
 	@Override
 	public void stop() throws Exception{
-		LOG.info("Stopping ZooKeeperLeaderElectionService.");
+		synchronized (lock) {
+			if (!running) {
+				return;
+			}
+
+			running = false;
+			confirmedLeaderSessionID = null;
+			issuedLeaderSessionID = null;
+		}
+
+		LOG.info("Stopping ZooKeeperLeaderElectionService {}.", this);
 
 		client.getConnectionStateListenable().removeListener(listener);
 
-		cache.close();
-		leaderLatch.close();
-		client.close();
+		Exception exception = null;
 
-		synchronized (lock) {
-			confirmedLeaderSessionID = null;
-			issuedLeaderSessionID = null;
+		try {
+			cache.close();
+		} catch (Exception e) {
+			exception = ExceptionUtils.firstOrSuppressed(e, exception);
+		}
+
+		try {
+			leaderLatch.close();
+		} catch(Exception e) {
+			exception = ExceptionUtils.firstOrSuppressed(e, exception);
+		}
+
+		if (exception != null) {
+			throw new Exception("Could not properly stop the ZooKeeperLeaderElectionService.", exception);
 		}
 	}
 
@@ -148,9 +181,14 @@ public class ZooKeeperLeaderElectionService implements LeaderElectionService, Le
 		if(leaderLatch.hasLeadership()) {
 			// check if this is an old confirmation call
 			synchronized (lock) {
-				if (leaderSessionID.equals(this.issuedLeaderSessionID)) {
-					confirmedLeaderSessionID = leaderSessionID;
-					writeLeaderInformation(confirmedLeaderSessionID);
+				if (running) {
+					if (leaderSessionID.equals(this.issuedLeaderSessionID)) {
+						confirmedLeaderSessionID = leaderSessionID;
+						writeLeaderInformation(confirmedLeaderSessionID);
+					}
+				} else {
+					LOG.debug("Ignoring the leader session Id {} confirmation, since the " +
+						"ZooKeeperLeaderElectionService has already been stopped.", leaderSessionID);
 				}
 			}
 		} else {
@@ -167,31 +205,41 @@ public class ZooKeeperLeaderElectionService implements LeaderElectionService, Le
 	@Override
 	public void isLeader() {
 		synchronized (lock) {
-			issuedLeaderSessionID = UUID.randomUUID();
-			confirmedLeaderSessionID = null;
+			if (running) {
+				issuedLeaderSessionID = UUID.randomUUID();
+				confirmedLeaderSessionID = null;
+
+				if (LOG.isDebugEnabled()) {
+					LOG.debug(
+						"Grant leadership to contender {} with session ID {}.",
+						leaderContender.getAddress(),
+						issuedLeaderSessionID);
+				}
 
-			if (LOG.isDebugEnabled()) {
-				LOG.debug(
-					"Grant leadership to contender {} with session ID {}.",
-					leaderContender.getAddress(),
-					issuedLeaderSessionID);
+				leaderContender.grantLeadership(issuedLeaderSessionID);
+			} else {
+				LOG.debug("Ignoring the grant leadership notification since the service has " +
+					"already been stopped.");
 			}
-
-			leaderContender.grantLeadership(issuedLeaderSessionID);
 		}
 	}
 
 	@Override
 	public void notLeader() {
 		synchronized (lock) {
-			issuedLeaderSessionID = null;
-			confirmedLeaderSessionID = null;
+			if (running) {
+				issuedLeaderSessionID = null;
+				confirmedLeaderSessionID = null;
 
-			if (LOG.isDebugEnabled()) {
-				LOG.debug("Revoke leadership of {}.", leaderContender.getAddress());
-			}
+				if (LOG.isDebugEnabled()) {
+					LOG.debug("Revoke leadership of {}.", leaderContender.getAddress());
+				}
 
-			leaderContender.revokeLeadership();
+				leaderContender.revokeLeadership();
+			} else {
+				LOG.debug("Ignoring the revoke leadership notification since the service " +
+					"has already been stopped.");
+			}
 		}
 	}
 
@@ -201,53 +249,57 @@ public class ZooKeeperLeaderElectionService implements LeaderElectionService, Le
 			// leaderSessionID is null if the leader contender has not yet confirmed the session ID
 			if (leaderLatch.hasLeadership()) {
 				synchronized (lock) {
-					if (LOG.isDebugEnabled()) {
-						LOG.debug(
-							"Leader node changed while {} is the leader with session ID {}.",
-							leaderContender.getAddress(),
-							confirmedLeaderSessionID);
-					}
-
-					if (confirmedLeaderSessionID != null) {
-						ChildData childData = cache.getCurrentData();
+					if (running) {
+						if (LOG.isDebugEnabled()) {
+							LOG.debug(
+								"Leader node changed while {} is the leader with session ID {}.",
+								leaderContender.getAddress(),
+								confirmedLeaderSessionID);
+						}
 
-						if (childData == null) {
-							if (LOG.isDebugEnabled()) {
-								LOG.debug(
-									"Writing leader information into empty node by {}.",
-									leaderContender.getAddress());
-							}
-							writeLeaderInformation(confirmedLeaderSessionID);
-						} else {
-							byte[] data = childData.getData();
+						if (confirmedLeaderSessionID != null) {
+							ChildData childData = cache.getCurrentData();
 
-							if (data == null || data.length == 0) {
-								// the data field seems to be empty, rewrite information
+							if (childData == null) {
 								if (LOG.isDebugEnabled()) {
 									LOG.debug(
-										"Writing leader information into node with empty data field by {}.",
+										"Writing leader information into empty node by {}.",
 										leaderContender.getAddress());
 								}
 								writeLeaderInformation(confirmedLeaderSessionID);
 							} else {
-								ByteArrayInputStream bais = new ByteArrayInputStream(data);
-								ObjectInputStream ois = new ObjectInputStream(bais);
-
-								String leaderAddress = ois.readUTF();
-								UUID leaderSessionID = (UUID) ois.readObject();
+								byte[] data = childData.getData();
 
-								if (!leaderAddress.equals(this.leaderContender.getAddress()) ||
-										(leaderSessionID == null || !leaderSessionID.equals(confirmedLeaderSessionID))) {
-									// the data field does not correspond to the expected leader information
+								if (data == null || data.length == 0) {
+									// the data field seems to be empty, rewrite information
 									if (LOG.isDebugEnabled()) {
 										LOG.debug(
-											"Correcting leader information by {}.",
+											"Writing leader information into node with empty data field by {}.",
 											leaderContender.getAddress());
 									}
 									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
+										if (LOG.isDebugEnabled()) {
+											LOG.debug(
+												"Correcting leader information by {}.",
+												leaderContender.getAddress());
+										}
+										writeLeaderInformation(confirmedLeaderSessionID);
+									}
 								}
 							}
 						}
+					} else {
+						LOG.debug("Ignoring node change notification since the service has already been stopped.");
 					}
 				}
 			}

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/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
index f74fb1a..7f18fd9 100644
--- 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
@@ -29,6 +29,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.ByteArrayInputStream;
+import java.io.IOException;
 import java.io.ObjectInputStream;
 import java.util.Objects;
 import java.util.UUID;
@@ -43,6 +44,8 @@ public class ZooKeeperLeaderRetrievalService implements LeaderRetrievalService,
 	private static final Logger LOG = LoggerFactory.getLogger(
 		ZooKeeperLeaderRetrievalService.class);
 
+	private final Object lock = new Object();
+
 	/** Connection to the used ZooKeeper quorum */
 	private final CuratorFramework client;
 
@@ -53,8 +56,11 @@ public class ZooKeeperLeaderRetrievalService implements LeaderRetrievalService,
 	private volatile LeaderRetrievalListener leaderListener;
 
 	private String lastLeaderAddress;
+
 	private UUID lastLeaderSessionID;
 
+	private volatile boolean running;
+
 	private final ConnectionStateListener connectionStateListener = new ConnectionStateListener() {
 		@Override
 		public void stateChanged(CuratorFramework client, ConnectionState newState) {
@@ -69,8 +75,14 @@ public class ZooKeeperLeaderRetrievalService implements LeaderRetrievalService,
 	 * @param retrievalPath Path of the ZooKeeper node which contains the leader information
 	 */
 	public ZooKeeperLeaderRetrievalService(CuratorFramework client, String retrievalPath) {
-		this.client = client;
+		this.client = Preconditions.checkNotNull(client, "CuratorFramework client");
 		this.cache = new NodeCache(client, retrievalPath);
+
+		this.leaderListener = null;
+		this.lastLeaderAddress = null;
+		this.lastLeaderSessionID = null;
+
+		running = false;
 	}
 
 	@Override
@@ -81,65 +93,87 @@ public class ZooKeeperLeaderRetrievalService implements LeaderRetrievalService,
 
 		LOG.info("Starting ZooKeeperLeaderRetrievalService.");
 
-		leaderListener = listener;
+		synchronized (lock) {
+			leaderListener = listener;
+
+			cache.getListenable().addListener(this);
+			cache.start();
 
-		cache.getListenable().addListener(this);
-		cache.start();
+			client.getConnectionStateListenable().addListener(connectionStateListener);
 
-		client.getConnectionStateListenable().addListener(connectionStateListener);
+			running = true;
+		}
 	}
 
 	@Override
 	public void stop() throws Exception {
 		LOG.info("Stopping ZooKeeperLeaderRetrievalService.");
 
+		synchronized (lock) {
+			if (!running) {
+				return;
+			}
+
+			running = false;
+		}
+
 		client.getConnectionStateListenable().removeListener(connectionStateListener);
 
-		cache.close();
+		try {
+			cache.close();
+		} catch (IOException e) {
+			throw new Exception("Could not properly stop the ZooKeeperLeaderRetrievalService.", e);
+		}
 	}
 
 	@Override
 	public void nodeChanged() throws Exception {
-		try {
-			LOG.debug("Leader node has changed.");
-
-			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();
+		synchronized (lock) {
+			if (running) {
+				try {
+					LOG.debug("Leader node has changed.");
+
+					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))) {
+						LOG.debug(
+							"New leader information: Leader={}, session ID={}.",
+							leaderAddress,
+							leaderSessionID);
+
+						lastLeaderAddress = leaderAddress;
+						lastLeaderSessionID = leaderSessionID;
+						leaderListener.notifyLeaderAddress(leaderAddress, leaderSessionID);
+					}
+				} catch (Exception e) {
+					leaderListener.handleError(new Exception("Could not handle node changed event.", e));
+					throw e;
 				}
+			} else {
+				LOG.debug("Ignoring node change notification since the service has already been stopped.");
 			}
-
-			if(!(Objects.equals(leaderAddress, lastLeaderAddress) &&
-					Objects.equals(leaderSessionID, lastLeaderSessionID))) {
-				LOG.debug(
-					"New leader information: Leader={}, session ID={}.",
-					leaderAddress,
-					leaderSessionID);
-
-				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/ddd6a99a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/StandaloneMiniCluster.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/StandaloneMiniCluster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/StandaloneMiniCluster.java
new file mode 100644
index 0000000..07178fc
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/StandaloneMiniCluster.java
@@ -0,0 +1,154 @@
+/*
+ * 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.minicluster;
+
+import akka.actor.ActorRef;
+import akka.actor.ActorSystem;
+import akka.pattern.Patterns;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.JobManagerOptions;
+import org.apache.flink.runtime.akka.AkkaUtils;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.concurrent.Executors;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils;
+import org.apache.flink.runtime.jobmanager.JobManager;
+import org.apache.flink.runtime.jobmanager.MemoryArchivist;
+import org.apache.flink.runtime.messages.TaskManagerMessages;
+import org.apache.flink.runtime.taskmanager.TaskManager;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.Preconditions;
+import scala.Option;
+import scala.concurrent.Await;
+import scala.concurrent.Future;
+import scala.concurrent.duration.FiniteDuration;
+
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Mini cluster to run the old JobManager code without embedded high availability services. This
+ * class has been implemented because the normal {@link FlinkMiniCluster} has been changed to use
+ * the {@link HighAvailabilityServices}. With this change we can no longer use the
+ * {@link org.apache.flink.api.java.RemoteEnvironment} to connect against the
+ * {@link FlinkMiniCluster}, because the remote environment cannot retrieve the current leader
+ * session id.
+ */
+public class StandaloneMiniCluster {
+
+	private static final String LOCAL_HOSTNAME = "localhost";
+
+	private final Configuration configuration;
+
+	private final ActorSystem actorSystem;
+
+	private final ScheduledExecutorService scheduledExecutorService;
+
+	private final HighAvailabilityServices highAvailabilityServices;
+
+	private final FiniteDuration timeout;
+
+	private final int port;
+
+	public StandaloneMiniCluster(Configuration configuration) throws Exception {
+		this.configuration = Preconditions.checkNotNull(configuration);
+
+		timeout = AkkaUtils.getTimeout(configuration);
+
+		actorSystem = JobManager.startActorSystem(
+			configuration,
+			LOCAL_HOSTNAME,
+			0);
+
+		port = configuration.getInteger(JobManagerOptions.PORT);
+
+		scheduledExecutorService = new ScheduledThreadPoolExecutor(1);
+
+		highAvailabilityServices = HighAvailabilityServicesUtils.createHighAvailabilityServices(
+			configuration,
+			Executors.directExecutor(),
+			HighAvailabilityServicesUtils.AddressResolution.TRY_ADDRESS_RESOLUTION);
+
+		JobManager.startJobManagerActors(
+			configuration,
+			actorSystem,
+			scheduledExecutorService,
+			scheduledExecutorService,
+			highAvailabilityServices,
+			JobManager.class,
+			MemoryArchivist.class);
+
+		ActorRef taskManager = TaskManager.startTaskManagerComponentsAndActor(
+			configuration,
+			ResourceID.generate(),
+			actorSystem,
+			highAvailabilityServices,
+			LOCAL_HOSTNAME,
+			Option.<String>empty(),
+			true,
+			TaskManager.class);
+
+		Future<Object> registrationFuture = Patterns.ask(
+			taskManager,
+			TaskManagerMessages.getNotifyWhenRegisteredAtJobManagerMessage(),
+			timeout.toMillis());
+
+		Await.ready(registrationFuture, timeout);
+	}
+
+	public String getHostname() {
+		return LOCAL_HOSTNAME;
+	}
+
+	public int getPort() {
+		return port;
+	}
+
+	public Configuration getConfiguration() {
+		return configuration;
+	}
+
+	public void close() throws Exception {
+		Exception exception = null;
+
+		actorSystem.shutdown();
+		actorSystem.awaitTermination();
+
+		try {
+			highAvailabilityServices.closeAndCleanupAllData();
+		} catch (Exception e) {
+			exception = e;
+		}
+
+		scheduledExecutorService.shutdownNow();
+
+		try {
+			scheduledExecutorService.awaitTermination(timeout.toMillis(), TimeUnit.MILLISECONDS);
+		} catch (InterruptedException e) {
+			Thread.currentThread().interrupt();
+
+			exception = ExceptionUtils.firstOrSuppressed(e, exception);
+		}
+
+		if (exception != null) {
+			throw exception;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/flink-runtime/src/main/java/org/apache/flink/runtime/query/QueryableStateClient.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/query/QueryableStateClient.java b/flink-runtime/src/main/java/org/apache/flink/runtime/query/QueryableStateClient.java
index f9cf01d..9b05273 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/query/QueryableStateClient.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/query/QueryableStateClient.java
@@ -29,6 +29,7 @@ import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.IllegalConfigurationException;
 import org.apache.flink.configuration.QueryableStateOptions;
 import org.apache.flink.runtime.akka.AkkaUtils;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
 import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
 import org.apache.flink.runtime.query.AkkaKvStateLocationLookupService.FixedDelayLookupRetryStrategyFactory;
 import org.apache.flink.runtime.query.AkkaKvStateLocationLookupService.LookupRetryStrategyFactory;
@@ -38,7 +39,6 @@ import org.apache.flink.runtime.query.netty.KvStateServer;
 import org.apache.flink.runtime.query.netty.UnknownKeyOrNamespace;
 import org.apache.flink.runtime.query.netty.UnknownKvStateID;
 import org.apache.flink.runtime.state.KeyGroupRangeAssignment;
-import org.apache.flink.runtime.util.LeaderRetrievalUtils;
 import org.apache.flink.util.Preconditions;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -102,14 +102,16 @@ public class QueryableStateClient {
 	 * system and another for the network client.
 	 *
 	 * @param config Configuration to use.
+	 * @param highAvailabilityServices Service factory for high availability services
 	 * @throws Exception Failures are forwarded
 	 */
-	public QueryableStateClient(Configuration config) throws Exception {
+	public QueryableStateClient(
+			Configuration config,
+			HighAvailabilityServices highAvailabilityServices) throws Exception {
 		Preconditions.checkNotNull(config, "Configuration");
 
 		// Create a leader retrieval service
-		LeaderRetrievalService leaderRetrievalService = LeaderRetrievalUtils
-				.createLeaderRetrievalService(config, true);
+		LeaderRetrievalService leaderRetrievalService = highAvailabilityServices.getJobManagerLeaderRetriever(HighAvailabilityServices.DEFAULT_JOB_ID);
 
 		// Get the ask timeout
 		String askTimeoutString = config.getString(

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceUtils.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceUtils.java
index eab4de8..8789eed 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceUtils.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceUtils.java
@@ -112,10 +112,12 @@ public class AkkaRpcServiceUtils {
 
 	/**
 	 *
-	 * @param hostname     The hostname or address where the target RPC service is listening.
-	 * @param port         The port where the target RPC service is listening.
+	 * @param hostname The hostname or address where the target RPC service is listening.
+	 * @param port The port where the target RPC service is listening.
 	 * @param endpointName The name of the RPC endpoint.
-	 * @param config       The configuration from which to deduce further settings.
+	 * @param addressResolution Whether to try address resolution of the given hostname or not.
+	 *                          This allows to fail fast in case that the hostname cannot be resolved.
+	 * @param config The configuration from which to deduce further settings.
 	 *
 	 * @return The RPC URL of the specified RPC endpoint.
 	 */
@@ -143,10 +145,12 @@ public class AkkaRpcServiceUtils {
 
 	/**
 	 * 
-	 * @param hostname     The hostname or address where the target RPC service is listening.
-	 * @param port         The port where the target RPC service is listening.
+	 * @param hostname The hostname or address where the target RPC service is listening.
+	 * @param port The port where the target RPC service is listening.
 	 * @param endpointName The name of the RPC endpoint.
-	 * @param akkaProtocol       True, if security/encryption is enabled, false otherwise.
+	 * @param addressResolution Whether to try address resolution of the given hostname or not.
+	 *                          This allows to fail fast in case that the hostname cannot be resolved.
+	 * @param akkaProtocol True, if security/encryption is enabled, false otherwise.
 	 * 
 	 * @return The RPC URL of the specified RPC endpoint.
 	 */

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/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
index 05749c4..073c52b 100644
--- 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
@@ -33,7 +33,6 @@ import org.apache.flink.runtime.jobmanager.HighAvailabilityMode;
 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.leaderretrieval.StandaloneLeaderRetrievalService;
 import org.apache.flink.runtime.net.ConnectionUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -51,57 +50,6 @@ 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}
-	 * @param resolveInitialHostName If true, resolves the initial hostname
-	 * @return The {@link LeaderRetrievalService} specified in the configuration object
-	 * @throws Exception
-	 */
-	public static LeaderRetrievalService createLeaderRetrievalService(
-			Configuration configuration,
-			boolean resolveInitialHostName)
-		throws Exception {
-
-		HighAvailabilityMode highAvailabilityMode = getRecoveryMode(configuration);
-
-		switch (highAvailabilityMode) {
-			case NONE:
-				return StandaloneUtils.createLeaderRetrievalService(configuration, resolveInitialHostName, null);
-			case ZOOKEEPER:
-				return ZooKeeperUtils.createLeaderRetrievalService(configuration);
-			default:
-				throw new Exception("Recovery mode " + highAvailabilityMode + " is not supported.");
-		}
-	}
-
-	/**
-	 * Creates a {@link LeaderRetrievalService} that either uses the distributed leader election
-	 * configured in the configuration, or, in standalone mode, the given actor reference.
-	 *
-	 * @param configuration Configuration containing the settings for the {@link LeaderRetrievalService}
-	 * @param standaloneRef Actor reference to be used in standalone mode. 
-	 *                      
-	 * @return The {@link LeaderRetrievalService} specified in the configuration object
-	 * @throws Exception
-	 */
-	public static LeaderRetrievalService createLeaderRetrievalService(
-				Configuration configuration, ActorRef standaloneRef) throws Exception {
-
-		HighAvailabilityMode highAvailabilityMode = getRecoveryMode(configuration);
-
-		switch (highAvailabilityMode) {
-			case NONE:
-				String akkaUrl = standaloneRef.path().toSerializationFormat();
-				return new StandaloneLeaderRetrievalService(akkaUrl);
-			case ZOOKEEPER:
-				return ZooKeeperUtils.createLeaderRetrievalService(configuration);
-			default:
-				throw new Exception("Recovery mode " + highAvailabilityMode + " is not supported.");
-		}
-	}
 	
 	/**
 	 * Retrieves the current leader gateway using the given {@link LeaderRetrievalService}. If the
@@ -173,6 +121,21 @@ public class LeaderRetrievalUtils {
 		}
 	}
 
+	/**
+	 * Retrieves the current leader session id of the component identified by the given leader
+	 * retrieval service.
+	 *
+	 * @param leaderRetrievalService Leader retrieval service to be used for the leader retrieval
+	 * @param timeout Timeout for the leader retrieval
+	 * @return The leader session id of the retrieved leader
+	 * @throws LeaderRetrievalException if the leader retrieval operation fails (including a timeout)
+	 */
+	public static UUID retrieveLeaderSessionId(
+			LeaderRetrievalService leaderRetrievalService,
+			FiniteDuration timeout) throws LeaderRetrievalException {
+		return retrieveLeaderConnectionInfo(leaderRetrievalService, timeout).getLeaderSessionID();
+	}
+
 	public static InetAddress findConnectingAddress(
 		LeaderRetrievalService leaderRetrievalService,
 		Time timeout) throws LeaderRetrievalException {

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/flink-runtime/src/main/java/org/apache/flink/runtime/util/SerializedThrowable.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/SerializedThrowable.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/SerializedThrowable.java
index ed485d0..f6f08e4 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/util/SerializedThrowable.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/SerializedThrowable.java
@@ -135,6 +135,10 @@ public class SerializedThrowable extends Exception implements Serializable {
 		return cached;
 	}
 
+	public String getOriginalErrorClassName() {
+		return originalErrorClassName;
+	}
+
 	// ------------------------------------------------------------------------
 	//  Override the behavior of Throwable
 	// ------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/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
index 1b73dc7..9ade5ec 100644
--- 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
@@ -159,18 +159,6 @@ public class ZooKeeperUtils {
 	/**
 	 * Creates a {@link ZooKeeperLeaderRetrievalService} instance.
 	 *
-	 * @param configuration {@link Configuration} object containing the configuration values
-	 * @return {@link ZooKeeperLeaderRetrievalService} instance.
-	 */
-	public static ZooKeeperLeaderRetrievalService createLeaderRetrievalService(
-			Configuration configuration) throws Exception {
-		final CuratorFramework client = startCuratorFramework(configuration);
-		return createLeaderRetrievalService(client, configuration);
-	}
-
-	/**
-	 * Creates a {@link ZooKeeperLeaderRetrievalService} instance.
-	 *
 	 * @param client        The {@link CuratorFramework} ZooKeeper client to use
 	 * @param configuration {@link Configuration} object containing the configuration values
 	 * @return {@link ZooKeeperLeaderRetrievalService} instance.
@@ -207,21 +195,6 @@ public class ZooKeeperUtils {
 	}
 
 	/**
-	 * Creates a {@link ZooKeeperLeaderElectionService} instance and a new {@link
-	 * CuratorFramework} client.
-	 *
-	 * @param configuration {@link Configuration} object containing the configuration values
-	 * @return {@link ZooKeeperLeaderElectionService} instance.
-	 */
-	public static ZooKeeperLeaderElectionService createLeaderElectionService(
-			Configuration configuration) throws Exception {
-
-		CuratorFramework client = startCuratorFramework(configuration);
-
-		return createLeaderElectionService(client, configuration);
-	}
-
-	/**
 	 * Creates a {@link ZooKeeperLeaderElectionService} instance.
 	 *
 	 * @param client        The {@link CuratorFramework} ZooKeeper client to use


[04/16] flink git commit: [FLINK-6136] Separate EmbeddedHaServices and StandaloneHaServices

Posted by tr...@apache.org.
[FLINK-6136] Separate EmbeddedHaServices and StandaloneHaServices

This PR introduces a standalone high availability services implementation which can be used
in a distributed setting with no HA guarantees. Additionally, it introduces a common base
class which is also used by the EmbeddedHaServices. This base class instantiates the
standalone variants of the checkpoint recovery factory, submitted job graphs store, running
jobs registry and blob store.

The StandaloneHaServices are instantiated with a fixed address for the Job- and
ResourceManager. This address and the HighAvailability.DEFAULT_LEADER_ID is returned by
the corresponding LeaderRetrievalServices when being started.

This closes #3622.


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

Branch: refs/heads/master
Commit: a0bb99c70703f6eaf0953b9ea6e70c5dbb9c1c77
Parents: 43fa507
Author: Till Rohrmann <tr...@apache.org>
Authored: Tue Mar 21 15:10:15 2017 +0100
Committer: Till Rohrmann <tr...@apache.org>
Committed: Fri May 5 11:05:55 2017 +0200

----------------------------------------------------------------------
 .../flink/client/program/ClusterClient.java     |   8 +-
 .../RemoteExecutorHostnameResolutionTest.java   |   3 +-
 .../apache/flink/client/program/ClientTest.java |  41 +-
 ...rRetrievalServiceHostnameResolutionTest.java |  11 +-
 .../org/apache/flink/storm/api/FlinkClient.java |  16 +-
 .../flink/util/ConfigurationException.java      |  38 ++
 .../MesosApplicationMasterRunner.java           |   7 +-
 .../handlers/HandlerRedirectUtils.java          |   5 +-
 .../apache/flink/runtime/blob/BlobUtils.java    |   4 +-
 .../runtime/client/JobListeningContext.java     |   2 +-
 .../highavailability/EmbeddedNonHaServices.java |  70 ---
 .../HighAvailabilityServicesUtils.java          |  90 +++-
 .../runtime/highavailability/NonHaServices.java |  69 ---
 .../highavailability/ZookeeperHaServices.java   | 215 ---------
 .../highavailability/ZookeeperRegistry.java     | 127 -----
 .../SingleLeaderElectionService.java            | 386 ---------------
 .../nonha/AbstractNonHaServices.java            | 117 ++---
 .../nonha/EmbeddedLeaderService.java            | 469 -------------------
 .../highavailability/nonha/NonHaRegistry.java   |  72 ---
 .../nonha/embedded/EmbeddedHaServices.java      | 126 +++++
 .../nonha/embedded/EmbeddedLeaderService.java   | 468 ++++++++++++++++++
 .../SingleLeaderElectionService.java            | 386 +++++++++++++++
 .../nonha/standalone/StandaloneHaServices.java  | 101 ++++
 .../StandaloneRunningJobsRegistry.java          |  72 +++
 .../zookeeper/ZooKeeperHaServices.java          | 217 +++++++++
 .../zookeeper/ZooKeeperRunningJobsRegistry.java | 128 +++++
 .../flink/runtime/jobmaster/JobMaster.java      |   9 +-
 .../flink/runtime/minicluster/MiniCluster.java  |   4 +-
 .../runtime/query/QueryableStateClient.java     |   2 +-
 .../resourcemanager/ResourceManager.java        |   2 +
 .../ResourceManagerConfiguration.java           |   2 +-
 ...urceManagerRuntimeServicesConfiguration.java |   2 +-
 .../exceptions/ConfigurationException.java      |  38 --
 .../slotmanager/SlotManagerConfiguration.java   |   2 +-
 .../flink/runtime/rpc/RpcServiceUtils.java      | 174 -------
 .../runtime/rpc/akka/AkkaRpcServiceUtils.java   | 205 ++++++++
 .../runtime/taskexecutor/TaskExecutor.java      |   7 +-
 .../runtime/taskexecutor/TaskManagerRunner.java |   4 +-
 .../runtime/util/LeaderRetrievalUtils.java      |  17 +-
 .../flink/runtime/util/StandaloneUtils.java     |  74 +--
 .../apache/flink/runtime/akka/AkkaUtils.scala   |  19 +-
 .../flink/runtime/jobmanager/JobManager.scala   | 148 +-----
 .../minicluster/LocalFlinkMiniCluster.scala     |  15 +-
 .../flink/runtime/taskmanager/TaskManager.scala |  45 +-
 .../TestingHighAvailabilityServices.java        |   4 +-
 .../highavailability/ZooKeeperRegistryTest.java |  82 ----
 .../SingleLeaderElectionServiceTest.java        | 226 ---------
 .../nonha/embedded/EmbeddedHaServicesTest.java  | 167 +++++++
 .../SingleLeaderElectionServiceTest.java        | 226 +++++++++
 .../standalone/StandaloneHaServicesTest.java    | 104 ++++
 .../zookeeper/ZooKeeperRegistryTest.java        |  85 ++++
 .../JobManagerProcessReapingTest.java           |  16 +-
 .../flink/runtime/jobmanager/JobSubmitTest.java |   2 +-
 .../ZooKeeperLeaderElectionTest.java            |   4 +-
 .../ZooKeeperLeaderRetrievalTest.java           |  28 +-
 .../runtime/taskexecutor/TaskExecutorTest.java  |   7 +-
 .../TaskManagerProcessReapingTestBase.java      |   3 +-
 .../TaskManagerRegistrationTest.java            |   5 +-
 .../runtime/testutils/JobManagerProcess.java    |  18 +-
 .../flink/runtime/akka/AkkaUtilsTest.scala      |  17 +-
 .../jobmanager/JobManagerConnectionTest.scala   |  10 +-
 .../runtime/testingUtils/TestingUtils.scala     |   9 +-
 .../ZooKeeperLeaderElectionITCase.java          |   3 +-
 .../flink/yarn/YARNHighAvailabilityITCase.java  |   4 +-
 .../flink/yarn/YarnApplicationMasterRunner.java |   3 +-
 .../apache/flink/yarn/YarnClusterClient.java    |   2 +-
 .../yarn/YarnFlinkApplicationMasterRunner.java  |   7 +-
 .../flink/yarn/YarnTaskExecutorRunner.java      |   9 +-
 .../YarnHighAvailabilityServices.java           |   6 +-
 .../YarnIntraNonHaMasterServices.java           |   2 +-
 .../YarnPreConfiguredMasterNonHaServices.java   |  16 +-
 .../YarnPreConfiguredMasterHaServicesTest.java  |  26 +-
 72 files changed, 2712 insertions(+), 2396 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/a0bb99c7/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java b/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java
index ab4daa9..0f88f7c 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java
@@ -241,7 +241,7 @@ public abstract class ClusterClient {
 		try {
 			LeaderConnectionInfo leaderConnectionInfo =
 				LeaderRetrievalUtils.retrieveLeaderConnectionInfo(
-					LeaderRetrievalUtils.createLeaderRetrievalService(flinkConfig), timeout);
+					LeaderRetrievalUtils.createLeaderRetrievalService(flinkConfig, true), timeout);
 
 			return AkkaUtils.getInetSockeAddressFromAkkaURL(leaderConnectionInfo.getAddress());
 		} catch (Exception e) {
@@ -464,7 +464,7 @@ public abstract class ClusterClient {
 	public JobExecutionResult retrieveJob(JobID jobID) throws JobExecutionException {
 		final LeaderRetrievalService leaderRetrievalService;
 		try {
-			leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(flinkConfig);
+			leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(flinkConfig, true);
 		} catch (Exception e) {
 			throw new JobRetrievalException(jobID, "Could not create the leader retrieval service", e);
 		}
@@ -498,7 +498,7 @@ public abstract class ClusterClient {
 	public JobListeningContext connectToJob(JobID jobID) throws JobExecutionException {
 		final LeaderRetrievalService leaderRetrievalService;
 		try {
-			leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(flinkConfig);
+			leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(flinkConfig, true);
 		} catch (Exception e) {
 			throw new JobRetrievalException(jobID, "Could not create the leader retrieval service", e);
 		}
@@ -721,7 +721,7 @@ public abstract class ClusterClient {
 	public ActorGateway getJobManagerGateway() throws Exception {
 		LOG.debug("Looking up JobManager");
 		return LeaderRetrievalUtils.retrieveLeaderGateway(
-			LeaderRetrievalUtils.createLeaderRetrievalService(flinkConfig),
+			LeaderRetrievalUtils.createLeaderRetrievalService(flinkConfig, true),
 			actorSystemLoader.get(),
 			lookupTimeout);
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/a0bb99c7/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 07edb3a..d8fb3de 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
@@ -23,6 +23,7 @@ import org.apache.flink.api.java.ExecutionEnvironment;
 import org.apache.flink.api.java.io.DiscardingOutputFormat;
 import org.apache.flink.client.program.ProgramInvocationException;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.util.TestLogger;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
@@ -36,7 +37,7 @@ import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 import static org.junit.Assume.assumeTrue;
 
-public class RemoteExecutorHostnameResolutionTest {
+public class RemoteExecutorHostnameResolutionTest extends TestLogger {
 
 	private static final String nonExistingHostname = "foo.bar.com.invalid";
 	private static final int port = 14451;

http://git-wip-us.apache.org/repos/asf/flink/blob/a0bb99c7/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 75cb0e7..da297d6 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
@@ -43,7 +43,7 @@ import org.apache.flink.runtime.akka.AkkaUtils;
 import org.apache.flink.runtime.akka.FlinkUntypedActor;
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
-import org.apache.flink.runtime.jobmanager.JobManager;
+import org.apache.flink.runtime.jobmaster.JobMaster;
 import org.apache.flink.runtime.messages.JobManagerMessages;
 import org.apache.flink.runtime.util.SerializedThrowable;
 import org.apache.flink.util.NetUtils;
@@ -67,7 +67,6 @@ import static org.mockito.Mockito.doAnswer;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
-
 /**
  * Simple and maybe stupid test to check the {@link ClusterClient} class.
  */
@@ -129,7 +128,9 @@ public class ClientTest extends TestLogger {
 	 */
 	@Test
 	public void testDetachedMode() throws Exception{
-		jobManagerSystem.actorOf(Props.create(SuccessReturningActor.class), JobManager.JOB_MANAGER_NAME());
+		jobManagerSystem.actorOf(
+			Props.create(SuccessReturningActor.class),
+			JobMaster.JOB_MANAGER_NAME);
 		ClusterClient out = new StandaloneClusterClient(config);
 		out.setDetached(true);
 
@@ -198,22 +199,18 @@ public class ClientTest extends TestLogger {
 	 * This test verifies correct job submission messaging logic and plan translation calls.
 	 */
 	@Test
-	public void shouldSubmitToJobClient() {
-		try {
-			jobManagerSystem.actorOf(Props.create(SuccessReturningActor.class), JobManager.JOB_MANAGER_NAME());
+	public void shouldSubmitToJobClient() throws IOException, ProgramInvocationException {
+		jobManagerSystem.actorOf(
+			Props.create(SuccessReturningActor.class),
+			JobMaster.JOB_MANAGER_NAME);
 
-			ClusterClient out = new StandaloneClusterClient(config);
-			out.setDetached(true);
-			JobSubmissionResult result = out.run(program.getPlanWithJars(), 1);
+		ClusterClient out = new StandaloneClusterClient(config);
+		out.setDetached(true);
+		JobSubmissionResult result = out.run(program.getPlanWithJars(), 1);
 
-			assertNotNull(result);
+		assertNotNull(result);
 
-			program.deleteExtractedLibraries();
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
+		program.deleteExtractedLibraries();
 	}
 
 	/**
@@ -221,7 +218,9 @@ public class ClientTest extends TestLogger {
 	 */
 	@Test
 	public void shouldSubmitToJobClientFails() throws IOException {
-		jobManagerSystem.actorOf(Props.create(FailureReturningActor.class), JobManager.JOB_MANAGER_NAME());
+			jobManagerSystem.actorOf(
+				Props.create(FailureReturningActor.class),
+				JobMaster.JOB_MANAGER_NAME);
 
 		ClusterClient out = new StandaloneClusterClient(config);
 		out.setDetached(true);
@@ -245,7 +244,9 @@ public class ClientTest extends TestLogger {
 	@Test
 	public void tryLocalExecution() {
 		try {
-			jobManagerSystem.actorOf(Props.create(SuccessReturningActor.class), JobManager.JOB_MANAGER_NAME());
+			jobManagerSystem.actorOf(
+				Props.create(SuccessReturningActor.class),
+				JobMaster.JOB_MANAGER_NAME);
 			
 			PackagedProgram packagedProgramMock = mock(PackagedProgram.class);
 			when(packagedProgramMock.isUsingInteractiveMode()).thenReturn(true);
@@ -276,7 +277,9 @@ public class ClientTest extends TestLogger {
 	@Test
 	public void testGetExecutionPlan() {
 		try {
-			jobManagerSystem.actorOf(Props.create(FailureReturningActor.class), JobManager.JOB_MANAGER_NAME());
+			jobManagerSystem.actorOf(
+				Props.create(FailureReturningActor.class),
+				JobMaster.JOB_MANAGER_NAME);
 			
 			PackagedProgram prg = new PackagedProgram(TestOptimizerPlan.class, "/dev/random", "/tmp");
 			assertNotNull(prg.getPreviewPlan());

http://git-wip-us.apache.org/repos/asf/flink/blob/a0bb99c7/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
index dd7d8bc..fc10f65 100644
--- 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
@@ -56,7 +56,7 @@ public class LeaderRetrievalServiceHostnameResolutionTest extends TestLogger {
 			config.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, nonExistingHostname);
 			config.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, 17234);
 
-			LeaderRetrievalUtils.createLeaderRetrievalService(config);
+			LeaderRetrievalUtils.createLeaderRetrievalService(config, false);
 		}
 		catch (Exception e) {
 			System.err.println("Shouldn't throw an exception!");
@@ -69,7 +69,7 @@ public class LeaderRetrievalServiceHostnameResolutionTest extends TestLogger {
 	 * Tests that the StandaloneLeaderRetrievalService does not resolve host names by default.
 	 */
 	@Test
-	public void testUnresolvableHostname2() {
+	public void testUnresolvableHostname2() throws Exception {
 
 		try {
 			Configuration config = new Configuration();
@@ -78,16 +78,11 @@ public class LeaderRetrievalServiceHostnameResolutionTest extends TestLogger {
 			config.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, 17234);
 
 			LeaderRetrievalUtils.createLeaderRetrievalService(config, true);
-			fail("This should fail with an IllegalConfigurationException");
+			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() {

http://git-wip-us.apache.org/repos/asf/flink/blob/a0bb99c7/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkClient.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkClient.java b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkClient.java
index c58a8ee..2b7f357 100644
--- a/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkClient.java
+++ b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkClient.java
@@ -22,6 +22,9 @@ import akka.actor.ActorRef;
 import akka.actor.ActorSystem;
 import akka.pattern.Patterns;
 import akka.util.Timeout;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils.AddressResolution;
+import org.apache.flink.runtime.jobmaster.JobMaster;
+import org.apache.flink.runtime.rpc.akka.AkkaRpcServiceUtils;
 import org.apache.storm.Config;
 import org.apache.storm.generated.AlreadyAliveException;
 import org.apache.storm.generated.InvalidTopologyException;
@@ -43,12 +46,10 @@ import org.apache.flink.core.fs.Path;
 import org.apache.flink.runtime.akka.AkkaUtils;
 import org.apache.flink.runtime.client.JobStatusMessage;
 import org.apache.flink.runtime.jobgraph.JobGraph;
-import org.apache.flink.runtime.jobmanager.JobManager;
 import org.apache.flink.runtime.messages.JobManagerMessages;
 import org.apache.flink.runtime.messages.JobManagerMessages.RunningJobsStatus;
 import org.apache.flink.storm.util.StormConfig;
 import org.apache.flink.streaming.api.graph.StreamGraph;
-import org.apache.flink.util.NetUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import scala.Some;
@@ -326,9 +327,14 @@ public class FlinkClient {
 			throw new RuntimeException("Could not start actor system to communicate with JobManager", e);
 		}
 
-		return JobManager.getJobManagerActorRef(AkkaUtils.getAkkaProtocol(configuration),
-				NetUtils.unresolvedHostAndPortToNormalizedString(this.jobManagerHost, this.jobManagerPort),
-				actorSystem, AkkaUtils.getLookupTimeout(configuration));
+		final String jobManagerAkkaUrl = AkkaRpcServiceUtils.getRpcUrl(
+			jobManagerHost,
+			jobManagerPort,
+			JobMaster.JOB_MANAGER_NAME,
+			AddressResolution.TRY_ADDRESS_RESOLUTION,
+			configuration);
+
+		return AkkaUtils.getActorRef(jobManagerAkkaUrl, actorSystem, AkkaUtils.getLookupTimeout(configuration));
 	}
 
 	@SuppressWarnings({ "unchecked", "rawtypes" })

http://git-wip-us.apache.org/repos/asf/flink/blob/a0bb99c7/flink-core/src/main/java/org/apache/flink/util/ConfigurationException.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/util/ConfigurationException.java b/flink-core/src/main/java/org/apache/flink/util/ConfigurationException.java
new file mode 100644
index 0000000..4623fa3
--- /dev/null
+++ b/flink-core/src/main/java/org/apache/flink/util/ConfigurationException.java
@@ -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.
+ */
+
+package org.apache.flink.util;
+
+/**
+ * Exception which occurs when creating a configuration object fails.
+ */
+public class ConfigurationException extends FlinkException {
+	private static final long serialVersionUID = 3971647332059381556L;
+
+	public ConfigurationException(String message) {
+		super(message);
+	}
+
+	public ConfigurationException(String message, Throwable cause) {
+		super(message, cause);
+	}
+
+	public ConfigurationException(Throwable cause) {
+		super(cause);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/a0bb99c7/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosApplicationMasterRunner.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosApplicationMasterRunner.java b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosApplicationMasterRunner.java
index a23c9f6..0c12745 100644
--- a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosApplicationMasterRunner.java
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosApplicationMasterRunner.java
@@ -50,6 +50,7 @@ import org.apache.flink.runtime.clusterframework.overlays.Krb5ConfOverlay;
 import org.apache.flink.runtime.clusterframework.overlays.SSLStoreOverlay;
 import org.apache.flink.runtime.jobmanager.JobManager;
 import org.apache.flink.runtime.jobmanager.MemoryArchivist;
+import org.apache.flink.runtime.jobmaster.JobMaster;
 import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
 import org.apache.flink.runtime.process.ProcessReaper;
 import org.apache.flink.runtime.security.SecurityUtils;
@@ -65,6 +66,8 @@ import org.apache.mesos.Protos;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import scala.Option;
+import scala.Some;
 import scala.concurrent.duration.Duration;
 import scala.concurrent.duration.FiniteDuration;
 
@@ -301,8 +304,8 @@ public class MesosApplicationMasterRunner {
 				actorSystem,
 				futureExecutor,
 				ioExecutor,
-				new scala.Some<>(JobManager.JOB_MANAGER_NAME()),
-				scala.Option.<String>empty(),
+				new Some<>(JobMaster.JOB_MANAGER_NAME),
+				Option.<String>empty(),
 				getJobManagerClass(),
 				getArchivistClass())._1();
 

http://git-wip-us.apache.org/repos/asf/flink/blob/a0bb99c7/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/HandlerRedirectUtils.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/HandlerRedirectUtils.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/HandlerRedirectUtils.java
index 6616a2a..144e519 100644
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/HandlerRedirectUtils.java
+++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/HandlerRedirectUtils.java
@@ -25,12 +25,11 @@ import io.netty.handler.codec.http.HttpResponse;
 import io.netty.handler.codec.http.HttpResponseStatus;
 import io.netty.handler.codec.http.HttpVersion;
 import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.runtime.akka.AkkaUtils;
 import org.apache.flink.runtime.instance.ActorGateway;
-import org.apache.flink.runtime.jobmanager.JobManager;
 import org.apache.flink.runtime.webmonitor.files.MimeTypes;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import scala.Option;
 import scala.Tuple2;
 
 import java.util.regex.Matcher;
@@ -62,7 +61,7 @@ public class HandlerRedirectUtils {
 		final String jobManagerName = localJobManagerAddress.substring(localJobManagerAddress.lastIndexOf("/") + 1);
 
 		if (!localJobManagerAddress.equals(leaderAddress) &&
-			!leaderAddress.equals(JobManager.getLocalJobManagerAkkaURL(Option.apply(jobManagerName)))) {
+			!leaderAddress.equals(AkkaUtils.getLocalAkkaURL(jobManagerName))) {
 			// We are not the leader and need to redirect
 			Matcher matcher = LeaderAddressHostPattern.matcher(leaderAddress);
 

http://git-wip-us.apache.org/repos/asf/flink/blob/a0bb99c7/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobUtils.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobUtils.java
index c540f74..b0cab78 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobUtils.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobUtils.java
@@ -24,7 +24,7 @@ import org.apache.flink.api.common.JobID;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.IllegalConfigurationException;
-import org.apache.flink.runtime.highavailability.ZookeeperHaServices;
+import org.apache.flink.runtime.highavailability.zookeeper.ZooKeeperHaServices;
 import org.apache.flink.runtime.jobmanager.HighAvailabilityMode;
 import org.apache.flink.util.StringUtils;
 import org.slf4j.Logger;
@@ -84,7 +84,7 @@ public class BlobUtils {
 		if (highAvailabilityMode == HighAvailabilityMode.NONE) {
 			return new VoidBlobStore();
 		} else if (highAvailabilityMode == HighAvailabilityMode.ZOOKEEPER) {
-			return ZookeeperHaServices.createBlobStore(config);
+			return ZooKeeperHaServices.createBlobStore(config);
 		} else {
 			throw new IllegalConfigurationException("Unexpected high availability mode '" + highAvailabilityMode + "'.");
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/a0bb99c7/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobListeningContext.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobListeningContext.java b/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobListeningContext.java
index b5d7cb7..b944ba8 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobListeningContext.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobListeningContext.java
@@ -135,7 +135,7 @@ public final class JobListeningContext {
 	private ActorGateway getJobManager() throws JobRetrievalException {
 		try {
 			return LeaderRetrievalUtils.retrieveLeaderGateway(
-				LeaderRetrievalUtils.createLeaderRetrievalService(configuration),
+				LeaderRetrievalUtils.createLeaderRetrievalService(configuration, true),
 				actorSystem,
 				AkkaUtils.getLookupTimeout(configuration));
 		} catch (Exception e) {

http://git-wip-us.apache.org/repos/asf/flink/blob/a0bb99c7/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/EmbeddedNonHaServices.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/EmbeddedNonHaServices.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/EmbeddedNonHaServices.java
deleted file mode 100644
index 8bf81eb..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/EmbeddedNonHaServices.java
+++ /dev/null
@@ -1,70 +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.highavailability;
-
-import org.apache.flink.runtime.highavailability.nonha.AbstractNonHaServices;
-import org.apache.flink.runtime.highavailability.nonha.EmbeddedLeaderService;
-import org.apache.flink.runtime.leaderelection.LeaderElectionService;
-import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
-
-/**
- * An implementation of the {@link HighAvailabilityServices} for the non-high-availability case
- * where all participants (ResourceManager, JobManagers, TaskManagers) run in the same process.
- *
- * <p>This implementation has no dependencies on any external services. It returns a fix
- * pre-configured ResourceManager, and stores checkpoints and metadata simply on the heap or
- * on a local file system and therefore in a storage without guarantees.
- */
-public class EmbeddedNonHaServices extends AbstractNonHaServices implements HighAvailabilityServices {
-
-	private final EmbeddedLeaderService resourceManagerLeaderService;
-
-	public EmbeddedNonHaServices() {
-		super();
-		this.resourceManagerLeaderService = new EmbeddedLeaderService(getExecutorService());
-	}
-
-	// ------------------------------------------------------------------------
-
-	@Override
-	public LeaderRetrievalService getResourceManagerLeaderRetriever() {
-		return resourceManagerLeaderService.createLeaderRetrievalService();
-	}
-
-	@Override
-	public LeaderElectionService getResourceManagerLeaderElectionService() {
-		return resourceManagerLeaderService.createLeaderElectionService();
-	}
-
-	// ------------------------------------------------------------------------
-
-	@Override
-	public void close() throws Exception {
-		try {
-			super.close();
-		} finally {
-			resourceManagerLeaderService.shutdown();
-		}
-	}
-
-	@Override
-	public void closeAndCleanupAllData() throws Exception {
-		close();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/a0bb99c7/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServicesUtils.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServicesUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServicesUtils.java
index fe180de..106be5a 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServicesUtils.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServicesUtils.java
@@ -18,43 +18,111 @@
 
 package org.apache.flink.runtime.highavailability;
 
+import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.concurrent.Executors;
+import org.apache.flink.configuration.JobManagerOptions;
+import org.apache.flink.runtime.highavailability.nonha.embedded.EmbeddedHaServices;
+import org.apache.flink.runtime.highavailability.nonha.standalone.StandaloneHaServices;
+import org.apache.flink.runtime.highavailability.zookeeper.ZooKeeperHaServices;
 import org.apache.flink.runtime.jobmanager.HighAvailabilityMode;
+import org.apache.flink.runtime.jobmaster.JobMaster;
+import org.apache.flink.runtime.resourcemanager.ResourceManager;
+import org.apache.flink.runtime.rpc.akka.AkkaRpcServiceUtils;
 import org.apache.flink.runtime.util.LeaderRetrievalUtils;
 import org.apache.flink.runtime.util.ZooKeeperUtils;
+import org.apache.flink.util.ConfigurationException;
 
+import java.util.concurrent.Executor;
+
+/**
+ * Utils class to instantiate {@link HighAvailabilityServices} implementations.
+ */
 public class HighAvailabilityServicesUtils {
 
-	public static HighAvailabilityServices createAvailableOrEmbeddedServices(Configuration config) throws Exception {
+	public static HighAvailabilityServices createAvailableOrEmbeddedServices(
+		Configuration config,
+		Executor executor) throws Exception {
 		HighAvailabilityMode highAvailabilityMode = LeaderRetrievalUtils.getRecoveryMode(config);
 
 		switch (highAvailabilityMode) {
 			case NONE:
-				return new EmbeddedNonHaServices();
+				return new EmbeddedHaServices(executor);
 
 			case ZOOKEEPER:
-				return new ZookeeperHaServices(ZooKeeperUtils.startCuratorFramework(config), 
-						Executors.directExecutor(), config);
+				return new ZooKeeperHaServices(
+					ZooKeeperUtils.startCuratorFramework(config),
+					executor,
+					config);
 
 			default:
 				throw new Exception("High availability mode " + highAvailabilityMode + " is not supported.");
 		}
 	}
 	
-	
-	public static HighAvailabilityServices createHighAvailabilityServices(Configuration configuration) throws Exception {
+	public static HighAvailabilityServices createHighAvailabilityServices(
+		Configuration configuration,
+		Executor executor,
+		AddressResolution addressResolution) throws Exception {
+
 		HighAvailabilityMode highAvailabilityMode = LeaderRetrievalUtils.getRecoveryMode(configuration);
 
 		switch(highAvailabilityMode) {
 			case NONE:
-				final String resourceManagerAddress = null;
-				return new NonHaServices(resourceManagerAddress);
+				final Tuple2<String, Integer> hostnamePort = getJobManagerAddress(configuration);
+
+				final String jobManagerRpcUrl = AkkaRpcServiceUtils.getRpcUrl(
+					hostnamePort.f0,
+					hostnamePort.f1,
+					JobMaster.JOB_MANAGER_NAME,
+					addressResolution,
+					configuration);
+				final String resourceManagerRpcUrl = AkkaRpcServiceUtils.getRpcUrl(
+					hostnamePort.f0,
+					hostnamePort.f1,
+					ResourceManager.RESOURCE_MANAGER_NAME,
+					addressResolution,
+					configuration);
+
+				return new StandaloneHaServices(resourceManagerRpcUrl, jobManagerRpcUrl);
 			case ZOOKEEPER:
-				return new ZookeeperHaServices(ZooKeeperUtils.startCuratorFramework(configuration), 
-						Executors.directExecutor(), configuration);
+				return new ZooKeeperHaServices(
+					ZooKeeperUtils.startCuratorFramework(configuration),
+					executor,
+					configuration);
 			default:
 				throw new Exception("Recovery mode " + highAvailabilityMode + " is not supported.");
 		}
 	}
+
+	/**
+	 * Returns the JobManager's hostname and port extracted from the given
+	 * {@link Configuration}.
+	 *
+	 * @param configuration Configuration to extract the JobManager's address from
+	 * @return The JobManager's hostname and port
+	 * @throws ConfigurationException if the JobManager's address cannot be extracted from the configuration
+	 */
+	public static Tuple2<String, Integer> getJobManagerAddress(Configuration configuration) throws ConfigurationException {
+
+		final String hostname = configuration.getString(JobManagerOptions.ADDRESS);
+		final int port = configuration.getInteger(JobManagerOptions.PORT);
+
+		if (hostname == null) {
+			throw new ConfigurationException("Config parameter '" + JobManagerOptions.ADDRESS +
+				"' is missing (hostname/address of JobManager to connect to).");
+		}
+
+		if (port <= 0 || port >= 65536) {
+			throw new ConfigurationException("Invalid value for '" + JobManagerOptions.PORT +
+				"' (port of the JobManager actor system) : " + port +
+				".  it must be great than 0 and less than 65536.");
+		}
+
+		return Tuple2.of(hostname, port);
+	}
+
+	public enum AddressResolution {
+		TRY_ADDRESS_RESOLUTION,
+		NO_ADDRESS_RESOLUTION
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/a0bb99c7/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/NonHaServices.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/NonHaServices.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/NonHaServices.java
deleted file mode 100644
index beb5963..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/NonHaServices.java
+++ /dev/null
@@ -1,69 +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.highavailability;
-
-import org.apache.flink.runtime.highavailability.leaderelection.SingleLeaderElectionService;
-import org.apache.flink.runtime.highavailability.nonha.AbstractNonHaServices;
-import org.apache.flink.runtime.leaderelection.LeaderElectionService;
-import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
-import org.apache.flink.runtime.leaderretrieval.StandaloneLeaderRetrievalService;
-
-import static org.apache.flink.util.Preconditions.checkNotNull;
-
-/**
- * An implementation of the {@link HighAvailabilityServices} for the non-high-availability case.
- * This implementation can be used for testing, and for cluster setups that do not
- * tolerate failures of the master processes (JobManager, ResourceManager).
- * 
- * <p>This implementation has no dependencies on any external services. It returns a fix
- * pre-configured ResourceManager, and stores checkpoints and metadata simply on the heap or
- * on a local file system and therefore in a storage without guarantees.
- */
-public class NonHaServices extends AbstractNonHaServices implements HighAvailabilityServices {
-
-	/** The constant name of the ResourceManager RPC endpoint */
-	private static final String RESOURCE_MANAGER_RPC_ENDPOINT_NAME = "resource_manager";
-
-	/** The fix address of the ResourceManager */
-	private final String resourceManagerAddress;
-
-	/**
-	 * Creates a new services class for the fix pre-defined leaders.
-	 * 
-	 * @param resourceManagerAddress    The fix address of the ResourceManager
-	 */
-	public NonHaServices(String resourceManagerAddress) {
-		super();
-		this.resourceManagerAddress = checkNotNull(resourceManagerAddress);
-	}
-
-	// ------------------------------------------------------------------------
-	//  Services
-	// ------------------------------------------------------------------------
-
-	@Override
-	public LeaderRetrievalService getResourceManagerLeaderRetriever() {
-		return new StandaloneLeaderRetrievalService(resourceManagerAddress, DEFAULT_LEADER_ID);
-	}
-
-	@Override
-	public LeaderElectionService getResourceManagerLeaderElectionService() {
-		return new SingleLeaderElectionService(getExecutorService(), DEFAULT_LEADER_ID);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/a0bb99c7/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/ZookeeperHaServices.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/ZookeeperHaServices.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/ZookeeperHaServices.java
deleted file mode 100644
index 4d0db0a..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/ZookeeperHaServices.java
+++ /dev/null
@@ -1,215 +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.highavailability;
-
-import org.apache.curator.framework.CuratorFramework;
-
-import org.apache.flink.api.common.JobID;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.configuration.HighAvailabilityOptions;
-import org.apache.flink.configuration.IllegalConfigurationException;
-import org.apache.flink.core.fs.FileSystem;
-import org.apache.flink.core.fs.Path;
-import org.apache.flink.runtime.blob.BlobStore;
-import org.apache.flink.runtime.blob.FileSystemBlobStore;
-import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory;
-import org.apache.flink.runtime.checkpoint.ZooKeeperCheckpointRecoveryFactory;
-import org.apache.flink.runtime.jobmanager.SubmittedJobGraphStore;
-import org.apache.flink.runtime.leaderelection.LeaderElectionService;
-import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
-import org.apache.flink.runtime.util.ZooKeeperUtils;
-
-import java.io.IOException;
-import java.util.concurrent.Executor;
-
-import static org.apache.flink.util.Preconditions.checkNotNull;
-import static org.apache.flink.util.StringUtils.isNullOrWhitespaceOnly;
-
-/**
- * An implementation of the {@link HighAvailabilityServices} using Apache ZooKeeper.
- * The services store data in ZooKeeper's nodes as illustrated by teh following tree structure:
- * 
- * <pre>
- * /flink
- *      +/cluster_id_1/resource_manager_lock
- *      |            |
- *      |            +/job-id-1/job_manager_lock
- *      |            |         /checkpoints/latest
- *      |            |                     /latest-1
- *      |            |                     /latest-2
- *      |            |
- *      |            +/job-id-2/job_manager_lock
- *      |      
- *      +/cluster_id_2/resource_manager_lock
- *                   |
- *                   +/job-id-1/job_manager_lock
- *                            |/checkpoints/latest
- *                            |            /latest-1
- *                            |/persisted_job_graph
- * </pre>
- * 
- * <p>The root path "/flink" is configurable via the option {@link HighAvailabilityOptions#HA_ZOOKEEPER_ROOT}.
- * This makes sure Flink stores its data under specific subtrees in ZooKeeper, for example to
- * accommodate specific permission.
- * 
- * <p>The "cluster_id" part identifies the data stored for a specific Flink "cluster". 
- * This "cluster" can be either a standalone or containerized Flink cluster, or it can be job
- * on a framework like YARN or Mesos (in a "per-job-cluster" mode).
- * 
- * <p>In case of a "per-job-cluster" on YARN or Mesos, the cluster-id is generated and configured
- * automatically by the client or dispatcher that submits the Job to YARN or Mesos.
- * 
- * <p>In the case of a standalone cluster, that cluster-id needs to be configured via
- * {@link HighAvailabilityOptions#HA_CLUSTER_ID}. All nodes with the same cluster id will join the same
- * cluster and participate in the execution of the same set of jobs.
- */
-public class ZookeeperHaServices implements HighAvailabilityServices {
-
-	private static final String RESOURCE_MANAGER_LEADER_PATH = "/resource_manager_lock";
-
-	private static final String JOB_MANAGER_LEADER_PATH = "/job_manager_lock";
-
-	// ------------------------------------------------------------------------
-	
-	
-	/** The ZooKeeper client to use */
-	private final CuratorFramework client;
-
-	/** The executor to run ZooKeeper callbacks on */
-	private final Executor executor;
-
-	/** The runtime configuration */
-	private final Configuration configuration;
-
-	/** The zookeeper based running jobs registry */
-	private final RunningJobsRegistry runningJobsRegistry;
-
-	public ZookeeperHaServices(CuratorFramework client, Executor executor, Configuration configuration) {
-		this.client = checkNotNull(client);
-		this.executor = checkNotNull(executor);
-		this.configuration = checkNotNull(configuration);
-		this.runningJobsRegistry = new ZookeeperRegistry(client, configuration);
-	}
-
-	// ------------------------------------------------------------------------
-	//  Services
-	// ------------------------------------------------------------------------
-
-	@Override
-	public LeaderRetrievalService getResourceManagerLeaderRetriever() {
-		return ZooKeeperUtils.createLeaderRetrievalService(client, configuration, RESOURCE_MANAGER_LEADER_PATH);
-	}
-
-	@Override
-	public LeaderRetrievalService getJobManagerLeaderRetriever(JobID jobID) {
-		return ZooKeeperUtils.createLeaderRetrievalService(client, configuration, getPathForJobManager(jobID));
-	}
-
-	@Override
-	public LeaderElectionService getResourceManagerLeaderElectionService() {
-		return ZooKeeperUtils.createLeaderElectionService(client, configuration, RESOURCE_MANAGER_LEADER_PATH);
-	}
-
-	@Override
-	public LeaderElectionService getJobManagerLeaderElectionService(JobID jobID) {
-		return ZooKeeperUtils.createLeaderElectionService(client, configuration, getPathForJobManager(jobID));
-	}
-
-	@Override
-	public CheckpointRecoveryFactory getCheckpointRecoveryFactory() {
-		return new ZooKeeperCheckpointRecoveryFactory(client, configuration, executor);
-	}
-
-	@Override
-	public SubmittedJobGraphStore getSubmittedJobGraphStore() throws Exception {
-		return ZooKeeperUtils.createSubmittedJobGraphs(client, configuration, executor);
-	}
-
-	@Override
-	public RunningJobsRegistry getRunningJobsRegistry() {
-		return runningJobsRegistry;
-	}
-
-	@Override
-	public BlobStore createBlobStore() throws IOException {
-		return createBlobStore(configuration);
-	}
-
-	/**
-	 * Creates the BLOB store in which BLOBs are stored in a highly-available
-	 * fashion.
-	 *
-	 * @param configuration configuration to extract the storage path from
-	 * @return Blob store
-	 * @throws IOException if the blob store could not be created
-	 */
-	public static BlobStore createBlobStore(
-		final Configuration configuration) throws IOException {
-		String storagePath = configuration.getValue(
-			HighAvailabilityOptions.HA_STORAGE_PATH);
-		if (isNullOrWhitespaceOnly(storagePath)) {
-			throw new IllegalConfigurationException("Configuration is missing the mandatory parameter: " +
-					HighAvailabilityOptions.HA_STORAGE_PATH);
-		}
-
-		final Path path;
-		try {
-			path = new Path(storagePath);
-		} catch (Exception e) {
-			throw new IOException("Invalid path for highly available storage (" +
-					HighAvailabilityOptions.HA_STORAGE_PATH.key() + ')', e);
-		}
-
-		final FileSystem fileSystem;
-		try {
-			fileSystem = path.getFileSystem();
-		} catch (Exception e) {
-			throw new IOException("Could not create FileSystem for highly available storage (" +
-					HighAvailabilityOptions.HA_STORAGE_PATH.key() + ')', e);
-		}
-
-		final String clusterId =
-			configuration.getValue(HighAvailabilityOptions.HA_CLUSTER_ID);
-		storagePath += "/" + clusterId;
-
-		return new FileSystemBlobStore(fileSystem, storagePath);
-	}
-
-	// ------------------------------------------------------------------------
-	//  Shutdown
-	// ------------------------------------------------------------------------
-
-	@Override
-	public void close() throws Exception {
-		client.close();
-	}
-
-	@Override
-	public void closeAndCleanupAllData() throws Exception {
-		close();
-	}
-
-	// ------------------------------------------------------------------------
-	//  Utilities
-	// ------------------------------------------------------------------------
-
-	private static String getPathForJobManager(final JobID jobID) {
-		return "/" + jobID + JOB_MANAGER_LEADER_PATH;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/a0bb99c7/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/ZookeeperRegistry.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/ZookeeperRegistry.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/ZookeeperRegistry.java
deleted file mode 100644
index a8be35a..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/ZookeeperRegistry.java
+++ /dev/null
@@ -1,127 +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.highavailability;
-
-import org.apache.curator.framework.CuratorFramework;
-import org.apache.flink.api.common.JobID;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.configuration.HighAvailabilityOptions;
-import org.apache.zookeeper.data.Stat;
-
-import java.io.IOException;
-import java.nio.charset.Charset;
-import java.util.Arrays;
-
-import static org.apache.flink.util.Preconditions.checkNotNull;
-
-/**
- * A zookeeper based registry for running jobs, highly available.
- */
-public class ZookeeperRegistry implements RunningJobsRegistry {
-
-	private static final Charset ENCODING = Charset.forName("utf-8");
-
-	/** The ZooKeeper client to use */
-	private final CuratorFramework client;
-
-	private final String runningJobPath;
-
-	public ZookeeperRegistry(final CuratorFramework client, final Configuration configuration) {
-		this.client = checkNotNull(client, "client");
-		this.runningJobPath = configuration.getString(HighAvailabilityOptions.ZOOKEEPER_RUNNING_JOB_REGISTRY_PATH);
-	}
-
-	@Override
-	public void setJobRunning(JobID jobID) throws IOException {
-		checkNotNull(jobID);
-
-		try {
-			writeEnumToZooKeeper(jobID, JobSchedulingStatus.RUNNING);
-		}
-		catch (Exception e) {
-			throw new IOException("Failed to set RUNNING state in ZooKeeper for job " + jobID, e);
-		}
-	}
-
-	@Override
-	public void setJobFinished(JobID jobID) throws IOException {
-		checkNotNull(jobID);
-
-		try {
-			writeEnumToZooKeeper(jobID, JobSchedulingStatus.DONE);
-		}
-		catch (Exception e) {
-			throw new IOException("Failed to set DONE state in ZooKeeper for job " + jobID, e);
-		}
-	}
-
-	@Override
-	public JobSchedulingStatus getJobSchedulingStatus(JobID jobID) throws IOException {
-		checkNotNull(jobID);
-
-		try {
-			final String zkPath = createZkPath(jobID);
-			final Stat stat = client.checkExists().forPath(zkPath);
-			if (stat != null) {
-				// found some data, try to parse it
-				final byte[] data = client.getData().forPath(zkPath);
-				if (data != null) {
-					try {
-						final String name = new String(data, ENCODING);
-						return JobSchedulingStatus.valueOf(name);
-					}
-					catch (IllegalArgumentException e) {
-						throw new IOException("Found corrupt data in ZooKeeper: " + 
-								Arrays.toString(data) + " is no valid job status");
-					}
-				}
-			}
-
-			// nothing found, yet, must be in status 'PENDING'
-			return JobSchedulingStatus.PENDING;
-		}
-		catch (Exception e) {
-			throw new IOException("Get finished state from zk fail for job " + jobID.toString(), e);
-		}
-	}
-
-	@Override
-	public void clearJob(JobID jobID) throws IOException {
-		checkNotNull(jobID);
-
-		try {
-			final String zkPath = createZkPath(jobID);
-			this.client.newNamespaceAwareEnsurePath(zkPath).ensure(client.getZookeeperClient());
-			this.client.delete().forPath(zkPath);
-		}
-		catch (Exception e) {
-			throw new IOException("Failed to clear job state from ZooKeeper for job " + jobID, e);
-		}
-	}
-
-	private String createZkPath(JobID jobID) {
-		return runningJobPath + jobID.toString();
-	}
-
-	private void writeEnumToZooKeeper(JobID jobID, JobSchedulingStatus status) throws Exception {
-		final String zkPath = createZkPath(jobID);
-		this.client.newNamespaceAwareEnsurePath(zkPath).ensure(client.getZookeeperClient());
-		this.client.setData().forPath(zkPath, status.name().getBytes(ENCODING));
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/a0bb99c7/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/leaderelection/SingleLeaderElectionService.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/leaderelection/SingleLeaderElectionService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/leaderelection/SingleLeaderElectionService.java
deleted file mode 100644
index 96e1390..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/leaderelection/SingleLeaderElectionService.java
+++ /dev/null
@@ -1,386 +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.highavailability.leaderelection;
-
-import org.apache.flink.runtime.leaderelection.LeaderContender;
-import org.apache.flink.runtime.leaderelection.LeaderElectionService;
-import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener;
-import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import javax.annotation.Nullable;
-import javax.annotation.concurrent.GuardedBy;
-import java.util.HashSet;
-import java.util.UUID;
-import java.util.concurrent.Executor;
-
-import static org.apache.flink.util.Preconditions.checkArgument;
-import static org.apache.flink.util.Preconditions.checkNotNull;
-import static org.apache.flink.util.Preconditions.checkState;
-
-/**
- * An implementation of the {@link LeaderElectionService} interface that handles a single
- * leader contender. When started, this service immediately grants the contender the leadership.
- * 
- * <p>The implementation accepts a single static leader session ID and is hence compatible with
- * pre-configured single leader (no leader failover) setups.
- * 
- * <p>This implementation supports a series of leader listeners that receive notifications about
- * the leader contender.
- */
-public class SingleLeaderElectionService implements LeaderElectionService {
-
-	private static final Logger LOG = LoggerFactory.getLogger(SingleLeaderElectionService.class);
-
-	// ------------------------------------------------------------------------
-
-	/** lock for all operations on this instance */
-	private final Object lock = new Object();
-
-	/** The executor service that dispatches notifications */
-	private final Executor notificationExecutor;
-
-	/** The leader ID assigned to the immediate leader */
-	private final UUID leaderId;
-
-	@GuardedBy("lock")
-	private final HashSet<EmbeddedLeaderRetrievalService> listeners;
-
-	/** The currently proposed leader */
-	@GuardedBy("lock")
-	private volatile LeaderContender proposedLeader;
-
-	/** The confirmed leader */
-	@GuardedBy("lock")
-	private volatile LeaderContender leader;
-
-	/** The address of the confirmed leader */
-	@GuardedBy("lock")
-	private volatile String leaderAddress;
-
-	/** Flag marking this service as shutdown, meaning it cannot be started again */
-	@GuardedBy("lock")
-	private volatile boolean shutdown;
-
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Creates a new leader election service. The service assigns the given leader ID
-	 * to the leader contender.
-	 * 
-	 * @param leaderId The constant leader ID assigned to the leader.
-	 */
-	public SingleLeaderElectionService(Executor notificationsDispatcher, UUID leaderId) {
-		this.notificationExecutor = checkNotNull(notificationsDispatcher);
-		this.leaderId = checkNotNull(leaderId);
-		this.listeners = new HashSet<>();
-
-		shutdown = false;
-	}
-
-	// ------------------------------------------------------------------------
-	//  leader election service
-	// ------------------------------------------------------------------------
-
-	@Override
-	public void start(LeaderContender contender) throws Exception {
-		checkNotNull(contender, "contender");
-
-		synchronized (lock) {
-			checkState(!shutdown, "service is shut down");
-			checkState(proposedLeader == null, "service already started");
-
-			// directly grant leadership to the given contender
-			proposedLeader = contender;
-			notificationExecutor.execute(new GrantLeadershipCall(contender, leaderId));
-		}
-	}
-
-	@Override
-	public void stop() {
-		synchronized (lock) {
-			// notify all listeners that there is no leader
-			for (EmbeddedLeaderRetrievalService listener : listeners) {
-				notificationExecutor.execute(
-						new NotifyOfLeaderCall(null, null, listener.listener, LOG));
-			}
-
-			// if there was a leader, revoke its leadership
-			if (leader != null) {
-				try {
-					leader.revokeLeadership();
-				} catch (Throwable t) {
-					leader.handleError(t instanceof Exception ? (Exception) t : new Exception(t));
-				}
-			}
-
-			proposedLeader = null;
-			leader = null;
-			leaderAddress = null;
-		}
-	}
-
-	@Override
-	public void confirmLeaderSessionID(UUID leaderSessionID) {
-		checkNotNull(leaderSessionID, "leaderSessionID");
-		checkArgument(leaderSessionID.equals(leaderId), "confirmed wrong leader session id");
-
-		synchronized (lock) {
-			checkState(!shutdown, "service is shut down");
-			checkState(proposedLeader != null, "no leader proposed yet");
-			checkState(leader == null, "leader already confirmed");
-
-			// accept the confirmation
-			final String address = proposedLeader.getAddress();
-			leaderAddress = address;
-			leader = proposedLeader;
-
-			// notify all listeners
-			for (EmbeddedLeaderRetrievalService listener : listeners) {
-				notificationExecutor.execute(
-						new NotifyOfLeaderCall(address, leaderId, listener.listener, LOG));
-			}
-		}
-	}
-
-	@Override
-	public boolean hasLeadership() {
-		synchronized (lock) {
-			return leader != null;
-		}
-	}
-
-	void errorOnGrantLeadership(LeaderContender contender, Throwable error) {
-		LOG.warn("Error notifying leader listener about new leader", error);
-		contender.handleError(error instanceof Exception ? (Exception) error : new Exception(error));
-		
-		synchronized (lock) {
-			if (proposedLeader == contender) {
-				proposedLeader = null;
-				leader = null;
-			}
-		}
-	}
-
-	// ------------------------------------------------------------------------
-	//  shutdown
-	// ------------------------------------------------------------------------
-
-	public boolean isShutdown() {
-		return shutdown;
-	}
-
-	public void shutdown() {
-		shutdownInternally(new Exception("The leader service is shutting down"));
-	}
-
-	private void shutdownInternally(Exception exceptionForHandlers) {
-		synchronized (lock) {
-			if (shutdown) {
-				return;
-			}
-
-			shutdown = true;
-
-			// fail the leader (if there is one)
-			if (leader != null) {
-				try {
-					leader.handleError(exceptionForHandlers);
-				} catch (Throwable ignored) {}
-			}
-
-			// clear all leader status
-			leader = null;
-			proposedLeader = null;
-			leaderAddress = null;
-
-			// fail all registered listeners
-			for (EmbeddedLeaderRetrievalService service : listeners) {
-				service.shutdown(exceptionForHandlers);
-			}
-			listeners.clear();
-		}
-	}
-
-	private void fatalError(Throwable error) {
-		LOG.error("Embedded leader election service encountered a fatal error. Shutting down service.", error);
-
-		shutdownInternally(new Exception("Leader election service is shutting down after a fatal error", error));
-	}
-
-	// ------------------------------------------------------------------------
-	//  leader listeners
-	// ------------------------------------------------------------------------
-
-	public LeaderRetrievalService createLeaderRetrievalService() {
-		checkState(!shutdown, "leader election service is shut down");
-		return new EmbeddedLeaderRetrievalService();
-	}
-
-	void addListener(EmbeddedLeaderRetrievalService service, LeaderRetrievalListener listener) {
-		synchronized (lock) {
-			checkState(!shutdown, "leader election service is shut down");
-			checkState(!service.running, "leader retrieval service is already started");
-
-			try {
-				if (!listeners.add(service)) {
-					throw new IllegalStateException("leader retrieval service was added to this service multiple times");
-				}
-
-				service.listener = listener;
-				service.running = true;
-
-				// if we already have a leader, immediately notify this new listener
-				if (leader != null) {
-					notificationExecutor.execute(
-							new NotifyOfLeaderCall(leaderAddress, leaderId, listener, LOG));
-				}
-			}
-			catch (Throwable t) {
-				fatalError(t);
-			}
-		}
-	}
-
-	void removeListener(EmbeddedLeaderRetrievalService service) {
-		synchronized (lock) {
-			// if the service was not even started, simply do nothing
-			if (!service.running || shutdown) {
-				return;
-			}
-
-			try {
-				if (!listeners.remove(service)) {
-					throw new IllegalStateException("leader retrieval service does not belong to this service");
-				}
-
-				// stop the service
-				service.listener = null;
-				service.running = false;
-			}
-			catch (Throwable t) {
-				fatalError(t);
-			}
-		}
-	}
-
-	// ------------------------------------------------------------------------
-
-	private class EmbeddedLeaderRetrievalService implements LeaderRetrievalService {
-
-		volatile LeaderRetrievalListener listener;
-
-		volatile boolean running;
-
-		@Override
-		public void start(LeaderRetrievalListener listener) throws Exception {
-			checkNotNull(listener);
-			addListener(this, listener);
-		}
-
-		@Override
-		public void stop() throws Exception {
-			removeListener(this);
-		}
-
-		void shutdown(Exception cause) {
-			if (running) {
-				final LeaderRetrievalListener lst = listener;
-				running = false;
-				listener = null;
-
-				try {
-					lst.handleError(cause);
-				} catch (Throwable ignored) {}
-			}
-		}
-	}
-
-	// ------------------------------------------------------------------------
-	//  asynchronous notifications
-	// ------------------------------------------------------------------------
-
-	/**
-	 * This runnable informs a leader contender that it gained leadership.
-	 */
-	private class GrantLeadershipCall implements Runnable {
-
-		private final LeaderContender contender;
-		private final UUID leaderSessionId;
-
-		GrantLeadershipCall(LeaderContender contender, UUID leaderSessionId) {
-
-			this.contender = checkNotNull(contender);
-			this.leaderSessionId = checkNotNull(leaderSessionId);
-		}
-
-		@Override
-		public void run() {
-			try {
-				contender.grantLeadership(leaderSessionId);
-			}
-			catch (Throwable t) {
-				errorOnGrantLeadership(contender, t);
-			}
-		}
-	}
-
-	// ------------------------------------------------------------------------
-
-	/**
-	 * This runnable informs a leader listener of a new leader
-	 */
-	private static class NotifyOfLeaderCall implements Runnable {
-
-		@Nullable
-		private final String address;       // null if leader revoked without new leader
-		@Nullable
-		private final UUID leaderSessionId; // null if leader revoked without new leader
-
-		private final LeaderRetrievalListener listener;
-		private final Logger logger;
-
-		NotifyOfLeaderCall(
-				@Nullable String address,
-				@Nullable UUID leaderSessionId,
-				LeaderRetrievalListener listener,
-				Logger logger) {
-
-			this.address = address;
-			this.leaderSessionId = leaderSessionId;
-			this.listener = checkNotNull(listener);
-			this.logger = checkNotNull(logger);
-		}
-
-		@Override
-		public void run() {
-			try {
-				listener.notifyLeaderAddress(address, leaderSessionId);
-			}
-			catch (Throwable t) {
-				logger.warn("Error notifying leader listener about new leader", t);
-				listener.handleError(t instanceof Exception ? (Exception) t : new Exception(t));
-			}
-		}
-	}
-
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/a0bb99c7/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/AbstractNonHaServices.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/AbstractNonHaServices.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/AbstractNonHaServices.java
index b10e414..ac90e3f 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/AbstractNonHaServices.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/AbstractNonHaServices.java
@@ -18,131 +18,85 @@
 
 package org.apache.flink.runtime.highavailability.nonha;
 
-import org.apache.flink.api.common.JobID;
 import org.apache.flink.runtime.blob.BlobStore;
 import org.apache.flink.runtime.blob.VoidBlobStore;
 import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory;
 import org.apache.flink.runtime.checkpoint.StandaloneCheckpointRecoveryFactory;
 import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
 import org.apache.flink.runtime.highavailability.RunningJobsRegistry;
-import org.apache.flink.runtime.highavailability.ServicesThreadFactory;
+import org.apache.flink.runtime.highavailability.nonha.standalone.StandaloneRunningJobsRegistry;
 import org.apache.flink.runtime.jobmanager.StandaloneSubmittedJobGraphStore;
 import org.apache.flink.runtime.jobmanager.SubmittedJobGraphStore;
-import org.apache.flink.runtime.leaderelection.LeaderElectionService;
-import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
 
 import javax.annotation.concurrent.GuardedBy;
 import java.io.IOException;
-import java.util.HashMap;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
 
-import static org.apache.flink.util.Preconditions.checkNotNull;
 import static org.apache.flink.util.Preconditions.checkState;
 
 /**
- * Base for all {@link HighAvailabilityServices} that are not highly available, but are backed
- * by storage that has no availability guarantees and leader election services that cannot
- * elect among multiple distributed leader contenders.
+ * Abstract base class for non high-availability services.
+ *
+ * This class returns the standalone variants for the checkpoint recovery factory, the submitted
+ * job graph store, the running jobs registry and the blob store.
  */
 public abstract class AbstractNonHaServices implements HighAvailabilityServices {
+	protected final Object lock = new Object();
 
-	private final Object lock = new Object();
-
-	private final ExecutorService executor;
-
-	private final HashMap<JobID, EmbeddedLeaderService> jobManagerLeaderServices;
-
-	private final NonHaRegistry runningJobsRegistry;
+	private final RunningJobsRegistry runningJobsRegistry;
 
 	private boolean shutdown;
 
-	// ------------------------------------------------------------------------
-
 	public AbstractNonHaServices() {
-		this.executor = Executors.newCachedThreadPool(new ServicesThreadFactory());
-		this.jobManagerLeaderServices = new HashMap<>();
-		this.runningJobsRegistry = new NonHaRegistry();
+		this.runningJobsRegistry = new StandaloneRunningJobsRegistry();
+
+		shutdown = false;
 	}
 
-	// ------------------------------------------------------------------------
-	//  services
-	// ------------------------------------------------------------------------
+	// ----------------------------------------------------------------------
+	// HighAvailabilityServices method implementations
+	// ----------------------------------------------------------------------
 
 	@Override
-	public LeaderRetrievalService getJobManagerLeaderRetriever(JobID jobID) {
-		checkNotNull(jobID);
-
+	public CheckpointRecoveryFactory getCheckpointRecoveryFactory() {
 		synchronized (lock) {
 			checkNotShutdown();
-			EmbeddedLeaderService service = getOrCreateJobManagerService(jobID);
-			return service.createLeaderRetrievalService();
+
+			return new StandaloneCheckpointRecoveryFactory();
 		}
 	}
 
 	@Override
-	public LeaderElectionService getJobManagerLeaderElectionService(JobID jobID) {
-		checkNotNull(jobID);
-
+	public SubmittedJobGraphStore getSubmittedJobGraphStore() throws Exception {
 		synchronized (lock) {
 			checkNotShutdown();
-			EmbeddedLeaderService service = getOrCreateJobManagerService(jobID);
-			return service.createLeaderElectionService();
-		}
-	}
 
-	@GuardedBy("lock")
-	private EmbeddedLeaderService getOrCreateJobManagerService(JobID jobID) {
-		EmbeddedLeaderService service = jobManagerLeaderServices.get(jobID);
-		if (service == null) {
-			service = new EmbeddedLeaderService(executor);
-			jobManagerLeaderServices.put(jobID, service);
+			return new StandaloneSubmittedJobGraphStore();
 		}
-		return service;
-	}
-
-	@Override
-	public CheckpointRecoveryFactory getCheckpointRecoveryFactory() {
-		checkNotShutdown();
-		return new StandaloneCheckpointRecoveryFactory();
 	}
 
 	@Override
-	public SubmittedJobGraphStore getSubmittedJobGraphStore() {
-		checkNotShutdown();
-		return new StandaloneSubmittedJobGraphStore();
-	}
+	public RunningJobsRegistry getRunningJobsRegistry() throws Exception {
+		synchronized (lock) {
+			checkNotShutdown();
 
-	@Override
-	public RunningJobsRegistry getRunningJobsRegistry() {
-		checkNotShutdown();
-		return runningJobsRegistry;
+			return runningJobsRegistry;
+		}
 	}
 
 	@Override
 	public BlobStore createBlobStore() throws IOException {
-		checkNotShutdown();
-		return new VoidBlobStore();
-	}
+		synchronized (lock) {
+			checkNotShutdown();
 
-	// ------------------------------------------------------------------------
-	//  shutdown
-	// ------------------------------------------------------------------------
+			return new VoidBlobStore();
+		}
+	}
 
 	@Override
 	public void close() throws Exception {
 		synchronized (lock) {
 			if (!shutdown) {
 				shutdown = true;
-
-				// no further calls should be dispatched
-				executor.shutdownNow();
-
-				// stop all job manager leader services
-				for (EmbeddedLeaderService service : jobManagerLeaderServices.values()) {
-					service.shutdown();
-				}
-				jobManagerLeaderServices.clear();
 			}
 		}
 	}
@@ -153,15 +107,16 @@ public abstract class AbstractNonHaServices implements HighAvailabilityServices
 		close();
 	}
 
-	private void checkNotShutdown() {
+	// ----------------------------------------------------------------------
+	// Helper methods
+	// ----------------------------------------------------------------------
+
+	@GuardedBy("lock")
+	protected void checkNotShutdown() {
 		checkState(!shutdown, "high availability services are shut down");
 	}
 
-	// ------------------------------------------------------------------------
-	//  utilities
-	// ------------------------------------------------------------------------
-
-	protected ExecutorService getExecutorService() {
-		return executor;
+	protected boolean isShutDown() {
+		return shutdown;
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/a0bb99c7/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/EmbeddedLeaderService.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/EmbeddedLeaderService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/EmbeddedLeaderService.java
deleted file mode 100644
index d4eba26..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/EmbeddedLeaderService.java
+++ /dev/null
@@ -1,469 +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.highavailability.nonha;
-
-import org.apache.flink.runtime.leaderelection.LeaderContender;
-import org.apache.flink.runtime.leaderelection.LeaderElectionService;
-import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener;
-import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import javax.annotation.Nullable;
-import javax.annotation.concurrent.GuardedBy;
-import java.util.HashSet;
-import java.util.Set;
-import java.util.UUID;
-import java.util.concurrent.Executor;
-import java.util.concurrent.ExecutorService;
-
-import static org.apache.flink.util.Preconditions.checkNotNull;
-import static org.apache.flink.util.Preconditions.checkState;
-
-/**
- * A simple leader election service, which selects a leader among contenders and notifies listeners.
- * 
- * <p>An election service for contenders can be created via {@link #createLeaderElectionService()},
- * a listener service for leader observers can be created via {@link #createLeaderRetrievalService()}.
- */
-public class EmbeddedLeaderService {
-
-	private static final Logger LOG = LoggerFactory.getLogger(EmbeddedLeaderService.class);
-
-	private final Object lock = new Object();
-
-	private final Executor notificationExecutor;
-
-	private final Set<EmbeddedLeaderElectionService> allLeaderContenders;
-
-	private final Set<EmbeddedLeaderRetrievalService> listeners;
-
-	/** proposed leader, which has been notified of leadership grant, but has not confirmed */
-	private EmbeddedLeaderElectionService currentLeaderProposed;
-
-	/** actual leader that has confirmed leadership and of which listeners have been notified */
-	private EmbeddedLeaderElectionService currentLeaderConfirmed;
-
-	/** fencing UID for the current leader (or proposed leader) */
-	private UUID currentLeaderSessionId;
-
-	/** the cached address of the current leader */
-	private String currentLeaderAddress;
-
-	/** flag marking the service as terminated */
-	private boolean shutdown;
-
-	// ------------------------------------------------------------------------
-
-	public EmbeddedLeaderService(ExecutorService notificationsDispatcher) {
-		this.notificationExecutor = checkNotNull(notificationsDispatcher);
-		this.allLeaderContenders = new HashSet<>();
-		this.listeners = new HashSet<>();
-	}
-
-	// ------------------------------------------------------------------------
-	//  shutdown and errors
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Shuts down this leader election service.
-	 * 
-	 * <p>This method does not perform a clean revocation of the leader status and
-	 * no notification to any leader listeners. It simply notifies all contenders
-	 * and listeners that the service is no longer available.
-	 */
-	public void shutdown() {
-		synchronized (lock) {
-			shutdownInternally(new Exception("Leader election service is shutting down"));
-		}
-	}
-
-	private void fatalError(Throwable error) {
-		LOG.error("Embedded leader election service encountered a fatal error. Shutting down service.", error);
-
-		synchronized (lock) {
-			shutdownInternally(new Exception("Leader election service is shutting down after a fatal error", error));
-		}
-	}
-
-	@GuardedBy("lock")
-	private void shutdownInternally(Exception exceptionForHandlers) {
-		assert Thread.holdsLock(lock);
-
-		if (!shutdown) {
-			// clear all leader status
-			currentLeaderProposed = null;
-			currentLeaderConfirmed = null;
-			currentLeaderSessionId = null;
-			currentLeaderAddress = null;
-
-			// fail all registered listeners
-			for (EmbeddedLeaderElectionService service : allLeaderContenders) {
-				service.shutdown(exceptionForHandlers);
-			}
-			allLeaderContenders.clear();
-
-			// fail all registered listeners
-			for (EmbeddedLeaderRetrievalService service : listeners) {
-				service.shutdown(exceptionForHandlers);
-			}
-			listeners.clear();
-
-			shutdown = true;
-		}
-	}
-
-	// ------------------------------------------------------------------------
-	//  creating contenders and listeners
-	// ------------------------------------------------------------------------
-
-	public LeaderElectionService createLeaderElectionService() {
-		checkState(!shutdown, "leader election service is shut down");
-		return new EmbeddedLeaderElectionService();
-	}
-
-	public LeaderRetrievalService createLeaderRetrievalService() {
-		checkState(!shutdown, "leader election service is shut down");
-		return new EmbeddedLeaderRetrievalService();
-	}
-
-	// ------------------------------------------------------------------------
-	//  adding and removing contenders & listeners
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Callback from leader contenders when they start their service.
-	 */
-	void addContender(EmbeddedLeaderElectionService service, LeaderContender contender) {
-		synchronized (lock) {
-			checkState(!shutdown, "leader election service is shut down");
-			checkState(!service.running, "leader election service is already started");
-
-			try {
-				if (!allLeaderContenders.add(service)) {
-					throw new IllegalStateException("leader election service was added to this service multiple times");
-				}
-
-				service.contender = contender;
-				service.running = true;
-
-				updateLeader();
-			}
-			catch (Throwable t) {
-				fatalError(t);
-			}
-		}
-	}
-
-	/**
-	 * Callback from leader contenders when they stop their service.
-	 */
-	void removeContender(EmbeddedLeaderElectionService service) {
-		synchronized (lock) {
-			// if the service was not even started, simply do nothing
-			if (!service.running || shutdown) {
-				return;
-			}
-
-			try {
-				if (!allLeaderContenders.remove(service)) {
-					throw new IllegalStateException("leader election service does not belong to this service");
-				}
-
-				// stop the service
-				service.contender = null;
-				service.running = false;
-				service.isLeader = false;
-
-				// if that was the current leader, unset its status
-				if (currentLeaderConfirmed == service) {
-					currentLeaderConfirmed = null;
-					currentLeaderSessionId = null;
-					currentLeaderAddress = null;
-				}
-				if (currentLeaderProposed == service) {
-					currentLeaderProposed = null;
-					currentLeaderSessionId = null;
-				}
-
-				updateLeader();
-			}
-			catch (Throwable t) {
-				fatalError(t);
-			}
-		}
-	}
-
-	/**
-	 * Callback from leader contenders when they confirm a leader grant
-	 */
-	void confirmLeader(final EmbeddedLeaderElectionService service, final UUID leaderSessionId) {
-		synchronized (lock) {
-			// if the service was shut down in the meantime, ignore this confirmation
-			if (!service.running || shutdown) {
-				return;
-			}
-
-			try {
-				// check if the confirmation is for the same grant, or whether it is a stale grant 
-				if (service == currentLeaderProposed && currentLeaderSessionId.equals(leaderSessionId)) {
-					final String address = service.contender.getAddress();
-					LOG.info("Received confirmation of leadership for leader {} , session={}", address, leaderSessionId);
-
-					// mark leadership
-					currentLeaderConfirmed = service;
-					currentLeaderAddress = address;
-					currentLeaderProposed = null;
-					service.isLeader = true;
-
-					// notify all listeners
-					for (EmbeddedLeaderRetrievalService listener : listeners) {
-						notificationExecutor.execute(
-								new NotifyOfLeaderCall(address, leaderSessionId, listener.listener, LOG));
-					}
-				}
-				else {
-					LOG.debug("Received confirmation of leadership for a stale leadership grant. Ignoring.");
-				}
-			}
-			catch (Throwable t) {
-				fatalError(t);
-			}
-		}
-	}
-
-	@GuardedBy("lock")
-	private void updateLeader() {
-		// this must be called under the lock
-		assert Thread.holdsLock(lock);
-
-		if (currentLeaderConfirmed == null && currentLeaderProposed == null) {
-			// we need a new leader
-			if (allLeaderContenders.isEmpty()) {
-				// no new leader available, tell everyone that there is no leader currently
-				for (EmbeddedLeaderRetrievalService listener : listeners) {
-					notificationExecutor.execute(
-							new NotifyOfLeaderCall(null, null, listener.listener, LOG));
-				}
-			}
-			else {
-				// propose a leader and ask it
-				final UUID leaderSessionId = UUID.randomUUID();
-				EmbeddedLeaderElectionService leaderService = allLeaderContenders.iterator().next();
-
-				currentLeaderSessionId = leaderSessionId;
-				currentLeaderProposed = leaderService;
-
-				LOG.info("Proposing leadership to contender {} @ {}",
-						leaderService.contender, leaderService.contender.getAddress());
-
-				notificationExecutor.execute(
-						new GrantLeadershipCall(leaderService.contender, leaderSessionId, LOG));
-			}
-		}
-	}
-
-	void addListener(EmbeddedLeaderRetrievalService service, LeaderRetrievalListener listener) {
-		synchronized (lock) {
-			checkState(!shutdown, "leader election service is shut down");
-			checkState(!service.running, "leader retrieval service is already started");
-
-			try {
-				if (!listeners.add(service)) {
-					throw new IllegalStateException("leader retrieval service was added to this service multiple times");
-				}
-
-				service.listener = listener;
-				service.running = true;
-
-				// if we already have a leader, immediately notify this new listener
-				if (currentLeaderConfirmed != null) {
-					notificationExecutor.execute(
-							new NotifyOfLeaderCall(currentLeaderAddress, currentLeaderSessionId, listener, LOG));
-				}
-			}
-			catch (Throwable t) {
-				fatalError(t);
-			}
-		}
-	}
-
-	void removeListener(EmbeddedLeaderRetrievalService service) {
-		synchronized (lock) {
-			// if the service was not even started, simply do nothing
-			if (!service.running || shutdown) {
-				return;
-			}
-
-			try {
-				if (!listeners.remove(service)) {
-					throw new IllegalStateException("leader retrieval service does not belong to this service");
-				}
-
-				// stop the service
-				service.listener = null;
-				service.running = false;
-			}
-			catch (Throwable t) {
-				fatalError(t);
-			}
-		}
-	}
-
-	// ------------------------------------------------------------------------
-	//  election and retrieval service implementations 
-	// ------------------------------------------------------------------------
-
-	private class EmbeddedLeaderElectionService implements LeaderElectionService {
-
-		volatile LeaderContender contender;
-
-		volatile boolean isLeader;
-
-		volatile boolean running;
-
-		@Override
-		public void start(LeaderContender contender) throws Exception {
-			checkNotNull(contender);
-			addContender(this, contender);
-		}
-
-		@Override
-		public void stop() throws Exception {
-			removeContender(this);
-		}
-
-		@Override
-		public void confirmLeaderSessionID(UUID leaderSessionID) {
-			checkNotNull(leaderSessionID);
-			confirmLeader(this, leaderSessionID);
-		}
-
-		@Override
-		public boolean hasLeadership() {
-			return isLeader;
-		}
-
-		void shutdown(Exception cause) {
-			if (running) {
-				running = false;
-				isLeader = false;
-				contender.handleError(cause);
-				contender = null;
-			}
-		}
-	}
-
-	// ------------------------------------------------------------------------
-
-	private class EmbeddedLeaderRetrievalService implements LeaderRetrievalService {
-
-		volatile LeaderRetrievalListener listener;
-
-		volatile boolean running;
-
-		@Override
-		public void start(LeaderRetrievalListener listener) throws Exception {
-			checkNotNull(listener);
-			addListener(this, listener);
-		}
-
-		@Override
-		public void stop() throws Exception {
-			removeListener(this);
-		}
-
-		public void shutdown(Exception cause) {
-			if (running) {
-				running = false;
-				listener.handleError(cause);
-				listener = null;
-			}
-		}
-	}
-
-	// ------------------------------------------------------------------------
-	//  asynchronous notifications
-	// ------------------------------------------------------------------------
-
-	private static class NotifyOfLeaderCall implements Runnable {
-
-		@Nullable
-		private final String address;       // null if leader revoked without new leader
-		@Nullable
-		private final UUID leaderSessionId; // null if leader revoked without new leader
-
-		private final LeaderRetrievalListener listener;
-		private final Logger logger;
-
-		NotifyOfLeaderCall(
-				@Nullable String address,
-				@Nullable UUID leaderSessionId,
-				LeaderRetrievalListener listener,
-				Logger logger) {
-
-			this.address = address;
-			this.leaderSessionId = leaderSessionId;
-			this.listener = checkNotNull(listener);
-			this.logger = checkNotNull(logger);
-		}
-
-		@Override
-		public void run() {
-			try {
-				listener.notifyLeaderAddress(address, leaderSessionId);
-			}
-			catch (Throwable t) {
-				logger.warn("Error notifying leader listener about new leader", t);
-				listener.handleError(t instanceof Exception ? (Exception) t : new Exception(t));
-			}
-		}
-	}
-
-	// ------------------------------------------------------------------------
-
-	private static class GrantLeadershipCall implements Runnable {
-
-		private final LeaderContender contender;
-		private final UUID leaderSessionId;
-		private final Logger logger;
-
-		GrantLeadershipCall(
-				LeaderContender contender,
-				UUID leaderSessionId,
-				Logger logger) {
-
-			this.contender = checkNotNull(contender);
-			this.leaderSessionId = checkNotNull(leaderSessionId);
-			this.logger = checkNotNull(logger);
-		}
-
-		@Override
-		public void run() {
-			try {
-				contender.grantLeadership(leaderSessionId);
-			}
-			catch (Throwable t) {
-				logger.warn("Error granting leadership to contender", t);
-				contender.handleError(t instanceof Exception ? (Exception) t : new Exception(t));
-			}
-		}
-	}
-}


[05/16] flink git commit: [FLINK-6078] Remove CuratorFramework#close calls from ZooKeeper based HA services

Posted by tr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureCancelingITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureCancelingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureCancelingITCase.java
index 5ba651f..9d2806c 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureCancelingITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureCancelingITCase.java
@@ -31,8 +31,11 @@ import org.apache.flink.api.java.io.DiscardingOutputFormat;
 import org.apache.flink.client.program.ProgramInvocationException;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.JobManagerOptions;
 import org.apache.flink.runtime.akka.AkkaUtils;
 import org.apache.flink.runtime.client.JobStatusMessage;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils;
 import org.apache.flink.runtime.jobmanager.JobManager;
 import org.apache.flink.runtime.jobmanager.MemoryArchivist;
 import org.apache.flink.runtime.messages.JobManagerMessages;
@@ -70,12 +73,13 @@ import static org.junit.Assert.fail;
 public class ProcessFailureCancelingITCase extends TestLogger {
 	
 	@Test
-	public void testCancelingOnProcessFailure() {
+	public void testCancelingOnProcessFailure() throws Exception {
 		final StringWriter processOutput = new StringWriter();
 
 		ActorSystem jmActorSystem = null;
 		Process taskManagerProcess = null;
-		
+		HighAvailabilityServices highAvailabilityServices = null;
+
 		try {
 			// check that we run this test only if the java command
 			// is available on this machine
@@ -101,6 +105,13 @@ public class ProcessFailureCancelingITCase extends TestLogger {
 			jmConfig.setString(ConfigConstants.AKKA_WATCH_HEARTBEAT_PAUSE, "2000 s");
 			jmConfig.setInteger(ConfigConstants.AKKA_WATCH_THRESHOLD, 10);
 			jmConfig.setString(ConfigConstants.AKKA_ASK_TIMEOUT, "100 s");
+			jmConfig.setString(JobManagerOptions.ADDRESS, localAddress._1());
+			jmConfig.setInteger(JobManagerOptions.PORT, jobManagerPort);
+
+			highAvailabilityServices = HighAvailabilityServicesUtils.createHighAvailabilityServices(
+				jmConfig,
+				TestingUtils.defaultExecutor(),
+				HighAvailabilityServicesUtils.AddressResolution.NO_ADDRESS_RESOLUTION);
 
 			jmActorSystem = AkkaUtils.createActorSystem(jmConfig, new Some<>(localAddress));
 			ActorRef jmActor = JobManager.startJobManagerActors(
@@ -108,6 +119,7 @@ public class ProcessFailureCancelingITCase extends TestLogger {
 				jmActorSystem,
 				TestingUtils.defaultExecutor(),
 				TestingUtils.defaultExecutor(),
+				highAvailabilityServices,
 				JobManager.class,
 				MemoryArchivist.class)._1();
 
@@ -193,12 +205,10 @@ public class ProcessFailureCancelingITCase extends TestLogger {
 			// all seems well :-)
 		}
 		catch (Exception e) {
-			e.printStackTrace();
 			printProcessLog("TaskManager", processOutput.toString());
-			fail(e.getMessage());
+			throw e;
 		}
 		catch (Error e) {
-			e.printStackTrace();
 			printProcessLog("TaskManager 1", processOutput.toString());
 			throw e;
 		}
@@ -209,6 +219,10 @@ public class ProcessFailureCancelingITCase extends TestLogger {
 			if (jmActorSystem != null) {
 				jmActorSystem.shutdown();
 			}
+
+			if (highAvailabilityServices != null) {
+				highAvailabilityServices.closeAndCleanupAllData();
+			}
 		}
 	}
 	
@@ -250,7 +264,11 @@ public class ProcessFailureCancelingITCase extends TestLogger {
 		}
 		
 		// tell the JobManager to cancel the job
-		jobManager.tell(new JobManagerMessages.CancelJob(jobId), ActorRef.noSender());
+		jobManager.tell(
+			new JobManagerMessages.LeaderSessionMessage(
+				HighAvailabilityServices.DEFAULT_LEADER_ID,
+				new JobManagerMessages.CancelJob(jobId)),
+			ActorRef.noSender());
 	}
 
 	private void waitUntilNumTaskManagersAreRegistered(ActorRef jobManager, int numExpected, long maxDelay)

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/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 fa7e071..bafdd9f 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
@@ -32,6 +32,7 @@ import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.TaskManagerOptions;
 import org.apache.flink.runtime.messages.TaskManagerMessages;
 import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
+import org.apache.flink.test.util.TestEnvironment;
 import org.apache.flink.util.TestLogger;
 import org.junit.Test;
 import scala.concurrent.Await;
@@ -87,8 +88,7 @@ public class TaskManagerFailureRecoveryITCase extends TestLogger {
 			// for the result
 			List<Long> resultCollection = new ArrayList<Long>();
 
-			final ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment(
-					"localhost", cluster.getLeaderRPCPort());
+			final ExecutionEnvironment env = new TestEnvironment(cluster, PARALLELISM, false);
 
 			env.setParallelism(PARALLELISM);
 			env.setRestartStrategy(RestartStrategies.fixedDelayRestart(1, 1000));

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/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
index 5eadba6..40a8f09 100644
--- 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
@@ -34,7 +34,6 @@ 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.LeaderRetrievalService;
 import org.apache.flink.runtime.messages.JobManagerMessages;
 import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
 import org.apache.flink.runtime.testingUtils.TestingCluster;
@@ -42,7 +41,6 @@ import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages;
 import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages.WaitForAllVerticesToBeRunningOrFinished;
 import org.apache.flink.runtime.testingUtils.TestingUtils;
 import org.apache.flink.runtime.testutils.ZooKeeperTestUtils;
-import org.apache.flink.runtime.util.LeaderRetrievalUtils;
 import org.apache.flink.util.TestLogger;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -80,6 +78,7 @@ public class ZooKeeperLeaderElectionITCase extends TestLogger {
 	public static void tearDown() throws Exception {
 		if (zkServer != null) {
 			zkServer.close();
+			zkServer = null;
 		}
 	}
 
@@ -112,8 +111,8 @@ public class ZooKeeperLeaderElectionITCase extends TestLogger {
 				cluster.waitForTaskManagersToBeRegisteredAtJobManager(leadingJM.actor());
 
 				Future<Object> registeredTMs = leadingJM.ask(
-						JobManagerMessages.getRequestNumberRegisteredTaskManager(),
-						timeout);
+					JobManagerMessages.getRequestNumberRegisteredTaskManager(),
+					timeout);
 
 				int numRegisteredTMs = (Integer) Await.result(registeredTMs, timeout);
 
@@ -122,8 +121,7 @@ public class ZooKeeperLeaderElectionITCase extends TestLogger {
 				cluster.clearLeader();
 				leadingJM.tell(PoisonPill.getInstance());
 			}
-		}
-		finally {
+		} finally {
 			cluster.stop();
 		}
 	}
@@ -273,15 +271,10 @@ public class ZooKeeperLeaderElectionITCase extends TestLogger {
 		@Override
 		public void run() {
 			try {
-				LeaderRetrievalService lrService =
-						LeaderRetrievalUtils.createLeaderRetrievalService(
-							cluster.configuration(),
-							false);
-
 				JobExecutionResult result = JobClient.submitJobAndWait(
 						clientActorSystem,
 						cluster.configuration(),
-						lrService,
+						cluster.highAvailabilityServices(),
 						graph,
 						timeout,
 						false,

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/AbstractOperatorRestoreTestBase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/AbstractOperatorRestoreTestBase.java b/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/AbstractOperatorRestoreTestBase.java
index 2eecf49..82e8d94 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/AbstractOperatorRestoreTestBase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/AbstractOperatorRestoreTestBase.java
@@ -27,14 +27,13 @@ import org.apache.flink.runtime.akka.AkkaUtils;
 import org.apache.flink.runtime.akka.ListeningBehaviour;
 import org.apache.flink.runtime.clusterframework.types.ResourceID;
 import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils;
 import org.apache.flink.runtime.instance.ActorGateway;
 import org.apache.flink.runtime.instance.AkkaActorGateway;
 import org.apache.flink.runtime.jobgraph.JobGraph;
 import org.apache.flink.runtime.jobgraph.JobStatus;
 import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings;
 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.state.memory.MemoryStateBackend;
 import org.apache.flink.runtime.taskmanager.TaskManager;
@@ -44,9 +43,11 @@ import org.apache.flink.runtime.testingUtils.TestingMemoryArchivist;
 import org.apache.flink.runtime.testingUtils.TestingTaskManager;
 import org.apache.flink.runtime.testingUtils.TestingTaskManagerMessages;
 import org.apache.flink.runtime.testingUtils.TestingUtils;
+import org.apache.flink.runtime.util.LeaderRetrievalUtils;
 import org.apache.flink.streaming.api.CheckpointingMode;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.api.graph.StreamingJobGraphGenerator;
+import org.apache.flink.util.TestLogger;
 import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.BeforeClass;
@@ -71,36 +72,48 @@ import java.util.concurrent.TimeUnit;
  * Step 1: Migrate the job to 1.3 by submitting the same job used for the 1.2 savepoint, and create a new savepoint.
  * Step 2: Modify the job topology, and restore from the savepoint created in step 1.
  */
-public abstract class AbstractOperatorRestoreTestBase {
+public abstract class AbstractOperatorRestoreTestBase extends TestLogger {
 
 	@Rule
 	public final TemporaryFolder tmpFolder = new TemporaryFolder();
 
 	private static ActorSystem actorSystem = null;
+	private static HighAvailabilityServices highAvailabilityServices = null;
 	private static ActorGateway jobManager = null;
 	private static ActorGateway archiver = null;
 	private static ActorGateway taskManager = null;
 
-	private static final FiniteDuration timeout = new FiniteDuration(30, TimeUnit.SECONDS);
+	private static final FiniteDuration timeout = new FiniteDuration(30L, TimeUnit.SECONDS);
 
 	@BeforeClass
 	public static void setupCluster() throws Exception {
-		FiniteDuration timeout = new FiniteDuration(30, TimeUnit.SECONDS);
+		final Configuration configuration = new Configuration();
+
+		FiniteDuration timeout = new FiniteDuration(30L, TimeUnit.SECONDS);
 
 		actorSystem = AkkaUtils.createLocalActorSystem(new Configuration());
 
+		highAvailabilityServices = HighAvailabilityServicesUtils.createAvailableOrEmbeddedServices(
+			configuration,
+			TestingUtils.defaultExecutor());
+
 		Tuple2<ActorRef, ActorRef> master = JobManager.startJobManagerActors(
-			new Configuration(),
+			configuration,
 			actorSystem,
 			TestingUtils.defaultExecutor(),
 			TestingUtils.defaultExecutor(),
+			highAvailabilityServices,
 			Option.apply("jm"),
 			Option.apply("arch"),
 			TestingJobManager.class,
 			TestingMemoryArchivist.class);
 
-		jobManager = new AkkaActorGateway(master._1(), HighAvailabilityServices.DEFAULT_LEADER_ID);
-		archiver = new AkkaActorGateway(master._2(), HighAvailabilityServices.DEFAULT_LEADER_ID);
+		jobManager = LeaderRetrievalUtils.retrieveLeaderGateway(
+			highAvailabilityServices.getJobManagerLeaderRetriever(HighAvailabilityServices.DEFAULT_JOB_ID),
+			actorSystem,
+			timeout);
+
+		archiver = new AkkaActorGateway(master._2(), jobManager.leaderSessionID());
 
 		Configuration tmConfig = new Configuration();
 		tmConfig.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 4);
@@ -109,13 +122,13 @@ public abstract class AbstractOperatorRestoreTestBase {
 			tmConfig,
 			ResourceID.generate(),
 			actorSystem,
+			highAvailabilityServices,
 			"localhost",
 			Option.apply("tm"),
-			Option.<LeaderRetrievalService>apply(new StandaloneLeaderRetrievalService(jobManager.path(), HighAvailabilityServices.DEFAULT_LEADER_ID)),
 			true,
 			TestingTaskManager.class);
 
-		taskManager = new AkkaActorGateway(taskManagerRef, HighAvailabilityServices.DEFAULT_LEADER_ID);
+		taskManager = new AkkaActorGateway(taskManagerRef, jobManager.leaderSessionID());
 
 		// Wait until connected
 		Object msg = new TestingTaskManagerMessages.NotifyWhenRegisteredAtJobManager(jobManager.actor());
@@ -123,7 +136,11 @@ public abstract class AbstractOperatorRestoreTestBase {
 	}
 
 	@AfterClass
-	public static void tearDownCluster() {
+	public static void tearDownCluster() throws Exception {
+		if (highAvailabilityServices != null) {
+			highAvailabilityServices.closeAndCleanupAllData();
+		}
+
 		if (actorSystem != null) {
 			actorSystem.shutdown();
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/TimestampITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/TimestampITCase.java b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/TimestampITCase.java
index 229d3fd..45686ef 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/TimestampITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/TimestampITCase.java
@@ -45,6 +45,7 @@ import org.apache.flink.streaming.api.watermark.Watermark;
 import org.apache.flink.streaming.api.windowing.assigners.TumblingEventTimeWindows;
 import org.apache.flink.streaming.api.windowing.time.Time;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.streaming.util.TestStreamEnvironment;
 import org.apache.flink.util.TestLogger;
 
 import org.junit.AfterClass;
@@ -86,32 +87,24 @@ public class TimestampITCase extends TestLogger {
 
 	@BeforeClass
 	public static void startCluster() {
-		try {
-			Configuration config = new Configuration();
-			config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, NUM_TASK_MANAGERS);
-			config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, NUM_TASK_SLOTS);
-			config.setLong(TaskManagerOptions.MANAGED_MEMORY_SIZE, 12L);
+		Configuration config = new Configuration();
+		config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, NUM_TASK_MANAGERS);
+		config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, NUM_TASK_SLOTS);
+		config.setLong(TaskManagerOptions.MANAGED_MEMORY_SIZE, 12L);
 
-			cluster = new LocalFlinkMiniCluster(config, false);
+		cluster = new LocalFlinkMiniCluster(config, false);
 
-			cluster.start();
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail("Failed to start test cluster: " + e.getMessage());
-		}
+		cluster.start();
+
+		TestStreamEnvironment.setAsContext(cluster, PARALLELISM);
 	}
 
 	@AfterClass
 	public static void shutdownCluster() {
-		try {
-			cluster.shutdown();
-			cluster = null;
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail("Failed to stop test cluster: " + e.getMessage());
-		}
+		cluster.shutdown();
+		cluster = null;
+
+		TestStreamEnvironment.unsetAsContext();
 	}
 
 	/**
@@ -132,8 +125,7 @@ public class TimestampITCase extends TestLogger {
 
 		long initialTime = 0L;
 
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment(
-				"localhost", cluster.getLeaderRPCPort());
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
 		
 		env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
 		env.setParallelism(PARALLELISM);
@@ -182,8 +174,7 @@ public class TimestampITCase extends TestLogger {
 
 		long initialTime = 0L;
 
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment(
-				"localhost", cluster.getLeaderRPCPort());
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
 
 		env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
 		env.setParallelism(PARALLELISM);
@@ -270,8 +261,7 @@ public class TimestampITCase extends TestLogger {
 		final int NUM_ELEMENTS = 10;
 
 
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment(
-				"localhost", cluster.getLeaderRPCPort());
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
 
 		env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
 		env.setParallelism(PARALLELISM);
@@ -297,8 +287,7 @@ public class TimestampITCase extends TestLogger {
 	public void testDisabledTimestamps() throws Exception {
 		final int NUM_ELEMENTS = 10;
 		
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment(
-				"localhost", cluster.getLeaderRPCPort());
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
 		
 		env.setStreamTimeCharacteristic(TimeCharacteristic.ProcessingTime);
 		env.setParallelism(PARALLELISM);
@@ -325,7 +314,7 @@ public class TimestampITCase extends TestLogger {
 	public void testTimestampExtractorWithAutoInterval() throws Exception {
 		final int NUM_ELEMENTS = 10;
 
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", cluster.getLeaderRPCPort());
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
 
 		env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
 		env.getConfig().setAutoWatermarkInterval(10);
@@ -389,7 +378,7 @@ public class TimestampITCase extends TestLogger {
 	public void testTimestampExtractorWithCustomWatermarkEmit() throws Exception {
 		final int NUM_ELEMENTS = 10;
 
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", cluster.getLeaderRPCPort());
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
 
 		env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
 		env.getConfig().setAutoWatermarkInterval(10);
@@ -451,7 +440,7 @@ public class TimestampITCase extends TestLogger {
 	public void testTimestampExtractorWithDecreasingCustomWatermarkEmit() throws Exception {
 		final int NUM_ELEMENTS = 10;
 
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", cluster.getLeaderRPCPort());
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
 
 		env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
 		env.getConfig().setAutoWatermarkInterval(1);
@@ -513,7 +502,7 @@ public class TimestampITCase extends TestLogger {
 	public void testTimestampExtractorWithLongMaxWatermarkFromSource() throws Exception {
 		final int NUM_ELEMENTS = 10;
 
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", cluster.getLeaderRPCPort());
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
 
 		env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
 		env.getConfig().setAutoWatermarkInterval(1);
@@ -574,7 +563,7 @@ public class TimestampITCase extends TestLogger {
 		final int NUM_ELEMENTS = 10;
 
 		StreamExecutionEnvironment env = StreamExecutionEnvironment
-				.createRemoteEnvironment("localhost", cluster.getLeaderRPCPort());
+				.getExecutionEnvironment();
 
 		env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
 		env.getConfig().setAutoWatermarkInterval(10);
@@ -630,7 +619,7 @@ public class TimestampITCase extends TestLogger {
 	@Test
 	public void testEventTimeSourceWithProcessingTime() throws Exception {
 		StreamExecutionEnvironment env = 
-				StreamExecutionEnvironment.createRemoteEnvironment("localhost", cluster.getLeaderRPCPort());
+				StreamExecutionEnvironment.getExecutionEnvironment();
 		
 		env.setParallelism(2);
 		env.getConfig().disableSysoutLogging();
@@ -651,8 +640,7 @@ public class TimestampITCase extends TestLogger {
 	
 	@Test
 	public void testErrorOnEventTimeOverProcessingTime() {
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment(
-				"localhost", cluster.getLeaderRPCPort());
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
 
 		env.setParallelism(2);
 		env.getConfig().disableSysoutLogging();
@@ -682,8 +670,7 @@ public class TimestampITCase extends TestLogger {
 
 	@Test
 	public void testErrorOnEventTimeWithoutTimestamps() {
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment(
-				"localhost", cluster.getLeaderRPCPort());
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
 
 		env.setParallelism(2);
 		env.getConfig().disableSysoutLogging();

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/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 5db02d1..e5f26c5 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
@@ -18,7 +18,7 @@
 
 package org.apache.flink.api.scala.runtime.jobmanager
 
-import akka.actor.{ActorSystem, PoisonPill}
+import akka.actor.ActorSystem
 import akka.testkit.{ImplicitSender, TestKit}
 import org.apache.flink.configuration.{ConfigConstants, Configuration}
 import org.apache.flink.runtime.akka.{AkkaUtils, ListeningBehaviour}
@@ -28,7 +28,6 @@ import org.apache.flink.runtime.messages.Acknowledge
 import org.apache.flink.runtime.messages.JobManagerMessages._
 import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages.NotifyWhenAtLeastNumTaskManagerAreRegistered
 import org.apache.flink.runtime.testingUtils.TestingMessages.DisableDisconnect
-import org.apache.flink.runtime.testingUtils.TestingTaskManagerMessages.{JobManagerTerminated, NotifyWhenJobManagerTerminated}
 import org.apache.flink.runtime.testingUtils.{ScalaTestingUtils, TestingCluster, TestingUtils}
 import org.junit.runner.RunWith
 import org.scalatest.junit.JUnitRunner
@@ -66,11 +65,8 @@ class JobManagerFailsITCase(_system: ActorSystem)
           jmGateway.tell(RequestNumberRegisteredTaskManager, self)
           expectMsg(1)
 
-          tm ! NotifyWhenJobManagerTerminated(jmGateway.leaderSessionID())
-
-          jmGateway.tell(PoisonPill, self)
-
-          expectMsgClass(classOf[JobManagerTerminated])
+          // stop the current leader and make sure that he is gone
+          TestingUtils.stopActorGracefully(jmGateway)
 
           cluster.restartLeadingJobManager()
 
@@ -109,11 +105,8 @@ class JobManagerFailsITCase(_system: ActorSystem)
           jmGateway.tell(SubmitJob(jobGraph, ListeningBehaviour.DETACHED), self)
           expectMsg(JobSubmitSuccess(jobGraph.getJobID))
 
-          tm.tell(NotifyWhenJobManagerTerminated(jmGateway.leaderSessionID()), self)
-
-          jmGateway.tell(PoisonPill, self)
-
-          expectMsgClass(classOf[JobManagerTerminated])
+          // stop the current leader and make sure that he is gone
+          TestingUtils.stopActorGracefully(jmGateway)
 
           cluster.restartLeadingJobManager()
 

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/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 424fc8b..61cb8cc 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
@@ -226,7 +226,7 @@ class TaskManagerFailsITCase(_system: ActorSystem)
 
           tm ! NotifyWhenRegisteredAtJobManager
 
-          expectMsg(RegisteredAtJobManager)
+          expectMsgClass(classOf[RegisteredAtJobManager])
 
           jmGateway.tell(SubmitJob(jobGraph2, ListeningBehaviour.EXECUTION_RESULT), self)
 

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/FlinkYarnSessionCliTest.java
----------------------------------------------------------------------
diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/FlinkYarnSessionCliTest.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/FlinkYarnSessionCliTest.java
index a503115..264b6aa 100644
--- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/FlinkYarnSessionCliTest.java
+++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/FlinkYarnSessionCliTest.java
@@ -18,7 +18,6 @@
 
 package org.apache.flink.yarn;
 
-import akka.actor.ActorSystem;
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.CommandLineParser;
 import org.apache.commons.cli.DefaultParser;
@@ -27,7 +26,6 @@ import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.client.CliFrontend;
 import org.apache.flink.client.cli.CliFrontendParser;
 import org.apache.flink.client.cli.RunOptions;
-import org.apache.flink.client.deployment.ClusterDescriptor;
 import org.apache.flink.client.program.ClusterClient;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.yarn.cli.FlinkYarnSessionCli;
@@ -36,7 +34,6 @@ import org.apache.flink.test.util.TestBaseUtils;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
 import org.apache.hadoop.yarn.client.api.YarnClient;
-import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.junit.Assert;
 import org.junit.Rule;
 import org.junit.Test;
@@ -129,7 +126,7 @@ public class FlinkYarnSessionCliTest {
 		Assert.assertEquals(2, descriptor.getTaskManagerCount());
 
 		Configuration config = new Configuration();
-		CliFrontend.setJobManagerAddressInConfig(config, new InetSocketAddress("test", 9000));
+		CliFrontend.setJobManagerAddressInConfig(config, new InetSocketAddress("localhost", 9000));
 		ClusterClient client = new TestingYarnClusterClient(descriptor, config);
 		Assert.assertEquals(6, client.getMaxSlots());
 	}
@@ -175,7 +172,7 @@ public class FlinkYarnSessionCliTest {
 
 	private static class TestingYarnClusterClient extends YarnClusterClient {
 
-		public TestingYarnClusterClient(AbstractYarnClusterDescriptor descriptor, Configuration config) throws IOException, YarnException {
+		public TestingYarnClusterClient(AbstractYarnClusterDescriptor descriptor, Configuration config) throws Exception {
 			super(descriptor,
 				Mockito.mock(YarnClient.class),
 				Mockito.mock(ApplicationReport.class),

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNHighAvailabilityITCase.java
----------------------------------------------------------------------
diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNHighAvailabilityITCase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNHighAvailabilityITCase.java
index bc1af65..1f043ef 100644
--- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNHighAvailabilityITCase.java
+++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNHighAvailabilityITCase.java
@@ -24,14 +24,15 @@ import akka.testkit.JavaTestKit;
 import org.apache.curator.test.TestingServer;
 import org.apache.flink.client.program.ClusterClient;
 import org.apache.flink.configuration.ConfigConstants;
-import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.CoreOptions;
 import org.apache.flink.configuration.GlobalConfiguration;
 import org.apache.flink.configuration.HighAvailabilityOptions;
 import org.apache.flink.runtime.akka.AkkaUtils;
+import org.apache.flink.runtime.concurrent.Executors;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils;
 import org.apache.flink.runtime.instance.ActorGateway;
 import org.apache.flink.runtime.instance.AkkaActorGateway;
-import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
 import org.apache.flink.runtime.messages.Acknowledge;
 import org.apache.flink.runtime.state.filesystem.FsStateBackendFactory;
 import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages;
@@ -56,7 +57,7 @@ public class YARNHighAvailabilityITCase extends YarnTestBase {
 
 	protected static ActorSystem actorSystem;
 
-	protected static final int numberApplicationAttempts = 10;
+	protected static final int numberApplicationAttempts = 3;
 
 	@Rule
 	public TemporaryFolder temp = new TemporaryFolder();
@@ -128,9 +129,19 @@ public class YARNHighAvailabilityITCase extends YarnTestBase {
 
 		final FiniteDuration timeout = new FiniteDuration(2, TimeUnit.MINUTES);
 
+		HighAvailabilityServices highAvailabilityServices = null;
+
 		try {
 			yarnCluster = flinkYarnClient.deploy();
-			final Configuration config = yarnCluster.getFlinkConfiguration();
+
+			final ClusterClient finalYarnCluster = yarnCluster;
+
+			highAvailabilityServices = HighAvailabilityServicesUtils.createHighAvailabilityServices(
+				yarnCluster.getFlinkConfiguration(),
+				Executors.directExecutor(),
+				HighAvailabilityServicesUtils.AddressResolution.TRY_ADDRESS_RESOLUTION);
+
+			final HighAvailabilityServices finalHighAvailabilityServices = highAvailabilityServices;
 
 			new JavaTestKit(actorSystem) {{
 				for (int attempt = 0; attempt < numberKillingAttempts; attempt++) {
@@ -138,8 +149,10 @@ public class YARNHighAvailabilityITCase extends YarnTestBase {
 						@Override
 						protected void run() {
 							try {
-								LeaderRetrievalService lrs = LeaderRetrievalUtils.createLeaderRetrievalService(config, false);
-								ActorGateway gateway = LeaderRetrievalUtils.retrieveLeaderGateway(lrs, actorSystem, timeout);
+								ActorGateway gateway = LeaderRetrievalUtils.retrieveLeaderGateway(
+									finalHighAvailabilityServices.getJobManagerLeaderRetriever(HighAvailabilityServices.DEFAULT_JOB_ID),
+									actorSystem,
+									timeout);
 								ActorGateway selfGateway = new AkkaActorGateway(getRef(), gateway.leaderSessionID());
 
 								gateway.tell(new TestingJobManagerMessages.NotifyWhenAtLeastNumTaskManagerAreRegistered(1), selfGateway);
@@ -158,10 +171,13 @@ public class YARNHighAvailabilityITCase extends YarnTestBase {
 					@Override
 					protected void run() {
 						try {
-							LeaderRetrievalService lrs = LeaderRetrievalUtils.createLeaderRetrievalService(config, false);
-							ActorGateway gateway2 = LeaderRetrievalUtils.retrieveLeaderGateway(lrs, actorSystem, timeout);
-							ActorGateway selfGateway = new AkkaActorGateway(getRef(), gateway2.leaderSessionID());
-							gateway2.tell(new TestingJobManagerMessages.NotifyWhenAtLeastNumTaskManagerAreRegistered(1), selfGateway);
+							ActorGateway gateway = LeaderRetrievalUtils.retrieveLeaderGateway(
+								finalHighAvailabilityServices.getJobManagerLeaderRetriever(HighAvailabilityServices.DEFAULT_JOB_ID),
+								actorSystem,
+								timeout);
+							ActorGateway selfGateway = new AkkaActorGateway(getRef(), gateway.leaderSessionID());
+
+							gateway.tell(new TestingJobManagerMessages.NotifyWhenAtLeastNumTaskManagerAreRegistered(1), selfGateway);
 
 							expectMsgEquals(Acknowledge.get());
 						} catch (Exception e) {
@@ -175,6 +191,10 @@ public class YARNHighAvailabilityITCase extends YarnTestBase {
 			if (yarnCluster != null) {
 				yarnCluster.shutdown();
 			}
+
+			if (highAvailabilityServices != null) {
+				highAvailabilityServices.closeAndCleanupAllData();
+			}
 		}
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java
----------------------------------------------------------------------
diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java
index 68cc73d..f45fe82 100644
--- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java
+++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java
@@ -224,7 +224,7 @@ public class YARNSessionFIFOITCase extends YarnTestBase {
 	 * Test the YARN Java API
 	 */
 	@Test
-	public void testJavaAPI() {
+	public void testJavaAPI() throws Exception {
 		final int WAIT_TIME = 15;
 		LOG.info("Starting testJavaAPI()");
 

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java b/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java
index a5a6c36..65525f2 100644
--- a/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java
+++ b/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java
@@ -1307,7 +1307,7 @@ public abstract class AbstractYarnClusterDescriptor implements ClusterDescriptor
 			ApplicationReport report,
 			org.apache.flink.configuration.Configuration flinkConfiguration,
 			Path sessionFilesDir,
-			boolean perJobCluster) throws IOException, YarnException {
+			boolean perJobCluster) throws Exception {
 		return new YarnClusterClient(
 			descriptor,
 			yarnClient,

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnApplicationMasterRunner.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnApplicationMasterRunner.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnApplicationMasterRunner.java
index ee87cfe..b62f957 100644
--- a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnApplicationMasterRunner.java
+++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnApplicationMasterRunner.java
@@ -26,14 +26,16 @@ import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.GlobalConfiguration;
 import org.apache.flink.configuration.HighAvailabilityOptions;
+import org.apache.flink.configuration.JobManagerOptions;
 import org.apache.flink.configuration.SecurityOptions;
 import org.apache.flink.runtime.akka.AkkaUtils;
 import org.apache.flink.runtime.clusterframework.BootstrapTools;
 import org.apache.flink.runtime.clusterframework.ContaineredTaskManagerParameters;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils;
 import org.apache.flink.runtime.jobmanager.JobManager;
 import org.apache.flink.runtime.jobmanager.MemoryArchivist;
 import org.apache.flink.runtime.jobmaster.JobMaster;
-import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
 import org.apache.flink.runtime.net.SSLUtils;
 import org.apache.flink.runtime.process.ProcessReaper;
 import org.apache.flink.runtime.security.SecurityUtils;
@@ -42,7 +44,6 @@ import org.apache.flink.runtime.util.EnvironmentInformation;
 import org.apache.flink.runtime.util.ExecutorThreadFactory;
 import org.apache.flink.runtime.util.Hardware;
 import org.apache.flink.runtime.util.JvmShutdownSafeguard;
-import org.apache.flink.runtime.util.LeaderRetrievalUtils;
 import org.apache.flink.runtime.util.SignalHandler;
 import org.apache.flink.runtime.webmonitor.WebMonitor;
 import org.apache.flink.yarn.cli.FlinkYarnSessionCli;
@@ -215,6 +216,7 @@ public class YarnApplicationMasterRunner {
 	protected int runApplicationMaster(Configuration config) {
 		ActorSystem actorSystem = null;
 		WebMonitor webMonitor = null;
+		HighAvailabilityServices highAvailabilityServices = null;
 
 		int numberProcessors = Hardware.getNumberCPUCores();
 
@@ -332,6 +334,16 @@ public class YarnApplicationMasterRunner {
 			// 3) Resource Master for YARN
 			// 4) Process reapers for the JobManager and Resource Master
 
+			// 0: Start the JobManager services
+
+			// update the configuration used to create the high availability services
+			config.setString(JobManagerOptions.ADDRESS, akkaHostname);
+			config.setInteger(JobManagerOptions.PORT, akkaPort);
+
+			highAvailabilityServices = HighAvailabilityServicesUtils.createHighAvailabilityServices(
+				config,
+				ioExecutor,
+				HighAvailabilityServicesUtils.AddressResolution.NO_ADDRESS_RESOLUTION);
 
 			// 1: the JobManager
 			LOG.debug("Starting JobManager actor");
@@ -342,6 +354,7 @@ public class YarnApplicationMasterRunner {
 				actorSystem,
 				futureExecutor,
 				ioExecutor,
+				highAvailabilityServices,
 				new Some<>(JobMaster.JOB_MANAGER_NAME),
 				Option.<String>empty(),
 				getJobManagerClass(),
@@ -351,7 +364,12 @@ public class YarnApplicationMasterRunner {
 			// 2: the web monitor
 			LOG.debug("Starting Web Frontend");
 
-			webMonitor = BootstrapTools.startWebMonitorIfConfigured(config, actorSystem, jobManager, LOG);
+			webMonitor = BootstrapTools.startWebMonitorIfConfigured(
+				config,
+				highAvailabilityServices,
+				actorSystem,
+				jobManager,
+				LOG);
 
 			String protocol = "http://";
 			if (config.getBoolean(ConfigConstants.JOB_MANAGER_WEB_SSL_ENABLED,
@@ -364,15 +382,11 @@ public class YarnApplicationMasterRunner {
 			// 3: Flink's Yarn ResourceManager
 			LOG.debug("Starting YARN Flink Resource Manager");
 
-			// we need the leader retrieval service here to be informed of new leaders and session IDs
-			LeaderRetrievalService leaderRetriever = 
-				LeaderRetrievalUtils.createLeaderRetrievalService(config, jobManager);
-
 			Props resourceMasterProps = YarnFlinkResourceManager.createActorProps(
 				getResourceManagerClass(),
 				config,
 				yarnConfig,
-				leaderRetriever,
+				highAvailabilityServices.getJobManagerLeaderRetriever(HighAvailabilityServices.DEFAULT_JOB_ID),
 				appMasterHostname,
 				webMonitorURL,
 				taskManagerParameters,
@@ -437,6 +451,14 @@ public class YarnApplicationMasterRunner {
 			}
 		}
 
+		if (highAvailabilityServices != null) {
+			try {
+				highAvailabilityServices.close();
+			} catch (Throwable t) {
+				LOG.error("Failed to stop the high availability services.", t);
+			}
+		}
+
 		org.apache.flink.runtime.concurrent.Executors.gracefulShutdown(
 			AkkaUtils.getTimeout(config).toMillis(),
 			TimeUnit.MILLISECONDS,

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterClient.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterClient.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterClient.java
index f044cdd..e70af09 100644
--- a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterClient.java
+++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterClient.java
@@ -32,9 +32,8 @@ import org.apache.flink.runtime.clusterframework.messages.GetClusterStatus;
 import org.apache.flink.runtime.clusterframework.messages.GetClusterStatusResponse;
 import org.apache.flink.runtime.clusterframework.messages.InfoMessage;
 import org.apache.flink.runtime.clusterframework.messages.ShutdownClusterAfterJob;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
 import org.apache.flink.runtime.jobgraph.JobGraph;
-import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
-import org.apache.flink.runtime.util.LeaderRetrievalUtils;
 import org.apache.flink.util.Preconditions;
 import org.apache.flink.yarn.cli.FlinkYarnSessionCli;
 import org.apache.hadoop.conf.Configuration;
@@ -109,7 +108,7 @@ public class YarnClusterClient extends ClusterClient {
 		final ApplicationReport appReport,
 		org.apache.flink.configuration.Configuration flinkConfig,
 		Path sessionFilesDir,
-		boolean newlyCreatedCluster) throws IOException, YarnException {
+		boolean newlyCreatedCluster) throws Exception {
 
 		super(flinkConfig);
 
@@ -123,7 +122,10 @@ public class YarnClusterClient extends ClusterClient {
 		this.trackingURL = appReport.getTrackingUrl();
 		this.newlyCreatedCluster = newlyCreatedCluster;
 
-		this.applicationClient = new LazApplicationClientLoader(flinkConfig, actorSystemLoader);
+		this.applicationClient = new LazApplicationClientLoader(
+			flinkConfig,
+			actorSystemLoader,
+			highAvailabilityServices);
 
 		this.pollingRunner = new PollingThread(yarnClient, appId);
 		this.pollingRunner.setDaemon(true);
@@ -443,7 +445,12 @@ public class YarnClusterClient extends ClusterClient {
 		@Override
 		public void run() {
 			LOG.info("Shutting down YarnClusterClient from the client shutdown hook");
-			shutdown();
+
+			try {
+				shutdown();
+			} catch (Throwable t) {
+				LOG.warn("Could not properly shut down the yarn cluster client.", t);
+			}
 		}
 	}
 
@@ -545,14 +552,17 @@ public class YarnClusterClient extends ClusterClient {
 
 		private final org.apache.flink.configuration.Configuration flinkConfig;
 		private final LazyActorSystemLoader actorSystemLoader;
+		private final HighAvailabilityServices highAvailabilityServices;
 
 		private ActorRef applicationClient;
 
 		private LazApplicationClientLoader(
 				org.apache.flink.configuration.Configuration flinkConfig,
-				LazyActorSystemLoader actorSystemLoader) {
-			this.flinkConfig = flinkConfig;
-			this.actorSystemLoader = actorSystemLoader;
+				LazyActorSystemLoader actorSystemLoader,
+				HighAvailabilityServices highAvailabilityServices) {
+			this.flinkConfig = Preconditions.checkNotNull(flinkConfig, "flinkConfig");
+			this.actorSystemLoader = Preconditions.checkNotNull(actorSystemLoader, "actorSystemLoader");
+			this.highAvailabilityServices = Preconditions.checkNotNull(highAvailabilityServices, "highAvailabilityServices");
 		}
 
 		/**
@@ -561,14 +571,6 @@ public class YarnClusterClient extends ClusterClient {
 		 */
 		public ActorRef get() {
 			if (applicationClient == null) {
-				/* The leader retrieval service for connecting to the cluster and finding the active leader. */
-				LeaderRetrievalService leaderRetrievalService;
-				try {
-					leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(flinkConfig, true);
-				} catch (Exception e) {
-					throw new RuntimeException("Could not create the leader retrieval service.", e);
-				}
-
 				// start application client
 				LOG.info("Start application client.");
 
@@ -576,7 +578,7 @@ public class YarnClusterClient extends ClusterClient {
 					Props.create(
 						ApplicationClient.class,
 						flinkConfig,
-						leaderRetrievalService),
+						highAvailabilityServices.getJobManagerLeaderRetriever(HighAvailabilityServices.DEFAULT_JOB_ID)),
 					"applicationClient");
 			}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterClientV2.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterClientV2.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterClientV2.java
index 5a3a3c0..33d5987 100644
--- a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterClientV2.java
+++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterClientV2.java
@@ -31,7 +31,6 @@ import org.apache.hadoop.yarn.client.api.YarnClientApplication;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.net.URL;
 import java.util.List;
@@ -58,11 +57,11 @@ public class YarnClusterClientV2 extends ClusterClient {
 	 *
 	 * @param clusterDescriptor The descriptor used to create yarn job
 	 * @param flinkConfig Flink configuration
-	 * @throws java.io.IOException
+	 * @throws Exception if the cluster client could not be created
 	 */
 	public YarnClusterClientV2(
 			final AbstractYarnClusterDescriptor clusterDescriptor,
-			org.apache.flink.configuration.Configuration flinkConfig) throws IOException {
+			org.apache.flink.configuration.Configuration flinkConfig) throws Exception {
 
 		super(flinkConfig);
 

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnFlinkApplicationMasterRunner.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnFlinkApplicationMasterRunner.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnFlinkApplicationMasterRunner.java
index 05b2be8..3f4d4f6 100644
--- a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnFlinkApplicationMasterRunner.java
+++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnFlinkApplicationMasterRunner.java
@@ -138,9 +138,10 @@ public class YarnFlinkApplicationMasterRunner extends AbstractYarnFlinkApplicati
 				LOG.info("Starting High Availability Services");
 				commonRpcService = createRpcService(config, appMasterHostname, amPortRange);
 
-				haServices = HighAvailabilityServicesUtils.createAvailableOrEmbeddedServices(
+				haServices = HighAvailabilityServicesUtils.createHighAvailabilityServices(
 					config,
-					commonRpcService.getExecutor());
+					commonRpcService.getExecutor(),
+					HighAvailabilityServicesUtils.AddressResolution.NO_ADDRESS_RESOLUTION);
 
 				heartbeatServices = HeartbeatServices.fromConfiguration(config);
 				

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnCLI.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnCLI.java b/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnCLI.java
index ca5049c..557fa38 100644
--- a/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnCLI.java
+++ b/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnCLI.java
@@ -33,7 +33,6 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.File;
-import java.io.IOException;
 import java.io.UnsupportedEncodingException;
 import java.net.URL;
 import java.net.URLDecoder;
@@ -224,22 +223,14 @@ public class FlinkYarnCLI implements CustomCommandLine<YarnClusterClientV2> {
 			String applicationName,
 			CommandLine cmdLine,
 			Configuration config,
-			List<URL> userJarFiles) {
+			List<URL> userJarFiles) throws Exception {
 		Preconditions.checkNotNull(userJarFiles, "User jar files should not be null.");
 
 		YarnClusterDescriptorV2 yarnClusterDescriptor = createDescriptor(applicationName, cmdLine);
 		yarnClusterDescriptor.setFlinkConfiguration(config);
 		yarnClusterDescriptor.setProvidedUserJarFiles(userJarFiles);
 
-		YarnClusterClientV2 client = null;
-		try {
-			client = new YarnClusterClientV2(yarnClusterDescriptor, config);
-		}
-		catch (IOException e) {
-			throw new RuntimeException("Fail to create YarnClusterClientV2", e.getCause());
-		}
-		return client;
-
+		return new YarnClusterClientV2(yarnClusterDescriptor, config);
 	}
 
 	/**

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java b/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java
index f4557c9..9277d21 100644
--- a/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java
+++ b/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java
@@ -678,7 +678,12 @@ public class FlinkYarnSessionCli implements CustomCommandLine<YarnClusterClient>
 	public void stop() {
 		if (yarnCluster != null) {
 			LOG.info("Command line interface is shutting down the yarnCluster");
-			yarnCluster.shutdown();
+
+			try {
+				yarnCluster.shutdown();
+			} catch (Throwable t) {
+				LOG.warn("Could not properly shutdown the yarn cluster.", t);
+			}
 		}
 	}
 


[14/16] flink git commit: [FLINK-6217] ContaineredTaskManagerParameters sets off-heap memory size incorrectly.

Posted by tr...@apache.org.
[FLINK-6217] ContaineredTaskManagerParameters sets off-heap memory size incorrectly.

This closes #3648.


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

Branch: refs/heads/master
Commit: 5cf22f411962a1199da7843252386b77e2461851
Parents: 9375808
Author: Haohui Mai <wh...@apache.org>
Authored: Wed Mar 29 16:57:52 2017 -0700
Committer: Till Rohrmann <tr...@apache.org>
Committed: Fri May 5 11:08:10 2017 +0200

----------------------------------------------------------------------
 .../clusterframework/LaunchableMesosWorker.java |  4 +-
 .../clusterframework/BootstrapTools.java        | 18 +++++--
 .../ContaineredTaskManagerParameters.java       |  5 +-
 .../ContaineredTaskManagerParametersTest.java   | 49 ++++++++++++++++++++
 4 files changed, 69 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/5cf22f41/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/LaunchableMesosWorker.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/LaunchableMesosWorker.java b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/LaunchableMesosWorker.java
index 3d5350a..2408ac6 100644
--- a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/LaunchableMesosWorker.java
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/LaunchableMesosWorker.java
@@ -223,7 +223,9 @@ public class LaunchableMesosWorker implements LaunchableTask {
 		// finalize the memory parameters
 		jvmArgs.append(" -Xms").append(tmParams.taskManagerHeapSizeMB()).append("m");
 		jvmArgs.append(" -Xmx").append(tmParams.taskManagerHeapSizeMB()).append("m");
-		jvmArgs.append(" -XX:MaxDirectMemorySize=").append(tmParams.taskManagerDirectMemoryLimitMB()).append("m");
+		if (tmParams.taskManagerDirectMemoryLimitMB() >= 0) {
+			jvmArgs.append(" -XX:MaxDirectMemorySize=").append(tmParams.taskManagerDirectMemoryLimitMB()).append("m");
+		}
 
 		// pass dynamic system properties
 		jvmArgs.append(' ').append(

http://git-wip-us.apache.org/repos/asf/flink/blob/5cf22f41/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/BootstrapTools.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/BootstrapTools.java b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/BootstrapTools.java
index 9bcaa18..e9d3cbd 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/BootstrapTools.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/BootstrapTools.java
@@ -25,6 +25,7 @@ import com.typesafe.config.Config;
 
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.Option;
+import org.apache.commons.lang3.StringUtils;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.CoreOptions;
@@ -46,6 +47,7 @@ import java.io.IOException;
 import java.io.PrintWriter;
 import java.net.BindException;
 import java.net.ServerSocket;
+import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.Map;
@@ -358,10 +360,18 @@ public class BootstrapTools {
 
 		final Map<String, String> startCommandValues = new HashMap<>();
 		startCommandValues.put("java", "$JAVA_HOME/bin/java");
-		startCommandValues
-			.put("jvmmem", 	"-Xms" + tmParams.taskManagerHeapSizeMB() + "m " +
-							"-Xmx" + tmParams.taskManagerHeapSizeMB() + "m " +
-							"-XX:MaxDirectMemorySize=" + tmParams.taskManagerDirectMemoryLimitMB() + "m");
+
+		ArrayList<String> params = new ArrayList<>();
+		params.add(String.format("-Xms%dm", tmParams.taskManagerHeapSizeMB()));
+		params.add(String.format("-Xmx%dm", tmParams.taskManagerHeapSizeMB()));
+
+		if (tmParams.taskManagerDirectMemoryLimitMB() >= 0) {
+			params.add(String.format("-XX:MaxDirectMemorySize=%dm",
+				tmParams.taskManagerDirectMemoryLimitMB()));
+		}
+
+		startCommandValues.put("jvmmem", StringUtils.join(params, ' '));
+
 		String javaOpts = flinkConfig.getString(CoreOptions.FLINK_JVM_OPTIONS);
 		if (flinkConfig.getString(CoreOptions.FLINK_TM_JVM_OPTIONS).length() > 0) {
 			javaOpts += " " + flinkConfig.getString(CoreOptions.FLINK_TM_JVM_OPTIONS);

http://git-wip-us.apache.org/repos/asf/flink/blob/5cf22f41/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/ContaineredTaskManagerParameters.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/ContaineredTaskManagerParameters.java b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/ContaineredTaskManagerParameters.java
index 0fc0870..8ff3c25 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/ContaineredTaskManagerParameters.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/ContaineredTaskManagerParameters.java
@@ -146,8 +146,9 @@ public class ContaineredTaskManagerParameters implements java.io.Serializable {
 		final boolean useOffHeap = config.getBoolean(TaskManagerOptions.MEMORY_OFF_HEAP);
 
 		final long heapSizeMB;
+		long offHeapSize = -1;
 		if (useOffHeap) {
-			long offHeapSize = config.getLong(TaskManagerOptions.MANAGED_MEMORY_SIZE);
+			offHeapSize = config.getLong(TaskManagerOptions.MANAGED_MEMORY_SIZE);
 
 			if (offHeapSize <= 0) {
 				double fraction = config.getFloat(TaskManagerOptions.MANAGED_MEMORY_FRACTION);
@@ -174,6 +175,6 @@ public class ContaineredTaskManagerParameters implements java.io.Serializable {
 		
 		// done
 		return new ContaineredTaskManagerParameters(
-			containerMemoryMB, heapSizeMB, javaMemorySizeMB, numSlots, envVars);
+			containerMemoryMB, heapSizeMB, offHeapSize, numSlots, envVars);
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/5cf22f41/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ContaineredTaskManagerParametersTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ContaineredTaskManagerParametersTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ContaineredTaskManagerParametersTest.java
new file mode 100644
index 0000000..c0c48f9
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ContaineredTaskManagerParametersTest.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.clusterframework;
+
+import org.apache.flink.configuration.Configuration;
+import org.junit.Test;
+
+import static org.apache.flink.configuration.ConfigConstants.TASK_MANAGER_MEMORY_OFF_HEAP_KEY;
+import static org.apache.flink.configuration.TaskManagerOptions.MANAGED_MEMORY_SIZE;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class ContaineredTaskManagerParametersTest {
+	private static final long CONTAINER_MEMORY = 8192;
+
+	@Test
+	public void testDefaultOffHeapMemory() {
+		Configuration conf = new Configuration();
+		ContaineredTaskManagerParameters params =
+			ContaineredTaskManagerParameters.create(conf, CONTAINER_MEMORY, 1);
+		assertEquals(-1, params.taskManagerDirectMemoryLimitMB());
+	}
+
+	@Test
+	public void testTotalMemoryDoesNotExceedContainerMemory() {
+		Configuration conf = new Configuration();
+		conf.setBoolean(MANAGED_MEMORY_SIZE.key(), true);
+		ContaineredTaskManagerParameters params =
+			ContaineredTaskManagerParameters.create(conf, CONTAINER_MEMORY, 1);
+		assertTrue(params.taskManagerHeapSizeMB() +
+			params.taskManagerDirectMemoryLimitMB() <= CONTAINER_MEMORY);
+	}
+}


[09/16] flink git commit: [FLINK-6078] Remove CuratorFramework#close calls from ZooKeeper based HA services

Posted by tr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/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 68f43da..5092643 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
@@ -49,7 +49,7 @@ import org.apache.flink.runtime.execution.SuppressRestartsException
 import org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager
 import org.apache.flink.runtime.executiongraph.restart.RestartStrategyFactory
 import org.apache.flink.runtime.executiongraph._
-import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils
+import org.apache.flink.runtime.highavailability.{HighAvailabilityServices, HighAvailabilityServicesUtils}
 import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils.AddressResolution
 import org.apache.flink.runtime.instance.{AkkaActorGateway, InstanceID, InstanceManager}
 import org.apache.flink.runtime.jobgraph.{JobGraph, JobStatus}
@@ -57,8 +57,8 @@ import org.apache.flink.runtime.jobmanager.SubmittedJobGraphStore.SubmittedJobGr
 import org.apache.flink.runtime.jobmanager.scheduler.{Scheduler => FlinkScheduler}
 import org.apache.flink.runtime.jobmanager.slots.ActorTaskManagerGateway
 import org.apache.flink.runtime.jobmaster.JobMaster
+import org.apache.flink.runtime.leaderelection.{LeaderContender, LeaderElectionService}
 import org.apache.flink.runtime.jobmaster.JobMaster.{ARCHIVE_NAME, JOB_MANAGER_NAME}
-import org.apache.flink.runtime.leaderelection.{LeaderContender, LeaderElectionService, StandaloneLeaderElectionService}
 import org.apache.flink.runtime.messages.ArchiveMessages.ArchiveExecutionGraph
 import org.apache.flink.runtime.messages.ExecutionGraphMessages.JobStatusChanged
 import org.apache.flink.runtime.messages.JobManagerMessages._
@@ -2037,14 +2037,27 @@ object JobManager {
 
     val timeout = AkkaUtils.getTimeout(configuration)
 
-    val (jobManagerSystem, _, _, webMonitorOption, _) = try {
-      startActorSystemAndJobManagerActors(
+    // we have to first start the JobManager ActorSystem because this determines the port if 0
+    // was chosen before. The method startActorSystem will update the configuration correspondingly.
+    val jobManagerSystem = startActorSystem(
+      configuration,
+      listeningAddress,
+      listeningPort)
+
+    val highAvailabilityServices = HighAvailabilityServicesUtils.createHighAvailabilityServices(
+      configuration,
+      ioExecutor,
+      AddressResolution.NO_ADDRESS_RESOLUTION);
+
+    val (_, _, webMonitorOption, _) = try {
+      startJobManagerActors(
+        jobManagerSystem,
         configuration,
         executionMode,
         listeningAddress,
-        listeningPort,
         futureExecutor,
         ioExecutor,
+        highAvailabilityServices,
         classOf[JobManager],
         classOf[MemoryArchivist],
         Option(classOf[StandaloneResourceManager])
@@ -2070,6 +2083,13 @@ object JobManager {
         }
     }
 
+    try {
+      highAvailabilityServices.close()
+    } catch {
+      case t: Throwable =>
+        LOG.warn("Could not properly stop the high availability services.", t)
+    }
+
     FlinkExecutors.gracefulShutdown(
       timeout.toMillis,
       TimeUnit.MILLISECONDS,
@@ -2175,32 +2195,18 @@ object JobManager {
     }
   }
 
-  /** Starts an ActorSystem, the JobManager and all its components including the WebMonitor.
+  /**
+    * Starts the JobManager actor system.
     *
-    * @param configuration The configuration object for the JobManager
-    * @param executionMode The execution mode in which to run. Execution mode LOCAL with spawn an
-    *                      additional TaskManager in the same process.
-    * @param externalHostname The hostname where the JobManager is reachable for rpc communication
-    * @param port The port where the JobManager is reachable for rpc communication
-    * @param futureExecutor to run the JobManager's futures
-    * @param ioExecutor to run blocking io operations
-    * @param jobManagerClass The class of the JobManager to be started
-    * @param archiveClass The class of the Archivist to be started
-    * @param resourceManagerClass Optional class of resource manager if one should be started
-    * @return A tuple containing the started ActorSystem, ActorRefs to the JobManager and the
-    *         Archivist and an Option containing a possibly started WebMonitor
+    * @param configuration Configuration to use for the job manager actor system
+    * @param externalHostname External hostname to bind to
+    * @param port Port to bind to
+    * @return Actor system for the JobManager and its components
     */
-  def startActorSystemAndJobManagerActors(
+  def startActorSystem(
       configuration: Configuration,
-      executionMode: JobManagerMode,
       externalHostname: String,
-      port: Int,
-      futureExecutor: ScheduledExecutorService,
-      ioExecutor: Executor,
-      jobManagerClass: Class[_ <: JobManager],
-      archiveClass: Class[_ <: MemoryArchivist],
-      resourceManagerClass: Option[Class[_ <: FlinkResourceManager[_]]])
-    : (ActorSystem, ActorRef, ActorRef, Option[WebMonitor], Option[ActorRef]) = {
+      port: Int): ActorSystem = {
 
     val hostPort = NetUtils.unresolvedHostAndPortToNormalizedString(externalHostname, port)
 
@@ -2223,7 +2229,7 @@ object JobManager {
           val cause = t.getCause()
           if (cause != null && t.getCause().isInstanceOf[java.net.BindException]) {
             throw new Exception("Unable to create JobManager at address " + hostPort +
-              " - " + cause.getMessage(), t)
+                                  " - " + cause.getMessage(), t)
           }
         }
         throw new Exception("Could not create JobManager actor system", t)
@@ -2234,11 +2240,42 @@ object JobManager {
     configuration.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, address.host.get)
     configuration.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, address.port.get)
 
+    jobManagerSystem
+  }
+
+  /** Starts the JobManager and all its components including the WebMonitor.
+    *
+    * @param configuration The configuration object for the JobManager
+    * @param executionMode The execution mode in which to run. Execution mode LOCAL with spawn an
+    *                      additional TaskManager in the same process.
+    * @param externalHostname The hostname where the JobManager is reachable for rpc communication
+    * @param futureExecutor to run the JobManager's futures
+    * @param ioExecutor to run blocking io operations
+    * @param highAvailabilityServices to instantiate high availability services
+    * @param jobManagerClass The class of the JobManager to be started
+    * @param archiveClass The class of the Archivist to be started
+    * @param resourceManagerClass Optional class of resource manager if one should be started
+    * @return A tuple containing the started ActorSystem, ActorRefs to the JobManager and the
+    *         Archivist and an Option containing a possibly started WebMonitor
+    */
+  def startJobManagerActors(
+      jobManagerSystem: ActorSystem,
+      configuration: Configuration,
+      executionMode: JobManagerMode,
+      externalHostname: String,
+      futureExecutor: ScheduledExecutorService,
+      ioExecutor: Executor,
+      highAvailabilityServices: HighAvailabilityServices,
+      jobManagerClass: Class[_ <: JobManager],
+      archiveClass: Class[_ <: MemoryArchivist],
+      resourceManagerClass: Option[Class[_ <: FlinkResourceManager[_]]])
+    : (ActorRef, ActorRef, Option[WebMonitor], Option[ActorRef]) = {
+
     val webMonitor: Option[WebMonitor] =
       if (configuration.getInteger(ConfigConstants.JOB_MANAGER_WEB_PORT_KEY, 0) >= 0) {
         LOG.info("Starting JobManager web frontend")
-        val leaderRetrievalService = LeaderRetrievalUtils
-          .createLeaderRetrievalService(configuration, false)
+        val leaderRetrievalService = highAvailabilityServices.getJobManagerLeaderRetriever(
+          HighAvailabilityServices.DEFAULT_JOB_ID)
 
         // start the web frontend. we need to load this dynamically
         // because it is not in the same project/dependencies
@@ -2265,6 +2302,7 @@ object JobManager {
         jobManagerSystem,
         futureExecutor,
         ioExecutor,
+        highAvailabilityServices,
         jobManagerClass,
         archiveClass)
 
@@ -2287,9 +2325,9 @@ object JobManager {
           configuration,
           ResourceID.generate(),
           jobManagerSystem,
+          highAvailabilityServices,
           externalHostname,
           Some(TaskExecutor.TASK_MANAGER_NAME),
-          None,
           localTaskManagerCommunication = true,
           classOf[TaskManager])
 
@@ -2324,14 +2362,15 @@ object JobManager {
               FlinkResourceManager.startResourceManagerActors(
                 configuration,
                 jobManagerSystem,
-                LeaderRetrievalUtils.createLeaderRetrievalService(configuration, false),
+                highAvailabilityServices.getJobManagerLeaderRetriever(
+                  HighAvailabilityServices.DEFAULT_JOB_ID),
                 rmClass))
           case None =>
             LOG.info("Resource Manager class not provided. No resource manager will be started.")
             None
         }
 
-      (jobManagerSystem, jobManager, archive, webMonitor, resourceManager)
+      (jobManager, archive, webMonitor, resourceManager)
     }
     catch {
       case t: Throwable =>
@@ -2468,15 +2507,12 @@ object JobManager {
    * @param configuration The configuration from which to parse the config values.
    * @param futureExecutor to run JobManager's futures
    * @param ioExecutor to run blocking io operations
-   * @param leaderElectionServiceOption LeaderElectionService which shall be returned if the option
-   *                                    is defined
    * @return The members for a default JobManager.
    */
   def createJobManagerComponents(
       configuration: Configuration,
       futureExecutor: ScheduledExecutorService,
-      ioExecutor: Executor,
-      leaderElectionServiceOption: Option[LeaderElectionService]) :
+      ioExecutor: Executor) :
     (InstanceManager,
     FlinkScheduler,
     BlobLibraryCacheManager,
@@ -2484,9 +2520,6 @@ object JobManager {
     FiniteDuration, // timeout
     Int, // number of archived jobs
     Option[Path], // archive path
-    LeaderElectionService,
-    SubmittedJobGraphStore,
-    CheckpointRecoveryFactory,
     FiniteDuration, // timeout for job recovery
     Option[FlinkMetricRegistry]
    ) = {
@@ -2550,32 +2583,6 @@ object JobManager {
         throw t
     }
 
-    // Create recovery related components
-    val (leaderElectionService, submittedJobGraphs, checkpointRecoveryFactory) =
-      HighAvailabilityMode.fromConfig(configuration) match {
-        case HighAvailabilityMode.NONE =>
-          val leaderElectionService = leaderElectionServiceOption match {
-            case Some(les) => les
-            case None => new StandaloneLeaderElectionService()
-          }
-
-          (leaderElectionService,
-            new StandaloneSubmittedJobGraphStore(),
-            new StandaloneCheckpointRecoveryFactory())
-
-        case HighAvailabilityMode.ZOOKEEPER =>
-          val client = ZooKeeperUtils.startCuratorFramework(configuration)
-
-          val leaderElectionService = leaderElectionServiceOption match {
-            case Some(les) => les
-            case None => ZooKeeperUtils.createLeaderElectionService(client, configuration)
-          }
-
-          (leaderElectionService,
-            ZooKeeperUtils.createSubmittedJobGraphs(client, configuration, ioExecutor),
-            new ZooKeeperCheckpointRecoveryFactory(client, configuration, ioExecutor))
-      }
-
     val jobRecoveryTimeoutStr = configuration.getValue(HighAvailabilityOptions.HA_JOB_DELAY)
 
     val jobRecoveryTimeout = if (jobRecoveryTimeoutStr == null || jobRecoveryTimeoutStr.isEmpty) {
@@ -2605,9 +2612,6 @@ object JobManager {
       timeout,
       archiveCount,
       archivePath,
-      leaderElectionService,
-      submittedJobGraphs,
-      checkpointRecoveryFactory,
       jobRecoveryTimeout,
       metricRegistry)
   }
@@ -2629,6 +2633,7 @@ object JobManager {
       actorSystem: ActorSystem,
       futureExecutor: ScheduledExecutorService,
       ioExecutor: Executor,
+      highAvailabilityServices: HighAvailabilityServices,
       jobManagerClass: Class[_ <: JobManager],
       archiveClass: Class[_ <: MemoryArchivist])
     : (ActorRef, ActorRef) = {
@@ -2638,6 +2643,7 @@ object JobManager {
       actorSystem,
       futureExecutor,
       ioExecutor,
+      highAvailabilityServices,
       Some(JobMaster.JOB_MANAGER_NAME),
       Some(JobMaster.ARCHIVE_NAME),
       jobManagerClass,
@@ -2665,6 +2671,7 @@ object JobManager {
       actorSystem: ActorSystem,
       futureExecutor: ScheduledExecutorService,
       ioExecutor: Executor,
+      highAvailabilityServices: HighAvailabilityServices,
       jobManagerActorName: Option[String],
       archiveActorName: Option[String],
       jobManagerClass: Class[_ <: JobManager],
@@ -2678,15 +2685,11 @@ object JobManager {
     timeout,
     archiveCount,
     archivePath,
-    leaderElectionService,
-    submittedJobGraphs,
-    checkpointRecoveryFactory,
     jobRecoveryTimeout,
     metricsRegistry) = createJobManagerComponents(
       configuration,
       futureExecutor,
-      ioExecutor,
-      None)
+      ioExecutor)
 
     val archiveProps = getArchiveProps(archiveClass, archiveCount, archivePath)
 
@@ -2707,9 +2710,10 @@ object JobManager {
       archive,
       restartStrategy,
       timeout,
-      leaderElectionService,
-      submittedJobGraphs,
-      checkpointRecoveryFactory,
+      highAvailabilityServices.getJobManagerLeaderElectionService(
+        HighAvailabilityServices.DEFAULT_JOB_ID),
+      highAvailabilityServices.getSubmittedJobGraphStore(),
+      highAvailabilityServices.getCheckpointRecoveryFactory(),
       jobRecoveryTimeout,
       metricsRegistry)
 

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/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 a493b3d..79141f1 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
@@ -101,9 +101,10 @@ object TaskManagerMessages {
 
   /**
    * Acknowledges that the task manager has been successfully registered at any job manager. This
-   * message is a response to [[NotifyWhenRegisteredAtJobManager]].
+   * message is a response to [[NotifyWhenRegisteredAtJobManager]] and contains the current leader
+   * session id.
    */
-  case object RegisteredAtJobManager
+  case class RegisteredAtJobManager(leaderId: UUID)
 
   /** Tells the address of the new leading [[org.apache.flink.runtime.jobmanager.JobManager]]
     * and the new leader session ID.
@@ -151,14 +152,6 @@ object TaskManagerMessages {
   NotifyWhenRegisteredAtJobManager.type = NotifyWhenRegisteredAtJobManager
 
   /**
-   * Accessor for the case object instance, to simplify Java interoperability.
-   *
-   * @return The RegisteredAtJobManager case object instance.
-   */
-  def getRegisteredAtJobManagerMessage:
-            RegisteredAtJobManager.type = RegisteredAtJobManager
-
-  /**
     * Accessor for the case object instance, to simplify Java interoperability.
     * @return The RequestTaskManagerLog case object instance.
     */

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/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 07fb996..46c4404 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
@@ -18,6 +18,7 @@
 
 package org.apache.flink.runtime.minicluster
 
+import java.net.URL
 import java.util.UUID
 import java.util.concurrent.{Executors, TimeUnit}
 
@@ -25,20 +26,21 @@ import akka.pattern.Patterns.gracefulStop
 import akka.pattern.ask
 import akka.actor.{ActorRef, ActorSystem}
 import com.typesafe.config.Config
-
 import org.apache.flink.api.common.{JobExecutionResult, JobID, JobSubmissionResult}
 import org.apache.flink.configuration.{ConfigConstants, Configuration}
+import org.apache.flink.core.fs.Path
 import org.apache.flink.runtime.akka.AkkaUtils
 import org.apache.flink.runtime.client.{JobClient, JobExecutionException}
 import org.apache.flink.runtime.concurrent.{Executors => FlinkExecutors}
+import org.apache.flink.runtime.execution.librarycache.FlinkUserCodeClassLoader
+import org.apache.flink.runtime.highavailability.{HighAvailabilityServices, HighAvailabilityServicesUtils}
 import org.apache.flink.runtime.instance.{ActorGateway, AkkaActorGateway}
 import org.apache.flink.runtime.jobgraph.JobGraph
 import org.apache.flink.runtime.jobmanager.HighAvailabilityMode
 import org.apache.flink.runtime.leaderretrieval.{LeaderRetrievalListener, LeaderRetrievalService, StandaloneLeaderRetrievalService}
 import org.apache.flink.runtime.messages.TaskManagerMessages.NotifyWhenRegisteredAtJobManager
-import org.apache.flink.runtime.util.{ExecutorThreadFactory, Hardware, ZooKeeperUtils}
+import org.apache.flink.runtime.util.{ExecutorThreadFactory, Hardware}
 import org.apache.flink.runtime.webmonitor.{WebMonitor, WebMonitorUtils}
-
 import org.slf4j.LoggerFactory
 
 import scala.concurrent.duration.{Duration, FiniteDuration}
@@ -56,6 +58,7 @@ import scala.concurrent._
  */
 abstract class FlinkMiniCluster(
     val userConfiguration: Configuration,
+    val highAvailabilityServices: HighAvailabilityServices,
     val useSingleActorSystem: Boolean)
   extends LeaderRetrievalListener {
 
@@ -115,6 +118,15 @@ abstract class FlinkMiniCluster(
     Hardware.getNumberCPUCores(),
     new ExecutorThreadFactory("mini-cluster-io"))
 
+  def this(configuration: Configuration, useSingleActorSystem: Boolean) {
+    this(
+      configuration,
+      HighAvailabilityServicesUtils.createAvailableOrEmbeddedServices(
+        configuration,
+        ExecutionContext.global),
+      useSingleActorSystem)
+  }
+
   def configuration: Configuration = {
     if (originalConfiguration.getInteger(
       ConfigConstants.JOB_MANAGER_IPC_PORT_KEY,
@@ -148,25 +160,17 @@ abstract class FlinkMiniCluster(
   // --------------------------------------------------------------------------
 
   def getNumberOfJobManagers: Int = {
-    if(haMode == HighAvailabilityMode.NONE) {
-      1
-    } else {
-      originalConfiguration.getInteger(
-        ConfigConstants.LOCAL_NUMBER_JOB_MANAGER,
-        ConfigConstants.DEFAULT_LOCAL_NUMBER_JOB_MANAGER
-      )
-    }
+    originalConfiguration.getInteger(
+      ConfigConstants.LOCAL_NUMBER_JOB_MANAGER,
+      ConfigConstants.DEFAULT_LOCAL_NUMBER_JOB_MANAGER
+    )
   }
 
   def getNumberOfResourceManagers: Int = {
-    if(haMode == HighAvailabilityMode.NONE) {
-      1
-    } else {
-      originalConfiguration.getInteger(
-        ConfigConstants.LOCAL_NUMBER_RESOURCE_MANAGER,
-        ConfigConstants.DEFAULT_LOCAL_NUMBER_RESOURCE_MANAGER
-      )
-    }
+    originalConfiguration.getInteger(
+      ConfigConstants.LOCAL_NUMBER_RESOURCE_MANAGER,
+      ConfigConstants.DEFAULT_LOCAL_NUMBER_RESOURCE_MANAGER
+    )
   }
 
   def getJobManagersAsJava = {
@@ -327,10 +331,11 @@ abstract class FlinkMiniCluster(
     jobManagerActorSystems = Some(jmActorSystems)
     jobManagerActors = Some(jmActors)
 
-    // start leader retrieval service
-    val lrs = createLeaderRetrievalService()
-    jobManagerLeaderRetrievalService = Some(lrs)
-    lrs.start(this)
+    // find out which job manager the leader is
+    jobManagerLeaderRetrievalService = Option(highAvailabilityServices.getJobManagerLeaderRetriever(
+      HighAvailabilityServices.DEFAULT_JOB_ID))
+
+    jobManagerLeaderRetrievalService.foreach(_.start(this))
 
     // start as many resource managers as job managers
     val (rmActorSystems, rmActors) =
@@ -383,7 +388,8 @@ abstract class FlinkMiniCluster(
         config.getInteger(ConfigConstants.JOB_MANAGER_WEB_PORT_KEY, 0) >= 0) {
 
       // TODO: Add support for HA: Make web server work independently from the JM
-      val leaderRetrievalService = new StandaloneLeaderRetrievalService(jobManagerAkkaURL)
+      val leaderRetrievalService = highAvailabilityServices.getJobManagerLeaderRetriever(
+        HighAvailabilityServices.DEFAULT_JOB_ID)
 
       LOG.info("Starting JobManger web frontend")
       // start the new web frontend. we need to load this dynamically
@@ -409,6 +415,9 @@ abstract class FlinkMiniCluster(
     awaitTermination()
 
     jobManagerLeaderRetrievalService.foreach(_.stop())
+
+    highAvailabilityServices.closeAndCleanupAllData()
+
     isRunning = false
 
     FlinkExecutors.gracefulShutdown(
@@ -509,33 +518,37 @@ abstract class FlinkMiniCluster(
     submitJobAndWait(jobGraph, printUpdates, timeout)
   }
 
-  def submitJobAndWait(
-    jobGraph: JobGraph,
-    printUpdates: Boolean,
-    timeout: FiniteDuration)
-  : JobExecutionResult = {
-    submitJobAndWait(jobGraph, printUpdates, timeout, createLeaderRetrievalService())
-  }
-
   @throws(classOf[JobExecutionException])
   def submitJobAndWait(
       jobGraph: JobGraph,
       printUpdates: Boolean,
-      timeout: FiniteDuration,
-      leaderRetrievalService: LeaderRetrievalService)
+      timeout: FiniteDuration)
     : JobExecutionResult = {
 
     val clientActorSystem = startJobClientActorSystem(jobGraph.getJobID)
 
-     try {
-     JobClient.submitJobAndWait(
+    val userCodeClassLoader =
+      try {
+        createUserCodeClassLoader(
+          jobGraph.getUserJars,
+          jobGraph.getClasspaths,
+          Thread.currentThread().getContextClassLoader)
+      } catch {
+        case e: Exception => throw new JobExecutionException(
+          jobGraph.getJobID,
+          "Could not create the user code class loader.",
+          e)
+      }
+
+    try {
+      JobClient.submitJobAndWait(
        clientActorSystem,
        configuration,
-       leaderRetrievalService,
+       highAvailabilityServices,
        jobGraph,
        timeout,
        printUpdates,
-       this.getClass.getClassLoader())
+       userCodeClassLoader)
     } finally {
        if(!useSingleActorSystem) {
          // we have to shutdown the just created actor system
@@ -558,11 +571,24 @@ abstract class FlinkMiniCluster(
         )
     }
 
+    val userCodeClassLoader =
+      try {
+        createUserCodeClassLoader(
+          jobGraph.getUserJars,
+          jobGraph.getClasspaths,
+          Thread.currentThread().getContextClassLoader)
+      } catch {
+        case e: Exception => throw new JobExecutionException(
+          jobGraph.getJobID,
+          "Could not create the user code class loader.",
+          e)
+      }
+
     JobClient.submitJobDetached(jobManagerGateway,
       configuration,
       jobGraph,
       timeout,
-      this.getClass.getClassLoader())
+      userCodeClassLoader)
 
     new JobSubmissionResult(jobGraph.getJobID)
   }
@@ -573,20 +599,6 @@ abstract class FlinkMiniCluster(
     }
   }
 
-  protected def createLeaderRetrievalService(): LeaderRetrievalService = {
-    (jobManagerActorSystems, jobManagerActors) match {
-      case (Some(jmActorSystems), Some(jmActors)) =>
-        if (haMode == HighAvailabilityMode.NONE) {
-          new StandaloneLeaderRetrievalService(
-            AkkaUtils.getAkkaURL(jmActorSystems(0), jmActors(0)))
-        } else {
-          ZooKeeperUtils.createLeaderRetrievalService(originalConfiguration)
-        }
-
-      case _ => throw new Exception("The FlinkMiniCluster has not been started properly.")
-    }
-  }
-
   protected def clearLeader(): Unit = {
     futureLock.synchronized{
       leaderGateway = Promise()
@@ -643,4 +655,28 @@ abstract class FlinkMiniCluster(
       }
     }
   }
+
+  private def createUserCodeClassLoader(
+      jars: java.util.List[Path],
+      classPaths: java.util.List[URL],
+      parentClassLoader: ClassLoader): FlinkUserCodeClassLoader = {
+
+    val urls = new Array[URL](jars.size() + classPaths.size())
+
+    import scala.collection.JavaConverters._
+
+    var counter = 0
+
+    for (path <- jars.asScala) {
+      urls(counter) = path.makeQualified(path.getFileSystem).toUri.toURL
+      counter += 1
+    }
+
+    for (classPath <- classPaths.asScala) {
+      urls(counter) = classPath
+      counter += 1
+    }
+
+    new FlinkUserCodeClassLoader(urls, parentClassLoader)
+  }
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/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 2f83548..3db91b7 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
@@ -32,6 +32,7 @@ import org.apache.flink.runtime.clusterframework.standalone.StandaloneResourceMa
 import org.apache.flink.runtime.clusterframework.types.{ResourceID, ResourceIDRetrievable}
 import org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager
 import org.apache.flink.runtime.executiongraph.restart.RestartStrategyFactory
+import org.apache.flink.runtime.highavailability.{HighAvailabilityServices, HighAvailabilityServicesUtils}
 import org.apache.flink.runtime.instance.InstanceManager
 import org.apache.flink.runtime.io.disk.iomanager.IOManager
 import org.apache.flink.runtime.io.network.NetworkEnvironment
@@ -49,7 +50,7 @@ import org.apache.flink.runtime.taskexecutor.{TaskExecutor, TaskManagerConfigura
 import org.apache.flink.runtime.taskmanager.{TaskManager, TaskManagerLocation}
 import org.apache.flink.runtime.util.EnvironmentInformation
 
-import scala.concurrent.Await
+import scala.concurrent.{Await, ExecutionContext}
 import scala.concurrent.duration.FiniteDuration
 
 /**
@@ -63,7 +64,20 @@ import scala.concurrent.duration.FiniteDuration
  */
 class LocalFlinkMiniCluster(
     userConfiguration: Configuration,
-    singleActorSystem: Boolean) extends FlinkMiniCluster(userConfiguration, singleActorSystem) {
+    highAvailabilityServices: HighAvailabilityServices,
+    singleActorSystem: Boolean) extends FlinkMiniCluster(
+  userConfiguration,
+  highAvailabilityServices,
+  singleActorSystem) {
+
+  def this(userConfiguration: Configuration, useSingleActorSystem: Boolean) = {
+    this(
+      userConfiguration,
+       HighAvailabilityServicesUtils.createAvailableOrEmbeddedServices(
+         userConfiguration,
+         ExecutionContext.global),
+      useSingleActorSystem)
+  }
 
   def this(userConfiguration: Configuration) = this(userConfiguration, true)
 
@@ -125,15 +139,11 @@ class LocalFlinkMiniCluster(
     timeout,
     archiveCount,
     archivePath,
-    leaderElectionService,
-    submittedJobGraphStore,
-    checkpointRecoveryFactory,
     jobRecoveryTimeout,
     metricsRegistry) = JobManager.createJobManagerComponents(
       config,
       futureExecutor,
-      ioExecutor,
-      createLeaderElectionService())
+      ioExecutor)
 
     if (config.getBoolean(ConfigConstants.LOCAL_START_WEBSERVER, false)) {
       metricsRegistry.get.startQueryService(system, null)
@@ -158,9 +168,10 @@ class LocalFlinkMiniCluster(
         archive,
         restartStrategyFactory,
         timeout,
-        leaderElectionService,
-        submittedJobGraphStore,
-        checkpointRecoveryFactory,
+        highAvailabilityServices.getJobManagerLeaderElectionService(
+          HighAvailabilityServices.DEFAULT_JOB_ID),
+        highAvailabilityServices.getSubmittedJobGraphStore(),
+        highAvailabilityServices.getCheckpointRecoveryFactory(),
         jobRecoveryTimeout,
         metricsRegistry),
       jobManagerName)
@@ -182,7 +193,8 @@ class LocalFlinkMiniCluster(
     val resourceManagerProps = getResourceManagerProps(
       resourceManagerClass,
       config,
-      createLeaderRetrievalService())
+      highAvailabilityServices.getJobManagerLeaderRetriever(
+        HighAvailabilityServices.DEFAULT_JOB_ID))
 
     system.actorOf(resourceManagerProps, resourceManagerName)
   }
@@ -237,7 +249,8 @@ class LocalFlinkMiniCluster(
       taskManagerServices.getMemoryManager(),
       taskManagerServices.getIOManager(),
       taskManagerServices.getNetworkEnvironment,
-      createLeaderRetrievalService(),
+      highAvailabilityServices.getJobManagerLeaderRetriever(
+        HighAvailabilityServices.DEFAULT_JOB_ID),
       metricRegistry)
 
     if (config.getBoolean(ConfigConstants.LOCAL_START_WEBSERVER, false)) {
@@ -332,10 +345,6 @@ class LocalFlinkMiniCluster(
   // Helper methods
   //------------------------------------------------------------------------------------------------
 
-  def createLeaderElectionService(): Option[LeaderElectionService] = {
-    None
-  }
-
   def initializeIOFormatClasses(configuration: Configuration): Unit = {
     try {
       val om = classOf[FileOutputFormat[_]].getDeclaredMethod("initDefaultsFromConfiguration",

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/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 4065660..a3110a4 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,11 +39,14 @@ import org.apache.flink.runtime.blob.{BlobCache, BlobClient, BlobService}
 import org.apache.flink.runtime.broadcast.BroadcastVariableManager
 import org.apache.flink.runtime.clusterframework.messages.StopCluster
 import org.apache.flink.runtime.clusterframework.types.ResourceID
+import org.apache.flink.runtime.concurrent.Executors
 import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor
 import org.apache.flink.runtime.execution.ExecutionState
 import org.apache.flink.runtime.execution.librarycache.{BlobLibraryCacheManager, FallbackLibraryCacheManager, LibraryCacheManager}
 import org.apache.flink.runtime.executiongraph.{ExecutionAttemptID, PartitionInfo}
 import org.apache.flink.runtime.filecache.FileCache
+import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils.AddressResolution
+import org.apache.flink.runtime.highavailability.{HighAvailabilityServices, HighAvailabilityServicesUtils}
 import org.apache.flink.runtime.instance.{AkkaActorGateway, HardwareDescription, InstanceID}
 import org.apache.flink.runtime.io.disk.iomanager.IOManager
 import org.apache.flink.runtime.io.network.NetworkEnvironment
@@ -293,7 +296,7 @@ class TaskManager(
     // its registration at the JobManager
     case NotifyWhenRegisteredAtJobManager =>
       if (isConnected) {
-        sender ! decorateMessage(RegisteredAtJobManager)
+        sender ! decorateMessage(RegisteredAtJobManager(leaderSessionID.orNull))
       } else {
         waitForRegistration += sender
       }
@@ -993,7 +996,7 @@ class TaskManager(
 
     // notify all the actors that listen for a successful registration
     for (listener <- waitForRegistration) {
-      listener ! RegisteredAtJobManager
+      listener ! RegisteredAtJobManager(leaderSessionID.orNull)
     }
     waitForRegistration.clear()
   }
@@ -1638,20 +1641,37 @@ object TaskManager {
       taskManagerClass: Class[_ <: TaskManager])
     : Unit = {
 
-    val (taskManagerHostname, actorSystemPort) = selectNetworkInterfaceAndPort(configuration)
+    val highAvailabilityServices = HighAvailabilityServicesUtils.createHighAvailabilityServices(
+      configuration,
+      Executors.directExecutor(),
+      AddressResolution.TRY_ADDRESS_RESOLUTION)
 
-    runTaskManager(
-      taskManagerHostname,
-      resourceID,
-      actorSystemPort,
+    val (taskManagerHostname, actorSystemPort) = selectNetworkInterfaceAndPort(
       configuration,
-      taskManagerClass)
+      highAvailabilityServices)
+
+    try {
+      runTaskManager(
+        taskManagerHostname,
+        resourceID,
+        actorSystemPort,
+        configuration,
+        highAvailabilityServices,
+        taskManagerClass)
+    } finally {
+      try {
+        highAvailabilityServices.close()
+      } catch {
+        case t: Throwable => LOG.warn("Could not properly stop the high availability services.", t)
+      }
+    }
   }
 
   @throws(classOf[IOException])
   @throws(classOf[IllegalConfigurationException])
   def selectNetworkInterfaceAndPort(
-      configuration: Configuration)
+      configuration: Configuration,
+      highAvailabilityServices: HighAvailabilityServices)
     : (String, Int) = {
 
     var taskManagerHostname = configuration.getString(
@@ -1661,13 +1681,11 @@ object TaskManager {
       LOG.info("Using configured hostname/address for TaskManager: " + taskManagerHostname)
     }
     else {
-      val leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(
-        configuration,
-        true)
       val lookupTimeout = AkkaUtils.getLookupTimeout(configuration)
 
       val taskManagerAddress = LeaderRetrievalUtils.findConnectingAddress(
-        leaderRetrievalService,
+        highAvailabilityServices.getJobManagerLeaderRetriever(
+          HighAvailabilityServices.DEFAULT_JOB_ID),
         lookupTimeout)
 
       taskManagerHostname = taskManagerAddress.getHostName()
@@ -1700,13 +1718,15 @@ object TaskManager {
    * @param resourceID The id of the resource which the task manager will run on.
    * @param actorSystemPort The port at which the actor system will communicate.
    * @param configuration The configuration for the TaskManager.
+   * @param highAvailabilityServices Service factory for high availability services
    */
   @throws(classOf[Exception])
   def runTaskManager(
       taskManagerHostname: String,
       resourceID: ResourceID,
       actorSystemPort: Int,
-      configuration: Configuration)
+      configuration: Configuration,
+      highAvailabilityServices: HighAvailabilityServices)
     : Unit = {
 
     runTaskManager(
@@ -1714,6 +1734,7 @@ object TaskManager {
       resourceID,
       actorSystemPort,
       configuration,
+      highAvailabilityServices,
       classOf[TaskManager])
   }
 
@@ -1730,6 +1751,7 @@ object TaskManager {
    * @param resourceID The id of the resource which the task manager will run on.
    * @param actorSystemPort The port at which the actor system will communicate.
    * @param configuration The configuration for the TaskManager.
+   * @param highAvailabilityServices Service factory for high availability services
    * @param taskManagerClass The actor class to instantiate. Allows the use of TaskManager
    *                         subclasses for example for YARN.
    */
@@ -1739,6 +1761,7 @@ object TaskManager {
       resourceID: ResourceID,
       actorSystemPort: Int,
       configuration: Configuration,
+      highAvailabilityServices: HighAvailabilityServices,
       taskManagerClass: Class[_ <: TaskManager])
     : Unit = {
 
@@ -1778,9 +1801,9 @@ object TaskManager {
         configuration,
         resourceID,
         taskManagerSystem,
+        highAvailabilityServices,
         taskManagerHostname,
         Some(TaskExecutor.TASK_MANAGER_NAME),
-        None,
         localTaskManagerCommunication = false,
         taskManagerClass)
 
@@ -1821,8 +1844,7 @@ object TaskManager {
 
       // block until everything is done
       taskManagerSystem.awaitTermination()
-    }
-    catch {
+    } catch {
       case t: Throwable =>
         LOG.error("Error while starting up taskManager", t)
         try {
@@ -1840,17 +1862,15 @@ object TaskManager {
    * @param configuration The configuration for the TaskManager.
    * @param resourceID The id of the resource which the task manager will run on.
    * @param actorSystem The actor system that should run the TaskManager actor.
+   * @param highAvailabilityServices Factory to create high availability services
    * @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 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 taskManagerClass The class of the TaskManager actor. May be used to give
    *                         subclasses that understand additional actor messages.
-    * @throws org.apache.flink.configuration.IllegalConfigurationException
+   * @throws org.apache.flink.configuration.IllegalConfigurationException
    *                              Thrown, if the given config contains illegal values.
    * @throws java.io.IOException Thrown, if any of the I/O components (such as buffer pools,
    *                             I/O manager, ...) cannot be properly started.
@@ -1865,9 +1885,9 @@ object TaskManager {
       configuration: Configuration,
       resourceID: ResourceID,
       actorSystem: ActorSystem,
+      highAvailabilityServices: HighAvailabilityServices,
       taskManagerHostname: String,
       taskManagerActorName: Option[String],
-      leaderRetrievalServiceOption: Option[LeaderRetrievalService],
       localTaskManagerCommunication: Boolean,
       taskManagerClass: Class[_ <: TaskManager])
     : ActorRef = {
@@ -1885,11 +1905,8 @@ object TaskManager {
 
     val metricRegistry = taskManagerServices.getMetricRegistry()
 
-    val leaderRetrievalService = leaderRetrievalServiceOption match {
-      case Some(lrs) => lrs
-      // validate the address if possible (e.g. we're in Standalone mode)
-      case None => LeaderRetrievalUtils.createLeaderRetrievalService(configuration, true)
-    }
+    val leaderRetrievalService = highAvailabilityServices.getJobManagerLeaderRetriever(
+      HighAvailabilityServices.DEFAULT_JOB_ID)
 
     // create the actor properties (which define the actor constructor parameters)
     val tmProps = getTaskManagerProps(

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/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 99a3815..ec1bbd8 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
@@ -33,6 +33,7 @@ import org.apache.flink.runtime.messages.JobManagerMessages;
 import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
 import org.apache.flink.runtime.testingUtils.TestingUtils;
 
+import org.apache.flink.util.TestLogger;
 import org.junit.Test;
 
 import scala.concurrent.Await;
@@ -47,7 +48,7 @@ import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
-public class CoordinatorShutdownTest {
+public class CoordinatorShutdownTest extends TestLogger {
 	
 	@Test
 	public void testCoordinatorShutsDownOnFailure() {

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/flink-runtime/src/test/java/org/apache/flink/runtime/client/JobClientActorTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/client/JobClientActorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/client/JobClientActorTest.java
index 0ec00df..8530b0f 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/client/JobClientActorTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/client/JobClientActorTest.java
@@ -29,6 +29,7 @@ import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.akka.AkkaUtils;
 import org.apache.flink.runtime.akka.FlinkUntypedActor;
 import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
+import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices;
 import org.apache.flink.runtime.jobgraph.JobGraph;
 import org.apache.flink.runtime.leaderelection.TestingLeaderRetrievalService;
 import org.apache.flink.runtime.messages.Acknowledge;
@@ -306,11 +307,14 @@ public class JobClientActorTest extends TestLogger {
 			leaderSessionID
 		);
 
+		TestingHighAvailabilityServices highAvailabilityServices = new TestingHighAvailabilityServices();
+		highAvailabilityServices.setJobMasterLeaderRetriever(HighAvailabilityServices.DEFAULT_JOB_ID, testingLeaderRetrievalService);
+
 		JobListeningContext jobListeningContext =
 			JobClient.submitJob(
 				system,
 				clientConfig,
-				testingLeaderRetrievalService,
+				highAvailabilityServices,
 				testJobGraph,
 				timeout,
 				false,
@@ -328,6 +332,8 @@ public class JobClientActorTest extends TestLogger {
 			Assert.fail();
 		} catch (JobExecutionException e) {
 			// this is what we want
+		} finally {
+			highAvailabilityServices.closeAndCleanupAllData();
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ClusterShutdownITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ClusterShutdownITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ClusterShutdownITCase.java
index 41018dd..388572c 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ClusterShutdownITCase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ClusterShutdownITCase.java
@@ -25,6 +25,8 @@ import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.akka.AkkaUtils;
 import org.apache.flink.runtime.clusterframework.messages.StopCluster;
 import org.apache.flink.runtime.clusterframework.messages.StopClusterSuccessful;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
+import org.apache.flink.runtime.highavailability.nonha.embedded.EmbeddedHaServices;
 import org.apache.flink.runtime.instance.ActorGateway;
 import org.apache.flink.runtime.messages.Acknowledge;
 import org.apache.flink.runtime.testingUtils.TestingMessages;
@@ -32,12 +34,16 @@ import org.apache.flink.runtime.testingUtils.TestingUtils;
 import org.apache.flink.runtime.testutils.TestingResourceManager;
 import org.apache.flink.util.TestLogger;
 
+import org.junit.After;
 import org.junit.AfterClass;
+import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
 import scala.Option;
 
+import java.util.Arrays;
+
 /**
  * Runs tests to ensure that a cluster is shutdown properly.
  */
@@ -47,6 +53,19 @@ public class ClusterShutdownITCase extends TestLogger {
 
 	private static Configuration config = new Configuration();
 
+	private HighAvailabilityServices highAvailabilityServices;
+
+	@Before
+	public void setupTest() {
+		highAvailabilityServices = new EmbeddedHaServices(TestingUtils.defaultExecutor());
+	}
+
+	@After
+	public void tearDownTest() throws Exception {
+		highAvailabilityServices.closeAndCleanupAllData();
+		highAvailabilityServices = null;
+	}
+
 	@BeforeClass
 	public static void setup() {
 		system = AkkaUtils.createActorSystem(AkkaUtils.getDefaultAkkaConfig());
@@ -68,37 +87,51 @@ public class ClusterShutdownITCase extends TestLogger {
 		@Override
 		protected void run() {
 
-			ActorGateway me =
-				TestingUtils.createForwardingActor(system, getTestActor(), Option.<String>empty());
-
-			// start job manager which doesn't shutdown the actor system
-			ActorGateway jobManager =
-				TestingUtils.createJobManager(
-					system,
-					TestingUtils.defaultExecutor(),
-					TestingUtils.defaultExecutor(),
-					config,
-					"jobmanager1");
-
-			// Tell the JobManager to inform us of shutdown actions
-			jobManager.tell(TestingMessages.getNotifyOfComponentShutdown(), me);
-
-			// Register a TaskManager
-			ActorGateway taskManager =
-				TestingUtils.createTaskManager(system, jobManager, config, true, true);
-
-			// Tell the TaskManager to inform us of TaskManager shutdowns
-			taskManager.tell(TestingMessages.getNotifyOfComponentShutdown(), me);
-
-
-			// No resource manager connected
-			jobManager.tell(new StopCluster(ApplicationStatus.SUCCEEDED, "Shutting down."), me);
-
-			expectMsgAllOf(
-				new TestingMessages.ComponentShutdown(taskManager.actor()),
-				new TestingMessages.ComponentShutdown(jobManager.actor()),
-				StopClusterSuccessful.getInstance()
-			);
+			ActorGateway jobManager = null;
+			ActorGateway taskManager = null;
+			ActorGateway forwardingActor = null;
+
+			try {
+				// start job manager which doesn't shutdown the actor system
+				jobManager =
+					TestingUtils.createJobManager(
+						system,
+						TestingUtils.defaultExecutor(),
+						TestingUtils.defaultExecutor(),
+						config,
+						highAvailabilityServices,
+						"jobmanager1");
+
+				forwardingActor =
+					TestingUtils.createForwardingActor(
+						system,
+						getTestActor(),
+						jobManager.leaderSessionID(),
+						Option.<String>empty());
+
+				// Tell the JobManager to inform us of shutdown actions
+				jobManager.tell(TestingMessages.getNotifyOfComponentShutdown(), forwardingActor);
+
+				// Register a TaskManager
+				taskManager =
+					TestingUtils.createTaskManager(system, highAvailabilityServices, config, true, true);
+
+				// Tell the TaskManager to inform us of TaskManager shutdowns
+				taskManager.tell(TestingMessages.getNotifyOfComponentShutdown(), forwardingActor);
+
+
+				// No resource manager connected
+				jobManager.tell(new StopCluster(ApplicationStatus.SUCCEEDED, "Shutting down."), forwardingActor);
+
+				expectMsgAllOf(
+					new TestingMessages.ComponentShutdown(taskManager.actor()),
+					new TestingMessages.ComponentShutdown(jobManager.actor()),
+					StopClusterSuccessful.getInstance()
+				);
+			} finally {
+				TestingUtils.stopActorGatewaysGracefully(Arrays.asList(
+					jobManager, taskManager, forwardingActor));
+			}
 
 		}};
 		}};
@@ -115,50 +148,68 @@ public class ClusterShutdownITCase extends TestLogger {
 		@Override
 		protected void run() {
 
-			ActorGateway me =
-				TestingUtils.createForwardingActor(system, getTestActor(), Option.<String>empty());
-
-			// start job manager which doesn't shutdown the actor system
-			ActorGateway jobManager =
-				TestingUtils.createJobManager(
-					system,
-					TestingUtils.defaultExecutor(),
-					TestingUtils.defaultExecutor(),
-					config,
-					"jobmanager2");
-
-			// Tell the JobManager to inform us of shutdown actions
-			jobManager.tell(TestingMessages.getNotifyOfComponentShutdown(), me);
-
-			// Register a TaskManager
-			ActorGateway taskManager =
-				TestingUtils.createTaskManager(system, jobManager, config, true, true);
-
-			// Tell the TaskManager to inform us of TaskManager shutdowns
-			taskManager.tell(TestingMessages.getNotifyOfComponentShutdown(), me);
-
-			// Start resource manager and let it register
-			ActorGateway resourceManager =
-				TestingUtils.createResourceManager(system, jobManager.actor(), config);
-
-			// Tell the ResourceManager to inform us of ResourceManager shutdowns
-			resourceManager.tell(TestingMessages.getNotifyOfComponentShutdown(), me);
-
-			// notify about a resource manager registration at the job manager
-			resourceManager.tell(new TestingResourceManager.NotifyWhenResourceManagerConnected(), me);
-
-			// Wait for resource manager
-			expectMsgEquals(Acknowledge.get());
-
-			// Shutdown cluster with resource manager connected
-			jobManager.tell(new StopCluster(ApplicationStatus.SUCCEEDED, "Shutting down."), me);
-
-			expectMsgAllOf(
-				new TestingMessages.ComponentShutdown(taskManager.actor()),
-				new TestingMessages.ComponentShutdown(jobManager.actor()),
-				new TestingMessages.ComponentShutdown(resourceManager.actor()),
-				StopClusterSuccessful.getInstance()
-			);
+			ActorGateway jobManager = null;
+			ActorGateway taskManager = null;
+			ActorGateway resourceManager = null;
+			ActorGateway forwardingActor = null;
+
+			try {
+				// start job manager which doesn't shutdown the actor system
+				jobManager =
+					TestingUtils.createJobManager(
+						system,
+						TestingUtils.defaultExecutor(),
+						TestingUtils.defaultExecutor(),
+						config,
+						highAvailabilityServices,
+						"jobmanager2");
+
+				forwardingActor =
+					TestingUtils.createForwardingActor(
+						system,
+						getTestActor(),
+						jobManager.leaderSessionID(),
+						Option.<String>empty());
+
+				// Tell the JobManager to inform us of shutdown actions
+				jobManager.tell(TestingMessages.getNotifyOfComponentShutdown(), forwardingActor);
+
+				// Register a TaskManager
+				taskManager =
+					TestingUtils.createTaskManager(system, highAvailabilityServices, config, true, true);
+
+				// Tell the TaskManager to inform us of TaskManager shutdowns
+				taskManager.tell(TestingMessages.getNotifyOfComponentShutdown(), forwardingActor);
+
+				// Start resource manager and let it register
+				resourceManager =
+					TestingUtils.createResourceManager(
+						system,
+						config,
+						highAvailabilityServices);
+
+				// Tell the ResourceManager to inform us of ResourceManager shutdowns
+				resourceManager.tell(TestingMessages.getNotifyOfComponentShutdown(), forwardingActor);
+
+				// notify about a resource manager registration at the job manager
+				resourceManager.tell(new TestingResourceManager.NotifyWhenResourceManagerConnected(), forwardingActor);
+
+				// Wait for resource manager
+				expectMsgEquals(Acknowledge.get());
+
+				// Shutdown cluster with resource manager connected
+				jobManager.tell(new StopCluster(ApplicationStatus.SUCCEEDED, "Shutting down."), forwardingActor);
+
+				expectMsgAllOf(
+					new TestingMessages.ComponentShutdown(taskManager.actor()),
+					new TestingMessages.ComponentShutdown(jobManager.actor()),
+					new TestingMessages.ComponentShutdown(resourceManager.actor()),
+					StopClusterSuccessful.getInstance()
+				);
+			} finally {
+				TestingUtils.stopActorGatewaysGracefully(Arrays.asList(
+					jobManager, taskManager, resourceManager, forwardingActor));
+			}
 
 		}};
 		}};

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ResourceManagerITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ResourceManagerITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ResourceManagerITCase.java
index 6191195..d52afe4 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ResourceManagerITCase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ResourceManagerITCase.java
@@ -23,6 +23,8 @@ import akka.testkit.JavaTestKit;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.akka.AkkaUtils;
 import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
+import org.apache.flink.runtime.highavailability.nonha.embedded.EmbeddedHaServices;
 import org.apache.flink.runtime.instance.ActorGateway;
 import org.apache.flink.runtime.instance.HardwareDescription;
 import org.apache.flink.runtime.messages.Acknowledge;
@@ -32,12 +34,17 @@ import org.apache.flink.runtime.testingUtils.TestingUtils;
 import org.apache.flink.runtime.testutils.TestingResourceManager;
 import org.apache.flink.util.TestLogger;
 
+import org.junit.After;
 import org.junit.AfterClass;
+import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
 import scala.Option;
 
+
+import java.util.Arrays;
+
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.mockito.Mockito.mock;
@@ -53,6 +60,8 @@ public class ResourceManagerITCase extends TestLogger {
 
 	private static Configuration config = new Configuration();
 
+	private HighAvailabilityServices highAvailabilityServices;
+
 	@BeforeClass
 	public static void setup() {
 		system = AkkaUtils.createActorSystem(AkkaUtils.getDefaultAkkaConfig());
@@ -63,6 +72,17 @@ public class ResourceManagerITCase extends TestLogger {
 		JavaTestKit.shutdownActorSystem(system);
 	}
 
+	@Before
+	public void setupTest() {
+		highAvailabilityServices = new EmbeddedHaServices(TestingUtils.defaultExecutor());
+	}
+
+	@After
+	public void tearDownTest() throws Exception {
+		highAvailabilityServices.closeAndCleanupAllData();
+		highAvailabilityServices = null;
+	}
+
 	/**
 	 * Tests whether the resource manager connects and reconciles existing task managers.
 	 */
@@ -74,49 +94,67 @@ public class ResourceManagerITCase extends TestLogger {
 		@Override
 		protected void run() {
 
-			ActorGateway jobManager =
-				TestingUtils.createJobManager(
-					system,
-					TestingUtils.defaultExecutor(),
-					TestingUtils.defaultExecutor(),
-					config,
-					"ReconciliationTest");
-			ActorGateway me =
-				TestingUtils.createForwardingActor(system, getTestActor(), Option.<String>empty());
+			ActorGateway jobManager = null;
+			ActorGateway resourceManager = null;
+			ActorGateway forwardingActor = null;
+
+			try {
+				jobManager =
+					TestingUtils.createJobManager(
+						system,
+						TestingUtils.defaultExecutor(),
+						TestingUtils.defaultExecutor(),
+						config,
+						highAvailabilityServices,
+						"ReconciliationTest");
 
-			// !! no resource manager started !!
+				forwardingActor =
+					TestingUtils.createForwardingActor(
+						system,
+						getTestActor(),
+						jobManager.leaderSessionID(),
+						Option.<String>empty());
 
-			ResourceID resourceID = ResourceID.generate();
+				// !! no resource manager started !!
 
-			TaskManagerLocation location = mock(TaskManagerLocation.class);
-			when(location.getResourceID()).thenReturn(resourceID);
+				ResourceID resourceID = ResourceID.generate();
 
-			HardwareDescription resourceProfile = HardwareDescription.extractFromSystem(1_000_000);
+				TaskManagerLocation location = mock(TaskManagerLocation.class);
+				when(location.getResourceID()).thenReturn(resourceID);
 
-			jobManager.tell(
-				new RegistrationMessages.RegisterTaskManager(resourceID, location, resourceProfile, 1),
-				me);
+				HardwareDescription resourceProfile = HardwareDescription.extractFromSystem(1_000_000);
 
-			expectMsgClass(RegistrationMessages.AcknowledgeRegistration.class);
+				jobManager.tell(
+					new RegistrationMessages.RegisterTaskManager(resourceID, location, resourceProfile, 1),
+					forwardingActor);
 
-			// now start the resource manager
-			ActorGateway resourceManager =
-				TestingUtils.createResourceManager(system, jobManager.actor(), config);
+				expectMsgClass(RegistrationMessages.AcknowledgeRegistration.class);
 
-			// register at testing job manager to receive a message once a resource manager registers
-			resourceManager.tell(new TestingResourceManager.NotifyWhenResourceManagerConnected(), me);
+				// now start the resource manager
+				resourceManager =
+					TestingUtils.createResourceManager(
+						system,
+						config,
+						highAvailabilityServices);
 
-			// Wait for resource manager
-			expectMsgEquals(Acknowledge.get());
+				// register at testing job manager to receive a message once a resource manager registers
+				resourceManager.tell(new TestingResourceManager.NotifyWhenResourceManagerConnected(), forwardingActor);
 
-			// check if we registered the task manager resource
-			resourceManager.tell(new TestingResourceManager.GetRegisteredResources(), me);
+				// Wait for resource manager
+				expectMsgEquals(Acknowledge.get());
 
-			TestingResourceManager.GetRegisteredResourcesReply reply =
-				expectMsgClass(TestingResourceManager.GetRegisteredResourcesReply.class);
+				// check if we registered the task manager resource
+				resourceManager.tell(new TestingResourceManager.GetRegisteredResources(), forwardingActor);
 
-			assertEquals(1, reply.resources.size());
-			assertTrue(reply.resources.contains(resourceID));
+				TestingResourceManager.GetRegisteredResourcesReply reply =
+					expectMsgClass(TestingResourceManager.GetRegisteredResourcesReply.class);
+
+				assertEquals(1, reply.resources.size());
+				assertTrue(reply.resources.contains(resourceID));
+			} finally {
+				TestingUtils.stopActorGatewaysGracefully(Arrays.asList(
+					jobManager, resourceManager, forwardingActor));
+			}
 
 		}};
 		}};
@@ -133,37 +171,53 @@ public class ResourceManagerITCase extends TestLogger {
 		@Override
 		protected void run() {
 
-			ActorGateway jobManager =
-				TestingUtils.createJobManager(
-					system,
-					TestingUtils.defaultExecutor(),
-					TestingUtils.defaultExecutor(),
-					config,
-					"RegTest");
-			ActorGateway me =
-				TestingUtils.createForwardingActor(system, getTestActor(), Option.<String>empty());
-
-			// start the resource manager
-			ActorGateway resourceManager =
-				TestingUtils.createResourceManager(system, jobManager.actor(), config);
-
-			// notify about a resource manager registration at the job manager
-			resourceManager.tell(new TestingResourceManager.NotifyWhenResourceManagerConnected(), me);
-
-			// Wait for resource manager
-			expectMsgEquals(Acknowledge.get());
-
-			// start task manager and wait for registration
-			ActorGateway taskManager =
-				TestingUtils.createTaskManager(system, jobManager.actor(), config, true, true);
-
-			// check if we registered the task manager resource
-			resourceManager.tell(new TestingResourceManager.GetRegisteredResources(), me);
-
-			TestingResourceManager.GetRegisteredResourcesReply reply =
-				expectMsgClass(TestingResourceManager.GetRegisteredResourcesReply.class);
-
-			assertEquals(1, reply.resources.size());
+			ActorGateway jobManager = null;
+			ActorGateway taskManager = null;
+			ActorGateway resourceManager = null;
+			ActorGateway forwardingActor = null;
+
+			try {
+				jobManager =
+					TestingUtils.createJobManager(
+						system,
+						TestingUtils.defaultExecutor(),
+						TestingUtils.defaultExecutor(),
+						config,
+						highAvailabilityServices,
+						"RegTest");
+
+				forwardingActor =
+					TestingUtils.createForwardingActor(
+						system,
+						getTestActor(),
+						jobManager.leaderSessionID(),
+						Option.<String>empty());
+
+				// start the resource manager
+				resourceManager =
+					TestingUtils.createResourceManager(system, config, highAvailabilityServices);
+
+				// notify about a resource manager registration at the job manager
+				resourceManager.tell(new TestingResourceManager.NotifyWhenResourceManagerConnected(), forwardingActor);
+
+				// Wait for resource manager
+				expectMsgEquals(Acknowledge.get());
+
+				// start task manager and wait for registration
+				taskManager =
+					TestingUtils.createTaskManager(system, highAvailabilityServices, config, true, true);
+
+				// check if we registered the task manager resource
+				resourceManager.tell(new TestingResourceManager.GetRegisteredResources(), forwardingActor);
+
+				TestingResourceManager.GetRegisteredResourcesReply reply =
+					expectMsgClass(TestingResourceManager.GetRegisteredResourcesReply.class);
+
+				assertEquals(1, reply.resources.size());
+			} finally {
+				TestingUtils.stopActorGatewaysGracefully(Arrays.asList(
+					jobManager, resourceManager, taskManager, forwardingActor));
+			}
 
 		}};
 		}};

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ResourceManagerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ResourceManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ResourceManagerTest.java
index c740518..5aa31ff 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ResourceManagerTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ResourceManagerTest.java
@@ -36,6 +36,7 @@ import org.apache.flink.runtime.concurrent.Future;
 import org.apache.flink.runtime.concurrent.ScheduledExecutor;
 import org.apache.flink.runtime.heartbeat.HeartbeatServices;
 import org.apache.flink.runtime.heartbeat.TestingHeartbeatServices;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
 import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices;
 import org.apache.flink.runtime.instance.ActorGateway;
 import org.apache.flink.runtime.jobmaster.JobMasterGateway;
@@ -58,7 +59,9 @@ import org.apache.flink.runtime.testingUtils.TestingUtils;
 import org.apache.flink.runtime.testutils.TestingResourceManager;
 import org.apache.flink.runtime.util.TestingFatalErrorHandler;
 import org.apache.flink.util.TestLogger;
+import org.junit.After;
 import org.junit.AfterClass;
+import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.mockito.ArgumentCaptor;
@@ -90,6 +93,9 @@ public class ResourceManagerTest extends TestLogger {
 
 	private static Configuration config = new Configuration();
 
+	private TestingHighAvailabilityServices highAvailabilityServices;
+	private TestingLeaderRetrievalService jobManagerLeaderRetrievalService;
+
 	@BeforeClass
 	public static void setup() {
 		system = AkkaUtils.createLocalActorSystem(config);
@@ -100,6 +106,32 @@ public class ResourceManagerTest extends TestLogger {
 		JavaTestKit.shutdownActorSystem(system);
 	}
 
+	@Before
+	public void setupTest() {
+		jobManagerLeaderRetrievalService = new TestingLeaderRetrievalService();
+
+		highAvailabilityServices = new TestingHighAvailabilityServices();
+
+		highAvailabilityServices.setJobMasterLeaderRetriever(
+			HighAvailabilityServices.DEFAULT_JOB_ID,
+			jobManagerLeaderRetrievalService);
+	}
+
+	@After
+	public void teardownTest() throws Exception {
+		if (jobManagerLeaderRetrievalService != null) {
+			jobManagerLeaderRetrievalService.stop();
+
+			jobManagerLeaderRetrievalService = null;
+		}
+
+		if (highAvailabilityServices != null) {
+			highAvailabilityServices.closeAndCleanupAllData();
+
+			highAvailabilityServices = null;
+		}
+	}
+
 	/**
 	 * Tests the registration and reconciliation of the ResourceManager with the JobManager
 	 */
@@ -109,8 +141,20 @@ public class ResourceManagerTest extends TestLogger {
 		new Within(duration("10 seconds")) {
 		@Override
 		protected void run() {
-			fakeJobManager = TestingUtils.createForwardingActor(system, getTestActor(), Option.<String>empty());
-			resourceManager = TestingUtils.createResourceManager(system, fakeJobManager.actor(), config);
+			fakeJobManager = TestingUtils.createForwardingActor(
+				system,
+				getTestActor(),
+				HighAvailabilityServices.DEFAULT_LEADER_ID,
+				Option.<String>empty());
+
+			jobManagerLeaderRetrievalService.notifyListener(
+				fakeJobManager.path(),
+				HighAvailabilityServices.DEFAULT_LEADER_ID);
+
+			resourceManager = TestingUtils.createResourceManager(
+				system,
+				config,
+				highAvailabilityServices);
 
 			expectMsgClass(RegisterResourceManager.class);
 
@@ -150,8 +194,20 @@ public class ResourceManagerTest extends TestLogger {
 			Configuration shortTimeoutConfig = config.clone();
 			shortTimeoutConfig.setString(ConfigConstants.AKKA_LOOKUP_TIMEOUT, "1 s");
 
-			fakeJobManager = TestingUtils.createForwardingActor(system, getTestActor(), Option.<String>empty());
-			resourceManager = TestingUtils.createResourceManager(system, fakeJobManager.actor(), shortTimeoutConfig);
+			fakeJobManager = TestingUtils.createForwardingActor(
+				system,
+				getTestActor(),
+				HighAvailabilityServices.DEFAULT_LEADER_ID,
+				Option.<String>empty());
+
+			jobManagerLeaderRetrievalService.notifyListener(
+				fakeJobManager.path(),
+				HighAvailabilityServices.DEFAULT_LEADER_ID);
+
+			resourceManager = TestingUtils.createResourceManager(
+				system,
+				shortTimeoutConfig,
+				highAvailabilityServices);
 
 			// wait for registration message
 			RegisterResourceManager msg = expectMsgClass(RegisterResourceManager.class);
@@ -180,8 +236,20 @@ public class ResourceManagerTest extends TestLogger {
 			Configuration shortTimeoutConfig = config.clone();
 			shortTimeoutConfig.setString(ConfigConstants.AKKA_LOOKUP_TIMEOUT, "99999 s");
 
-			fakeJobManager = TestingUtils.createForwardingActor(system, getTestActor(), Option.<String>empty());
-			resourceManager = TestingUtils.createResourceManager(system, fakeJobManager.actor(), shortTimeoutConfig);
+			fakeJobManager = TestingUtils.createForwardingActor(
+				system,
+				getTestActor(),
+				HighAvailabilityServices.DEFAULT_LEADER_ID,
+				Option.<String>empty());
+
+			jobManagerLeaderRetrievalService.notifyListener(
+				fakeJobManager.path(),
+				HighAvailabilityServices.DEFAULT_LEADER_ID);
+
+			resourceManager = TestingUtils.createResourceManager(
+				system,
+				shortTimeoutConfig,
+				highAvailabilityServices);
 
 			// wait for registration message
 			RegisterResourceManager msg = expectMsgClass(RegisterResourceManager.class);
@@ -212,8 +280,20 @@ public class ResourceManagerTest extends TestLogger {
 		@Override
 		protected void run() {
 
-			fakeJobManager = TestingUtils.createForwardingActor(system, getTestActor(), Option.<String>empty());
-			resourceManager = TestingUtils.createResourceManager(system, fakeJobManager.actor(), config);
+			fakeJobManager = TestingUtils.createForwardingActor(
+				system,
+				getTestActor(),
+				HighAvailabilityServices.DEFAULT_LEADER_ID,
+				Option.<String>empty());
+
+			jobManagerLeaderRetrievalService.notifyListener(
+				fakeJobManager.path(),
+				HighAvailabilityServices.DEFAULT_LEADER_ID);
+
+			resourceManager = TestingUtils.createResourceManager(
+				system,
+				config,
+				highAvailabilityServices);
 
 			// register with JM
 			expectMsgClass(RegisterResourceManager.class);
@@ -270,8 +350,20 @@ public class ResourceManagerTest extends TestLogger {
 		@Override
 		protected void run() {
 
-			fakeJobManager = TestingUtils.createForwardingActor(system, getTestActor(), Option.<String>empty());
-			resourceManager = TestingUtils.createResourceManager(system, fakeJobManager.actor(), config);
+			fakeJobManager = TestingUtils.createForwardingActor(
+				system,
+				getTestActor(),
+				HighAvailabilityServices.DEFAULT_LEADER_ID,
+				Option.<String>empty());
+
+			jobManagerLeaderRetrievalService.notifyListener(
+				fakeJobManager.path(),
+				HighAvailabilityServices.DEFAULT_LEADER_ID);
+
+			resourceManager = TestingUtils.createResourceManager(
+				system,
+				config,
+				highAvailabilityServices);
 
 			// register with JM
 			expectMsgClass(RegisterResourceManager.class);
@@ -321,8 +413,20 @@ public class ResourceManagerTest extends TestLogger {
 		@Override
 		protected void run() {
 
-			fakeJobManager = TestingUtils.createForwardingActor(system, getTestActor(), Option.<String>empty());
-			resourceManager = TestingUtils.createResourceManager(system, fakeJobManager.actor(), config);
+			fakeJobManager = TestingUtils.createForwardingActor(
+				system,
+				getTestActor(),
+				HighAvailabilityServices.DEFAULT_LEADER_ID,
+				Option.<String>empty());
+
+			jobManagerLeaderRetrievalService.notifyListener(
+				fakeJobManager.path(),
+				HighAvailabilityServices.DEFAULT_LEADER_ID);
+
+			resourceManager = TestingUtils.createResourceManager(
+				system,
+				config,
+				highAvailabilityServices);
 
 			// register with JM
 			expectMsgClass(RegisterResourceManager.class);

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/ManualLeaderService.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/ManualLeaderService.java b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/ManualLeaderService.java
new file mode 100644
index 0000000..423c0ce
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/ManualLeaderService.java
@@ -0,0 +1,116 @@
+/*
+ * 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.highavailability;
+
+import org.apache.flink.runtime.leaderelection.LeaderElectionService;
+import org.apache.flink.runtime.leaderelection.TestingLeaderElectionService;
+import org.apache.flink.runtime.leaderelection.TestingLeaderRetrievalService;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+import org.apache.flink.util.Preconditions;
+
+import javax.annotation.Nullable;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.UUID;
+
+/**
+ * Leader service for {@link TestingManualHighAvailabilityServices} implementation. The leader
+ * service allows to create multiple {@link TestingLeaderElectionService} and
+ * {@link TestingLeaderRetrievalService} and allows to manually trigger the services identified
+ * by a continuous index.
+ */
+public class ManualLeaderService {
+
+	private final List<TestingLeaderElectionService> leaderElectionServices;
+	private final List<TestingLeaderRetrievalService> leaderRetrievalServices;
+
+	private int currentLeaderIndex;
+
+	@Nullable
+	private UUID currentLeaderId;
+
+	public ManualLeaderService() {
+		leaderElectionServices = new ArrayList<>(4);
+		leaderRetrievalServices = new ArrayList<>(4);
+
+		currentLeaderIndex = -1;
+		currentLeaderId = null;
+	}
+
+	public LeaderRetrievalService createLeaderRetrievalService() {
+		final TestingLeaderRetrievalService testingLeaderRetrievalService = new TestingLeaderRetrievalService(
+			getLeaderAddress(currentLeaderIndex),
+			currentLeaderId);
+
+		leaderRetrievalServices.add(testingLeaderRetrievalService);
+
+		return testingLeaderRetrievalService;
+	}
+
+	public LeaderElectionService createLeaderElectionService() {
+		TestingLeaderElectionService testingLeaderElectionService = new TestingLeaderElectionService();
+
+		leaderElectionServices.add(testingLeaderElectionService);
+
+		return testingLeaderElectionService;
+	}
+
+	public void grantLeadership(int index, UUID leaderId) {
+		if (currentLeaderId != null) {
+			revokeLeadership();
+		}
+
+		Preconditions.checkNotNull(leaderId);
+		Preconditions.checkArgument(0 <= index && index < leaderElectionServices.size());
+
+		TestingLeaderElectionService testingLeaderElectionService = leaderElectionServices.get(index);
+
+		testingLeaderElectionService.isLeader(leaderId);
+
+		currentLeaderIndex = index;
+		currentLeaderId = leaderId;
+	}
+
+	public void revokeLeadership() {
+		assert(currentLeaderId != null);
+		assert(0 <= currentLeaderIndex &&  currentLeaderIndex < leaderElectionServices.size());
+
+		TestingLeaderElectionService testingLeaderElectionService = leaderElectionServices.get(currentLeaderIndex);
+
+		testingLeaderElectionService.notLeader();
+
+		currentLeaderIndex = -1;
+		currentLeaderId = null;
+	}
+
+	public void notifyRetrievers(int index, UUID leaderId) {
+		for (TestingLeaderRetrievalService retrievalService: leaderRetrievalServices) {
+			retrievalService.notifyListener(getLeaderAddress(index), leaderId);
+		}
+	}
+
+	private String getLeaderAddress(int index) {
+		if (0 <= index && index < leaderElectionServices.size()) {
+			TestingLeaderElectionService testingLeaderElectionService = leaderElectionServices.get(index);
+			return testingLeaderElectionService.getAddress();
+		} else {
+			return null;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/TestingManualHighAvailabilityServices.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/TestingManualHighAvailabilityServices.java b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/TestingManualHighAvailabilityServices.java
new file mode 100644
index 0000000..676b0d8
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/TestingManualHighAvailabilityServices.java
@@ -0,0 +1,150 @@
+/*
+ * 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.highavailability;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.blob.BlobStore;
+import org.apache.flink.runtime.blob.VoidBlobStore;
+import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory;
+import org.apache.flink.runtime.checkpoint.StandaloneCheckpointRecoveryFactory;
+import org.apache.flink.runtime.highavailability.nonha.standalone.StandaloneRunningJobsRegistry;
+import org.apache.flink.runtime.jobmanager.StandaloneSubmittedJobGraphStore;
+import org.apache.flink.runtime.jobmanager.SubmittedJobGraphStore;
+import org.apache.flink.runtime.leaderelection.LeaderElectionService;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.UUID;
+
+/**
+ * Testing high availability service which can be manually controlled. The leader election and
+ * notification of new leaders is triggered manually via {@link #grantLeadership(JobID, int, UUID)}
+ * and {@link #notifyRetrievers(JobID, int, UUID)}.
+ */
+public class TestingManualHighAvailabilityServices implements HighAvailabilityServices {
+
+	private final Map<JobID, ManualLeaderService> jobManagerLeaderServices;
+
+	private final ManualLeaderService resourceManagerLeaderService;
+
+	public TestingManualHighAvailabilityServices() {
+		jobManagerLeaderServices = new HashMap<>(4);
+		resourceManagerLeaderService = new ManualLeaderService();
+	}
+
+	@Override
+	public LeaderRetrievalService getResourceManagerLeaderRetriever() {
+		return resourceManagerLeaderService.createLeaderRetrievalService();
+	}
+
+	@Override
+	public LeaderRetrievalService getJobManagerLeaderRetriever(JobID jobID) {
+		ManualLeaderService leaderService = getOrCreateJobManagerLeaderService(jobID);
+
+		return leaderService.createLeaderRetrievalService();
+	}
+
+	@Override
+	public LeaderElectionService getResourceManagerLeaderElectionService() {
+		return resourceManagerLeaderService.createLeaderElectionService();
+	}
+
+	@Override
+	public LeaderElectionService getJobManagerLeaderElectionService(JobID jobID) {
+		ManualLeaderService leaderService = getOrCreateJobManagerLeaderService(jobID);
+
+		return leaderService.createLeaderElectionService();
+	}
+
+	@Override
+	public CheckpointRecoveryFactory getCheckpointRecoveryFactory() {
+		return new StandaloneCheckpointRecoveryFactory();
+	}
+
+	@Override
+	public SubmittedJobGraphStore getSubmittedJobGraphStore() throws Exception {
+		return new StandaloneSubmittedJobGraphStore();
+	}
+
+	@Override
+	public RunningJobsRegistry getRunningJobsRegistry() throws Exception {
+		return new StandaloneRunningJobsRegistry();
+	}
+
+	@Override
+	public BlobStore createBlobStore() throws IOException {
+		return new VoidBlobStore();
+	}
+
+	@Override
+	public void close() throws Exception {
+		// nothing to do
+	}
+
+	@Override
+	public void closeAndCleanupAllData() throws Exception {
+		// nothing to do
+	}
+
+	public void grantLeadership(JobID jobId, int index, UUID leaderId) {
+		ManualLeaderService manualLeaderService = jobManagerLeaderServices.get(jobId);
+
+		if (manualLeaderService != null) {
+			manualLeaderService.grantLeadership(index, leaderId);
+		} else {
+			throw new IllegalStateException("No manual leader service for job id " + jobId +
+				" has been initialized.");
+		}
+	}
+
+	public void revokeLeadership(JobID jobId) {
+		ManualLeaderService manualLeaderService = jobManagerLeaderServices.get(jobId);
+
+		if (manualLeaderService != null) {
+			manualLeaderService.revokeLeadership();
+		} else {
+			throw new IllegalStateException("No manual leader service for job id " + jobId +
+				" has been initialized.");
+		}
+	}
+
+	public void notifyRetrievers(JobID jobId, int index, UUID leaderId) {
+		ManualLeaderService manualLeaderService = jobManagerLeaderServices.get(jobId);
+
+		if (manualLeaderService != null) {
+			manualLeaderService.notifyRetrievers(index, leaderId);
+		} else {
+			throw new IllegalStateException("No manual leader service for job id " + jobId +
+				" has been initialized.");
+		}
+	}
+
+	private ManualLeaderService getOrCreateJobManagerLeaderService(JobID jobId) {
+		ManualLeaderService manualLeaderService = jobManagerLeaderServices.get(jobId);
+
+		if (manualLeaderService == null) {
+			manualLeaderService = new ManualLeaderService();
+			jobManagerLeaderServices.put(jobId, manualLeaderService);
+		}
+
+		return manualLeaderService;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerHARecoveryTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerHARecoveryTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerHARecoveryTest.java
index 77eb566..b8b5984 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerHARecoveryTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerHARecoveryTest.java
@@ -52,6 +52,7 @@ import org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager;
 import org.apache.flink.runtime.executiongraph.restart.FixedDelayRestartStrategy;
 import org.apache.flink.runtime.executiongraph.restart.RestartStrategyFactory;
 import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
+import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices;
 import org.apache.flink.runtime.instance.ActorGateway;
 import org.apache.flink.runtime.instance.AkkaActorGateway;
 import org.apache.flink.runtime.instance.InstanceManager;
@@ -67,7 +68,6 @@ import org.apache.flink.runtime.jobmanager.scheduler.Scheduler;
 import org.apache.flink.runtime.leaderelection.LeaderElectionService;
 import org.apache.flink.runtime.leaderelection.TestingLeaderElectionService;
 import org.apache.flink.runtime.leaderelection.TestingLeaderRetrievalService;
-import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
 import org.apache.flink.runtime.messages.JobManagerMessages;
 import org.apache.flink.runtime.metrics.MetricRegistry;
 import org.apache.flink.runtime.state.ChainedStateHandle;
@@ -86,6 +86,7 @@ import org.apache.flink.runtime.testutils.RecoverableCompletedCheckpointStore;
 import org.apache.flink.runtime.util.TestByteStreamStateHandleDeepCompare;
 import org.apache.flink.util.InstantiationUtil;
 
+import org.apache.flink.util.TestLogger;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Rule;
@@ -123,7 +124,7 @@ import static org.mockito.Matchers.eq;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
-public class JobManagerHARecoveryTest {
+public class JobManagerHARecoveryTest extends TestLogger {
 
 	private static ActorSystem system;
 
@@ -171,8 +172,11 @@ public class JobManagerHARecoveryTest {
 			CheckpointRecoveryFactory checkpointStateFactory = new MyCheckpointRecoveryFactory(checkpointStore, checkpointCounter);
 			TestingLeaderElectionService myLeaderElectionService = new TestingLeaderElectionService();
 			TestingLeaderRetrievalService myLeaderRetrievalService = new TestingLeaderRetrievalService(
-				"localhost",
-				HighAvailabilityServices.DEFAULT_LEADER_ID);
+				null,
+				null);
+			TestingHighAvailabilityServices testingHighAvailabilityServices = new TestingHighAvailabilityServices();
+
+			testingHighAvailabilityServices.setJobMasterLeaderRetriever(HighAvailabilityServices.DEFAULT_JOB_ID, myLeaderRetrievalService);
 
 			InstanceManager instanceManager = new InstanceManager();
 			instanceManager.addInstanceListener(scheduler);
@@ -200,14 +204,14 @@ public class JobManagerHARecoveryTest {
 			ActorGateway gateway = new AkkaActorGateway(jobManager, leaderSessionID);
 
 			taskManager = TaskManager.startTaskManagerComponentsAndActor(
-					flinkConfiguration,
-					ResourceID.generate(),
-					system,
-					"localhost",
-					Option.apply("taskmanager"),
-					Option.apply((LeaderRetrievalService) myLeaderRetrievalService),
-					true,
-					TestingTaskManager.class);
+				flinkConfiguration,
+				ResourceID.generate(),
+				system,
+				testingHighAvailabilityServices,
+				"localhost",
+				Option.apply("taskmanager"),
+				true,
+				TestingTaskManager.class);
 
 			ActorGateway tmGateway = new AkkaActorGateway(taskManager, leaderSessionID);
 
@@ -226,9 +230,9 @@ public class JobManagerHARecoveryTest {
 					vertexId,
 					vertexId,
 					vertexId,
-					100,
-					10 * 60 * 1000,
-					0,
+					100L,
+					10L * 60L * 1000L,
+					0L,
 					1,
 					ExternalizedCheckpointSettings.none(),
 					null,
@@ -604,5 +608,4 @@ public class JobManagerHARecoveryTest {
 			return recoveredStates;
 		}
 	}
-
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/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 2ac3ea7..35ac1e5 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
@@ -32,6 +32,7 @@ import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils;
 import org.apache.flink.runtime.jobmaster.JobMaster;
 import org.apache.flink.runtime.rpc.akka.AkkaRpcServiceUtils;
 import org.apache.flink.runtime.testutils.CommonTestUtils;
+import org.apache.flink.util.TestLogger;
 import org.junit.Test;
 
 import org.apache.flink.configuration.Configuration;
@@ -50,7 +51,7 @@ import java.util.regex.Pattern;
 /**
  * Tests that the JobManager process properly exits when the JobManager actor dies.
  */
-public class JobManagerProcessReapingTest {
+public class JobManagerProcessReapingTest extends TestLogger {
 
 	@Test
 	public void testReapProcessOnFailure() {

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerStartupTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerStartupTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerStartupTest.java
index 0ab1b67..9ac6873 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerStartupTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerStartupTest.java
@@ -36,6 +36,7 @@ import org.apache.flink.runtime.util.StartupUtils;
 import org.apache.flink.util.NetUtils;
 
 import org.apache.flink.util.OperatingSystem;
+import org.apache.flink.util.TestLogger;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -44,7 +45,7 @@ import org.junit.Test;
  * Tests that verify the startup behavior of the JobManager in failure
  * situations, when the JobManager cannot be started.
  */
-public class JobManagerStartupTest {
+public class JobManagerStartupTest extends TestLogger {
 
 	private final static String DOES_NOT_EXISTS_NO_SIR = "does-not-exist-no-sir";
 
@@ -101,7 +102,7 @@ public class JobManagerStartupTest {
 					throw (BindException) cause;
 				}	
 			}
-			fail("this should throw a BindException");
+			throw e;
 		}
 		finally {
 			try {


[15/16] flink git commit: [FLINK-5974] [mesos] Added configurations to support mesos-dns hostname resolution

Posted by tr...@apache.org.
[FLINK-5974] [mesos] Added configurations to support mesos-dns hostname resolution

This closes #3692.


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

Branch: refs/heads/master
Commit: 1e53b75e7df039dd45e7497a353163319ffa6182
Parents: b1f1864
Author: Vijay Srinivasaraghavan <vi...@emc.com>
Authored: Thu Apr 6 09:48:39 2017 -0700
Committer: Till Rohrmann <tr...@apache.org>
Committed: Fri May 5 11:36:10 2017 +0200

----------------------------------------------------------------------
 .../flink/configuration/ConfigConstants.java    |  4 ++
 .../clusterframework/LaunchableMesosWorker.java | 26 +++++++++++--
 .../MesosApplicationMasterRunner.java           | 17 +++++++--
 .../clusterframework/MesosConfigKeys.java       | 10 +++++
 .../MesosFlinkResourceManager.java              |  3 +-
 .../MesosTaskManagerParameters.java             | 40 +++++++++++++++++++-
 .../MesosFlinkResourceManagerTest.java          |  9 ++++-
 7 files changed, 98 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/1e53b75e/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 975a3d4..cafef2e 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
@@ -1286,6 +1286,10 @@ public final class ConfigConstants {
 
 	public static final String DEFAULT_MESOS_RESOURCEMANAGER_FRAMEWORK_USER = "";
 
+	public static final String MESOS_RESOURCEMANAGER_TASKS_HOSTNAME = "mesos.resourcemanager.tasks.hostname";
+
+	public static final String MESOS_RESOURCEMANAGER_TASKS_BOOTSTRAP_CMD = "mesos.resourcemanager.tasks.cmd-prefix";
+
 	/** Default value to override SSL support for the Artifact Server */
 	public static final boolean DEFAULT_MESOS_ARTIFACT_SERVER_SSL_ENABLED = true;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/1e53b75e/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/LaunchableMesosWorker.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/LaunchableMesosWorker.java b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/LaunchableMesosWorker.java
index 2408ac6..3e66a5d 100644
--- a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/LaunchableMesosWorker.java
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/LaunchableMesosWorker.java
@@ -22,10 +22,12 @@ import com.netflix.fenzo.ConstraintEvaluator;
 import com.netflix.fenzo.TaskAssignmentResult;
 import com.netflix.fenzo.TaskRequest;
 import com.netflix.fenzo.VMTaskFitnessCalculator;
+import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.mesos.Utils;
 import org.apache.flink.mesos.scheduler.LaunchableTask;
 import org.apache.flink.mesos.util.MesosArtifactResolver;
+import org.apache.flink.mesos.util.MesosConfiguration;
 import org.apache.flink.runtime.clusterframework.ContaineredTaskManagerParameters;
 import org.apache.flink.runtime.clusterframework.ContainerSpecification;
 import org.apache.flink.util.Preconditions;
@@ -64,6 +66,7 @@ public class LaunchableMesosWorker implements LaunchableTask {
 	private final MesosTaskManagerParameters params;
 	private final Protos.TaskID taskID;
 	private final Request taskRequest;
+	private final MesosConfiguration mesosConfiguration;
 
 	/**
 	 * Construct a launchable Mesos worker.
@@ -76,11 +79,14 @@ public class LaunchableMesosWorker implements LaunchableTask {
 			MesosArtifactResolver resolver,
 			MesosTaskManagerParameters params,
 			ContainerSpecification containerSpec,
-			Protos.TaskID taskID) {
+			Protos.TaskID taskID,
+			MesosConfiguration mesosConfiguration) {
 		this.resolver = Preconditions.checkNotNull(resolver);
-		this.params = Preconditions.checkNotNull(params);
 		this.containerSpec = Preconditions.checkNotNull(containerSpec);
+		this.params = Preconditions.checkNotNull(params);
 		this.taskID = Preconditions.checkNotNull(taskID);
+		this.mesosConfiguration = Preconditions.checkNotNull(mesosConfiguration);
+		
 		this.taskRequest = new Request();
 	}
 
@@ -193,6 +199,12 @@ public class LaunchableMesosWorker implements LaunchableTask {
 		final Protos.Environment.Builder env = cmd.getEnvironmentBuilder();
 		final StringBuilder jvmArgs = new StringBuilder();
 
+		//configure task manager hostname property if hostname override property is supplied
+		if(params.getTaskManagerHostname().isDefined()) {
+			final String taskManagerHostName = params.getTaskManagerHostname().get().replace("_TASK",taskID.getValue());
+			dynamicProperties.setString(ConfigConstants.TASK_MANAGER_HOSTNAME_KEY, taskManagerHostName);
+		}
+
 		// use the assigned ports for the TM
 		if (assignment.getAssignedPorts().size() < TM_PORT_KEYS.length) {
 			throw new IllegalArgumentException("unsufficient # of ports assigned");
@@ -234,8 +246,16 @@ public class LaunchableMesosWorker implements LaunchableTask {
 		// finalize JVM args
 		env.addVariables(variable(MesosConfigKeys.ENV_JVM_ARGS, jvmArgs.toString()));
 
+		// populate TASK_NAME and FRAMEWORK_NAME environment variables to the TM container
+		env.addVariables(variable(MesosConfigKeys.ENV_TASK_NAME, taskInfo.getTaskId().getValue()));
+		env.addVariables(variable(MesosConfigKeys.ENV_FRAMEWORK_NAME, mesosConfiguration.frameworkInfo().getName()));
+
 		// build the launch command w/ dynamic application properties
-		StringBuilder launchCommand = new StringBuilder("$FLINK_HOME/bin/mesos-taskmanager.sh ");
+		StringBuilder launchCommand = new StringBuilder();
+		if(params.bootstrapCommand().isDefined()) {
+			launchCommand.append(params.bootstrapCommand().get()).append(" && ");
+		}
+		launchCommand.append("$FLINK_HOME/bin/mesos-taskmanager.sh ");
 		launchCommand.append(ContainerSpecification.formatSystemProperties(dynamicProperties));
 		cmd.setValue(launchCommand.toString());
 

http://git-wip-us.apache.org/repos/asf/flink/blob/1e53b75e/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosApplicationMasterRunner.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosApplicationMasterRunner.java b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosApplicationMasterRunner.java
index 5513df4..1cedcc3 100644
--- a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosApplicationMasterRunner.java
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosApplicationMasterRunner.java
@@ -210,10 +210,19 @@ public class MesosApplicationMasterRunner {
 		try {
 			// ------- (1) load and parse / validate all configurations -------
 
-			// Note that we use the "appMasterHostname" given by the system, to make sure
-			// we use the hostnames consistently throughout akka.
-			// for akka "localhost" and "localhost.localdomain" are different actors.
-			final String appMasterHostname = InetAddress.getLocalHost().getHostName();
+			final String appMasterHostname;
+			//We will use JM RPC address property if it is supplied through configuration
+			final String jmRpcAddress = config.getString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, null);
+			if(jmRpcAddress != null) {
+				LOG.info("JM RPC address from Flink configuration file: {} ", jmRpcAddress);
+				appMasterHostname = jmRpcAddress;
+			} else {
+				// Note that we use the "appMasterHostname" given by the system, to make sure
+				// we use the hostnames consistently throughout akka.
+				// for akka "localhost" and "localhost.localdomain" are different actors.
+				appMasterHostname = InetAddress.getLocalHost().getHostName();
+			}
+			LOG.info("App Master Hostname to use: {}", appMasterHostname);
 
 			// Mesos configuration
 			final MesosConfiguration mesosConfig = createMesosConfig(config, appMasterHostname);

http://git-wip-us.apache.org/repos/asf/flink/blob/1e53b75e/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosConfigKeys.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosConfigKeys.java b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosConfigKeys.java
index ebd9af5..35da95f 100644
--- a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosConfigKeys.java
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosConfigKeys.java
@@ -41,6 +41,16 @@ public class MesosConfigKeys {
 	 */
 	public static final String ENV_JVM_ARGS = "JVM_ARGS";
 
+	/**
+	 * Standard environment variables used in DCOS environment
+	 */
+	public static final String ENV_TASK_NAME = "TASK_NAME";
+
+	/**
+ 	 * Standard environment variables used in DCOS environment
+ 	 */
+	public static final String ENV_FRAMEWORK_NAME = "FRAMEWORK_NAME";
+
 	/** Private constructor to prevent instantiation */
 	private MesosConfigKeys() {}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/1e53b75e/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosFlinkResourceManager.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosFlinkResourceManager.java b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosFlinkResourceManager.java
index a7321a3..a9ff6cb 100644
--- a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosFlinkResourceManager.java
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosFlinkResourceManager.java
@@ -669,7 +669,8 @@ public class MesosFlinkResourceManager extends FlinkResourceManager<RegisteredMe
 
 	private LaunchableMesosWorker createLaunchableMesosWorker(Protos.TaskID taskID) {
 		LaunchableMesosWorker launchable =
-			new LaunchableMesosWorker(artifactResolver, taskManagerParameters, taskManagerContainerSpec, taskID);
+			new LaunchableMesosWorker(artifactResolver, taskManagerParameters, taskManagerContainerSpec,
+					taskID, mesosConfig);
 		return launchable;
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/1e53b75e/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosTaskManagerParameters.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosTaskManagerParameters.java b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosTaskManagerParameters.java
index aaab027..b3b8162 100644
--- a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosTaskManagerParameters.java
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosTaskManagerParameters.java
@@ -64,6 +64,14 @@ public class MesosTaskManagerParameters {
 		key("mesos.resourcemanager.tasks.container.image.name")
 			.noDefaultValue();
 
+	public static final ConfigOption<String> MESOS_TM_HOSTNAME =
+			key(ConfigConstants.MESOS_RESOURCEMANAGER_TASKS_HOSTNAME)
+			.noDefaultValue();
+
+	public static final ConfigOption<String> MESOS_TM_BOOTSTRAP_CMD =
+			key(ConfigConstants.MESOS_RESOURCEMANAGER_TASKS_BOOTSTRAP_CMD)
+			.noDefaultValue();
+	
 	public static final ConfigOption<String> MESOS_RM_CONTAINER_VOLUMES =
 		key("mesos.resourcemanager.tasks.container.volumes")
 		.noDefaultValue();
@@ -92,6 +100,10 @@ public class MesosTaskManagerParameters {
 	private final List<Protos.Volume> containerVolumes;
 	
 	private final List<ConstraintEvaluator> constraints;
+	
+	private final Option<String> bootstrapCommand;
+
+	private final Option<String> taskManagerHostname;
 
 	public MesosTaskManagerParameters(
 			double cpus,
@@ -99,7 +111,9 @@ public class MesosTaskManagerParameters {
 			Option<String> containerImageName,
 			ContaineredTaskManagerParameters containeredParameters,
 			List<Protos.Volume> containerVolumes,
-			List<ConstraintEvaluator> constraints) {
+			List<ConstraintEvaluator> constraints,
+			Option<String> bootstrapCommand,
+			Option<String> taskManagerHostname) {
 
 		this.cpus = cpus;
 		this.containerType = Preconditions.checkNotNull(containerType);
@@ -107,6 +121,8 @@ public class MesosTaskManagerParameters {
 		this.containeredParameters = Preconditions.checkNotNull(containeredParameters);
 		this.containerVolumes = Preconditions.checkNotNull(containerVolumes);
 		this.constraints = Preconditions.checkNotNull(constraints);
+		this.bootstrapCommand = Preconditions.checkNotNull(bootstrapCommand);
+		this.taskManagerHostname = Preconditions.checkNotNull(taskManagerHostname);
 	}
 
 
@@ -154,6 +170,16 @@ public class MesosTaskManagerParameters {
 		return constraints;
 	}
 
+	/**
+ 	 * Get the taskManager hostname.
+ 	 */
+	public Option<String> getTaskManagerHostname() { return taskManagerHostname; }
+
+	/**
+ 	 * Get the bootstrap command.
+ 	 */
+	public Option<String> bootstrapCommand() { return bootstrapCommand;	}	
+
 	@Override
 	public String toString() {
 		return "MesosTaskManagerParameters{" +
@@ -163,6 +189,8 @@ public class MesosTaskManagerParameters {
 			", containeredParameters=" + containeredParameters +
 			", containerVolumes=" + containerVolumes +
 			", constraints=" + constraints +
+			", taskManagerHostName=" + taskManagerHostname +
+			", bootstrapCommand=" + bootstrapCommand +
 			'}';
 	}
 
@@ -208,13 +236,21 @@ public class MesosTaskManagerParameters {
 
 		List<Protos.Volume> containerVolumes = buildVolumes(containerVolOpt);
 
+		//obtain Task Manager Host Name from the configuration
+		Option<String> tmHostname = Option.apply(flinkConfig.getString(MESOS_TM_HOSTNAME));
+
+		//obtain bootstrap command from the configuration
+		Option<String> tmBootstrapCommand = Option.apply(flinkConfig.getString(MESOS_TM_BOOTSTRAP_CMD));
+
 		return new MesosTaskManagerParameters(
 			cpus,
 			containerType,
 			Option.apply(imageName),
 			containeredParameters,			
 			containerVolumes,
-			constraints);
+			constraints,
+			tmBootstrapCommand,
+			tmHostname);
 	}
 
 	private static List<ConstraintEvaluator> parseConstraints(String mesosConstraints) {

http://git-wip-us.apache.org/repos/asf/flink/blob/1e53b75e/flink-mesos/src/test/java/org/apache/flink/mesos/runtime/clusterframework/MesosFlinkResourceManagerTest.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/test/java/org/apache/flink/mesos/runtime/clusterframework/MesosFlinkResourceManagerTest.java b/flink-mesos/src/test/java/org/apache/flink/mesos/runtime/clusterframework/MesosFlinkResourceManagerTest.java
index 7fe5db5..7ab4e40 100644
--- a/flink-mesos/src/test/java/org/apache/flink/mesos/runtime/clusterframework/MesosFlinkResourceManagerTest.java
+++ b/flink-mesos/src/test/java/org/apache/flink/mesos/runtime/clusterframework/MesosFlinkResourceManagerTest.java
@@ -214,7 +214,14 @@ public class MesosFlinkResourceManagerTest extends TestLogger {
 			ContaineredTaskManagerParameters containeredParams =
 				new ContaineredTaskManagerParameters(1024, 768, 256, 4, new HashMap<String, String>());
 			MesosTaskManagerParameters tmParams = new MesosTaskManagerParameters(
-				1.0, MesosTaskManagerParameters.ContainerType.MESOS, Option.<String>empty(), containeredParams, Collections.<Protos.Volume>emptyList(), Collections.<ConstraintEvaluator>emptyList());
+				1.0, 
+				MesosTaskManagerParameters.ContainerType.MESOS, 
+				Option.<String>empty(), 
+				containeredParams, 
+				Collections.<Protos.Volume>emptyList(), 
+				Collections.<ConstraintEvaluator>emptyList(),
+				Option.<String>empty(),
+				Option.<String>empty());
 
 			TestActorRef<TestingMesosFlinkResourceManager> resourceManagerRef =
 				TestActorRef.create(system, MesosFlinkResourceManager.createActorProps(


[13/16] flink git commit: [FLINK-6217] Fix ContaineredTaskManagerParametersTest to properly test off heap settings

Posted by tr...@apache.org.
[FLINK-6217] Fix ContaineredTaskManagerParametersTest to properly test off heap settings


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

Branch: refs/heads/master
Commit: b1f186425c3a15de9d7c0f66e3d3d4d3a7e6b259
Parents: 5cf22f4
Author: Till Rohrmann <tr...@apache.org>
Authored: Wed May 3 17:33:01 2017 +0200
Committer: Till Rohrmann <tr...@apache.org>
Committed: Fri May 5 11:08:10 2017 +0200

----------------------------------------------------------------------
 .../ContaineredTaskManagerParametersTest.java   | 26 ++++++++++++++------
 1 file changed, 19 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/b1f18642/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ContaineredTaskManagerParametersTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ContaineredTaskManagerParametersTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ContaineredTaskManagerParametersTest.java
index c0c48f9..ad11f70 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ContaineredTaskManagerParametersTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ContaineredTaskManagerParametersTest.java
@@ -19,30 +19,42 @@
 package org.apache.flink.runtime.clusterframework;
 
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.util.TestLogger;
 import org.junit.Test;
 
-import static org.apache.flink.configuration.ConfigConstants.TASK_MANAGER_MEMORY_OFF_HEAP_KEY;
-import static org.apache.flink.configuration.TaskManagerOptions.MANAGED_MEMORY_SIZE;
+import static org.apache.flink.configuration.TaskManagerOptions.MEMORY_OFF_HEAP;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
-public class ContaineredTaskManagerParametersTest {
-	private static final long CONTAINER_MEMORY = 8192;
+public class ContaineredTaskManagerParametersTest extends TestLogger {
+	private static final long CONTAINER_MEMORY = 8192L;
 
+	/**
+	 * This tests that per default the off heap memory is set to -1.
+	 */
 	@Test
-	public void testDefaultOffHeapMemory() {
+	public void testOffHeapMemoryWithDefaultConfiguration() {
 		Configuration conf = new Configuration();
+
 		ContaineredTaskManagerParameters params =
 			ContaineredTaskManagerParameters.create(conf, CONTAINER_MEMORY, 1);
-		assertEquals(-1, params.taskManagerDirectMemoryLimitMB());
+		assertEquals(-1L, params.taskManagerDirectMemoryLimitMB());
 	}
 
+	/**
+	 * This tests that when using off heap memory the sum of on and off heap memory does not exceeds the container
+	 * maximum.
+	 */
 	@Test
 	public void testTotalMemoryDoesNotExceedContainerMemory() {
 		Configuration conf = new Configuration();
-		conf.setBoolean(MANAGED_MEMORY_SIZE.key(), true);
+		conf.setBoolean(MEMORY_OFF_HEAP, true);
+
 		ContaineredTaskManagerParameters params =
 			ContaineredTaskManagerParameters.create(conf, CONTAINER_MEMORY, 1);
+
+		assertTrue(params.taskManagerDirectMemoryLimitMB() > 0L);
+
 		assertTrue(params.taskManagerHeapSizeMB() +
 			params.taskManagerDirectMemoryLimitMB() <= CONTAINER_MEMORY);
 	}


[03/16] flink git commit: [FLINK-6136] Separate EmbeddedHaServices and StandaloneHaServices

Posted by tr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/a0bb99c7/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/NonHaRegistry.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/NonHaRegistry.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/NonHaRegistry.java
deleted file mode 100644
index ab1ce47..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/NonHaRegistry.java
+++ /dev/null
@@ -1,72 +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.highavailability.nonha;
-
-import org.apache.flink.api.common.JobID;
-import org.apache.flink.runtime.highavailability.RunningJobsRegistry;
-
-import java.util.HashMap;
-
-import static org.apache.flink.util.Preconditions.checkNotNull;
-
-/**
- * A registry for running jobs, not-highly available.
- */
-public class NonHaRegistry implements RunningJobsRegistry {
-
-	/** The currently running jobs */
-	private final HashMap<JobID, JobSchedulingStatus> jobStatus = new HashMap<>();
-
-	@Override
-	public void setJobRunning(JobID jobID) {
-		checkNotNull(jobID);
-
-		synchronized (jobStatus) {
-			jobStatus.put(jobID, JobSchedulingStatus.RUNNING);
-		}
-	}
-
-	@Override
-	public void setJobFinished(JobID jobID) {
-		checkNotNull(jobID);
-
-		synchronized (jobStatus) {
-			jobStatus.put(jobID, JobSchedulingStatus.DONE);
-		}
-	}
-
-	@Override
-	public JobSchedulingStatus getJobSchedulingStatus(JobID jobID) {
-		checkNotNull(jobID);
-		
-		synchronized (jobStatus) {
-			JobSchedulingStatus status = jobStatus.get(jobID);
-			return status == null ? JobSchedulingStatus.PENDING : status;
-		}
-	}
-
-	@Override
-	public void clearJob(JobID jobID) {
-		checkNotNull(jobID);
-
-		synchronized (jobStatus) {
-			jobStatus.remove(jobID);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/a0bb99c7/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/embedded/EmbeddedHaServices.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/embedded/EmbeddedHaServices.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/embedded/EmbeddedHaServices.java
new file mode 100644
index 0000000..a338edc
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/embedded/EmbeddedHaServices.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.highavailability.nonha.embedded;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
+import org.apache.flink.runtime.highavailability.nonha.AbstractNonHaServices;
+import org.apache.flink.runtime.leaderelection.LeaderElectionService;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+import org.apache.flink.util.Preconditions;
+
+import javax.annotation.concurrent.GuardedBy;
+import java.util.HashMap;
+import java.util.concurrent.Executor;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * An implementation of the {@link HighAvailabilityServices} for the non-high-availability case
+ * where all participants (ResourceManager, JobManagers, TaskManagers) run in the same process.
+ *
+ * <p>This implementation has no dependencies on any external services. It returns a fix
+ * pre-configured ResourceManager, and stores checkpoints and metadata simply on the heap or
+ * on a local file system and therefore in a storage without guarantees.
+ */
+public class EmbeddedHaServices extends AbstractNonHaServices {
+
+	private final Executor executor;
+
+	private final EmbeddedLeaderService resourceManagerLeaderService;
+
+	private final HashMap<JobID, EmbeddedLeaderService> jobManagerLeaderServices;
+
+	public EmbeddedHaServices(Executor executor) {
+		this.executor = Preconditions.checkNotNull(executor);
+		this.resourceManagerLeaderService = new EmbeddedLeaderService(executor);
+		this.jobManagerLeaderServices = new HashMap<>();
+	}
+
+	// ------------------------------------------------------------------------
+	//  services
+	// ------------------------------------------------------------------------
+
+	@Override
+	public LeaderRetrievalService getResourceManagerLeaderRetriever() {
+		return resourceManagerLeaderService.createLeaderRetrievalService();
+	}
+
+	@Override
+	public LeaderElectionService getResourceManagerLeaderElectionService() {
+		return resourceManagerLeaderService.createLeaderElectionService();
+	}
+
+	@Override
+	public LeaderRetrievalService getJobManagerLeaderRetriever(JobID jobID) {
+		checkNotNull(jobID);
+
+		synchronized (lock) {
+			checkNotShutdown();
+			EmbeddedLeaderService service = getOrCreateJobManagerService(jobID);
+			return service.createLeaderRetrievalService();
+		}
+	}
+
+	@Override
+	public LeaderElectionService getJobManagerLeaderElectionService(JobID jobID) {
+		checkNotNull(jobID);
+
+		synchronized (lock) {
+			checkNotShutdown();
+			EmbeddedLeaderService service = getOrCreateJobManagerService(jobID);
+			return service.createLeaderElectionService();
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	// internal
+	// ------------------------------------------------------------------------
+
+	@GuardedBy("lock")
+	private EmbeddedLeaderService getOrCreateJobManagerService(JobID jobID) {
+		EmbeddedLeaderService service = jobManagerLeaderServices.get(jobID);
+		if (service == null) {
+			service = new EmbeddedLeaderService(executor);
+			jobManagerLeaderServices.put(jobID, service);
+		}
+		return service;
+	}
+
+	// ------------------------------------------------------------------------
+	//  shutdown
+	// ------------------------------------------------------------------------
+
+	@Override
+	public void close() throws Exception {
+		synchronized (lock) {
+			if (!isShutDown()) {
+				// stop all job manager leader services
+				for (EmbeddedLeaderService service : jobManagerLeaderServices.values()) {
+					service.shutdown();
+				}
+				jobManagerLeaderServices.clear();
+
+				resourceManagerLeaderService.shutdown();
+			}
+
+			super.close();
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/a0bb99c7/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/embedded/EmbeddedLeaderService.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/embedded/EmbeddedLeaderService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/embedded/EmbeddedLeaderService.java
new file mode 100644
index 0000000..5eb4375
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/embedded/EmbeddedLeaderService.java
@@ -0,0 +1,468 @@
+/*
+ * 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.highavailability.nonha.embedded;
+
+import org.apache.flink.runtime.leaderelection.LeaderContender;
+import org.apache.flink.runtime.leaderelection.LeaderElectionService;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.Executor;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * A simple leader election service, which selects a leader among contenders and notifies listeners.
+ * 
+ * <p>An election service for contenders can be created via {@link #createLeaderElectionService()},
+ * a listener service for leader observers can be created via {@link #createLeaderRetrievalService()}.
+ */
+public class EmbeddedLeaderService {
+
+	private static final Logger LOG = LoggerFactory.getLogger(EmbeddedLeaderService.class);
+
+	private final Object lock = new Object();
+
+	private final Executor notificationExecutor;
+
+	private final Set<EmbeddedLeaderElectionService> allLeaderContenders;
+
+	private final Set<EmbeddedLeaderRetrievalService> listeners;
+
+	/** proposed leader, which has been notified of leadership grant, but has not confirmed */
+	private EmbeddedLeaderElectionService currentLeaderProposed;
+
+	/** actual leader that has confirmed leadership and of which listeners have been notified */
+	private EmbeddedLeaderElectionService currentLeaderConfirmed;
+
+	/** fencing UID for the current leader (or proposed leader) */
+	private UUID currentLeaderSessionId;
+
+	/** the cached address of the current leader */
+	private String currentLeaderAddress;
+
+	/** flag marking the service as terminated */
+	private boolean shutdown;
+
+	// ------------------------------------------------------------------------
+
+	public EmbeddedLeaderService(Executor notificationsDispatcher) {
+		this.notificationExecutor = checkNotNull(notificationsDispatcher);
+		this.allLeaderContenders = new HashSet<>();
+		this.listeners = new HashSet<>();
+	}
+
+	// ------------------------------------------------------------------------
+	//  shutdown and errors
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Shuts down this leader election service.
+	 * 
+	 * <p>This method does not perform a clean revocation of the leader status and
+	 * no notification to any leader listeners. It simply notifies all contenders
+	 * and listeners that the service is no longer available.
+	 */
+	public void shutdown() {
+		synchronized (lock) {
+			shutdownInternally(new Exception("Leader election service is shutting down"));
+		}
+	}
+
+	private void fatalError(Throwable error) {
+		LOG.error("Embedded leader election service encountered a fatal error. Shutting down service.", error);
+
+		synchronized (lock) {
+			shutdownInternally(new Exception("Leader election service is shutting down after a fatal error", error));
+		}
+	}
+
+	@GuardedBy("lock")
+	private void shutdownInternally(Exception exceptionForHandlers) {
+		assert Thread.holdsLock(lock);
+
+		if (!shutdown) {
+			// clear all leader status
+			currentLeaderProposed = null;
+			currentLeaderConfirmed = null;
+			currentLeaderSessionId = null;
+			currentLeaderAddress = null;
+
+			// fail all registered listeners
+			for (EmbeddedLeaderElectionService service : allLeaderContenders) {
+				service.shutdown(exceptionForHandlers);
+			}
+			allLeaderContenders.clear();
+
+			// fail all registered listeners
+			for (EmbeddedLeaderRetrievalService service : listeners) {
+				service.shutdown(exceptionForHandlers);
+			}
+			listeners.clear();
+
+			shutdown = true;
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  creating contenders and listeners
+	// ------------------------------------------------------------------------
+
+	public LeaderElectionService createLeaderElectionService() {
+		checkState(!shutdown, "leader election service is shut down");
+		return new EmbeddedLeaderElectionService();
+	}
+
+	public LeaderRetrievalService createLeaderRetrievalService() {
+		checkState(!shutdown, "leader election service is shut down");
+		return new EmbeddedLeaderRetrievalService();
+	}
+
+	// ------------------------------------------------------------------------
+	//  adding and removing contenders & listeners
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Callback from leader contenders when they start their service.
+	 */
+	void addContender(EmbeddedLeaderElectionService service, LeaderContender contender) {
+		synchronized (lock) {
+			checkState(!shutdown, "leader election service is shut down");
+			checkState(!service.running, "leader election service is already started");
+
+			try {
+				if (!allLeaderContenders.add(service)) {
+					throw new IllegalStateException("leader election service was added to this service multiple times");
+				}
+
+				service.contender = contender;
+				service.running = true;
+
+				updateLeader();
+			}
+			catch (Throwable t) {
+				fatalError(t);
+			}
+		}
+	}
+
+	/**
+	 * Callback from leader contenders when they stop their service.
+	 */
+	void removeContender(EmbeddedLeaderElectionService service) {
+		synchronized (lock) {
+			// if the service was not even started, simply do nothing
+			if (!service.running || shutdown) {
+				return;
+			}
+
+			try {
+				if (!allLeaderContenders.remove(service)) {
+					throw new IllegalStateException("leader election service does not belong to this service");
+				}
+
+				// stop the service
+				service.contender = null;
+				service.running = false;
+				service.isLeader = false;
+
+				// if that was the current leader, unset its status
+				if (currentLeaderConfirmed == service) {
+					currentLeaderConfirmed = null;
+					currentLeaderSessionId = null;
+					currentLeaderAddress = null;
+				}
+				if (currentLeaderProposed == service) {
+					currentLeaderProposed = null;
+					currentLeaderSessionId = null;
+				}
+
+				updateLeader();
+			}
+			catch (Throwable t) {
+				fatalError(t);
+			}
+		}
+	}
+
+	/**
+	 * Callback from leader contenders when they confirm a leader grant
+	 */
+	void confirmLeader(final EmbeddedLeaderElectionService service, final UUID leaderSessionId) {
+		synchronized (lock) {
+			// if the service was shut down in the meantime, ignore this confirmation
+			if (!service.running || shutdown) {
+				return;
+			}
+
+			try {
+				// check if the confirmation is for the same grant, or whether it is a stale grant 
+				if (service == currentLeaderProposed && currentLeaderSessionId.equals(leaderSessionId)) {
+					final String address = service.contender.getAddress();
+					LOG.info("Received confirmation of leadership for leader {} , session={}", address, leaderSessionId);
+
+					// mark leadership
+					currentLeaderConfirmed = service;
+					currentLeaderAddress = address;
+					currentLeaderProposed = null;
+					service.isLeader = true;
+
+					// notify all listeners
+					for (EmbeddedLeaderRetrievalService listener : listeners) {
+						notificationExecutor.execute(
+								new NotifyOfLeaderCall(address, leaderSessionId, listener.listener, LOG));
+					}
+				}
+				else {
+					LOG.debug("Received confirmation of leadership for a stale leadership grant. Ignoring.");
+				}
+			}
+			catch (Throwable t) {
+				fatalError(t);
+			}
+		}
+	}
+
+	@GuardedBy("lock")
+	private void updateLeader() {
+		// this must be called under the lock
+		assert Thread.holdsLock(lock);
+
+		if (currentLeaderConfirmed == null && currentLeaderProposed == null) {
+			// we need a new leader
+			if (allLeaderContenders.isEmpty()) {
+				// no new leader available, tell everyone that there is no leader currently
+				for (EmbeddedLeaderRetrievalService listener : listeners) {
+					notificationExecutor.execute(
+							new NotifyOfLeaderCall(null, null, listener.listener, LOG));
+				}
+			}
+			else {
+				// propose a leader and ask it
+				final UUID leaderSessionId = UUID.randomUUID();
+				EmbeddedLeaderElectionService leaderService = allLeaderContenders.iterator().next();
+
+				currentLeaderSessionId = leaderSessionId;
+				currentLeaderProposed = leaderService;
+
+				LOG.info("Proposing leadership to contender {} @ {}",
+						leaderService.contender, leaderService.contender.getAddress());
+
+				notificationExecutor.execute(
+						new GrantLeadershipCall(leaderService.contender, leaderSessionId, LOG));
+			}
+		}
+	}
+
+	void addListener(EmbeddedLeaderRetrievalService service, LeaderRetrievalListener listener) {
+		synchronized (lock) {
+			checkState(!shutdown, "leader election service is shut down");
+			checkState(!service.running, "leader retrieval service is already started");
+
+			try {
+				if (!listeners.add(service)) {
+					throw new IllegalStateException("leader retrieval service was added to this service multiple times");
+				}
+
+				service.listener = listener;
+				service.running = true;
+
+				// if we already have a leader, immediately notify this new listener
+				if (currentLeaderConfirmed != null) {
+					notificationExecutor.execute(
+							new NotifyOfLeaderCall(currentLeaderAddress, currentLeaderSessionId, listener, LOG));
+				}
+			}
+			catch (Throwable t) {
+				fatalError(t);
+			}
+		}
+	}
+
+	void removeListener(EmbeddedLeaderRetrievalService service) {
+		synchronized (lock) {
+			// if the service was not even started, simply do nothing
+			if (!service.running || shutdown) {
+				return;
+			}
+
+			try {
+				if (!listeners.remove(service)) {
+					throw new IllegalStateException("leader retrieval service does not belong to this service");
+				}
+
+				// stop the service
+				service.listener = null;
+				service.running = false;
+			}
+			catch (Throwable t) {
+				fatalError(t);
+			}
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  election and retrieval service implementations 
+	// ------------------------------------------------------------------------
+
+	private class EmbeddedLeaderElectionService implements LeaderElectionService {
+
+		volatile LeaderContender contender;
+
+		volatile boolean isLeader;
+
+		volatile boolean running;
+
+		@Override
+		public void start(LeaderContender contender) throws Exception {
+			checkNotNull(contender);
+			addContender(this, contender);
+		}
+
+		@Override
+		public void stop() throws Exception {
+			removeContender(this);
+		}
+
+		@Override
+		public void confirmLeaderSessionID(UUID leaderSessionID) {
+			checkNotNull(leaderSessionID);
+			confirmLeader(this, leaderSessionID);
+		}
+
+		@Override
+		public boolean hasLeadership() {
+			return isLeader;
+		}
+
+		void shutdown(Exception cause) {
+			if (running) {
+				running = false;
+				isLeader = false;
+				contender.handleError(cause);
+				contender = null;
+			}
+		}
+	}
+
+	// ------------------------------------------------------------------------
+
+	private class EmbeddedLeaderRetrievalService implements LeaderRetrievalService {
+
+		volatile LeaderRetrievalListener listener;
+
+		volatile boolean running;
+
+		@Override
+		public void start(LeaderRetrievalListener listener) throws Exception {
+			checkNotNull(listener);
+			addListener(this, listener);
+		}
+
+		@Override
+		public void stop() throws Exception {
+			removeListener(this);
+		}
+
+		public void shutdown(Exception cause) {
+			if (running) {
+				running = false;
+				listener.handleError(cause);
+				listener = null;
+			}
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  asynchronous notifications
+	// ------------------------------------------------------------------------
+
+	private static class NotifyOfLeaderCall implements Runnable {
+
+		@Nullable
+		private final String address;       // null if leader revoked without new leader
+		@Nullable
+		private final UUID leaderSessionId; // null if leader revoked without new leader
+
+		private final LeaderRetrievalListener listener;
+		private final Logger logger;
+
+		NotifyOfLeaderCall(
+				@Nullable String address,
+				@Nullable UUID leaderSessionId,
+				LeaderRetrievalListener listener,
+				Logger logger) {
+
+			this.address = address;
+			this.leaderSessionId = leaderSessionId;
+			this.listener = checkNotNull(listener);
+			this.logger = checkNotNull(logger);
+		}
+
+		@Override
+		public void run() {
+			try {
+				listener.notifyLeaderAddress(address, leaderSessionId);
+			}
+			catch (Throwable t) {
+				logger.warn("Error notifying leader listener about new leader", t);
+				listener.handleError(t instanceof Exception ? (Exception) t : new Exception(t));
+			}
+		}
+	}
+
+	// ------------------------------------------------------------------------
+
+	private static class GrantLeadershipCall implements Runnable {
+
+		private final LeaderContender contender;
+		private final UUID leaderSessionId;
+		private final Logger logger;
+
+		GrantLeadershipCall(
+				LeaderContender contender,
+				UUID leaderSessionId,
+				Logger logger) {
+
+			this.contender = checkNotNull(contender);
+			this.leaderSessionId = checkNotNull(leaderSessionId);
+			this.logger = checkNotNull(logger);
+		}
+
+		@Override
+		public void run() {
+			try {
+				contender.grantLeadership(leaderSessionId);
+			}
+			catch (Throwable t) {
+				logger.warn("Error granting leadership to contender", t);
+				contender.handleError(t instanceof Exception ? (Exception) t : new Exception(t));
+			}
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/a0bb99c7/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/leaderelection/SingleLeaderElectionService.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/leaderelection/SingleLeaderElectionService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/leaderelection/SingleLeaderElectionService.java
new file mode 100644
index 0000000..a56b077
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/leaderelection/SingleLeaderElectionService.java
@@ -0,0 +1,386 @@
+/*
+ * 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.highavailability.nonha.leaderelection;
+
+import org.apache.flink.runtime.leaderelection.LeaderContender;
+import org.apache.flink.runtime.leaderelection.LeaderElectionService;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+import java.util.HashSet;
+import java.util.UUID;
+import java.util.concurrent.Executor;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * An implementation of the {@link LeaderElectionService} interface that handles a single
+ * leader contender. When started, this service immediately grants the contender the leadership.
+ * 
+ * <p>The implementation accepts a single static leader session ID and is hence compatible with
+ * pre-configured single leader (no leader failover) setups.
+ * 
+ * <p>This implementation supports a series of leader listeners that receive notifications about
+ * the leader contender.
+ */
+public class SingleLeaderElectionService implements LeaderElectionService {
+
+	private static final Logger LOG = LoggerFactory.getLogger(SingleLeaderElectionService.class);
+
+	// ------------------------------------------------------------------------
+
+	/** lock for all operations on this instance */
+	private final Object lock = new Object();
+
+	/** The executor service that dispatches notifications */
+	private final Executor notificationExecutor;
+
+	/** The leader ID assigned to the immediate leader */
+	private final UUID leaderId;
+
+	@GuardedBy("lock")
+	private final HashSet<EmbeddedLeaderRetrievalService> listeners;
+
+	/** The currently proposed leader */
+	@GuardedBy("lock")
+	private volatile LeaderContender proposedLeader;
+
+	/** The confirmed leader */
+	@GuardedBy("lock")
+	private volatile LeaderContender leader;
+
+	/** The address of the confirmed leader */
+	@GuardedBy("lock")
+	private volatile String leaderAddress;
+
+	/** Flag marking this service as shutdown, meaning it cannot be started again */
+	@GuardedBy("lock")
+	private volatile boolean shutdown;
+
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Creates a new leader election service. The service assigns the given leader ID
+	 * to the leader contender.
+	 * 
+	 * @param leaderId The constant leader ID assigned to the leader.
+	 */
+	public SingleLeaderElectionService(Executor notificationsDispatcher, UUID leaderId) {
+		this.notificationExecutor = checkNotNull(notificationsDispatcher);
+		this.leaderId = checkNotNull(leaderId);
+		this.listeners = new HashSet<>();
+
+		shutdown = false;
+	}
+
+	// ------------------------------------------------------------------------
+	//  leader election service
+	// ------------------------------------------------------------------------
+
+	@Override
+	public void start(LeaderContender contender) throws Exception {
+		checkNotNull(contender, "contender");
+
+		synchronized (lock) {
+			checkState(!shutdown, "service is shut down");
+			checkState(proposedLeader == null, "service already started");
+
+			// directly grant leadership to the given contender
+			proposedLeader = contender;
+			notificationExecutor.execute(new GrantLeadershipCall(contender, leaderId));
+		}
+	}
+
+	@Override
+	public void stop() {
+		synchronized (lock) {
+			// notify all listeners that there is no leader
+			for (EmbeddedLeaderRetrievalService listener : listeners) {
+				notificationExecutor.execute(
+						new NotifyOfLeaderCall(null, null, listener.listener, LOG));
+			}
+
+			// if there was a leader, revoke its leadership
+			if (leader != null) {
+				try {
+					leader.revokeLeadership();
+				} catch (Throwable t) {
+					leader.handleError(t instanceof Exception ? (Exception) t : new Exception(t));
+				}
+			}
+
+			proposedLeader = null;
+			leader = null;
+			leaderAddress = null;
+		}
+	}
+
+	@Override
+	public void confirmLeaderSessionID(UUID leaderSessionID) {
+		checkNotNull(leaderSessionID, "leaderSessionID");
+		checkArgument(leaderSessionID.equals(leaderId), "confirmed wrong leader session id");
+
+		synchronized (lock) {
+			checkState(!shutdown, "service is shut down");
+			checkState(proposedLeader != null, "no leader proposed yet");
+			checkState(leader == null, "leader already confirmed");
+
+			// accept the confirmation
+			final String address = proposedLeader.getAddress();
+			leaderAddress = address;
+			leader = proposedLeader;
+
+			// notify all listeners
+			for (EmbeddedLeaderRetrievalService listener : listeners) {
+				notificationExecutor.execute(
+						new NotifyOfLeaderCall(address, leaderId, listener.listener, LOG));
+			}
+		}
+	}
+
+	@Override
+	public boolean hasLeadership() {
+		synchronized (lock) {
+			return leader != null;
+		}
+	}
+
+	void errorOnGrantLeadership(LeaderContender contender, Throwable error) {
+		LOG.warn("Error notifying leader listener about new leader", error);
+		contender.handleError(error instanceof Exception ? (Exception) error : new Exception(error));
+		
+		synchronized (lock) {
+			if (proposedLeader == contender) {
+				proposedLeader = null;
+				leader = null;
+			}
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  shutdown
+	// ------------------------------------------------------------------------
+
+	public boolean isShutdown() {
+		return shutdown;
+	}
+
+	public void shutdown() {
+		shutdownInternally(new Exception("The leader service is shutting down"));
+	}
+
+	private void shutdownInternally(Exception exceptionForHandlers) {
+		synchronized (lock) {
+			if (shutdown) {
+				return;
+			}
+
+			shutdown = true;
+
+			// fail the leader (if there is one)
+			if (leader != null) {
+				try {
+					leader.handleError(exceptionForHandlers);
+				} catch (Throwable ignored) {}
+			}
+
+			// clear all leader status
+			leader = null;
+			proposedLeader = null;
+			leaderAddress = null;
+
+			// fail all registered listeners
+			for (EmbeddedLeaderRetrievalService service : listeners) {
+				service.shutdown(exceptionForHandlers);
+			}
+			listeners.clear();
+		}
+	}
+
+	private void fatalError(Throwable error) {
+		LOG.error("Embedded leader election service encountered a fatal error. Shutting down service.", error);
+
+		shutdownInternally(new Exception("Leader election service is shutting down after a fatal error", error));
+	}
+
+	// ------------------------------------------------------------------------
+	//  leader listeners
+	// ------------------------------------------------------------------------
+
+	public LeaderRetrievalService createLeaderRetrievalService() {
+		checkState(!shutdown, "leader election service is shut down");
+		return new EmbeddedLeaderRetrievalService();
+	}
+
+	void addListener(EmbeddedLeaderRetrievalService service, LeaderRetrievalListener listener) {
+		synchronized (lock) {
+			checkState(!shutdown, "leader election service is shut down");
+			checkState(!service.running, "leader retrieval service is already started");
+
+			try {
+				if (!listeners.add(service)) {
+					throw new IllegalStateException("leader retrieval service was added to this service multiple times");
+				}
+
+				service.listener = listener;
+				service.running = true;
+
+				// if we already have a leader, immediately notify this new listener
+				if (leader != null) {
+					notificationExecutor.execute(
+							new NotifyOfLeaderCall(leaderAddress, leaderId, listener, LOG));
+				}
+			}
+			catch (Throwable t) {
+				fatalError(t);
+			}
+		}
+	}
+
+	void removeListener(EmbeddedLeaderRetrievalService service) {
+		synchronized (lock) {
+			// if the service was not even started, simply do nothing
+			if (!service.running || shutdown) {
+				return;
+			}
+
+			try {
+				if (!listeners.remove(service)) {
+					throw new IllegalStateException("leader retrieval service does not belong to this service");
+				}
+
+				// stop the service
+				service.listener = null;
+				service.running = false;
+			}
+			catch (Throwable t) {
+				fatalError(t);
+			}
+		}
+	}
+
+	// ------------------------------------------------------------------------
+
+	private class EmbeddedLeaderRetrievalService implements LeaderRetrievalService {
+
+		volatile LeaderRetrievalListener listener;
+
+		volatile boolean running;
+
+		@Override
+		public void start(LeaderRetrievalListener listener) throws Exception {
+			checkNotNull(listener);
+			addListener(this, listener);
+		}
+
+		@Override
+		public void stop() throws Exception {
+			removeListener(this);
+		}
+
+		void shutdown(Exception cause) {
+			if (running) {
+				final LeaderRetrievalListener lst = listener;
+				running = false;
+				listener = null;
+
+				try {
+					lst.handleError(cause);
+				} catch (Throwable ignored) {}
+			}
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  asynchronous notifications
+	// ------------------------------------------------------------------------
+
+	/**
+	 * This runnable informs a leader contender that it gained leadership.
+	 */
+	private class GrantLeadershipCall implements Runnable {
+
+		private final LeaderContender contender;
+		private final UUID leaderSessionId;
+
+		GrantLeadershipCall(LeaderContender contender, UUID leaderSessionId) {
+
+			this.contender = checkNotNull(contender);
+			this.leaderSessionId = checkNotNull(leaderSessionId);
+		}
+
+		@Override
+		public void run() {
+			try {
+				contender.grantLeadership(leaderSessionId);
+			}
+			catch (Throwable t) {
+				errorOnGrantLeadership(contender, t);
+			}
+		}
+	}
+
+	// ------------------------------------------------------------------------
+
+	/**
+	 * This runnable informs a leader listener of a new leader
+	 */
+	private static class NotifyOfLeaderCall implements Runnable {
+
+		@Nullable
+		private final String address;       // null if leader revoked without new leader
+		@Nullable
+		private final UUID leaderSessionId; // null if leader revoked without new leader
+
+		private final LeaderRetrievalListener listener;
+		private final Logger logger;
+
+		NotifyOfLeaderCall(
+				@Nullable String address,
+				@Nullable UUID leaderSessionId,
+				LeaderRetrievalListener listener,
+				Logger logger) {
+
+			this.address = address;
+			this.leaderSessionId = leaderSessionId;
+			this.listener = checkNotNull(listener);
+			this.logger = checkNotNull(logger);
+		}
+
+		@Override
+		public void run() {
+			try {
+				listener.notifyLeaderAddress(address, leaderSessionId);
+			}
+			catch (Throwable t) {
+				logger.warn("Error notifying leader listener about new leader", t);
+				listener.handleError(t instanceof Exception ? (Exception) t : new Exception(t));
+			}
+		}
+	}
+
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/a0bb99c7/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/standalone/StandaloneHaServices.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/standalone/StandaloneHaServices.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/standalone/StandaloneHaServices.java
new file mode 100644
index 0000000..c7b54c3
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/standalone/StandaloneHaServices.java
@@ -0,0 +1,101 @@
+/*
+ * 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.highavailability.nonha.standalone;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
+import org.apache.flink.runtime.highavailability.nonha.AbstractNonHaServices;
+import org.apache.flink.runtime.leaderelection.LeaderElectionService;
+import org.apache.flink.runtime.leaderelection.StandaloneLeaderElectionService;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+import org.apache.flink.runtime.leaderretrieval.StandaloneLeaderRetrievalService;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * An implementation of the {@link HighAvailabilityServices} for the non-high-availability case.
+ * This implementation can be used for testing, and for cluster setups that do not
+ * tolerate failures of the master processes (JobManager, ResourceManager).
+ * 
+ * <p>This implementation has no dependencies on any external services. It returns a fix
+ * pre-configured ResourceManager and JobManager, and stores checkpoints and metadata simply on the
+ * heap or on a local file system and therefore in a storage without guarantees.
+ */
+public class StandaloneHaServices extends AbstractNonHaServices {
+
+	/** The constant name of the ResourceManager RPC endpoint */
+	private static final String RESOURCE_MANAGER_RPC_ENDPOINT_NAME = "resource_manager";
+
+	/** The fix address of the ResourceManager */
+	private final String resourceManagerAddress;
+
+	/** The fix address of the JobManager */
+	private final String jobManagerAddress;
+
+	/**
+	 * Creates a new services class for the fix pre-defined leaders.
+	 * 
+	 * @param resourceManagerAddress    The fix address of the ResourceManager
+	 */
+	public StandaloneHaServices(String resourceManagerAddress, String jobManagerAddress) {
+		this.resourceManagerAddress = checkNotNull(resourceManagerAddress, "resourceManagerAddress");
+		this.jobManagerAddress = checkNotNull(jobManagerAddress, "jobManagerAddress");
+	}
+
+	// ------------------------------------------------------------------------
+	//  Services
+	// ------------------------------------------------------------------------
+
+	@Override
+	public LeaderRetrievalService getResourceManagerLeaderRetriever() {
+		synchronized (lock) {
+			checkNotShutdown();
+
+			return new StandaloneLeaderRetrievalService(resourceManagerAddress, DEFAULT_LEADER_ID);
+		}
+
+	}
+
+	@Override
+	public LeaderElectionService getResourceManagerLeaderElectionService() {
+		synchronized (lock) {
+			checkNotShutdown();
+
+			return new StandaloneLeaderElectionService();
+		}
+	}
+
+	@Override
+	public LeaderRetrievalService getJobManagerLeaderRetriever(JobID jobID) {
+		synchronized (lock) {
+			checkNotShutdown();
+
+			return new StandaloneLeaderRetrievalService(jobManagerAddress, DEFAULT_LEADER_ID);
+		}
+	}
+
+	@Override
+	public LeaderElectionService getJobManagerLeaderElectionService(JobID jobID) {
+		synchronized (lock) {
+			checkNotShutdown();
+
+			return new StandaloneLeaderElectionService();
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/a0bb99c7/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/standalone/StandaloneRunningJobsRegistry.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/standalone/StandaloneRunningJobsRegistry.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/standalone/StandaloneRunningJobsRegistry.java
new file mode 100644
index 0000000..585ef34
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/standalone/StandaloneRunningJobsRegistry.java
@@ -0,0 +1,72 @@
+/*
+ * 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.highavailability.nonha.standalone;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.highavailability.RunningJobsRegistry;
+
+import java.util.HashMap;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * A registry for running jobs, not-highly available.
+ */
+public class StandaloneRunningJobsRegistry implements RunningJobsRegistry {
+
+	/** The currently running jobs */
+	private final HashMap<JobID, JobSchedulingStatus> jobStatus = new HashMap<>();
+
+	@Override
+	public void setJobRunning(JobID jobID) {
+		checkNotNull(jobID);
+
+		synchronized (jobStatus) {
+			jobStatus.put(jobID, JobSchedulingStatus.RUNNING);
+		}
+	}
+
+	@Override
+	public void setJobFinished(JobID jobID) {
+		checkNotNull(jobID);
+
+		synchronized (jobStatus) {
+			jobStatus.put(jobID, JobSchedulingStatus.DONE);
+		}
+	}
+
+	@Override
+	public JobSchedulingStatus getJobSchedulingStatus(JobID jobID) {
+		checkNotNull(jobID);
+		
+		synchronized (jobStatus) {
+			JobSchedulingStatus status = jobStatus.get(jobID);
+			return status == null ? JobSchedulingStatus.PENDING : status;
+		}
+	}
+
+	@Override
+	public void clearJob(JobID jobID) {
+		checkNotNull(jobID);
+
+		synchronized (jobStatus) {
+			jobStatus.remove(jobID);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/a0bb99c7/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/zookeeper/ZooKeeperHaServices.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/zookeeper/ZooKeeperHaServices.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/zookeeper/ZooKeeperHaServices.java
new file mode 100644
index 0000000..5d895c1
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/zookeeper/ZooKeeperHaServices.java
@@ -0,0 +1,217 @@
+/*
+ * 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.highavailability.zookeeper;
+
+import org.apache.curator.framework.CuratorFramework;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
+import org.apache.flink.configuration.IllegalConfigurationException;
+import org.apache.flink.core.fs.FileSystem;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.runtime.blob.BlobStore;
+import org.apache.flink.runtime.blob.FileSystemBlobStore;
+import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory;
+import org.apache.flink.runtime.checkpoint.ZooKeeperCheckpointRecoveryFactory;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
+import org.apache.flink.runtime.highavailability.RunningJobsRegistry;
+import org.apache.flink.runtime.jobmanager.SubmittedJobGraphStore;
+import org.apache.flink.runtime.leaderelection.LeaderElectionService;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+import org.apache.flink.runtime.util.ZooKeeperUtils;
+
+import java.io.IOException;
+import java.util.concurrent.Executor;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.StringUtils.isNullOrWhitespaceOnly;
+
+/**
+ * An implementation of the {@link HighAvailabilityServices} using Apache ZooKeeper.
+ * The services store data in ZooKeeper's nodes as illustrated by teh following tree structure:
+ * 
+ * <pre>
+ * /flink
+ *      +/cluster_id_1/resource_manager_lock
+ *      |            |
+ *      |            +/job-id-1/job_manager_lock
+ *      |            |         /checkpoints/latest
+ *      |            |                     /latest-1
+ *      |            |                     /latest-2
+ *      |            |
+ *      |            +/job-id-2/job_manager_lock
+ *      |      
+ *      +/cluster_id_2/resource_manager_lock
+ *                   |
+ *                   +/job-id-1/job_manager_lock
+ *                            |/checkpoints/latest
+ *                            |            /latest-1
+ *                            |/persisted_job_graph
+ * </pre>
+ * 
+ * <p>The root path "/flink" is configurable via the option {@link HighAvailabilityOptions#HA_ZOOKEEPER_ROOT}.
+ * This makes sure Flink stores its data under specific subtrees in ZooKeeper, for example to
+ * accommodate specific permission.
+ * 
+ * <p>The "cluster_id" part identifies the data stored for a specific Flink "cluster". 
+ * This "cluster" can be either a standalone or containerized Flink cluster, or it can be job
+ * on a framework like YARN or Mesos (in a "per-job-cluster" mode).
+ * 
+ * <p>In case of a "per-job-cluster" on YARN or Mesos, the cluster-id is generated and configured
+ * automatically by the client or dispatcher that submits the Job to YARN or Mesos.
+ * 
+ * <p>In the case of a standalone cluster, that cluster-id needs to be configured via
+ * {@link HighAvailabilityOptions#HA_CLUSTER_ID}. All nodes with the same cluster id will join the same
+ * cluster and participate in the execution of the same set of jobs.
+ */
+public class ZooKeeperHaServices implements HighAvailabilityServices {
+
+	private static final String RESOURCE_MANAGER_LEADER_PATH = "/resource_manager_lock";
+
+	private static final String JOB_MANAGER_LEADER_PATH = "/job_manager_lock";
+
+	// ------------------------------------------------------------------------
+	
+	
+	/** The ZooKeeper client to use */
+	private final CuratorFramework client;
+
+	/** The executor to run ZooKeeper callbacks on */
+	private final Executor executor;
+
+	/** The runtime configuration */
+	private final Configuration configuration;
+
+	/** The zookeeper based running jobs registry */
+	private final RunningJobsRegistry runningJobsRegistry;
+
+	public ZooKeeperHaServices(CuratorFramework client, Executor executor, Configuration configuration) {
+		this.client = checkNotNull(client);
+		this.executor = checkNotNull(executor);
+		this.configuration = checkNotNull(configuration);
+		this.runningJobsRegistry = new ZooKeeperRunningJobsRegistry(client, configuration);
+	}
+
+	// ------------------------------------------------------------------------
+	//  Services
+	// ------------------------------------------------------------------------
+
+	@Override
+	public LeaderRetrievalService getResourceManagerLeaderRetriever() {
+		return ZooKeeperUtils.createLeaderRetrievalService(client, configuration, RESOURCE_MANAGER_LEADER_PATH);
+	}
+
+	@Override
+	public LeaderRetrievalService getJobManagerLeaderRetriever(JobID jobID) {
+		return ZooKeeperUtils.createLeaderRetrievalService(client, configuration, getPathForJobManager(jobID));
+	}
+
+	@Override
+	public LeaderElectionService getResourceManagerLeaderElectionService() {
+		return ZooKeeperUtils.createLeaderElectionService(client, configuration, RESOURCE_MANAGER_LEADER_PATH);
+	}
+
+	@Override
+	public LeaderElectionService getJobManagerLeaderElectionService(JobID jobID) {
+		return ZooKeeperUtils.createLeaderElectionService(client, configuration, getPathForJobManager(jobID));
+	}
+
+	@Override
+	public CheckpointRecoveryFactory getCheckpointRecoveryFactory() {
+		return new ZooKeeperCheckpointRecoveryFactory(client, configuration, executor);
+	}
+
+	@Override
+	public SubmittedJobGraphStore getSubmittedJobGraphStore() throws Exception {
+		return ZooKeeperUtils.createSubmittedJobGraphs(client, configuration, executor);
+	}
+
+	@Override
+	public RunningJobsRegistry getRunningJobsRegistry() {
+		return runningJobsRegistry;
+	}
+
+	@Override
+	public BlobStore createBlobStore() throws IOException {
+		return createBlobStore(configuration);
+	}
+
+	/**
+	 * Creates the BLOB store in which BLOBs are stored in a highly-available
+	 * fashion.
+	 *
+	 * @param configuration configuration to extract the storage path from
+	 * @return Blob store
+	 * @throws IOException if the blob store could not be created
+	 */
+	public static BlobStore createBlobStore(
+		final Configuration configuration) throws IOException {
+		String storagePath = configuration.getValue(
+			HighAvailabilityOptions.HA_STORAGE_PATH);
+		if (isNullOrWhitespaceOnly(storagePath)) {
+			throw new IllegalConfigurationException("Configuration is missing the mandatory parameter: " +
+					HighAvailabilityOptions.HA_STORAGE_PATH);
+		}
+
+		final Path path;
+		try {
+			path = new Path(storagePath);
+		} catch (Exception e) {
+			throw new IOException("Invalid path for highly available storage (" +
+					HighAvailabilityOptions.HA_STORAGE_PATH.key() + ')', e);
+		}
+
+		final FileSystem fileSystem;
+		try {
+			fileSystem = path.getFileSystem();
+		} catch (Exception e) {
+			throw new IOException("Could not create FileSystem for highly available storage (" +
+					HighAvailabilityOptions.HA_STORAGE_PATH.key() + ')', e);
+		}
+
+		final String clusterId =
+			configuration.getValue(HighAvailabilityOptions.HA_CLUSTER_ID);
+		storagePath += "/" + clusterId;
+
+		return new FileSystemBlobStore(fileSystem, storagePath);
+	}
+
+	// ------------------------------------------------------------------------
+	//  Shutdown
+	// ------------------------------------------------------------------------
+
+	@Override
+	public void close() throws Exception {
+		client.close();
+	}
+
+	@Override
+	public void closeAndCleanupAllData() throws Exception {
+		close();
+	}
+
+	// ------------------------------------------------------------------------
+	//  Utilities
+	// ------------------------------------------------------------------------
+
+	private static String getPathForJobManager(final JobID jobID) {
+		return "/" + jobID + JOB_MANAGER_LEADER_PATH;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/a0bb99c7/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/zookeeper/ZooKeeperRunningJobsRegistry.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/zookeeper/ZooKeeperRunningJobsRegistry.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/zookeeper/ZooKeeperRunningJobsRegistry.java
new file mode 100644
index 0000000..8a083d1
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/zookeeper/ZooKeeperRunningJobsRegistry.java
@@ -0,0 +1,128 @@
+/*
+ * 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.highavailability.zookeeper;
+
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
+import org.apache.flink.runtime.highavailability.RunningJobsRegistry;
+import org.apache.zookeeper.data.Stat;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.util.Arrays;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * A zookeeper based registry for running jobs, highly available.
+ */
+public class ZooKeeperRunningJobsRegistry implements RunningJobsRegistry {
+
+	private static final Charset ENCODING = Charset.forName("utf-8");
+
+	/** The ZooKeeper client to use */
+	private final CuratorFramework client;
+
+	private final String runningJobPath;
+
+	public ZooKeeperRunningJobsRegistry(final CuratorFramework client, final Configuration configuration) {
+		this.client = checkNotNull(client, "client");
+		this.runningJobPath = configuration.getString(HighAvailabilityOptions.ZOOKEEPER_RUNNING_JOB_REGISTRY_PATH);
+	}
+
+	@Override
+	public void setJobRunning(JobID jobID) throws IOException {
+		checkNotNull(jobID);
+
+		try {
+			writeEnumToZooKeeper(jobID, JobSchedulingStatus.RUNNING);
+		}
+		catch (Exception e) {
+			throw new IOException("Failed to set RUNNING state in ZooKeeper for job " + jobID, e);
+		}
+	}
+
+	@Override
+	public void setJobFinished(JobID jobID) throws IOException {
+		checkNotNull(jobID);
+
+		try {
+			writeEnumToZooKeeper(jobID, JobSchedulingStatus.DONE);
+		}
+		catch (Exception e) {
+			throw new IOException("Failed to set DONE state in ZooKeeper for job " + jobID, e);
+		}
+	}
+
+	@Override
+	public JobSchedulingStatus getJobSchedulingStatus(JobID jobID) throws IOException {
+		checkNotNull(jobID);
+
+		try {
+			final String zkPath = createZkPath(jobID);
+			final Stat stat = client.checkExists().forPath(zkPath);
+			if (stat != null) {
+				// found some data, try to parse it
+				final byte[] data = client.getData().forPath(zkPath);
+				if (data != null) {
+					try {
+						final String name = new String(data, ENCODING);
+						return JobSchedulingStatus.valueOf(name);
+					}
+					catch (IllegalArgumentException e) {
+						throw new IOException("Found corrupt data in ZooKeeper: " + 
+								Arrays.toString(data) + " is no valid job status");
+					}
+				}
+			}
+
+			// nothing found, yet, must be in status 'PENDING'
+			return JobSchedulingStatus.PENDING;
+		}
+		catch (Exception e) {
+			throw new IOException("Get finished state from zk fail for job " + jobID.toString(), e);
+		}
+	}
+
+	@Override
+	public void clearJob(JobID jobID) throws IOException {
+		checkNotNull(jobID);
+
+		try {
+			final String zkPath = createZkPath(jobID);
+			this.client.newNamespaceAwareEnsurePath(zkPath).ensure(client.getZookeeperClient());
+			this.client.delete().forPath(zkPath);
+		}
+		catch (Exception e) {
+			throw new IOException("Failed to clear job state from ZooKeeper for job " + jobID, e);
+		}
+	}
+
+	private String createZkPath(JobID jobID) {
+		return runningJobPath + jobID.toString();
+	}
+
+	private void writeEnumToZooKeeper(JobID jobID, JobSchedulingStatus status) throws Exception {
+		final String zkPath = createZkPath(jobID);
+		this.client.newNamespaceAwareEnsurePath(zkPath).ensure(client.getZookeeperClient());
+		this.client.setData().forPath(zkPath, status.name().getBytes(ENCODING));
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/a0bb99c7/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java
index e60ff77..3a55f2e 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java
@@ -64,7 +64,6 @@ import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
 import org.apache.flink.runtime.jobgraph.JobGraph;
 import org.apache.flink.runtime.jobgraph.JobStatus;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
-import org.apache.flink.runtime.jobmanager.JobManager;
 import org.apache.flink.runtime.jobmanager.OnCompletionActions;
 import org.apache.flink.runtime.jobmanager.PartitionProducerDisposedException;
 import org.apache.flink.runtime.jobmanager.slots.AllocatedSlot;
@@ -88,8 +87,8 @@ import org.apache.flink.runtime.rpc.FatalErrorHandler;
 import org.apache.flink.runtime.rpc.RpcEndpoint;
 import org.apache.flink.runtime.rpc.RpcMethod;
 import org.apache.flink.runtime.rpc.RpcService;
-import org.apache.flink.runtime.rpc.RpcServiceUtils;
 import org.apache.flink.runtime.rpc.StartStoppable;
+import org.apache.flink.runtime.rpc.akka.AkkaRpcServiceUtils;
 import org.apache.flink.runtime.state.KeyGroupRange;
 import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway;
 import org.apache.flink.runtime.taskexecutor.slot.SlotOffer;
@@ -127,6 +126,10 @@ import static org.apache.flink.util.Preconditions.checkNotNull;
  */
 public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 
+	/** Default names for Flink's distributed components */
+	public static final String JOB_MANAGER_NAME = "jobmanager";
+	public static final String ARCHIVE_NAME = "archive";
+
 	private static final AtomicReferenceFieldUpdater<JobMaster, UUID> LEADER_ID_UPDATER =
 			AtomicReferenceFieldUpdater.newUpdater(JobMaster.class, UUID.class, "leaderSessionID");
 
@@ -208,7 +211,7 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 			FatalErrorHandler errorHandler,
 			ClassLoader userCodeLoader) throws Exception {
 
-		super(rpcService, RpcServiceUtils.createRandomName(JobManager.JOB_MANAGER_NAME()));
+		super(rpcService, AkkaRpcServiceUtils.createRandomName(JobMaster.JOB_MANAGER_NAME));
 
 		this.resourceId = checkNotNull(resourceId);
 		this.jobGraph = checkNotNull(jobGraph);

http://git-wip-us.apache.org/repos/asf/flink/blob/a0bb99c7/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java
index 03fbef5..3e7f2f3 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java
@@ -235,7 +235,9 @@ public class MiniCluster {
 
 				// create the high-availability services
 				LOG.info("Starting high-availability services");
-				haServices = HighAvailabilityServicesUtils.createAvailableOrEmbeddedServices(configuration);
+				haServices = HighAvailabilityServicesUtils.createAvailableOrEmbeddedServices(
+					configuration,
+					commonRpcService.getExecutor());
 
 				heartbeatServices = HeartbeatServices.fromConfiguration(configuration);
 

http://git-wip-us.apache.org/repos/asf/flink/blob/a0bb99c7/flink-runtime/src/main/java/org/apache/flink/runtime/query/QueryableStateClient.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/query/QueryableStateClient.java b/flink-runtime/src/main/java/org/apache/flink/runtime/query/QueryableStateClient.java
index 5d9db19..f9cf01d 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/query/QueryableStateClient.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/query/QueryableStateClient.java
@@ -109,7 +109,7 @@ public class QueryableStateClient {
 
 		// Create a leader retrieval service
 		LeaderRetrievalService leaderRetrievalService = LeaderRetrievalUtils
-				.createLeaderRetrievalService(config);
+				.createLeaderRetrievalService(config, true);
 
 		// Get the ask timeout
 		String askTimeoutString = config.getString(

http://git-wip-us.apache.org/repos/asf/flink/blob/a0bb99c7/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java
index 2ef8276..bef0aa3 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java
@@ -87,6 +87,8 @@ public abstract class ResourceManager<WorkerType extends Serializable>
 		extends RpcEndpoint<ResourceManagerGateway>
 		implements LeaderContender {
 
+	public static final String RESOURCE_MANAGER_NAME = "resourcemanager";
+
 	/** Unique id of the resource manager */
 	private final ResourceID resourceId;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/a0bb99c7/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerConfiguration.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerConfiguration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerConfiguration.java
index fa75bbb..2c64f08 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerConfiguration.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerConfiguration.java
@@ -21,7 +21,7 @@ package org.apache.flink.runtime.resourcemanager;
 import org.apache.flink.api.common.time.Time;
 import org.apache.flink.configuration.AkkaOptions;
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.resourcemanager.exceptions.ConfigurationException;
+import org.apache.flink.util.ConfigurationException;
 import org.apache.flink.util.Preconditions;
 import scala.concurrent.duration.Duration;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/a0bb99c7/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerRuntimeServicesConfiguration.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerRuntimeServicesConfiguration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerRuntimeServicesConfiguration.java
index 79d1c02..d4bc2d5 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerRuntimeServicesConfiguration.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerRuntimeServicesConfiguration.java
@@ -21,7 +21,7 @@ package org.apache.flink.runtime.resourcemanager;
 import org.apache.flink.api.common.time.Time;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.ResourceManagerOptions;
-import org.apache.flink.runtime.resourcemanager.exceptions.ConfigurationException;
+import org.apache.flink.util.ConfigurationException;
 import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManagerConfiguration;
 import org.apache.flink.util.Preconditions;
 import scala.concurrent.duration.Duration;

http://git-wip-us.apache.org/repos/asf/flink/blob/a0bb99c7/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/exceptions/ConfigurationException.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/exceptions/ConfigurationException.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/exceptions/ConfigurationException.java
deleted file mode 100644
index 0007318..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/exceptions/ConfigurationException.java
+++ /dev/null
@@ -1,38 +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.resourcemanager.exceptions;
-
-/**
- * Exception which occures when creating a configuration object fails.
- */
-public class ConfigurationException extends Exception {
-	private static final long serialVersionUID = 3971647332059381556L;
-
-	public ConfigurationException(String message) {
-		super(message);
-	}
-
-	public ConfigurationException(String message, Throwable cause) {
-		super(message, cause);
-	}
-
-	public ConfigurationException(Throwable cause) {
-		super(cause);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/a0bb99c7/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerConfiguration.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerConfiguration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerConfiguration.java
index d21c251..a651168 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerConfiguration.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerConfiguration.java
@@ -23,7 +23,7 @@ import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.ConfigOption;
 import org.apache.flink.configuration.ConfigOptions;
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.resourcemanager.exceptions.ConfigurationException;
+import org.apache.flink.util.ConfigurationException;
 import org.apache.flink.util.Preconditions;
 import scala.concurrent.duration.Duration;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/a0bb99c7/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcServiceUtils.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcServiceUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcServiceUtils.java
deleted file mode 100644
index e555e7f..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcServiceUtils.java
+++ /dev/null
@@ -1,174 +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.rpc;
-
-import akka.actor.ActorSystem;
-import com.typesafe.config.Config;
-import org.apache.flink.api.common.time.Time;
-import org.apache.flink.configuration.ConfigConstants;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.akka.AkkaUtils;
-import org.apache.flink.runtime.net.SSLUtils;
-import org.apache.flink.runtime.rpc.akka.AkkaRpcService;
-import org.apache.flink.util.NetUtils;
-
-import org.apache.flink.util.Preconditions;
-import org.jboss.netty.channel.ChannelException;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.net.UnknownHostException;
-import java.util.concurrent.atomic.AtomicLong;
-
-import static org.apache.flink.util.Preconditions.checkArgument;
-import static org.apache.flink.util.Preconditions.checkNotNull;
-
-/**
- * These RPC utilities contain helper methods around RPC use, such as starting an RPC service,
- * or constructing RPC addresses.
- */
-public class RpcServiceUtils {
-
-	private static final Logger LOG = LoggerFactory.getLogger(RpcServiceUtils.class);
-
-	private static final AtomicLong nextNameOffset = new AtomicLong(0L);
-
-	// ------------------------------------------------------------------------
-	//  RPC instantiation
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Utility method to create RPC service from configuration and hostname, port.
-	 *
-	 * @param hostname   The hostname/address that describes the TaskManager's data location.
-	 * @param port           If true, the TaskManager will not initiate the TCP network stack.
-	 * @param configuration                 The configuration for the TaskManager.
-	 * @return   The rpc service which is used to start and connect to the TaskManager RpcEndpoint .
-	 * @throws IOException      Thrown, if the actor system can not bind to the address
-	 * @throws Exception      Thrown is some other error occurs while creating akka actor system
-	 */
-	public static RpcService createRpcService(String hostname, int port, Configuration configuration) throws Exception {
-		LOG.info("Starting AkkaRpcService at {}.", NetUtils.hostAndPortToUrlString(hostname, port));
-
-		final ActorSystem actorSystem;
-
-		try {
-			Config akkaConfig;
-
-			if (hostname != null && !hostname.isEmpty()) {
-				// remote akka config
-				akkaConfig = AkkaUtils.getAkkaConfig(configuration, hostname, port);
-			} else {
-				// local akka config
-				akkaConfig = AkkaUtils.getAkkaConfig(configuration);
-			}
-
-			LOG.debug("Using akka configuration \n {}.", akkaConfig);
-
-			actorSystem = AkkaUtils.createActorSystem(akkaConfig);
-		} catch (Throwable t) {
-			if (t instanceof ChannelException) {
-				Throwable cause = t.getCause();
-				if (cause != null && t.getCause() instanceof java.net.BindException) {
-					String address = NetUtils.hostAndPortToUrlString(hostname, port);
-					throw new IOException("Unable to bind AkkaRpcService actor system to address " +
-						address + " - " + cause.getMessage(), t);
-				}
-			}
-			throw new Exception("Could not create TaskManager actor system", t);
-		}
-
-		final Time timeout = Time.milliseconds(AkkaUtils.getTimeout(configuration).toMillis());
-		return new AkkaRpcService(actorSystem, timeout);
-	}
-
-	// ------------------------------------------------------------------------
-	//  RPC endpoint addressing
-	// ------------------------------------------------------------------------
-
-	/**
-	 *
-	 * @param hostname     The hostname or address where the target RPC service is listening.
-	 * @param port         The port where the target RPC service is listening.
-	 * @param endpointName The name of the RPC endpoint.
-	 * @param config       The configuration from which to deduce further settings.
-	 *
-	 * @return The RPC URL of the specified RPC endpoint.
-	 */
-	public static String getRpcUrl(String hostname, int port, String endpointName, Configuration config)
-			throws UnknownHostException {
-
-		checkNotNull(config, "config is null");
-
-		final boolean sslEnabled = config.getBoolean(
-					ConfigConstants.AKKA_SSL_ENABLED,
-					ConfigConstants.DEFAULT_AKKA_SSL_ENABLED) &&
-				SSLUtils.getSSLEnabled(config);
-
-		return getRpcUrl(hostname, port, endpointName, sslEnabled);
-	}
-
-	/**
-	 * 
-	 * @param hostname     The hostname or address where the target RPC service is listening.
-	 * @param port         The port where the target RPC service is listening.
-	 * @param endpointName The name of the RPC endpoint.
-	 * @param secure       True, if security/encryption is enabled, false otherwise.
-	 * 
-	 * @return The RPC URL of the specified RPC endpoint.
-	 */
-	public static String getRpcUrl(String hostname, int port, String endpointName, boolean secure)
-			throws UnknownHostException {
-
-		checkNotNull(hostname, "hostname is null");
-		checkNotNull(endpointName, "endpointName is null");
-		checkArgument(port > 0 && port <= 65535, "port must be in [1, 65535]");
-
-		final String protocol = secure ? "akka.ssl.tcp" : "akka.tcp";
-		final String hostPort = NetUtils.hostAndPortToUrlString(hostname, port);
-
-		return String.format("%s://flink@%s/user/%s", protocol, hostPort, endpointName);
-	}
-
-	/**
-	 * Creates a random name of the form prefix_X, where X is an increasing number.
-	 *
-	 * @param prefix Prefix string to prepend to the monotonically increasing name offset number
-	 * @return A random name of the form prefix_X where X is an increasing number
-	 */
-	public static String createRandomName(String prefix) {
-		Preconditions.checkNotNull(prefix, "Prefix must not be null.");
-
-		long nameOffset;
-
-		// obtain the next name offset by incrementing it atomically
-		do {
-			nameOffset = nextNameOffset.get();
-		} while (!nextNameOffset.compareAndSet(nameOffset, nameOffset + 1L));
-
-		return prefix + '_' + nameOffset;
-	}
-
-	// ------------------------------------------------------------------------
-
-	/** This class is not meant to be instantiated */
-	private RpcServiceUtils() {}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/a0bb99c7/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceUtils.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceUtils.java
new file mode 100644
index 0000000..eab4de8
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceUtils.java
@@ -0,0 +1,205 @@
+/*
+ * 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.rpc.akka;
+
+import akka.actor.ActorSystem;
+import com.typesafe.config.Config;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.akka.AkkaUtils;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils.AddressResolution;
+import org.apache.flink.runtime.net.SSLUtils;
+import org.apache.flink.runtime.rpc.RpcService;
+import org.apache.flink.util.NetUtils;
+
+import org.apache.flink.util.Preconditions;
+import org.jboss.netty.channel.ChannelException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * These RPC utilities contain helper methods around RPC use, such as starting an RPC service,
+ * or constructing RPC addresses.
+ */
+public class AkkaRpcServiceUtils {
+
+	private static final Logger LOG = LoggerFactory.getLogger(AkkaRpcServiceUtils.class);
+
+	private static final String AKKA_TCP = "akka.tcp";
+	private static final String AkKA_SSL_TCP = "akka.ssl.tcp";
+
+	private static final AtomicLong nextNameOffset = new AtomicLong(0L);
+
+	// ------------------------------------------------------------------------
+	//  RPC instantiation
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Utility method to create RPC service from configuration and hostname, port.
+	 *
+	 * @param hostname   The hostname/address that describes the TaskManager's data location.
+	 * @param port           If true, the TaskManager will not initiate the TCP network stack.
+	 * @param configuration                 The configuration for the TaskManager.
+	 * @return   The rpc service which is used to start and connect to the TaskManager RpcEndpoint .
+	 * @throws IOException      Thrown, if the actor system can not bind to the address
+	 * @throws Exception      Thrown is some other error occurs while creating akka actor system
+	 */
+	public static RpcService createRpcService(String hostname, int port, Configuration configuration) throws Exception {
+		LOG.info("Starting AkkaRpcService at {}.", NetUtils.hostAndPortToUrlString(hostname, port));
+
+		final ActorSystem actorSystem;
+
+		try {
+			Config akkaConfig;
+
+			if (hostname != null && !hostname.isEmpty()) {
+				// remote akka config
+				akkaConfig = AkkaUtils.getAkkaConfig(configuration, hostname, port);
+			} else {
+				// local akka config
+				akkaConfig = AkkaUtils.getAkkaConfig(configuration);
+			}
+
+			LOG.debug("Using akka configuration \n {}.", akkaConfig);
+
+			actorSystem = AkkaUtils.createActorSystem(akkaConfig);
+		} catch (Throwable t) {
+			if (t instanceof ChannelException) {
+				Throwable cause = t.getCause();
+				if (cause != null && t.getCause() instanceof java.net.BindException) {
+					String address = NetUtils.hostAndPortToUrlString(hostname, port);
+					throw new IOException("Unable to bind AkkaRpcService actor system to address " +
+						address + " - " + cause.getMessage(), t);
+				}
+			}
+			throw new Exception("Could not create TaskManager actor system", t);
+		}
+
+		final Time timeout = Time.milliseconds(AkkaUtils.getTimeout(configuration).toMillis());
+		return new AkkaRpcService(actorSystem, timeout);
+	}
+
+	// ------------------------------------------------------------------------
+	//  RPC endpoint addressing
+	// ------------------------------------------------------------------------
+
+	/**
+	 *
+	 * @param hostname     The hostname or address where the target RPC service is listening.
+	 * @param port         The port where the target RPC service is listening.
+	 * @param endpointName The name of the RPC endpoint.
+	 * @param config       The configuration from which to deduce further settings.
+	 *
+	 * @return The RPC URL of the specified RPC endpoint.
+	 */
+	public static String getRpcUrl(
+		String hostname,
+		int port,
+		String endpointName,
+		HighAvailabilityServicesUtils.AddressResolution addressResolution,
+		Configuration config) throws UnknownHostException {
+
+		checkNotNull(config, "config is null");
+
+		final boolean sslEnabled = config.getBoolean(
+					ConfigConstants.AKKA_SSL_ENABLED,
+					ConfigConstants.DEFAULT_AKKA_SSL_ENABLED) &&
+				SSLUtils.getSSLEnabled(config);
+
+		return getRpcUrl(
+			hostname,
+			port,
+			endpointName,
+			addressResolution,
+			sslEnabled ? AkkaProtocol.SSL_TCP : AkkaProtocol.TCP);
+	}
+
+	/**
+	 * 
+	 * @param hostname     The hostname or address where the target RPC service is listening.
+	 * @param port         The port where the target RPC service is listening.
+	 * @param endpointName The name of the RPC endpoint.
+	 * @param akkaProtocol       True, if security/encryption is enabled, false otherwise.
+	 * 
+	 * @return The RPC URL of the specified RPC endpoint.
+	 */
+	public static String getRpcUrl(
+			String hostname,
+			int port,
+			String endpointName,
+			HighAvailabilityServicesUtils.AddressResolution addressResolution,
+			AkkaProtocol akkaProtocol) throws UnknownHostException {
+
+		checkNotNull(hostname, "hostname is null");
+		checkNotNull(endpointName, "endpointName is null");
+		checkArgument(port > 0 && port <= 65535, "port must be in [1, 65535]");
+
+		final String protocolPrefix = akkaProtocol == AkkaProtocol.SSL_TCP ? AkKA_SSL_TCP : AKKA_TCP;
+
+		if (addressResolution == AddressResolution.TRY_ADDRESS_RESOLUTION) {
+			// Fail fast if the hostname cannot be resolved
+			//noinspection ResultOfMethodCallIgnored
+			InetAddress.getByName(hostname);
+		}
+
+		final String hostPort = NetUtils.unresolvedHostAndPortToNormalizedString(hostname, port);
+
+		return String.format("%s://flink@%s/user/%s", protocolPrefix, hostPort, endpointName);
+	}
+
+	public enum AkkaProtocol {
+		TCP,
+		SSL_TCP
+	}
+
+	/**
+	 * Creates a random name of the form prefix_X, where X is an increasing number.
+	 *
+	 * @param prefix Prefix string to prepend to the monotonically increasing name offset number
+	 * @return A random name of the form prefix_X where X is an increasing number
+	 */
+	public static String createRandomName(String prefix) {
+		Preconditions.checkNotNull(prefix, "Prefix must not be null.");
+
+		long nameOffset;
+
+		// obtain the next name offset by incrementing it atomically
+		do {
+			nameOffset = nextNameOffset.get();
+		} while (!nextNameOffset.compareAndSet(nameOffset, nameOffset + 1L));
+
+		return prefix + '_' + nameOffset;
+	}
+
+	// ------------------------------------------------------------------------
+
+	/** This class is not meant to be instantiated */
+	private AkkaRpcServiceUtils() {}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/a0bb99c7/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java
index 5b8c8ee..d05d900 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java
@@ -65,7 +65,7 @@ import org.apache.flink.runtime.rpc.FatalErrorHandler;
 import org.apache.flink.runtime.rpc.RpcEndpoint;
 import org.apache.flink.runtime.rpc.RpcMethod;
 import org.apache.flink.runtime.rpc.RpcService;
-import org.apache.flink.runtime.rpc.RpcServiceUtils;
+import org.apache.flink.runtime.rpc.akka.AkkaRpcServiceUtils;
 import org.apache.flink.runtime.taskexecutor.exceptions.CheckpointException;
 import org.apache.flink.runtime.taskexecutor.exceptions.PartitionException;
 import org.apache.flink.runtime.taskexecutor.exceptions.SlotAllocationException;
@@ -85,7 +85,6 @@ import org.apache.flink.runtime.taskexecutor.slot.TaskSlotTable;
 import org.apache.flink.runtime.taskmanager.CheckpointResponder;
 import org.apache.flink.runtime.taskmanager.Task;
 import org.apache.flink.runtime.taskmanager.TaskExecutionState;
-import org.apache.flink.runtime.taskmanager.TaskManager;
 import org.apache.flink.runtime.taskmanager.TaskManagerActions;
 import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
 import org.apache.flink.util.ExceptionUtils;
@@ -111,6 +110,8 @@ import static org.apache.flink.util.Preconditions.checkNotNull;
  */
 public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 
+	public static final String TASK_MANAGER_NAME = "taskmanager";
+
 	/** The connection information of this task manager */
 	private final TaskManagerLocation taskManagerLocation;
 
@@ -186,7 +187,7 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 			JobLeaderService jobLeaderService,
 			FatalErrorHandler fatalErrorHandler) {
 
-		super(rpcService, RpcServiceUtils.createRandomName(TaskManager.TASK_MANAGER_NAME()));
+		super(rpcService, AkkaRpcServiceUtils.createRandomName(TaskExecutor.TASK_MANAGER_NAME));
 
 		checkArgument(taskManagerConfiguration.getNumberSlots() > 0, "The number of slots has to be larger than 0.");
 

http://git-wip-us.apache.org/repos/asf/flink/blob/a0bb99c7/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java
index 2be8ff1..2ed1578 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java
@@ -30,7 +30,7 @@ import org.apache.flink.runtime.metrics.MetricRegistry;
 import org.apache.flink.runtime.metrics.groups.TaskManagerMetricGroup;
 import org.apache.flink.runtime.rpc.FatalErrorHandler;
 import org.apache.flink.runtime.rpc.RpcService;
-import org.apache.flink.runtime.rpc.RpcServiceUtils;
+import org.apache.flink.runtime.rpc.akka.AkkaRpcServiceUtils;
 import org.apache.flink.runtime.taskexecutor.utils.TaskExecutorMetricsInitializer;
 import org.apache.flink.runtime.util.LeaderRetrievalUtils;
 
@@ -219,6 +219,6 @@ public class TaskManagerRunner implements FatalErrorHandler {
 				"use 0 to let the system choose port automatically.",
 			ConfigConstants.TASK_MANAGER_IPC_PORT_KEY, rpcPort);
 
-		return RpcServiceUtils.createRpcService(taskManagerHostname, rpcPort, configuration);
+		return AkkaRpcServiceUtils.createRpcService(taskManagerHostname, rpcPort, configuration);
 	}
 }


[06/16] flink git commit: [FLINK-6078] Remove CuratorFramework#close calls from ZooKeeper based HA services

Posted by tr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorErrorITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorErrorITCase.java b/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorErrorITCase.java
index 56d1525..25d9228 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorErrorITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorErrorITCase.java
@@ -30,6 +30,7 @@ import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.TaskManagerOptions;
 import org.apache.flink.runtime.client.JobExecutionException;
 import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
+import org.apache.flink.test.util.TestEnvironment;
 import org.apache.flink.util.TestLogger;
 import org.junit.AfterClass;
 import org.junit.Assert;
@@ -48,39 +49,30 @@ public class AccumulatorErrorITCase extends TestLogger {
 
 	private static LocalFlinkMiniCluster cluster;
 
+	private static ExecutionEnvironment env;
+
 	@BeforeClass
 	public static void startCluster() {
-		try {
-			Configuration config = new Configuration();
-			config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, 2);
-			config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 3);
-			config.setLong(TaskManagerOptions.MANAGED_MEMORY_SIZE, 12L);
-			cluster = new LocalFlinkMiniCluster(config, false);
+		Configuration config = new Configuration();
+		config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, 2);
+		config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 3);
+		config.setLong(TaskManagerOptions.MANAGED_MEMORY_SIZE, 12L);
+		cluster = new LocalFlinkMiniCluster(config, false);
 
-			cluster.start();
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail("Failed to start test cluster: " + e.getMessage());
-		}
+		cluster.start();
+
+		env = new TestEnvironment(cluster, 6, false);
 	}
 
 	@AfterClass
 	public static void shutdownCluster() {
-		try {
-			cluster.shutdown();
-			cluster = null;
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail("Failed to stop test cluster: " + e.getMessage());
-		}
+		cluster.shutdown();
+		cluster = null;
 	}
 
 	@Test
 	public void testFaultyAccumulator() throws Exception {
 
-		ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment("localhost", cluster.getLeaderRPCPort());
 		env.getConfig().disableSysoutLogging();
 
 		// Test Exception forwarding with faulty Accumulator implementation
@@ -93,11 +85,9 @@ public class AccumulatorErrorITCase extends TestLogger {
 		try {
 			env.execute();
 			fail("Should have failed.");
-		} catch (ProgramInvocationException e) {
-			Assert.assertTrue("Exception should be passed:",
-					e.getCause() instanceof JobExecutionException);
+		} catch (JobExecutionException e) {
 			Assert.assertTrue("Root cause should be:",
-					e.getCause().getCause() instanceof CustomException);
+					e.getCause() instanceof CustomException);
 		}
 	}
 
@@ -105,7 +95,6 @@ public class AccumulatorErrorITCase extends TestLogger {
 	@Test
 	public void testInvalidTypeAccumulator() throws Exception {
 
-		ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment("localhost", cluster.getLeaderRPCPort());
 		env.getConfig().disableSysoutLogging();
 
 		// Test Exception forwarding with faulty Accumulator implementation
@@ -119,13 +108,11 @@ public class AccumulatorErrorITCase extends TestLogger {
 		try {
 			env.execute();
 			fail("Should have failed.");
-		} catch (ProgramInvocationException e) {
-			Assert.assertTrue("Exception should be passed:",
-					e.getCause() instanceof JobExecutionException);
+		} catch (JobExecutionException e) {
 			Assert.assertTrue("Root cause should be:",
-					e.getCause().getCause() instanceof Exception);
+					e.getCause() instanceof Exception);
 			Assert.assertTrue("Root cause should be:",
-					e.getCause().getCause().getCause() instanceof UnsupportedOperationException);
+					e.getCause().getCause() instanceof UnsupportedOperationException);
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/AbstractEventTimeWindowCheckpointingITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/AbstractEventTimeWindowCheckpointingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/AbstractEventTimeWindowCheckpointingITCase.java
index e8ceeba..d91c57f 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/AbstractEventTimeWindowCheckpointingITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/AbstractEventTimeWindowCheckpointingITCase.java
@@ -36,13 +36,13 @@ import org.apache.flink.runtime.state.filesystem.FsStateBackend;
 import org.apache.flink.runtime.state.memory.MemoryStateBackend;
 import org.apache.flink.streaming.api.TimeCharacteristic;
 import org.apache.flink.streaming.api.checkpoint.ListCheckpointed;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
 import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
 import org.apache.flink.streaming.api.functions.windowing.RichWindowFunction;
 import org.apache.flink.streaming.api.watermark.Watermark;
 import org.apache.flink.streaming.api.windowing.time.Time;
 import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
+import org.apache.flink.streaming.util.TestStreamEnvironment;
 import org.apache.flink.test.util.SuccessException;
 import org.apache.flink.util.Collector;
 import org.apache.flink.util.TestLogger;
@@ -81,6 +81,8 @@ public abstract class AbstractEventTimeWindowCheckpointingITCase extends TestLog
 
 	private static LocalFlinkMiniCluster cluster;
 
+	private static TestStreamEnvironment env;
+
 	@Rule
 	public TemporaryFolder tempFolder = new TemporaryFolder();
 
@@ -104,6 +106,8 @@ public abstract class AbstractEventTimeWindowCheckpointingITCase extends TestLog
 
 		cluster = new LocalFlinkMiniCluster(config, false);
 		cluster.start();
+
+		env = new TestStreamEnvironment(cluster, PARALLELISM);
 	}
 
 	@AfterClass
@@ -153,9 +157,6 @@ public abstract class AbstractEventTimeWindowCheckpointingITCase extends TestLog
 		FailingSource.reset();
 		
 		try {
-			StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment(
-					"localhost", cluster.getLeaderRPCPort());
-			
 			env.setParallelism(PARALLELISM);
 			env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
 			env.enableCheckpointing(100);
@@ -226,9 +227,6 @@ public abstract class AbstractEventTimeWindowCheckpointingITCase extends TestLog
 		FailingSource.reset();
 
 		try {
-			StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment(
-					"localhost", cluster.getLeaderRPCPort());
-
 			env.setParallelism(PARALLELISM);
 			env.setMaxParallelism(maxParallelism);
 			env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
@@ -296,9 +294,6 @@ public abstract class AbstractEventTimeWindowCheckpointingITCase extends TestLog
 		FailingSource.reset();
 
 		try {
-			StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment(
-					"localhost", cluster.getLeaderRPCPort());
-
 			env.setMaxParallelism(2 * PARALLELISM);
 			env.setParallelism(PARALLELISM);
 			env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
@@ -361,9 +356,6 @@ public abstract class AbstractEventTimeWindowCheckpointingITCase extends TestLog
 		FailingSource.reset();
 
 		try {
-			StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment(
-					"localhost", cluster.getLeaderRPCPort());
-
 			env.setParallelism(PARALLELISM);
 			env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
 			env.enableCheckpointing(100);
@@ -434,9 +426,6 @@ public abstract class AbstractEventTimeWindowCheckpointingITCase extends TestLog
 		FailingSource.reset();
 
 		try {
-			StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment(
-					"localhost", cluster.getLeaderRPCPort());
-
 			env.setParallelism(PARALLELISM);
 			env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
 			env.enableCheckpointing(100);

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/EventTimeAllWindowCheckpointingITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/EventTimeAllWindowCheckpointingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/EventTimeAllWindowCheckpointingITCase.java
index 3345b9c..a573be6 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/EventTimeAllWindowCheckpointingITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/EventTimeAllWindowCheckpointingITCase.java
@@ -30,13 +30,13 @@ import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
 import org.apache.flink.streaming.api.TimeCharacteristic;
 import org.apache.flink.runtime.state.CheckpointListener;
 import org.apache.flink.streaming.api.checkpoint.ListCheckpointed;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
 import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
 import org.apache.flink.streaming.api.functions.windowing.RichAllWindowFunction;
 import org.apache.flink.streaming.api.watermark.Watermark;
 import org.apache.flink.streaming.api.windowing.time.Time;
 import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
+import org.apache.flink.streaming.util.TestStreamEnvironment;
 import org.apache.flink.test.util.SuccessException;
 import org.apache.flink.util.Collector;
 import org.apache.flink.util.TestLogger;
@@ -65,6 +65,8 @@ public class EventTimeAllWindowCheckpointingITCase extends TestLogger {
 
 	private static LocalFlinkMiniCluster cluster;
 
+	private static TestStreamEnvironment env;
+
 
 	@BeforeClass
 	public static void startTestCluster() {
@@ -76,6 +78,8 @@ public class EventTimeAllWindowCheckpointingITCase extends TestLogger {
 		config.setString(ConfigConstants.DEFAULT_AKKA_ASK_TIMEOUT, "60 s");
 		cluster = new LocalFlinkMiniCluster(config, false);
 		cluster.start();
+
+		env = new TestStreamEnvironment(cluster, PARALLELISM);
 	}
 
 	@AfterClass
@@ -95,9 +99,6 @@ public class EventTimeAllWindowCheckpointingITCase extends TestLogger {
 		FailingSource.reset();
 		
 		try {
-			StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment(
-					"localhost", cluster.getLeaderRPCPort());
-			
 			env.setParallelism(PARALLELISM);
 			env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
 			env.enableCheckpointing(100);
@@ -159,9 +160,6 @@ public class EventTimeAllWindowCheckpointingITCase extends TestLogger {
 		FailingSource.reset();
 
 		try {
-			StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment(
-					"localhost", cluster.getLeaderRPCPort());
-
 			env.setParallelism(PARALLELISM);
 			env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
 			env.enableCheckpointing(100);
@@ -220,9 +218,6 @@ public class EventTimeAllWindowCheckpointingITCase extends TestLogger {
 		FailingSource.reset();
 
 		try {
-			StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment(
-					"localhost", cluster.getLeaderRPCPort());
-
 			env.setParallelism(PARALLELISM);
 			env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
 			env.enableCheckpointing(100);
@@ -292,9 +287,6 @@ public class EventTimeAllWindowCheckpointingITCase extends TestLogger {
 		FailingSource.reset();
 
 		try {
-			StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment(
-					"localhost", cluster.getLeaderRPCPort());
-
 			env.setParallelism(PARALLELISM);
 			env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
 			env.enableCheckpointing(100);
@@ -364,9 +356,6 @@ public class EventTimeAllWindowCheckpointingITCase extends TestLogger {
 		FailingSource.reset();
 
 		try {
-			StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment(
-					"localhost", cluster.getLeaderRPCPort());
-
 			env.setParallelism(PARALLELISM);
 			env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
 			env.enableCheckpointing(100);

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/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 2839bc1..5f56def 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
@@ -24,6 +24,7 @@ import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.TaskManagerOptions;
 import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.util.TestStreamEnvironment;
 import org.apache.flink.test.util.TestUtils;
 import org.apache.flink.util.TestLogger;
 
@@ -95,8 +96,7 @@ public abstract class StreamFaultToleranceTestBase extends TestLogger {
 	@Test
 	public void runCheckpointedProgram() throws Exception {
 		try {
-			StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment(
-					"localhost", cluster.getLeaderRPCPort());
+			TestStreamEnvironment env = new TestStreamEnvironment(cluster, PARALLELISM);
 			env.setParallelism(PARALLELISM);
 			env.enableCheckpointing(500);
 			env.getConfig().disableSysoutLogging();

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/WindowCheckpointingITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/WindowCheckpointingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/WindowCheckpointingITCase.java
index 56d8c66..7004f75 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/WindowCheckpointingITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/WindowCheckpointingITCase.java
@@ -30,12 +30,12 @@ import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
 import org.apache.flink.streaming.api.TimeCharacteristic;
 import org.apache.flink.runtime.state.CheckpointListener;
 import org.apache.flink.streaming.api.checkpoint.ListCheckpointed;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
 import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
 import org.apache.flink.streaming.api.functions.windowing.RichWindowFunction;
 import org.apache.flink.streaming.api.windowing.time.Time;
 import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
+import org.apache.flink.streaming.util.TestStreamEnvironment;
 import org.apache.flink.test.util.SuccessException;
 import org.apache.flink.util.Collector;
 import org.apache.flink.util.TestLogger;
@@ -76,6 +76,8 @@ public class WindowCheckpointingITCase extends TestLogger {
 
 	private static LocalFlinkMiniCluster cluster;
 
+	private static TestStreamEnvironment env;
+
 
 	@BeforeClass
 	public static void startTestCluster() {
@@ -86,6 +88,8 @@ public class WindowCheckpointingITCase extends TestLogger {
 
 		cluster = new LocalFlinkMiniCluster(config, false);
 		cluster.start();
+
+		env = new TestStreamEnvironment(cluster, PARALLELISM);
 	}
 
 	@AfterClass
@@ -103,9 +107,6 @@ public class WindowCheckpointingITCase extends TestLogger {
 		FailingSource.reset();
 		
 		try {
-			StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment(
-					"localhost", cluster.getLeaderRPCPort());
-			
 			env.setParallelism(PARALLELISM);
 			env.setStreamTimeCharacteristic(timeCharacteristic);
 			env.getConfig().setAutoWatermarkInterval(10);
@@ -161,9 +162,6 @@ public class WindowCheckpointingITCase extends TestLogger {
 		FailingSource.reset();
 
 		try {
-			StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment(
-					"localhost", cluster.getLeaderRPCPort());
-			
 			env.setParallelism(PARALLELISM);
 			env.setStreamTimeCharacteristic(timeCharacteristic);
 			env.getConfig().setAutoWatermarkInterval(10);
@@ -219,9 +217,6 @@ public class WindowCheckpointingITCase extends TestLogger {
 		FailingSource.reset();
 
 		try {
-			StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment(
-					"localhost", cluster.getLeaderRPCPort());
-
 			env.setParallelism(PARALLELISM);
 			env.setStreamTimeCharacteristic(timeCharacteristic);
 			env.getConfig().setAutoWatermarkInterval(10);
@@ -267,9 +262,6 @@ public class WindowCheckpointingITCase extends TestLogger {
 		FailingSource.reset();
 
 		try {
-			StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment(
-					"localhost", cluster.getLeaderRPCPort());
-
 			env.setParallelism(PARALLELISM);
 			env.setStreamTimeCharacteristic(timeCharacteristic);
 			env.getConfig().setAutoWatermarkInterval(10);

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/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 f25a302..75eb112 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
@@ -38,8 +38,10 @@ import org.apache.flink.runtime.messages.JobManagerMessages.TriggerSavepointSucc
 import org.apache.flink.runtime.state.filesystem.FsStateBackendFactory;
 import org.apache.flink.runtime.testingUtils.TestingCluster;
 import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages.WaitForAllVerticesToBeRunning;
+import org.apache.flink.streaming.util.TestStreamEnvironment;
 import org.apache.flink.test.testdata.KMeansData;
 import org.apache.flink.test.util.SuccessException;
+import org.apache.flink.test.util.TestEnvironment;
 import org.apache.flink.util.TestLogger;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -54,13 +56,14 @@ import scala.concurrent.duration.Deadline;
 import scala.concurrent.duration.FiniteDuration;
 
 import java.io.File;
+import java.io.IOException;
+import java.net.URL;
 import java.util.Collections;
 import java.util.List;
 import java.util.concurrent.TimeUnit;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.fail;
 
 public class ClassLoaderITCase extends TestLogger {
 
@@ -117,61 +120,69 @@ public class ClassLoaderITCase extends TestLogger {
 		}
 
 		FOLDER.delete();
+
+		TestStreamEnvironment.unsetAsContext();
+		TestEnvironment.unsetAsContext();
 	}
 
 	@Test
-	public void testJobsWithCustomClassLoader() {
+	public void testJobsWithCustomClassLoader() throws IOException, ProgramInvocationException {
 		try {
 			int port = testCluster.getLeaderRPCPort();
 
-			PackagedProgram inputSplitTestProg = new PackagedProgram(
-					new File(INPUT_SPLITS_PROG_JAR_FILE),
-					new String[] { INPUT_SPLITS_PROG_JAR_FILE,
-							"", // classpath
-							"localhost",
-							String.valueOf(port),
-							"4" // parallelism
-					});
+			PackagedProgram inputSplitTestProg = new PackagedProgram(new File(INPUT_SPLITS_PROG_JAR_FILE));
+
+			TestEnvironment.setAsContext(
+				testCluster,
+				parallelism,
+				Collections.singleton(new Path(INPUT_SPLITS_PROG_JAR_FILE)),
+				Collections.<URL>emptyList());
+
 			inputSplitTestProg.invokeInteractiveModeForExecution();
 
-			PackagedProgram streamingInputSplitTestProg = new PackagedProgram(
-					new File(STREAMING_INPUT_SPLITS_PROG_JAR_FILE),
-					new String[] { STREAMING_INPUT_SPLITS_PROG_JAR_FILE,
-							"localhost",
-							String.valueOf(port),
-							"4" // parallelism
-					});
+			PackagedProgram streamingInputSplitTestProg = new PackagedProgram(new File(STREAMING_INPUT_SPLITS_PROG_JAR_FILE));
+
+			TestStreamEnvironment.setAsContext(
+				testCluster,
+				parallelism,
+				Collections.singleton(new Path(STREAMING_INPUT_SPLITS_PROG_JAR_FILE)),
+				Collections.<URL>emptyList());
+
 			streamingInputSplitTestProg.invokeInteractiveModeForExecution();
 
-			String classpath = new File(INPUT_SPLITS_PROG_JAR_FILE).toURI().toURL().toString();
-			PackagedProgram inputSplitTestProg2 = new PackagedProgram(new File(INPUT_SPLITS_PROG_JAR_FILE),
-					new String[] { "",
-							classpath, // classpath
-							"localhost",
-							String.valueOf(port),
-							"4" // parallelism
-					});
+			URL classpath = new File(INPUT_SPLITS_PROG_JAR_FILE).toURI().toURL();
+			PackagedProgram inputSplitTestProg2 = new PackagedProgram(new File(INPUT_SPLITS_PROG_JAR_FILE));
+
+			TestEnvironment.setAsContext(
+				testCluster,
+				parallelism,
+				Collections.<Path>emptyList(),
+				Collections.singleton(classpath));
+
 			inputSplitTestProg2.invokeInteractiveModeForExecution();
 
 			// regular streaming job
-			PackagedProgram streamingProg = new PackagedProgram(
-					new File(STREAMING_PROG_JAR_FILE),
-					new String[] {
-							STREAMING_PROG_JAR_FILE,
-							"localhost",
-							String.valueOf(port)
-					});
+			PackagedProgram streamingProg = new PackagedProgram(new File(STREAMING_PROG_JAR_FILE));
+
+			TestStreamEnvironment.setAsContext(
+				testCluster,
+				parallelism,
+				Collections.singleton(new Path(STREAMING_PROG_JAR_FILE)),
+				Collections.<URL>emptyList());
+
 			streamingProg.invokeInteractiveModeForExecution();
 
 			// checkpointed streaming job with custom classes for the checkpoint (FLINK-2543)
 			// the test also ensures that user specific exceptions are serializable between JobManager <--> JobClient.
 			try {
-				PackagedProgram streamingCheckpointedProg = new PackagedProgram(
-						new File(STREAMING_CHECKPOINTED_PROG_JAR_FILE),
-						new String[] {
-								STREAMING_CHECKPOINTED_PROG_JAR_FILE,
-								"localhost",
-								String.valueOf(port) });
+				PackagedProgram streamingCheckpointedProg = new PackagedProgram(new File(STREAMING_CHECKPOINTED_PROG_JAR_FILE));
+
+				TestStreamEnvironment.setAsContext(
+					testCluster,
+					parallelism,
+					Collections.singleton(new Path(STREAMING_CHECKPOINTED_PROG_JAR_FILE)),
+					Collections.<URL>emptyList());
+
 				streamingCheckpointedProg.invokeInteractiveModeForExecution();
 			} catch (Exception e) {
 				// we can not access the SuccessException here when executing the tests with maven, because its not available in the jar.
@@ -182,14 +193,18 @@ public class ClassLoaderITCase extends TestLogger {
 
 			PackagedProgram kMeansProg = new PackagedProgram(
 					new File(KMEANS_JAR_PATH),
-					new String[] { KMEANS_JAR_PATH,
-							"localhost",
-							String.valueOf(port),
-							"4", // parallelism
-							KMeansData.DATAPOINTS,
-							KMeansData.INITIAL_CENTERS,
-							"25"
+					new String[] {
+						KMeansData.DATAPOINTS,
+						KMeansData.INITIAL_CENTERS,
+						"25"
 					});
+
+			TestEnvironment.setAsContext(
+				testCluster,
+				parallelism,
+				Collections.singleton(new Path(KMEANS_JAR_PATH)),
+				Collections.<URL>emptyList());
+
 			kMeansProg.invokeInteractiveModeForExecution();
 
 			// test FLINK-3633
@@ -200,6 +215,12 @@ public class ClassLoaderITCase extends TestLogger {
 							String.valueOf(port),
 					});
 
+			TestEnvironment.setAsContext(
+				testCluster,
+				parallelism,
+				Collections.singleton(new Path(USERCODETYPE_JAR_PATH)),
+				Collections.<URL>emptyList());
+
 			userCodeTypeProg.invokeInteractiveModeForExecution();
 
 			File checkpointDir = FOLDER.newFolder();
@@ -208,18 +229,21 @@ public class ClassLoaderITCase extends TestLogger {
 			final PackagedProgram program = new PackagedProgram(
 					new File(CHECKPOINTING_CUSTOM_KV_STATE_JAR_PATH),
 					new String[] {
-							CHECKPOINTING_CUSTOM_KV_STATE_JAR_PATH,
-							"localhost",
-							String.valueOf(port),
 							checkpointDir.toURI().toString(),
 							outputDir.toURI().toString()
 					});
 
+			TestStreamEnvironment.setAsContext(
+				testCluster,
+				parallelism,
+				Collections.singleton(new Path(CHECKPOINTING_CUSTOM_KV_STATE_JAR_PATH)),
+				Collections.<URL>emptyList());
+
 			program.invokeInteractiveModeForExecution();
 
 		} catch (Exception e) {
 			if (!(e.getCause().getCause() instanceof SuccessException)) {
-				fail(e.getMessage());
+				throw e;
 			}
 		}
 	}
@@ -231,23 +255,25 @@ public class ClassLoaderITCase extends TestLogger {
 	public void testDisposeSavepointWithCustomKvState() throws Exception {
 		Deadline deadline = new FiniteDuration(100, TimeUnit.SECONDS).fromNow();
 
-		int port = testCluster.getLeaderRPCPort();
-
 		File checkpointDir = FOLDER.newFolder();
 		File outputDir = FOLDER.newFolder();
 
 		final PackagedProgram program = new PackagedProgram(
 				new File(CUSTOM_KV_STATE_JAR_PATH),
 				new String[] {
-						CUSTOM_KV_STATE_JAR_PATH,
-						"localhost",
-						String.valueOf(port),
 						String.valueOf(parallelism),
 						checkpointDir.toURI().toString(),
 						"5000",
 						outputDir.toURI().toString()
 				});
 
+		TestStreamEnvironment.setAsContext(
+			testCluster,
+			parallelism,
+			Collections.singleton(new Path(CUSTOM_KV_STATE_JAR_PATH)),
+			Collections.<URL>emptyList()
+		);
+
 		// Execute detached
 		Thread invokeThread = new Thread(new Runnable() {
 			@Override
@@ -283,7 +309,7 @@ public class ClassLoaderITCase extends TestLogger {
 
 			// Retry if job is not available yet
 			if (jobId == null) {
-				Thread.sleep(100);
+				Thread.sleep(100L);
 			}
 		}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/CheckpointedStreamingProgram.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/CheckpointedStreamingProgram.java b/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/CheckpointedStreamingProgram.java
index 52a3ba8..795ae41 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/CheckpointedStreamingProgram.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/CheckpointedStreamingProgram.java
@@ -41,14 +41,11 @@ public class CheckpointedStreamingProgram {
 	private static final int CHECKPOINT_INTERVALL = 100;
 	
 	public static void main(String[] args) throws Exception {
-		final String jarFile = args[0];
-		final String host = args[1];
-		final int port = Integer.parseInt(args[2]);
-		
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment(host, port, jarFile);
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+
 		env.getConfig().disableSysoutLogging();
 		env.enableCheckpointing(CHECKPOINT_INTERVALL);
-		env.setRestartStrategy(RestartStrategies.fixedDelayRestart(1, 10000));
+		env.setRestartStrategy(RestartStrategies.fixedDelayRestart(1, 100L));
 		env.disableOperatorChaining();
 		
 		DataStream<String> text = env.addSource(new SimpleStringGenerator());

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/CheckpointingCustomKvStateProgram.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/CheckpointingCustomKvStateProgram.java b/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/CheckpointingCustomKvStateProgram.java
index d3baa7d..a24a3a8 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/CheckpointingCustomKvStateProgram.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/CheckpointingCustomKvStateProgram.java
@@ -49,14 +49,11 @@ import java.util.concurrent.ThreadLocalRandom;
 public class CheckpointingCustomKvStateProgram {
 
 	public static void main(String[] args) throws Exception {
-		final String jarFile = args[0];
-		final String host = args[1];
-		final int port = Integer.parseInt(args[2]);
-		final String checkpointPath = args[3];
-		final String outputPath = args[4];
+		final String checkpointPath = args[0];
+		final String outputPath = args[1];
 		final int parallelism = 1;
 
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment(host, port, jarFile);
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
 
 		env.setParallelism(parallelism);
 		env.getConfig().disableSysoutLogging();

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/CustomInputSplitProgram.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/CustomInputSplitProgram.java b/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/CustomInputSplitProgram.java
index b18e8e8..2caa7cf 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/CustomInputSplitProgram.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/CustomInputSplitProgram.java
@@ -18,7 +18,6 @@
 
 package org.apache.flink.test.classloading.jar;
 
-import java.net.URL;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
@@ -30,7 +29,6 @@ import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.RemoteEnvironment;
 import org.apache.flink.api.java.io.DiscardingOutputFormat;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
@@ -42,14 +40,8 @@ import org.apache.flink.core.io.InputSplitAssigner;
 public class CustomInputSplitProgram {
 	
 	public static void main(String[] args) throws Exception {
-		final String[] jarFile = (args[0].equals(""))? null : new String[] { args[0] };
-		final URL[] classpath = (args[1].equals(""))? null : new URL[] { new URL(args[1]) };
-		final String host = args[2];
-		final int port = Integer.parseInt(args[3]);
-		final int parallelism = Integer.parseInt(args[4]);
-
-		RemoteEnvironment env = new RemoteEnvironment(host, port, null, jarFile, classpath);
-		env.setParallelism(parallelism);
+
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
 		env.getConfig().disableSysoutLogging();
 
 		DataSet<Integer> data = env.createInput(new CustomInputFormat());

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/CustomKvStateProgram.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/CustomKvStateProgram.java b/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/CustomKvStateProgram.java
index 8de4797..cbd553c 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/CustomKvStateProgram.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/CustomKvStateProgram.java
@@ -43,15 +43,12 @@ import java.util.concurrent.ThreadLocalRandom;
 public class CustomKvStateProgram {
 
 	public static void main(String[] args) throws Exception {
-		final String jarFile = args[0];
-		final String host = args[1];
-		final int port = Integer.parseInt(args[2]);
-		final int parallelism = Integer.parseInt(args[3]);
-		final String checkpointPath = args[4];
-		final int checkpointingInterval = Integer.parseInt(args[5]);
-		final String outputPath = args[6];
-
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment(host, port, jarFile);
+		final int parallelism = Integer.parseInt(args[0]);
+		final String checkpointPath = args[1];
+		final int checkpointingInterval = Integer.parseInt(args[2]);
+		final String outputPath = args[3];
+
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
 		env.setParallelism(parallelism);
 		env.getConfig().disableSysoutLogging();
 		env.enableCheckpointing(checkpointingInterval);

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/KMeansForTest.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/KMeansForTest.java b/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/KMeansForTest.java
index 785464a..b8e6c85 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/KMeansForTest.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/KMeansForTest.java
@@ -52,22 +52,15 @@ public class KMeansForTest {
 	// *************************************************************************
 
 	public static void main(String[] args) throws Exception {
-		if (args.length < 7) {
+		if (args.length < 3) {
 			throw new IllegalArgumentException("Missing parameters");
 		}
 
-		final String jarFile = args[0];
-		final String host = args[1];
-		final int port = Integer.parseInt(args[2]);
+		final String pointsData = args[0];
+		final String centersData = args[1];
+		final int numIterations = Integer.parseInt(args[2]);
 
-		final int parallelism = Integer.parseInt(args[3]);
-
-		final String pointsData = args[4];
-		final String centersData = args[5];
-		final int numIterations = Integer.parseInt(args[6]);
-
-		ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment(host, port, jarFile);
-		env.setParallelism(parallelism);
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
 		env.getConfig().disableSysoutLogging();
 
 		// get input data

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/LegacyCheckpointedStreamingProgram.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/LegacyCheckpointedStreamingProgram.java b/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/LegacyCheckpointedStreamingProgram.java
index 60253fa..210973f 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/LegacyCheckpointedStreamingProgram.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/LegacyCheckpointedStreamingProgram.java
@@ -38,11 +38,7 @@ public class LegacyCheckpointedStreamingProgram {
 	private static final int CHECKPOINT_INTERVALL = 100;
 
 	public static void main(String[] args) throws Exception {
-		final String jarFile = args[0];
-		final String host = args[1];
-		final int port = Integer.parseInt(args[2]);
-
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment(host, port, jarFile);
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
 		env.getConfig().disableSysoutLogging();
 		env.enableCheckpointing(CHECKPOINT_INTERVALL);
 		env.setRestartStrategy(RestartStrategies.fixedDelayRestart(1, 10000));

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/StreamingCustomInputSplitProgram.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/StreamingCustomInputSplitProgram.java b/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/StreamingCustomInputSplitProgram.java
index 0f0ee0c..e7bd522 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/StreamingCustomInputSplitProgram.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/StreamingCustomInputSplitProgram.java
@@ -42,18 +42,13 @@ import java.util.List;
 public class StreamingCustomInputSplitProgram {
 	
 	public static void main(String[] args) throws Exception {
-		final String jarFile = args[0];
-		final String host = args[1];
-		final int port = Integer.parseInt(args[2]);
-		final int parallelism = Integer.parseInt(args[3]);
-
-		Configuration config = new Configuration();
+				Configuration config = new Configuration();
 
 		config.setString(ConfigConstants.AKKA_ASK_TIMEOUT, "5 s");
 
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment(host, port, config, jarFile);
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+
 		env.getConfig().disableSysoutLogging();
-		env.setParallelism(parallelism);
 
 		DataStream<Integer> data = env.createInput(new CustomInputFormat());
 

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/StreamingProgram.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/StreamingProgram.java b/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/StreamingProgram.java
index a19d8f2..0fdc744 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/StreamingProgram.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/StreamingProgram.java
@@ -31,12 +31,7 @@ import org.apache.flink.util.Collector;
 public class StreamingProgram {
 	
 	public static void main(String[] args) throws Exception {
-		
-		final String jarFile = args[0];
-		final String host = args[1];
-		final int port = Integer.parseInt(args[2]);
-		
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment(host, port, jarFile);
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
 		env.getConfig().disableSysoutLogging();
 		
 		DataStream<String> text = env.fromElements(WordCountData.TEXT).rebalance();

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/UserCodeType.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/UserCodeType.java b/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/UserCodeType.java
index a073cba..f12fd5f 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/UserCodeType.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/UserCodeType.java
@@ -43,11 +43,7 @@ public class UserCodeType {
 	}
 
 	public static void main(String[] args) throws Exception {
-		String jarFile = args[0];
-		String host = args[1];
-		int port = Integer.parseInt(args[2]);
-
-		ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment(host, port, jarFile);
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
 		env.getConfig().disableSysoutLogging();
 
 		DataSet<Integer> input = env.fromElements(1,2,3,4,5);

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/flink-tests/src/test/java/org/apache/flink/test/clients/examples/JobRetrievalITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/clients/examples/JobRetrievalITCase.java b/flink-tests/src/test/java/org/apache/flink/test/clients/examples/JobRetrievalITCase.java
index a74ed34..61595f2 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/clients/examples/JobRetrievalITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/clients/examples/JobRetrievalITCase.java
@@ -53,7 +53,8 @@ public class JobRetrievalITCase extends TestLogger {
 
 	@BeforeClass
 	public static void before() {
-		cluster = new TestingCluster(new Configuration(), false);
+		Configuration configuration = new Configuration();
+		cluster = new TestingCluster(configuration, false);
 		cluster.start();
 	}
 
@@ -72,7 +73,7 @@ public class JobRetrievalITCase extends TestLogger {
 
 		final JobGraph jobGraph = new JobGraph(jobID, "testjob", imalock);
 
-		final ClusterClient client = new StandaloneClusterClient(cluster.configuration());
+		final ClusterClient client = new StandaloneClusterClient(cluster.configuration(), cluster.highAvailabilityServices());
 
 		// acquire the lock to make sure that the job cannot complete until the job client
 		// has been attached in resumingThread
@@ -122,7 +123,7 @@ public class JobRetrievalITCase extends TestLogger {
 		try {
 			client.retrieveJob(jobID);
 			fail();
-		} catch (JobRetrievalException e) {
+		} catch (JobRetrievalException ignored) {
 			// this is what we want
 		}
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/CustomDistributionITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/CustomDistributionITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/CustomDistributionITCase.java
index 133ebd0..da92c05 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/CustomDistributionITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/CustomDistributionITCase.java
@@ -66,7 +66,7 @@ public class CustomDistributionITCase extends TestLogger {
 
 	@Before
 	public void prepare() {
-		TestEnvironment clusterEnv = new TestEnvironment(cluster, 1);
+		TestEnvironment clusterEnv = new TestEnvironment(cluster, 1, false);
 		clusterEnv.setAsContext();
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/RemoteEnvironmentITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/RemoteEnvironmentITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/RemoteEnvironmentITCase.java
index d9d1b42..0091571 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/RemoteEnvironmentITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/RemoteEnvironmentITCase.java
@@ -26,7 +26,7 @@ import org.apache.flink.api.java.io.LocalCollectionOutputFormat;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.core.io.GenericInputSplit;
-import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
+import org.apache.flink.runtime.minicluster.StandaloneMiniCluster;
 import org.apache.flink.util.Collector;
 import org.apache.flink.util.TestLogger;
 import org.junit.AfterClass;
@@ -39,47 +39,35 @@ import java.util.ArrayList;
 import java.util.List;
 
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
 
 @SuppressWarnings("serial")
 public class RemoteEnvironmentITCase extends TestLogger {
 
 	private static final int TM_SLOTS = 4;
 
-	private static final int NUM_TM = 1;
-
 	private static final int USER_DOP = 2;
 
 	private static final String INVALID_STARTUP_TIMEOUT = "0.001 ms";
 
 	private static final String VALID_STARTUP_TIMEOUT = "100 s";
 
-	private static LocalFlinkMiniCluster cluster;
+	private static Configuration configuration;
+
+	private static StandaloneMiniCluster cluster;
+
 
 	@BeforeClass
-	public static void setupCluster() {
-		try {
-			Configuration config = new Configuration();
-			config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, NUM_TM);
-			config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, TM_SLOTS);
-			cluster = new LocalFlinkMiniCluster(config, false);
-			cluster.start();
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail("Error starting test cluster: " + e.getMessage());
-		}
+	public static void setupCluster() throws Exception {
+		configuration = new Configuration();
+
+		configuration.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, TM_SLOTS);
+
+		cluster = new StandaloneMiniCluster(configuration);
 	}
 
 	@AfterClass
-	public static void tearDownCluster() {
-		try {
-			cluster.stop();
-		}
-		catch (Throwable t) {
-			t.printStackTrace();
-			fail("ClusterClient shutdown caused an exception: " + t.getMessage());
-		}
+	public static void tearDownCluster() throws Exception {
+		cluster.close();
 	}
 
 	/**
@@ -91,8 +79,8 @@ public class RemoteEnvironmentITCase extends TestLogger {
 		config.setString(ConfigConstants.AKKA_STARTUP_TIMEOUT, INVALID_STARTUP_TIMEOUT);
 
 		final ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment(
-				cluster.hostname(),
-				cluster.getLeaderRPCPort(),
+				cluster.getHostname(),
+				cluster.getPort(),
 				config
 		);
 		env.getConfig().disableSysoutLogging();
@@ -116,8 +104,8 @@ public class RemoteEnvironmentITCase extends TestLogger {
 		config.setString(ConfigConstants.AKKA_STARTUP_TIMEOUT, VALID_STARTUP_TIMEOUT);
 
 		final ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment(
-				cluster.hostname(),
-				cluster.getLeaderRPCPort(),
+				cluster.getHostname(),
+				cluster.getPort(),
 				config
 		);
 		env.setParallelism(USER_DOP);

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/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 2910f06..eea2509 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
@@ -30,6 +30,7 @@ import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.core.io.GenericInputSplit;
 import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
+import org.apache.flink.test.util.TestEnvironment;
 import org.apache.flink.util.Collector;
 import org.apache.flink.util.TestLogger;
 import org.junit.AfterClass;
@@ -52,6 +53,8 @@ public class AutoParallelismITCase extends TestLogger {
 
 	private static LocalFlinkMiniCluster cluster;
 
+	private static TestEnvironment env;
+
 	@BeforeClass
 	public static void setupCluster() {
 		Configuration config = new Configuration();
@@ -60,6 +63,8 @@ public class AutoParallelismITCase extends TestLogger {
 		cluster = new LocalFlinkMiniCluster(config, false);
 
 		cluster.start();
+
+		env = new TestEnvironment(cluster, NUM_TM * SLOTS_PER_TM, false);
 	}
 
 	@AfterClass
@@ -78,9 +83,6 @@ public class AutoParallelismITCase extends TestLogger {
 	@Test
 	public void testProgramWithAutoParallelism() {
 		try {
-			ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment(
-					"localhost", cluster.getLeaderRPCPort());
-
 			env.setParallelism(ExecutionConfig.PARALLELISM_AUTO_MAX);
 			env.getConfig().disableSysoutLogging();
 

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/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 b4a7f99..76480ba 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
@@ -19,7 +19,6 @@
 package org.apache.flink.test.misc;
 
 import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.java.ExecutionEnvironment;
 import org.apache.flink.api.java.io.DiscardingOutputFormat;
 import org.apache.flink.client.program.ProgramInvocationException;
 import org.apache.flink.configuration.ConfigConstants;
@@ -27,7 +26,9 @@ import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.TaskManagerOptions;
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
+import org.apache.flink.runtime.client.JobExecutionException;
 import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
+import org.apache.flink.test.util.TestEnvironment;
 import org.apache.flink.types.Value;
 
 import org.apache.flink.util.TestLogger;
@@ -47,39 +48,28 @@ public class CustomSerializationITCase extends TestLogger {
 	
 	private static LocalFlinkMiniCluster cluster;
 
+	private static TestEnvironment env;
+
 	@BeforeClass
 	public static void startCluster() {
-		try {
-			Configuration config = new Configuration();
-			config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, PARLLELISM);
-			config.setLong(TaskManagerOptions.MANAGED_MEMORY_SIZE, 30L);
-			cluster = new LocalFlinkMiniCluster(config, false);
-			cluster.start();
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail("Failed to start test cluster: " + e.getMessage());
-		}
+		Configuration config = new Configuration();
+		config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, PARLLELISM);
+		config.setLong(TaskManagerOptions.MANAGED_MEMORY_SIZE, 30L);
+		cluster = new LocalFlinkMiniCluster(config, false);
+		cluster.start();
+
+		env = new TestEnvironment(cluster, PARLLELISM, false);
 	}
 
 	@AfterClass
 	public static void shutdownCluster() {
-		try {
-			cluster.shutdown();
-			cluster = null;
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail("Failed to stop test cluster: " + e.getMessage());
-		}
+		cluster.shutdown();
+		cluster = null;
 	}
 	
 	@Test
 	public void testIncorrectSerializer1() {
 		try {
-			ExecutionEnvironment env =
-					ExecutionEnvironment.createRemoteEnvironment("localhost", cluster.getLeaderRPCPort());
-			
 			env.setParallelism(PARLLELISM);
 			env.getConfig().disableSysoutLogging();
 			
@@ -96,8 +86,8 @@ public class CustomSerializationITCase extends TestLogger {
 			
 			env.execute();
 		}
-		catch (ProgramInvocationException e) {
-			Throwable rootCause = e.getCause().getCause();
+		catch (JobExecutionException e) {
+			Throwable rootCause = e.getCause();
 			assertTrue(rootCause instanceof IOException);
 			assertTrue(rootCause.getMessage().contains("broken serialization"));
 		}
@@ -110,9 +100,6 @@ public class CustomSerializationITCase extends TestLogger {
 	@Test
 	public void testIncorrectSerializer2() {
 		try {
-			ExecutionEnvironment env =
-					ExecutionEnvironment.createRemoteEnvironment("localhost", cluster.getLeaderRPCPort());
-
 			env.setParallelism(PARLLELISM);
 			env.getConfig().disableSysoutLogging();
 
@@ -129,8 +116,8 @@ public class CustomSerializationITCase extends TestLogger {
 
 			env.execute();
 		}
-		catch (ProgramInvocationException e) {
-			Throwable rootCause = e.getCause().getCause();
+		catch (JobExecutionException e) {
+			Throwable rootCause = e.getCause();
 			assertTrue(rootCause instanceof IOException);
 			assertTrue(rootCause.getMessage().contains("broken serialization"));
 		}
@@ -143,9 +130,6 @@ public class CustomSerializationITCase extends TestLogger {
 	@Test
 	public void testIncorrectSerializer3() {
 		try {
-			ExecutionEnvironment env =
-					ExecutionEnvironment.createRemoteEnvironment("localhost", cluster.getLeaderRPCPort());
-
 			env.setParallelism(PARLLELISM);
 			env.getConfig().disableSysoutLogging();
 
@@ -162,8 +146,8 @@ public class CustomSerializationITCase extends TestLogger {
 
 			env.execute();
 		}
-		catch (ProgramInvocationException e) {
-			Throwable rootCause = e.getCause().getCause();
+		catch (JobExecutionException e) {
+			Throwable rootCause = e.getCause();
 			assertTrue(rootCause instanceof IOException);
 			assertTrue(rootCause.getMessage().contains("broken serialization"));
 		}
@@ -176,9 +160,6 @@ public class CustomSerializationITCase extends TestLogger {
 	@Test
 	public void testIncorrectSerializer4() {
 		try {
-			ExecutionEnvironment env =
-					ExecutionEnvironment.createRemoteEnvironment("localhost", cluster.getLeaderRPCPort());
-
 			env.setParallelism(PARLLELISM);
 			env.getConfig().disableSysoutLogging();
 

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/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 f885321..7dab0f1 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
@@ -23,15 +23,15 @@ import org.apache.flink.api.common.accumulators.LongCounter;
 import org.apache.flink.api.common.functions.MapFunction;
 import org.apache.flink.api.common.functions.RichFlatMapFunction;
 import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
 import org.apache.flink.api.java.io.DiscardingOutputFormat;
-import org.apache.flink.client.program.ProgramInvocationException;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.TaskManagerOptions;
 import org.apache.flink.core.fs.FileSystem;
+import org.apache.flink.runtime.client.JobExecutionException;
 import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
 
+import org.apache.flink.test.util.TestEnvironment;
 import org.apache.flink.util.Collector;
 
 import org.apache.flink.util.TestLogger;
@@ -54,43 +54,34 @@ import static org.junit.Assert.*;
 @SuppressWarnings("serial")
 public class MiscellaneousIssuesITCase extends TestLogger {
 
+	private static final int PARALLELISM = 6;
+
 	private static LocalFlinkMiniCluster cluster;
+
+	private static TestEnvironment env;
 	
 	@BeforeClass
 	public static void startCluster() {
-		try {
-			Configuration config = new Configuration();
-			config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, 2);
-			config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 3);
-			config.setLong(TaskManagerOptions.MANAGED_MEMORY_SIZE, 12L);
-			cluster = new LocalFlinkMiniCluster(config, false);
+		Configuration config = new Configuration();
+		config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, 2);
+		config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 3);
+		config.setLong(TaskManagerOptions.MANAGED_MEMORY_SIZE, 12L);
+		cluster = new LocalFlinkMiniCluster(config, false);
 
-			cluster.start();
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail("Failed to start test cluster: " + e.getMessage());
-		}
+		cluster.start();
+
+		env = new TestEnvironment(cluster, PARALLELISM, false);
 	}
 	
 	@AfterClass
 	public static void shutdownCluster() {
-		try {
-			cluster.shutdown();
-			cluster = null;
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail("Failed to stop test cluster: " + e.getMessage());
-		}
+		cluster.shutdown();
+		cluster = null;
 	}
 	
 	@Test
 	public void testNullValues() {
 		try {
-			ExecutionEnvironment env =
-					ExecutionEnvironment.createRemoteEnvironment("localhost", cluster.getLeaderRPCPort());
-
 			env.setParallelism(1);
 			env.getConfig().disableSysoutLogging();
 
@@ -107,10 +98,9 @@ public class MiscellaneousIssuesITCase extends TestLogger {
 				env.execute();
 				fail("this should fail due to null values.");
 			}
-			catch (ProgramInvocationException e) {
+			catch (JobExecutionException e) {
 				assertNotNull(e.getCause());
-				assertNotNull(e.getCause().getCause());
-				assertTrue(e.getCause().getCause() instanceof NullPointerException);
+				assertTrue(e.getCause() instanceof NullPointerException);
 			}
 		}
 		catch (Exception e) {
@@ -122,9 +112,6 @@ public class MiscellaneousIssuesITCase extends TestLogger {
 	@Test
 	public void testDisjointDataflows() {
 		try {
-			ExecutionEnvironment env =
-					ExecutionEnvironment.createRemoteEnvironment("localhost", cluster.getLeaderRPCPort());
-
 			env.setParallelism(5);
 			env.getConfig().disableSysoutLogging();
 
@@ -145,9 +132,6 @@ public class MiscellaneousIssuesITCase extends TestLogger {
 		final String ACC_NAME = "test_accumulator";
 		
 		try {
-			ExecutionEnvironment env =
-					ExecutionEnvironment.createRemoteEnvironment("localhost", cluster.getLeaderRPCPort());
-
 			env.setParallelism(6);
 			env.getConfig().disableSysoutLogging();
 			

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/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 4f24452..a5103cc 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
@@ -25,7 +25,6 @@ import org.apache.flink.api.java.io.DiscardingOutputFormat;
 import org.apache.flink.api.java.operators.DeltaIteration;
 import org.apache.flink.api.java.operators.IterativeDataSet;
 import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.client.program.ProgramInvocationException;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.TaskManagerOptions;
@@ -34,14 +33,17 @@ import org.apache.flink.examples.java.clustering.util.KMeansData;
 import org.apache.flink.examples.java.graph.ConnectedComponents;
 import org.apache.flink.examples.java.graph.util.ConnectedComponentsData;
 
+import org.apache.flink.runtime.client.JobExecutionException;
 import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
+import org.apache.flink.test.util.TestEnvironment;
 import org.apache.flink.util.TestLogger;
 import org.junit.Test;
 
 import static org.junit.Assert.*;
 
 public class SuccessAfterNetworkBuffersFailureITCase extends TestLogger {
-	
+
+	private static final int PARALLELISM = 16;
 	@Test
 	public void testSuccessfulProgramAfterFailure() {
 		LocalFlinkMiniCluster cluster = null;
@@ -56,9 +58,11 @@ public class SuccessAfterNetworkBuffersFailureITCase extends TestLogger {
 			cluster = new LocalFlinkMiniCluster(config, false);
 
 			cluster.start();
+
+			TestEnvironment env = new TestEnvironment(cluster, PARALLELISM, false);
 			
 			try {
-				runConnectedComponents(cluster.getLeaderRPCPort());
+				runConnectedComponents(env);
 			}
 			catch (Exception e) {
 				e.printStackTrace();
@@ -66,15 +70,15 @@ public class SuccessAfterNetworkBuffersFailureITCase extends TestLogger {
 			}
 	
 			try {
-				runKMeans(cluster.getLeaderRPCPort());
+				runKMeans(env);
 				fail("This program execution should have failed.");
 			}
-			catch (ProgramInvocationException e) {
-				assertTrue(e.getCause().getCause().getMessage().contains("Insufficient number of network buffers"));
+			catch (JobExecutionException e) {
+				assertTrue(e.getCause().getMessage().contains("Insufficient number of network buffers"));
 			}
 	
 			try {
-				runConnectedComponents(cluster.getLeaderRPCPort());
+				runConnectedComponents(env);
 			}
 			catch (Exception e) {
 				e.printStackTrace();
@@ -92,10 +96,9 @@ public class SuccessAfterNetworkBuffersFailureITCase extends TestLogger {
 		}
 	}
 	
-	private static void runConnectedComponents(int jmPort) throws Exception {
+	private static void runConnectedComponents(ExecutionEnvironment env) throws Exception {
 		
-		ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment("localhost", jmPort);
-		env.setParallelism(16);
+		env.setParallelism(PARALLELISM);
 		env.getConfig().disableSysoutLogging();
 
 		// read vertex and edge data
@@ -134,10 +137,9 @@ public class SuccessAfterNetworkBuffersFailureITCase extends TestLogger {
 		env.execute();
 	}
 
-	private static void runKMeans(int jmPort) throws Exception {
+	private static void runKMeans(ExecutionEnvironment env) throws Exception {
 
-		ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment("localhost", jmPort);
-		env.setParallelism(16);
+		env.setParallelism(PARALLELISM);
 		env.getConfig().disableSysoutLogging();
 
 		// get input data

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/flink-tests/src/test/java/org/apache/flink/test/query/AbstractQueryableStateITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/query/AbstractQueryableStateITCase.java b/flink-tests/src/test/java/org/apache/flink/test/query/AbstractQueryableStateITCase.java
index 3c8eb48..6c8e758 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/query/AbstractQueryableStateITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/query/AbstractQueryableStateITCase.java
@@ -170,7 +170,9 @@ public abstract class AbstractQueryableStateITCase extends TestLogger {
 		final Deadline deadline = TEST_TIMEOUT.fromNow();
 		final int numKeys = 256;
 
-		final QueryableStateClient client = new QueryableStateClient(cluster.configuration());
+		final QueryableStateClient client = new QueryableStateClient(
+			cluster.configuration(),
+			cluster.highAvailabilityServices());
 
 		JobID jobId = null;
 
@@ -396,7 +398,9 @@ public abstract class AbstractQueryableStateITCase extends TestLogger {
 
 		final int numElements = 1024;
 
-		final QueryableStateClient client = new QueryableStateClient(cluster.configuration());
+		final QueryableStateClient client = new QueryableStateClient(
+			cluster.configuration(),
+			cluster.highAvailabilityServices());
 
 		JobID jobId = null;
 		try {
@@ -461,7 +465,9 @@ public abstract class AbstractQueryableStateITCase extends TestLogger {
 
 		final int numElements = 1024;
 
-		final QueryableStateClient client = new QueryableStateClient(cluster.configuration());
+		final QueryableStateClient client = new QueryableStateClient(
+			cluster.configuration(),
+			cluster.highAvailabilityServices());
 
 		JobID jobId = null;
 		try {
@@ -586,7 +592,9 @@ public abstract class AbstractQueryableStateITCase extends TestLogger {
 
 		final int numElements = 1024;
 
-		final QueryableStateClient client = new QueryableStateClient(cluster.configuration());
+		final QueryableStateClient client = new QueryableStateClient(
+			cluster.configuration(),
+			cluster.highAvailabilityServices());
 
 		JobID jobId = null;
 		try {
@@ -679,7 +687,9 @@ public abstract class AbstractQueryableStateITCase extends TestLogger {
 
 		final int numElements = 1024;
 
-		final QueryableStateClient client = new QueryableStateClient(cluster.configuration());
+		final QueryableStateClient client = new QueryableStateClient(
+			cluster.configuration(),
+			cluster.highAvailabilityServices());
 
 		JobID jobId = null;
 		try {
@@ -743,7 +753,9 @@ public abstract class AbstractQueryableStateITCase extends TestLogger {
 
 		final int numElements = 1024;
 
-		final QueryableStateClient client = new QueryableStateClient(cluster.configuration());
+		final QueryableStateClient client = new QueryableStateClient(
+			cluster.configuration(),
+			cluster.highAvailabilityServices());
 
 		JobID jobId = null;
 		try {
@@ -844,7 +856,9 @@ public abstract class AbstractQueryableStateITCase extends TestLogger {
 
 		final int numElements = 1024;
 
-		final QueryableStateClient client = new QueryableStateClient(cluster.configuration());
+		final QueryableStateClient client = new QueryableStateClient(
+			cluster.configuration(),
+			cluster.highAvailabilityServices());
 
 		JobID jobId = null;
 		try {
@@ -1008,6 +1022,7 @@ public abstract class AbstractQueryableStateITCase extends TestLogger {
 	 */
 	private static class TestKeyRangeSource extends RichParallelSourceFunction<Tuple2<Integer, Long>>
 			implements CheckpointListener {
+		private static final long serialVersionUID = -5744725196953582710L;
 
 		private final static AtomicLong LATEST_CHECKPOINT_ID = new AtomicLong();
 		private final int numKeys;
@@ -1055,6 +1070,8 @@ public abstract class AbstractQueryableStateITCase extends TestLogger {
 	}
 
 	private static class SumFold implements FoldFunction<Tuple2<Integer, Long>, String> {
+		private static final long serialVersionUID = -6249227626701264599L;
+
 		@Override
 		public String fold(String accumulator, Tuple2<Integer, Long> value) throws Exception {
 			long acc = Long.valueOf(accumulator);
@@ -1064,6 +1081,8 @@ public abstract class AbstractQueryableStateITCase extends TestLogger {
 	}
 
 	private static class SumReduce implements ReduceFunction<Tuple2<Integer, Long>> {
+		private static final long serialVersionUID = -8651235077342052336L;
+
 		@Override
 		public Tuple2<Integer, Long> reduce(Tuple2<Integer, Long> value1, Tuple2<Integer, Long> value2) throws Exception {
 			value1.f1 += value2.f1;

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/flink-tests/src/test/java/org/apache/flink/test/recovery/AbstractTaskManagerProcessFailureRecoveryTest.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/AbstractTaskManagerProcessFailureRecoveryTest.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/AbstractTaskManagerProcessFailureRecoveryTest.java
index 27d1aa1..c7c07ce 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/recovery/AbstractTaskManagerProcessFailureRecoveryTest.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/AbstractTaskManagerProcessFailureRecoveryTest.java
@@ -26,9 +26,12 @@ import akka.util.Timeout;
 import org.apache.commons.io.FileUtils;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.JobManagerOptions;
 import org.apache.flink.configuration.TaskManagerOptions;
 import org.apache.flink.runtime.akka.AkkaUtils;
 import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils;
 import org.apache.flink.runtime.jobmanager.JobManager;
 import org.apache.flink.runtime.jobmanager.MemoryArchivist;
 import org.apache.flink.runtime.messages.JobManagerMessages;
@@ -86,13 +89,14 @@ public abstract class AbstractTaskManagerProcessFailureRecoveryTest extends Test
 	protected static final int PARALLELISM = 4;
 
 	@Test
-	public void testTaskManagerProcessFailure() {
+	public void testTaskManagerProcessFailure() throws Exception {
 
 		final StringWriter processOutput1 = new StringWriter();
 		final StringWriter processOutput2 = new StringWriter();
 		final StringWriter processOutput3 = new StringWriter();
 
 		ActorSystem jmActorSystem = null;
+		HighAvailabilityServices highAvailabilityServices = null;
 		Process taskManagerProcess1 = null;
 		Process taskManagerProcess2 = null;
 		Process taskManagerProcess3 = null;
@@ -128,6 +132,13 @@ public abstract class AbstractTaskManagerProcessFailureRecoveryTest extends Test
 			jmConfig.setInteger(ConfigConstants.AKKA_WATCH_THRESHOLD, 9);
 			jmConfig.setString(ConfigConstants.RESTART_STRATEGY_FIXED_DELAY_DELAY, "10 s");
 			jmConfig.setString(ConfigConstants.AKKA_ASK_TIMEOUT, "100 s");
+			jmConfig.setString(JobManagerOptions.ADDRESS, localAddress._1());
+			jmConfig.setInteger(JobManagerOptions.PORT, jobManagerPort);
+
+			highAvailabilityServices = HighAvailabilityServicesUtils.createHighAvailabilityServices(
+				jmConfig,
+				TestingUtils.defaultExecutor(),
+				HighAvailabilityServicesUtils.AddressResolution.NO_ADDRESS_RESOLUTION);
 
 			jmActorSystem = AkkaUtils.createActorSystem(jmConfig, new Some<>(localAddress));
 			ActorRef jmActor = JobManager.startJobManagerActors(
@@ -135,6 +146,7 @@ public abstract class AbstractTaskManagerProcessFailureRecoveryTest extends Test
 				jmActorSystem,
 				TestingUtils.defaultExecutor(),
 				TestingUtils.defaultExecutor(),
+				highAvailabilityServices,
 				JobManager.class,
 				MemoryArchivist.class)._1();
 
@@ -263,6 +275,10 @@ public abstract class AbstractTaskManagerProcessFailureRecoveryTest extends Test
 					// we can ignore this
 				}
 			}
+
+			if (highAvailabilityServices != null) {
+				highAvailabilityServices.closeAndCleanupAllData();
+			}
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/flink-tests/src/test/java/org/apache/flink/test/recovery/ChaosMonkeyITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/ChaosMonkeyITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/ChaosMonkeyITCase.java
index bba218f..6d53b9f 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/recovery/ChaosMonkeyITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/ChaosMonkeyITCase.java
@@ -27,6 +27,8 @@ import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.HighAvailabilityOptions;
 import org.apache.flink.runtime.akka.AkkaUtils;
 import org.apache.flink.runtime.akka.ListeningBehaviour;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils;
 import org.apache.flink.runtime.instance.AkkaActorGateway;
 import org.apache.flink.runtime.jobgraph.JobGraph;
 import org.apache.flink.runtime.jobgraph.JobStatus;
@@ -34,13 +36,13 @@ import org.apache.flink.runtime.leaderelection.TestingListener;
 import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
 import org.apache.flink.runtime.messages.JobManagerMessages;
 import org.apache.flink.runtime.state.filesystem.FsStateBackendFactory;
+import org.apache.flink.runtime.testingUtils.TestingUtils;
 import org.apache.flink.runtime.testutils.CommonTestUtils;
 import org.apache.flink.runtime.testutils.JobManagerActorTestUtils;
 import org.apache.flink.runtime.testutils.JobManagerProcess;
 import org.apache.flink.runtime.testutils.TaskManagerProcess;
 import org.apache.flink.runtime.testutils.TestJvmProcess;
 import org.apache.flink.runtime.testutils.ZooKeeperTestUtils;
-import org.apache.flink.runtime.util.ZooKeeperUtils;
 import org.apache.flink.runtime.zookeeper.ZooKeeperTestEnvironment;
 import org.apache.flink.runtime.state.CheckpointListener;
 import org.apache.flink.streaming.api.checkpoint.ListCheckpointed;
@@ -169,6 +171,11 @@ public class ChaosMonkeyITCase extends TestLogger {
 		// Task manager
 		config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, numberOfSlotsPerTaskManager);
 
+		final HighAvailabilityServices highAvailabilityServices = HighAvailabilityServicesUtils.createHighAvailabilityServices(
+			config,
+			TestingUtils.defaultExecutor(),
+			HighAvailabilityServicesUtils.AddressResolution.NO_ADDRESS_RESOLUTION);
+
 		ActorSystem testActorSystem = null;
 		LeaderRetrievalService leaderRetrievalService = null;
 		List<JobManagerProcess> jobManagerProcesses = new ArrayList<>();
@@ -187,7 +194,7 @@ public class ChaosMonkeyITCase extends TestLogger {
 			testActorSystem = AkkaUtils.createDefaultActorSystem();
 
 			// Leader listener
-			leaderRetrievalService = ZooKeeperUtils.createLeaderRetrievalService(config);
+			leaderRetrievalService = highAvailabilityServices.getJobManagerLeaderRetriever(HighAvailabilityServices.DEFAULT_JOB_ID);
 			TestingListener leaderListener = new TestingListener();
 			leaderRetrievalService.start(leaderListener);
 
@@ -358,6 +365,8 @@ public class ChaosMonkeyITCase extends TestLogger {
 			if (testActorSystem != null) {
 				testActorSystem.shutdown();
 			}
+
+			highAvailabilityServices.closeAndCleanupAllData();
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/flink-tests/src/test/java/org/apache/flink/test/recovery/FastFailuresITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/FastFailuresITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/FastFailuresITCase.java
index d80c826..0b49814 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/recovery/FastFailuresITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/FastFailuresITCase.java
@@ -30,6 +30,7 @@ import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.api.functions.sink.SinkFunction;
 import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
 
+import org.apache.flink.streaming.util.TestStreamEnvironment;
 import org.apache.flink.util.TestLogger;
 import org.junit.Test;
 
@@ -45,18 +46,19 @@ public class FastFailuresITCase extends TestLogger {
 	
 	@Test
 	public void testThis() {
+		final int parallelism = 4;
+
 		Configuration config = new Configuration();
 		config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, 2);
 		config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 2);
 		
 		LocalFlinkMiniCluster cluster = new LocalFlinkMiniCluster(config, false);
 		cluster.start();
-		
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment(
-				"localhost", cluster.getLeaderRPCPort());
 
+		TestStreamEnvironment env = new TestStreamEnvironment(cluster, parallelism);
+		
 		env.getConfig().disableSysoutLogging();
-		env.setParallelism(4);
+		env.setParallelism(parallelism);
 		env.enableCheckpointing(1000);
 		env.getConfig().setRestartStrategy(RestartStrategies.fixedDelayRestart(210, 0));
 		

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHACheckpointRecoveryITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHACheckpointRecoveryITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHACheckpointRecoveryITCase.java
index f910e49..6c70b87 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHACheckpointRecoveryITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHACheckpointRecoveryITCase.java
@@ -26,6 +26,8 @@ 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.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils;
 import org.apache.flink.runtime.instance.ActorGateway;
 import org.apache.flink.runtime.instance.AkkaActorGateway;
 import org.apache.flink.runtime.jobgraph.JobGraph;
@@ -34,12 +36,12 @@ import org.apache.flink.runtime.jobgraph.JobVertex;
 import org.apache.flink.runtime.leaderelection.TestingListener;
 import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
 import org.apache.flink.runtime.taskmanager.TaskManager;
+import org.apache.flink.runtime.testingUtils.TestingUtils;
 import org.apache.flink.runtime.testtasks.BlockingNoOpInvokable;
 import org.apache.flink.runtime.testutils.CommonTestUtils;
 import org.apache.flink.runtime.testutils.JobManagerActorTestUtils;
 import org.apache.flink.runtime.testutils.JobManagerProcess;
 import org.apache.flink.runtime.testutils.ZooKeeperTestUtils;
-import org.apache.flink.runtime.util.ZooKeeperUtils;
 import org.apache.flink.runtime.zookeeper.ZooKeeperTestEnvironment;
 import org.apache.flink.runtime.state.CheckpointListener;
 import org.apache.flink.streaming.api.checkpoint.ListCheckpointed;
@@ -170,6 +172,10 @@ public class JobManagerHACheckpointRecoveryITCase extends TestLogger {
 		final JobManagerProcess[] jobManagerProcess = new JobManagerProcess[2];
 		LeaderRetrievalService leaderRetrievalService = null;
 		ActorSystem taskManagerSystem = null;
+		final HighAvailabilityServices highAvailabilityServices = HighAvailabilityServicesUtils.createHighAvailabilityServices(
+			config,
+			TestingUtils.defaultExecutor(),
+			HighAvailabilityServicesUtils.AddressResolution.NO_ADDRESS_RESOLUTION);
 
 		try {
 			final Deadline deadline = TestTimeOut.fromNow();
@@ -187,16 +193,21 @@ public class JobManagerHACheckpointRecoveryITCase extends TestLogger {
 
 			// Leader listener
 			TestingListener leaderListener = new TestingListener();
-			leaderRetrievalService = ZooKeeperUtils.createLeaderRetrievalService(config);
+			leaderRetrievalService = highAvailabilityServices.getJobManagerLeaderRetriever(HighAvailabilityServices.DEFAULT_JOB_ID);
 			leaderRetrievalService.start(leaderListener);
 
 			// The task manager
 			taskManagerSystem = AkkaUtils.createActorSystem(
 					config, Option.apply(new Tuple2<String, Object>("localhost", 0)));
 			TaskManager.startTaskManagerComponentsAndActor(
-					config, ResourceID.generate(), taskManagerSystem, "localhost",
-					Option.<String>empty(), Option.<LeaderRetrievalService>empty(),
-					false, TaskManager.class);
+				config,
+				ResourceID.generate(),
+				taskManagerSystem,
+				highAvailabilityServices,
+				"localhost",
+				Option.<String>empty(),
+				false,
+				TaskManager.class);
 
 			{
 				// Initial submission
@@ -298,6 +309,8 @@ public class JobManagerHACheckpointRecoveryITCase extends TestLogger {
 			if (testSystem != null) {
 				testSystem.shutdown();
 			}
+
+			highAvailabilityServices.closeAndCleanupAllData();
 		}
 	}
 
@@ -323,6 +336,10 @@ public class JobManagerHACheckpointRecoveryITCase extends TestLogger {
 		LeaderRetrievalService leaderRetrievalService = null;
 		ActorSystem taskManagerSystem = null;
 		ActorSystem testActorSystem = null;
+		final HighAvailabilityServices highAvailabilityServices = HighAvailabilityServicesUtils.createHighAvailabilityServices(
+			config,
+			TestingUtils.defaultExecutor(),
+			HighAvailabilityServicesUtils.AddressResolution.NO_ADDRESS_RESOLUTION);
 
 		try {
 			// Test actor system
@@ -338,16 +355,21 @@ public class JobManagerHACheckpointRecoveryITCase extends TestLogger {
 
 			// Leader listener
 			TestingListener leaderListener = new TestingListener();
-			leaderRetrievalService = ZooKeeperUtils.createLeaderRetrievalService(config);
+			leaderRetrievalService = highAvailabilityServices.getJobManagerLeaderRetriever(HighAvailabilityServices.DEFAULT_JOB_ID);
 			leaderRetrievalService.start(leaderListener);
 
 			// The task manager
 			taskManagerSystem = AkkaUtils.createActorSystem(
 					config, Option.apply(new Tuple2<String, Object>("localhost", 0)));
 			TaskManager.startTaskManagerComponentsAndActor(
-					config, ResourceID.generate(), taskManagerSystem, "localhost",
-					Option.<String>empty(), Option.<LeaderRetrievalService>empty(),
-					false, TaskManager.class);
+				config,
+				ResourceID.generate(),
+				taskManagerSystem,
+				highAvailabilityServices,
+				"localhost",
+				Option.<String>empty(),
+				false,
+				TaskManager.class);
 
 			// Get the leader
 			leaderListener.waitForNewLeader(testDeadline.timeLeft().toMillis());
@@ -453,6 +475,8 @@ public class JobManagerHACheckpointRecoveryITCase extends TestLogger {
 			if (testActorSystem != null) {
 				testActorSystem.shutdown();
 			}
+
+			highAvailabilityServices.closeAndCleanupAllData();
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHAJobGraphRecoveryITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHAJobGraphRecoveryITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHAJobGraphRecoveryITCase.java
index e4d0f65..052195a 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHAJobGraphRecoveryITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHAJobGraphRecoveryITCase.java
@@ -32,6 +32,8 @@ 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.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils;
 import org.apache.flink.runtime.instance.ActorGateway;
 import org.apache.flink.runtime.instance.AkkaActorGateway;
 import org.apache.flink.runtime.jobgraph.JobGraph;
@@ -45,12 +47,12 @@ import org.apache.flink.runtime.messages.JobManagerMessages.LeaderSessionMessage
 import org.apache.flink.runtime.messages.JobManagerMessages.SubmitJob;
 import org.apache.flink.runtime.taskmanager.TaskManager;
 import org.apache.flink.runtime.testingUtils.TestingCluster;
+import org.apache.flink.runtime.testingUtils.TestingUtils;
 import org.apache.flink.runtime.testtasks.BlockingNoOpInvokable;
 import org.apache.flink.runtime.testutils.CommonTestUtils;
 import org.apache.flink.runtime.testutils.JobManagerActorTestUtils;
 import org.apache.flink.runtime.testutils.JobManagerProcess;
 import org.apache.flink.runtime.testutils.ZooKeeperTestUtils;
-import org.apache.flink.runtime.util.ZooKeeperUtils;
 import org.apache.flink.runtime.zookeeper.ZooKeeperTestEnvironment;
 import org.apache.flink.util.TestLogger;
 import org.apache.zookeeper.data.Stat;
@@ -183,6 +185,11 @@ public class JobManagerHAJobGraphRecoveryITCase extends TestLogger {
 
 		ActorSystem taskManagerSystem = null;
 
+		final HighAvailabilityServices highAvailabilityServices = HighAvailabilityServicesUtils.createHighAvailabilityServices(
+			config,
+			TestingUtils.defaultExecutor(),
+			HighAvailabilityServicesUtils.AddressResolution.NO_ADDRESS_RESOLUTION);
+
 		try {
 			final Deadline deadline = TestTimeOut.fromNow();
 
@@ -199,15 +206,20 @@ public class JobManagerHAJobGraphRecoveryITCase extends TestLogger {
 
 			// Leader listener
 			TestingListener leaderListener = new TestingListener();
-			leaderRetrievalService = ZooKeeperUtils.createLeaderRetrievalService(config);
+			leaderRetrievalService = highAvailabilityServices.getJobManagerLeaderRetriever(HighAvailabilityServices.DEFAULT_JOB_ID);
 			leaderRetrievalService.start(leaderListener);
 
 			// The task manager
 			taskManagerSystem = AkkaUtils.createActorSystem(AkkaUtils.getDefaultAkkaConfig());
 			TaskManager.startTaskManagerComponentsAndActor(
-					config, ResourceID.generate(), taskManagerSystem, "localhost",
-					Option.<String>empty(), Option.<LeaderRetrievalService>empty(),
-					false, TaskManager.class);
+				config,
+				ResourceID.generate(),
+				taskManagerSystem,
+				highAvailabilityServices,
+				"localhost",
+				Option.<String>empty(),
+				false,
+				TaskManager.class);
 
 			// Client test actor
 			TestActorRef<RecordingTestClient> clientRef = TestActorRef.create(
@@ -327,6 +339,8 @@ public class JobManagerHAJobGraphRecoveryITCase extends TestLogger {
 			if (testSystem != null) {
 				testSystem.shutdown();
 			}
+
+			highAvailabilityServices.closeAndCleanupAllData();
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHAProcessFailureBatchRecoveryITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHAProcessFailureBatchRecoveryITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHAProcessFailureBatchRecoveryITCase.java
index b6a1bd4..5f9d178 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHAProcessFailureBatchRecoveryITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHAProcessFailureBatchRecoveryITCase.java
@@ -34,16 +34,18 @@ import org.apache.flink.configuration.HighAvailabilityOptions;
 import org.apache.flink.configuration.TaskManagerOptions;
 import org.apache.flink.runtime.akka.AkkaUtils;
 import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils;
 import org.apache.flink.runtime.instance.ActorGateway;
 import org.apache.flink.runtime.instance.AkkaActorGateway;
 import org.apache.flink.runtime.leaderelection.TestingListener;
 import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
 import org.apache.flink.runtime.taskmanager.TaskManager;
+import org.apache.flink.runtime.testingUtils.TestingUtils;
 import org.apache.flink.runtime.testutils.CommonTestUtils;
 import org.apache.flink.runtime.testutils.JobManagerActorTestUtils;
 import org.apache.flink.runtime.testutils.JobManagerProcess;
 import org.apache.flink.runtime.testutils.ZooKeeperTestUtils;
-import org.apache.flink.runtime.util.ZooKeeperUtils;
 import org.apache.flink.runtime.zookeeper.ZooKeeperTestEnvironment;
 import org.apache.flink.util.Collector;
 import org.apache.flink.util.TestLogger;
@@ -238,7 +240,8 @@ public class JobManagerHAProcessFailureBatchRecoveryITCase extends TestLogger {
 		// Task managers
 		final ActorSystem[] tmActorSystem = new ActorSystem[numberOfTaskManagers];
 
-		// Leader election service
+		HighAvailabilityServices highAvailabilityServices = null;
+
 		LeaderRetrievalService leaderRetrievalService = null;
 
 		// Coordination between the processes goes through a directory
@@ -263,13 +266,22 @@ public class JobManagerHAProcessFailureBatchRecoveryITCase extends TestLogger {
 			config.setInteger(TaskManagerOptions.NETWORK_NUM_BUFFERS, 100);
 			config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 2);
 
+			highAvailabilityServices = HighAvailabilityServicesUtils.createAvailableOrEmbeddedServices(
+				config,
+				TestingUtils.defaultExecutor());
+
 			// Start the task manager process
 			for (int i = 0; i < numberOfTaskManagers; i++) {
 				tmActorSystem[i] = AkkaUtils.createActorSystem(AkkaUtils.getDefaultAkkaConfig());
 				TaskManager.startTaskManagerComponentsAndActor(
-						config, ResourceID.generate(), tmActorSystem[i], "localhost",
-						Option.<String>empty(), Option.<LeaderRetrievalService>empty(),
-						false, TaskManager.class);
+					config,
+					ResourceID.generate(),
+					tmActorSystem[i],
+					highAvailabilityServices,
+					"localhost",
+					Option.<String>empty(),
+					false,
+					TaskManager.class);
 			}
 
 			// Test actor system
@@ -279,7 +291,7 @@ public class JobManagerHAProcessFailureBatchRecoveryITCase extends TestLogger {
 
 			// Leader listener
 			TestingListener leaderListener = new TestingListener();
-			leaderRetrievalService = ZooKeeperUtils.createLeaderRetrievalService(config);
+			leaderRetrievalService = highAvailabilityServices.getJobManagerLeaderRetriever(HighAvailabilityServices.DEFAULT_JOB_ID);
 			leaderRetrievalService.start(leaderListener);
 
 			// Initial submission
@@ -378,6 +390,10 @@ public class JobManagerHAProcessFailureBatchRecoveryITCase extends TestLogger {
 				}
 			}
 
+			if (highAvailabilityServices != null) {
+				highAvailabilityServices.closeAndCleanupAllData();
+			}
+
 			// Delete coordination directory
 			if (coordinateTempDir != null) {
 				try {


[12/16] flink git commit: [FLINK-6435] [async] React to exceptionally completed StreamElementQueueEntry

Posted by tr...@apache.org.
[FLINK-6435] [async] React to exceptionally completed StreamElementQueueEntry

The AsyncWaitOperator should not only react to orderly completed
StreamElementQueueEntries but also to those completed with a user exception
or those which timed out.

This PR fixes the problem by calling the onComplete function passed to
StreamElementQueueEntry#onComplete also in the exceptional case.

This closes #3814.


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

Branch: refs/heads/master
Commit: 93758082273618d9fdbb3a9b3ed916a4b637760f
Parents: ddd6a99
Author: Till Rohrmann <tr...@apache.org>
Authored: Wed May 3 14:40:46 2017 +0200
Committer: Till Rohrmann <tr...@apache.org>
Committed: Fri May 5 11:08:09 2017 +0200

----------------------------------------------------------------------
 .../async/queue/StreamElementQueueEntry.java    |   9 +-
 .../operators/async/AsyncWaitOperatorTest.java  | 196 +++++++++++++++----
 2 files changed, 161 insertions(+), 44 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/93758082/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/queue/StreamElementQueueEntry.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/queue/StreamElementQueueEntry.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/queue/StreamElementQueueEntry.java
index 66872df..4a50201 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/queue/StreamElementQueueEntry.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/queue/StreamElementQueueEntry.java
@@ -21,6 +21,7 @@ package org.apache.flink.streaming.api.operators.async.queue;
 import java.util.concurrent.Executor;
 import org.apache.flink.annotation.Internal;
 import org.apache.flink.runtime.concurrent.AcceptFunction;
+import org.apache.flink.runtime.concurrent.BiFunction;
 import org.apache.flink.runtime.concurrent.Future;
 import org.apache.flink.streaming.runtime.streamrecord.StreamElement;
 import org.apache.flink.util.Preconditions;
@@ -65,10 +66,14 @@ public abstract class StreamElementQueueEntry<T> implements AsyncResult {
 			Executor executor) {
 		final StreamElementQueueEntry<T> thisReference = this;
 
-		getFuture().thenAcceptAsync(new AcceptFunction<T>() {
+		getFuture().handleAsync(new BiFunction<T, Throwable, Void>() {
 			@Override
-			public void accept(T value) {
+			public Void apply(T t, Throwable throwable) {
+				// call the complete function for normal completion as well as exceptional completion
+				// see FLINK-6435
 				completeFunction.accept(thisReference);
+
+				return null;
 			}
 		}, executor);
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/93758082/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperatorTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperatorTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperatorTest.java
index c4867ff..1d83229 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperatorTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperatorTest.java
@@ -71,6 +71,7 @@ import org.mockito.ArgumentCaptor;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 
+import javax.annotation.Nonnull;
 import java.util.ArrayDeque;
 import java.util.Collections;
 import java.util.Comparator;
@@ -90,13 +91,7 @@ import static org.junit.Assert.assertTrue;
 import static org.mockito.Matchers.any;
 import static org.mockito.Matchers.anyLong;
 import static org.mockito.Matchers.eq;
-import static org.mockito.Mockito.doAnswer;
-import static org.mockito.Mockito.doNothing;
-import static org.mockito.Mockito.doReturn;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.never;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
+import static org.mockito.Mockito.*;
 
 /**
  * Tests for {@link AsyncWaitOperator}. These test that:
@@ -173,7 +168,7 @@ public class AsyncWaitOperatorTest extends TestLogger {
 	}
 
 	/**
-	 * A special {@link org.apache.flink.streaming.api.functions.async.AsyncFunction} without issuing
+	 * A special {@link AsyncFunction} without issuing
 	 * {@link AsyncCollector#collect} until the latch counts to zero.
 	 * This function is used in the testStateSnapshotAndRestore, ensuring
 	 * that {@link StreamElementQueueEntry} can stay
@@ -637,20 +632,7 @@ public class AsyncWaitOperatorTest extends TestLogger {
 			2,
 			AsyncDataStream.OutputMode.ORDERED);
 
-		final Environment mockEnvironment = mock(Environment.class);
-
-		final Configuration taskConfiguration = new Configuration();
-		final ExecutionConfig executionConfig = new ExecutionConfig();
-		final TaskMetricGroup metricGroup = new UnregisteredTaskMetricsGroup();
-		final TaskManagerRuntimeInfo taskManagerRuntimeInfo = new TestingTaskManagerRuntimeInfo();
-		final TaskInfo taskInfo = new TaskInfo("foobarTask", 1, 0, 1, 1);
-
-		when(mockEnvironment.getTaskConfiguration()).thenReturn(taskConfiguration);
-		when(mockEnvironment.getExecutionConfig()).thenReturn(executionConfig);
-		when(mockEnvironment.getMetricGroup()).thenReturn(metricGroup);
-		when(mockEnvironment.getTaskManagerInfo()).thenReturn(taskManagerRuntimeInfo);
-		when(mockEnvironment.getTaskInfo()).thenReturn(taskInfo);
-		when(mockEnvironment.getUserClassLoader()).thenReturn(AsyncWaitOperatorTest.class.getClassLoader());
+		final Environment mockEnvironment = createMockEnvironment();
 
 		final OneInputStreamOperatorTestHarness<Integer, Integer> testHarness =
 			new OneInputStreamOperatorTestHarness<>(operator, IntSerializer.INSTANCE, mockEnvironment);
@@ -696,6 +678,25 @@ public class AsyncWaitOperatorTest extends TestLogger {
 		Assert.assertTrue(failureCause.getCause().getCause() instanceof TimeoutException);
 	}
 
+	@Nonnull
+	private Environment createMockEnvironment() {
+		final Environment mockEnvironment = mock(Environment.class);
+
+		final Configuration taskConfiguration = new Configuration();
+		final ExecutionConfig executionConfig = new ExecutionConfig();
+		final TaskMetricGroup metricGroup = new UnregisteredTaskMetricsGroup();
+		final TaskManagerRuntimeInfo taskManagerRuntimeInfo = new TestingTaskManagerRuntimeInfo();
+		final TaskInfo taskInfo = new TaskInfo("foobarTask", 1, 0, 1, 1);
+
+		when(mockEnvironment.getTaskConfiguration()).thenReturn(taskConfiguration);
+		when(mockEnvironment.getExecutionConfig()).thenReturn(executionConfig);
+		when(mockEnvironment.getMetricGroup()).thenReturn(metricGroup);
+		when(mockEnvironment.getTaskManagerInfo()).thenReturn(taskManagerRuntimeInfo);
+		when(mockEnvironment.getTaskInfo()).thenReturn(taskInfo);
+		when(mockEnvironment.getUserClassLoader()).thenReturn(Thread.currentThread().getContextClassLoader());
+		return mockEnvironment;
+	}
+
 	/**
 	 * Test case for FLINK-5638: Tests that the async wait operator can be closed even if the
 	 * emitter is currently waiting on the checkpoint lock (e.g. in the case of two chained async
@@ -710,16 +711,7 @@ public class AsyncWaitOperatorTest extends TestLogger {
 
 		ArgumentCaptor<Throwable> failureReason = ArgumentCaptor.forClass(Throwable.class);
 
-		Environment environment = mock(Environment.class);
-		when(environment.getMetricGroup()).thenReturn(new UnregisteredTaskMetricsGroup());
-		when(environment.getTaskManagerInfo()).thenReturn(new TestingTaskManagerRuntimeInfo());
-		when(environment.getUserClassLoader()).thenReturn(getClass().getClassLoader());
-		when(environment.getTaskInfo()).thenReturn(new TaskInfo(
-			"testTask",
-			1,
-			0,
-			1,
-			0));
+		Environment environment = createMockEnvironment();
 		doNothing().when(environment).failExternally(failureReason.capture());
 
 		StreamTask<?, ?> containingTask = mock(StreamTask.class);
@@ -827,16 +819,7 @@ public class AsyncWaitOperatorTest extends TestLogger {
 		final long timeout = 100000L;
 		final long timestamp = 1L;
 
-		Environment environment = mock(Environment.class);
-		when(environment.getMetricGroup()).thenReturn(new UnregisteredTaskMetricsGroup());
-		when(environment.getTaskManagerInfo()).thenReturn(new TestingTaskManagerRuntimeInfo());
-		when(environment.getUserClassLoader()).thenReturn(getClass().getClassLoader());
-		when(environment.getTaskInfo()).thenReturn(new TaskInfo(
-			"testTask",
-			1,
-			0,
-			1,
-			0));
+		Environment environment = createMockEnvironment();
 
 		ScheduledFuture<?> scheduledFuture = mock(ScheduledFuture.class);
 
@@ -893,4 +876,133 @@ public class AsyncWaitOperatorTest extends TestLogger {
 		// check that we have cancelled our registered timeout
 		verify(scheduledFuture).cancel(eq(true));
 	}
+
+	/**
+	 * FLINK-6435
+	 *
+	 * Tests that a user exception triggers the completion of a StreamElementQueueEntry and does not wait to until
+	 * another StreamElementQueueEntry is properly completed before it is collected.
+	 */
+	@Test(timeout = 2000)
+	public void testOrderedWaitUserExceptionHandling() throws Exception {
+		testUserExceptionHandling(AsyncDataStream.OutputMode.ORDERED);
+	}
+
+	/**
+	 * FLINK-6435
+	 *
+	 * Tests that a user exception triggers the completion of a StreamElementQueueEntry and does not wait to until
+	 * another StreamElementQueueEntry is properly completed before it is collected.
+	 */
+	@Test(timeout = 2000)
+	public void testUnorderedWaitUserExceptionHandling() throws Exception {
+		testUserExceptionHandling(AsyncDataStream.OutputMode.UNORDERED);
+	}
+
+	private void testUserExceptionHandling(AsyncDataStream.OutputMode outputMode) throws Exception {
+		UserExceptionAsyncFunction asyncWaitFunction = new UserExceptionAsyncFunction();
+		long timeout = 2000L;
+
+		AsyncWaitOperator<Integer, Integer> asyncWaitOperator = new AsyncWaitOperator<>(
+			asyncWaitFunction,
+			TIMEOUT,
+			2,
+			outputMode);
+
+		final Environment mockEnvironment = createMockEnvironment();
+
+		OneInputStreamOperatorTestHarness<Integer, Integer> harness = new OneInputStreamOperatorTestHarness<>(
+			asyncWaitOperator,
+			IntSerializer.INSTANCE,
+			mockEnvironment);
+
+		harness.open();
+
+		synchronized (harness.getCheckpointLock()) {
+			harness.processElement(1, 1L);
+		}
+
+		verify(harness.getEnvironment(), timeout(timeout)).failExternally(any(Exception.class));
+
+		synchronized (harness.getCheckpointLock()) {
+			harness.close();
+		}
+	}
+
+	/**
+	 * AsyncFunction which completes the result with an {@link Exception}.
+	 */
+	private static class UserExceptionAsyncFunction implements AsyncFunction<Integer, Integer> {
+
+		private static final long serialVersionUID = 6326568632967110990L;
+
+		@Override
+		public void asyncInvoke(Integer input, AsyncCollector<Integer> collector) throws Exception {
+			collector.collect(new Exception("Test exception"));
+		}
+	}
+
+	/**
+	 * FLINK-6435
+	 *
+	 * Tests that timeout exceptions are properly handled in ordered output mode. The proper handling means that
+	 * a StreamElementQueueEntry is completed in case of a timeout exception.
+	 */
+	@Test
+	public void testOrderedWaitTimeoutHandling() throws Exception {
+		testTimeoutExceptionHandling(AsyncDataStream.OutputMode.ORDERED);
+	}
+
+	/**
+	 * FLINK-6435
+	 *
+	 * Tests that timeout exceptions are properly handled in ordered output mode. The proper handling means that
+	 * a StreamElementQueueEntry is completed in case of a timeout exception.
+	 */
+	@Test
+	public void testUnorderedWaitTimeoutHandling() throws Exception {
+		testTimeoutExceptionHandling(AsyncDataStream.OutputMode.UNORDERED);
+	}
+
+	private void testTimeoutExceptionHandling(AsyncDataStream.OutputMode outputMode) throws Exception {
+		AsyncFunction<Integer, Integer> asyncFunction = new NoOpAsyncFunction<>();
+		long timeout = 10L; // 1 milli second
+
+		AsyncWaitOperator<Integer, Integer> asyncWaitOperator = new AsyncWaitOperator<>(
+			asyncFunction,
+			timeout,
+			2,
+			outputMode);
+
+		final Environment mockenvironment = createMockEnvironment();
+
+		OneInputStreamOperatorTestHarness<Integer, Integer> harness = new OneInputStreamOperatorTestHarness<>(
+			asyncWaitOperator,
+			IntSerializer.INSTANCE,
+			mockenvironment);
+
+		harness.open();
+
+		synchronized (harness.getCheckpointLock()) {
+			harness.processElement(1, 1L);
+		}
+
+		harness.setProcessingTime(10L);
+
+		verify(harness.getEnvironment(), timeout(100L * timeout)).failExternally(any(Exception.class));
+
+		synchronized (harness.getCheckpointLock()) {
+			harness.close();
+		}
+	}
+
+	private static class NoOpAsyncFunction<IN, OUT> implements AsyncFunction<IN, OUT> {
+		private static final long serialVersionUID = -3060481953330480694L;
+
+		@Override
+		public void asyncInvoke(IN input, AsyncCollector<OUT> collector) throws Exception {
+			// no op
+		}
+	}
+
 }


[11/16] flink git commit: [FLINK-6078] Remove CuratorFramework#close calls from ZooKeeper based HA services

Posted by tr...@apache.org.
[FLINK-6078] Remove CuratorFramework#close calls from ZooKeeper based HA services

Remove client less factory methods from ZooKeeperUtils

Introduce default job id

This closes #3781.


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

Branch: refs/heads/master
Commit: ddd6a99a95b56c52ea5b5153b7270b578f5479bc
Parents: a0bb99c
Author: Till Rohrmann <tr...@apache.org>
Authored: Thu Mar 16 17:03:03 2017 +0100
Committer: Till Rohrmann <tr...@apache.org>
Committed: Fri May 5 11:06:07 2017 +0200

----------------------------------------------------------------------
 .../org/apache/flink/client/CliFrontend.java    |   6 +-
 .../flink/client/cli/CustomCommandLine.java     |   4 +-
 .../flink/client/program/ClusterClient.java     |  89 ++++----
 .../client/program/StandaloneClusterClient.java |   8 +-
 .../RemoteExecutorHostnameResolutionTest.java   |  22 +-
 .../apache/flink/client/program/ClientTest.java |   5 +-
 ...rRetrievalServiceHostnameResolutionTest.java |  32 +--
 .../api/avro/AvroExternalJarProgramITCase.java  |  22 +-
 .../src/test/resources/log4j-test.properties    |   2 +-
 .../connectors/kafka/Kafka010ITCase.java        |  14 +-
 .../connectors/kafka/Kafka08ITCase.java         |   4 +-
 .../kafka/Kafka09SecuredRunITCase.java          |   2 +-
 .../connectors/kafka/KafkaConsumerTestBase.java |  96 ++++----
 .../connectors/kafka/KafkaProducerTestBase.java |   2 +-
 .../kafka/KafkaShortRetentionTestBase.java      |  18 +-
 .../connectors/kafka/KafkaTestBase.java         |  23 +-
 .../src/test/resources/log4j-test.properties    |   2 +-
 .../org/apache/flink/storm/api/FlinkClient.java |   4 +-
 .../flink/contrib/streaming/CollectITCase.java  |   9 +-
 .../operations/DegreesWithExceptionITCase.java  |  42 ++--
 .../ReduceOnEdgesWithExceptionITCase.java       |  37 ++--
 .../ReduceOnNeighborsWithExceptionITCase.java   |  43 ++--
 .../apache/flink/ml/util/FlinkTestBase.scala    |   2 +-
 .../src/test/resources/log4j-test.properties    |  38 ++++
 .../src/test/resources/logback-test.xml         |  42 ++++
 .../MesosApplicationMasterRunner.java           |  39 +++-
 .../MesosFlinkResourceManagerTest.java          |  37 +++-
 .../BackPressureStatsTrackerITCase.java         |  19 +-
 .../StackTraceSampleCoordinatorITCase.java      |  19 +-
 .../webmonitor/WebRuntimeMonitorITCase.java     |  31 ++-
 .../ZooKeeperCheckpointRecoveryFactory.java     |   2 +-
 .../apache/flink/runtime/client/JobClient.java  |  47 ++--
 .../flink/runtime/client/JobClientActor.java    |   5 +-
 .../runtime/client/JobListeningContext.java     |  22 +-
 .../clusterframework/BootstrapTools.java        |  17 +-
 .../HighAvailabilityServices.java               |   7 +
 .../HighAvailabilityServicesUtils.java          |   2 +-
 .../apache/flink/runtime/jobgraph/JobGraph.java |   9 +
 .../ZooKeeperSubmittedJobGraphStore.java        |  12 +-
 .../ZooKeeperLeaderElectionService.java         | 184 ++++++++++------
 .../ZooKeeperLeaderRetrievalService.java        | 122 +++++++----
 .../minicluster/StandaloneMiniCluster.java      | 154 +++++++++++++
 .../runtime/query/QueryableStateClient.java     |  10 +-
 .../runtime/rpc/akka/AkkaRpcServiceUtils.java   |  16 +-
 .../runtime/util/LeaderRetrievalUtils.java      |  67 ++----
 .../flink/runtime/util/SerializedThrowable.java |   4 +
 .../flink/runtime/util/ZooKeeperUtils.java      |  27 ---
 .../flink/runtime/jobmanager/JobManager.scala   | 158 +++++++-------
 .../runtime/messages/TaskManagerMessages.scala  |  13 +-
 .../runtime/minicluster/FlinkMiniCluster.scala  | 142 +++++++-----
 .../minicluster/LocalFlinkMiniCluster.scala     |  41 ++--
 .../flink/runtime/taskmanager/TaskManager.scala |  71 +++---
 .../checkpoint/CoordinatorShutdownTest.java     |   3 +-
 .../runtime/client/JobClientActorTest.java      |   8 +-
 .../clusterframework/ClusterShutdownITCase.java | 201 ++++++++++-------
 .../clusterframework/ResourceManagerITCase.java | 178 +++++++++------
 .../clusterframework/ResourceManagerTest.java   | 128 ++++++++++-
 .../highavailability/ManualLeaderService.java   | 116 ++++++++++
 .../TestingManualHighAvailabilityServices.java  | 150 +++++++++++++
 .../jobmanager/JobManagerHARecoveryTest.java    |  35 +--
 .../JobManagerProcessReapingTest.java           |   3 +-
 .../jobmanager/JobManagerStartupTest.java       |   5 +-
 .../runtime/jobmanager/JobManagerTest.java      | 188 ++++++++++------
 .../flink/runtime/jobmanager/JobSubmitTest.java |  15 +-
 .../jobmaster/JobManagerRunnerMockTest.java     |   3 +-
 .../LeaderChangeJobRecoveryTest.java            |  32 ++-
 .../LeaderChangeStateCleanupTest.java           |  50 +++--
 .../LeaderElectionRetrievalTestingCluster.java  | 121 -----------
 .../TestingLeaderElectionService.java           |   4 +
 .../TestingLeaderRetrievalService.java          |   4 +
 .../ZooKeeperLeaderElectionTest.java            | 109 +++++-----
 .../ZooKeeperLeaderRetrievalTest.java           |  91 ++++----
 .../runtime/metrics/TaskManagerMetricsTest.java |  26 ++-
 ...askManagerComponentsStartupShutdownTest.java |  47 ++--
 .../TaskManagerConfigurationTest.java           |  76 ++++---
 .../TaskManagerProcessReapingTestBase.java      |  73 +++++--
 .../TaskManagerRegistrationTest.java            | 217 +++++++++++--------
 .../taskmanager/TaskManagerStartupTest.java     |  62 +++++-
 .../runtime/taskmanager/TaskManagerTest.java    | 136 ++++++++++--
 .../jobmanager/JobManagerRegistrationTest.scala |  88 +++++---
 .../runtime/testingUtils/TestingCluster.scala   |  37 +++-
 .../runtime/testingUtils/TestingUtils.scala     | 176 ++++++---------
 .../org/apache/flink/api/scala/FlinkShell.scala |  14 +-
 .../flink/api/scala/ScalaShellITCase.scala      | 104 ++++-----
 .../environment/RemoteStreamEnvironment.java    |   6 +-
 .../streaming/util/TestStreamEnvironment.java   |  79 ++++++-
 .../flink/test/util/JavaProgramTestBase.java    |   4 +-
 .../test/util/MultipleProgramsTestBase.java     |   2 +-
 .../apache/flink/test/util/TestEnvironment.java | 133 ++++++++++--
 .../accumulators/AccumulatorErrorITCase.java    |  49 ++---
 ...tractEventTimeWindowCheckpointingITCase.java |  21 +-
 .../EventTimeAllWindowCheckpointingITCase.java  |  21 +-
 .../StreamFaultToleranceTestBase.java           |   4 +-
 .../WindowCheckpointingITCase.java              |  18 +-
 .../test/classloading/ClassLoaderITCase.java    | 136 +++++++-----
 .../jar/CheckpointedStreamingProgram.java       |   9 +-
 .../jar/CheckpointingCustomKvStateProgram.java  |   9 +-
 .../jar/CustomInputSplitProgram.java            |  12 +-
 .../classloading/jar/CustomKvStateProgram.java  |  15 +-
 .../test/classloading/jar/KMeansForTest.java    |  17 +-
 .../jar/LegacyCheckpointedStreamingProgram.java |   6 +-
 .../jar/StreamingCustomInputSplitProgram.java   |  11 +-
 .../test/classloading/jar/StreamingProgram.java |   7 +-
 .../test/classloading/jar/UserCodeType.java     |   6 +-
 .../clients/examples/JobRetrievalITCase.java    |   7 +-
 .../CustomDistributionITCase.java               |   2 +-
 .../RemoteEnvironmentITCase.java                |  46 ++--
 .../flink/test/misc/AutoParallelismITCase.java  |   8 +-
 .../test/misc/CustomSerializationITCase.java    |  57 ++---
 .../test/misc/MiscellaneousIssuesITCase.java    |  52 ++---
 ...SuccessAfterNetworkBuffersFailureITCase.java |  28 +--
 .../query/AbstractQueryableStateITCase.java     |  33 ++-
 ...ctTaskManagerProcessFailureRecoveryTest.java |  18 +-
 .../flink/test/recovery/ChaosMonkeyITCase.java  |  13 +-
 .../flink/test/recovery/FastFailuresITCase.java |  10 +-
 .../JobManagerHACheckpointRecoveryITCase.java   |  42 +++-
 .../JobManagerHAJobGraphRecoveryITCase.java     |  24 +-
 ...agerHAProcessFailureBatchRecoveryITCase.java |  28 ++-
 .../recovery/ProcessFailureCancelingITCase.java |  30 ++-
 .../TaskManagerFailureRecoveryITCase.java       |   4 +-
 .../ZooKeeperLeaderElectionITCase.java          |  17 +-
 .../AbstractOperatorRestoreTestBase.java        |  39 +++-
 .../test/streaming/runtime/TimestampITCase.java |  63 +++---
 .../jobmanager/JobManagerFailsITCase.scala      |  17 +-
 .../taskmanager/TaskManagerFailsITCase.scala    |   2 +-
 .../flink/yarn/FlinkYarnSessionCliTest.java     |   7 +-
 .../flink/yarn/YARNHighAvailabilityITCase.java  |  40 +++-
 .../flink/yarn/YARNSessionFIFOITCase.java       |   2 +-
 .../yarn/AbstractYarnClusterDescriptor.java     |   2 +-
 .../flink/yarn/YarnApplicationMasterRunner.java |  38 +++-
 .../apache/flink/yarn/YarnClusterClient.java    |  36 +--
 .../apache/flink/yarn/YarnClusterClientV2.java  |   5 +-
 .../yarn/YarnFlinkApplicationMasterRunner.java  |   5 +-
 .../org/apache/flink/yarn/cli/FlinkYarnCLI.java |  13 +-
 .../flink/yarn/cli/FlinkYarnSessionCli.java     |   7 +-
 135 files changed, 3474 insertions(+), 2131 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/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 0d61cbd..74d5f5d 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
@@ -263,7 +263,11 @@ public class CliFrontend {
 		}
 		finally {
 			if (client != null) {
-				client.shutdown();
+				try {
+					client.shutdown();
+				} catch (Exception e) {
+					LOG.warn("Could not properly shut down the cluster client.", e);
+				}
 			}
 			if (program != null) {
 				program.deleteExtractedLibraries();

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/flink-clients/src/main/java/org/apache/flink/client/cli/CustomCommandLine.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/cli/CustomCommandLine.java b/flink-clients/src/main/java/org/apache/flink/client/cli/CustomCommandLine.java
index c58c74c..a4cb479 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/cli/CustomCommandLine.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/cli/CustomCommandLine.java
@@ -75,11 +75,11 @@ public interface CustomCommandLine<ClusterType extends ClusterClient> {
 	 * @param config The Flink config to use
 	 * @param userJarFiles User jar files to include in the classpath of the cluster.
 	 * @return The client to communicate with the cluster which the CustomCommandLine brought up.
-	 * @throws UnsupportedOperationException if the operation is not supported
+	 * @throws Exception if the cluster could not be created
 	 */
 	ClusterType createCluster(
 			String applicationName,
 			CommandLine commandLine,
 			Configuration config,
-			List<URL> userJarFiles) throws UnsupportedOperationException;
+			List<URL> userJarFiles) throws Exception;
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java b/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java
index 0f88f7c..6770eee 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java
@@ -42,6 +42,9 @@ import org.apache.flink.runtime.client.JobExecutionException;
 import org.apache.flink.runtime.client.JobListeningContext;
 import org.apache.flink.runtime.client.JobRetrievalException;
 import org.apache.flink.runtime.clusterframework.messages.GetClusterStatusResponse;
+import org.apache.flink.runtime.concurrent.Executors;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils;
 import org.apache.flink.runtime.instance.ActorGateway;
 import org.apache.flink.runtime.jobgraph.JobGraph;
 import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings;
@@ -72,7 +75,6 @@ import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 
-
 /**
  * Encapsulates the functionality necessary to submit a program to a remote cluster.
  */
@@ -95,6 +97,9 @@ public abstract class ClusterClient {
 	/** Lookup timeout for the job manager retrieval service */
 	private final FiniteDuration lookupTimeout;
 
+	/** Service factory for high available */
+	protected final HighAvailabilityServices highAvailabilityServices;
+
 	/** Flag indicating whether to sysout print execution updates */
 	private boolean printStatusDuringExecution = true;
 
@@ -119,10 +124,25 @@ public abstract class ClusterClient {
 	 *
 	 * @param flinkConfig The config used to obtain the job-manager's address, and used to configure the optimizer.
 	 *
-	 * @throws java.io.IOException Thrown, if the client's actor system could not be started.
+	 * @throws Exception we cannot create the high availability services
 	 */
-	public ClusterClient(Configuration flinkConfig) throws IOException {
+	public ClusterClient(Configuration flinkConfig) throws Exception {
+		this(flinkConfig,
+			HighAvailabilityServicesUtils.createHighAvailabilityServices(
+				flinkConfig,
+				Executors.directExecutor(),
+				HighAvailabilityServicesUtils.AddressResolution.TRY_ADDRESS_RESOLUTION));
+	}
 
+	/**
+	 * 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.
+	 *
+	 * @param flinkConfig The config used to obtain the job-manager's address, and used to configure the optimizer.
+	 * @param highAvailabilityServices HighAvailabilityServices to use for leader retrieval
+	 */
+	public ClusterClient(Configuration flinkConfig, HighAvailabilityServices highAvailabilityServices) {
 		this.flinkConfig = Preconditions.checkNotNull(flinkConfig);
 		this.compiler = new Optimizer(new DataStatistics(), new DefaultCostEstimator(), flinkConfig);
 
@@ -130,6 +150,8 @@ public abstract class ClusterClient {
 		this.lookupTimeout = AkkaUtils.getLookupTimeout(flinkConfig);
 
 		this.actorSystemLoader = new LazyActorSystemLoader(flinkConfig, LOG);
+
+		this.highAvailabilityServices = Preconditions.checkNotNull(highAvailabilityServices);
 	}
 
 	// ------------------------------------------------------------------------
@@ -202,12 +224,16 @@ public abstract class ClusterClient {
 	/**
 	 * Shuts down the client. This stops the internal actor system and actors.
 	 */
-	public void shutdown() {
+	public void shutdown() throws Exception {
 		synchronized (this) {
 			try {
 				finalizeCluster();
 			} finally {
-				this.actorSystemLoader.shutdown();
+				actorSystemLoader.shutdown();
+			}
+
+			if (highAvailabilityServices != null) {
+				highAvailabilityServices.closeAndCleanupAllData();
 			}
 		}
 	}
@@ -241,7 +267,8 @@ public abstract class ClusterClient {
 		try {
 			LeaderConnectionInfo leaderConnectionInfo =
 				LeaderRetrievalUtils.retrieveLeaderConnectionInfo(
-					LeaderRetrievalUtils.createLeaderRetrievalService(flinkConfig, true), timeout);
+					highAvailabilityServices.getJobManagerLeaderRetriever(HighAvailabilityServices.DEFAULT_JOB_ID),
+					timeout);
 
 			return AkkaUtils.getInetSockeAddressFromAkkaURL(leaderConnectionInfo.getAddress());
 		} catch (Exception e) {
@@ -411,17 +438,17 @@ public abstract class ClusterClient {
 
 		waitForClusterToBeReady();
 
-		final LeaderRetrievalService leaderRetrievalService;
-		try {
-			leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(flinkConfig, true);
-		} catch (Exception e) {
-			throw new ProgramInvocationException("Could not create the leader retrieval service", e);
-		}
-
 		try {
 			logAndSysout("Submitting job with JobID: " + jobGraph.getJobID() + ". Waiting for job completion.");
-			this.lastJobExecutionResult = JobClient.submitJobAndWait(actorSystemLoader.get(), flinkConfig,
-				leaderRetrievalService, jobGraph, timeout, printStatusDuringExecution, classLoader);
+			this.lastJobExecutionResult = JobClient.submitJobAndWait(
+				actorSystemLoader.get(),
+				flinkConfig,
+				highAvailabilityServices,
+				jobGraph,
+				timeout,
+				printStatusDuringExecution,
+				classLoader);
+
 			return this.lastJobExecutionResult;
 		} catch (JobExecutionException e) {
 			throw new ProgramInvocationException("The program execution failed: " + e.getMessage(), e);
@@ -462,13 +489,6 @@ public abstract class ClusterClient {
 	 * @throws JobExecutionException if an error occurs during monitoring the job execution
 	 */
 	public JobExecutionResult retrieveJob(JobID jobID) throws JobExecutionException {
-		final LeaderRetrievalService leaderRetrievalService;
-		try {
-			leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(flinkConfig, true);
-		} catch (Exception e) {
-			throw new JobRetrievalException(jobID, "Could not create the leader retrieval service", e);
-		}
-
 		ActorGateway jobManagerGateway;
 		try {
 			jobManagerGateway = getJobManagerGateway();
@@ -477,13 +497,13 @@ public abstract class ClusterClient {
 		}
 
 		final JobListeningContext listeningContext = JobClient.attachToRunningJob(
-				jobID,
-				jobManagerGateway,
-				flinkConfig,
-				actorSystemLoader.get(),
-				leaderRetrievalService,
-				timeout,
-				printStatusDuringExecution);
+			jobID,
+			jobManagerGateway,
+			flinkConfig,
+			actorSystemLoader.get(),
+			highAvailabilityServices,
+			timeout,
+			printStatusDuringExecution);
 
 		return JobClient.awaitJobResult(listeningContext);
 	}
@@ -496,13 +516,6 @@ public abstract class ClusterClient {
 	 * @throws JobExecutionException if an error occurs during monitoring the job execution
 	 */
 	public JobListeningContext connectToJob(JobID jobID) throws JobExecutionException {
-		final LeaderRetrievalService leaderRetrievalService;
-		try {
-			leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(flinkConfig, true);
-		} catch (Exception e) {
-			throw new JobRetrievalException(jobID, "Could not create the leader retrieval service", e);
-		}
-
 		ActorGateway jobManagerGateway;
 		try {
 			jobManagerGateway = getJobManagerGateway();
@@ -515,7 +528,7 @@ public abstract class ClusterClient {
 				jobManagerGateway,
 				flinkConfig,
 				actorSystemLoader.get(),
-				leaderRetrievalService,
+				highAvailabilityServices,
 				timeout,
 				printStatusDuringExecution);
 	}
@@ -721,7 +734,7 @@ public abstract class ClusterClient {
 	public ActorGateway getJobManagerGateway() throws Exception {
 		LOG.debug("Looking up JobManager");
 		return LeaderRetrievalUtils.retrieveLeaderGateway(
-			LeaderRetrievalUtils.createLeaderRetrievalService(flinkConfig, true),
+			highAvailabilityServices.getJobManagerLeaderRetriever(HighAvailabilityServices.DEFAULT_JOB_ID),
 			actorSystemLoader.get(),
 			lookupTimeout);
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/flink-clients/src/main/java/org/apache/flink/client/program/StandaloneClusterClient.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/StandaloneClusterClient.java b/flink-clients/src/main/java/org/apache/flink/client/program/StandaloneClusterClient.java
index 296ddc9..fd179c0 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/program/StandaloneClusterClient.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/program/StandaloneClusterClient.java
@@ -22,12 +22,12 @@ import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.clusterframework.messages.GetClusterStatus;
 import org.apache.flink.runtime.clusterframework.messages.GetClusterStatusResponse;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
 import org.apache.flink.runtime.instance.ActorGateway;
 import org.apache.flink.runtime.jobgraph.JobGraph;
 import scala.concurrent.Await;
 import scala.concurrent.Future;
 
-import java.io.IOException;
 import java.net.URL;
 import java.util.Collections;
 import java.util.List;
@@ -38,10 +38,14 @@ import java.util.List;
  */
 public class StandaloneClusterClient extends ClusterClient {
 
-	public StandaloneClusterClient(Configuration config) throws IOException {
+	public StandaloneClusterClient(Configuration config) throws Exception {
 		super(config);
 	}
 
+	public StandaloneClusterClient(Configuration config, HighAvailabilityServices highAvailabilityServices) {
+		super(config, highAvailabilityServices);
+	}
+
 	@Override
 	public void waitForClusterToBeReady() {}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/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 d8fb3de..be93949 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,7 +21,6 @@ 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.apache.flink.configuration.Configuration;
 import org.apache.flink.util.TestLogger;
 import org.junit.BeforeClass;
@@ -33,7 +32,6 @@ import java.net.URL;
 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;
 
@@ -48,26 +46,20 @@ public class RemoteExecutorHostnameResolutionTest extends TestLogger {
 	}
 
 	@Test
-	public void testUnresolvableHostname1() {
+	public void testUnresolvableHostname1() throws Exception {
 
 		RemoteExecutor exec = new RemoteExecutor(nonExistingHostname, port);
 		try {
 			exec.executePlan(getProgram());
 			fail("This should fail with an ProgramInvocationException");
 		}
-		catch (ProgramInvocationException e) {
+		catch (UnknownHostException ignored) {
 			// that is what we want!
-			assertTrue(e.getCause() instanceof UnknownHostException);
-		}
-		catch (Exception e) {
-			System.err.println("Wrong exception!");
-			e.printStackTrace();
-			fail(e.getMessage());
 		}
 	}
 
 	@Test
-	public void testUnresolvableHostname2() {
+	public void testUnresolvableHostname2() throws Exception {
 
 		InetSocketAddress add = new InetSocketAddress(nonExistingHostname, port);
 		RemoteExecutor exec = new RemoteExecutor(add, new Configuration(),
@@ -76,14 +68,8 @@ public class RemoteExecutorHostnameResolutionTest extends TestLogger {
 			exec.executePlan(getProgram());
 			fail("This should fail with an ProgramInvocationException");
 		}
-		catch (ProgramInvocationException e) {
+		catch (UnknownHostException ignored) {
 			// that is what we want!
-			assertTrue(e.getCause() instanceof UnknownHostException);
-		}
-		catch (Exception e) {
-			System.err.println("Wrong exception!");
-			e.printStackTrace();
-			fail(e.getMessage());
 		}
 	}
 	

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/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 da297d6..b7ade2a 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
@@ -56,7 +56,6 @@ import org.junit.Test;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 
-import java.io.IOException;
 import java.net.URL;
 import java.util.Collections;
 import java.util.UUID;
@@ -199,7 +198,7 @@ public class ClientTest extends TestLogger {
 	 * This test verifies correct job submission messaging logic and plan translation calls.
 	 */
 	@Test
-	public void shouldSubmitToJobClient() throws IOException, ProgramInvocationException {
+	public void shouldSubmitToJobClient() throws Exception {
 		jobManagerSystem.actorOf(
 			Props.create(SuccessReturningActor.class),
 			JobMaster.JOB_MANAGER_NAME);
@@ -217,7 +216,7 @@ public class ClientTest extends TestLogger {
 	 * This test verifies correct that the correct exception is thrown when the job submission fails.
 	 */
 	@Test
-	public void shouldSubmitToJobClientFails() throws IOException {
+	public void shouldSubmitToJobClientFails() throws Exception {
 			jobManagerSystem.actorOf(
 				Props.create(FailureReturningActor.class),
 				JobMaster.JOB_MANAGER_NAME);

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/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
index fc10f65..0ecdc2c 100644
--- 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
@@ -20,7 +20,9 @@ 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.runtime.jobmaster.JobMaster;
+import org.apache.flink.runtime.util.StandaloneUtils;
+import org.apache.flink.util.ConfigurationException;
 import org.apache.flink.util.TestLogger;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -32,7 +34,7 @@ import static org.junit.Assert.*;
 import static org.junit.Assume.assumeTrue;
 
 /**
- * Tests that verify that the LeaderRetrievalSevice correctly handles non-resolvable host names
+ * Tests that verify that the LeaderRetrievalService correctly handles non-resolvable host names
  * and does not fail with another exception
  */
 public class LeaderRetrievalServiceHostnameResolutionTest extends TestLogger {
@@ -48,21 +50,16 @@ public class LeaderRetrievalServiceHostnameResolutionTest extends TestLogger {
 	 * Tests that the StandaloneLeaderRetrievalService resolves host names if specified.
 	 */
 	@Test
-	public void testUnresolvableHostname1() {
+	public void testUnresolvableHostname1() throws UnknownHostException, ConfigurationException {
+		Configuration config = new Configuration();
 
-		try {
-			Configuration config = new Configuration();
+		config.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, nonExistingHostname);
+		config.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, 17234);
 
-			config.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, nonExistingHostname);
-			config.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, 17234);
-
-			LeaderRetrievalUtils.createLeaderRetrievalService(config, false);
-		}
-		catch (Exception e) {
-			System.err.println("Shouldn't throw an exception!");
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
+		StandaloneUtils.createLeaderRetrievalService(
+			config,
+			false,
+			JobMaster.JOB_MANAGER_NAME);
 	}
 
 	/*
@@ -77,7 +74,10 @@ public class LeaderRetrievalServiceHostnameResolutionTest extends TestLogger {
 			config.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, nonExistingHostname);
 			config.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, 17234);
 
-			LeaderRetrievalUtils.createLeaderRetrievalService(config, true);
+			StandaloneUtils.createLeaderRetrievalService(
+				config,
+				true,
+				JobMaster.JOB_MANAGER_NAME);
 			fail("This should fail with an UnknownHostException");
 		}
 		catch (UnknownHostException e) {

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/flink-connectors/flink-avro/src/test/java/org/apache/flink/api/avro/AvroExternalJarProgramITCase.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-avro/src/test/java/org/apache/flink/api/avro/AvroExternalJarProgramITCase.java b/flink-connectors/flink-avro/src/test/java/org/apache/flink/api/avro/AvroExternalJarProgramITCase.java
index 5f5209a..063a363 100644
--- a/flink-connectors/flink-avro/src/test/java/org/apache/flink/api/avro/AvroExternalJarProgramITCase.java
+++ b/flink-connectors/flink-avro/src/test/java/org/apache/flink/api/avro/AvroExternalJarProgramITCase.java
@@ -19,14 +19,16 @@
 package org.apache.flink.api.avro;
 
 import java.io.File;
+import java.net.URL;
+import java.util.Collections;
 
-import org.apache.flink.client.program.ClusterClient;
 import org.apache.flink.client.program.PackagedProgram;
-import org.apache.flink.client.program.StandaloneClusterClient;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.core.fs.Path;
 import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
 
+import org.apache.flink.test.util.TestEnvironment;
 import org.apache.flink.util.TestLogger;
 import org.junit.Assert;
 import org.junit.Test;
@@ -43,8 +45,9 @@ public class AvroExternalJarProgramITCase extends TestLogger {
 		LocalFlinkMiniCluster testMiniCluster = null;
 
 		try {
+			int parallelism = 4;
 			Configuration config = new Configuration();
-			config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 4);
+			config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, parallelism);
 			testMiniCluster = new LocalFlinkMiniCluster(config, false);
 			testMiniCluster.start();
 
@@ -53,15 +56,16 @@ public class AvroExternalJarProgramITCase extends TestLogger {
 
 			PackagedProgram program = new PackagedProgram(new File(jarFile), new String[] { testData });
 
+			TestEnvironment.setAsContext(
+				testMiniCluster,
+				parallelism,
+				Collections.singleton(new Path(jarFile)),
+				Collections.<URL>emptyList());
 
 			config.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, "localhost");
 			config.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, testMiniCluster.getLeaderRPCPort());
 
-			ClusterClient client = new StandaloneClusterClient(config);
-
-			client.setPrintStatusDuringExecution(false);
-			client.run(program, 4);
-
+			program.invokeInteractiveModeForExecution();
 		}
 		catch (Throwable t) {
 			System.err.println(t.getMessage());
@@ -69,6 +73,8 @@ public class AvroExternalJarProgramITCase extends TestLogger {
 			Assert.fail("Error during the packaged program execution: " + t.getMessage());
 		}
 		finally {
+			TestEnvironment.unsetAsContext();
+
 			if (testMiniCluster != null) {
 				try {
 					testMiniCluster.stop();

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/flink-connectors/flink-avro/src/test/resources/log4j-test.properties
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-avro/src/test/resources/log4j-test.properties b/flink-connectors/flink-avro/src/test/resources/log4j-test.properties
index 0b686e5..881dc06 100644
--- a/flink-connectors/flink-avro/src/test/resources/log4j-test.properties
+++ b/flink-connectors/flink-avro/src/test/resources/log4j-test.properties
@@ -24,4 +24,4 @@ log4j.appender.A1=org.apache.log4j.ConsoleAppender
 
 # A1 uses PatternLayout.
 log4j.appender.A1.layout=org.apache.log4j.PatternLayout
-log4j.appender.A1.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n
\ No newline at end of file
+log4j.appender.A1.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010ITCase.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010ITCase.java b/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010ITCase.java
index 2085169..39b2b8f 100644
--- a/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010ITCase.java
+++ b/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010ITCase.java
@@ -180,13 +180,14 @@ public class Kafka010ITCase extends KafkaConsumerTestBase {
 
 		// ---------- Produce an event time stream into Kafka -------------------
 
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
 		env.setParallelism(1);
 		env.getConfig().setRestartStrategy(RestartStrategies.noRestart());
 		env.getConfig().disableSysoutLogging();
 		env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
 
 		DataStream<Long> streamWithTimestamps = env.addSource(new SourceFunction<Long>() {
+			private static final long serialVersionUID = -2255105836471289626L;
 			boolean running = true;
 
 			@Override
@@ -208,6 +209,8 @@ public class Kafka010ITCase extends KafkaConsumerTestBase {
 
 		final TypeInformationSerializationSchema<Long> longSer = new TypeInformationSerializationSchema<>(TypeInfoParser.<Long>parse("Long"), env.getConfig());
 		FlinkKafkaProducer010.FlinkKafkaProducer010Configuration prod = FlinkKafkaProducer010.writeToKafkaWithTimestamps(streamWithTimestamps, topic, new KeyedSerializationSchemaWrapper<>(longSer), standardProps, new KafkaPartitioner<Long>() {
+			private static final long serialVersionUID = -6730989584364230617L;
+
 			@Override
 			public int partition(Long next, byte[] serializedKey, byte[] serializedValue, int numPartitions) {
 				return (int)(next % 3);
@@ -219,7 +222,7 @@ public class Kafka010ITCase extends KafkaConsumerTestBase {
 
 		// ---------- Consume stream from Kafka -------------------
 
-		env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
+		env = StreamExecutionEnvironment.getExecutionEnvironment();
 		env.setParallelism(1);
 		env.getConfig().setRestartStrategy(RestartStrategies.noRestart());
 		env.getConfig().disableSysoutLogging();
@@ -227,6 +230,8 @@ public class Kafka010ITCase extends KafkaConsumerTestBase {
 
 		FlinkKafkaConsumer010<Long> kafkaSource = new FlinkKafkaConsumer010<>(topic, new LimitedLongDeserializer(), standardProps);
 		kafkaSource.assignTimestampsAndWatermarks(new AssignerWithPunctuatedWatermarks<Long>() {
+			private static final long serialVersionUID = -4834111073247835189L;
+
 			@Nullable
 			@Override
 			public Watermark checkAndGetNextWatermark(Long lastElement, long extractedTimestamp) {
@@ -253,8 +258,12 @@ public class Kafka010ITCase extends KafkaConsumerTestBase {
 
 	private static class TimestampValidatingOperator extends StreamSink<Long> {
 
+		private static final long serialVersionUID = 1353168781235526806L;
+
 		public TimestampValidatingOperator() {
 			super(new SinkFunction<Long>() {
+				private static final long serialVersionUID = -6676565693361786524L;
+
 				@Override
 				public void invoke(Long value) throws Exception {
 					throw new RuntimeException("Unexpected");
@@ -304,6 +313,7 @@ public class Kafka010ITCase extends KafkaConsumerTestBase {
 
 	private static class LimitedLongDeserializer implements KeyedDeserializationSchema<Long> {
 
+		private static final long serialVersionUID = 6966177118923713521L;
 		private final TypeInformation<Long> ti;
 		private final TypeSerializer<Long> ser;
 		long cnt = 0;

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08ITCase.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08ITCase.java b/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08ITCase.java
index f5cb8c0..8cc735d 100644
--- a/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08ITCase.java
+++ b/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08ITCase.java
@@ -87,7 +87,7 @@ public class Kafka08ITCase extends KafkaConsumerTestBase {
 		final int valuesCount = 20;
 		final int startFrom = 0;
 
-		final StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
+		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
 		env.getConfig().disableSysoutLogging();
 
 		readSequence(env, StartupMode.GROUP_OFFSETS, null, standardProps, parallelism, topic, valuesCount, startFrom);
@@ -190,7 +190,7 @@ public class Kafka08ITCase extends KafkaConsumerTestBase {
 		// write a sequence from 0 to 99 to each of the 3 partitions.
 		final String topicName = writeSequence("testOffsetAutocommit", 100, parallelism, 1);
 
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
 		// NOTE: We are not enabling the checkpointing!
 		env.getConfig().disableSysoutLogging();
 		env.getConfig().setRestartStrategy(RestartStrategies.noRestart());

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09SecuredRunITCase.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09SecuredRunITCase.java b/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09SecuredRunITCase.java
index e748537..16a13c0 100644
--- a/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09SecuredRunITCase.java
+++ b/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09SecuredRunITCase.java
@@ -34,7 +34,7 @@ public class Kafka09SecuredRunITCase extends KafkaConsumerTestBase {
 	protected static final Logger LOG = LoggerFactory.getLogger(Kafka09SecuredRunITCase.class);
 
 	@BeforeClass
-	public static void prepare() throws IOException, ClassNotFoundException {
+	public static void prepare() throws ClassNotFoundException {
 		LOG.info("-------------------------------------------------------------------------");
 		LOG.info("    Starting Kafka09SecuredRunITCase ");
 		LOG.info("-------------------------------------------------------------------------");

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java
index ddac61c..ba83460 100644
--- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java
+++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java
@@ -154,7 +154,7 @@ public abstract class KafkaConsumerTestBase extends KafkaTestBase {
 		try {
 			Properties properties = new Properties();
 
-			StreamExecutionEnvironment see = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
+			StreamExecutionEnvironment see = StreamExecutionEnvironment.getExecutionEnvironment();
 			see.getConfig().disableSysoutLogging();
 			see.setRestartStrategy(RestartStrategies.noRestart());
 			see.setParallelism(1);
@@ -173,22 +173,14 @@ public abstract class KafkaConsumerTestBase extends KafkaTestBase {
 			DataStream<String> stream = see.addSource(source);
 			stream.print();
 			see.execute("No broker test");
-		} catch(ProgramInvocationException pie) {
+		} catch(JobExecutionException jee) {
 			if(kafkaServer.getVersion().equals("0.9") || kafkaServer.getVersion().equals("0.10")) {
-				assertTrue(pie.getCause() instanceof JobExecutionException);
-
-				JobExecutionException jee = (JobExecutionException) pie.getCause();
-
 				assertTrue(jee.getCause() instanceof TimeoutException);
 
 				TimeoutException te = (TimeoutException) jee.getCause();
 
 				assertEquals("Timeout expired while fetching topic metadata", te.getMessage());
 			} else {
-				assertTrue(pie.getCause() instanceof JobExecutionException);
-
-				JobExecutionException jee = (JobExecutionException) pie.getCause();
-
 				assertTrue(jee.getCause() instanceof RuntimeException);
 
 				RuntimeException re = (RuntimeException) jee.getCause();
@@ -208,7 +200,7 @@ public abstract class KafkaConsumerTestBase extends KafkaTestBase {
 
 		final String topicName = writeSequence("testCommitOffsetsToKafkaTopic", recordsInEachPartition, parallelism, 1);
 
-		final StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
+		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
 		env.getConfig().disableSysoutLogging();
 		env.getConfig().setRestartStrategy(RestartStrategies.noRestart());
 		env.setParallelism(parallelism);
@@ -280,6 +272,8 @@ public abstract class KafkaConsumerTestBase extends KafkaTestBase {
 	public void runStartFromKafkaCommitOffsets() throws Exception {
 		final int parallelism = 3;
 		final int recordsInEachPartition = 300;
+		final int recordsToConsume = 150;
+		final int consumePause = 50;
 
 		final String topicName = writeSequence("testStartFromKafkaCommitOffsetsTopic", recordsInEachPartition, parallelism, 1);
 
@@ -294,7 +288,7 @@ public abstract class KafkaConsumerTestBase extends KafkaTestBase {
 			attempt++;
 			LOG.info("Attempt " + attempt + " to read records and commit some offsets to Kafka");
 
-			final StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
+			final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
 			env.getConfig().disableSysoutLogging();
 			env.getConfig().setRestartStrategy(RestartStrategies.noRestart());
 			env.setParallelism(parallelism);
@@ -302,13 +296,13 @@ public abstract class KafkaConsumerTestBase extends KafkaTestBase {
 
 			env
 				.addSource(kafkaServer.getConsumer(topicName, new SimpleStringSchema(), standardProps))
-				.map(new ThrottledMapper<String>(50))
+				.map(new ThrottledMapper<String>(consumePause))
 				.map(new MapFunction<String, Object>() {
 					int count = 0;
 					@Override
 					public Object map(String value) throws Exception {
 						count++;
-						if (count == 150) {
+						if (count == recordsToConsume) {
 							throw new SuccessException();
 						}
 						return null;
@@ -329,7 +323,7 @@ public abstract class KafkaConsumerTestBase extends KafkaTestBase {
 
 		LOG.info("Got final committed offsets from Kafka o1={}, o2={}, o3={}", o1, o2, o3);
 
-		final StreamExecutionEnvironment env2 = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
+		final StreamExecutionEnvironment env2 = StreamExecutionEnvironment.getExecutionEnvironment();
 		env2.getConfig().disableSysoutLogging();
 		env2.getConfig().setRestartStrategy(RestartStrategies.noRestart());
 		env2.setParallelism(parallelism);
@@ -375,7 +369,7 @@ public abstract class KafkaConsumerTestBase extends KafkaTestBase {
 
 		final String topicName = writeSequence("testAutoOffsetRetrievalAndCommitToKafkaTopic", recordsInEachPartition, parallelism, 1);
 
-		final StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
+		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
 		env.getConfig().disableSysoutLogging();
 		env.getConfig().setRestartStrategy(RestartStrategies.noRestart());
 		env.setParallelism(parallelism);
@@ -452,7 +446,7 @@ public abstract class KafkaConsumerTestBase extends KafkaTestBase {
 
 		final String topicName = writeSequence("testStartFromEarliestOffsetsTopic", recordsInEachPartition, parallelism, 1);
 
-		final StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
+		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
 		env.getConfig().disableSysoutLogging();
 		env.setParallelism(parallelism);
 
@@ -510,7 +504,7 @@ public abstract class KafkaConsumerTestBase extends KafkaTestBase {
 				new TypeInformationSerializationSchema<>(resultType, new ExecutionConfig()));
 
 		// setup and run the latest-consuming job
-		final StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
+		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
 		env.getConfig().disableSysoutLogging();
 		env.setParallelism(parallelism);
 
@@ -541,7 +535,7 @@ public abstract class KafkaConsumerTestBase extends KafkaTestBase {
 				try {
 					env.execute(consumeExtraRecordsJobName);
 				} catch (Throwable t) {
-					if (!(t.getCause() instanceof JobCancellationException)) {
+					if (!(t instanceof JobCancellationException)) {
 						error.set(t);
 					}
 				}
@@ -555,7 +549,9 @@ public abstract class KafkaConsumerTestBase extends KafkaTestBase {
 			consumeExtraRecordsJobName);
 
 		// setup the extra records writing job
-		final StreamExecutionEnvironment env2 = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
+		final StreamExecutionEnvironment env2 = StreamExecutionEnvironment.getExecutionEnvironment();
+
+		env2.setParallelism(parallelism);
 
 		DataStream<Tuple2<Integer, Integer>> extraRecordsStream = env2
 			.addSource(new RichParallelSourceFunction<Tuple2<Integer, Integer>>() {
@@ -577,7 +573,7 @@ public abstract class KafkaConsumerTestBase extends KafkaTestBase {
 				public void cancel() {
 					running = false;
 				}
-			}).setParallelism(parallelism);
+			});
 
 		kafkaServer.produceIntoKafka(extraRecordsStream, topicName, serSchema, readProps, null);
 
@@ -626,7 +622,7 @@ public abstract class KafkaConsumerTestBase extends KafkaTestBase {
 
 		final String topicName = writeSequence("testStartFromGroupOffsetsTopic", recordsInEachPartition, parallelism, 1);
 
-		final StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
+		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
 		env.getConfig().disableSysoutLogging();
 		env.setParallelism(parallelism);
 
@@ -685,7 +681,7 @@ public abstract class KafkaConsumerTestBase extends KafkaTestBase {
 
 		final String topicName = writeSequence("testStartFromSpecificOffsetsTopic", recordsInEachPartition, parallelism, 1);
 
-		final StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
+		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
 		env.getConfig().disableSysoutLogging();
 		env.setParallelism(parallelism);
 
@@ -751,7 +747,7 @@ public abstract class KafkaConsumerTestBase extends KafkaTestBase {
 		createTestTopic(additionalEmptyTopic, parallelism, 1); // create an empty topic which will remain empty all the time
 
 		final StreamExecutionEnvironment env =
-				StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
+				StreamExecutionEnvironment.getExecutionEnvironment();
 		env.setParallelism(parallelism);
 		env.enableCheckpointing(500);
 		env.setRestartStrategy(RestartStrategies.noRestart()); // fail immediately
@@ -878,7 +874,7 @@ public abstract class KafkaConsumerTestBase extends KafkaTestBase {
 		createTestTopic(topic, parallelism, 1);
 
 		DataGenerators.generateRandomizedIntegerSequence(
-				StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort),
+				StreamExecutionEnvironment.getExecutionEnvironment(),
 				kafkaServer,
 				topic, parallelism, numElementsPerPartition, true);
 
@@ -887,7 +883,7 @@ public abstract class KafkaConsumerTestBase extends KafkaTestBase {
 		DeserializationSchema<Integer> schema =
 				new TypeInformationSerializationSchema<>(BasicTypeInfo.INT_TYPE_INFO, new ExecutionConfig());
 
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
 		env.enableCheckpointing(500);
 		env.setParallelism(parallelism);
 		env.setRestartStrategy(RestartStrategies.fixedDelayRestart(1, 0));
@@ -927,7 +923,7 @@ public abstract class KafkaConsumerTestBase extends KafkaTestBase {
 		createTestTopic(topic, numPartitions, 1);
 
 		DataGenerators.generateRandomizedIntegerSequence(
-				StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort),
+				StreamExecutionEnvironment.getExecutionEnvironment(),
 				kafkaServer,
 				topic, numPartitions, numElementsPerPartition, false);
 
@@ -936,7 +932,7 @@ public abstract class KafkaConsumerTestBase extends KafkaTestBase {
 		DeserializationSchema<Integer> schema =
 				new TypeInformationSerializationSchema<>(BasicTypeInfo.INT_TYPE_INFO, new ExecutionConfig());
 
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
 		env.enableCheckpointing(500);
 		env.setParallelism(parallelism);
 		env.setRestartStrategy(RestartStrategies.fixedDelayRestart(1, 0));
@@ -975,7 +971,7 @@ public abstract class KafkaConsumerTestBase extends KafkaTestBase {
 		createTestTopic(topic, numPartitions, 1);
 
 		DataGenerators.generateRandomizedIntegerSequence(
-				StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort),
+				StreamExecutionEnvironment.getExecutionEnvironment(),
 				kafkaServer,
 				topic, numPartitions, numElementsPerPartition, true);
 
@@ -984,7 +980,7 @@ public abstract class KafkaConsumerTestBase extends KafkaTestBase {
 		DeserializationSchema<Integer> schema =
 				new TypeInformationSerializationSchema<>(BasicTypeInfo.INT_TYPE_INFO, new ExecutionConfig());
 
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
 		env.enableCheckpointing(500);
 		env.setParallelism(parallelism);
 		// set the number of restarts to one. The failing mapper will fail once, then it's only success exceptions.
@@ -1033,7 +1029,7 @@ public abstract class KafkaConsumerTestBase extends KafkaTestBase {
 			@Override
 			public void run() {
 				try {
-					final StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
+					final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
 					env.setParallelism(parallelism);
 					env.enableCheckpointing(100);
 					env.getConfig().disableSysoutLogging();
@@ -1107,7 +1103,7 @@ public abstract class KafkaConsumerTestBase extends KafkaTestBase {
 			@Override
 			public void run() {
 				try {
-					final StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
+					final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
 					env.setParallelism(parallelism);
 					env.enableCheckpointing(100);
 					env.getConfig().disableSysoutLogging();
@@ -1163,7 +1159,7 @@ public abstract class KafkaConsumerTestBase extends KafkaTestBase {
 		DeserializationSchema<Integer> schema =
 				new TypeInformationSerializationSchema<>(BasicTypeInfo.INT_TYPE_INFO, new ExecutionConfig());
 
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
 		env.setParallelism(12); // needs to be more that the mini cluster has slots
 		env.getConfig().disableSysoutLogging();
 
@@ -1180,7 +1176,7 @@ public abstract class KafkaConsumerTestBase extends KafkaTestBase {
 			env.execute("test fail on deploy");
 			fail("this test should fail with an exception");
 		}
-		catch (ProgramInvocationException e) {
+		catch (JobExecutionException e) {
 
 			// validate that we failed due to a NoResourceAvailableException
 			Throwable cause = e.getCause();
@@ -1209,7 +1205,7 @@ public abstract class KafkaConsumerTestBase extends KafkaTestBase {
 		final int NUM_TOPICS = 5;
 		final int NUM_ELEMENTS = 20;
 
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
 		env.getConfig().disableSysoutLogging();
 		
 		// create topics with content
@@ -1220,6 +1216,10 @@ public abstract class KafkaConsumerTestBase extends KafkaTestBase {
 			// create topic
 			createTestTopic(topic, i + 1 /*partitions*/, 1);
 		}
+
+		// before FLINK-6078 the RemoteExecutionEnvironment set the parallelism to 1 as well
+		env.setParallelism(1);
+
 		// run first job, producing into all topics
 		DataStream<Tuple3<Integer, Integer, String>> stream = env.addSource(new RichParallelSourceFunction<Tuple3<Integer, Integer, String>>() {
 
@@ -1249,7 +1249,7 @@ public abstract class KafkaConsumerTestBase extends KafkaTestBase {
 		env.execute("Write to topics");
 
 		// run second job consuming from multiple topics
-		env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
+		env = StreamExecutionEnvironment.getExecutionEnvironment();
 		env.getConfig().disableSysoutLogging();
 
 		stream = env.addSource(kafkaServer.getConsumer(topics, schema, props));
@@ -1357,7 +1357,7 @@ public abstract class KafkaConsumerTestBase extends KafkaTestBase {
 		final TypeInformationSerializationSchema<Tuple2<Long, byte[]>> serSchema =
 				new TypeInformationSerializationSchema<>(longBytesInfo, new ExecutionConfig());
 
-		final StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
+		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
 		env.setRestartStrategy(RestartStrategies.noRestart());
 		env.getConfig().disableSysoutLogging();
 		env.enableCheckpointing(100);
@@ -1457,7 +1457,7 @@ public abstract class KafkaConsumerTestBase extends KafkaTestBase {
 		createTestTopic(topic, parallelism, 2);
 
 		DataGenerators.generateRandomizedIntegerSequence(
-				StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort),
+				StreamExecutionEnvironment.getExecutionEnvironment(),
 				kafkaServer,
 				topic, parallelism, numElementsPerPartition, true);
 
@@ -1472,7 +1472,7 @@ public abstract class KafkaConsumerTestBase extends KafkaTestBase {
 		DeserializationSchema<Integer> schema =
 				new TypeInformationSerializationSchema<>(BasicTypeInfo.INT_TYPE_INFO, new ExecutionConfig());
 
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
 		env.setParallelism(parallelism);
 		env.enableCheckpointing(500);
 		env.setRestartStrategy(RestartStrategies.noRestart());
@@ -1503,7 +1503,7 @@ public abstract class KafkaConsumerTestBase extends KafkaTestBase {
 
 		// ----------- Write some data into Kafka -------------------
 
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
 		env.setParallelism(1);
 		env.setRestartStrategy(RestartStrategies.noRestart());
 		env.getConfig().disableSysoutLogging();
@@ -1535,7 +1535,7 @@ public abstract class KafkaConsumerTestBase extends KafkaTestBase {
 
 		// ----------- Read the data again -------------------
 
-		env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
+		env = StreamExecutionEnvironment.getExecutionEnvironment();
 		env.setParallelism(1);
 		env.setRestartStrategy(RestartStrategies.noRestart());
 		env.getConfig().disableSysoutLogging();
@@ -1590,7 +1590,7 @@ public abstract class KafkaConsumerTestBase extends KafkaTestBase {
 
 		// ----------- Write some data into Kafka -------------------
 
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
 		env.setParallelism(1);
 		env.getConfig().setRestartStrategy(RestartStrategies.noRestart());
 		env.getConfig().disableSysoutLogging();
@@ -1621,7 +1621,7 @@ public abstract class KafkaConsumerTestBase extends KafkaTestBase {
 
 		// ----------- Read the data again -------------------
 
-		env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
+		env = StreamExecutionEnvironment.getExecutionEnvironment();
 		env.setParallelism(1);
 		env.getConfig().setRestartStrategy(RestartStrategies.noRestart());
 		env.getConfig().disableSysoutLogging();
@@ -1661,7 +1661,7 @@ public abstract class KafkaConsumerTestBase extends KafkaTestBase {
 		final String topic = writeSequence("testEndOfStream", ELEMENT_COUNT, 1, 1);
 
 		// read using custom schema
-		final StreamExecutionEnvironment env1 = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
+		final StreamExecutionEnvironment env1 = StreamExecutionEnvironment.getExecutionEnvironment();
 		env1.setParallelism(1);
 		env1.getConfig().setRestartStrategy(RestartStrategies.noRestart());
 		env1.getConfig().disableSysoutLogging();
@@ -1700,7 +1700,7 @@ public abstract class KafkaConsumerTestBase extends KafkaTestBase {
 			public void run() {
 				try {
 					// start job writing & reading data.
-					final StreamExecutionEnvironment env1 = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
+					final StreamExecutionEnvironment env1 = StreamExecutionEnvironment.getExecutionEnvironment();
 					env1.setParallelism(1);
 					env1.getConfig().setRestartStrategy(RestartStrategies.noRestart());
 					env1.getConfig().disableSysoutLogging();
@@ -1741,7 +1741,7 @@ public abstract class KafkaConsumerTestBase extends KafkaTestBase {
 					env1.execute("Metrics test job");
 				} catch(Throwable t) {
 					LOG.warn("Got exception during execution", t);
-					if(!(t.getCause() instanceof JobCancellationException)) { // we'll cancel the job
+					if(!(t instanceof JobCancellationException)) { // we'll cancel the job
 						error.f0 = t;
 					}
 				}
@@ -1994,7 +1994,7 @@ public abstract class KafkaConsumerTestBase extends KafkaTestBase {
 
 			createTestTopic(topicName, parallelism, replicationFactor);
 
-			StreamExecutionEnvironment writeEnv = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
+			StreamExecutionEnvironment writeEnv = StreamExecutionEnvironment.getExecutionEnvironment();
 			writeEnv.getConfig().setRestartStrategy(RestartStrategies.noRestart());
 			writeEnv.getConfig().disableSysoutLogging();
 			
@@ -2046,7 +2046,7 @@ public abstract class KafkaConsumerTestBase extends KafkaTestBase {
 
 			JobManagerCommunicationUtils.waitUntilNoJobIsRunning(flink.getLeaderGateway(timeout));
 			
-			final StreamExecutionEnvironment readEnv = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
+			final StreamExecutionEnvironment readEnv = StreamExecutionEnvironment.getExecutionEnvironment();
 			readEnv.getConfig().setRestartStrategy(RestartStrategies.noRestart());
 			readEnv.getConfig().disableSysoutLogging();
 			readEnv.setParallelism(parallelism);

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTestBase.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTestBase.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTestBase.java
index c925c8f..6f61392 100644
--- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTestBase.java
+++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTestBase.java
@@ -71,7 +71,7 @@ public abstract class KafkaProducerTestBase extends KafkaTestBase {
 
 			TypeInformation<Tuple2<Long, String>> longStringInfo = TypeInfoParser.parse("Tuple2<Long, String>");
 
-			StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
+			StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
 			env.setRestartStrategy(RestartStrategies.noRestart());
 			env.getConfig().disableSysoutLogging();
 

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaShortRetentionTestBase.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaShortRetentionTestBase.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaShortRetentionTestBase.java
index 954dc7d..f688660 100644
--- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaShortRetentionTestBase.java
+++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaShortRetentionTestBase.java
@@ -29,6 +29,7 @@ import org.apache.flink.streaming.api.datastream.DataStreamSource;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
 import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
+import org.apache.flink.streaming.util.TestStreamEnvironment;
 import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema;
 import org.apache.flink.streaming.util.serialization.KeyedSerializationSchemaWrapper;
 import org.apache.flink.streaming.util.serialization.SimpleStringSchema;
@@ -59,6 +60,12 @@ import static org.junit.Assert.fail;
 public class KafkaShortRetentionTestBase implements Serializable {
 	
 	protected static final Logger LOG = LoggerFactory.getLogger(KafkaShortRetentionTestBase.class);
+
+	protected static final int NUM_TMS = 1;
+
+	protected static final int TM_SLOTS = 8;
+
+	protected static final int PARALLELISM = NUM_TMS * TM_SLOTS;
 	
 	private static KafkaTestEnvironment kafkaServer;
 	private static Properties standardProps;
@@ -97,17 +104,21 @@ public class KafkaShortRetentionTestBase implements Serializable {
 		standardProps = kafkaServer.getStandardProperties();
 
 		// start also a re-usable Flink mini cluster
-		flinkConfig.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, 1);
-		flinkConfig.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 8);
+		flinkConfig.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, NUM_TMS);
+		flinkConfig.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, TM_SLOTS);
 		flinkConfig.setLong(TaskManagerOptions.MANAGED_MEMORY_SIZE, 16L);
 		flinkConfig.setString(ConfigConstants.RESTART_STRATEGY_FIXED_DELAY_DELAY, "0 s");
 
 		flink = new LocalFlinkMiniCluster(flinkConfig, false);
 		flink.start();
+
+		TestStreamEnvironment.setAsContext(flink, PARALLELISM);
 	}
 
 	@AfterClass
 	public static void shutDownServices() {
+		TestStreamEnvironment.unsetAsContext();
+
 		if (flink != null) {
 			flink.shutdown();
 		}
@@ -135,8 +146,7 @@ public class KafkaShortRetentionTestBase implements Serializable {
 		tprops.setProperty("retention.ms", "250");
 		kafkaServer.createTestTopic(topic, parallelism, 1, tprops);
 
-		final StreamExecutionEnvironment env =
-				StreamExecutionEnvironment.createRemoteEnvironment("localhost", flink.getLeaderRPCPort());
+		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
 		env.setParallelism(parallelism);
 		env.setRestartStrategy(RestartStrategies.noRestart()); // fail immediately
 		env.getConfig().disableSysoutLogging();

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java
index a21a239..1837af6 100644
--- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java
+++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java
@@ -26,6 +26,7 @@ import org.apache.flink.metrics.jmx.JMXReporter;
 import org.apache.flink.runtime.client.JobExecutionException;
 import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.util.TestStreamEnvironment;
 import org.apache.flink.test.util.SuccessException;
 import org.apache.flink.util.InstantiationUtil;
 import org.apache.flink.util.TestLogger;
@@ -40,7 +41,6 @@ import org.slf4j.LoggerFactory;
 
 import scala.concurrent.duration.FiniteDuration;
 
-import java.io.IOException;
 import java.util.Properties;
 import java.util.concurrent.TimeUnit;
 
@@ -65,14 +65,18 @@ public abstract class KafkaTestBase extends TestLogger {
 	
 	protected static final int NUMBER_OF_KAFKA_SERVERS = 3;
 
+	protected static final int NUM_TMS = 1;
+
+	protected static final int TM_SLOTS = 8;
+
+	protected static final int PARALLELISM = NUM_TMS * TM_SLOTS;
+
 	protected static String brokerConnectionStrings;
 
 	protected static Properties standardProps;
 	
 	protected static LocalFlinkMiniCluster flink;
 
-	protected static int flinkPort;
-
 	protected static FiniteDuration timeout = new FiniteDuration(10, TimeUnit.SECONDS);
 
 	protected static KafkaTestEnvironment kafkaServer;
@@ -87,7 +91,7 @@ public abstract class KafkaTestBase extends TestLogger {
 	// ------------------------------------------------------------------------
 	
 	@BeforeClass
-	public static void prepare() throws IOException, ClassNotFoundException {
+	public static void prepare() throws ClassNotFoundException {
 
 		LOG.info("-------------------------------------------------------------------------");
 		LOG.info("    Starting KafkaTestBase ");
@@ -95,6 +99,7 @@ public abstract class KafkaTestBase extends TestLogger {
 
 		startClusters(false);
 
+		TestStreamEnvironment.setAsContext(flink, PARALLELISM);
 	}
 
 	@AfterClass
@@ -104,6 +109,8 @@ public abstract class KafkaTestBase extends TestLogger {
 		LOG.info("    Shut down KafkaTestBase ");
 		LOG.info("-------------------------------------------------------------------------");
 
+		TestStreamEnvironment.unsetAsContext();
+
 		shutdownClusters();
 
 		LOG.info("-------------------------------------------------------------------------");
@@ -113,8 +120,8 @@ public abstract class KafkaTestBase extends TestLogger {
 
 	protected static Configuration getFlinkConfiguration() {
 		Configuration flinkConfig = new Configuration();
-		flinkConfig.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, 1);
-		flinkConfig.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 8);
+		flinkConfig.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, NUM_TMS);
+		flinkConfig.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, TM_SLOTS);
 		flinkConfig.setLong(TaskManagerOptions.MANAGED_MEMORY_SIZE, 16L);
 		flinkConfig.setString(ConfigConstants.RESTART_STRATEGY_FIXED_DELAY_DELAY, "0 s");
 		flinkConfig.setString(MetricOptions.REPORTERS_LIST, "my_reporter");
@@ -147,14 +154,10 @@ public abstract class KafkaTestBase extends TestLogger {
 		// start also a re-usable Flink mini cluster
 		flink = new LocalFlinkMiniCluster(getFlinkConfiguration(), false);
 		flink.start();
-
-		flinkPort = flink.getLeaderRPCPort();
-
 	}
 
 	protected static void shutdownClusters() {
 
-		flinkPort = -1;
 		if (flink != null) {
 			flink.shutdown();
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/flink-connectors/flink-connector-kafka-base/src/test/resources/log4j-test.properties
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-base/src/test/resources/log4j-test.properties b/flink-connectors/flink-connector-kafka-base/src/test/resources/log4j-test.properties
index 6bdfb48..16c226f 100644
--- a/flink-connectors/flink-connector-kafka-base/src/test/resources/log4j-test.properties
+++ b/flink-connectors/flink-connector-kafka-base/src/test/resources/log4j-test.properties
@@ -16,7 +16,7 @@
 # limitations under the License.
 ################################################################################
 
-log4j.rootLogger=INFO, testlogger
+log4j.rootLogger=OFF, testlogger
 
 log4j.appender.testlogger=org.apache.log4j.ConsoleAppender
 log4j.appender.testlogger.target = System.err

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkClient.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkClient.java b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkClient.java
index 2b7f357..21794f9 100644
--- a/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkClient.java
+++ b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkClient.java
@@ -205,7 +205,7 @@ public class FlinkClient {
 		final ClusterClient client;
 		try {
 			client = new StandaloneClusterClient(configuration);
-		} catch (final IOException e) {
+		} catch (final Exception e) {
 			throw new RuntimeException("Could not establish a connection to the job manager", e);
 		}
 
@@ -245,7 +245,7 @@ public class FlinkClient {
 		final ClusterClient client;
 		try {
 			client = new StandaloneClusterClient(configuration);
-		} catch (final IOException e) {
+		} catch (final Exception e) {
 			throw new RuntimeException("Could not establish a connection to the job manager", e);
 		}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/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 7a25636..f9b6a21 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,8 +22,9 @@ import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
 import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-
+import org.apache.flink.streaming.util.TestStreamEnvironment;
 import org.apache.flink.util.TestLogger;
+
 import org.junit.Test;
 
 import java.util.Iterator;
@@ -41,8 +42,9 @@ public class CollectITCase extends TestLogger {
 		try {
 			cluster.start();
 
-			final StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment(
-					"localhost", cluster.getLeaderRPCPort());
+			TestStreamEnvironment.setAsContext(cluster, 1);
+
+			final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
 	
 			final long N = 10;
 			DataStream<Long> stream = env.generateSequence(1, N);
@@ -57,6 +59,7 @@ public class CollectITCase extends TestLogger {
 			assertEquals("received wrong number of elements", N + 1, i);
 		}
 		finally {
+			TestStreamEnvironment.unsetAsContext();
 			cluster.stop();
 		}
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/DegreesWithExceptionITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/DegreesWithExceptionITCase.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/DegreesWithExceptionITCase.java
index ad434d4..111d421 100644
--- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/DegreesWithExceptionITCase.java
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/DegreesWithExceptionITCase.java
@@ -28,6 +28,7 @@ import org.apache.flink.graph.Graph;
 import org.apache.flink.graph.test.TestGraphUtils;
 import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
 
+import org.apache.flink.test.util.TestEnvironment;
 import org.apache.flink.types.LongValue;
 import org.apache.flink.util.TestLogger;
 import org.junit.AfterClass;
@@ -45,27 +46,19 @@ public class DegreesWithExceptionITCase extends TestLogger {
 
 	@BeforeClass
 	public static void setupCluster() {
-		try {
-			Configuration config = new Configuration();
-			config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, PARALLELISM);
-			cluster = new LocalFlinkMiniCluster(config, false);
-			cluster.start();
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail("Error starting test cluster: " + e.getMessage());
-		}
+		Configuration config = new Configuration();
+		config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, PARALLELISM);
+		cluster = new LocalFlinkMiniCluster(config, false);
+		cluster.start();
+
+		TestEnvironment.setAsContext(cluster, PARALLELISM);
 	}
 
 	@AfterClass
 	public static void tearDownCluster() {
-		try {
-			cluster.stop();
-		}
-		catch (Throwable t) {
-			t.printStackTrace();
-			fail("ClusterClient shutdown caused an exception: " + t.getMessage());
-		}
+		cluster.stop();
+
+		TestEnvironment.unsetAsContext();
 	}
 
 	/**
@@ -74,8 +67,7 @@ public class DegreesWithExceptionITCase extends TestLogger {
 	@Test
 	public void testOutDegreesInvalidEdgeSrcId() throws Exception {
 
-		final ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment(
-				"localhost", cluster.getLeaderRPCPort());
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
 		env.setParallelism(PARALLELISM);
 		env.getConfig().disableSysoutLogging();
 		
@@ -98,8 +90,7 @@ public class DegreesWithExceptionITCase extends TestLogger {
 	@Test
 	public void testInDegreesInvalidEdgeTrgId() throws Exception {
 
-		final ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment(
-				"localhost", cluster.getLeaderRPCPort());
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
 		env.setParallelism(PARALLELISM);
 		env.getConfig().disableSysoutLogging();
 
@@ -122,8 +113,7 @@ public class DegreesWithExceptionITCase extends TestLogger {
 	@Test
 	public void testGetDegreesInvalidEdgeTrgId() throws Exception {
 
-		final ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment(
-				"localhost", cluster.getLeaderRPCPort());
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
 		env.setParallelism(PARALLELISM);
 		env.getConfig().disableSysoutLogging();
 
@@ -146,8 +136,7 @@ public class DegreesWithExceptionITCase extends TestLogger {
 	@Test
 	public void testGetDegreesInvalidEdgeSrcId() throws Exception {
 
-		final ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment(
-				"localhost", cluster.getLeaderRPCPort());
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
 		env.setParallelism(PARALLELISM);
 		env.getConfig().disableSysoutLogging();
 
@@ -170,8 +159,7 @@ public class DegreesWithExceptionITCase extends TestLogger {
 	@Test
 	public void testGetDegreesInvalidEdgeSrcTrgId() throws Exception {
 
-		final ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment(
-				"localhost", cluster.getLeaderRPCPort());
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
 		env.setParallelism(PARALLELISM);
 		env.getConfig().disableSysoutLogging();
 

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnEdgesWithExceptionITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnEdgesWithExceptionITCase.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnEdgesWithExceptionITCase.java
index d090d3c..7a0a30c 100644
--- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnEdgesWithExceptionITCase.java
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnEdgesWithExceptionITCase.java
@@ -31,6 +31,7 @@ import org.apache.flink.graph.Graph;
 import org.apache.flink.graph.Vertex;
 import org.apache.flink.graph.test.TestGraphUtils;
 import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
+import org.apache.flink.test.util.TestEnvironment;
 import org.apache.flink.util.Collector;
 import org.apache.flink.util.TestLogger;
 import org.junit.AfterClass;
@@ -48,27 +49,19 @@ public class ReduceOnEdgesWithExceptionITCase extends TestLogger {
 
 	@BeforeClass
 	public static void setupCluster() {
-		try {
-			Configuration config = new Configuration();
-			config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, PARALLELISM);
-			cluster = new LocalFlinkMiniCluster(config, false);
-			cluster.start();
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail("Error starting test cluster: " + e.getMessage());
-		}
+		Configuration config = new Configuration();
+		config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, PARALLELISM);
+		cluster = new LocalFlinkMiniCluster(config, false);
+		cluster.start();
+
+		TestEnvironment.setAsContext(cluster, PARALLELISM);
 	}
 
 	@AfterClass
 	public static void tearDownCluster() {
-		try {
-			cluster.stop();
-		}
-		catch (Throwable t) {
-			t.printStackTrace();
-			fail("ClusterClient shutdown caused an exception: " + t.getMessage());
-		}
+		cluster.stop();
+
+		TestEnvironment.unsetAsContext();
 	}
 
 	/**
@@ -77,8 +70,7 @@ public class ReduceOnEdgesWithExceptionITCase extends TestLogger {
 	@Test
 	public void testGroupReduceOnEdgesInvalidEdgeSrcId() throws Exception {
 
-		final ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment(
-				"localhost", cluster.getLeaderRPCPort());
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
 		env.setParallelism(PARALLELISM);
 		env.getConfig().disableSysoutLogging();
 
@@ -91,6 +83,8 @@ public class ReduceOnEdgesWithExceptionITCase extends TestLogger {
 
 			verticesWithAllNeighbors.output(new DiscardingOutputFormat<Tuple2<Long, Long>>());
 			env.execute();
+
+			fail("Expected an exception.");
 		} catch (Exception e) {
 			// We expect the job to fail with an exception
 		}
@@ -102,8 +96,7 @@ public class ReduceOnEdgesWithExceptionITCase extends TestLogger {
 	@Test
 	public void testGroupReduceOnEdgesInvalidEdgeTrgId() throws Exception {
 
-		final ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment(
-				"localhost", cluster.getLeaderRPCPort());
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
 		env.setParallelism(PARALLELISM);
 		env.getConfig().disableSysoutLogging();
 
@@ -116,6 +109,8 @@ public class ReduceOnEdgesWithExceptionITCase extends TestLogger {
 
 			verticesWithAllNeighbors.output(new DiscardingOutputFormat<Tuple2<Long, Long>>());
 			env.execute();
+
+			fail("Expected an exception.");
 		} catch (Exception e) {
 			// We expect the job to fail with an exception
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnNeighborsWithExceptionITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnNeighborsWithExceptionITCase.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnNeighborsWithExceptionITCase.java
index afe2e18..b337bca 100644
--- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnNeighborsWithExceptionITCase.java
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnNeighborsWithExceptionITCase.java
@@ -32,6 +32,7 @@ import org.apache.flink.graph.ReduceNeighborsFunction;
 import org.apache.flink.graph.Vertex;
 import org.apache.flink.graph.test.TestGraphUtils;
 import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
+import org.apache.flink.test.util.TestEnvironment;
 import org.apache.flink.util.Collector;
 import org.apache.flink.util.TestLogger;
 import org.junit.AfterClass;
@@ -49,27 +50,19 @@ public class ReduceOnNeighborsWithExceptionITCase extends TestLogger {
 
 	@BeforeClass
 	public static void setupCluster() {
-		try {
-			Configuration config = new Configuration();
-			config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, PARALLELISM);
-			cluster = new LocalFlinkMiniCluster(config, false);
-			cluster.start();
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail("Error starting test cluster: " + e.getMessage());
-		}
+		Configuration config = new Configuration();
+		config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, PARALLELISM);
+		cluster = new LocalFlinkMiniCluster(config, false);
+		cluster.start();
+
+		TestEnvironment.setAsContext(cluster, PARALLELISM);
 	}
 
 	@AfterClass
 	public static void tearDownCluster() {
-		try {
-			cluster.stop();
-		}
-		catch (Throwable t) {
-			t.printStackTrace();
-			fail("ClusterClient shutdown caused an exception: " + t.getMessage());
-		}
+		cluster.stop();
+
+		TestEnvironment.unsetAsContext();
 	}
 
 	/**
@@ -79,8 +72,7 @@ public class ReduceOnNeighborsWithExceptionITCase extends TestLogger {
 	@Test
 	public void testGroupReduceOnNeighborsWithVVInvalidEdgeSrcId() throws Exception {
 
-		final ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment(
-				"localhost", cluster.getLeaderRPCPort());
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
 		env.setParallelism(PARALLELISM);
 		env.getConfig().disableSysoutLogging();
 
@@ -93,6 +85,8 @@ public class ReduceOnNeighborsWithExceptionITCase extends TestLogger {
 
 			verticesWithSumOfOutNeighborValues.output(new DiscardingOutputFormat<Tuple2<Long, Long>>());
 			env.execute();
+
+			fail("Expected an exception.");
 		} catch (Exception e) {
 			// We expect the job to fail with an exception
 		}
@@ -105,8 +99,7 @@ public class ReduceOnNeighborsWithExceptionITCase extends TestLogger {
 	@Test
 	public void testGroupReduceOnNeighborsWithVVInvalidEdgeTrgId() throws Exception {
 
-		final ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment(
-				"localhost", cluster.getLeaderRPCPort());
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
 		env.setParallelism(PARALLELISM);
 		env.getConfig().disableSysoutLogging();
 
@@ -119,6 +112,8 @@ public class ReduceOnNeighborsWithExceptionITCase extends TestLogger {
 
 			verticesWithSumOfOutNeighborValues.output(new DiscardingOutputFormat<Tuple2<Long, Long>>());
 			env.execute();
+
+			fail("Expected an exception.");
 		} catch (Exception e) {
 			// We expect the job to fail with an exception
 		}
@@ -131,8 +126,7 @@ public class ReduceOnNeighborsWithExceptionITCase extends TestLogger {
 	@Test
 	public void testGroupReduceOnNeighborsInvalidEdgeSrcId() throws Exception {
 
-		final ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment(
-				"localhost", cluster.getLeaderRPCPort());
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
 		env.setParallelism(PARALLELISM);
 		env.getConfig().disableSysoutLogging();
 
@@ -157,8 +151,7 @@ public class ReduceOnNeighborsWithExceptionITCase extends TestLogger {
 	@Test
 	public void testGroupReduceOnNeighborsInvalidEdgeTrgId() throws Exception {
 
-		final ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment(
-				"localhost", cluster.getLeaderRPCPort());
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
 		env.setParallelism(PARALLELISM);
 		env.getConfig().disableSysoutLogging();
 

http://git-wip-us.apache.org/repos/asf/flink/blob/ddd6a99a/flink-libraries/flink-ml/src/test/scala/org/apache/flink/ml/util/FlinkTestBase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-ml/src/test/scala/org/apache/flink/ml/util/FlinkTestBase.scala b/flink-libraries/flink-ml/src/test/scala/org/apache/flink/ml/util/FlinkTestBase.scala
index 6353d6a..3ee7a99 100644
--- a/flink-libraries/flink-ml/src/test/scala/org/apache/flink/ml/util/FlinkTestBase.scala
+++ b/flink-libraries/flink-ml/src/test/scala/org/apache/flink/ml/util/FlinkTestBase.scala
@@ -62,7 +62,7 @@ trait FlinkTestBase extends BeforeAndAfter {
       false,
       true)
 
-    val clusterEnvironment = new TestEnvironment(cl, parallelism)
+    val clusterEnvironment = new TestEnvironment(cl, parallelism, false)
     clusterEnvironment.setAsContext()
 
     cluster = Some(cl)