You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by se...@apache.org on 2016/10/13 15:40:11 UTC

[1/8] flink git commit: [FLINK-4375] [distributed coordination] Implement new JobManager creation, initialization, and basic RPC methods

Repository: flink
Updated Branches:
  refs/heads/flip-6 35a44daa6 -> 48c936eed


http://git-wip-us.apache.org/repos/asf/flink/blob/34fef475/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 a2716e5..9f9234f 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
@@ -18,13 +18,13 @@
 
 package org.apache.flink.runtime.taskexecutor;
 
+import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.runtime.accumulators.AccumulatorSnapshot;
 import org.apache.flink.runtime.blob.BlobCache;
 import org.apache.flink.runtime.broadcast.BroadcastVariableManager;
 import org.apache.flink.runtime.clusterframework.types.AllocationID;
 import org.apache.flink.runtime.clusterframework.types.ResourceID;
 import org.apache.flink.runtime.clusterframework.types.SlotID;
-import org.apache.flink.runtime.instance.InstanceID;
 import org.apache.flink.runtime.concurrent.ApplyFunction;
 import org.apache.flink.runtime.concurrent.Future;
 import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor;
@@ -33,20 +33,29 @@ import org.apache.flink.runtime.execution.librarycache.LibraryCacheManager;
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
 import org.apache.flink.runtime.executiongraph.PartitionInfo;
 import org.apache.flink.runtime.filecache.FileCache;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.io.disk.iomanager.IOManager;
+import org.apache.flink.runtime.io.network.NetworkEnvironment;
 import org.apache.flink.runtime.io.network.netty.PartitionStateChecker;
 import org.apache.flink.runtime.io.network.partition.ResultPartitionConsumableNotifier;
 import org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate;
 import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
 import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider;
 import org.apache.flink.runtime.jobmaster.JobMasterGateway;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener;
+import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.runtime.messages.Acknowledge;
 import org.apache.flink.runtime.metrics.MetricRegistry;
-import org.apache.flink.runtime.resourcemanager.messages.taskexecutor.TMSlotRequestRegistered;
-import org.apache.flink.runtime.resourcemanager.messages.taskexecutor.TMSlotRequestRejected;
 import org.apache.flink.runtime.metrics.groups.TaskManagerMetricGroup;
 import org.apache.flink.runtime.metrics.groups.TaskMetricGroup;
+import org.apache.flink.runtime.resourcemanager.messages.taskexecutor.TMSlotRequestRegistered;
+import org.apache.flink.runtime.resourcemanager.messages.taskexecutor.TMSlotRequestRejected;
 import org.apache.flink.runtime.resourcemanager.messages.taskexecutor.TMSlotRequestReply;
 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.taskexecutor.exceptions.CheckpointException;
 import org.apache.flink.runtime.taskexecutor.exceptions.PartitionException;
 import org.apache.flink.runtime.taskexecutor.exceptions.TaskException;
@@ -60,26 +69,16 @@ import org.apache.flink.runtime.taskmanager.Task;
 import org.apache.flink.runtime.taskmanager.TaskExecutionState;
 import org.apache.flink.runtime.taskmanager.TaskManagerActions;
 import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
-import org.apache.flink.annotation.VisibleForTesting;
-import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
-import org.apache.flink.runtime.io.disk.iomanager.IOManager;
-import org.apache.flink.runtime.io.network.NetworkEnvironment;
-import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener;
-import org.apache.flink.runtime.memory.MemoryManager;
-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.util.Preconditions;
 
-import java.util.HashSet;
-import java.util.Set;
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.util.Collection;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.Iterator;
 import java.util.Map;
+import java.util.Set;
 import java.util.UUID;
 
 import static org.apache.flink.util.Preconditions.checkArgument;
@@ -276,11 +275,12 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 		TaskMetricGroup taskMetricGroup = taskManagerMetricGroup.addTaskForJob(tdd);
 
 		InputSplitProvider inputSplitProvider = new RpcInputSplitProvider(
-			jobManagerConnection.getJobManagerGateway(),
-			tdd.getJobID(),
-			tdd.getVertexID(),
-			tdd.getExecutionId(),
-			taskManagerConfiguration.getTimeout());
+				jobManagerConnection.getJobMasterLeaderId(),
+				jobManagerConnection.getJobManagerGateway(),
+				tdd.getJobID(),
+				tdd.getVertexID(),
+				tdd.getExecutionId(),
+				taskManagerConfiguration.getTimeout());
 
 		TaskManagerActions taskManagerActions = jobManagerConnection.getTaskManagerActions();
 		CheckpointResponder checkpointResponder = jobManagerConnection.getCheckpointResponder();
@@ -580,10 +580,15 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 		clearTasks();
 	}
 
-	private void updateTaskExecutionState(final JobMasterGateway jobMasterGateway, final TaskExecutionState taskExecutionState) {
+	private void updateTaskExecutionState(
+			final UUID jobMasterLeaderId,
+			final JobMasterGateway jobMasterGateway,
+			final TaskExecutionState taskExecutionState)
+	{
 		final ExecutionAttemptID executionAttemptID = taskExecutionState.getID();
 
-		Future<Acknowledge> futureAcknowledge = jobMasterGateway.updateTaskExecutionState(taskExecutionState);
+		Future<Acknowledge> futureAcknowledge = jobMasterGateway.updateTaskExecutionState(
+				jobMasterLeaderId, taskExecutionState);
 
 		futureAcknowledge.exceptionallyAsync(new ApplyFunction<Throwable, Void>() {
 			@Override
@@ -595,7 +600,11 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 		}, getMainThreadExecutor());
 	}
 
-	private void unregisterTaskAndNotifyFinalState(final JobMasterGateway jobMasterGateway, ExecutionAttemptID executionAttemptID) {
+	private void unregisterTaskAndNotifyFinalState(
+			final UUID jobMasterLeaderId,
+			final JobMasterGateway jobMasterGateway,
+			final ExecutionAttemptID executionAttemptID)
+	{
 		Task task = removeTask(executionAttemptID);
 
 		if (task != null) {
@@ -613,13 +622,14 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 			AccumulatorSnapshot accumulatorSnapshot = task.getAccumulatorRegistry().getSnapshot();
 
 			updateTaskExecutionState(
-				jobMasterGateway,
-				new TaskExecutionState(
-					task.getJobID(),
-					task.getExecutionId(),
-					task.getExecutionState(),
-					task.getFailureCause(),
-					accumulatorSnapshot));
+					jobMasterLeaderId,
+					jobMasterGateway,
+					new TaskExecutionState(
+							task.getJobID(),
+							task.getExecutionId(),
+							task.getExecutionState(),
+							task.getFailureCause(),
+							accumulatorSnapshot));
 		} else {
 			log.error("Cannot find task with ID {} to unregister.", executionAttemptID);
 		}
@@ -661,11 +671,14 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 		}
 	}
 
-	private JobManagerConnection associateWithJobManager(JobMasterGateway jobMasterGateway, int blobPort) {
+	private JobManagerConnection associateWithJobManager(UUID jobMasterLeaderId,
+			JobMasterGateway jobMasterGateway, int blobPort)
+	{
+		Preconditions.checkNotNull(jobMasterLeaderId);
 		Preconditions.checkNotNull(jobMasterGateway);
 		Preconditions.checkArgument(blobPort > 0 || blobPort <= 65535, "Blob port is out of range.");
 
-		TaskManagerActions taskManagerActions = new TaskManagerActionsImpl(jobMasterGateway);
+		TaskManagerActions taskManagerActions = new TaskManagerActionsImpl(jobMasterLeaderId, jobMasterGateway);
 
 		CheckpointResponder checkpointResponder = new RpcCheckpointResponder(jobMasterGateway);
 
@@ -678,19 +691,21 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 			taskManagerConfiguration.getCleanupInterval());
 
 		ResultPartitionConsumableNotifier resultPartitionConsumableNotifier = new RpcResultPartitionConsumableNotifier(
-			jobMasterGateway,
-			getRpcService().getExecutor(),
-			taskManagerConfiguration.getTimeout());
+				jobMasterLeaderId,
+				jobMasterGateway,
+				getRpcService().getExecutor(),
+				taskManagerConfiguration.getTimeout());
 
-		PartitionStateChecker partitionStateChecker = new RpcPartitionStateChecker(jobMasterGateway);
+		PartitionStateChecker partitionStateChecker = new RpcPartitionStateChecker(jobMasterLeaderId, jobMasterGateway);
 
 		return new JobManagerConnection(
-			jobMasterGateway,
-			taskManagerActions,
-			checkpointResponder,
-			libraryCacheManager,
-			resultPartitionConsumableNotifier,
-			partitionStateChecker);
+				jobMasterLeaderId,
+				jobMasterGateway,
+				taskManagerActions,
+				checkpointResponder,
+				libraryCacheManager,
+				resultPartitionConsumableNotifier,
+				partitionStateChecker);
 	}
 
 	private void disassociateFromJobManager(JobManagerConnection jobManagerConnection) throws IOException {
@@ -782,9 +797,11 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 	}
 
 	private class TaskManagerActionsImpl implements TaskManagerActions {
+		private final UUID jobMasterLeaderId;
 		private final JobMasterGateway jobMasterGateway;
 
-		private TaskManagerActionsImpl(JobMasterGateway jobMasterGateway) {
+		private TaskManagerActionsImpl(UUID jobMasterLeaderId, JobMasterGateway jobMasterGateway) {
+			this.jobMasterLeaderId = Preconditions.checkNotNull(jobMasterLeaderId);
 			this.jobMasterGateway = Preconditions.checkNotNull(jobMasterGateway);
 		}
 
@@ -793,7 +810,7 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 			runAsync(new Runnable() {
 				@Override
 				public void run() {
-					unregisterTaskAndNotifyFinalState(jobMasterGateway, executionAttemptID);
+					unregisterTaskAndNotifyFinalState(jobMasterLeaderId, jobMasterGateway, executionAttemptID);
 				}
 			});
 		}
@@ -816,7 +833,7 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 
 		@Override
 		public void updateTaskExecutionState(final TaskExecutionState taskExecutionState) {
-			TaskExecutor.this.updateTaskExecutionState(jobMasterGateway, taskExecutionState);
+			TaskExecutor.this.updateTaskExecutionState(jobMasterLeaderId, jobMasterGateway, taskExecutionState);
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/34fef475/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcInputSplitProvider.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcInputSplitProvider.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcInputSplitProvider.java
index 4850d63..3b9da48 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcInputSplitProvider.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcInputSplitProvider.java
@@ -31,7 +31,10 @@ import org.apache.flink.runtime.jobmaster.SerializedInputSplit;
 import org.apache.flink.util.InstantiationUtil;
 import org.apache.flink.util.Preconditions;
 
+import java.util.UUID;
+
 public class RpcInputSplitProvider implements InputSplitProvider {
+	private final UUID jobMasterLeaderId;
 	private final JobMasterGateway jobMasterGateway;
 	private final JobID jobID;
 	private final JobVertexID jobVertexID;
@@ -39,11 +42,13 @@ public class RpcInputSplitProvider implements InputSplitProvider {
 	private final Time timeout;
 
 	public RpcInputSplitProvider(
+			UUID jobMasterLeaderId,
 			JobMasterGateway jobMasterGateway,
 			JobID jobID,
 			JobVertexID jobVertexID,
 			ExecutionAttemptID executionAttemptID,
 			Time timeout) {
+		this.jobMasterLeaderId = Preconditions.checkNotNull(jobMasterLeaderId);
 		this.jobMasterGateway = Preconditions.checkNotNull(jobMasterGateway);
 		this.jobID = Preconditions.checkNotNull(jobID);
 		this.jobVertexID = Preconditions.checkNotNull(jobVertexID);
@@ -56,7 +61,8 @@ public class RpcInputSplitProvider implements InputSplitProvider {
 	public InputSplit getNextInputSplit(ClassLoader userCodeClassLoader) throws InputSplitProviderException {
 		Preconditions.checkNotNull(userCodeClassLoader);
 
-		Future<SerializedInputSplit> futureInputSplit = jobMasterGateway.requestNextInputSplit(jobVertexID, executionAttemptID);
+		Future<SerializedInputSplit> futureInputSplit = jobMasterGateway.requestNextInputSplit(
+				jobMasterLeaderId, jobVertexID, executionAttemptID);
 
 		try {
 			SerializedInputSplit serializedInputSplit = futureInputSplit.get(timeout.getSize(), timeout.getUnit());

http://git-wip-us.apache.org/repos/asf/flink/blob/34fef475/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcPartitionStateChecker.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcPartitionStateChecker.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcPartitionStateChecker.java
index ab111ad..1c91b87 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcPartitionStateChecker.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcPartitionStateChecker.java
@@ -28,11 +28,15 @@ import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
 import org.apache.flink.runtime.jobmaster.JobMasterGateway;
 import org.apache.flink.util.Preconditions;
 
+import java.util.UUID;
+
 public class RpcPartitionStateChecker implements PartitionStateChecker {
 
+	private final UUID jobMasterLeaderId;
 	private final JobMasterGateway jobMasterGateway;
 
-	public RpcPartitionStateChecker(JobMasterGateway jobMasterGateway) {
+	public RpcPartitionStateChecker(UUID jobMasterLeaderId, JobMasterGateway jobMasterGateway) {
+		this.jobMasterLeaderId = Preconditions.checkNotNull(jobMasterLeaderId);
 		this.jobMasterGateway = Preconditions.checkNotNull(jobMasterGateway);
 	}
 
@@ -43,6 +47,6 @@ public class RpcPartitionStateChecker implements PartitionStateChecker {
 		IntermediateDataSetID resultId,
 		ResultPartitionID partitionId) {
 
-		return jobMasterGateway.requestPartitionState(partitionId, executionId, resultId);
+		return jobMasterGateway.requestPartitionState(jobMasterLeaderId, partitionId, executionId, resultId);
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/34fef475/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcResultPartitionConsumableNotifier.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcResultPartitionConsumableNotifier.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcResultPartitionConsumableNotifier.java
index 29ad3b6..cf01d5a 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcResultPartitionConsumableNotifier.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcResultPartitionConsumableNotifier.java
@@ -31,27 +31,32 @@ import org.apache.flink.util.Preconditions;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.util.UUID;
 import java.util.concurrent.Executor;
 
 public class RpcResultPartitionConsumableNotifier implements ResultPartitionConsumableNotifier {
 
 	private static final Logger LOG = LoggerFactory.getLogger(RpcResultPartitionConsumableNotifier.class);
 
+	private final UUID jobMasterLeaderId;
 	private final JobMasterGateway jobMasterGateway;
 	private final Executor executor;
 	private final Time timeout;
 
 	public RpcResultPartitionConsumableNotifier(
+			UUID jobMasterLeaderId,
 			JobMasterGateway jobMasterGateway,
 			Executor executor,
 			Time timeout) {
+		this.jobMasterLeaderId = Preconditions.checkNotNull(jobMasterLeaderId);
 		this.jobMasterGateway = Preconditions.checkNotNull(jobMasterGateway);
 		this.executor = Preconditions.checkNotNull(executor);
 		this.timeout = Preconditions.checkNotNull(timeout);
 	}
 	@Override
 	public void notifyPartitionConsumable(JobID jobId, ResultPartitionID partitionId, final TaskActions taskActions) {
-		Future<Acknowledge> acknowledgeFuture = jobMasterGateway.scheduleOrUpdateConsumers(partitionId, timeout);
+		Future<Acknowledge> acknowledgeFuture = jobMasterGateway.scheduleOrUpdateConsumers(
+				jobMasterLeaderId, partitionId, timeout);
 
 		acknowledgeFuture.exceptionallyAsync(new ApplyFunction<Throwable, Void>() {
 			@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/34fef475/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 84f5ac7..9209d15 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
@@ -581,4 +581,10 @@ object AkkaUtils {
         throw new Exception(s"Could not retrieve InetSocketAddress from Akka URL $akkaURL")
     }
   }
+
+  def formatDurationParingErrorMessage: String = {
+    "Duration format must be \"val unit\", where 'val' is a number and 'unit' is " + 
+      "(d|day)|(h|hour)|(min|minute)|s|sec|second)|(ms|milli|millisecond)|"+
+      "(�s|micro|microsecond)|(ns|nano|nanosecond)"
+  }
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/34fef475/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 faf69cc..a255027 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
@@ -19,11 +19,15 @@
 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.highavailability.nonha.NonHaRegistry;
 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.concurrent.ConcurrentHashMap;
 
 /**
@@ -140,4 +144,14 @@ public class TestingHighAvailabilityServices implements HighAvailabilityServices
 
 		}
 	}
+
+	@Override
+	public RunningJobsRegistry getRunningJobsRegistry() throws Exception {
+		return new NonHaRegistry();
+	}
+
+	@Override
+	public BlobStore createBlobStore() throws IOException {
+		return new VoidBlobStore();
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/34fef475/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 30dfef5..f709cbd 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
@@ -21,14 +21,21 @@ package org.apache.flink.runtime.jobmaster;
 import org.apache.flink.api.common.JobExecutionResult;
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.blob.BlobStore;
 import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
+import org.apache.flink.runtime.highavailability.RunningJobsRegistry;
 import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobgraph.JobVertex;
 import org.apache.flink.runtime.jobmanager.OnCompletionActions;
 import org.apache.flink.runtime.jobmanager.SubmittedJobGraphStore;
 import org.apache.flink.runtime.leaderelection.LeaderElectionService;
+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.junit.After;
 import org.junit.Before;
+import org.junit.Ignore;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.powermock.api.mockito.PowerMockito;
@@ -61,11 +68,19 @@ public class JobManagerRunnerMockTest {
 
 	private TestingOnCompletionActions jobCompletion;
 
+	private BlobStore blobStore;
+
+	private RunningJobsRegistry runningJobsRegistry;
+
 	@Before
 	public void setUp() throws Exception {
+		RpcService mockRpc = mock(RpcService.class);
+		when(mockRpc.getAddress()).thenReturn("localhost");
+
 		jobManager = mock(JobMaster.class);
 		jobManagerGateway = mock(JobMasterGateway.class);
 		when(jobManager.getSelf()).thenReturn(jobManagerGateway);
+		when(jobManager.getRpcService()).thenReturn(mockRpc);
 
 		PowerMockito.whenNew(JobMaster.class).withAnyArguments().thenReturn(jobManager);
 
@@ -74,19 +89,25 @@ public class JobManagerRunnerMockTest {
 		leaderElectionService = mock(LeaderElectionService.class);
 		when(leaderElectionService.hasLeadership()).thenReturn(true);
 
-		submittedJobGraphStore = mock(SubmittedJobGraphStore.class);
-		when(submittedJobGraphStore.contains(any(JobID.class))).thenReturn(true);
+		runningJobsRegistry = mock(RunningJobsRegistry.class);
+		when(runningJobsRegistry.isJobRunning(any(JobID.class))).thenReturn(true);
 
+		blobStore = mock(BlobStore.class);
+		
 		HighAvailabilityServices haServices = mock(HighAvailabilityServices.class);
 		when(haServices.getJobManagerLeaderElectionService(any(JobID.class))).thenReturn(leaderElectionService);
 		when(haServices.getSubmittedJobGraphStore()).thenReturn(submittedJobGraphStore);
+		when(haServices.createBlobStore()).thenReturn(blobStore);
+		when(haServices.getRunningJobsRegistry()).thenReturn(runningJobsRegistry);
 
 		runner = PowerMockito.spy(new JobManagerRunner(
-			new JobGraph("test"),
+			new JobGraph("test", new JobVertex("vertex")),
 			mock(Configuration.class),
-			mock(RpcService.class),
+			mockRpc,
 			haServices,
-			mock(JobManagerServices.class),
+			JobManagerServices.fromConfiguration(new Configuration(), haServices),
+			new MetricRegistry(MetricRegistryConfiguration.defaultMetricRegistryConfiguration()),
+			jobCompletion,
 			jobCompletion));
 	}
 
@@ -94,25 +115,26 @@ public class JobManagerRunnerMockTest {
 	public void tearDown() throws Exception {
 	}
 
+	@Ignore
 	@Test
 	public void testStartAndShutdown() throws Exception {
 		runner.start();
-		verify(jobManager).init();
-		verify(jobManager).start();
 		verify(leaderElectionService).start(runner);
 
 		assertTrue(!jobCompletion.isJobFinished());
 		assertTrue(!jobCompletion.isJobFailed());
 
+		verify(jobManager).start(any(UUID.class));
+		
 		runner.shutdown();
 		verify(leaderElectionService).stop();
 		verify(jobManager).shutDown();
 	}
 
+	@Ignore
 	@Test
 	public void testShutdownBeforeGrantLeadership() throws Exception {
 		runner.start();
-		verify(jobManager).init();
 		verify(jobManager).start();
 		verify(leaderElectionService).start(runner);
 
@@ -129,13 +151,14 @@ public class JobManagerRunnerMockTest {
 
 	}
 
+	@Ignore
 	@Test
 	public void testJobFinished() throws Exception {
 		runner.start();
 
 		UUID leaderSessionID = UUID.randomUUID();
 		runner.grantLeadership(leaderSessionID);
-		verify(jobManagerGateway).startJob(leaderSessionID);
+		verify(jobManager).start(leaderSessionID);
 		assertTrue(!jobCompletion.isJobFinished());
 
 		// runner been told by JobManager that job is finished
@@ -148,13 +171,14 @@ public class JobManagerRunnerMockTest {
 		assertTrue(runner.isShutdown());
 	}
 
+	@Ignore
 	@Test
 	public void testJobFailed() throws Exception {
 		runner.start();
 
 		UUID leaderSessionID = UUID.randomUUID();
 		runner.grantLeadership(leaderSessionID);
-		verify(jobManagerGateway).startJob(leaderSessionID);
+		verify(jobManager).start(leaderSessionID);
 		assertTrue(!jobCompletion.isJobFinished());
 
 		// runner been told by JobManager that job is failed
@@ -166,39 +190,41 @@ public class JobManagerRunnerMockTest {
 		assertTrue(runner.isShutdown());
 	}
 
+	@Ignore
 	@Test
 	public void testLeadershipRevoked() throws Exception {
 		runner.start();
 
 		UUID leaderSessionID = UUID.randomUUID();
 		runner.grantLeadership(leaderSessionID);
-		verify(jobManagerGateway).startJob(leaderSessionID);
+		verify(jobManager).start(leaderSessionID);
 		assertTrue(!jobCompletion.isJobFinished());
 
 		runner.revokeLeadership();
-		verify(jobManagerGateway).suspendJob(any(Throwable.class));
+		verify(jobManager).suspendExecution(any(Throwable.class));
 		assertFalse(runner.isShutdown());
 	}
 
+	@Ignore
 	@Test
 	public void testRegainLeadership() throws Exception {
 		runner.start();
 
 		UUID leaderSessionID = UUID.randomUUID();
 		runner.grantLeadership(leaderSessionID);
-		verify(jobManagerGateway).startJob(leaderSessionID);
+		verify(jobManager).start(leaderSessionID);
 		assertTrue(!jobCompletion.isJobFinished());
 
 		runner.revokeLeadership();
-		verify(jobManagerGateway).suspendJob(any(Throwable.class));
+		verify(jobManager).suspendExecution(any(Throwable.class));
 		assertFalse(runner.isShutdown());
 
 		UUID leaderSessionID2 = UUID.randomUUID();
 		runner.grantLeadership(leaderSessionID2);
-		verify(jobManagerGateway).startJob(leaderSessionID2);
+		verify(jobManager).start(leaderSessionID2);
 	}
 
-	private static class TestingOnCompletionActions implements OnCompletionActions {
+	private static class TestingOnCompletionActions implements OnCompletionActions, FatalErrorHandler {
 
 		private volatile JobExecutionResult result;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/34fef475/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobManagerRunnerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobManagerRunnerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobManagerRunnerTest.java
new file mode 100644
index 0000000..174422f
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobManagerRunnerTest.java
@@ -0,0 +1,24 @@
+/*
+ * 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.jobmaster;
+
+public class JobManagerRunnerTest {
+	
+	// TODO: Test that 
+}


[3/8] flink git commit: [FLINK-4375] [distributed coordination] Implement new JobManager creation, initialization, and basic RPC methods

Posted by se...@apache.org.
[FLINK-4375] [distributed coordination] Implement new JobManager creation, initialization, and basic RPC methods


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

Branch: refs/heads/flip-6
Commit: 34fef4752e94b3d0c7afe7a9525799bb651a07b4
Parents: c8dc074
Author: Kurt Young <yk...@gmail.com>
Authored: Wed Oct 12 23:25:16 2016 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Thu Oct 13 16:25:49 2016 +0200

----------------------------------------------------------------------
 .../org/apache/flink/util/ExceptionUtils.java   |  20 +
 .../java/org/apache/flink/util/StringUtils.java |  14 +
 .../apache/flink/runtime/blob/BlobServer.java   |  65 +-
 .../apache/flink/runtime/blob/BlobStore.java    |   3 +-
 .../apache/flink/runtime/blob/BlobUtils.java    |   8 +-
 .../flink/runtime/blob/FileSystemBlobStore.java |  53 +-
 .../flink/runtime/blob/VoidBlobStore.java       |   2 +-
 .../CheckpointCoordinatorGateway.java           |  26 +-
 .../HighAvailabilityServices.java               |  13 +
 .../runtime/highavailability/NonHaServices.java |  13 +
 .../highavailability/RunningJobsRegistry.java   |  66 ++
 .../highavailability/ZookeeperHaServices.java   | 115 ++-
 .../highavailability/nonha/NonHaRegistry.java   |  62 ++
 .../runtime/jobmanager/OnCompletionActions.java |   3 +-
 .../runtime/jobmaster/JobManagerRunner.java     | 270 ++++--
 .../runtime/jobmaster/JobManagerServices.java   |  93 +-
 .../flink/runtime/jobmaster/JobMaster.java      | 902 +++++++------------
 .../runtime/jobmaster/JobMasterGateway.java     | 114 +--
 .../jobmaster/MiniClusterJobDispatcher.java     |  61 +-
 .../message/DisposeSavepointResponse.java       |  49 -
 .../message/TriggerSavepointResponse.java       |  74 --
 .../apache/flink/runtime/rpc/RpcService.java    |   4 +-
 .../taskexecutor/JobManagerConnection.java      |  25 +-
 .../runtime/taskexecutor/TaskExecutor.java      | 107 ++-
 .../taskexecutor/rpc/RpcInputSplitProvider.java |   8 +-
 .../rpc/RpcPartitionStateChecker.java           |   8 +-
 .../RpcResultPartitionConsumableNotifier.java   |   7 +-
 .../apache/flink/runtime/akka/AkkaUtils.scala   |   6 +
 .../TestingHighAvailabilityServices.java        |  14 +
 .../jobmaster/JobManagerRunnerMockTest.java     |  58 +-
 .../runtime/jobmaster/JobManagerRunnerTest.java |  24 +
 31 files changed, 1278 insertions(+), 1009 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/34fef475/flink-core/src/main/java/org/apache/flink/util/ExceptionUtils.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/util/ExceptionUtils.java b/flink-core/src/main/java/org/apache/flink/util/ExceptionUtils.java
index 7227006..0f6f24f 100644
--- a/flink-core/src/main/java/org/apache/flink/util/ExceptionUtils.java
+++ b/flink-core/src/main/java/org/apache/flink/util/ExceptionUtils.java
@@ -99,6 +99,26 @@ public final class ExceptionUtils {
 	}
 
 	/**
+	 * Throws the given {@code Throwable} in scenarios where the signatures do allow to
+	 * throw a Exception. Errors and Exceptions are thrown directly, other "exotic"
+	 * subclasses of Throwable are wrapped in an Exception.
+	 *
+	 * @param t The throwable to be thrown.
+	 * @param parentMessage The message for the parent Exception, if one is needed.
+	 */
+	public static void rethrowException(Throwable t, String parentMessage) throws Exception {
+		if (t instanceof Error) {
+			throw (Error) t;
+		}
+		else if (t instanceof Exception) {
+			throw (Exception) t;
+		}
+		else {
+			throw new Exception(parentMessage, t);
+		}
+	}
+
+	/**
 	 * Tries to throw the given {@code Throwable} in scenarios where the signatures allows only IOExceptions
 	 * (and RuntimeException and Error). Throws this exception directly, if it is an IOException,
 	 * a RuntimeException, or an Error. Otherwise does nothing.

http://git-wip-us.apache.org/repos/asf/flink/blob/34fef475/flink-core/src/main/java/org/apache/flink/util/StringUtils.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/util/StringUtils.java b/flink-core/src/main/java/org/apache/flink/util/StringUtils.java
index 10b6304..3c32d77 100644
--- a/flink-core/src/main/java/org/apache/flink/util/StringUtils.java
+++ b/flink-core/src/main/java/org/apache/flink/util/StringUtils.java
@@ -335,4 +335,18 @@ public final class StringUtils {
 			return null;
 		}
 	}
+
+	public static boolean isNullOrWhitespaceOnly(String str) {
+		if (str == null || str.length() == 0) {
+			return true;
+		}
+
+		final int len = str.length();
+		for (int i = 0; i < len; i++) {
+			if (!Character.isWhitespace(str.charAt(i))) {
+				return false;
+			}
+		}
+		return true;
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/34fef475/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServer.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServer.java
index ff54b67..5395d34 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServer.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServer.java
@@ -22,7 +22,11 @@ import org.apache.commons.io.FileUtils;
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.configuration.ConfigConstants;
 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.highavailability.HighAvailabilityServices;
 import org.apache.flink.runtime.jobmanager.HighAvailabilityMode;
 import org.apache.flink.util.NetUtils;
 import org.slf4j.Logger;
@@ -43,6 +47,7 @@ import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.StringUtils.isNullOrWhitespaceOnly;
 
 /**
  * This class implements the BLOB server. The BLOB server is responsible for listening for incoming requests and
@@ -88,23 +93,22 @@ public class BlobServer extends Thread implements BlobService {
 	 *         thrown if the BLOB server cannot bind to a free network port
 	 */
 	public BlobServer(Configuration config) throws IOException {
-		checkNotNull(config, "Configuration");
+		this(config, createBlobStoreFromConfig(config));
+	}
 
-		HighAvailabilityMode highAvailabilityMode = HighAvailabilityMode.fromConfig(config);
+	public BlobServer(Configuration config, HighAvailabilityServices haServices) throws IOException {
+		this(config, haServices.createBlobStore());
+	}
+
+	private BlobServer(Configuration config, BlobStore blobStore) throws IOException {
+		checkNotNull(config);
+		this.blobStore = checkNotNull(blobStore);
 
 		// configure and create the storage directory
 		String storageDirectory = config.getString(ConfigConstants.BLOB_STORAGE_DIRECTORY_KEY, null);
 		this.storageDir = BlobUtils.initStorageDirectory(storageDirectory);
 		LOG.info("Created BLOB server storage directory {}", storageDir);
 
-		if (highAvailabilityMode == HighAvailabilityMode.NONE) {
-			this.blobStore = new VoidBlobStore();
-		} else if (highAvailabilityMode == HighAvailabilityMode.ZOOKEEPER) {
-			this.blobStore = new FileSystemBlobStore(config);
-		} else {
-			throw new IllegalConfigurationException("Unexpected high availability mode '" + highAvailabilityMode + ".");
-		}
-
 		// configure the maximum number of concurrent connections
 		final int maxConnections = config.getInteger(
 				ConfigConstants.BLOB_FETCH_CONCURRENT_KEY, ConfigConstants.DEFAULT_BLOB_FETCH_CONCURRENT);
@@ -125,13 +129,7 @@ public class BlobServer extends Thread implements BlobService {
 			backlog = ConfigConstants.DEFAULT_BLOB_FETCH_BACKLOG;
 		}
 
-		if (highAvailabilityMode == HighAvailabilityMode.NONE) {
-			// Add shutdown hook to delete storage directory
-			this.shutdownHook = BlobUtils.addShutdownHook(this, LOG);
-		}
-		else {
-			this.shutdownHook = null;
-		}
+		this.shutdownHook = BlobUtils.addShutdownHook(this, LOG);
 
 		//  ----------------------- start the server -------------------
 
@@ -426,4 +424,37 @@ public class BlobServer extends Thread implements BlobService {
 		}
 	}
 
+	private static BlobStore createBlobStoreFromConfig(Configuration config) throws IOException {
+		HighAvailabilityMode highAvailabilityMode = HighAvailabilityMode.fromConfig(config);
+
+		if (highAvailabilityMode == HighAvailabilityMode.NONE) {
+		return new VoidBlobStore();
+		} else if (highAvailabilityMode == HighAvailabilityMode.ZOOKEEPER) {
+			final String storagePath = config.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);
+			}
+
+			return new FileSystemBlobStore(fileSystem, storagePath);
+		} else {
+			throw new IllegalConfigurationException("Unexpected high availability mode '" + highAvailabilityMode + ".");
+		}
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/34fef475/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobStore.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobStore.java
index 1e72d91..7050338 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobStore.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobStore.java
@@ -25,7 +25,7 @@ import java.io.File;
 /**
  * A blob store.
  */
-interface BlobStore {
+public interface BlobStore {
 
 	/**
 	 * Copies the local file to the blob store.
@@ -93,5 +93,4 @@ interface BlobStore {
 	 * Cleans up the store and deletes all blobs.
 	 */
 	void cleanUp();
-
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/34fef475/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 e74fa6f..136df09 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,6 +24,7 @@ import org.apache.flink.api.common.JobID;
 import org.apache.flink.core.fs.FileSystem;
 import org.apache.flink.core.fs.Path;
 import org.apache.flink.util.IOUtils;
+import org.apache.flink.util.StringUtils;
 import org.slf4j.Logger;
 
 import java.io.EOFException;
@@ -73,7 +74,7 @@ public class BlobUtils {
 	 */
 	static File initStorageDirectory(String storageDirectory) {
 		File baseDir;
-		if (storageDirectory == null || storageDirectory.trim().isEmpty()) {
+		if (StringUtils.isNullOrWhitespaceOnly(storageDirectory)) {
 			baseDir = new File(System.getProperty("java.io.tmpdir"));
 		}
 		else {
@@ -81,10 +82,9 @@ public class BlobUtils {
 		}
 
 		File storageDir;
-		final int MAX_ATTEMPTS = 10;
-		int attempt;
 
-		for(attempt = 0; attempt < MAX_ATTEMPTS; attempt++) {
+		final int MAX_ATTEMPTS = 10;
+		for(int attempt = 0; attempt < MAX_ATTEMPTS; attempt++) {
 			storageDir = new File(baseDir, String.format(
 					"blobStore-%s", UUID.randomUUID().toString()));
 

http://git-wip-us.apache.org/repos/asf/flink/blob/34fef475/flink-runtime/src/main/java/org/apache/flink/runtime/blob/FileSystemBlobStore.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/FileSystemBlobStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/FileSystemBlobStore.java
index deba738..2c05002 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/FileSystemBlobStore.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/FileSystemBlobStore.java
@@ -20,12 +20,7 @@ package org.apache.flink.runtime.blob;
 
 import com.google.common.io.Files;
 
-import org.apache.commons.lang3.StringUtils;
-
 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.util.IOUtils;
@@ -38,7 +33,6 @@ import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
-import java.net.URI;
 
 import static org.apache.flink.util.Preconditions.checkNotNull;
 
@@ -47,25 +41,24 @@ import static org.apache.flink.util.Preconditions.checkNotNull;
  *
  * <p>This is used in addition to the local blob storage for high availability.
  */
-class FileSystemBlobStore implements BlobStore {
+public class FileSystemBlobStore implements BlobStore {
 
 	private static final Logger LOG = LoggerFactory.getLogger(FileSystemBlobStore.class);
 
+	/** The file system in which blobs are stored */
+	private final FileSystem fileSystem;
+	
 	/** The base path of the blob store */
 	private final String basePath;
 
-	FileSystemBlobStore(Configuration config) throws IOException {
-		String storagePath = config.getValue(HighAvailabilityOptions.HA_STORAGE_PATH);
-
-		if (storagePath == null || StringUtils.isBlank(storagePath)) {
-			throw new IllegalConfigurationException("Missing high-availability storage path for metadata." +
-					" Specify via configuration key '" + HighAvailabilityOptions.HA_STORAGE_PATH + "'.");
-		}
+	public FileSystemBlobStore(FileSystem fileSystem, String storagePath) throws IOException {
+		this.fileSystem = checkNotNull(fileSystem);
+		this.basePath = checkNotNull(storagePath) + "/blob";
 
-		this.basePath = storagePath + "/blob";
+		LOG.info("Creating highly available BLOB storage directory at {}", basePath);
 
-		FileSystem.get(new Path(basePath).toUri()).mkdirs(new Path(basePath));
-		LOG.info("Created blob directory {}.", basePath);
+		fileSystem.mkdirs(new Path(basePath));
+		LOG.debug("Created highly available BLOB storage directory at {}", basePath);
 	}
 
 	// - Put ------------------------------------------------------------------
@@ -81,9 +74,7 @@ class FileSystemBlobStore implements BlobStore {
 	}
 
 	private void put(File fromFile, String toBlobPath) throws Exception {
-		try (OutputStream os = FileSystem.get(new URI(toBlobPath))
-				.create(new Path(toBlobPath), true)) {
-
+		try (OutputStream os = fileSystem.create(new Path(toBlobPath), true)) {
 			LOG.debug("Copying from {} to {}.", fromFile, toBlobPath);
 			Files.copy(fromFile, os);
 		}
@@ -106,16 +97,15 @@ class FileSystemBlobStore implements BlobStore {
 		checkNotNull(toFile, "File");
 
 		if (!toFile.exists() && !toFile.createNewFile()) {
-			throw new IllegalStateException("Failed to create target file to copy to");
+			throw new IOException("Failed to create target file to copy to");
 		}
 
-		final URI fromUri = new URI(fromBlobPath);
 		final Path fromPath = new Path(fromBlobPath);
 
-		if (FileSystem.get(fromUri).exists(fromPath)) {
-			try (InputStream is = FileSystem.get(fromUri).open(fromPath)) {
-				FileOutputStream fos = new FileOutputStream(toFile);
-
+		if (fileSystem.exists(fromPath)) {
+			try (InputStream is = fileSystem.open(fromPath);
+				FileOutputStream fos = new FileOutputStream(toFile))
+			{
 				LOG.debug("Copying from {} to {}.", fromBlobPath, toFile);
 				IOUtils.copyBytes(is, fos); // closes the streams
 			}
@@ -145,17 +135,16 @@ class FileSystemBlobStore implements BlobStore {
 	private void delete(String blobPath) {
 		try {
 			LOG.debug("Deleting {}.", blobPath);
-
-			FileSystem fs = FileSystem.get(new URI(blobPath));
+			
 			Path path = new Path(blobPath);
 
-			fs.delete(path, true);
+			fileSystem.delete(path, true);
 
 			// send a call to delete the directory containing the file. This will
 			// fail (and be ignored) when some files still exist.
 			try {
-				fs.delete(path.getParent(), false);
-				fs.delete(new Path(basePath), false);
+				fileSystem.delete(path.getParent(), false);
+				fileSystem.delete(new Path(basePath), false);
 			} catch (IOException ignored) {}
 		}
 		catch (Exception e) {
@@ -168,7 +157,7 @@ class FileSystemBlobStore implements BlobStore {
 		try {
 			LOG.debug("Cleaning up {}.", basePath);
 
-			FileSystem.get(new URI(basePath)).delete(new Path(basePath), true);
+			fileSystem.delete(new Path(basePath), true);
 		}
 		catch (Exception e) {
 			LOG.error("Failed to clean up recovery directory.");

http://git-wip-us.apache.org/repos/asf/flink/blob/34fef475/flink-runtime/src/main/java/org/apache/flink/runtime/blob/VoidBlobStore.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/VoidBlobStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/VoidBlobStore.java
index 1b71add..ece2ac1 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/VoidBlobStore.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/VoidBlobStore.java
@@ -25,7 +25,7 @@ import java.io.File;
 /**
  * A blob store doing nothing.
  */
-class VoidBlobStore implements BlobStore {
+public class VoidBlobStore implements BlobStore {
 
 	@Override
 	public void put(File localFile, BlobKey blobKey) throws Exception {

http://git-wip-us.apache.org/repos/asf/flink/blob/34fef475/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorGateway.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorGateway.java
index e448ebc..196ef5c 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorGateway.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorGateway.java
@@ -23,21 +23,23 @@ import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
 import org.apache.flink.runtime.rpc.RpcGateway;
 import org.apache.flink.runtime.state.CheckpointStateHandles;
 
+import java.util.UUID;
+
 public interface CheckpointCoordinatorGateway extends RpcGateway {
 
 	void acknowledgeCheckpoint(
-		JobID jobID,
-		ExecutionAttemptID executionAttemptID,
-		long checkpointID,
-		CheckpointStateHandles checkpointStateHandles,
-		long synchronousDurationMillis,
-		long asynchronousDurationMillis,
-		long bytesBufferedInAlignment,
-		long alignmentDurationNanos);
+			final JobID jobID,
+			final ExecutionAttemptID executionAttemptID,
+			final long checkpointID,
+			final CheckpointStateHandles checkpointStateHandles,
+			final long synchronousDurationMillis,
+			final long asynchronousDurationMillis,
+			final long bytesBufferedInAlignment,
+			final long alignmentDurationNanos);
 
 	void declineCheckpoint(
-		JobID jobID,
-		ExecutionAttemptID executionAttemptID,
-		long checkpointID,
-		long checkpointTimestamp);
+			final JobID jobID,
+			final ExecutionAttemptID executionAttemptID,
+			final long checkpointID,
+			final long checkpointTimestamp);
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/34fef475/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 a26886a..5d78ffc 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
@@ -19,11 +19,14 @@
 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.checkpoint.CheckpointRecoveryFactory;
 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;
+
 /**
  * This class gives access to all services needed for
  *
@@ -72,4 +75,14 @@ public interface HighAvailabilityServices {
 	 * Gets the submitted job graph store for the job manager
 	 */
 	SubmittedJobGraphStore getSubmittedJobGraphStore() throws Exception;
+
+	/**
+	 * Gets the registry that holds information about whether jobs are currently running.
+	 */
+	RunningJobsRegistry getRunningJobsRegistry() throws Exception;
+
+	/**
+	 * Creates the BLOB store in which BLOBs are stored in a highly-available fashion.
+	 */
+	BlobStore createBlobStore() throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/34fef475/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
index 2c6295c..d7fd2bf 100644
--- 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
@@ -19,8 +19,11 @@
 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.NonHaRegistry;
 import org.apache.flink.runtime.jobmanager.StandaloneSubmittedJobGraphStore;
 import org.apache.flink.runtime.jobmanager.SubmittedJobGraphStore;
 import org.apache.flink.runtime.leaderelection.LeaderElectionService;
@@ -102,4 +105,14 @@ public class NonHaServices implements HighAvailabilityServices {
 	public SubmittedJobGraphStore getSubmittedJobGraphStore() throws Exception {
 		return new StandaloneSubmittedJobGraphStore();
 	}
+
+	@Override
+	public RunningJobsRegistry getRunningJobsRegistry() throws Exception {
+		return new NonHaRegistry();
+	}
+
+	@Override
+	public BlobStore createBlobStore() {
+		return new VoidBlobStore();
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/34fef475/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/RunningJobsRegistry.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/RunningJobsRegistry.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/RunningJobsRegistry.java
new file mode 100644
index 0000000..e7c131c
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/RunningJobsRegistry.java
@@ -0,0 +1,66 @@
+/*
+ * 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 java.io.IOException;
+
+/**
+ * This registry tracks if a certain job is running.
+ * 
+ * <p>This registry is used in highly-available setups with multiple master nodes,
+ * to determine whether a new leader should attempt to recover a certain job (because the 
+ * job is still running), or whether the job has already finished successfully (in case of a
+ * finite job) and the leader has only been granted leadership because the previous leader
+ * quit cleanly after the job was finished.
+ */
+public interface RunningJobsRegistry {
+
+	/**
+	 * Marks a job as running.
+	 * 
+	 * @param jobID The id of the job.
+	 *
+	 * @throws IOException Thrown when the communication with the highly-available storage or registry
+	 *                     failed and could not be retried.
+	 */
+	void setJobRunning(JobID jobID) throws IOException;
+
+	/**
+	 * Marks a job as running.
+	 *
+	 * @param jobID The id of the job.
+	 * 
+	 * @throws IOException Thrown when the communication with the highly-available storage or registry
+	 *                     failed and could not be retried.
+	 */
+	void setJobFinished(JobID jobID) throws IOException;
+
+	/**
+	 * Checks whether a job is running.
+	 *
+	 * @param jobID The id of the job to check.
+	 * @return True if the job is still running, false otherwise.
+	 * 
+	 * @throws IOException Thrown when the communication with the highly-available storage or registry
+	 *                     failed and could not be retried.
+	 */
+	boolean isJobRunning(JobID jobID) throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/34fef475/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
index d26b668..3a7736b 100644
--- 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
@@ -19,8 +19,15 @@
 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;
@@ -28,12 +35,56 @@ 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 static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.StringUtils.isNullOrWhitespaceOnly;
+
 /**
- * An implementation of the {@link HighAvailabilityServices} with zookeeper.
+ * 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 DEFAULT_RESOURCE_MANAGER_PATH_SUFFIX = "/resource-manager";
+	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;
@@ -41,29 +92,37 @@ public class ZookeeperHaServices implements HighAvailabilityServices {
 	/** The runtime configuration */
 	private final Configuration configuration;
 
+	public ZookeeperHaServices(Configuration configuration) {
+		this(ZooKeeperUtils.startCuratorFramework(configuration), configuration);
+	}
+
 	public ZookeeperHaServices(final CuratorFramework client, final Configuration configuration) {
-		this.client = client;
-		this.configuration = configuration;
+		this.client = checkNotNull(client);
+		this.configuration = checkNotNull(configuration);
 	}
 
+	// ------------------------------------------------------------------------
+	//  Services
+	// ------------------------------------------------------------------------
+
 	@Override
 	public LeaderRetrievalService getResourceManagerLeaderRetriever() throws Exception {
-		return ZooKeeperUtils.createLeaderRetrievalService(client, configuration, DEFAULT_RESOURCE_MANAGER_PATH_SUFFIX);
+		return ZooKeeperUtils.createLeaderRetrievalService(client, configuration, RESOURCE_MANAGER_LEADER_PATH);
 	}
 
 	@Override
 	public LeaderRetrievalService getJobManagerLeaderRetriever(JobID jobID) throws Exception {
-		return ZooKeeperUtils.createLeaderRetrievalService(client, configuration, getPathSuffixForJob(jobID));
+		return ZooKeeperUtils.createLeaderRetrievalService(client, configuration, getPathForJobManager(jobID));
 	}
 
 	@Override
 	public LeaderElectionService getResourceManagerLeaderElectionService() throws Exception {
-		return ZooKeeperUtils.createLeaderElectionService(client, configuration, DEFAULT_RESOURCE_MANAGER_PATH_SUFFIX);
+		return ZooKeeperUtils.createLeaderElectionService(client, configuration, RESOURCE_MANAGER_LEADER_PATH);
 	}
 
 	@Override
 	public LeaderElectionService getJobManagerLeaderElectionService(JobID jobID) throws Exception {
-		return ZooKeeperUtils.createLeaderElectionService(client, configuration, getPathSuffixForJob(jobID));
+		return ZooKeeperUtils.createLeaderElectionService(client, configuration, getPathForJobManager(jobID));
 	}
 
 	@Override
@@ -76,7 +135,43 @@ public class ZookeeperHaServices implements HighAvailabilityServices {
 		return ZooKeeperUtils.createSubmittedJobGraphs(client, configuration);
 	}
 
-	private static String getPathSuffixForJob(final JobID jobID) {
-		return String.format("/job-managers/%s", jobID);
+	@Override
+	public RunningJobsRegistry getRunningJobsRegistry() throws Exception {
+		throw new UnsupportedOperationException("not yet implemented");
+	}
+
+	@Override
+	public BlobStore createBlobStore() throws IOException {
+		final 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);
+		}
+
+		return new FileSystemBlobStore(fileSystem, storagePath);
+	}
+
+	// ------------------------------------------------------------------------
+	//  Utilities
+	// ------------------------------------------------------------------------
+
+	private static String getPathForJobManager(final JobID jobID) {
+		return "/" + jobID + JOB_MANAGER_LEADER_PATH;
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/34fef475/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
new file mode 100644
index 0000000..85dd711
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/NonHaRegistry.java
@@ -0,0 +1,62 @@
+/*
+ * 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.HashSet;
+
+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 HashSet<JobID> running = new HashSet<>();
+
+	@Override
+	public void setJobRunning(JobID jobID) {
+		checkNotNull(jobID);
+
+		synchronized (running) {
+			running.add(jobID);
+		}
+	}
+
+	@Override
+	public void setJobFinished(JobID jobID) {
+		checkNotNull(jobID);
+
+		synchronized (running) {
+			running.remove(jobID);
+		}
+	}
+
+	@Override
+	public boolean isJobRunning(JobID jobID) {
+		checkNotNull(jobID);
+
+		synchronized (running) {
+			return running.contains(jobID);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/34fef475/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/OnCompletionActions.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/OnCompletionActions.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/OnCompletionActions.java
index 6de4253..25a2a66 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/OnCompletionActions.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/OnCompletionActions.java
@@ -19,9 +19,8 @@
 package org.apache.flink.runtime.jobmanager;
 
 import org.apache.flink.api.common.JobExecutionResult;
-import org.apache.flink.runtime.rpc.FatalErrorHandler;
 
-public interface OnCompletionActions extends FatalErrorHandler {
+public interface OnCompletionActions {
 
 	void jobFinished(JobExecutionResult result);
 

http://git-wip-us.apache.org/repos/asf/flink/blob/34fef475/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunner.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunner.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunner.java
index a096932..74c1050 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunner.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunner.java
@@ -21,26 +21,38 @@ package org.apache.flink.runtime.jobmaster;
 import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.api.common.JobExecutionResult;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.client.JobExecutionException;
+import org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager;
 import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
+import org.apache.flink.runtime.highavailability.RunningJobsRegistry;
 import org.apache.flink.runtime.jobgraph.JobGraph;
 import org.apache.flink.runtime.jobmanager.OnCompletionActions;
-import org.apache.flink.runtime.jobmanager.SubmittedJobGraphStore;
-import org.apache.flink.runtime.jobmanager.scheduler.Scheduler;
 import org.apache.flink.runtime.leaderelection.LeaderContender;
 import org.apache.flink.runtime.leaderelection.LeaderElectionService;
+import org.apache.flink.runtime.metrics.MetricRegistry;
+import org.apache.flink.runtime.metrics.MetricRegistryConfiguration;
+import org.apache.flink.runtime.metrics.groups.JobManagerMetricGroup;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
 import org.apache.flink.runtime.rpc.RpcService;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.IOException;
 import java.util.UUID;
 
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
 /**
  * The runner for the job manager. It deals with job level leader election and make underlying job manager
  * properly reacted.
  */
-public class JobManagerRunner implements LeaderContender, OnCompletionActions {
+public class JobManagerRunner implements LeaderContender, OnCompletionActions, FatalErrorHandler {
 
-	private final Logger log = LoggerFactory.getLogger(JobManagerRunner.class);
+	private static final Logger log = LoggerFactory.getLogger(JobManagerRunner.class);
+
+	// ------------------------------------------------------------------------
 
 	/** Lock to ensure that this runner can deal with leader election event and job completion notifies simultaneously */
 	private final Object lock = new Object();
@@ -48,52 +60,141 @@ public class JobManagerRunner implements LeaderContender, OnCompletionActions {
 	/** The job graph needs to run */
 	private final JobGraph jobGraph;
 
-	private final OnCompletionActions toNotify;
+	/** The listener to notify once the job completes - either successfully or unsuccessfully */
+	private final OnCompletionActions toNotifyOnComplete;
+
+	/** The handler to call in case of fatal (unrecoverable) errors */ 
+	private final FatalErrorHandler errorHandler;
 
 	/** Used to check whether a job needs to be run */
-	private final SubmittedJobGraphStore submittedJobGraphStore;
+	private final RunningJobsRegistry runningJobsRegistry;
 
 	/** Leader election for this job */
 	private final LeaderElectionService leaderElectionService;
 
+	private final JobManagerServices jobManagerServices;
+
 	private final JobMaster jobManager;
 
+	private final JobManagerMetricGroup jobManagerMetricGroup;
+
 	/** flag marking the runner as shut down */
 	private volatile boolean shutdown;
 
+	// ------------------------------------------------------------------------
+
 	public JobManagerRunner(
-		final JobGraph jobGraph,
-		final Configuration configuration,
-		final RpcService rpcService,
-		final HighAvailabilityServices haServices,
-		final OnCompletionActions toNotify) throws Exception
+			final JobGraph jobGraph,
+			final Configuration configuration,
+			final RpcService rpcService,
+			final HighAvailabilityServices haServices,
+			final OnCompletionActions toNotifyOnComplete,
+			final FatalErrorHandler errorHandler) throws Exception
 	{
 		this(jobGraph, configuration, rpcService, haServices,
-			JobManagerServices.fromConfiguration(configuration), toNotify);
+				new MetricRegistry(MetricRegistryConfiguration.fromConfiguration(configuration)),
+				toNotifyOnComplete, errorHandler);
 	}
 
 	public JobManagerRunner(
-		final JobGraph jobGraph,
-		final Configuration configuration,
-		final RpcService rpcService,
-		final HighAvailabilityServices haServices,
-		final JobManagerServices jobManagerServices,
-		final OnCompletionActions toNotify) throws Exception
+			final JobGraph jobGraph,
+			final Configuration configuration,
+			final RpcService rpcService,
+			final HighAvailabilityServices haServices,
+			final MetricRegistry metricRegistry,
+			final OnCompletionActions toNotifyOnComplete,
+			final FatalErrorHandler errorHandler) throws Exception
+	{
+		this(jobGraph, configuration, rpcService, haServices,
+				JobManagerServices.fromConfiguration(configuration, haServices),
+				metricRegistry,
+				toNotifyOnComplete, errorHandler);
+	}
+
+	/**
+	 * 
+	 * <p>Exceptions that occur while creating the JobManager or JobManagerRunner are directly
+	 * thrown and not reported to the given {@code FatalErrorHandler}.
+	 * 
+	 * <p>This JobManagerRunner assumes that it owns the given {@code JobManagerServices}.
+	 * It will shut them down on error and on calls to {@link #shutdown()}.
+	 * 
+	 * @throws Exception Thrown if the runner cannot be set up, because either one of the
+	 *                   required services could not be started, ot the Job could not be initialized.
+	 */
+	public JobManagerRunner(
+			final JobGraph jobGraph,
+			final Configuration configuration,
+			final RpcService rpcService,
+			final HighAvailabilityServices haServices,
+			final JobManagerServices jobManagerServices,
+			final MetricRegistry metricRegistry,
+			final OnCompletionActions toNotifyOnComplete,
+			final FatalErrorHandler errorHandler) throws Exception
 	{
-		this.jobGraph = jobGraph;
-		this.toNotify = toNotify;
-		this.submittedJobGraphStore = haServices.getSubmittedJobGraphStore();
-		this.leaderElectionService = haServices.getJobManagerLeaderElectionService(jobGraph.getJobID());
-
-		this.jobManager = new JobMaster(
-			jobGraph, configuration, rpcService, haServices,
-			jobManagerServices.libraryCacheManager,
-			jobManagerServices.restartStrategyFactory,
-			jobManagerServices.savepointStore,
-			jobManagerServices.timeout,
-			new Scheduler(jobManagerServices.executorService),
-			jobManagerServices.jobManagerMetricGroup,
-			this);
+
+		JobManagerMetricGroup jobManagerMetrics = null;
+
+		// make sure we cleanly shut down out JobManager services if initialization fails
+		try {
+			this.jobGraph = checkNotNull(jobGraph);
+			this.toNotifyOnComplete = checkNotNull(toNotifyOnComplete);
+			this.errorHandler = checkNotNull(errorHandler);
+			this.jobManagerServices = checkNotNull(jobManagerServices);
+
+			checkArgument(jobGraph.getNumberOfVertices() > 0, "The given job is empty");
+
+			final String hostAddress = rpcService.getAddress().isEmpty() ? "localhost" : rpcService.getAddress();
+			jobManagerMetrics = new JobManagerMetricGroup(metricRegistry, hostAddress);
+			this.jobManagerMetricGroup = jobManagerMetrics;
+
+			// libraries and class loader first
+			final BlobLibraryCacheManager libraryCacheManager = jobManagerServices.libraryCacheManager;
+			try {
+				libraryCacheManager.registerJob(
+						jobGraph.getJobID(), jobGraph.getUserJarBlobKeys(), jobGraph.getClasspaths());
+			} catch (IOException e) {
+				throw new Exception("Cannot set up the user code libraries: " + e.getMessage(), e);
+			}
+
+			final ClassLoader userCodeLoader = libraryCacheManager.getClassLoader(jobGraph.getJobID());
+			if (userCodeLoader == null) {
+				throw new Exception("The user code class loader could not be initialized.");
+			}
+
+			// high availability services next
+			this.runningJobsRegistry = haServices.getRunningJobsRegistry();
+			this.leaderElectionService = haServices.getJobManagerLeaderElectionService(jobGraph.getJobID());
+
+			// now start the JobManager
+			this.jobManager = new JobMaster(
+					jobGraph, configuration,
+					rpcService,
+					haServices,
+					jobManagerServices.executorService,
+					jobManagerServices.libraryCacheManager,
+					jobManagerServices.restartStrategyFactory,
+					jobManagerServices.savepointStore,
+					jobManagerServices.rpcAskTimeout,
+					jobManagerMetrics,
+					this,
+					this,
+					userCodeLoader);
+		}
+		catch (Throwable t) {
+			// clean up everything
+			try {
+				jobManagerServices.shutdown();
+			} catch (Throwable tt) {
+				log.error("Error while shutting down JobManager services", tt);
+			}
+
+			if (jobManagerMetrics != null) {
+				jobManagerMetrics.close();
+			}
+
+			throw new JobExecutionException(jobGraph.getJobID(), "Could not set up JobManager", t);
+		}
 	}
 
 	//----------------------------------------------------------------------------------------------
@@ -101,9 +202,6 @@ public class JobManagerRunner implements LeaderContender, OnCompletionActions {
 	//----------------------------------------------------------------------------------------------
 
 	public void start() throws Exception {
-		jobManager.init();
-		jobManager.start();
-
 		try {
 			leaderElectionService.start(this);
 		}
@@ -114,11 +212,6 @@ public class JobManagerRunner implements LeaderContender, OnCompletionActions {
 	}
 
 	public void shutdown() {
-		shutdown(new Exception("The JobManager runner is shutting down"));
-	}
-
-	public void shutdown(Throwable cause) {
-		// TODO what is the cause used for ?
 		shutdownInternally();
 	}
 
@@ -129,12 +222,29 @@ public class JobManagerRunner implements LeaderContender, OnCompletionActions {
 			if (leaderElectionService != null) {
 				try {
 					leaderElectionService.stop();
-				} catch (Exception e) {
-					log.error("Could not properly shutdown the leader election service.");
+				} catch (Throwable t) {
+					log.error("Could not properly shutdown the leader election service", t);
 				}
 			}
 
-			jobManager.shutDown();
+			try {
+				jobManager.shutDown();
+			} catch (Throwable t) {
+				log.error("Error shutting down JobManager", t);
+			}
+
+			try {
+				jobManagerServices.shutdown();
+			} catch (Throwable t) {
+				log.error("Error shutting down JobManager services", t);
+			}
+
+			// make all registered metrics go away
+			try {
+				jobManagerMetricGroup.close();
+			} catch (Throwable t) {
+				log.error("Error while unregistering metrics", t);
+			}
 		}
 	}
 
@@ -148,11 +258,12 @@ public class JobManagerRunner implements LeaderContender, OnCompletionActions {
 	@Override
 	public void jobFinished(JobExecutionResult result) {
 		try {
+			unregisterJobFromHighAvailability();
 			shutdownInternally();
 		}
 		finally {
-			if (toNotify != null) {
-				toNotify.jobFinished(result);
+			if (toNotifyOnComplete != null) {
+				toNotifyOnComplete.jobFinished(result);
 			}
 		}
 	}
@@ -163,11 +274,12 @@ public class JobManagerRunner implements LeaderContender, OnCompletionActions {
 	@Override
 	public void jobFailed(Throwable cause) {
 		try {
+			unregisterJobFromHighAvailability();
 			shutdownInternally();
 		}
 		finally {
-			if (toNotify != null) {
-				toNotify.jobFailed(cause);
+			if (toNotifyOnComplete != null) {
+				toNotifyOnComplete.jobFailed(cause);
 			}
 		}
 	}
@@ -178,11 +290,12 @@ public class JobManagerRunner implements LeaderContender, OnCompletionActions {
 	@Override
 	public void jobFinishedByOther() {
 		try {
+			unregisterJobFromHighAvailability();
 			shutdownInternally();
 		}
 		finally {
-			if (toNotify != null) {
-				toNotify.jobFinishedByOther();
+			if (toNotifyOnComplete != null) {
+				toNotifyOnComplete.jobFinishedByOther();
 			}
 		}
 	}
@@ -192,18 +305,43 @@ public class JobManagerRunner implements LeaderContender, OnCompletionActions {
 	 */
 	@Override
 	public void onFatalError(Throwable exception) {
-		// first and in any case, notify our handler, so it can react fast
+		// we log first to make sure an explaining message goes into the log
+		// we even guard the log statement here to increase chances that the error handler
+		// gets the notification on hard critical situations like out-of-memory errors
+		try {
+			log.error("JobManager runner encountered a fatal error.", exception);
+		} catch (Throwable ignored) {}
+
+		// in any case, notify our handler, so it can react fast
 		try {
-			if (toNotify != null) {
-				toNotify.onFatalError(exception);
+			if (errorHandler != null) {
+				errorHandler.onFatalError(exception);
 			}
 		}
 		finally {
-			log.error("JobManager runner encountered a fatal error.", exception);
+			// the shutdown may not even needed any more, if the fatal error
+			// handler kills the process. that is fine, a process kill cleans up better than anything.
 			shutdownInternally();
 		}
 	}
 
+	/**
+	 * Marks this runner's job as not running. Other JobManager will not recover the job
+	 * after this call.
+	 * 
+	 * <p>This method never throws an exception.
+	 */
+	private void unregisterJobFromHighAvailability() {
+		try {
+			runningJobsRegistry.setJobFinished(jobGraph.getJobID());
+		}
+		catch (Throwable t) {
+			log.error("Could not un-register from high-availability services job {} ({})." +
+					"Other JobManager's may attempt to recover it and re-execute it.",
+					jobGraph.getName(), jobGraph.getJobID(), t);
+		}
+	}
+
 	//----------------------------------------------------------------------------------------------
 	// Leadership methods
 	//----------------------------------------------------------------------------------------------
@@ -223,15 +361,25 @@ public class JobManagerRunner implements LeaderContender, OnCompletionActions {
 			// it's okay that job manager wait for the operation complete
 			leaderElectionService.confirmLeaderSessionID(leaderSessionID);
 
+			boolean jobRunning;
+			try {
+				jobRunning = runningJobsRegistry.isJobRunning(jobGraph.getJobID());
+			} catch (Throwable t) {
+				log.error("Could not access status (running/finished) of job {}. " +
+						"Falling back to assumption that job is running and attempting recovery...",
+						jobGraph.getJobID(), t);
+				jobRunning = true;
+			}
+
 			// Double check the leadership after we confirm that, there is a small chance that multiple
 			// job managers schedule the same job after if they try to recover at the same time.
 			// This will eventually be noticed, but can not be ruled out from the beginning.
 			if (leaderElectionService.hasLeadership()) {
-				if (isJobFinishedByOthers()) {
+				if (jobRunning) {
+					jobManager.start(leaderSessionID);
+				} else {
 					log.info("Job {} ({}) already finished by others.", jobGraph.getName(), jobGraph.getJobID());
 					jobFinishedByOther();
-				} else {
-					jobManager.getSelf().startJob(leaderSessionID);
 				}
 			}
 		}
@@ -248,7 +396,7 @@ public class JobManagerRunner implements LeaderContender, OnCompletionActions {
 			log.info("JobManager for job {} ({}) was revoked leadership at {}.",
 				jobGraph.getName(), jobGraph.getJobID(), getAddress());
 
-			jobManager.getSelf().suspendJob(new Exception("JobManager is no longer the leader."));
+			jobManager.getSelf().suspendExecution(new Exception("JobManager is no longer the leader."));
 		}
 	}
 
@@ -263,11 +411,9 @@ public class JobManagerRunner implements LeaderContender, OnCompletionActions {
 		onFatalError(exception);
 	}
 
-	@VisibleForTesting
-	boolean isJobFinishedByOthers() {
-		// TODO: Fix
-		return false;
-	}
+	//----------------------------------------------------------------------------------------------
+	// Testing
+	//----------------------------------------------------------------------------------------------
 
 	@VisibleForTesting
 	boolean isShutdown() {

http://git-wip-us.apache.org/repos/asf/flink/blob/34fef475/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerServices.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerServices.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerServices.java
index e6beba6..eebfbfa 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerServices.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerServices.java
@@ -19,13 +19,21 @@
 package org.apache.flink.runtime.jobmaster;
 
 import org.apache.flink.api.common.time.Time;
+import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.IllegalConfigurationException;
+import org.apache.flink.runtime.akka.AkkaUtils;
+import org.apache.flink.runtime.blob.BlobServer;
 import org.apache.flink.runtime.checkpoint.savepoint.SavepointStore;
+import org.apache.flink.runtime.checkpoint.savepoint.SavepointStoreFactory;
 import org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager;
 import org.apache.flink.runtime.executiongraph.restart.RestartStrategyFactory;
-import org.apache.flink.runtime.metrics.groups.JobManagerMetricGroup;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
+import org.apache.flink.util.ExceptionUtils;
+import scala.concurrent.duration.FiniteDuration;
 
 import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ForkJoinPool;
 
 import static org.apache.flink.util.Preconditions.checkNotNull;
 
@@ -42,32 +50,95 @@ public class JobManagerServices {
 
 	public final SavepointStore savepointStore;
 
-	public final Time timeout;
-
-	public final JobManagerMetricGroup jobManagerMetricGroup;
+	public final Time rpcAskTimeout;
 
 	public JobManagerServices(
 			ExecutorService executorService,
 			BlobLibraryCacheManager libraryCacheManager,
 			RestartStrategyFactory restartStrategyFactory,
 			SavepointStore savepointStore,
-			Time timeout,
-			JobManagerMetricGroup jobManagerMetricGroup) {
+			Time rpcAskTimeout) {
 
 		this.executorService = checkNotNull(executorService);
 		this.libraryCacheManager = checkNotNull(libraryCacheManager);
 		this.restartStrategyFactory = checkNotNull(restartStrategyFactory);
 		this.savepointStore = checkNotNull(savepointStore);
-		this.timeout = checkNotNull(timeout);
-		this.jobManagerMetricGroup = checkNotNull(jobManagerMetricGroup);
+		this.rpcAskTimeout = checkNotNull(rpcAskTimeout);
+	}
+
+	/**
+	 * 
+	 * <p>This method makes sure all services are closed or shut down, even when an exception occurred
+	 * in the shutdown of one component. The first encountered exception is thrown, with successive
+	 * exceptions added as suppressed exceptions.
+	 * 
+	 * @throws Exception The first Exception encountered during shutdown.
+	 */
+	public void shutdown() throws Exception {
+		Throwable firstException = null;
+
+		try {
+			executorService.shutdownNow();
+		} catch (Throwable t) {
+			firstException = t;
+		}
+
+		try {
+			savepointStore.shutdown();
+		}
+		catch (Throwable t) {
+			if (firstException == null) {
+				firstException = t;
+			} else {
+				firstException.addSuppressed(t);
+			}
+		}
+
+		try {
+			libraryCacheManager.shutdown();
+		}
+		catch (Throwable t) {
+			if (firstException == null) {
+				firstException = t;
+			} else {
+				firstException.addSuppressed(t);
+			}
+		}
+
+		if (firstException != null) {
+			ExceptionUtils.rethrowException(firstException, "Error while shutting down JobManager services");
+		}
 	}
 
 	// ------------------------------------------------------------------------
 	//  Creating the components from a configuration 
 	// ------------------------------------------------------------------------
 	
-	public static JobManagerServices fromConfiguration(Configuration config) throws Exception {
-		// TODO not yet implemented
-		return null;
+
+	public static JobManagerServices fromConfiguration(
+			Configuration config,
+			HighAvailabilityServices haServices) throws Exception {
+
+		final BlobServer blobServer = new BlobServer(config, haServices);
+
+		final long cleanupInterval = config.getLong(
+			ConfigConstants.LIBRARY_CACHE_MANAGER_CLEANUP_INTERVAL,
+			ConfigConstants.DEFAULT_LIBRARY_CACHE_MANAGER_CLEANUP_INTERVAL) * 1000;
+
+		final BlobLibraryCacheManager libraryCacheManager = new BlobLibraryCacheManager(blobServer, cleanupInterval);
+
+		final FiniteDuration timeout;
+		try {
+			timeout = AkkaUtils.getTimeout(config);
+		} catch (NumberFormatException e) {
+			throw new IllegalConfigurationException(AkkaUtils.formatDurationParingErrorMessage());
+		}
+
+		return new JobManagerServices(
+			new ForkJoinPool(),
+			libraryCacheManager,
+			RestartStrategyFactory.createRestartStrategyFactory(config),
+			SavepointStoreFactory.createFromConfig(config),
+			Time.of(timeout.length(), timeout.unit()));
 	}
 }


[7/8] flink git commit: [FLINK-4764] [core] Introduce Config Options

Posted by se...@apache.org.
[FLINK-4764] [core] Introduce Config Options

This is a more concise and maintainable way to define configuration keys, default values,
deprecated keys, etc.

This closes #2605


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

Branch: refs/heads/flip-6
Commit: d71a09cc2a36a877e8287db8d9fe84134a4901ba
Parents: 05436f4
Author: Stephan Ewen <se...@apache.org>
Authored: Fri Oct 7 15:24:44 2016 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Thu Oct 13 16:25:49 2016 +0200

----------------------------------------------------------------------
 .../flink/configuration/ConfigOption.java       | 171 ++++++++
 .../flink/configuration/ConfigOptions.java      | 116 ++++++
 .../flink/configuration/Configuration.java      | 407 +++++++++++++++----
 .../configuration/DelegatingConfiguration.java  | 118 +++++-
 .../flink/configuration/ConfigurationTest.java  |  95 ++++-
 .../DelegatingConfigurationTest.java            |  55 +--
 .../UnmodifiableConfigurationTest.java          |  16 +-
 7 files changed, 844 insertions(+), 134 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/d71a09cc/flink-core/src/main/java/org/apache/flink/configuration/ConfigOption.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/configuration/ConfigOption.java b/flink-core/src/main/java/org/apache/flink/configuration/ConfigOption.java
new file mode 100644
index 0000000..3531f6d
--- /dev/null
+++ b/flink-core/src/main/java/org/apache/flink/configuration/ConfigOption.java
@@ -0,0 +1,171 @@
+/*
+ * 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.configuration;
+
+import org.apache.flink.annotation.PublicEvolving;
+
+import java.util.Arrays;
+import java.util.Collections;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * A {@code ConfigOption} describes a configuration parameter. It encapsulates
+ * the configuration key, deprecated older versions of the key, and an optional
+ * default value for the configuration parameter.
+ * 
+ * <p>{@code ConfigOptions} are built via the {@link ConfigOptions} class.
+ * Once created, a config option is immutable.
+ * 
+ * @param <T> The type of value associated with the configuration option.
+ */
+@PublicEvolving
+public class ConfigOption<T> {
+
+	private static final String[] EMPTY = new String[0];
+
+	// ------------------------------------------------------------------------
+
+	/** The current key for that config option */
+	private final String key;
+
+	/** The list of deprecated keys, in the order to be checked */
+	private final String[] deprecatedKeys;
+
+	/** The default value for this option */
+	private final T defaultValue;
+
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Creates a new config option with no deprecated keys.
+	 *
+	 * @param key             The current key for that config option
+	 * @param defaultValue    The default value for this option
+	 */
+	ConfigOption(String key, T defaultValue) {
+		this.key = checkNotNull(key);
+		this.defaultValue = defaultValue;
+		this.deprecatedKeys = EMPTY;
+	}
+
+	/**
+	 * Creates a new config option with deprecated keys.
+	 *
+	 * @param key             The current key for that config option
+	 * @param defaultValue    The default value for this option
+	 * @param deprecatedKeys  The list of deprecated keys, in the order to be checked
+	 */
+	ConfigOption(String key, T defaultValue, String... deprecatedKeys) {
+		this.key = checkNotNull(key);
+		this.defaultValue = defaultValue;
+		this.deprecatedKeys = deprecatedKeys == null || deprecatedKeys.length == 0 ? EMPTY : deprecatedKeys;
+	}
+
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Creates a new config option, using this option's key and default value, and
+	 * adding the given deprecated keys.
+	 * 
+	 * <p>When obtaining a value from the configuration via {@link Configuration#getValue(ConfigOption)},
+	 * the deprecated keys will be checked in the order provided to this method. The first key for which
+	 * a value is found will be used - that value will be returned.
+	 * 
+	 * @param deprecatedKeys The deprecated keys, in the order in which they should be checked.
+	 * @return A new config options, with the given deprecated keys.
+	 */
+	public ConfigOption<T> withDeprecatedKeys(String... deprecatedKeys) {
+		return new ConfigOption<>(key, defaultValue, deprecatedKeys);
+	}
+
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Gets the configuration key.
+	 * @return The configuration key
+	 */
+	public String key() {
+		return key;
+	}
+
+	/**
+	 * Checks if this option has a default value.
+	 * @return True if it has a default value, false if not.
+	 */
+	public boolean hasDefaultValue() {
+		return defaultValue != null;
+	}
+
+	/**
+	 * Returns the default value, or null, if there is no default value.
+	 * @return The default value, or null.
+	 */
+	public T defaultValue() {
+		return defaultValue;
+	}
+
+	/**
+	 * Checks whether this option has deprecated keys.
+	 * @return True if the option has deprecated keys, false if not.
+	 */
+	public boolean hasDeprecatedKeys() {
+		return deprecatedKeys != EMPTY;
+	}
+
+	/**
+	 * Gets the deprecated keys, in the order to be checked.
+	 * @return The option's deprecated keys.
+	 */
+	public Iterable<String> deprecatedKeys() {
+		return deprecatedKeys == EMPTY ? Collections.<String>emptyList() : Arrays.asList(deprecatedKeys);
+	}
+
+	// ------------------------------------------------------------------------
+
+	@Override
+	public boolean equals(Object o) {
+		if (this == o) {
+			return true;
+		}
+		else if (o != null && o.getClass() == ConfigOption.class) {
+			ConfigOption<?> that = (ConfigOption<?>) o;
+			return this.key.equals(that.key) &&
+					Arrays.equals(this.deprecatedKeys, that.deprecatedKeys) &&
+					(this.defaultValue == null ? that.defaultValue == null :
+							(that.defaultValue != null && this.defaultValue.equals(that.defaultValue)));
+		}
+		else {
+			return false;
+		}
+	}
+
+	@Override
+	public int hashCode() {
+		return 31 * key.hashCode() +
+				17 * Arrays.hashCode(deprecatedKeys) +
+				(defaultValue != null ? defaultValue.hashCode() : 0);
+	}
+
+	@Override
+	public String toString() {
+		return String.format("Key: '%s' , default: %s (deprecated keys: %s)",
+				key, defaultValue, Arrays.toString(deprecatedKeys));
+	}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/d71a09cc/flink-core/src/main/java/org/apache/flink/configuration/ConfigOptions.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/configuration/ConfigOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/ConfigOptions.java
new file mode 100644
index 0000000..f87da0a
--- /dev/null
+++ b/flink-core/src/main/java/org/apache/flink/configuration/ConfigOptions.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.configuration;
+
+import org.apache.flink.annotation.PublicEvolving;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * {@code ConfigOptions} are used to build a {@link ConfigOption}.
+ * The option is typically built in one of the following pattern:
+ * 
+ * <pre>{@code
+ * // simple string-valued option with a default value
+ * ConfigOption<String> tempDirs = ConfigOptions
+ *     .key("tmp.dir")
+ *     .defaultValue("/tmp");
+ * 
+ * // simple integer-valued option with a default value
+ * ConfigOption<Integer> parallelism = ConfigOptions
+ *     .key("application.parallelism")
+ *     .defaultValue(100);
+ * 
+ * // option with no default value
+ * ConfigOption<String> userName = ConfigOptions
+ *     .key("user.name")
+ *     .noDefaultValue();
+ * 
+ * // option with deprecated keys to check
+ * ConfigOption<Double> threshold = ConfigOptions
+ *     .key("cpu.utilization.threshold")
+ *     .defaultValue(0.9).
+ *     .withDeprecatedKeys("cpu.threshold");
+ * }</pre>
+ */
+@PublicEvolving
+public class ConfigOptions {
+
+	/**
+	 * Starts building a new {@link ConfigOption}.
+	 * 
+	 * @param key The key for the config option.
+	 * @return The builder for the config option with the given key.
+	 */
+	public static OptionBuilder key(String key) {
+		checkNotNull(key);
+		return new OptionBuilder(key);
+	}
+
+	// ------------------------------------------------------------------------
+
+	/**
+	 * The option builder is used to create a {@link ConfigOption}.
+	 * It is instantiated via {@link ConfigOptions#key(String)}.
+	 */
+	public static final class OptionBuilder {
+
+		/** The key for the config option */
+		private final String key;
+
+		/**
+		 * Creates a new OptionBuilder.
+		 * @param key The key for the config option
+		 */
+		OptionBuilder(String key) {
+			this.key = key;
+		}
+
+		/**
+		 * Creates a ConfigOption with the given default value.
+		 * 
+		 * <p>This method does not accept "null". For options with no default value, choose
+		 * one of the {@code noDefaultValue} methods.
+		 * 
+		 * @param value The default value for the config option
+		 * @param <T> The type of the default value.
+		 * @return The config option with the default value.
+		 */
+		public <T> ConfigOption<T> defaultValue(T value) {
+			checkNotNull(value);
+			return new ConfigOption<T>(key, value);
+		}
+
+		/**
+		 * Creates a string-valued option with no default value.
+		 * String-valued options are the only ones that can have no
+		 * default value.
+		 * 
+		 * @return The created ConfigOption.
+		 */
+		public ConfigOption<String> noDefaultValue() {
+			return new ConfigOption<>(key, null);
+		}
+	}
+
+	// ------------------------------------------------------------------------
+
+	/** Not intended to be instantiated */
+	private ConfigOptions() {}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/d71a09cc/flink-core/src/main/java/org/apache/flink/configuration/Configuration.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/configuration/Configuration.java b/flink-core/src/main/java/org/apache/flink/configuration/Configuration.java
index 8ca5d07..f15c669 100644
--- a/flink-core/src/main/java/org/apache/flink/configuration/Configuration.java
+++ b/flink-core/src/main/java/org/apache/flink/configuration/Configuration.java
@@ -27,6 +27,7 @@ import java.util.Properties;
 import java.util.Set;
 
 import org.apache.flink.annotation.Public;
+import org.apache.flink.annotation.PublicEvolving;
 import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.core.io.IOReadableWritable;
 import org.apache.flink.core.memory.DataInputView;
@@ -134,7 +135,33 @@ public class Configuration extends ExecutionConfig.GlobalJobParameters
 			return o.toString();
 		}
 	}
-	
+
+	/**
+	 * Returns the value associated with the given config option as a string.
+	 *
+	 * @param configOption The configuration option
+	 * @return the (default) value associated with the given config option
+	 */
+	@PublicEvolving
+	public String getString(ConfigOption<String> configOption) {
+		Object o = getValueOrDefaultFromOption(configOption);
+		return o == null ? null : o.toString();
+	}
+
+	/**
+	 * Returns the value associated with the given config option as a string.
+	 * If no value is mapped under any key of the option, it returns the specified
+	 * default instead of the option's default value.
+	 *
+	 * @param configOption The configuration option
+	 * @return the (default) value associated with the given config option
+	 */
+	@PublicEvolving
+	public String getString(ConfigOption<String> configOption, String overrideDefault) {
+		Object o = getRawValueFromOption(configOption);
+		return o == null ? overrideDefault : o.toString();
+	}
+
 	/**
 	 * Adds the given key/value pair to the configuration object.
 	 * 
@@ -148,6 +175,20 @@ public class Configuration extends ExecutionConfig.GlobalJobParameters
 	}
 
 	/**
+	 * Adds the given value to the configuration object.
+	 * The main key of the config option will be used to map the value.
+	 *
+	 * @param key
+	 *        the option specifying the key to be added
+	 * @param value
+	 *        the value of the key/value pair to be added
+	 */
+	@PublicEvolving
+	public void setString(ConfigOption<String> key, String value) {
+		setValueInternal(key.key(), value);
+	}
+
+	/**
 	 * Returns the value associated with the given key as an integer.
 	 * 
 	 * @param key
@@ -161,28 +202,20 @@ public class Configuration extends ExecutionConfig.GlobalJobParameters
 		if (o == null) {
 			return defaultValue;
 		}
-		
-		if (o.getClass() == Integer.class) {
-			return (Integer) o;
-		}
-		else if (o.getClass() == Long.class) {
-			long value = (Long) o;
-			if (value <= Integer.MAX_VALUE && value >= Integer.MIN_VALUE) {
-				return (int) value;
-			} else {
-				LOG.warn("Configuration value {} overflows/underflows the integer type.", value);
-				return defaultValue;
-			}
-		}
-		else {
-			try {
-				return Integer.parseInt(o.toString());
-			}
-			catch (NumberFormatException e) {
-				LOG.warn("Configuration cannot evaluate value {} as an integer number", o);
-				return defaultValue;
-			}
-		}
+
+		return convertToInt(o, defaultValue);
+	}
+
+	/**
+	 * Returns the value associated with the given config option as an integer.
+	 *
+	 * @param configOption The configuration option
+	 * @return the (default) value associated with the given config option
+	 */
+	@PublicEvolving
+	public int getInteger(ConfigOption<Integer> configOption) {
+		Object o = getValueOrDefaultFromOption(configOption);
+		return convertToInt(o, configOption.defaultValue());
 	}
 
 	/**
@@ -198,6 +231,20 @@ public class Configuration extends ExecutionConfig.GlobalJobParameters
 	}
 
 	/**
+	 * Adds the given value to the configuration object.
+	 * The main key of the config option will be used to map the value.
+	 *
+	 * @param key
+	 *        the option specifying the key to be added
+	 * @param value
+	 *        the value of the key/value pair to be added
+	 */
+	@PublicEvolving
+	public void setInteger(ConfigOption<Integer> key, int value) {
+		setValueInternal(key.key(), value);
+	}
+
+	/**
 	 * Returns the value associated with the given key as a long.
 	 * 
 	 * @param key
@@ -211,22 +258,20 @@ public class Configuration extends ExecutionConfig.GlobalJobParameters
 		if (o == null) {
 			return defaultValue;
 		}
-		
-		if (o.getClass() == Long.class) {
-			return (Long) o;
-		}
-		else if (o.getClass() == Integer.class) {
-			return ((Integer) o).longValue();
-		}
-		else {
-			try {
-				return Long.parseLong(o.toString());
-			}
-			catch (NumberFormatException e) {
-				LOG.warn("Configuration cannot evaluate value " + o + " as a long integer number");
-				return defaultValue;
-			}
-		}
+
+		return convertToLong(o, defaultValue);
+	}
+
+	/**
+	 * Returns the value associated with the given config option as a long integer.
+	 *
+	 * @param configOption The configuration option
+	 * @return the (default) value associated with the given config option
+	 */
+	@PublicEvolving
+	public long getLong(ConfigOption<Long> configOption) {
+		Object o = getValueOrDefaultFromOption(configOption);
+		return convertToLong(o, configOption.defaultValue());
 	}
 
 	/**
@@ -242,6 +287,20 @@ public class Configuration extends ExecutionConfig.GlobalJobParameters
 	}
 
 	/**
+	 * Adds the given value to the configuration object.
+	 * The main key of the config option will be used to map the value.
+	 *
+	 * @param key
+	 *        the option specifying the key to be added
+	 * @param value
+	 *        the value of the key/value pair to be added
+	 */
+	@PublicEvolving
+	public void setLong(ConfigOption<Long> key, long value) {
+		setValueInternal(key.key(), value);
+	}
+
+	/**
 	 * Returns the value associated with the given key as a boolean.
 	 * 
 	 * @param key
@@ -255,13 +314,20 @@ public class Configuration extends ExecutionConfig.GlobalJobParameters
 		if (o == null) {
 			return defaultValue;
 		}
-		
-		if (o.getClass() == Boolean.class) {
-			return (Boolean) o;
-		}
-		else {
-			return Boolean.parseBoolean(o.toString());
-		}
+
+		return convertToBoolean(o);
+	}
+
+	/**
+	 * Returns the value associated with the given config option as a boolean.
+	 *
+	 * @param configOption The configuration option
+	 * @return the (default) value associated with the given config option
+	 */
+	@PublicEvolving
+	public boolean getBoolean(ConfigOption<Boolean> configOption) {
+		Object o = getValueOrDefaultFromOption(configOption);
+		return convertToBoolean(o);
 	}
 
 	/**
@@ -277,6 +343,20 @@ public class Configuration extends ExecutionConfig.GlobalJobParameters
 	}
 
 	/**
+	 * Adds the given value to the configuration object.
+	 * The main key of the config option will be used to map the value.
+	 *
+	 * @param key
+	 *        the option specifying the key to be added
+	 * @param value
+	 *        the value of the key/value pair to be added
+	 */
+	@PublicEvolving
+	public void setBoolean(ConfigOption<Boolean> key, boolean value) {
+		setValueInternal(key.key(), value);
+	}
+
+	/**
 	 * Returns the value associated with the given key as a float.
 	 * 
 	 * @param key
@@ -290,28 +370,20 @@ public class Configuration extends ExecutionConfig.GlobalJobParameters
 		if (o == null) {
 			return defaultValue;
 		}
-		
-		if (o.getClass() == Float.class) {
-			return (Float) o;
-		}
-		else if (o.getClass() == Double.class) {
-			double value = ((Double) o);
-			if (value <= Float.MAX_VALUE && value >= Float.MIN_VALUE) {
-				return (float) value;
-			} else {
-				LOG.warn("Configuration value {} overflows/underflows the float type.", value);
-				return defaultValue;
-			}
-		}
-		else {
-			try {
-				return Float.parseFloat(o.toString());
-			}
-			catch (NumberFormatException e) {
-				LOG.warn("Configuration cannot evaluate value {} as a float value", o);
-				return defaultValue;
-			}
-		}
+
+		return convertToFloat(o, defaultValue);
+	}
+
+	/**
+	 * Returns the value associated with the given config option as a float.
+	 *
+	 * @param configOption The configuration option
+	 * @return the (default) value associated with the given config option
+	 */
+	@PublicEvolving
+	public float getFloat(ConfigOption<Float> configOption) {
+		Object o = getValueOrDefaultFromOption(configOption);
+		return convertToFloat(o, configOption.defaultValue());
 	}
 
 	/**
@@ -325,7 +397,21 @@ public class Configuration extends ExecutionConfig.GlobalJobParameters
 	public void setFloat(String key, float value) {
 		setValueInternal(key, value);
 	}
-	
+
+	/**
+	 * Adds the given value to the configuration object.
+	 * The main key of the config option will be used to map the value.
+	 *
+	 * @param key
+	 *        the option specifying the key to be added
+	 * @param value
+	 *        the value of the key/value pair to be added
+	 */
+	@PublicEvolving
+	public void setFloat(ConfigOption<Float> key, float value) {
+		setValueInternal(key.key(), value);
+	}
+
 	/**
 	 * Returns the value associated with the given key as a double.
 	 * 
@@ -340,22 +426,20 @@ public class Configuration extends ExecutionConfig.GlobalJobParameters
 		if (o == null) {
 			return defaultValue;
 		}
-		
-		if (o.getClass() == Double.class) {
-			return (Double) o;
-		}
-		else if (o.getClass() == Float.class) {
-			return ((Float) o).doubleValue();
-		}
-		else {
-			try {
-				return Double.parseDouble(o.toString());
-			}
-			catch (NumberFormatException e) {
-				LOG.warn("Configuration cannot evaluate value {} as a double value", o);
-				return defaultValue;
-			}
-		}
+
+		return convertToDouble(o, defaultValue);
+	}
+
+	/**
+	 * Returns the value associated with the given config option as a {@code double}.
+	 *
+	 * @param configOption The configuration option
+	 * @return the (default) value associated with the given config option
+	 */
+	@PublicEvolving
+	public double getDouble(ConfigOption<Double> configOption) {
+		Object o = getValueOrDefaultFromOption(configOption);
+		return convertToDouble(o, configOption.defaultValue());
 	}
 
 	/**
@@ -369,7 +453,21 @@ public class Configuration extends ExecutionConfig.GlobalJobParameters
 	public void setDouble(String key, double value) {
 		setValueInternal(key, value);
 	}
-	
+
+	/**
+	 * Adds the given value to the configuration object.
+	 * The main key of the config option will be used to map the value.
+	 *
+	 * @param key
+	 *        the option specifying the key to be added
+	 * @param value
+	 *        the value of the key/value pair to be added
+	 */
+	@PublicEvolving
+	public void setDouble(ConfigOption<Double> key, double value) {
+		setValueInternal(key.key(), value);
+	}
+
 	/**
 	 * Returns the value associated with the given key as a byte array.
 	 * 
@@ -407,6 +505,18 @@ public class Configuration extends ExecutionConfig.GlobalJobParameters
 		setValueInternal(key, bytes);
 	}
 
+	/**
+	 * Returns the value associated with the given config option as a string.
+	 *
+	 * @param configOption The configuration option
+	 * @return the (default) value associated with the given config option
+	 */
+	@PublicEvolving
+	public String getValue(ConfigOption<?> configOption) {
+		Object o = getValueOrDefaultFromOption(configOption);
+		return o == null ? null : o.toString();
+	}
+
 	// --------------------------------------------------------------------------------------------
 	
 	/**
@@ -523,7 +633,130 @@ public class Configuration extends ExecutionConfig.GlobalJobParameters
 			return this.confData.get(key);
 		}
 	}
-	
+
+	private Object getRawValueFromOption(ConfigOption<?> configOption) {
+		// first try the current key
+		Object o = getRawValue(configOption.key());
+
+		if (o != null) {
+			return o;
+		}
+		else if (configOption.hasDeprecatedKeys()) {
+			for (String deprecatedKey : configOption.deprecatedKeys()) {
+				Object oo = getRawValue(deprecatedKey);
+				if (oo != null) {
+					return oo;
+				}
+			}
+		}
+
+		return null;
+	}
+
+	private Object getValueOrDefaultFromOption(ConfigOption<?> configOption) {
+		Object o = getRawValueFromOption(configOption);
+		return o != null ? o : configOption.defaultValue();
+	}
+
+	// --------------------------------------------------------------------------------------------
+	//  Type conversion
+	// --------------------------------------------------------------------------------------------
+
+	private int convertToInt(Object o, int defaultValue) {
+		if (o.getClass() == Integer.class) {
+			return (Integer) o;
+		}
+		else if (o.getClass() == Long.class) {
+			long value = (Long) o;
+			if (value <= Integer.MAX_VALUE && value >= Integer.MIN_VALUE) {
+				return (int) value;
+			} else {
+				LOG.warn("Configuration value {} overflows/underflows the integer type.", value);
+				return defaultValue;
+			}
+		}
+		else {
+			try {
+				return Integer.parseInt(o.toString());
+			}
+			catch (NumberFormatException e) {
+				LOG.warn("Configuration cannot evaluate value {} as an integer number", o);
+				return defaultValue;
+			}
+		}
+	}
+
+	private long convertToLong(Object o, long defaultValue) {
+		if (o.getClass() == Long.class) {
+			return (Long) o;
+		}
+		else if (o.getClass() == Integer.class) {
+			return ((Integer) o).longValue();
+		}
+		else {
+			try {
+				return Long.parseLong(o.toString());
+			}
+			catch (NumberFormatException e) {
+				LOG.warn("Configuration cannot evaluate value " + o + " as a long integer number");
+				return defaultValue;
+			}
+		}
+	}
+
+	private boolean convertToBoolean(Object o) {
+		if (o.getClass() == Boolean.class) {
+			return (Boolean) o;
+		}
+		else {
+			return Boolean.parseBoolean(o.toString());
+		}
+	}
+
+	private float convertToFloat(Object o, float defaultValue) {
+		if (o.getClass() == Float.class) {
+			return (Float) o;
+		}
+		else if (o.getClass() == Double.class) {
+			double value = ((Double) o);
+			if (value <= Float.MAX_VALUE && value >= Float.MIN_VALUE) {
+				return (float) value;
+			} else {
+				LOG.warn("Configuration value {} overflows/underflows the float type.", value);
+				return defaultValue;
+			}
+		}
+		else {
+			try {
+				return Float.parseFloat(o.toString());
+			}
+			catch (NumberFormatException e) {
+				LOG.warn("Configuration cannot evaluate value {} as a float value", o);
+				return defaultValue;
+			}
+		}
+	}
+
+	private double convertToDouble(Object o, double defaultValue) {
+		if (o.getClass() == Double.class) {
+			return (Double) o;
+		}
+		else if (o.getClass() == Float.class) {
+			return ((Float) o).doubleValue();
+		}
+		else {
+			try {
+				return Double.parseDouble(o.toString());
+			}
+			catch (NumberFormatException e) {
+				LOG.warn("Configuration cannot evaluate value {} as a double value", o);
+				return defaultValue;
+			}
+		}
+	}
+
+	// --------------------------------------------------------------------------------------------
+	//  Serialization
 	// --------------------------------------------------------------------------------------------
 
 	@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/d71a09cc/flink-core/src/main/java/org/apache/flink/configuration/DelegatingConfiguration.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/configuration/DelegatingConfiguration.java b/flink-core/src/main/java/org/apache/flink/configuration/DelegatingConfiguration.java
index dba77f3..bd9a962 100644
--- a/flink-core/src/main/java/org/apache/flink/configuration/DelegatingConfiguration.java
+++ b/flink-core/src/main/java/org/apache/flink/configuration/DelegatingConfiguration.java
@@ -15,6 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.flink.configuration;
 
 import org.apache.flink.core.memory.DataInputView;
@@ -22,7 +23,11 @@ import org.apache.flink.core.memory.DataOutputView;
 import org.apache.flink.util.Preconditions;
 
 import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
 import java.util.HashSet;
+import java.util.List;
 import java.util.Map;
 import java.util.Properties;
 import java.util.Set;
@@ -55,8 +60,7 @@ public final class DelegatingConfiguration extends Configuration {
 	 * @param backingConfig The configuration holding the actual config data.
 	 * @param prefix The prefix prepended to all config keys.
 	 */
-	public DelegatingConfiguration(Configuration backingConfig, String prefix)
-	{
+	public DelegatingConfiguration(Configuration backingConfig, String prefix) {
 		this.backingConfig = Preconditions.checkNotNull(backingConfig);
 		this.prefix = prefix;
 	}
@@ -69,11 +73,26 @@ public final class DelegatingConfiguration extends Configuration {
 	}
 
 	@Override
+	public String getString(ConfigOption<String> configOption) {
+		return  this.backingConfig.getString(prefixOption(configOption, prefix));
+	}
+
+	@Override
+	public String getString(ConfigOption<String> configOption, String overrideDefault) {
+		return  this.backingConfig.getString(prefixOption(configOption, prefix), overrideDefault);
+	}
+
+	@Override
 	public void setString(String key, String value) {
 		this.backingConfig.setString(this.prefix + key, value);
 	}
 
 	@Override
+	public void setString(ConfigOption<String> key, String value) {
+		this.backingConfig.setString(prefix + key.key(), value);
+	}
+
+	@Override
 	public <T> Class<T> getClass(String key, Class<? extends T> defaultValue, ClassLoader classLoader) throws ClassNotFoundException {
 		return this.backingConfig.getClass(this.prefix + key, defaultValue, classLoader);
 	}
@@ -89,51 +108,101 @@ public final class DelegatingConfiguration extends Configuration {
 	}
 
 	@Override
+	public int getInteger(ConfigOption<Integer> configOption) {
+		return  this.backingConfig.getInteger(prefixOption(configOption, prefix));
+	}
+
+	@Override
 	public void setInteger(String key, int value) {
 		this.backingConfig.setInteger(this.prefix + key, value);
 	}
 
 	@Override
+	public void setInteger(ConfigOption<Integer> key, int value) {
+		this.backingConfig.setInteger(prefix + key.key(), value);
+	}
+
+	@Override
 	public long getLong(String key, long defaultValue) {
 		return this.backingConfig.getLong(this.prefix + key, defaultValue);
 	}
 
 	@Override
+	public long getLong(ConfigOption<Long> configOption) {
+		return  this.backingConfig.getLong(prefixOption(configOption, prefix));
+	}
+
+	@Override
 	public void setLong(String key, long value) {
 		this.backingConfig.setLong(this.prefix + key, value);
 	}
 
 	@Override
+	public void setLong(ConfigOption<Long> key, long value) {
+		this.backingConfig.setLong(prefix + key.key(), value);
+	}
+
+	@Override
 	public boolean getBoolean(String key, boolean defaultValue) {
 		return this.backingConfig.getBoolean(this.prefix + key, defaultValue);
 	}
 
 	@Override
+	public boolean getBoolean(ConfigOption<Boolean> configOption) {
+		return  this.backingConfig.getBoolean(prefixOption(configOption, prefix));
+	}
+
+	@Override
 	public void setBoolean(String key, boolean value) {
 		this.backingConfig.setBoolean(this.prefix + key, value);
 	}
 
 	@Override
+	public void setBoolean(ConfigOption<Boolean> key, boolean value) {
+		this.backingConfig.setBoolean(prefix + key.key(), value);
+	}
+
+	@Override
 	public float getFloat(String key, float defaultValue) {
 		return this.backingConfig.getFloat(this.prefix + key, defaultValue);
 	}
 
 	@Override
+	public float getFloat(ConfigOption<Float> configOption) {
+		return this.backingConfig.getFloat(prefixOption(configOption, prefix));
+	}
+
+	@Override
 	public void setFloat(String key, float value) {
 		this.backingConfig.setFloat(this.prefix + key, value);
 	}
 
 	@Override
+	public void setFloat(ConfigOption<Float> key, float value) {
+		this.backingConfig.setFloat(prefix + key.key(), value);
+	}
+
+	@Override
 	public double getDouble(String key, double defaultValue) {
 		return this.backingConfig.getDouble(this.prefix + key, defaultValue);
 	}
 
 	@Override
+	public double getDouble(ConfigOption<Double> configOption) {
+		return this.backingConfig.getDouble(prefixOption(configOption, prefix));
+	}
+
+	@Override
 	public void setDouble(String key, double value) {
 		this.backingConfig.setDouble(this.prefix + key, value);
 	}
 
 	@Override
+	public void setDouble(ConfigOption<Double> key, double value) {
+		this.backingConfig.setDouble(prefix + key.key(), value);
+	}
+
+	@Override
 	public byte[] getBytes(final String key, final byte[] defaultValue) {
 		return this.backingConfig.getBytes(this.prefix + key, defaultValue);
 	}
@@ -144,6 +213,11 @@ public final class DelegatingConfiguration extends Configuration {
 	}
 
 	@Override
+	public String getValue(ConfigOption<?> configOption) {
+		return this.backingConfig.getValue(prefixOption(configOption, prefix));
+	}
+
+	@Override
 	public void addAllToProperties(Properties props) {
 		// only add keys with our prefix
 		synchronized (backingConfig.confData) {
@@ -195,6 +269,27 @@ public final class DelegatingConfiguration extends Configuration {
 		return set;
 	}
 
+	@Override
+	public Configuration clone() {
+		return new DelegatingConfiguration(backingConfig.clone(), prefix);
+	}
+
+	@Override
+	public Map<String, String> toMap() {
+		Map<String, String> map = backingConfig.toMap();
+		Map<String, String> prefixed = new HashMap<>(map.size());
+		for (Map.Entry<String, String> entry : map.entrySet()) {
+			prefixed.put(prefix + entry.getKey(), entry.getValue());
+		}
+
+		return prefixed; 
+	}
+
+	@Override
+	public boolean containsKey(String key) {
+		return backingConfig.containsKey(prefix + key);
+	}
+
 	// --------------------------------------------------------------------------------------------
 
 	@Override
@@ -225,4 +320,23 @@ public final class DelegatingConfiguration extends Configuration {
 			return false;
 		}
 	}
+
+	// --------------------------------------------------------------------------------------------
+
+	private static <T> ConfigOption<T> prefixOption(ConfigOption<T> option, String prefix) {
+		String key = prefix + option.key();
+
+		List<String> deprecatedKeys;
+		if (option.hasDeprecatedKeys()) {
+			deprecatedKeys = new ArrayList<>();
+			for (String dk : option.deprecatedKeys()) {
+				deprecatedKeys.add(prefix + dk);
+			}
+		} else {
+			deprecatedKeys = Collections.emptyList();
+		}
+
+		String[] deprecated = deprecatedKeys.toArray(new String[deprecatedKeys.size()]);
+		return new ConfigOption<T>(key, option.defaultValue(), deprecated);
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/d71a09cc/flink-core/src/test/java/org/apache/flink/configuration/ConfigurationTest.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/configuration/ConfigurationTest.java b/flink-core/src/test/java/org/apache/flink/configuration/ConfigurationTest.java
index cf3c908..91c5f65 100644
--- a/flink-core/src/test/java/org/apache/flink/configuration/ConfigurationTest.java
+++ b/flink-core/src/test/java/org/apache/flink/configuration/ConfigurationTest.java
@@ -20,6 +20,7 @@ package org.apache.flink.configuration;
 
 import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
@@ -33,7 +34,7 @@ import org.junit.Test;
  * objects is tested.
  */
 public class ConfigurationTest extends TestLogger {
-	
+
 	private static final byte[] EMPTY_BYTES = new byte[0];
 	private static final long TOO_LONG = Integer.MAX_VALUE + 10L;
 	private static final double TOO_LONG_DOUBLE = Double.MAX_VALUE;
@@ -73,7 +74,7 @@ public class ConfigurationTest extends TestLogger {
 			fail(e.getMessage());
 		}
 	}
-	
+
 	@Test
 	public void testConversions() {
 		try {
@@ -175,7 +176,7 @@ public class ConfigurationTest extends TestLogger {
 			fail(e.getMessage());
 		}
 	}
-	
+
 	@Test
 	public void testCopyConstructor() {
 		try {
@@ -194,4 +195,92 @@ public class ConfigurationTest extends TestLogger {
 			fail(e.getMessage());
 		}
 	}
+
+	@Test
+	public void testOptionWithDefault() {
+		Configuration cfg = new Configuration();
+		cfg.setInteger("int-key", 11);
+		cfg.setString("string-key", "abc");
+
+		ConfigOption<String> presentStringOption = ConfigOptions.key("string-key").defaultValue("my-beautiful-default");
+		ConfigOption<Integer> presentIntOption = ConfigOptions.key("int-key").defaultValue(87);
+
+		assertEquals("abc", cfg.getString(presentStringOption));
+		assertEquals("abc", cfg.getValue(presentStringOption));
+
+		assertEquals(11, cfg.getInteger(presentIntOption));
+		assertEquals("11", cfg.getValue(presentIntOption));
+
+		// test getting default when no value is present
+
+		ConfigOption<String> stringOption = ConfigOptions.key("test").defaultValue("my-beautiful-default");
+		ConfigOption<Integer> intOption = ConfigOptions.key("test2").defaultValue(87);
+
+		// getting strings with default value should work
+		assertEquals("my-beautiful-default", cfg.getValue(stringOption));
+		assertEquals("my-beautiful-default", cfg.getString(stringOption));
+
+		// overriding the default should work
+		assertEquals("override", cfg.getString(stringOption, "override"));
+
+		// getting a primitive with a default value should work
+		assertEquals(87, cfg.getInteger(intOption));
+		assertEquals("87", cfg.getValue(intOption));
+	}
+
+	@Test
+	public void testOptionWithNoDefault() {
+		Configuration cfg = new Configuration();
+		cfg.setInteger("int-key", 11);
+		cfg.setString("string-key", "abc");
+
+		ConfigOption<String> presentStringOption = ConfigOptions.key("string-key").noDefaultValue();
+
+		assertEquals("abc", cfg.getString(presentStringOption));
+		assertEquals("abc", cfg.getValue(presentStringOption));
+
+		// test getting default when no value is present
+
+		ConfigOption<String> stringOption = ConfigOptions.key("test").noDefaultValue();
+
+		// getting strings for null should work
+		assertNull(cfg.getValue(stringOption));
+		assertNull(cfg.getString(stringOption));
+
+		// overriding the null default should work
+		assertEquals("override", cfg.getString(stringOption, "override"));
+	}
+
+	@Test
+	public void testDeprecatedKeys() {
+		Configuration cfg = new Configuration();
+		cfg.setInteger("the-key", 11);
+		cfg.setInteger("old-key", 12);
+		cfg.setInteger("older-key", 13);
+
+		ConfigOption<Integer> matchesFirst = ConfigOptions
+				.key("the-key")
+				.defaultValue(-1)
+				.withDeprecatedKeys("old-key", "older-key");
+
+		ConfigOption<Integer> matchesSecond = ConfigOptions
+				.key("does-not-exist")
+				.defaultValue(-1)
+				.withDeprecatedKeys("old-key", "older-key");
+
+		ConfigOption<Integer> matchesThird = ConfigOptions
+				.key("does-not-exist")
+				.defaultValue(-1)
+				.withDeprecatedKeys("foo", "older-key");
+
+		ConfigOption<Integer> notContained = ConfigOptions
+				.key("does-not-exist")
+				.defaultValue(-1)
+				.withDeprecatedKeys("not-there", "also-not-there");
+
+		assertEquals(11, cfg.getInteger(matchesFirst));
+		assertEquals(12, cfg.getInteger(matchesSecond));
+		assertEquals(13, cfg.getInteger(matchesThird));
+		assertEquals(-1, cfg.getInteger(notContained));
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/d71a09cc/flink-core/src/test/java/org/apache/flink/configuration/DelegatingConfigurationTest.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/configuration/DelegatingConfigurationTest.java b/flink-core/src/test/java/org/apache/flink/configuration/DelegatingConfigurationTest.java
index d8b782d..9298a14 100644
--- a/flink-core/src/test/java/org/apache/flink/configuration/DelegatingConfigurationTest.java
+++ b/flink-core/src/test/java/org/apache/flink/configuration/DelegatingConfigurationTest.java
@@ -24,8 +24,6 @@ import org.junit.Test;
 import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
 import java.lang.reflect.Modifier;
-import java.util.Arrays;
-import java.util.Comparator;
 import java.util.Set;
 
 import static org.junit.Assert.assertTrue;
@@ -34,60 +32,43 @@ import static org.junit.Assert.assertEquals;
 
 public class DelegatingConfigurationTest {
 
-	/**
-	 * http://stackoverflow.com/questions/22225663/checking-in-a-unit-test-whether-all-methods-are-delegated
-	 */
 	@Test
 	public void testIfDelegatesImplementAllMethods() throws IllegalAccessException, IllegalArgumentException, InvocationTargetException {
 
-		Comparator<Method> methodComparator = new Comparator<Method>() {
-			@Override
-			public int compare(Method o1, Method o2) {
-				String o1Str = o1.getName() + typeParamToString(o1.getParameterTypes());
-				String o2Str = o2.getName() + typeParamToString(o2.getParameterTypes());
-				return o1Str.compareTo( o2Str ); 
-			}
-
-			private String typeParamToString(Class<?>[] classes) {
-				String str = "";
-				for(Object t : classes) {
-					str += t.toString();
-				}
-				return str;
-			}
-		};
-		
 		// For each method in the Configuration class...
 		Method[] confMethods = Configuration.class.getDeclaredMethods();
 		Method[] delegateMethods = DelegatingConfiguration.class.getDeclaredMethods();
-		Arrays.sort(confMethods, methodComparator);
-		Arrays.sort(delegateMethods, methodComparator);
-		match : for (Method configurationMethod : confMethods) {
-			boolean hasMethod = false;
-			if(!Modifier.isPublic(configurationMethod.getModifiers()) ) {
+
+		for (Method configurationMethod : confMethods) {
+			if (!Modifier.isPublic(configurationMethod.getModifiers()) ) {
 				continue;
 			}
+
+			boolean hasMethod = false;
+
 			// Find matching method in wrapper class and call it
-			mismatch: for (Method wrapperMethod : delegateMethods) {
+			lookForWrapper: for (Method wrapperMethod : delegateMethods) {
 				if (configurationMethod.getName().equals(wrapperMethod.getName())) {
-					
+
 					// Get parameters for method
 					Class<?>[] wrapperMethodParams = wrapperMethod.getParameterTypes();
 					Class<?>[] configMethodParams = configurationMethod.getParameterTypes();
-					if(wrapperMethodParams.length != configMethodParams.length) {
-						System.err.println("Length");
-						break mismatch;
+					if (wrapperMethodParams.length != configMethodParams.length) {
+						continue;
 					}
-					for(int i = 0; i < wrapperMethodParams.length; i++) {
-						if(wrapperMethodParams[i] != configMethodParams[i]) {
-							break mismatch;
+
+					for (int i = 0; i < wrapperMethodParams.length; i++) {
+						if (wrapperMethodParams[i] != configMethodParams[i]) {
+							continue lookForWrapper;
 						}
 					}
 					hasMethod = true;
-					break match;
+					break;
 				}
 			}
-			assertTrue("Foo method '" + configurationMethod.getName() + "' has not been wrapped correctly in DelegatingConfiguration wrapper", hasMethod);
+
+			assertTrue("Configuration method '" + configurationMethod.getName() + 
+					"' has not been wrapped correctly in DelegatingConfiguration wrapper", hasMethod);
 		}
 	}
 	

http://git-wip-us.apache.org/repos/asf/flink/blob/d71a09cc/flink-core/src/test/java/org/apache/flink/configuration/UnmodifiableConfigurationTest.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/configuration/UnmodifiableConfigurationTest.java b/flink-core/src/test/java/org/apache/flink/configuration/UnmodifiableConfigurationTest.java
index 386d03b..26e3d7a 100644
--- a/flink-core/src/test/java/org/apache/flink/configuration/UnmodifiableConfigurationTest.java
+++ b/flink-core/src/test/java/org/apache/flink/configuration/UnmodifiableConfigurationTest.java
@@ -56,6 +56,9 @@ public class UnmodifiableConfigurationTest extends TestLogger {
 	@Test
 	public void testExceptionOnSet() {
 		try {
+			@SuppressWarnings("rawtypes")
+			final ConfigOption rawOption = ConfigOptions.key("testkey").defaultValue("value");
+
 			Map<Class<?>, Object> parameters = new HashMap<Class<?>, Object>();
 			parameters.put(byte[].class, new byte[0]);
 			parameters.put(Class.class, Object.class);
@@ -65,19 +68,22 @@ public class UnmodifiableConfigurationTest extends TestLogger {
 			parameters.put(double.class, 0.0);
 			parameters.put(String.class, "");
 			parameters.put(boolean.class, false);
-					
+
 			Class<UnmodifiableConfiguration> clazz = UnmodifiableConfiguration.class;
 			UnmodifiableConfiguration config = new UnmodifiableConfiguration(new Configuration());
-			
+
 			for (Method m : clazz.getMethods()) {
 				if (m.getName().startsWith("set")) {
-					
+
+					Class<?> keyClass = m.getParameterTypes()[0];
 					Class<?> parameterClass = m.getParameterTypes()[1];
+					Object key = keyClass == String.class ? "key" : rawOption;
+
 					Object parameter = parameters.get(parameterClass);
 					assertNotNull("method " + m + " not covered by test", parameter);
-					
+
 					try {
-						m.invoke(config, "key", parameter);
+						m.invoke(config, key, parameter);
 						fail("should fail with an exception");
 					}
 					catch (InvocationTargetException e) {


[6/8] flink git commit: [FLINK-4776] [distributed coordination] Move ExecutionGraph initialization into the dedicated class ExecutionGraphBuilder

Posted by se...@apache.org.
[FLINK-4776] [distributed coordination] Move ExecutionGraph initialization into the dedicated class ExecutionGraphBuilder


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

Branch: refs/heads/flip-6
Commit: 05436f4b64e771b22f13f56ff9e0ea7aa94b4ff7
Parents: 21b9f16
Author: Stephan Ewen <se...@apache.org>
Authored: Fri Oct 7 19:58:24 2016 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Thu Oct 13 16:25:49 2016 +0200

----------------------------------------------------------------------
 .../checkpoint/CheckpointCoordinator.java       |   4 +-
 .../runtime/executiongraph/ExecutionGraph.java  |   8 +-
 .../executiongraph/ExecutionGraphBuilder.java   | 262 +++++++++++++++++++
 .../flink/runtime/jobmanager/JobManager.scala   | 168 ++----------
 4 files changed, 297 insertions(+), 145 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/05436f4b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java
index 4428427..e95afe0 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java
@@ -171,7 +171,7 @@ public class CheckpointCoordinator {
 			CheckpointIDCounter checkpointIDCounter,
 			CompletedCheckpointStore completedCheckpointStore,
 			SavepointStore savepointStore,
-			CheckpointStatsTracker statsTracker) throws Exception {
+			CheckpointStatsTracker statsTracker) {
 
 		// sanity checks
 		checkArgument(baseInterval > 0, "Checkpoint timeout must be larger than zero");
@@ -207,7 +207,7 @@ public class CheckpointCoordinator {
 			// issues a blocking call to ZooKeeper.
 			checkpointIDCounter.start();
 		} catch (Throwable t) {
-			throw new Exception("Failed to start checkpoint ID counter: " + t.getMessage(), t);
+			throw new RuntimeException("Failed to start checkpoint ID counter: " + t.getMessage(), t);
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/05436f4b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java
index 6023205..cf98ca6 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java
@@ -359,7 +359,7 @@ public class ExecutionGraph {
 			CheckpointIDCounter checkpointIDCounter,
 			CompletedCheckpointStore checkpointStore,
 			SavepointStore savepointStore,
-			CheckpointStatsTracker statsTracker) throws Exception {
+			CheckpointStatsTracker statsTracker) {
 
 		// simple sanity checks
 		if (interval < 10 || checkpointTimeout < 10) {
@@ -374,7 +374,11 @@ public class ExecutionGraph {
 		ExecutionVertex[] tasksToCommitTo = collectExecutionVertices(verticesToCommitTo);
 
 		// disable to make sure existing checkpoint coordinators are cleared
-		disableSnaphotCheckpointing();
+		try {
+			disableSnaphotCheckpointing();
+		} catch (Throwable t) {
+			LOG.error("Error while shutting down checkpointer.");
+		}
 
 		checkpointStatsTracker = Objects.requireNonNull(statsTracker, "Checkpoint stats tracker");
 

http://git-wip-us.apache.org/repos/asf/flink/blob/05436f4b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraphBuilder.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraphBuilder.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraphBuilder.java
new file mode 100644
index 0000000..1c6eb8d
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraphBuilder.java
@@ -0,0 +1,262 @@
+/*
+ * 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.executiongraph;
+
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.runtime.JobException;
+import org.apache.flink.runtime.checkpoint.CheckpointIDCounter;
+import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory;
+import org.apache.flink.runtime.checkpoint.CompletedCheckpointStore;
+import org.apache.flink.runtime.checkpoint.savepoint.SavepointStore;
+import org.apache.flink.runtime.checkpoint.stats.CheckpointStatsTracker;
+import org.apache.flink.runtime.checkpoint.stats.DisabledCheckpointStatsTracker;
+import org.apache.flink.runtime.checkpoint.stats.SimpleCheckpointStatsTracker;
+import org.apache.flink.runtime.client.JobExecutionException;
+import org.apache.flink.runtime.client.JobSubmissionException;
+import org.apache.flink.runtime.executiongraph.restart.RestartStrategy;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.jobgraph.jsonplan.JsonPlanGenerator;
+import org.apache.flink.runtime.jobgraph.tasks.JobSnapshottingSettings;
+
+import org.slf4j.Logger;
+
+import scala.concurrent.ExecutionContext;
+import scala.concurrent.ExecutionContext$;
+import scala.concurrent.duration.FiniteDuration;
+
+import javax.annotation.Nullable;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.Executor;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Utility class to encapsulate the logic of building an {@link ExecutionGraph} from a {@link JobGraph}.
+ */
+public class ExecutionGraphBuilder {
+
+	/**
+	 * Builds the ExecutionGraph from the JobGraph.
+	 * If a prior execution graph exists, the JobGraph will be attached. If no prior execution
+	 * graph exists, then the JobGraph will become attach to a new emoty execution graph.
+	 */
+	public static ExecutionGraph buildGraph(
+			@Nullable ExecutionGraph prior,
+			JobGraph jobGraph,
+			Configuration jobManagerConfig,
+			Executor executor,
+			ClassLoader classLoader,
+			CheckpointRecoveryFactory recoveryFactory,
+			SavepointStore savepointStore,
+			Time timeout,
+			RestartStrategy restartStrategy,
+			MetricGroup metrics,
+			int parallelismForAutoMax,
+			Logger log)
+			throws JobExecutionException, JobException
+	{
+		final ExecutionContext executionContext = ExecutionContext$.MODULE$.fromExecutor(executor);
+		
+		return buildGraph(prior, jobGraph, jobManagerConfig, executionContext,
+				classLoader, recoveryFactory, savepointStore, timeout, restartStrategy,
+				metrics, parallelismForAutoMax, log);
+	}
+
+	/**
+	 * Builds the ExecutionGraph from the JobGraph.
+	 * If a prior execution graph exists, the JobGraph will be attached. If no prior execution
+	 * graph exists, then the JobGraph will become attach to a new emoty execution graph.
+	 */
+	public static ExecutionGraph buildGraph(
+			@Nullable ExecutionGraph prior,
+			JobGraph jobGraph,
+			Configuration jobManagerConfig,
+			ExecutionContext executionContext,
+			ClassLoader classLoader,
+			CheckpointRecoveryFactory recoveryFactory,
+			SavepointStore savepointStore,
+			Time timeout,
+			RestartStrategy restartStrategy,
+			MetricGroup metrics,
+			int parallelismForAutoMax,
+			Logger log)
+		throws JobExecutionException, JobException
+	{
+		checkNotNull(jobGraph, "job graph cannot be null");
+
+		final String jobName = jobGraph.getName();
+		final JobID jobId = jobGraph.getJobID();
+
+		// create a new execution graph, if none exists so far
+		final ExecutionGraph executionGraph = (prior != null) ? prior :
+				new ExecutionGraph(
+						executionContext,
+						jobId,
+						jobName,
+						jobGraph.getJobConfiguration(),
+						jobGraph.getSerializedExecutionConfig(),
+						new FiniteDuration(timeout.getSize(), timeout.getUnit()),
+						restartStrategy,
+						jobGraph.getUserJarBlobKeys(),
+						jobGraph.getClasspaths(),
+						classLoader,
+						metrics);
+
+		// set the basic properties
+
+		executionGraph.setScheduleMode(jobGraph.getScheduleMode());
+		executionGraph.setQueuedSchedulingAllowed(jobGraph.getAllowQueuedScheduling());
+
+		try {
+			executionGraph.setJsonPlan(JsonPlanGenerator.generatePlan(jobGraph));
+		}
+		catch (Throwable t) {
+			log.warn("Cannot create JSON plan for job", t);
+			// give the graph an empty plan
+			executionGraph.setJsonPlan("{}");
+		}
+
+		// initialize the vertices that have a master initialization hook
+		// file output formats create directories here, input formats create splits
+
+		final long initMasterStart = System.nanoTime();
+		log.info("Running initialization on master for job {} ({}).", jobName, jobId);
+
+		for (JobVertex vertex : jobGraph.getVertices()) {
+			String executableClass = vertex.getInvokableClassName();
+			if (executableClass == null || executableClass.isEmpty()) {
+				throw new JobSubmissionException(jobId,
+						"The vertex " + vertex.getID() + " (" + vertex.getName() + ") has no invokable class.");
+			}
+
+			if (vertex.getParallelism() == ExecutionConfig.PARALLELISM_AUTO_MAX) {
+				vertex.setParallelism(parallelismForAutoMax);
+			}
+
+			try {
+				vertex.initializeOnMaster(classLoader);
+			}
+			catch (Throwable t) {
+					throw new JobExecutionException(jobId,
+							"Cannot initialize task '" + vertex.getName() + "': " + t.getMessage(), t);
+			}
+		}
+
+		log.info("Successfully ran initialization on master in {} ms.",
+				(System.nanoTime() - initMasterStart) / 1_000_000);
+
+		// topologically sort the job vertices and attach the graph to the existing one
+		List<JobVertex> sortedTopology = jobGraph.getVerticesSortedTopologicallyFromSources();
+		if (log.isDebugEnabled()) {
+			log.debug("Adding {} vertices from job graph {} ({}).", sortedTopology.size(), jobName, jobId);
+		}
+		executionGraph.attachJobGraph(sortedTopology);
+
+		if (log.isDebugEnabled()) {
+			log.debug("Successfully created execution graph from job graph {} ({}).", jobName, jobId);
+		}
+
+		// configure the state checkpointing
+		JobSnapshottingSettings snapshotSettings = jobGraph.getSnapshotSettings();
+		if (snapshotSettings != null) {
+
+			List<ExecutionJobVertex> triggerVertices = 
+					idToVertex(snapshotSettings.getVerticesToTrigger(), executionGraph);
+
+			List<ExecutionJobVertex> ackVertices =
+					idToVertex(snapshotSettings.getVerticesToAcknowledge(), executionGraph);
+
+			List<ExecutionJobVertex> confirmVertices =
+					idToVertex(snapshotSettings.getVerticesToConfirm(), executionGraph);
+
+			CompletedCheckpointStore completedCheckpoints;
+			CheckpointIDCounter checkpointIdCounter;
+			try {
+				completedCheckpoints = recoveryFactory.createCheckpointStore(jobId, classLoader);
+				checkpointIdCounter = recoveryFactory.createCheckpointIDCounter(jobId);
+			}
+			catch (Exception e) {
+				throw new JobExecutionException(jobId, "Failed to initialize high-availability checkpoint handler", e);
+			}
+
+			// Checkpoint stats tracker
+			boolean isStatsDisabled = jobManagerConfig.getBoolean(
+					ConfigConstants.JOB_MANAGER_WEB_CHECKPOINTS_DISABLE,
+					ConfigConstants.DEFAULT_JOB_MANAGER_WEB_CHECKPOINTS_DISABLE);
+
+			CheckpointStatsTracker checkpointStatsTracker;
+			if (isStatsDisabled) {
+				checkpointStatsTracker = new DisabledCheckpointStatsTracker();
+			}
+			else {
+				int historySize = jobManagerConfig.getInteger(
+						ConfigConstants.JOB_MANAGER_WEB_CHECKPOINTS_HISTORY_SIZE,
+						ConfigConstants.DEFAULT_JOB_MANAGER_WEB_CHECKPOINTS_HISTORY_SIZE);
+
+				checkpointStatsTracker = new SimpleCheckpointStatsTracker(historySize, ackVertices, metrics);
+			}
+
+			executionGraph.enableSnapshotCheckpointing(
+					snapshotSettings.getCheckpointInterval(),
+					snapshotSettings.getCheckpointTimeout(),
+					snapshotSettings.getMinPauseBetweenCheckpoints(),
+					snapshotSettings.getMaxConcurrentCheckpoints(),
+					triggerVertices,
+					ackVertices,
+					confirmVertices,
+					checkpointIdCounter,
+					completedCheckpoints,
+					savepointStore,
+					checkpointStatsTracker);
+		}
+
+		return executionGraph;
+	}
+
+	private static List<ExecutionJobVertex> idToVertex(
+			List<JobVertexID> jobVertices, ExecutionGraph executionGraph) throws IllegalArgumentException {
+
+		List<ExecutionJobVertex> result = new ArrayList<>(jobVertices.size());
+
+		for (JobVertexID id : jobVertices) {
+			ExecutionJobVertex vertex = executionGraph.getJobVertex(id);
+			if (vertex != null) {
+				result.add(vertex);
+			} else {
+				throw new IllegalArgumentException(
+						"The snapshot checkpointing settings refer to non-existent vertex " + id);
+			} 
+		}
+
+		return result;
+	}
+
+	// ------------------------------------------------------------------------
+
+	/** This class is not supposed to be instantiated */
+	private ExecutionGraphBuilder() {}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/05436f4b/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 01f9cec..e90f2d2 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
@@ -29,7 +29,8 @@ import akka.actor.Status.{Failure, Success}
 import akka.actor._
 import akka.pattern.ask
 import grizzled.slf4j.Logger
-import org.apache.flink.api.common.{ExecutionConfig, JobID}
+import org.apache.flink.api.common.JobID
+import org.apache.flink.api.common.time.Time
 import org.apache.flink.configuration.{ConfigConstants, Configuration, GlobalConfiguration}
 import org.apache.flink.core.fs.FileSystem
 import org.apache.flink.core.io.InputSplitAssigner
@@ -49,11 +50,10 @@ import org.apache.flink.runtime.clusterframework.standalone.StandaloneResourceMa
 import org.apache.flink.runtime.clusterframework.types.ResourceID
 import org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager
 import org.apache.flink.runtime.executiongraph.restart.RestartStrategyFactory
-import org.apache.flink.runtime.executiongraph.{ExecutionGraph, ExecutionJobVertex, StatusListenerMessenger}
+import org.apache.flink.runtime.executiongraph.{ExecutionGraphBuilder, ExecutionGraph, ExecutionJobVertex, StatusListenerMessenger}
 import org.apache.flink.runtime.instance.{AkkaActorGateway, InstanceManager}
 import org.apache.flink.runtime.io.network.PartitionState
-import org.apache.flink.runtime.jobgraph.jsonplan.JsonPlanGenerator
-import org.apache.flink.runtime.jobgraph.{JobGraph, JobStatus, JobVertexID}
+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.leaderelection.{LeaderContender, LeaderElectionService, StandaloneLeaderElectionService}
@@ -1114,7 +1114,7 @@ class JobManager(
           Option(jobGraph.getSerializedExecutionConfig()
             .deserializeValue(userCodeLoader)
             .getRestartStrategy())
-            .map(RestartStrategyFactory.createRestartStrategy(_)) match {
+            .map(RestartStrategyFactory.createRestartStrategy) match {
             case Some(strategy) => strategy
             case None => restartStrategyFactory.createRestartStrategy()
           }
@@ -1131,148 +1131,34 @@ class JobManager(
             new UnregisteredMetricsGroup()
         }
 
+        val numSlots = scheduler.getTotalNumberOfSlots()
+
         // see if there already exists an ExecutionGraph for the corresponding job ID
-        executionGraph = currentJobs.get(jobGraph.getJobID) match {
+        val registerNewGraph = currentJobs.get(jobGraph.getJobID) match {
           case Some((graph, currentJobInfo)) =>
+            executionGraph = graph
             currentJobInfo.setLastActive()
-            graph
+            false
           case None =>
-            val graph = new ExecutionGraph(
-              executionContext,
-              jobGraph.getJobID,
-              jobGraph.getName,
-              jobGraph.getJobConfiguration,
-              jobGraph.getSerializedExecutionConfig,
-              timeout,
-              restartStrategy,
-              jobGraph.getUserJarBlobKeys,
-              jobGraph.getClasspaths,
-              userCodeLoader,
-              jobMetrics)
-
-            currentJobs.put(jobGraph.getJobID, (graph, jobInfo))
-            graph
-        }
-
-        executionGraph.setScheduleMode(jobGraph.getScheduleMode())
-        executionGraph.setQueuedSchedulingAllowed(jobGraph.getAllowQueuedScheduling())
-
-        try {
-          executionGraph.setJsonPlan(JsonPlanGenerator.generatePlan(jobGraph))
-        }
-        catch {
-          case t: Throwable =>
-            log.warn("Cannot create JSON plan for job", t)
-            executionGraph.setJsonPlan("{}")
-        }
-
-        // initialize the vertices that have a master initialization hook
-        // file output formats create directories here, input formats create splits
-        if (log.isDebugEnabled) {
-          log.debug(s"Running initialization on master for job $jobId ($jobName).")
-        }
-
-        val numSlots = scheduler.getTotalNumberOfSlots()
-
-        for (vertex <- jobGraph.getVertices.asScala) {
-          val executableClass = vertex.getInvokableClassName
-          if (executableClass == null || executableClass.length == 0) {
-            throw new JobSubmissionException(jobId,
-              s"The vertex ${vertex.getID} (${vertex.getName}) has no invokable class.")
-          }
-
-          if (vertex.getParallelism() == ExecutionConfig.PARALLELISM_AUTO_MAX) {
-            vertex.setParallelism(numSlots)
-          }
-
-          try {
-            vertex.initializeOnMaster(userCodeLoader)
-          }
-          catch {
-            case t: Throwable =>
-              throw new JobExecutionException(jobId,
-                "Cannot initialize task '" + vertex.getName() + "': " + t.getMessage, t)
-          }
+            true
         }
 
-        // topologically sort the job vertices and attach the graph to the existing one
-        val sortedTopology = jobGraph.getVerticesSortedTopologicallyFromSources()
-        if (log.isDebugEnabled) {
-          log.debug(s"Adding ${sortedTopology.size()} vertices from " +
-            s"job graph $jobId ($jobName).")
-        }
-        executionGraph.attachJobGraph(sortedTopology)
-
-        if (log.isDebugEnabled) {
-          log.debug("Successfully created execution graph from job " +
-            s"graph $jobId ($jobName).")
-        }
-
-        // configure the state checkpointing
-        val snapshotSettings = jobGraph.getSnapshotSettings
-        if (snapshotSettings != null) {
-          val jobId = jobGraph.getJobID()
-
-          val idToVertex: JobVertexID => ExecutionJobVertex = id => {
-            val vertex = executionGraph.getJobVertex(id)
-            if (vertex == null) {
-              throw new JobSubmissionException(jobId,
-                "The snapshot checkpointing settings refer to non-existent vertex " + id)
-            }
-            vertex
-          }
-
-          val triggerVertices: java.util.List[ExecutionJobVertex] =
-            snapshotSettings.getVerticesToTrigger().asScala.map(idToVertex).asJava
-
-          val ackVertices: java.util.List[ExecutionJobVertex] =
-            snapshotSettings.getVerticesToAcknowledge().asScala.map(idToVertex).asJava
-
-          val confirmVertices: java.util.List[ExecutionJobVertex] =
-            snapshotSettings.getVerticesToConfirm().asScala.map(idToVertex).asJava
-
-          val completedCheckpoints = checkpointRecoveryFactory
-            .createCheckpointStore(jobId, userCodeLoader)
-
-          val checkpointIdCounter = checkpointRecoveryFactory.createCheckpointIDCounter(jobId)
-
-          // Checkpoint stats tracker
-          val isStatsDisabled: Boolean = flinkConfiguration.getBoolean(
-            ConfigConstants.JOB_MANAGER_WEB_CHECKPOINTS_DISABLE,
-            ConfigConstants.DEFAULT_JOB_MANAGER_WEB_CHECKPOINTS_DISABLE)
-
-          val checkpointStatsTracker : CheckpointStatsTracker =
-            if (isStatsDisabled) {
-              new DisabledCheckpointStatsTracker()
-            } else {
-              val historySize: Int = flinkConfiguration.getInteger(
-                ConfigConstants.JOB_MANAGER_WEB_CHECKPOINTS_HISTORY_SIZE,
-                ConfigConstants.DEFAULT_JOB_MANAGER_WEB_CHECKPOINTS_HISTORY_SIZE)
-
-              new SimpleCheckpointStatsTracker(historySize, ackVertices, jobMetrics)
-            }
-
-          val jobParallelism = jobGraph.getSerializedExecutionConfig
-            .deserializeValue(userCodeLoader).getParallelism()
-
-          val parallelism = if (jobParallelism == ExecutionConfig.PARALLELISM_AUTO_MAX) {
-            numSlots
-          } else {
-            jobParallelism
-          }
-
-          executionGraph.enableSnapshotCheckpointing(
-            snapshotSettings.getCheckpointInterval,
-            snapshotSettings.getCheckpointTimeout,
-            snapshotSettings.getMinPauseBetweenCheckpoints,
-            snapshotSettings.getMaxConcurrentCheckpoints,
-            triggerVertices,
-            ackVertices,
-            confirmVertices,
-            checkpointIdCounter,
-            completedCheckpoints,
-            savepointStore,
-            checkpointStatsTracker)
+        executionGraph = ExecutionGraphBuilder.buildGraph(
+          executionGraph,
+          jobGraph,
+          flinkConfiguration,
+          executionContext,
+          userCodeLoader,
+          checkpointRecoveryFactory,
+          savepointStore,
+          Time.of(timeout.length, timeout.unit),
+          restartStrategy,
+          jobMetrics,
+          numSlots,
+          log.logger)
+        
+        if (registerNewGraph) {
+          currentJobs.put(jobGraph.getJobID, (executionGraph, jobInfo))
         }
 
         // get notified about job status changes


[4/8] flink git commit: [FLINK-4768] [core] Migrate high-availability configuration parameters to ConfigOptions

Posted by se...@apache.org.
[FLINK-4768] [core] Migrate high-availability configuration parameters to ConfigOptions

This closes #2607


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

Branch: refs/heads/flip-6
Commit: c8dc074a1899fa0f7d6ce7c6377c5e3d30159c18
Parents: d71a09c
Author: Stephan Ewen <se...@apache.org>
Authored: Sat Oct 8 01:41:02 2016 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Thu Oct 13 16:25:49 2016 +0200

----------------------------------------------------------------------
 .../org/apache/flink/client/cli/DefaultCLI.java |   5 +-
 .../configuration/HighAvailabilityOptions.java  | 139 +++++++++++++++++++
 .../webmonitor/WebRuntimeMonitorITCase.java     |   7 +-
 .../flink/runtime/blob/FileSystemBlobStore.java |  22 ++-
 .../jobmanager/HighAvailabilityMode.java        |   8 +-
 .../flink/runtime/security/SecurityContext.java |  11 +-
 .../flink/runtime/util/ZooKeeperUtils.java      |  68 +++------
 .../zookeeper/FlinkZooKeeperQuorumPeer.java     |  46 +++---
 .../flink/runtime/jobmanager/JobManager.scala   |  14 +-
 .../flink/runtime/blob/BlobRecoveryITCase.java  |   5 +-
 .../BlobLibraryCacheRecoveryITCase.java         |   5 +-
 .../jobmanager/HighAvailabilityModeTest.java    |  13 +-
 .../jobmanager/JobManagerHARecoveryTest.java    |   5 +-
 .../ZooKeeperLeaderElectionTest.java            |  25 ++--
 .../ZooKeeperLeaderRetrievalTest.java           |  15 +-
 .../runtime/testutils/ZooKeeperTestUtils.java   |  13 +-
 .../flink/runtime/util/ZooKeeperUtilTest.java   |   3 +-
 .../zookeeper/ZooKeeperTestEnvironment.java     |  10 +-
 .../runtime/testingUtils/TestingUtils.scala     |  13 +-
 .../connectors/fs/RollingSinkSecuredITCase.java |   5 +-
 .../flink/test/util/SecureTestEnvironment.java  |   3 +-
 .../apache/flink/test/util/TestBaseUtils.java   |   3 +-
 .../flink/test/recovery/ChaosMonkeyITCase.java  |   3 +-
 ...agerHAProcessFailureBatchRecoveryITCase.java |   5 +-
 ...CliFrontendYarnAddressConfigurationTest.java |  11 +-
 .../flink/yarn/YARNHighAvailabilityITCase.java  |   3 +-
 .../yarn/AbstractYarnClusterDescriptor.java     |   5 +-
 .../flink/yarn/YarnApplicationMasterRunner.java |   3 +-
 .../flink/yarn/cli/FlinkYarnSessionCli.java     |   6 +-
 29 files changed, 302 insertions(+), 172 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/c8dc074a/flink-clients/src/main/java/org/apache/flink/client/cli/DefaultCLI.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/cli/DefaultCLI.java b/flink-clients/src/main/java/org/apache/flink/client/cli/DefaultCLI.java
index 18fa323..8f79403 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/cli/DefaultCLI.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/cli/DefaultCLI.java
@@ -19,11 +19,12 @@ package org.apache.flink.client.cli;
 
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.Options;
+
 import org.apache.flink.client.ClientUtils;
 import org.apache.flink.client.deployment.StandaloneClusterDescriptor;
 import org.apache.flink.client.program.StandaloneClusterClient;
-import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
 
 import java.net.InetSocketAddress;
 
@@ -64,7 +65,7 @@ public class DefaultCLI implements CustomCommandLine<StandaloneClusterClient> {
 
 		if (commandLine.hasOption(CliFrontendParser.ZOOKEEPER_NAMESPACE_OPTION.getOpt())) {
 			String zkNamespace = commandLine.getOptionValue(CliFrontendParser.ZOOKEEPER_NAMESPACE_OPTION.getOpt());
-			config.setString(ConfigConstants.HA_ZOOKEEPER_NAMESPACE_KEY, zkNamespace);
+			config.setString(HighAvailabilityOptions.HA_CLUSTER_ID.key(), zkNamespace);
 		}
 
 		StandaloneClusterDescriptor descriptor = new StandaloneClusterDescriptor(config);

http://git-wip-us.apache.org/repos/asf/flink/blob/c8dc074a/flink-core/src/main/java/org/apache/flink/configuration/HighAvailabilityOptions.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/configuration/HighAvailabilityOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/HighAvailabilityOptions.java
new file mode 100644
index 0000000..1ee988a
--- /dev/null
+++ b/flink-core/src/main/java/org/apache/flink/configuration/HighAvailabilityOptions.java
@@ -0,0 +1,139 @@
+/*
+ * 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.configuration;
+
+import org.apache.flink.annotation.PublicEvolving;
+
+import static org.apache.flink.configuration.ConfigOptions.key;
+
+/**
+ * The set of configuration options relating to high-availability settings.
+ */
+@PublicEvolving
+public class HighAvailabilityOptions {
+
+	// ------------------------------------------------------------------------
+	//  Required High Availability Options
+	// ------------------------------------------------------------------------
+
+	/** 
+	 * Defines high-availability mode used for the cluster execution.
+	 * A value of "NONE" signals no highly available setup.
+	 * To enable high-availability, set this mode to "ZOOKEEPER".
+	 */
+	public static final ConfigOption<String> HA_MODE = 
+			key("high-availability")
+			.defaultValue("NONE")
+			.withDeprecatedKeys("recovery.mode");
+
+	/**
+	 * The ID of the Flink cluster, used to separate multiple Flink clusters 
+	 * Needs to be set for standalone clusters, is automatically inferred in YARN and Mesos.
+	 */
+	public static final ConfigOption<String> HA_CLUSTER_ID = 
+			key("high-availability.cluster-id")
+			.defaultValue("/default")
+			.withDeprecatedKeys("high-availability.zookeeper.path.namespace", "recovery.zookeeper.path.namespace");
+
+	/**
+	 * File system path (URI) where Flink persists metadata in high-availability setups
+	 */
+	public static final ConfigOption<String> HA_STORAGE_PATH =
+			key("high-availability.storageDir")
+			.noDefaultValue()
+			.withDeprecatedKeys("high-availability.zookeeper.storageDir", "recovery.zookeeper.storageDir");
+
+	/**
+	 * The ZooKeeper quorum to use, when running Flink in a high-availability mode with ZooKeeper.
+	 */
+	public static final ConfigOption<String> HA_ZOOKEEPER_QUORUM =
+			key("high-availability.zookeeper.quorum")
+			.noDefaultValue()
+			.withDeprecatedKeys("recovery.zookeeper.quorum");
+	
+
+	// ------------------------------------------------------------------------
+	//  Recovery Options
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Optional port (range) used by the job manager in high-availability mode.
+	 */
+	public static final ConfigOption<String> HA_JOB_MANAGER_PORT_RANGE = 
+			key("high-availability.jobmanager.port")
+			.defaultValue("0")
+			.withDeprecatedKeys("recovery.jobmanager.port");
+
+	/**
+	 * The time before a JobManager after a fail over recovers the current jobs.
+	 */
+	public static final ConfigOption<String> HA_JOB_DELAY = 
+			key("high-availability.job.delay")
+			.noDefaultValue()
+			.withDeprecatedKeys("recovery.job.delay");
+
+	// ------------------------------------------------------------------------
+	//  ZooKeeper Options
+	// ------------------------------------------------------------------------
+
+	/**
+	 * The root path under which Flink stores its entries in ZooKeeper
+	 */
+	public static final ConfigOption<String> HA_ZOOKEEPER_ROOT =
+			key("high-availability.zookeeper.path.root")
+			.defaultValue("/flink")
+			.withDeprecatedKeys("recovery.zookeeper.path.root");
+
+	// ------------------------------------------------------------------------
+	//  ZooKeeper Client Settings
+	// ------------------------------------------------------------------------
+
+	public static final ConfigOption<Integer> ZOOKEEPER_SESSION_TIMEOUT = 
+			key("high-availability.zookeeper.client.session-timeout")
+			.defaultValue(60000)
+			.withDeprecatedKeys("recovery.zookeeper.client.session-timeout");
+
+	public static final ConfigOption<Integer> ZOOKEEPER_CONNECTION_TIMEOUT =
+			key("high-availability.zookeeper.client.connection-timeout")
+			.defaultValue(15000)
+			.withDeprecatedKeys("recovery.zookeeper.client.connection-timeout");
+
+	public static final ConfigOption<Integer> ZOOKEEPER_RETRY_WAIT = 
+			key("high-availability.zookeeper.client.retry-wait")
+			.defaultValue(5000)
+			.withDeprecatedKeys("recovery.zookeeper.client.retry-wait");
+
+	public static final ConfigOption<Integer> ZOOKEEPER_MAX_RETRY_ATTEMPTS = 
+			key("high-availability.zookeeper.client.max-retry-attempts")
+			.defaultValue(3)
+			.withDeprecatedKeys("recovery.zookeeper.client.max-retry-attempts");
+
+	public static final ConfigOption<Boolean> ZOOKEEPER_SASL_DISABLE = 
+			key("zookeeper.sasl.disable")
+			.defaultValue(true);
+
+	public static final ConfigOption<String> ZOOKEEPER_SASL_SERVICE_NAME = 
+			key("zookeeper.sasl.service-name")
+			.noDefaultValue();
+
+	// ------------------------------------------------------------------------
+
+	/** Not intended to be instantiated */
+	private HighAvailabilityOptions() {}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/c8dc074a/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 54c5e76..1ae776c 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
@@ -24,6 +24,7 @@ import io.netty.handler.codec.http.HttpResponseStatus;
 import org.apache.curator.test.TestingServer;
 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.jobmanager.JobManager;
 import org.apache.flink.runtime.jobmanager.MemoryArchivist;
@@ -237,7 +238,7 @@ public class WebRuntimeMonitorITCase extends TestLogger {
 				followingClient.sendGetRequest("index.html", deadline.timeLeft());
 				response = followingClient.getNextResponse(deadline.timeLeft());
 				assertEquals(HttpResponseStatus.TEMPORARY_REDIRECT, response.getStatus());
-				assertTrue(response.getLocation().contains("" + leadingWebMonitor.getServerPort()));
+				assertTrue(response.getLocation().contains(String.valueOf(leadingWebMonitor.getServerPort())));
 
 				// Kill the leader
 				leadingSystem.shutdown();
@@ -296,8 +297,8 @@ public class WebRuntimeMonitorITCase extends TestLogger {
 			final Configuration config = new Configuration();
 			config.setInteger(ConfigConstants.JOB_MANAGER_WEB_PORT_KEY, 0);
 			config.setString(ConfigConstants.JOB_MANAGER_WEB_LOG_PATH_KEY, logFile.toString());
-			config.setString(ConfigConstants.HA_MODE, "ZOOKEEPER");
-			config.setString(ConfigConstants.HA_ZOOKEEPER_QUORUM_KEY, zooKeeper.getConnectString());
+			config.setString(HighAvailabilityOptions.HA_MODE, "ZOOKEEPER");
+			config.setString(HighAvailabilityOptions.HA_ZOOKEEPER_QUORUM, zooKeeper.getConnectString());
 
 			actorSystem = AkkaUtils.createDefaultActorSystem();
 

http://git-wip-us.apache.org/repos/asf/flink/blob/c8dc074a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/FileSystemBlobStore.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/FileSystemBlobStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/FileSystemBlobStore.java
index ee189d4..deba738 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/FileSystemBlobStore.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/FileSystemBlobStore.java
@@ -19,14 +19,17 @@
 package org.apache.flink.runtime.blob;
 
 import com.google.common.io.Files;
+
+import org.apache.commons.lang3.StringUtils;
+
 import org.apache.flink.api.common.JobID;
-import org.apache.flink.configuration.ConfigConstants;
 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.util.ConfigurationUtil;
 import org.apache.flink.util.IOUtils;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -52,16 +55,11 @@ class FileSystemBlobStore implements BlobStore {
 	private final String basePath;
 
 	FileSystemBlobStore(Configuration config) throws IOException {
-		String storagePath = ConfigurationUtil.getStringWithDeprecatedKeys(
-				config,
-				ConfigConstants.HA_ZOOKEEPER_STORAGE_PATH,
-				null,
-				ConfigConstants.ZOOKEEPER_RECOVERY_PATH);
-
-		if (storagePath == null) {
-			throw new IllegalConfigurationException(String.format("Missing configuration for " +
-					"ZooKeeper file system path. Please specify via " +
-					"'%s' key.", ConfigConstants.HA_ZOOKEEPER_STORAGE_PATH));
+		String storagePath = config.getValue(HighAvailabilityOptions.HA_STORAGE_PATH);
+
+		if (storagePath == null || StringUtils.isBlank(storagePath)) {
+			throw new IllegalConfigurationException("Missing high-availability storage path for metadata." +
+					" Specify via configuration key '" + HighAvailabilityOptions.HA_STORAGE_PATH + "'.");
 		}
 
 		this.basePath = storagePath + "/blob";

http://git-wip-us.apache.org/repos/asf/flink/blob/c8dc074a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/HighAvailabilityMode.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/HighAvailabilityMode.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/HighAvailabilityMode.java
index 087ad3b..fa2db48 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/HighAvailabilityMode.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/HighAvailabilityMode.java
@@ -20,7 +20,7 @@ package org.apache.flink.runtime.jobmanager;
 
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.util.ConfigurationUtil;
+import org.apache.flink.configuration.HighAvailabilityOptions;
 
 /**
  * High availability mode for Flink's cluster execution. Currently supported modes are:
@@ -43,11 +43,7 @@ public enum HighAvailabilityMode {
 	 * configured.
 	 */
 	public static HighAvailabilityMode fromConfig(Configuration config) {
-		String haMode = ConfigurationUtil.getStringWithDeprecatedKeys(
-				config,
-				ConfigConstants.HA_MODE,
-				null,
-				ConfigConstants.RECOVERY_MODE);
+		String haMode = config.getValue(HighAvailabilityOptions.HA_MODE);
 
 		if (haMode == null) {
 			return HighAvailabilityMode.NONE;

http://git-wip-us.apache.org/repos/asf/flink/blob/c8dc074a/flink-runtime/src/main/java/org/apache/flink/runtime/security/SecurityContext.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/security/SecurityContext.java b/flink-runtime/src/main/java/org/apache/flink/runtime/security/SecurityContext.java
index be6611f..67dd78c 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/security/SecurityContext.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/security/SecurityContext.java
@@ -23,6 +23,7 @@ import org.apache.flink.annotation.Internal;
 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.util.Preconditions;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.security.Credentials;
@@ -182,9 +183,9 @@ public class SecurityContext {
 		//with pseudo JAAS configuration file if SASL auth is enabled for ZK
 		System.setProperty(JAVA_SECURITY_AUTH_LOGIN_CONFIG, "");
 
-		boolean disableSaslClient = configuration.getBoolean(ConfigConstants.ZOOKEEPER_SASL_DISABLE,
-				ConfigConstants.DEFAULT_ZOOKEEPER_SASL_DISABLE);
-		if(disableSaslClient) {
+		boolean disableSaslClient = configuration.getBoolean(HighAvailabilityOptions.ZOOKEEPER_SASL_DISABLE);
+
+		if (disableSaslClient) {
 			LOG.info("SASL client auth for ZK will be disabled");
 			//SASL auth is disabled by default but will be enabled if specified in configuration
 			System.setProperty(ZOOKEEPER_SASL_CLIENT,"false");
@@ -212,8 +213,8 @@ public class SecurityContext {
 		System.setProperty(JAVA_SECURITY_AUTH_LOGIN_CONFIG, jaasConfFile.getAbsolutePath());
 		System.setProperty(ZOOKEEPER_SASL_CLIENT, "true");
 
-		String zkSaslServiceName = configuration.getString(ConfigConstants.ZOOKEEPER_SASL_SERVICE_NAME, null);
-		if(!StringUtils.isBlank(zkSaslServiceName)) {
+		String zkSaslServiceName = configuration.getValue(HighAvailabilityOptions.ZOOKEEPER_SASL_SERVICE_NAME);
+		if (!StringUtils.isBlank(zkSaslServiceName)) {
 			LOG.info("ZK SASL service name: {} is provided in the configuration", zkSaslServiceName);
 			System.setProperty(ZOOKEEPER_SASL_CLIENT_USERNAME,zkSaslServiceName);
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/c8dc074a/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 5e69875..137a85b 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
@@ -18,12 +18,14 @@
 
 package org.apache.flink.runtime.util;
 
+import org.apache.commons.lang3.StringUtils;
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.framework.CuratorFrameworkFactory;
 import org.apache.curator.retry.ExponentialBackoffRetry;
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
 import org.apache.flink.configuration.IllegalConfigurationException;
 import org.apache.flink.runtime.checkpoint.CompletedCheckpoint;
 import org.apache.flink.runtime.checkpoint.CompletedCheckpointStore;
@@ -57,53 +59,25 @@ public class ZooKeeperUtils {
 	 * @return {@link CuratorFramework} instance
 	 */
 	public static CuratorFramework startCuratorFramework(Configuration configuration) {
-		String zkQuorum = ConfigurationUtil.getStringWithDeprecatedKeys(
-				configuration,
-				ConfigConstants.HA_ZOOKEEPER_QUORUM_KEY,
-				null,
-				ConfigConstants.ZOOKEEPER_QUORUM_KEY);
+		String zkQuorum = configuration.getValue(HighAvailabilityOptions.HA_ZOOKEEPER_QUORUM);
 
-		if (zkQuorum == null || zkQuorum.equals("")) {
+		if (zkQuorum == null || StringUtils.isBlank(zkQuorum)) {
 			throw new RuntimeException("No valid ZooKeeper quorum has been specified. " +
 					"You can specify the quorum via the configuration key '" +
-					ConfigConstants.HA_ZOOKEEPER_QUORUM_KEY + "'.");
+					HighAvailabilityOptions.HA_ZOOKEEPER_QUORUM.key() + "'.");
 		}
 
-		int sessionTimeout = ConfigurationUtil.getIntegerWithDeprecatedKeys(
-				configuration,
-				ConfigConstants.HA_ZOOKEEPER_SESSION_TIMEOUT,
-				ConfigConstants.DEFAULT_ZOOKEEPER_SESSION_TIMEOUT,
-				ConfigConstants.ZOOKEEPER_SESSION_TIMEOUT);
+		int sessionTimeout = configuration.getInteger(HighAvailabilityOptions.ZOOKEEPER_SESSION_TIMEOUT);
 
-		int connectionTimeout = ConfigurationUtil.getIntegerWithDeprecatedKeys(
-				configuration,
-				ConfigConstants.HA_ZOOKEEPER_CONNECTION_TIMEOUT,
-				ConfigConstants.DEFAULT_ZOOKEEPER_CONNECTION_TIMEOUT,
-				ConfigConstants.ZOOKEEPER_CONNECTION_TIMEOUT);
+		int connectionTimeout = configuration.getInteger(HighAvailabilityOptions.ZOOKEEPER_CONNECTION_TIMEOUT);
 
-		int retryWait = ConfigurationUtil.getIntegerWithDeprecatedKeys(
-				configuration,
-				ConfigConstants.HA_ZOOKEEPER_RETRY_WAIT,
-				ConfigConstants.DEFAULT_ZOOKEEPER_RETRY_WAIT,
-				ConfigConstants.ZOOKEEPER_RETRY_WAIT);
+		int retryWait = configuration.getInteger(HighAvailabilityOptions.ZOOKEEPER_RETRY_WAIT);
 
-		int maxRetryAttempts = ConfigurationUtil.getIntegerWithDeprecatedKeys(
-				configuration,
-				ConfigConstants.HA_ZOOKEEPER_MAX_RETRY_ATTEMPTS,
-				ConfigConstants.DEFAULT_ZOOKEEPER_MAX_RETRY_ATTEMPTS,
-				ConfigConstants.ZOOKEEPER_MAX_RETRY_ATTEMPTS);
+		int maxRetryAttempts = configuration.getInteger(HighAvailabilityOptions.ZOOKEEPER_MAX_RETRY_ATTEMPTS);
 
-		String root = ConfigurationUtil.getStringWithDeprecatedKeys(
-				configuration,
-				ConfigConstants.HA_ZOOKEEPER_DIR_KEY,
-				ConfigConstants.DEFAULT_ZOOKEEPER_DIR_KEY,
-				ConfigConstants.ZOOKEEPER_DIR_KEY);
+		String root = configuration.getValue(HighAvailabilityOptions.HA_ZOOKEEPER_ROOT);
 
-		String namespace = ConfigurationUtil.getStringWithDeprecatedKeys(
-				configuration,
-				ConfigConstants.HA_ZOOKEEPER_NAMESPACE_KEY,
-				ConfigConstants.DEFAULT_ZOOKEEPER_NAMESPACE_KEY,
-				ConfigConstants.ZOOKEEPER_NAMESPACE_KEY);
+		String namespace = configuration.getValue(HighAvailabilityOptions.HA_CLUSTER_ID);
 
 		String rootWithNamespace = generateZookeeperPath(root, namespace);
 
@@ -138,13 +112,9 @@ public class ZooKeeperUtils {
 	public static String getZooKeeperEnsemble(Configuration flinkConf)
 			throws IllegalConfigurationException {
 
-		String zkQuorum = ConfigurationUtil.getStringWithDeprecatedKeys(
-				flinkConf,
-				ConfigConstants.HA_ZOOKEEPER_QUORUM_KEY,
-				"",
-				ConfigConstants.ZOOKEEPER_QUORUM_KEY);
+		String zkQuorum = flinkConf.getValue(HighAvailabilityOptions.HA_ZOOKEEPER_QUORUM);
 
-		if (zkQuorum == null || zkQuorum.equals("")) {
+		if (zkQuorum == null || StringUtils.isBlank(zkQuorum)) {
 			throw new IllegalConfigurationException("No ZooKeeper quorum specified in config.");
 		}
 
@@ -367,15 +337,11 @@ public class ZooKeeperUtils {
 			Configuration configuration,
 			String prefix) throws IOException {
 
-		String rootPath = ConfigurationUtil.getStringWithDeprecatedKeys(
-				configuration,
-				ConfigConstants.HA_ZOOKEEPER_STORAGE_PATH,
-				"",
-				ConfigConstants.ZOOKEEPER_RECOVERY_PATH);
+		String rootPath = configuration.getValue(HighAvailabilityOptions.HA_STORAGE_PATH);
 
-		if (rootPath.equals("")) {
-			throw new IllegalConfigurationException("Missing recovery path. Specify via " +
-				"configuration key '" + ConfigConstants.HA_ZOOKEEPER_STORAGE_PATH + "'.");
+		if (rootPath == null || StringUtils.isBlank(rootPath)) {
+			throw new IllegalConfigurationException("Missing high-availability storage path for metadata." +
+					" Specify via configuration key '" + HighAvailabilityOptions.HA_STORAGE_PATH + "'.");
 		} else {
 			return new FileSystemStateStorageHelper<T>(rootPath, prefix);
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/c8dc074a/flink-runtime/src/main/java/org/apache/flink/runtime/zookeeper/FlinkZooKeeperQuorumPeer.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/zookeeper/FlinkZooKeeperQuorumPeer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/zookeeper/FlinkZooKeeperQuorumPeer.java
index 9fba529..c4140c0 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/zookeeper/FlinkZooKeeperQuorumPeer.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/zookeeper/FlinkZooKeeperQuorumPeer.java
@@ -19,9 +19,9 @@
 package org.apache.flink.runtime.zookeeper;
 
 import org.apache.flink.api.java.utils.ParameterTool;
-import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.IllegalConfigurationException;
 import org.apache.flink.runtime.util.EnvironmentInformation;
+
 import org.apache.zookeeper.server.ServerConfig;
 import org.apache.zookeeper.server.ZooKeeperServer;
 import org.apache.zookeeper.server.ZooKeeperServerMain;
@@ -47,8 +47,25 @@ import java.util.UUID;
  */
 public class FlinkZooKeeperQuorumPeer {
 
+	/** ZooKeeper default client port. */
+	public static final int DEFAULT_ZOOKEEPER_CLIENT_PORT = 2181;
+
+	/** ZooKeeper default init limit. */
+	public static final int DEFAULT_ZOOKEEPER_INIT_LIMIT = 10;
+
+	/** ZooKeeper default sync limit. */
+	public static final int DEFAULT_ZOOKEEPER_SYNC_LIMIT = 5;
+
+	/** ZooKeeper default peer port. */
+	public static final int DEFAULT_ZOOKEEPER_PEER_PORT = 2888;
+
+	/** ZooKeeper default leader port. */
+	public static final int DEFAULT_ZOOKEEPER_LEADER_PORT = 3888;
+
 	private static final Logger LOG = LoggerFactory.getLogger(FlinkZooKeeperQuorumPeer.class);
 
+	// ------------------------------------------------------------------------
+
 	public static void main(String[] args) {
 		try {
 			// startup checks and logging
@@ -67,6 +84,8 @@ public class FlinkZooKeeperQuorumPeer {
 		}
 	}
 
+	// ------------------------------------------------------------------------
+
 	/**
 	 * Runs a ZooKeeper {@link QuorumPeer} if further peers are configured or a single
 	 * {@link ZooKeeperServer} if no further peers are configured.
@@ -120,26 +139,23 @@ public class FlinkZooKeeperQuorumPeer {
 	private static void setRequiredProperties(Properties zkProps) {
 		// Set default client port
 		if (zkProps.getProperty("clientPort") == null) {
-			int clientPort = ConfigConstants.DEFAULT_ZOOKEEPER_CLIENT_PORT;
-			zkProps.setProperty("clientPort", String.valueOf(clientPort));
+			zkProps.setProperty("clientPort", String.valueOf(DEFAULT_ZOOKEEPER_CLIENT_PORT));
 
-			LOG.warn("No 'clientPort' configured. Set to '{}'.", clientPort);
+			LOG.warn("No 'clientPort' configured. Set to '{}'.", DEFAULT_ZOOKEEPER_CLIENT_PORT);
 		}
 
 		// Set default init limit
 		if (zkProps.getProperty("initLimit") == null) {
-			int initLimit = ConfigConstants.DEFAULT_ZOOKEEPER_INIT_LIMIT;
-			zkProps.setProperty("initLimit", String.valueOf(initLimit));
+			zkProps.setProperty("initLimit", String.valueOf(DEFAULT_ZOOKEEPER_INIT_LIMIT));
 
-			LOG.warn("No 'initLimit' configured. Set to '{}'.", initLimit);
+			LOG.warn("No 'initLimit' configured. Set to '{}'.", DEFAULT_ZOOKEEPER_INIT_LIMIT);
 		}
 
 		// Set default sync limit
 		if (zkProps.getProperty("syncLimit") == null) {
-			int syncLimit = ConfigConstants.DEFAULT_ZOOKEEPER_SYNC_LIMIT;
-			zkProps.setProperty("syncLimit", String.valueOf(syncLimit));
+			zkProps.setProperty("syncLimit", String.valueOf(DEFAULT_ZOOKEEPER_SYNC_LIMIT));
 
-			LOG.warn("No 'syncLimit' configured. Set to '{}'.", syncLimit);
+			LOG.warn("No 'syncLimit' configured. Set to '{}'.", DEFAULT_ZOOKEEPER_SYNC_LIMIT);
 		}
 
 		// Set default data dir
@@ -152,8 +168,8 @@ public class FlinkZooKeeperQuorumPeer {
 			LOG.warn("No 'dataDir' configured. Set to '{}'.", dataDir);
 		}
 
-		int peerPort = ConfigConstants.DEFAULT_ZOOKEEPER_PEER_PORT;
-		int leaderPort = ConfigConstants.DEFAULT_ZOOKEEPER_LEADER_PORT;
+		int peerPort = DEFAULT_ZOOKEEPER_PEER_PORT;
+		int leaderPort = DEFAULT_ZOOKEEPER_LEADER_PORT;
 
 		// Set peer and leader ports if none given, because ZooKeeper complains if multiple
 		// servers are configured, but no ports are given.
@@ -220,12 +236,8 @@ public class FlinkZooKeeperQuorumPeer {
 		
 		// Write myid to file. We use a File Writer, because that properly propagates errors,
 		// while the PrintWriter swallows errors
-		FileWriter writer = new FileWriter(new File(dataDir, "myid"));
-		try {
+		try (FileWriter writer = new FileWriter(new File(dataDir, "myid"))) {
 			writer.write(String.valueOf(id));
 		}
-		finally {
-			writer.close();
-		}
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/c8dc074a/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 e90f2d2..be820ae 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
@@ -31,7 +31,7 @@ import akka.pattern.ask
 import grizzled.slf4j.Logger
 import org.apache.flink.api.common.JobID
 import org.apache.flink.api.common.time.Time
-import org.apache.flink.configuration.{ConfigConstants, Configuration, GlobalConfiguration}
+import org.apache.flink.configuration.{ConfigConstants, Configuration, GlobalConfiguration, HighAvailabilityOptions}
 import org.apache.flink.core.fs.FileSystem
 import org.apache.flink.core.io.InputSplitAssigner
 import org.apache.flink.metrics.{Gauge, MetricGroup}
@@ -2367,9 +2367,7 @@ object JobManager {
         configuration.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, 0)
 
         // The port range of allowed job manager ports or 0 for random
-        configuration.getString(
-          ConfigConstants.RECOVERY_JOB_MANAGER_PORT,
-          ConfigConstants.DEFAULT_HA_JOB_MANAGER_PORT)
+        configuration.getValue(HighAvailabilityOptions.HA_JOB_MANAGER_PORT_RANGE)
       }
       else {
         LOG.info("Starting JobManager without high-availability")
@@ -2501,11 +2499,7 @@ object JobManager {
 
     val savepointStore = SavepointStoreFactory.createFromConfig(configuration)
 
-    val jobRecoveryTimeoutStr = ConfigurationUtil.getStringWithDeprecatedKeys(
-      configuration,
-      ConfigConstants.HA_JOB_DELAY,
-      null,
-      ConfigConstants.RECOVERY_JOB_DELAY)
+    val jobRecoveryTimeoutStr = configuration.getValue(HighAvailabilityOptions.HA_JOB_DELAY)
 
     val jobRecoveryTimeout = if (jobRecoveryTimeoutStr == null || jobRecoveryTimeoutStr.isEmpty) {
       timeout
@@ -2515,7 +2509,7 @@ object JobManager {
       } catch {
         case n: NumberFormatException =>
           throw new Exception(
-            s"Invalid config value for ${ConfigConstants.HA_JOB_DELAY}: " +
+            s"Invalid config value for ${HighAvailabilityOptions.HA_JOB_DELAY.key()}: " +
               s"$jobRecoveryTimeoutStr. Value must be a valid duration (such as '10 s' or '1 min')")
       }
     }

http://git-wip-us.apache.org/repos/asf/flink/blob/c8dc074a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobRecoveryITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobRecoveryITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobRecoveryITCase.java
index 8464d68..8ba20c9 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobRecoveryITCase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobRecoveryITCase.java
@@ -22,6 +22,7 @@ import org.apache.commons.io.FileUtils;
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
 import org.apache.flink.runtime.jobmanager.HighAvailabilityMode;
 import org.junit.After;
 import org.junit.Before;
@@ -68,9 +69,9 @@ public class BlobRecoveryITCase {
 
 		try {
 			Configuration config = new Configuration();
-			config.setString(ConfigConstants.HA_MODE, "ZOOKEEPER");
+			config.setString(HighAvailabilityOptions.HA_MODE, "ZOOKEEPER");
 			config.setString(ConfigConstants.STATE_BACKEND, "FILESYSTEM");
-			config.setString(ConfigConstants.HA_ZOOKEEPER_STORAGE_PATH, recoveryDir.getPath());
+			config.setString(HighAvailabilityOptions.HA_STORAGE_PATH, recoveryDir.getPath());
 
 			for (int i = 0; i < server.length; i++) {
 				server[i] = new BlobServer(config);

http://git-wip-us.apache.org/repos/asf/flink/blob/c8dc074a/flink-runtime/src/test/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheRecoveryITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheRecoveryITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheRecoveryITCase.java
index f6bed56..f6cdf09 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheRecoveryITCase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheRecoveryITCase.java
@@ -21,6 +21,7 @@ package org.apache.flink.runtime.execution.librarycache;
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
 import org.apache.flink.runtime.blob.BlobCache;
 import org.apache.flink.runtime.blob.BlobClient;
 import org.apache.flink.runtime.blob.BlobKey;
@@ -63,9 +64,9 @@ public class BlobLibraryCacheRecoveryITCase {
 
 		try {
 			Configuration config = new Configuration();
-			config.setString(ConfigConstants.HA_MODE, "ZOOKEEPER");
+			config.setString(HighAvailabilityOptions.HA_MODE, "ZOOKEEPER");
 			config.setString(ConfigConstants.STATE_BACKEND, "FILESYSTEM");
-			config.setString(ConfigConstants.HA_ZOOKEEPER_STORAGE_PATH, temporaryFolder.getRoot().getAbsolutePath());
+			config.setString(HighAvailabilityOptions.HA_STORAGE_PATH, temporaryFolder.getRoot().getAbsolutePath());
 
 			for (int i = 0; i < server.length; i++) {
 				server[i] = new BlobServer(config);

http://git-wip-us.apache.org/repos/asf/flink/blob/c8dc074a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/HighAvailabilityModeTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/HighAvailabilityModeTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/HighAvailabilityModeTest.java
index 04c0e48..91fb514 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/HighAvailabilityModeTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/HighAvailabilityModeTest.java
@@ -20,7 +20,8 @@ package org.apache.flink.runtime.jobmanager;
 
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.jobmanager.HighAvailabilityMode;
+import org.apache.flink.configuration.HighAvailabilityOptions;
+
 import org.junit.Test;
 
 import static org.junit.Assert.assertEquals;
@@ -42,7 +43,7 @@ public class HighAvailabilityModeTest {
 		assertEquals(DEFAULT_HA_MODE, HighAvailabilityMode.fromConfig(config));
 
 		// Check not equals default
-		config.setString(ConfigConstants.HA_MODE, HighAvailabilityMode.ZOOKEEPER.name().toLowerCase());
+		config.setString(HighAvailabilityOptions.HA_MODE, HighAvailabilityMode.ZOOKEEPER.name().toLowerCase());
 		assertEquals(HighAvailabilityMode.ZOOKEEPER, HighAvailabilityMode.fromConfig(config));
 	}
 
@@ -54,16 +55,16 @@ public class HighAvailabilityModeTest {
 		Configuration config = new Configuration();
 
 		// Check mapping of old default to new default
-		config.setString(ConfigConstants.RECOVERY_MODE, ConfigConstants.DEFAULT_RECOVERY_MODE);
+		config.setString("recovery.mode", ConfigConstants.DEFAULT_RECOVERY_MODE);
 		assertEquals(DEFAULT_HA_MODE, HighAvailabilityMode.fromConfig(config));
 
 		// Check deprecated config
-		config.setString(ConfigConstants.RECOVERY_MODE, HighAvailabilityMode.ZOOKEEPER.name().toLowerCase());
+		config.setString("recovery.mode", HighAvailabilityMode.ZOOKEEPER.name().toLowerCase());
 		assertEquals(HighAvailabilityMode.ZOOKEEPER, HighAvailabilityMode.fromConfig(config));
 
 		// Check precedence over deprecated config
-		config.setString(ConfigConstants.HA_MODE, HighAvailabilityMode.NONE.name().toLowerCase());
-		config.setString(ConfigConstants.RECOVERY_MODE, HighAvailabilityMode.ZOOKEEPER.name().toLowerCase());
+		config.setString("high-availability", HighAvailabilityMode.NONE.name().toLowerCase());
+		config.setString("recovery.mode", HighAvailabilityMode.ZOOKEEPER.name().toLowerCase());
 
 		assertEquals(HighAvailabilityMode.NONE, HighAvailabilityMode.fromConfig(config));
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/c8dc074a/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 360588d..5b12eee 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
@@ -26,6 +26,7 @@ import akka.testkit.JavaTestKit;
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
 import org.apache.flink.core.fs.FSDataInputStream;
 import org.apache.flink.runtime.akka.AkkaUtils;
 import org.apache.flink.runtime.akka.ListeningBehaviour;
@@ -131,8 +132,8 @@ public class JobManagerHARecoveryTest {
 		ActorRef jobManager = null;
 		ActorRef taskManager = null;
 
-		flinkConfiguration.setString(ConfigConstants.HA_MODE, "zookeeper");
-		flinkConfiguration.setString(ConfigConstants.HA_ZOOKEEPER_STORAGE_PATH, temporaryFolder.newFolder().toString());
+		flinkConfiguration.setString(HighAvailabilityOptions.HA_MODE, "zookeeper");
+		flinkConfiguration.setString(HighAvailabilityOptions.HA_STORAGE_PATH, temporaryFolder.newFolder().toString());
 		flinkConfiguration.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, slots);
 
 		try {

http://git-wip-us.apache.org/repos/asf/flink/blob/c8dc074a/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 e20985b..1f1eb62 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
@@ -27,6 +27,7 @@ import org.apache.curator.framework.recipes.cache.NodeCacheListener;
 import org.apache.curator.test.TestingServer;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
 import org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalService;
 import org.apache.flink.runtime.util.ZooKeeperUtils;
 import org.apache.flink.util.TestLogger;
@@ -89,8 +90,8 @@ public class ZooKeeperLeaderElectionTest extends TestLogger {
 	@Test
 	public void testZooKeeperLeaderElectionRetrieval() throws Exception {
 		Configuration configuration = new Configuration();
-		configuration.setString(ConfigConstants.HA_ZOOKEEPER_QUORUM_KEY, testingServer.getConnectString());
-		configuration.setString(ConfigConstants.HA_MODE, "zookeeper");
+		configuration.setString(HighAvailabilityOptions.HA_ZOOKEEPER_QUORUM, testingServer.getConnectString());
+		configuration.setString(HighAvailabilityOptions.HA_MODE, "zookeeper");
 
 		ZooKeeperLeaderElectionService leaderElectionService = null;
 		ZooKeeperLeaderRetrievalService leaderRetrievalService = null;
@@ -134,8 +135,8 @@ public class ZooKeeperLeaderElectionTest extends TestLogger {
 	@Test
 	public void testZooKeeperReelection() throws Exception {
 		Configuration configuration = new Configuration();
-		configuration.setString(ConfigConstants.HA_ZOOKEEPER_QUORUM_KEY, testingServer.getConnectString());
-		configuration.setString(ConfigConstants.HA_MODE, "zookeeper");
+		configuration.setString(HighAvailabilityOptions.HA_ZOOKEEPER_QUORUM, testingServer.getConnectString());
+		configuration.setString(HighAvailabilityOptions.HA_MODE, "zookeeper");
 
 		Deadline deadline = new FiniteDuration(5, TimeUnit.MINUTES).fromNow();
 
@@ -217,8 +218,8 @@ public class ZooKeeperLeaderElectionTest extends TestLogger {
 	@Test
 	public void testZooKeeperReelectionWithReplacement() throws Exception {
 		Configuration configuration = new Configuration();
-		configuration.setString(ConfigConstants.HA_ZOOKEEPER_QUORUM_KEY, testingServer.getConnectString());
-		configuration.setString(ConfigConstants.HA_MODE, "zookeeper");
+		configuration.setString(HighAvailabilityOptions.HA_ZOOKEEPER_QUORUM, testingServer.getConnectString());
+		configuration.setString(HighAvailabilityOptions.HA_MODE, "zookeeper");
 
 		int num = 3;
 		int numTries = 30;
@@ -295,8 +296,8 @@ public class ZooKeeperLeaderElectionTest extends TestLogger {
 		final String leaderPath = "/leader";
 
 		Configuration configuration = new Configuration();
-		configuration.setString(ConfigConstants.HA_ZOOKEEPER_QUORUM_KEY, testingServer.getConnectString());
-		configuration.setString(ConfigConstants.HA_MODE, "zookeeper");
+		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;
@@ -379,8 +380,8 @@ public class ZooKeeperLeaderElectionTest extends TestLogger {
 	@Test
 	public void testExceptionForwarding() throws Exception {
 		Configuration configuration = new Configuration();
-		configuration.setString(ConfigConstants.HA_ZOOKEEPER_QUORUM_KEY, testingServer.getConnectString());
-		configuration.setString(ConfigConstants.HA_MODE, "zookeeper");
+		configuration.setString(HighAvailabilityOptions.HA_ZOOKEEPER_QUORUM, testingServer.getConnectString());
+		configuration.setString(HighAvailabilityOptions.HA_MODE, "zookeeper");
 
 		ZooKeeperLeaderElectionService leaderElectionService = null;
 		ZooKeeperLeaderRetrievalService leaderRetrievalService = null;
@@ -448,8 +449,8 @@ public class ZooKeeperLeaderElectionTest extends TestLogger {
 	@Test
 	public void testEphemeralZooKeeperNodes() throws Exception {
 		Configuration configuration = new Configuration();
-		configuration.setString(ConfigConstants.HA_ZOOKEEPER_QUORUM_KEY, testingServer.getConnectString());
-		configuration.setString(ConfigConstants.HA_MODE, "zookeeper");
+		configuration.setString(HighAvailabilityOptions.HA_ZOOKEEPER_QUORUM, testingServer.getConnectString());
+		configuration.setString(HighAvailabilityOptions.HA_MODE, "zookeeper");
 
 		ZooKeeperLeaderElectionService leaderElectionService;
 		ZooKeeperLeaderRetrievalService leaderRetrievalService = null;

http://git-wip-us.apache.org/repos/asf/flink/blob/c8dc074a/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 0fe0644..70b1da0 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
@@ -20,16 +20,19 @@ package org.apache.flink.runtime.leaderelection;
 
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.test.TestingServer;
-import org.apache.flink.configuration.ConfigConstants;
+
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
 import org.apache.flink.runtime.jobmanager.JobManager;
 import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
 import org.apache.flink.runtime.util.LeaderRetrievalUtils;
 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 scala.Option;
 import scala.concurrent.duration.FiniteDuration;
 
@@ -82,8 +85,8 @@ public class ZooKeeperLeaderRetrievalTest extends TestLogger{
 
 		long sleepingTime = 1000;
 
-		config.setString(ConfigConstants.HA_MODE, "zookeeper");
-		config.setString(ConfigConstants.HA_ZOOKEEPER_QUORUM_KEY, testingServer.getConnectString());
+		config.setString(HighAvailabilityOptions.HA_MODE, "zookeeper");
+		config.setString(HighAvailabilityOptions.HA_ZOOKEEPER_QUORUM, testingServer.getConnectString());
 
 		LeaderElectionService leaderElectionService = null;
 		LeaderElectionService faultyLeaderElectionService;
@@ -179,8 +182,8 @@ public class ZooKeeperLeaderRetrievalTest extends TestLogger{
 	@Test
 	public void testTimeoutOfFindConnectingAddress() throws Exception {
 		Configuration config = new Configuration();
-		config.setString(ConfigConstants.HA_MODE, "zookeeper");
-		config.setString(ConfigConstants.HA_ZOOKEEPER_QUORUM_KEY, testingServer.getConnectString());
+		config.setString(HighAvailabilityOptions.HA_MODE, "zookeeper");
+		config.setString(HighAvailabilityOptions.HA_ZOOKEEPER_QUORUM, testingServer.getConnectString());
 
 		FiniteDuration timeout = new FiniteDuration(10, TimeUnit.SECONDS);
 
@@ -190,7 +193,7 @@ public class ZooKeeperLeaderRetrievalTest extends TestLogger{
 		assertEquals(InetAddress.getLocalHost(), result);
 	}
 
-	class FindConnectingAddress implements Runnable {
+	static class FindConnectingAddress implements Runnable {
 
 		private final Configuration config;
 		private final FiniteDuration timeout;

http://git-wip-us.apache.org/repos/asf/flink/blob/c8dc074a/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/ZooKeeperTestUtils.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/ZooKeeperTestUtils.java b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/ZooKeeperTestUtils.java
index 7dd7067..07cec32 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/ZooKeeperTestUtils.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/ZooKeeperTestUtils.java
@@ -20,6 +20,7 @@ package org.apache.flink.runtime.testutils;
 
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
 import org.apache.flink.runtime.jobmanager.HighAvailabilityMode;
 import org.apache.flink.runtime.state.filesystem.FsStateBackendFactory;
 
@@ -66,8 +67,8 @@ public class ZooKeeperTestUtils {
 		config.setInteger(ConfigConstants.JOB_MANAGER_WEB_PORT_KEY, -1);
 
 		// ZooKeeper recovery mode
-		config.setString(ConfigConstants.HA_MODE, "ZOOKEEPER");
-		config.setString(ConfigConstants.HA_ZOOKEEPER_QUORUM_KEY, zooKeeperQuorum);
+		config.setString(HighAvailabilityOptions.HA_MODE, "ZOOKEEPER");
+		config.setString(HighAvailabilityOptions.HA_ZOOKEEPER_QUORUM, zooKeeperQuorum);
 
 		int connTimeout = 5000;
 		if (System.getenv().containsKey("CI")) {
@@ -75,20 +76,20 @@ public class ZooKeeperTestUtils {
 			connTimeout = 30000;
 		}
 
-		config.setInteger(ConfigConstants.HA_ZOOKEEPER_CONNECTION_TIMEOUT, connTimeout);
-		config.setInteger(ConfigConstants.HA_ZOOKEEPER_SESSION_TIMEOUT, connTimeout);
+		config.setInteger(HighAvailabilityOptions.ZOOKEEPER_CONNECTION_TIMEOUT, connTimeout);
+		config.setInteger(HighAvailabilityOptions.ZOOKEEPER_SESSION_TIMEOUT, connTimeout);
 
 		// File system state backend
 		config.setString(ConfigConstants.STATE_BACKEND, "FILESYSTEM");
 		config.setString(FsStateBackendFactory.CHECKPOINT_DIRECTORY_URI_CONF_KEY, fsStateHandlePath + "/checkpoints");
-		config.setString(ConfigConstants.HA_ZOOKEEPER_STORAGE_PATH, fsStateHandlePath + "/recovery");
+		config.setString(HighAvailabilityOptions.HA_STORAGE_PATH, fsStateHandlePath + "/recovery");
 
 		// Akka failure detection and execution retries
 		config.setString(ConfigConstants.AKKA_WATCH_HEARTBEAT_INTERVAL, "1000 ms");
 		config.setString(ConfigConstants.AKKA_WATCH_HEARTBEAT_PAUSE, "6 s");
 		config.setInteger(ConfigConstants.AKKA_WATCH_THRESHOLD, 9);
 		config.setString(ConfigConstants.AKKA_ASK_TIMEOUT, "100 s");
-		config.setString(ConfigConstants.HA_JOB_DELAY, "10 s");
+		config.setString(HighAvailabilityOptions.HA_JOB_DELAY, "10 s");
 
 		return config;
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/c8dc074a/flink-runtime/src/test/java/org/apache/flink/runtime/util/ZooKeeperUtilTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/util/ZooKeeperUtilTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/ZooKeeperUtilTest.java
index daed4a4..d5895ff 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/util/ZooKeeperUtilTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/ZooKeeperUtilTest.java
@@ -20,6 +20,7 @@ package org.apache.flink.runtime.util;
 
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
 import org.apache.flink.util.TestLogger;
 import org.junit.Test;
 
@@ -71,7 +72,7 @@ public class ZooKeeperUtilTest extends TestLogger {
 	}
 
 	private Configuration setQuorum(Configuration conf, String quorum) {
-		conf.setString(ConfigConstants.HA_ZOOKEEPER_QUORUM_KEY, quorum);
+		conf.setString(HighAvailabilityOptions.HA_ZOOKEEPER_QUORUM, quorum);
 		return conf;
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/c8dc074a/flink-runtime/src/test/java/org/apache/flink/runtime/zookeeper/ZooKeeperTestEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/zookeeper/ZooKeeperTestEnvironment.java b/flink-runtime/src/test/java/org/apache/flink/runtime/zookeeper/ZooKeeperTestEnvironment.java
index bd58515..66c4fac 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/zookeeper/ZooKeeperTestEnvironment.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/zookeeper/ZooKeeperTestEnvironment.java
@@ -22,9 +22,11 @@ import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.test.TestingCluster;
 import org.apache.curator.test.TestingServer;
 import org.apache.curator.utils.ZKPaths;
-import org.apache.flink.configuration.ConfigConstants;
+
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
 import org.apache.flink.runtime.util.ZooKeeperUtils;
+
 import org.apache.zookeeper.KeeperException;
 
 import java.util.List;
@@ -58,7 +60,7 @@ public class ZooKeeperTestEnvironment {
 				zooKeeperServer = new TestingServer(true);
 				zooKeeperCluster = null;
 
-				conf.setString(ConfigConstants.HA_ZOOKEEPER_QUORUM_KEY,
+				conf.setString(HighAvailabilityOptions.HA_ZOOKEEPER_QUORUM,
 						zooKeeperServer.getConnectString());
 			}
 			else {
@@ -67,7 +69,7 @@ public class ZooKeeperTestEnvironment {
 
 				zooKeeperCluster.start();
 
-				conf.setString(ConfigConstants.HA_ZOOKEEPER_QUORUM_KEY,
+				conf.setString(HighAvailabilityOptions.HA_ZOOKEEPER_QUORUM,
 						zooKeeperCluster.getConnectString());
 			}
 
@@ -127,7 +129,7 @@ public class ZooKeeperTestEnvironment {
 	 */
 	public CuratorFramework createClient() {
 		Configuration config = new Configuration();
-		config.setString(ConfigConstants.HA_ZOOKEEPER_QUORUM_KEY, getConnectString());
+		config.setString(HighAvailabilityOptions.HA_ZOOKEEPER_QUORUM, getConnectString());
 		return ZooKeeperUtils.startCuratorFramework(config);
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/c8dc074a/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 e878097..97016e4 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
@@ -28,12 +28,12 @@ import com.typesafe.config.ConfigFactory
 import grizzled.slf4j.Logger
 import org.apache.flink.api.common.JobExecutionResult
 
-import org.apache.flink.configuration.{ConfigConstants, Configuration}
+import org.apache.flink.configuration.{HighAvailabilityOptions, ConfigConstants, Configuration}
 import org.apache.flink.runtime.client.JobClient
 import org.apache.flink.runtime.clusterframework.FlinkResourceManager
 import org.apache.flink.runtime.jobgraph.JobGraph
 import org.apache.flink.runtime.clusterframework.types.ResourceID
-import org.apache.flink.runtime.jobmanager.{MemoryArchivist, JobManager}
+import org.apache.flink.runtime.jobmanager.{HighAvailabilityMode, MemoryArchivist, JobManager}
 import org.apache.flink.runtime.testutils.TestingResourceManager
 import org.apache.flink.runtime.util.LeaderRetrievalUtils
 import org.apache.flink.runtime.{LogMessages, LeaderSessionMessageFilter, FlinkActor}
@@ -412,8 +412,7 @@ object TestingUtils {
     * @param configuration Configuration to use
     * @param jobManagerClass JobManager class to instantiate
     * @param prefix The prefix to use for the Actor names
-    *
-    * @return
+   * @return
     */
   def createJobManager(
       actorSystem: ActorSystem,
@@ -422,7 +421,8 @@ object TestingUtils {
       prefix: String)
     : ActorGateway = {
 
-    configuration.setString(ConfigConstants.HA_MODE,
+    configuration.setString(
+      HighAvailabilityOptions.HA_MODE,
       ConfigConstants.DEFAULT_HA_MODE)
 
       val (actor, _) = JobManager.startJobManagerActors(
@@ -502,7 +502,8 @@ object TestingUtils {
       configuration: Configuration)
   : ActorGateway = {
 
-    configuration.setString(ConfigConstants.HA_MODE,
+    configuration.setString(
+      HighAvailabilityOptions.HA_MODE,
       ConfigConstants.DEFAULT_HA_MODE)
 
     val actor = FlinkResourceManager.startResourceManagerActors(

http://git-wip-us.apache.org/repos/asf/flink/blob/c8dc074a/flink-streaming-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/RollingSinkSecuredITCase.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/RollingSinkSecuredITCase.java b/flink-streaming-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/RollingSinkSecuredITCase.java
index 051175a..c005814 100644
--- a/flink-streaming-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/RollingSinkSecuredITCase.java
+++ b/flink-streaming-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/RollingSinkSecuredITCase.java
@@ -20,6 +20,7 @@ package org.apache.flink.streaming.connectors.fs;
 
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
 import org.apache.flink.runtime.security.SecurityContext;
 import org.apache.flink.streaming.util.TestStreamEnvironment;
 import org.apache.flink.test.util.SecureTestEnvironment;
@@ -215,10 +216,10 @@ public class RollingSinkSecuredITCase extends RollingSinkITCase {
 			config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, DEFAULT_PARALLELISM);
 			config.setBoolean(ConfigConstants.LOCAL_START_WEBSERVER, false);
 			config.setInteger(ConfigConstants.LOCAL_NUMBER_JOB_MANAGER, 3);
-			config.setString(ConfigConstants.RECOVERY_MODE, "zookeeper");
+			config.setString(HighAvailabilityOptions.HA_MODE, "zookeeper");
 			config.setString(ConfigConstants.STATE_BACKEND, "filesystem");
 			config.setString(ConfigConstants.ZOOKEEPER_CHECKPOINTS_PATH, hdfsURI + "/flink/checkpoints");
-			config.setString(ConfigConstants.ZOOKEEPER_RECOVERY_PATH, hdfsURI + "/flink/recovery");
+			config.setString(HighAvailabilityOptions.HA_STORAGE_PATH, hdfsURI + "/flink/recovery");
 			config.setString("state.backend.fs.checkpointdir", hdfsURI + "/flink/checkpoints");
 
 			SecureTestEnvironment.populateFlinkSecureConfigurations(config);

http://git-wip-us.apache.org/repos/asf/flink/blob/c8dc074a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/SecureTestEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/SecureTestEnvironment.java b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/SecureTestEnvironment.java
index b5e622b..0250c16 100644
--- a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/SecureTestEnvironment.java
+++ b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/SecureTestEnvironment.java
@@ -21,6 +21,7 @@ package org.apache.flink.test.util;
 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.runtime.security.SecurityContext;
 import org.apache.hadoop.minikdc.MiniKdc;
 import org.junit.rules.TemporaryFolder;
@@ -115,7 +116,7 @@ public class SecureTestEnvironment {
 			Configuration flinkConfig = GlobalConfiguration.loadConfiguration();
 			flinkConfig.setString(ConfigConstants.SECURITY_KEYTAB_KEY, testKeytab);
 			flinkConfig.setString(ConfigConstants.SECURITY_PRINCIPAL_KEY, testPrincipal);
-			flinkConfig.setBoolean(ConfigConstants.ZOOKEEPER_SASL_DISABLE, false);
+			flinkConfig.setBoolean(HighAvailabilityOptions.ZOOKEEPER_SASL_DISABLE, false);
 			ctx.setFlinkConfiguration(flinkConfig);
 			TestingSecurityContext.install(ctx, getClientSecurityConfigurationMap());
 

http://git-wip-us.apache.org/repos/asf/flink/blob/c8dc074a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/TestBaseUtils.java
----------------------------------------------------------------------
diff --git a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/TestBaseUtils.java b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/TestBaseUtils.java
index b774f97..aa5e7d3 100644
--- a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/TestBaseUtils.java
+++ b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/TestBaseUtils.java
@@ -32,6 +32,7 @@ import org.apache.commons.io.IOUtils;
 import org.apache.flink.api.java.tuple.Tuple;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
 import org.apache.flink.runtime.messages.TaskManagerMessages;
 import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
 import org.apache.flink.util.TestLogger;
@@ -121,7 +122,7 @@ public class TestBaseUtils extends TestLogger {
 
 		if (startZooKeeper) {
 			config.setInteger(ConfigConstants.LOCAL_NUMBER_JOB_MANAGER, 3);
-			config.setString(ConfigConstants.HA_MODE, "zookeeper");
+			config.setString(HighAvailabilityOptions.HA_MODE, "zookeeper");
 		}
 
 		return startCluster(config, singleActorSystem);

http://git-wip-us.apache.org/repos/asf/flink/blob/c8dc074a/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 cc8ab80..4d10bf1 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
@@ -24,6 +24,7 @@ import org.apache.commons.io.FileUtils;
 import org.apache.flink.api.common.JobID;
 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.akka.ListeningBehaviour;
 import org.apache.flink.runtime.instance.AkkaActorGateway;
@@ -564,7 +565,7 @@ public class ChaosMonkeyITCase extends TestLogger {
 			fail(fsCheckpoints + " does not exist: " + Arrays.toString(FileStateBackendBasePath.listFiles()));
 		}
 
-		File fsRecovery = new File(new URI(config.getString(ConfigConstants.HA_ZOOKEEPER_STORAGE_PATH, "")).getPath());
+		File fsRecovery = new File(new URI(config.getString(HighAvailabilityOptions.HA_STORAGE_PATH)).getPath());
 
 		LOG.info("Checking " + fsRecovery);
 

http://git-wip-us.apache.org/repos/asf/flink/blob/c8dc074a/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 9b0d9de..a51f88b 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
@@ -30,6 +30,7 @@ import org.apache.flink.api.java.ExecutionEnvironment;
 import org.apache.flink.api.java.io.DiscardingOutputFormat;
 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.clusterframework.types.ResourceID;
 import org.apache.flink.runtime.instance.ActorGateway;
@@ -149,8 +150,8 @@ public class JobManagerHAProcessFailureBatchRecoveryITCase extends TestLogger {
 	 */
 	public void testJobManagerFailure(String zkQuorum, final File coordinateDir) throws Exception {
 		Configuration config = new Configuration();
-		config.setString(ConfigConstants.HA_MODE, "ZOOKEEPER");
-		config.setString(ConfigConstants.HA_ZOOKEEPER_QUORUM_KEY, zkQuorum);
+		config.setString(HighAvailabilityOptions.HA_MODE, "ZOOKEEPER");
+		config.setString(HighAvailabilityOptions.HA_ZOOKEEPER_QUORUM, zkQuorum);
 
 		ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment(
 				"leader", 1, config);

http://git-wip-us.apache.org/repos/asf/flink/blob/c8dc074a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/CliFrontendYarnAddressConfigurationTest.java
----------------------------------------------------------------------
diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/CliFrontendYarnAddressConfigurationTest.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/CliFrontendYarnAddressConfigurationTest.java
index 48ad7f5..4bcde16 100644
--- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/CliFrontendYarnAddressConfigurationTest.java
+++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/CliFrontendYarnAddressConfigurationTest.java
@@ -19,6 +19,7 @@
 package org.apache.flink.yarn;
 
 import org.apache.commons.cli.CommandLine;
+
 import org.apache.flink.client.CliFrontend;
 import org.apache.flink.client.cli.CliFrontendParser;
 import org.apache.flink.client.cli.CommandLineOptions;
@@ -27,7 +28,7 @@ import org.apache.flink.client.cli.RunOptions;
 import org.apache.flink.client.program.ClusterClient;
 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.IllegalConfigurationException;
 import org.apache.flink.test.util.TestBaseUtils;
 import org.apache.flink.yarn.cli.FlinkYarnSessionCli;
@@ -38,20 +39,20 @@ import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.client.api.YarnClient;
 import org.apache.hadoop.yarn.client.api.impl.YarnClientImpl;
 import org.apache.hadoop.yarn.exceptions.YarnException;
+
 import org.junit.AfterClass;
 import org.junit.Assert;
-import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.TemporaryFolder;
+
 import org.mockito.Mockito;
 
 import java.io.File;
 import java.io.IOException;
 import java.io.OutputStream;
 import java.io.PrintStream;
-import java.lang.reflect.Field;
 import java.net.InetSocketAddress;
 import java.nio.file.Files;
 import java.nio.file.StandardOpenOption;
@@ -202,7 +203,7 @@ public class CliFrontendYarnAddressConfigurationTest {
 				CliFrontendParser.parseRunCommand(new String[] {"-yid", TEST_YARN_APPLICATION_ID.toString()});
 
 		frontend.retrieveClient(options);
-		String zkNs = frontend.getConfiguration().getString(ConfigConstants.HA_ZOOKEEPER_NAMESPACE_KEY, "error");
+		String zkNs = frontend.getConfiguration().getValue(HighAvailabilityOptions.HA_CLUSTER_ID);
 		Assert.assertTrue(zkNs.matches("application_\\d+_0042"));
 	}
 
@@ -216,7 +217,7 @@ public class CliFrontendYarnAddressConfigurationTest {
 				CliFrontendParser.parseRunCommand(new String[] {"-yid", TEST_YARN_APPLICATION_ID.toString(), "-yz", overrideZkNamespace});
 
 		frontend.retrieveClient(options);
-		String zkNs = frontend.getConfiguration().getString(ConfigConstants.HA_ZOOKEEPER_NAMESPACE_KEY, "error");
+		String zkNs = frontend.getConfiguration().getValue(HighAvailabilityOptions.HA_CLUSTER_ID);
 		Assert.assertEquals(overrideZkNamespace, zkNs);
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/c8dc074a/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 9d6ff85..79f790f 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
@@ -26,6 +26,7 @@ import org.apache.flink.client.program.ClusterClient;
 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.runtime.akka.AkkaUtils;
 import org.apache.flink.runtime.instance.ActorGateway;
 import org.apache.flink.runtime.instance.AkkaActorGateway;
@@ -119,7 +120,7 @@ public class YARNHighAvailabilityITCase extends YarnTestBase {
 			zkServer.getConnectString() + "@@yarn.application-attempts=" + numberApplicationAttempts +
 			"@@" + ConfigConstants.STATE_BACKEND + "=FILESYSTEM" +
 			"@@" + FsStateBackendFactory.CHECKPOINT_DIRECTORY_URI_CONF_KEY + "=" + fsStateHandlePath + "/checkpoints" +
-			"@@" + ConfigConstants.HA_ZOOKEEPER_STORAGE_PATH + "=" + fsStateHandlePath + "/recovery");
+			"@@" + HighAvailabilityOptions.HA_STORAGE_PATH.key() + "=" + fsStateHandlePath + "/recovery");
 		flinkYarnClient.setConfigurationFilePath(new Path(confDirPath + File.separator + "flink-conf.yaml"));
 
 		ClusterClient yarnCluster = null;

http://git-wip-us.apache.org/repos/asf/flink/blob/c8dc074a/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 848013c..9481c24 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
@@ -22,6 +22,7 @@ import org.apache.flink.client.CliFrontend;
 import org.apache.flink.client.deployment.ClusterDescriptor;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.GlobalConfiguration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
 import org.apache.flink.runtime.akka.AkkaUtils;
 import org.apache.flink.runtime.jobmanager.HighAvailabilityMode;
 import org.apache.flink.runtime.security.SecurityContext;
@@ -539,11 +540,11 @@ public abstract class AbstractYarnClusterDescriptor implements ClusterDescriptor
 		// no user specified cli argument for namespace?
 		if (zkNamespace == null || zkNamespace.isEmpty()) {
 			// namespace defined in config? else use applicationId as default.
-			zkNamespace = flinkConfiguration.getString(ConfigConstants.HA_ZOOKEEPER_NAMESPACE_KEY, String.valueOf(appId));
+			zkNamespace = flinkConfiguration.getString(HighAvailabilityOptions.HA_CLUSTER_ID, String.valueOf(appId));
 			setZookeeperNamespace(zkNamespace);
 		}
 
-		flinkConfiguration.setString(ConfigConstants.HA_ZOOKEEPER_NAMESPACE_KEY, zkNamespace);
+		flinkConfiguration.setString(HighAvailabilityOptions.HA_CLUSTER_ID, zkNamespace);
 
 		if (HighAvailabilityMode.isHighAvailabilityModeActivated(flinkConfiguration)) {
 			// activate re-execution of failed applications

http://git-wip-us.apache.org/repos/asf/flink/blob/c8dc074a/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 b27876b..10e229e 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
@@ -25,6 +25,7 @@ import akka.actor.Props;
 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.runtime.akka.AkkaUtils;
 import org.apache.flink.runtime.clusterframework.BootstrapTools;
 import org.apache.flink.runtime.clusterframework.ContaineredTaskManagerParameters;
@@ -472,7 +473,7 @@ public class YarnApplicationMasterRunner {
 		// override zookeeper namespace with user cli argument (if provided)
 		String cliZKNamespace = ENV.get(YarnConfigKeys.ENV_ZOOKEEPER_NAMESPACE);
 		if (cliZKNamespace != null && !cliZKNamespace.isEmpty()) {
-			configuration.setString(ConfigConstants.HA_ZOOKEEPER_NAMESPACE_KEY, cliZKNamespace);
+			configuration.setString(HighAvailabilityOptions.HA_CLUSTER_ID, cliZKNamespace);
 		}
 
 		// if a web monitor shall be started, set the port to random binding

http://git-wip-us.apache.org/repos/asf/flink/blob/c8dc074a/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 d09340c..e4da140 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
@@ -29,6 +29,7 @@ import org.apache.flink.client.cli.CustomCommandLine;
 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.IllegalConfigurationException;
 import org.apache.flink.runtime.security.SecurityContext;
 import org.apache.flink.yarn.AbstractYarnClusterDescriptor;
@@ -60,7 +61,6 @@ import java.util.Map;
 import java.util.Properties;
 
 import static org.apache.flink.client.cli.CliFrontendParser.ADDRESS_OPTION;
-import static org.apache.flink.configuration.ConfigConstants.HA_ZOOKEEPER_NAMESPACE_KEY;
 
 /**
  * Class handling the command line interface to the YARN session.
@@ -513,8 +513,8 @@ public class FlinkYarnSessionCli implements CustomCommandLine<YarnClusterClient>
 		if(null != applicationID) {
 			String zkNamespace = cmdLine.hasOption(ZOOKEEPER_NAMESPACE.getOpt()) ?
 					cmdLine.getOptionValue(ZOOKEEPER_NAMESPACE.getOpt())
-					: config.getString(HA_ZOOKEEPER_NAMESPACE_KEY, applicationID);
-			config.setString(HA_ZOOKEEPER_NAMESPACE_KEY, zkNamespace);
+					: config.getString(HighAvailabilityOptions.HA_CLUSTER_ID, applicationID);
+			config.setString(HighAvailabilityOptions.HA_CLUSTER_ID, zkNamespace);
 
 			AbstractYarnClusterDescriptor yarnDescriptor = getClusterDescriptor();
 			yarnDescriptor.setFlinkConfiguration(config);


[8/8] flink git commit: [FLINK-4339] [cluster management] Implement Slot Pool core on JobManager side

Posted by se...@apache.org.
[FLINK-4339] [cluster management] Implement Slot Pool core on JobManager side


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

Branch: refs/heads/flip-6
Commit: 48c936eeda7b265f32bbf9f14d15f78eae15c06f
Parents: 34fef47
Author: Kurt Young <yk...@gmail.com>
Authored: Thu Oct 13 04:59:46 2016 +0800
Committer: Stephan Ewen <se...@apache.org>
Committed: Thu Oct 13 17:38:17 2016 +0200

----------------------------------------------------------------------
 .../CheckpointCoordinatorGateway.java           |   2 -
 .../clusterframework/types/ResourceProfile.java |   8 +
 .../runtime/clusterframework/types/SlotID.java  |  16 +-
 .../flink/runtime/instance/SlotDescriptor.java  | 161 +++++
 .../apache/flink/runtime/instance/SlotPool.java | 675 +++++++++++++++++++
 .../apache/flink/runtime/akka/AkkaUtils.scala   |   4 +-
 .../runtime/instance/AllocatedSlotsTest.java    | 135 ++++
 .../runtime/instance/AvailableSlotsTest.java    | 123 ++++
 .../flink/runtime/instance/SlotPoolTest.java    | 297 ++++++++
 9 files changed, 1411 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/48c936ee/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorGateway.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorGateway.java
index 196ef5c..fa09123 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorGateway.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorGateway.java
@@ -23,8 +23,6 @@ import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
 import org.apache.flink.runtime.rpc.RpcGateway;
 import org.apache.flink.runtime.state.CheckpointStateHandles;
 
-import java.util.UUID;
-
 public interface CheckpointCoordinatorGateway extends RpcGateway {
 
 	void acknowledgeCheckpoint(

http://git-wip-us.apache.org/repos/asf/flink/blob/48c936ee/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceProfile.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceProfile.java b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceProfile.java
index fa3aabc..1d8075e 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceProfile.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceProfile.java
@@ -70,4 +70,12 @@ public class ResourceProfile implements Serializable {
 	public boolean isMatching(ResourceProfile required) {
 		return cpuCores >= required.getCpuCores() && memoryInMB >= required.getMemoryInMB();
 	}
+
+	@Override
+	public String toString() {
+		return "ResourceProfile{" +
+			"cpuCores=" + cpuCores +
+			", memoryInMB=" + memoryInMB +
+			'}';
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/48c936ee/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/SlotID.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/SlotID.java b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/SlotID.java
index e831a5d..237597b 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/SlotID.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/SlotID.java
@@ -33,11 +33,11 @@ public class SlotID implements ResourceIDRetrievable, Serializable {
 	private final ResourceID resourceId;
 
 	/** The numeric id for single slot */
-	private final int slotId;
+	private final int slotNumber;
 
-	public SlotID(ResourceID resourceId, int slotId) {
+	public SlotID(ResourceID resourceId, int slotNumber) {
 		this.resourceId = checkNotNull(resourceId, "ResourceID must not be null");
-		this.slotId = slotId;
+		this.slotNumber = slotNumber;
 	}
 
 	// ------------------------------------------------------------------------
@@ -47,6 +47,10 @@ public class SlotID implements ResourceIDRetrievable, Serializable {
 		return resourceId;
 	}
 
+	public int getSlotNumber() {
+		return slotNumber;
+	}
+
 	// ------------------------------------------------------------------------
 
 	@Override
@@ -60,7 +64,7 @@ public class SlotID implements ResourceIDRetrievable, Serializable {
 
 		SlotID slotID = (SlotID) o;
 
-		if (slotId != slotID.slotId) {
+		if (slotNumber != slotID.slotNumber) {
 			return false;
 		}
 		return resourceId.equals(slotID.resourceId);
@@ -69,13 +73,13 @@ public class SlotID implements ResourceIDRetrievable, Serializable {
 	@Override
 	public int hashCode() {
 		int result = resourceId.hashCode();
-		result = 31 * result + slotId;
+		result = 31 * result + slotNumber;
 		return result;
 	}
 
 	@Override
 	public String toString() {
-		return resourceId + "_" + slotId;
+		return resourceId + "_" + slotNumber;
 	}
 
 	/**

http://git-wip-us.apache.org/repos/asf/flink/blob/48c936ee/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotDescriptor.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotDescriptor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotDescriptor.java
new file mode 100644
index 0000000..be7cf96
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotDescriptor.java
@@ -0,0 +1,161 @@
+/*
+ * 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.instance;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * The description of slots, TaskManagers offer one or more task slots, which define a slice of
+ * their resources. This description will contain some static information about the slot, such
+ * as the location and numeric id of the slot, rpc gateway to communicate with the TaskManager which
+ * owns the slot.
+ */
+public class SlotDescriptor {
+
+	/** The ID of the job this slice belongs to. */
+	private final JobID jobID;
+
+	/** The location information of the TaskManager to which this slot belongs */
+	private final TaskManagerLocation taskManagerLocation;
+
+	/** The number of the slot on which the task is deployed */
+	private final int slotNumber;
+
+	/** The resource profile of the slot provides */
+	private final ResourceProfile resourceProfile;
+
+	/** TEMP until the new RPC is in place: The actor gateway to communicate with the TaskManager */
+	private final ActorGateway taskManagerActorGateway;
+
+	public SlotDescriptor(
+		final JobID jobID,
+		final TaskManagerLocation location,
+		final int slotNumber,
+		final ResourceProfile resourceProfile,
+		final ActorGateway actorGateway)
+	{
+		this.jobID = checkNotNull(jobID);
+		this.taskManagerLocation = checkNotNull(location);
+		this.slotNumber = slotNumber;
+		this.resourceProfile = checkNotNull(resourceProfile);
+		this.taskManagerActorGateway = checkNotNull(actorGateway);
+	}
+
+	public SlotDescriptor(final SlotDescriptor other) {
+		this.jobID = other.jobID;
+		this.taskManagerLocation = other.taskManagerLocation;
+		this.slotNumber = other.slotNumber;
+		this.resourceProfile = other.resourceProfile;
+		this.taskManagerActorGateway = other.taskManagerActorGateway;
+	}
+	
+	// TODO - temporary workaround until we have the SlotDesriptor in the Slot
+	public SlotDescriptor(final Slot slot) {
+		this.jobID = slot.getJobID();
+		this.taskManagerLocation = slot.getTaskManagerLocation();
+		this.slotNumber = slot.getRootSlotNumber();
+		this.resourceProfile = new ResourceProfile(0, 0);
+		this.taskManagerActorGateway = slot.getTaskManagerActorGateway();
+	}
+
+	/**
+	 * Returns the ID of the job this allocated slot belongs to.
+	 *
+	 * @return the ID of the job this allocated slot belongs to
+	 */
+	public JobID getJobID() {
+		return jobID;
+	}
+
+	/**
+	 * Gets the number of the slot.
+	 *
+	 * @return The number of the slot on the TaskManager.
+	 */
+	public int getSlotNumber() {
+		return slotNumber;
+	}
+
+	/**
+	 * Gets the resource profile of the slot.
+	 *
+	 * @return The resource profile of the slot.
+	 */
+	public ResourceProfile getResourceProfile() {
+		return resourceProfile;
+	}
+
+	/**
+	 * Gets the location info of the TaskManager that offers this slot.
+	 *
+	 * @return The location info of the TaskManager that offers this slot
+	 */
+	public TaskManagerLocation getTaskManagerLocation() {
+		return taskManagerLocation;
+	}
+
+	/**
+	 * Gets the actor gateway that can be used to send messages to the TaskManager.
+	 * <p>
+	 * This method should be removed once the new interface-based RPC abstraction is in place
+	 *
+	 * @return The actor gateway that can be used to send messages to the TaskManager.
+	 */
+	public ActorGateway getTaskManagerActorGateway() {
+		return taskManagerActorGateway;
+	}
+
+	@Override
+	public boolean equals(Object o) {
+		if (this == o) {
+			return true;
+		}
+		if (o == null || getClass() != o.getClass()) {
+			return false;
+		}
+
+		SlotDescriptor that = (SlotDescriptor) o;
+
+		if (slotNumber != that.slotNumber) {
+			return false;
+		}
+		if (!jobID.equals(that.jobID)) {
+			return false;
+		}
+		return taskManagerLocation.equals(that.taskManagerLocation);
+
+	}
+
+	@Override
+	public int hashCode() {
+		int result = jobID.hashCode();
+		result = 31 * result + taskManagerLocation.hashCode();
+		result = 31 * result + slotNumber;
+		return result;
+	}
+
+	@Override
+	public String toString() {
+		return taskManagerLocation + " - " + slotNumber;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/48c936ee/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotPool.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotPool.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotPool.java
new file mode 100644
index 0000000..e7857c1
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotPool.java
@@ -0,0 +1,675 @@
+/*
+ * 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.instance;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.concurrent.ApplyFunction;
+import org.apache.flink.runtime.concurrent.BiFunction;
+import org.apache.flink.runtime.concurrent.Future;
+import org.apache.flink.runtime.concurrent.impl.FlinkCompletableFuture;
+import org.apache.flink.runtime.jobmanager.slots.SlotOwner;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway;
+import org.apache.flink.runtime.resourcemanager.SlotRequest;
+import org.apache.flink.runtime.resourcemanager.messages.jobmanager.RMSlotRequestRejected;
+import org.apache.flink.runtime.resourcemanager.messages.jobmanager.RMSlotRequestReply;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.Executor;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * The slot pool serves slot request issued by Scheduler or ExecutionGraph. It will will attempt to acquire new slots
+ * from the ResourceManager when it cannot serve a slot request. If no ResourceManager is currently available,
+ * or it gets a decline from the ResourceManager, or a request times out, it fails the slot request. The slot pool also
+ * holds all the slots that were offered to it and accepted, and can thus provides registered free slots even if the
+ * ResourceManager is down. The slots will only be released when they are useless, e.g. when the job is fully running
+ * but we still have some free slots.
+ * <p>
+ * All the allocation or the slot offering will be identified by self generated AllocationID, we will use it to
+ * eliminate ambiguities.
+ */
+public class SlotPool implements SlotOwner {
+
+	private static final Logger LOG = LoggerFactory.getLogger(SlotPool.class);
+
+	private final Object lock = new Object();
+
+	/** The executor which is used to execute futures */
+	private final Executor executor;
+
+	/** All registered resources, slots will be accepted and used only if the resource is registered */
+	private final Set<ResourceID> registeredResources;
+
+	/** The book-keeping of all allocated slots */
+	private final AllocatedSlots allocatedSlots;
+
+	/** The book-keeping of all available slots */
+	private final AvailableSlots availableSlots;
+
+	/** All pending requests waiting for slots */
+	private final Map<AllocationID, Tuple2<SlotRequest, FlinkCompletableFuture<SlotDescriptor>>> pendingRequests;
+
+	/** Timeout of slot allocation */
+	private final Time timeout;
+
+	/** the leader id of job manager */
+	private UUID jobManagerLeaderId;
+
+	/** The leader id of resource manager */
+	private UUID resourceManagerLeaderId;
+
+	/** The gateway to communicate with resource manager */
+	private ResourceManagerGateway resourceManagerGateway;
+
+	public SlotPool(final Executor executor) {
+		this.executor = executor;
+		this.registeredResources = new HashSet<>();
+		this.allocatedSlots = new AllocatedSlots();
+		this.availableSlots = new AvailableSlots();
+		this.pendingRequests = new HashMap<>();
+		this.timeout = Time.of(5, TimeUnit.SECONDS);
+	}
+
+	public void setJobManagerLeaderId(final UUID jobManagerLeaderId) {
+		this.jobManagerLeaderId = jobManagerLeaderId;
+	}
+
+	// ------------------------------------------------------------------------
+	//  Slot Allocation
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Try to allocate a simple slot with specified resource profile.
+	 *
+	 * @param jobID           The job id which the slot allocated for
+	 * @param resourceProfile The needed resource profile
+	 * @return The future of allocated simple slot
+	 */
+	public Future<SimpleSlot> allocateSimpleSlot(final JobID jobID, final ResourceProfile resourceProfile) {
+		return allocateSimpleSlot(jobID, resourceProfile, new AllocationID());
+	}
+
+
+	/**
+	 * Try to allocate a simple slot with specified resource profile and specified allocation id. It's mainly
+	 * for testing purpose since we need to specify whatever allocation id we want.
+	 */
+	@VisibleForTesting
+	Future<SimpleSlot> allocateSimpleSlot(
+		final JobID jobID,
+		final ResourceProfile resourceProfile,
+		final AllocationID allocationID)
+	{
+		final FlinkCompletableFuture<SlotDescriptor> future = new FlinkCompletableFuture<>();
+
+		internalAllocateSlot(jobID, allocationID, resourceProfile, future);
+
+		final SlotOwner owner = this;
+		return future.thenApplyAsync(
+			new ApplyFunction<SlotDescriptor, SimpleSlot>() {
+				@Override
+				public SimpleSlot apply(SlotDescriptor descriptor) {
+					SimpleSlot slot = new SimpleSlot(
+							descriptor.getJobID(), SlotPool.this,
+							descriptor.getTaskManagerLocation(), descriptor.getSlotNumber(),
+							descriptor.getTaskManagerActorGateway());
+					synchronized (lock) {
+						// double validation since we are out of the lock protection after the slot is granted
+						if (registeredResources.contains(descriptor.getTaskManagerLocation().getResourceID())) {
+							LOG.info("Allocation[{}] Allocated simple slot: {} for job {}.", allocationID, slot, jobID);
+							allocatedSlots.add(allocationID, descriptor, slot);
+						}
+						else {
+							throw new RuntimeException("Resource was marked dead asynchronously.");
+						}
+					}
+					return slot;
+				}
+			},
+			executor
+		);
+	}
+
+
+	/**
+	 * Try to allocate a shared slot with specified resource profile.
+	 *
+	 * @param jobID                  The job id which the slot allocated for
+	 * @param resourceProfile        The needed resource profile
+	 * @param sharingGroupAssignment The slot sharing group of the vertex
+	 * @return The future of allocated shared slot
+	 */
+	public Future<SharedSlot> allocateSharedSlot(
+		final JobID jobID,
+		final ResourceProfile resourceProfile,
+		final SlotSharingGroupAssignment sharingGroupAssignment)
+	{
+		return allocateSharedSlot(jobID, resourceProfile, sharingGroupAssignment, new AllocationID());
+	}
+
+	/**
+	 * Try to allocate a shared slot with specified resource profile and specified allocation id. It's mainly
+	 * for testing purpose since we need to specify whatever allocation id we want.
+	 */
+	@VisibleForTesting
+	Future<SharedSlot> allocateSharedSlot(
+		final JobID jobID,
+		final ResourceProfile resourceProfile,
+		final SlotSharingGroupAssignment sharingGroupAssignment,
+		final AllocationID allocationID)
+	{
+		final FlinkCompletableFuture<SlotDescriptor> future = new FlinkCompletableFuture<>();
+
+		internalAllocateSlot(jobID, allocationID, resourceProfile, future);
+
+		return future.thenApplyAsync(
+			new ApplyFunction<SlotDescriptor, SharedSlot>() {
+				@Override
+				public SharedSlot apply(SlotDescriptor descriptor) {
+					SharedSlot slot = new SharedSlot(
+							descriptor.getJobID(), SlotPool.this, descriptor.getTaskManagerLocation(),
+							descriptor.getSlotNumber(), descriptor.getTaskManagerActorGateway(),
+							sharingGroupAssignment);
+
+					synchronized (lock) {
+						// double validation since we are out of the lock protection after the slot is granted
+						if (registeredResources.contains(descriptor.getTaskManagerLocation().getResourceID())) {
+							LOG.info("Allocation[{}] Allocated shared slot: {} for job {}.", allocationID, slot, jobID);
+							allocatedSlots.add(allocationID, descriptor, slot);
+						}
+						else {
+							throw new RuntimeException("Resource was marked dead asynchronously.");
+						}
+					}
+					return slot;
+				}
+			},
+			executor
+		);
+	}
+
+	/**
+	 * Internally allocate the slot with specified resource profile. We will first check whether we have some
+	 * free slot which can meet the requirement already and allocate it immediately. Otherwise, we will try to
+	 * allocation the slot from resource manager.
+	 */
+	private void internalAllocateSlot(
+		final JobID jobID,
+		final AllocationID allocationID,
+		final ResourceProfile resourceProfile,
+		final FlinkCompletableFuture<SlotDescriptor> future)
+	{
+		LOG.info("Allocation[{}] Allocating slot with {} for Job {}.", allocationID, resourceProfile, jobID);
+
+		synchronized (lock) {
+			// check whether we have any free slot which can match the required resource profile
+			SlotDescriptor freeSlot = availableSlots.poll(resourceProfile);
+			if (freeSlot != null) {
+				future.complete(freeSlot);
+			}
+			else {
+				if (resourceManagerGateway != null) {
+					LOG.info("Allocation[{}] No available slot exists, trying to allocate from resource manager.",
+						allocationID);
+					SlotRequest slotRequest = new SlotRequest(jobID, allocationID, resourceProfile);
+					pendingRequests.put(allocationID, new Tuple2<>(slotRequest, future));
+					resourceManagerGateway.requestSlot(jobManagerLeaderId, resourceManagerLeaderId, slotRequest, timeout)
+						.handleAsync(new BiFunction<RMSlotRequestReply, Throwable, Void>() {
+							@Override
+							public Void apply(RMSlotRequestReply slotRequestReply, Throwable throwable) {
+								if (throwable != null) {
+									future.completeExceptionally(
+										new Exception("Slot allocation from resource manager failed", throwable));
+								} else if (slotRequestReply instanceof RMSlotRequestRejected) {
+									future.completeExceptionally(
+										new Exception("Slot allocation rejected by resource manager"));
+								}
+								return null;
+							}
+						}, executor);
+				}
+				else {
+					LOG.warn("Allocation[{}] Resource manager not available right now.", allocationID);
+					future.completeExceptionally(new Exception("Resource manager not available right now."));
+				}
+			}
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  Slot De-allocation
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Return the slot back to this pool without releasing it. It's mainly called by failed / cancelled tasks, and the
+	 * slot can be reused by other pending requests if the resource profile matches.n
+	 *
+	 * @param slot The slot needs to be returned
+	 * @return True if the returning slot been accepted
+	 */
+	@Override
+	public boolean returnAllocatedSlot(Slot slot) {
+		checkNotNull(slot);
+		checkArgument(!slot.isAlive(), "slot is still alive");
+		checkArgument(slot.getOwner() == this, "slot belongs to the wrong pool.");
+
+		if (slot.markReleased()) {
+			synchronized (lock) {
+				final SlotDescriptor slotDescriptor = allocatedSlots.remove(slot);
+				if (slotDescriptor != null) {
+					// check if this TaskManager is valid
+					if (!registeredResources.contains(slot.getTaskManagerID())) {
+						return false;
+					}
+
+					final FlinkCompletableFuture<SlotDescriptor> pendingRequest = pollPendingRequest(slotDescriptor);
+					if (pendingRequest != null) {
+						pendingRequest.complete(slotDescriptor);
+					}
+					else {
+						availableSlots.add(slotDescriptor);
+					}
+
+					return true;
+				}
+				else {
+					throw new IllegalArgumentException("Slot was not allocated from this pool.");
+				}
+			}
+		}
+		else {
+			return false;
+		}
+	}
+
+	private FlinkCompletableFuture<SlotDescriptor> pollPendingRequest(final SlotDescriptor slotDescriptor) {
+		for (Map.Entry<AllocationID, Tuple2<SlotRequest, FlinkCompletableFuture<SlotDescriptor>>> entry : pendingRequests.entrySet()) {
+			final Tuple2<SlotRequest, FlinkCompletableFuture<SlotDescriptor>> pendingRequest = entry.getValue();
+			if (slotDescriptor.getResourceProfile().isMatching(pendingRequest.f0.getResourceProfile())) {
+				pendingRequests.remove(entry.getKey());
+				return pendingRequest.f1;
+			}
+		}
+		return null;
+	}
+
+	// ------------------------------------------------------------------------
+	//  Slot Releasing
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Release slot to TaskManager, called for finished tasks or canceled jobs.
+	 *
+	 * @param slot The slot needs to be released.
+	 */
+	public void releaseSlot(final Slot slot) {
+		synchronized (lock) {
+			allocatedSlots.remove(slot);
+			availableSlots.remove(new SlotDescriptor(slot));
+			// TODO: send release request to task manager
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  Slot Offering
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Slot offering by TaskManager with AllocationID. The AllocationID is originally generated by this pool and
+	 * transfer through the ResourceManager to TaskManager. We use it to distinguish the different allocation
+	 * we issued. Slot offering may be rejected if we find something mismatching or there is actually no pending
+	 * request waiting for this slot (maybe fulfilled by some other returned slot).
+	 *
+	 * @param allocationID   The allocation id of the lo
+	 * @param slotDescriptor The offered slot descriptor
+	 * @return True if we accept the offering
+	 */
+	public boolean offerSlot(final AllocationID allocationID, final SlotDescriptor slotDescriptor) {
+		synchronized (lock) {
+			// check if this TaskManager is valid
+			final ResourceID resourceID = slotDescriptor.getTaskManagerLocation().getResourceID();
+			if (!registeredResources.contains(resourceID)) {
+				LOG.warn("Allocation[{}] Slot offering from unregistered TaskManager: {}",
+					allocationID, slotDescriptor);
+				return false;
+			}
+
+			// check whether we have already using this slot
+			final Slot allocatedSlot = allocatedSlots.get(allocationID);
+			if (allocatedSlot != null) {
+				final SlotDescriptor allocatedSlotDescriptor = new SlotDescriptor(allocatedSlot);
+
+				if (allocatedSlotDescriptor.equals(slotDescriptor)) {
+					LOG.debug("Allocation[{}] Duplicated slot offering: {}",
+						allocationID, slotDescriptor);
+					return true;
+				}
+				else {
+					LOG.info("Allocation[{}] Allocation had been fulfilled by slot {}, rejecting offered slot {}",
+						allocationID, allocatedSlotDescriptor, slotDescriptor);
+					return false;
+				}
+			}
+
+			// check whether we already have this slot in free pool
+			if (availableSlots.contains(slotDescriptor)) {
+				LOG.debug("Allocation[{}] Duplicated slot offering: {}",
+					allocationID, slotDescriptor);
+				return true;
+			}
+
+			// check whether we have request waiting for this slot
+			if (pendingRequests.containsKey(allocationID)) {
+				FlinkCompletableFuture<SlotDescriptor> future = pendingRequests.remove(allocationID).f1;
+				future.complete(slotDescriptor);
+				return true;
+			}
+
+			// unwanted slot, rejecting this offer
+			return false;
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  Resource
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Register TaskManager to this pool, only those slots come from registered TaskManager will be considered valid.
+	 * Also it provides a way for us to keep "dead" or "abnormal" TaskManagers out of this pool.
+	 *
+	 * @param resourceID The id of the TaskManager
+	 */
+	public void registerResource(final ResourceID resourceID) {
+		synchronized (lock) {
+			registeredResources.add(resourceID);
+		}
+	}
+
+	/**
+	 * Unregister TaskManager from this pool, all the related slots will be released and tasks be canceled. Called
+	 * when we find some TaskManager becomes "dead" or "abnormal", and we decide to not using slots from it anymore.
+	 *
+	 * @param resourceID The id of the TaskManager
+	 */
+	public void releaseResource(final ResourceID resourceID) {
+		synchronized (lock) {
+			registeredResources.remove(resourceID);
+			availableSlots.removeByResource(resourceID);
+
+			final Set<Slot> allocatedSlotsForResource = allocatedSlots.getSlotsByResource(resourceID);
+			for (Slot slot : allocatedSlotsForResource) {
+				slot.releaseSlot();
+			}
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  ResourceManager
+	// ------------------------------------------------------------------------
+
+	public void setResourceManager(
+		final UUID resourceManagerLeaderId,
+		final ResourceManagerGateway resourceManagerGateway)
+	{
+		synchronized (lock) {
+			this.resourceManagerLeaderId = resourceManagerLeaderId;
+			this.resourceManagerGateway = resourceManagerGateway;
+		}
+	}
+
+	public void disconnectResourceManager() {
+		synchronized (lock) {
+			this.resourceManagerLeaderId = null;
+			this.resourceManagerGateway = null;
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  Helper classes
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Organize allocated slots from different points of view.
+	 */
+	static class AllocatedSlots {
+
+		/** All allocated slots organized by TaskManager */
+		private final Map<ResourceID, Set<Slot>> allocatedSlotsByResource;
+
+		/** All allocated slots organized by Slot object */
+		private final Map<Slot, AllocationID> allocatedSlots;
+
+		private final Map<Slot, SlotDescriptor> allocatedSlotsWithDescriptor;
+
+		/** All allocated slots organized by AllocationID */
+		private final Map<AllocationID, Slot> allocatedSlotsById;
+
+		AllocatedSlots() {
+			this.allocatedSlotsByResource = new HashMap<>();
+			this.allocatedSlots = new HashMap<>();
+			this.allocatedSlotsWithDescriptor = new HashMap<>();
+			this.allocatedSlotsById = new HashMap<>();
+		}
+
+		/**
+		 * Add a new allocation
+		 *
+		 * @param allocationID The allocation id
+		 * @param slot         The allocated slot
+		 */
+		void add(final AllocationID allocationID, final SlotDescriptor descriptor, final Slot slot) {
+			allocatedSlots.put(slot, allocationID);
+			allocatedSlotsById.put(allocationID, slot);
+			allocatedSlotsWithDescriptor.put(slot, descriptor);
+
+			final ResourceID resourceID = slot.getTaskManagerID();
+			Set<Slot> slotsForResource = allocatedSlotsByResource.get(resourceID);
+			if (slotsForResource == null) {
+				slotsForResource = new HashSet<>();
+				allocatedSlotsByResource.put(resourceID, slotsForResource);
+			}
+			slotsForResource.add(slot);
+		}
+
+		/**
+		 * Get allocated slot with allocation id
+		 *
+		 * @param allocationID The allocation id
+		 * @return The allocated slot, null if we can't find a match
+		 */
+		Slot get(final AllocationID allocationID) {
+			return allocatedSlotsById.get(allocationID);
+		}
+
+		/**
+		 * Check whether we have allocated this slot
+		 *
+		 * @param slot The slot needs to checked
+		 * @return True if we contains this slot
+		 */
+		boolean contains(final Slot slot) {
+			return allocatedSlots.containsKey(slot);
+		}
+
+		/**
+		 * Remove an allocation with slot.
+		 *
+		 * @param slot The slot needs to be removed
+		 */
+		SlotDescriptor remove(final Slot slot) {
+			final SlotDescriptor descriptor = allocatedSlotsWithDescriptor.remove(slot);
+			if (descriptor != null) {
+				final AllocationID allocationID = allocatedSlots.remove(slot);
+				if (allocationID != null) {
+					allocatedSlotsById.remove(allocationID);
+				} else {
+					throw new IllegalStateException("Bug: maps are inconsistent");
+				}
+
+				final ResourceID resourceID = slot.getTaskManagerID();
+				final Set<Slot> slotsForResource = allocatedSlotsByResource.get(resourceID);
+				slotsForResource.remove(slot);
+				if (slotsForResource.isEmpty()) {
+					allocatedSlotsByResource.remove(resourceID);
+				}
+				
+				return descriptor;
+			} else {
+				return null;
+			}
+		}
+
+		/**
+		 * Get all allocated slot from same TaskManager.
+		 *
+		 * @param resourceID The id of the TaskManager
+		 * @return Set of slots which are allocated from the same TaskManager
+		 */
+		Set<Slot> getSlotsByResource(final ResourceID resourceID) {
+			Set<Slot> slotsForResource = allocatedSlotsByResource.get(resourceID);
+			if (slotsForResource != null) {
+				return new HashSet<>(slotsForResource);
+			}
+			else {
+				return new HashSet<>();
+			}
+		}
+
+		@VisibleForTesting
+		boolean containResource(final ResourceID resourceID) {
+			return allocatedSlotsByResource.containsKey(resourceID);
+		}
+
+		@VisibleForTesting
+		int size() {
+			return allocatedSlots.size();
+		}
+	}
+
+	/**
+	 * Organize all available slots from different points of view.
+	 */
+	static class AvailableSlots {
+
+		/** All available slots organized by TaskManager */
+		private final Map<ResourceID, Set<SlotDescriptor>> availableSlotsByResource;
+
+		/** All available slots */
+		private final Set<SlotDescriptor> availableSlots;
+
+		AvailableSlots() {
+			this.availableSlotsByResource = new HashMap<>();
+			this.availableSlots = new HashSet<>();
+		}
+
+		/**
+		 * Add an available slot.
+		 *
+		 * @param descriptor The descriptor of the slot
+		 */
+		void add(final SlotDescriptor descriptor) {
+			availableSlots.add(descriptor);
+
+			final ResourceID resourceID = descriptor.getTaskManagerLocation().getResourceID();
+			Set<SlotDescriptor> slotsForResource = availableSlotsByResource.get(resourceID);
+			if (slotsForResource == null) {
+				slotsForResource = new HashSet<>();
+				availableSlotsByResource.put(resourceID, slotsForResource);
+			}
+			slotsForResource.add(descriptor);
+		}
+
+		/**
+		 * Check whether we have this slot
+		 *
+		 * @param slotDescriptor The descriptor of the slot
+		 * @return True if we contains this slot
+		 */
+		boolean contains(final SlotDescriptor slotDescriptor) {
+			return availableSlots.contains(slotDescriptor);
+		}
+
+		/**
+		 * Poll a slot which matches the required resource profile
+		 *
+		 * @param resourceProfile The required resource profile
+		 * @return Slot which matches the resource profile, null if we can't find a match
+		 */
+		SlotDescriptor poll(final ResourceProfile resourceProfile) {
+			for (SlotDescriptor slotDescriptor : availableSlots) {
+				if (slotDescriptor.getResourceProfile().isMatching(resourceProfile)) {
+					remove(slotDescriptor);
+					return slotDescriptor;
+				}
+			}
+			return null;
+		}
+
+		/**
+		 * Remove all available slots come from specified TaskManager.
+		 *
+		 * @param resourceID The id of the TaskManager
+		 */
+		void removeByResource(final ResourceID resourceID) {
+			final Set<SlotDescriptor> slotsForResource = availableSlotsByResource.remove(resourceID);
+			if (slotsForResource != null) {
+				for (SlotDescriptor slotDescriptor : slotsForResource) {
+					availableSlots.remove(slotDescriptor);
+				}
+			}
+		}
+
+		private void remove(final SlotDescriptor slotDescriptor) {
+			availableSlots.remove(slotDescriptor);
+
+			final ResourceID resourceID = slotDescriptor.getTaskManagerLocation().getResourceID();
+			final Set<SlotDescriptor> slotsForResource = checkNotNull(availableSlotsByResource.get(resourceID));
+			slotsForResource.remove(slotDescriptor);
+			if (slotsForResource.isEmpty()) {
+				availableSlotsByResource.remove(resourceID);
+			}
+		}
+
+		@VisibleForTesting
+		boolean containResource(final ResourceID resourceID) {
+			return availableSlotsByResource.containsKey(resourceID);
+		}
+
+		@VisibleForTesting
+		int size() {
+			return availableSlots.size();
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/48c936ee/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 9209d15..2461340 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
@@ -583,8 +583,8 @@ object AkkaUtils {
   }
 
   def formatDurationParingErrorMessage: String = {
-    "Duration format must be \"val unit\", where 'val' is a number and 'unit' is " + 
-      "(d|day)|(h|hour)|(min|minute)|s|sec|second)|(ms|milli|millisecond)|"+
+    "Duration format must be \"val unit\", where 'val' is a number and 'unit' is " +
+      "(d|day)|(h|hour)|(min|minute)|s|sec|second)|(ms|milli|millisecond)|" +
       "(�s|micro|microsecond)|(ns|nano|nanosecond)"
   }
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/48c936ee/flink-runtime/src/test/java/org/apache/flink/runtime/instance/AllocatedSlotsTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/AllocatedSlotsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/AllocatedSlotsTest.java
new file mode 100644
index 0000000..655a3ea
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/AllocatedSlotsTest.java
@@ -0,0 +1,135 @@
+/*
+ * 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.instance;
+
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+public class AllocatedSlotsTest {
+
+	@Test
+	public void testOperations() throws Exception {
+		SlotPool.AllocatedSlots allocatedSlots = new SlotPool.AllocatedSlots();
+
+		final AllocationID allocation1 = new AllocationID();
+		final ResourceID resource1 = new ResourceID("resource1");
+		final Slot slot1 = createSlot(resource1);
+
+		allocatedSlots.add(allocation1, new SlotDescriptor(slot1), slot1);
+
+		assertTrue(allocatedSlots.contains(slot1));
+		assertTrue(allocatedSlots.containResource(resource1));
+
+		assertEquals(slot1, allocatedSlots.get(allocation1));
+		assertEquals(1, allocatedSlots.getSlotsByResource(resource1).size());
+		assertEquals(1, allocatedSlots.size());
+
+		final AllocationID allocation2 = new AllocationID();
+		final Slot slot2 = createSlot(resource1);
+
+		allocatedSlots.add(allocation2, new SlotDescriptor(slot2), slot2);
+
+		assertTrue(allocatedSlots.contains(slot1));
+		assertTrue(allocatedSlots.contains(slot2));
+		assertTrue(allocatedSlots.containResource(resource1));
+
+		assertEquals(slot1, allocatedSlots.get(allocation1));
+		assertEquals(slot2, allocatedSlots.get(allocation2));
+		assertEquals(2, allocatedSlots.getSlotsByResource(resource1).size());
+		assertEquals(2, allocatedSlots.size());
+
+		final AllocationID allocation3 = new AllocationID();
+		final ResourceID resource2 = new ResourceID("resource2");
+		final Slot slot3 = createSlot(resource2);
+
+		allocatedSlots.add(allocation3, new SlotDescriptor(slot2), slot3);
+
+		assertTrue(allocatedSlots.contains(slot1));
+		assertTrue(allocatedSlots.contains(slot2));
+		assertTrue(allocatedSlots.contains(slot3));
+		assertTrue(allocatedSlots.containResource(resource1));
+		assertTrue(allocatedSlots.containResource(resource2));
+
+		assertEquals(slot1, allocatedSlots.get(allocation1));
+		assertEquals(slot2, allocatedSlots.get(allocation2));
+		assertEquals(slot3, allocatedSlots.get(allocation3));
+		assertEquals(2, allocatedSlots.getSlotsByResource(resource1).size());
+		assertEquals(1, allocatedSlots.getSlotsByResource(resource2).size());
+		assertEquals(3, allocatedSlots.size());
+
+		allocatedSlots.remove(slot2);
+
+		assertTrue(allocatedSlots.contains(slot1));
+		assertFalse(allocatedSlots.contains(slot2));
+		assertTrue(allocatedSlots.contains(slot3));
+		assertTrue(allocatedSlots.containResource(resource1));
+		assertTrue(allocatedSlots.containResource(resource2));
+
+		assertEquals(slot1, allocatedSlots.get(allocation1));
+		assertNull(allocatedSlots.get(allocation2));
+		assertEquals(slot3, allocatedSlots.get(allocation3));
+		assertEquals(1, allocatedSlots.getSlotsByResource(resource1).size());
+		assertEquals(1, allocatedSlots.getSlotsByResource(resource2).size());
+		assertEquals(2, allocatedSlots.size());
+
+		allocatedSlots.remove(slot1);
+
+		assertFalse(allocatedSlots.contains(slot1));
+		assertFalse(allocatedSlots.contains(slot2));
+		assertTrue(allocatedSlots.contains(slot3));
+		assertFalse(allocatedSlots.containResource(resource1));
+		assertTrue(allocatedSlots.containResource(resource2));
+
+		assertNull(allocatedSlots.get(allocation1));
+		assertNull(allocatedSlots.get(allocation2));
+		assertEquals(slot3, allocatedSlots.get(allocation3));
+		assertEquals(0, allocatedSlots.getSlotsByResource(resource1).size());
+		assertEquals(1, allocatedSlots.getSlotsByResource(resource2).size());
+		assertEquals(1, allocatedSlots.size());
+
+		allocatedSlots.remove(slot3);
+
+		assertFalse(allocatedSlots.contains(slot1));
+		assertFalse(allocatedSlots.contains(slot2));
+		assertFalse(allocatedSlots.contains(slot3));
+		assertFalse(allocatedSlots.containResource(resource1));
+		assertFalse(allocatedSlots.containResource(resource2));
+
+		assertNull(allocatedSlots.get(allocation1));
+		assertNull(allocatedSlots.get(allocation2));
+		assertNull(allocatedSlots.get(allocation3));
+		assertEquals(0, allocatedSlots.getSlotsByResource(resource1).size());
+		assertEquals(0, allocatedSlots.getSlotsByResource(resource2).size());
+		assertEquals(0, allocatedSlots.size());
+	}
+
+	private Slot createSlot(final ResourceID resourceId) {
+		Slot slot = mock(Slot.class);
+		when(slot.getTaskManagerID()).thenReturn(resourceId);
+		return slot;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/48c936ee/flink-runtime/src/test/java/org/apache/flink/runtime/instance/AvailableSlotsTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/AvailableSlotsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/AvailableSlotsTest.java
new file mode 100644
index 0000000..872810f
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/AvailableSlotsTest.java
@@ -0,0 +1,123 @@
+/*
+ * 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.instance;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+public class AvailableSlotsTest {
+
+	static final ResourceProfile DEFAULT_TESTING_PROFILE = new ResourceProfile(1.0, 512);
+
+	static final ResourceProfile DEFAULT_TESTING_BIG_PROFILE = new ResourceProfile(2.0, 1024);
+
+	@Test
+	public void testAddAndRemove() throws Exception {
+		SlotPool.AvailableSlots availableSlots = new SlotPool.AvailableSlots();
+
+		final ResourceID resource1 = new ResourceID("resource1");
+		final ResourceID resource2 = new ResourceID("resource2");
+
+		final SlotDescriptor slot1 = createSlotDescriptor(resource1);
+		final SlotDescriptor slot2 = createSlotDescriptor(resource1);
+		final SlotDescriptor slot3 = createSlotDescriptor(resource2);
+
+		availableSlots.add(slot1);
+		availableSlots.add(slot2);
+		availableSlots.add(slot3);
+
+		assertEquals(3, availableSlots.size());
+		assertTrue(availableSlots.contains(slot1));
+		assertTrue(availableSlots.contains(slot2));
+		assertTrue(availableSlots.contains(slot3));
+		assertTrue(availableSlots.containResource(resource1));
+		assertTrue(availableSlots.containResource(resource2));
+
+		availableSlots.removeByResource(resource1);
+
+		assertEquals(1, availableSlots.size());
+		assertFalse(availableSlots.contains(slot1));
+		assertFalse(availableSlots.contains(slot2));
+		assertTrue(availableSlots.contains(slot3));
+		assertFalse(availableSlots.containResource(resource1));
+		assertTrue(availableSlots.containResource(resource2));
+
+		availableSlots.removeByResource(resource2);
+
+		assertEquals(0, availableSlots.size());
+		assertFalse(availableSlots.contains(slot1));
+		assertFalse(availableSlots.contains(slot2));
+		assertFalse(availableSlots.contains(slot3));
+		assertFalse(availableSlots.containResource(resource1));
+		assertFalse(availableSlots.containResource(resource2));
+	}
+
+	@Test
+	public void testPollFreeSlot() {
+		SlotPool.AvailableSlots availableSlots = new SlotPool.AvailableSlots();
+
+		final ResourceID resource1 = new ResourceID("resource1");
+		final SlotDescriptor slot1 = createSlotDescriptor(resource1);
+
+		availableSlots.add(slot1);
+
+		assertEquals(1, availableSlots.size());
+		assertTrue(availableSlots.contains(slot1));
+		assertTrue(availableSlots.containResource(resource1));
+
+		assertNull(availableSlots.poll(DEFAULT_TESTING_BIG_PROFILE));
+
+		assertEquals(slot1, availableSlots.poll(DEFAULT_TESTING_PROFILE));
+		assertEquals(0, availableSlots.size());
+		assertFalse(availableSlots.contains(slot1));
+		assertFalse(availableSlots.containResource(resource1));
+	}
+
+	static SlotDescriptor createSlotDescriptor(final ResourceID resourceID) {
+		return createSlotDescriptor(resourceID, new JobID());
+	}
+
+	static SlotDescriptor createSlotDescriptor(final ResourceID resourceID, final JobID jobID) {
+		return createSlotDescriptor(resourceID, jobID, DEFAULT_TESTING_PROFILE);
+	}
+
+	static SlotDescriptor createSlotDescriptor(final ResourceID resourceID, final JobID jobID,
+		final ResourceProfile resourceProfile)
+	{
+		return createSlotDescriptor(resourceID, jobID, resourceProfile, 0);
+	}
+
+	static SlotDescriptor createSlotDescriptor(final ResourceID resourceID, final JobID jobID,
+		final ResourceProfile resourceProfile, final int slotNumber)
+	{
+		TaskManagerLocation location = mock(TaskManagerLocation.class);
+		when(location.getResourceID()).thenReturn(resourceID);
+		return new SlotDescriptor(jobID, location, slotNumber, resourceProfile, mock(ActorGateway.class));
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/48c936ee/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SlotPoolTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SlotPoolTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SlotPoolTest.java
new file mode 100644
index 0000000..30cdbd6
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SlotPoolTest.java
@@ -0,0 +1,297 @@
+/*
+ * 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.instance;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.concurrent.BiFunction;
+import org.apache.flink.runtime.concurrent.Future;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway;
+import org.apache.flink.runtime.resourcemanager.SlotRequest;
+import org.apache.flink.util.TestLogger;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.UUID;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.flink.runtime.instance.AvailableSlotsTest.DEFAULT_TESTING_PROFILE;
+import static org.apache.flink.runtime.instance.AvailableSlotsTest.createSlotDescriptor;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+public class SlotPoolTest extends TestLogger {
+
+	private Executor executor;
+
+	private SlotPool slotPool;
+
+	private ResourceManagerGateway resourceManagerGateway;
+
+	@Before
+	public void setUp() throws Exception {
+		this.executor = Executors.newFixedThreadPool(1);
+		this.slotPool = new SlotPool(executor);
+		this.resourceManagerGateway = mock(ResourceManagerGateway.class);
+		when(resourceManagerGateway
+			.requestSlot(any(UUID.class), any(UUID.class), any(SlotRequest.class), any(Time.class)))
+			.thenReturn(mock(Future.class));
+		slotPool.setResourceManager(UUID.randomUUID(), resourceManagerGateway);
+		slotPool.setJobManagerLeaderId(UUID.randomUUID());
+	}
+
+	@After
+	public void tearDown() throws Exception {
+	}
+
+	@Test
+	public void testAllocateSimpleSlot() throws Exception {
+		ResourceID resourceID = new ResourceID("resource");
+		slotPool.registerResource(resourceID);
+
+		JobID jobID = new JobID();
+		AllocationID allocationID = new AllocationID();
+		Future<SimpleSlot> future = slotPool.allocateSimpleSlot(jobID, DEFAULT_TESTING_PROFILE, allocationID);
+		assertFalse(future.isDone());
+		verify(resourceManagerGateway).requestSlot(any(UUID.class), any(UUID.class), any(SlotRequest.class), any(Time.class));
+
+		SlotDescriptor slotDescriptor = createSlotDescriptor(resourceID, jobID, DEFAULT_TESTING_PROFILE);
+		assertTrue(slotPool.offerSlot(allocationID, slotDescriptor));
+
+		SimpleSlot slot = future.get(1, TimeUnit.SECONDS);
+		assertTrue(future.isDone());
+		assertTrue(slot.isAlive());
+		assertEquals(resourceID, slot.getTaskManagerID());
+		assertEquals(jobID, slot.getJobID());
+		assertEquals(slotPool, slot.getOwner());
+	}
+
+	@Test
+	public void testAllocateSharedSlot() throws Exception {
+		ResourceID resourceID = new ResourceID("resource");
+		slotPool.registerResource(resourceID);
+
+		JobVertexID vid = new JobVertexID();
+		SlotSharingGroup sharingGroup = new SlotSharingGroup(vid);
+		SlotSharingGroupAssignment assignment = sharingGroup.getTaskAssignment();
+
+		JobID jobID = new JobID();
+		AllocationID allocationID = new AllocationID();
+		Future<SharedSlot> future = slotPool.allocateSharedSlot(jobID, DEFAULT_TESTING_PROFILE, assignment, allocationID);
+
+		assertFalse(future.isDone());
+		verify(resourceManagerGateway).requestSlot(any(UUID.class), any(UUID.class), any(SlotRequest.class), any(Time.class));
+
+		SlotDescriptor slotDescriptor = createSlotDescriptor(resourceID, jobID, DEFAULT_TESTING_PROFILE);
+		assertTrue(slotPool.offerSlot(allocationID, slotDescriptor));
+
+		SharedSlot slot = future.get(1, TimeUnit.SECONDS);
+		assertTrue(future.isDone());
+		assertTrue(slot.isAlive());
+		assertEquals(resourceID, slot.getTaskManagerID());
+		assertEquals(jobID, slot.getJobID());
+		assertEquals(slotPool, slot.getOwner());
+
+		SimpleSlot simpleSlot = slot.allocateSubSlot(vid);
+		assertNotNull(simpleSlot);
+		assertTrue(simpleSlot.isAlive());
+	}
+
+	@Test
+	public void testAllocateSlotWithoutResourceManager() throws Exception {
+		slotPool.disconnectResourceManager();
+		Future<SimpleSlot> future = slotPool.allocateSimpleSlot(new JobID(), DEFAULT_TESTING_PROFILE);
+		future.handleAsync(
+			new BiFunction<SimpleSlot, Throwable, Void>() {
+				@Override
+				public Void apply(SimpleSlot simpleSlot, Throwable throwable) {
+					assertNull(simpleSlot);
+					assertNotNull(throwable);
+					return null;
+				}
+			},
+			executor);
+		try {
+			future.get(1, TimeUnit.SECONDS);
+			fail("We expected a ExecutionException.");
+		} catch (ExecutionException ex) {
+			// we expect the exception
+		}
+	}
+
+	@Test
+	public void testAllocationFulfilledByReturnedSlot() throws Exception {
+		ResourceID resourceID = new ResourceID("resource");
+		slotPool.registerResource(resourceID);
+
+		JobID jobID = new JobID();
+
+		AllocationID allocationID1 = new AllocationID();
+		Future<SimpleSlot> future1 = slotPool.allocateSimpleSlot(jobID, DEFAULT_TESTING_PROFILE, allocationID1);
+
+		AllocationID allocationID2 = new AllocationID();
+		Future<SimpleSlot> future2 = slotPool.allocateSimpleSlot(jobID, DEFAULT_TESTING_PROFILE, allocationID2);
+
+		assertFalse(future1.isDone());
+		assertFalse(future2.isDone());
+		verify(resourceManagerGateway, times(2))
+			.requestSlot(any(UUID.class), any(UUID.class), any(SlotRequest.class), any(Time.class));
+
+		SlotDescriptor slotDescriptor = createSlotDescriptor(resourceID, jobID, DEFAULT_TESTING_PROFILE);
+		assertTrue(slotPool.offerSlot(allocationID1, slotDescriptor));
+
+		SimpleSlot slot1 = future1.get(1, TimeUnit.SECONDS);
+		assertTrue(future1.isDone());
+		assertFalse(future2.isDone());
+
+		// return this slot to pool
+		slot1.releaseSlot();
+
+		// second allocation fulfilled by previous slot returning
+		SimpleSlot slot2 = future2.get(1, TimeUnit.SECONDS);
+		assertTrue(future2.isDone());
+
+		assertNotEquals(slot1, slot2);
+		assertTrue(slot1.isReleased());
+		assertTrue(slot2.isAlive());
+		assertEquals(slot1.getTaskManagerID(), slot2.getTaskManagerID());
+		assertEquals(slot1.getSlotNumber(), slot2.getSlotNumber());
+	}
+
+	@Test
+	public void testAllocateWithFreeSlot() throws Exception {
+		ResourceID resourceID = new ResourceID("resource");
+		slotPool.registerResource(resourceID);
+
+		JobID jobID = new JobID();
+		AllocationID allocationID1 = new AllocationID();
+		Future<SimpleSlot> future1 = slotPool.allocateSimpleSlot(jobID, DEFAULT_TESTING_PROFILE, allocationID1);
+		assertFalse(future1.isDone());
+
+		SlotDescriptor slotDescriptor = createSlotDescriptor(resourceID, jobID, DEFAULT_TESTING_PROFILE);
+		assertTrue(slotPool.offerSlot(allocationID1, slotDescriptor));
+
+		SimpleSlot slot1 = future1.get(1, TimeUnit.SECONDS);
+		assertTrue(future1.isDone());
+
+		// return this slot to pool
+		slot1.releaseSlot();
+
+		AllocationID allocationID2 = new AllocationID();
+		Future<SimpleSlot> future2 = slotPool.allocateSimpleSlot(jobID, DEFAULT_TESTING_PROFILE, allocationID2);
+
+		// second allocation fulfilled by previous slot returning
+		SimpleSlot slot2 = future2.get(1, TimeUnit.SECONDS);
+		assertTrue(future2.isDone());
+
+		assertNotEquals(slot1, slot2);
+		assertTrue(slot1.isReleased());
+		assertTrue(slot2.isAlive());
+		assertEquals(slot1.getTaskManagerID(), slot2.getTaskManagerID());
+		assertEquals(slot1.getSlotNumber(), slot2.getSlotNumber());
+	}
+
+	@Test
+	public void testOfferSlot() throws Exception {
+		ResourceID resourceID = new ResourceID("resource");
+		slotPool.registerResource(resourceID);
+
+		JobID jobID = new JobID();
+		AllocationID allocationID = new AllocationID();
+		Future<SimpleSlot> future = slotPool.allocateSimpleSlot(jobID, DEFAULT_TESTING_PROFILE, allocationID);
+		assertFalse(future.isDone());
+		verify(resourceManagerGateway).requestSlot(any(UUID.class), any(UUID.class), any(SlotRequest.class), any(Time.class));
+
+		// slot from unregistered resource
+		SlotDescriptor invalid = createSlotDescriptor(new ResourceID("unregistered"), jobID, DEFAULT_TESTING_PROFILE);
+		assertFalse(slotPool.offerSlot(allocationID, invalid));
+
+		SlotDescriptor slotDescriptor = createSlotDescriptor(resourceID, jobID, DEFAULT_TESTING_PROFILE);
+
+		// reject offering with mismatch allocation id
+		assertFalse(slotPool.offerSlot(new AllocationID(), slotDescriptor));
+
+		// accepted slot
+		assertTrue(slotPool.offerSlot(allocationID, slotDescriptor));
+		SimpleSlot slot = future.get(1, TimeUnit.SECONDS);
+		assertTrue(future.isDone());
+		assertTrue(slot.isAlive());
+
+		// conflict offer with using slot
+		SlotDescriptor conflict = createSlotDescriptor(resourceID, jobID, DEFAULT_TESTING_PROFILE);
+		assertFalse(slotPool.offerSlot(allocationID, conflict));
+
+		// duplicated offer with using slot
+		assertTrue(slotPool.offerSlot(allocationID, slotDescriptor));
+		assertTrue(future.isDone());
+		assertTrue(slot.isAlive());
+
+		// duplicated offer with free slot
+		slot.releaseSlot();
+		assertTrue(slot.isReleased());
+		assertTrue(slotPool.offerSlot(allocationID, slotDescriptor));
+	}
+
+	@Test
+	public void testReleaseResource() throws Exception {
+		ResourceID resourceID = new ResourceID("resource");
+		slotPool.registerResource(resourceID);
+
+		JobID jobID = new JobID();
+
+		AllocationID allocationID1 = new AllocationID();
+		Future<SimpleSlot> future1 = slotPool.allocateSimpleSlot(jobID, DEFAULT_TESTING_PROFILE, allocationID1);
+
+		AllocationID allocationID2 = new AllocationID();
+		Future<SimpleSlot> future2 = slotPool.allocateSimpleSlot(jobID, DEFAULT_TESTING_PROFILE, allocationID2);
+
+		SlotDescriptor slotDescriptor = createSlotDescriptor(resourceID, jobID, DEFAULT_TESTING_PROFILE);
+		assertTrue(slotPool.offerSlot(allocationID1, slotDescriptor));
+
+		SimpleSlot slot1 = future1.get(1, TimeUnit.SECONDS);
+		assertTrue(future1.isDone());
+		assertFalse(future2.isDone());
+
+		slotPool.releaseResource(resourceID);
+		assertTrue(slot1.isReleased());
+
+		// slot released and not usable, second allocation still not fulfilled
+		Thread.sleep(10);
+		assertFalse(future2.isDone());
+	}
+
+}


[2/8] flink git commit: [FLINK-4375] [distributed coordination] Implement new JobManager creation, initialization, and basic RPC methods

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/34fef475/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 3b8fc97..d9ff88f 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
@@ -18,38 +18,31 @@
 
 package org.apache.flink.runtime.jobmaster;
 
-import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.api.common.restartstrategy.RestartStrategies;
 import org.apache.flink.api.common.time.Time;
-import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.core.io.InputSplit;
 import org.apache.flink.core.io.InputSplitAssigner;
 import org.apache.flink.metrics.MetricGroup;
 import org.apache.flink.metrics.groups.UnregisteredMetricsGroup;
 import org.apache.flink.runtime.checkpoint.CheckpointCoordinator;
-import org.apache.flink.runtime.checkpoint.CheckpointIDCounter;
 import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory;
-import org.apache.flink.runtime.checkpoint.CompletedCheckpointStore;
 import org.apache.flink.runtime.checkpoint.savepoint.SavepointStore;
-import org.apache.flink.runtime.checkpoint.stats.CheckpointStatsTracker;
-import org.apache.flink.runtime.checkpoint.stats.DisabledCheckpointStatsTracker;
-import org.apache.flink.runtime.checkpoint.stats.SimpleCheckpointStatsTracker;
 import org.apache.flink.runtime.client.JobExecutionException;
 import org.apache.flink.runtime.client.JobSubmissionException;
 import org.apache.flink.runtime.client.SerializedJobExecutionResult;
 import org.apache.flink.runtime.clusterframework.types.ResourceID;
-import org.apache.flink.runtime.concurrent.BiFunction;
 import org.apache.flink.runtime.concurrent.Future;
-import org.apache.flink.runtime.concurrent.impl.FlinkCompletableFuture;
 import org.apache.flink.runtime.concurrent.impl.FlinkFuture;
 import org.apache.flink.runtime.execution.ExecutionState;
 import org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager;
 import org.apache.flink.runtime.executiongraph.Execution;
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
 import org.apache.flink.runtime.executiongraph.ExecutionGraph;
+import org.apache.flink.runtime.executiongraph.ExecutionGraphBuilder;
 import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
+import org.apache.flink.runtime.executiongraph.JobStatusListener;
 import org.apache.flink.runtime.executiongraph.restart.RestartStrategy;
 import org.apache.flink.runtime.executiongraph.restart.RestartStrategyFactory;
 import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
@@ -59,18 +52,13 @@ import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
 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.JobVertex;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
-import org.apache.flink.runtime.jobgraph.jsonplan.JsonPlanGenerator;
-import org.apache.flink.runtime.jobgraph.tasks.JobSnapshottingSettings;
 import org.apache.flink.runtime.jobmanager.OnCompletionActions;
 import org.apache.flink.runtime.jobmanager.scheduler.Scheduler;
 import org.apache.flink.runtime.jobmaster.message.ClassloadingProps;
-import org.apache.flink.runtime.jobmaster.message.DisposeSavepointResponse;
-import org.apache.flink.runtime.jobmaster.message.NextInputSplit;
-import org.apache.flink.runtime.jobmaster.message.TriggerSavepointResponse;
 import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener;
 import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+import org.apache.flink.runtime.messages.Acknowledge;
 import org.apache.flink.runtime.messages.checkpoint.AcknowledgeCheckpoint;
 import org.apache.flink.runtime.messages.checkpoint.DeclineCheckpoint;
 import org.apache.flink.runtime.metrics.groups.JobManagerMetricGroup;
@@ -83,9 +71,12 @@ import org.apache.flink.runtime.registration.RegisteredRpcConnection;
 import org.apache.flink.runtime.registration.RegistrationResponse;
 import org.apache.flink.runtime.registration.RetryingRegistration;
 import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway;
+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.StartStoppable;
+import org.apache.flink.runtime.state.CheckpointStateHandles;
 import org.apache.flink.runtime.state.KeyGroupRange;
 import org.apache.flink.runtime.taskmanager.TaskExecutionState;
 import org.apache.flink.runtime.util.SerializedThrowable;
@@ -93,15 +84,13 @@ import org.apache.flink.util.InstantiationUtil;
 import org.apache.flink.util.SerializedValue;
 import org.slf4j.Logger;
 
-import scala.concurrent.ExecutionContext$;
-import scala.concurrent.duration.FiniteDuration;
-
+import javax.annotation.Nullable;
 import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
 import java.util.Map;
 import java.util.UUID;
 import java.util.concurrent.Executor;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
 
 import static org.apache.flink.util.Preconditions.checkNotNull;
 
@@ -112,16 +101,21 @@ import static org.apache.flink.util.Preconditions.checkNotNull;
  * It offers the following methods as part of its rpc interface to interact with the JobMaster
  * remotely:
  * <ul>
- * <li>{@link #updateTaskExecutionState(TaskExecutionState)} updates the task execution state for
+ * <li>{@link #updateTaskExecutionState} updates the task execution state for
  * given task</li>
  * </ul>
  */
 public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 
+	private static final AtomicReferenceFieldUpdater<JobMaster, UUID> LEADER_ID_UPDATER =
+			AtomicReferenceFieldUpdater.newUpdater(JobMaster.class, UUID.class, "leaderSessionID");
+
+	// ------------------------------------------------------------------------
+
 	/** Logical representation of the job */
 	private final JobGraph jobGraph;
 
-	/** Configuration of the job */
+	/** Configuration of the JobManager */
 	private final Configuration configuration;
 
 	/** Service to contend for and retrieve the leadership of JM and RM */
@@ -130,37 +124,24 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 	/** Blob cache manager used across jobs */
 	private final BlobLibraryCacheManager libraryCacheManager;
 
-	/** Factory to create restart strategy for this job */
-	private final RestartStrategyFactory restartStrategyFactory;
-
-	/** Store for save points */
-	private final SavepointStore savepointStore;
-
-	/** The timeout for this job */
-	private final Time timeout;
-
-	/** The scheduler to use for scheduling new tasks as they are needed */
-	private final Scheduler scheduler;
+	/** The metrics for the JobManager itself */
+	private final MetricGroup jobManagerMetricGroup;
 
-	/** The metrics group used across jobs */
-	private final JobManagerMetricGroup jobManagerMetricGroup;
+	/** The metrics for the job */
+	private final MetricGroup jobMetricGroup;
 
 	/** The execution context which is used to execute futures */
-	private final Executor executionContext;
+	private final ExecutorService executionContext;
 
 	private final OnCompletionActions jobCompletionActions;
 
-	/** The execution graph of this job */
-	private volatile ExecutionGraph executionGraph;
-
-	/** The checkpoint recovery factory used by this job */
-	private CheckpointRecoveryFactory checkpointRecoveryFactory;
+	private final FatalErrorHandler errorHandler;
 
-	private ClassLoader userCodeLoader;
+	private final ClassLoader userCodeLoader;
 
-	private RestartStrategy restartStrategy;
+	/** The execution graph of this job */
+	private final ExecutionGraph executionGraph;
 
-	private MetricGroup jobMetrics;
 
 	private volatile UUID leaderSessionID;
 
@@ -170,22 +151,27 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 	private LeaderRetrievalService resourceManagerLeaderRetriever;
 
 	/** Connection with ResourceManager, null if not located address yet or we close it initiative */
-	private volatile ResourceManagerConnection resourceManagerConnection;
+	private ResourceManagerConnection resourceManagerConnection;
+
+	// TODO - we need to replace this with the slot pool
+	private final Scheduler scheduler;
 
 	// ------------------------------------------------------------------------
 
 	public JobMaster(
-		JobGraph jobGraph,
-		Configuration configuration,
-		RpcService rpcService,
-		HighAvailabilityServices highAvailabilityService,
-		BlobLibraryCacheManager libraryCacheManager,
-		RestartStrategyFactory restartStrategyFactory,
-		SavepointStore savepointStore,
-		Time timeout,
-		Scheduler scheduler,
-		JobManagerMetricGroup jobManagerMetricGroup,
-		OnCompletionActions jobCompletionActions)
+			JobGraph jobGraph,
+			Configuration configuration,
+			RpcService rpcService,
+			HighAvailabilityServices highAvailabilityService,
+			ExecutorService executorService,
+			BlobLibraryCacheManager libraryCacheManager,
+			RestartStrategyFactory restartStrategyFactory,
+			SavepointStore savepointStore,
+			Time rpcAskTimeout,
+			@Nullable JobManagerMetricGroup jobManagerMetricGroup,
+			OnCompletionActions jobCompletionActions,
+			FatalErrorHandler errorHandler,
+			ClassLoader userCodeLoader) throws Exception
 	{
 		super(rpcService);
 
@@ -193,289 +179,150 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 		this.configuration = checkNotNull(configuration);
 		this.highAvailabilityServices = checkNotNull(highAvailabilityService);
 		this.libraryCacheManager = checkNotNull(libraryCacheManager);
-		this.restartStrategyFactory = checkNotNull(restartStrategyFactory);
-		this.savepointStore = checkNotNull(savepointStore);
-		this.timeout = checkNotNull(timeout);
-		this.scheduler = checkNotNull(scheduler);
-		this.jobManagerMetricGroup = checkNotNull(jobManagerMetricGroup);
-		this.executionContext = checkNotNull(rpcService.getExecutor());
+		this.executionContext = checkNotNull(executorService);
 		this.jobCompletionActions = checkNotNull(jobCompletionActions);
-	}
+		this.errorHandler = checkNotNull(errorHandler);
+		this.userCodeLoader = checkNotNull(userCodeLoader);
 
-	//----------------------------------------------------------------------------------------------
-	// Lifecycle management
-	//----------------------------------------------------------------------------------------------
-
-	/**
-	 * Initializing the job execution environment, should be called before start. Any error occurred during
-	 * initialization will be treated as job submission failure.
-	 *
-	 * @throws JobSubmissionException
-	 */
-	public void init() throws JobSubmissionException {
-		log.info("Initializing job {} ({}).", jobGraph.getName(), jobGraph.getJobID());
-
-		try {
-			// IMPORTANT: We need to make sure that the library registration is the first action,
-			// because this makes sure that the uploaded jar files are removed in case of
-			// unsuccessful
-			try {
-				libraryCacheManager.registerJob(jobGraph.getJobID(), jobGraph.getUserJarBlobKeys(),
-					jobGraph.getClasspaths());
-			} catch (Throwable t) {
-				throw new JobSubmissionException(jobGraph.getJobID(),
-					"Cannot set up the user code libraries: " + t.getMessage(), t);
-			}
+		final String jobName = jobGraph.getName();
+		final JobID jid = jobGraph.getJobID();
 
-			userCodeLoader = libraryCacheManager.getClassLoader(jobGraph.getJobID());
-			if (userCodeLoader == null) {
-				throw new JobSubmissionException(jobGraph.getJobID(),
-					"The user code class loader could not be initialized.");
-			}
+		if (jobManagerMetricGroup != null) {
+			this.jobManagerMetricGroup = jobManagerMetricGroup;
+			this.jobMetricGroup = jobManagerMetricGroup.addJob(jobGraph);
+		} else {
+			this.jobManagerMetricGroup = new UnregisteredMetricsGroup();
+			this.jobMetricGroup = new UnregisteredMetricsGroup();
+		}
 
-			if (jobGraph.getNumberOfVertices() == 0) {
-				throw new JobSubmissionException(jobGraph.getJobID(), "The given job is empty");
-			}
+		log.info("Initializing job {} ({}).", jobName, jid);
 
-			final RestartStrategies.RestartStrategyConfiguration restartStrategyConfiguration =
+		final RestartStrategies.RestartStrategyConfiguration restartStrategyConfiguration =
 				jobGraph.getSerializedExecutionConfig()
-					.deserializeValue(userCodeLoader)
-					.getRestartStrategy();
-			if (restartStrategyConfiguration != null) {
-				restartStrategy = RestartStrategyFactory.createRestartStrategy(restartStrategyConfiguration);
-			}
-			else {
-				restartStrategy = restartStrategyFactory.createRestartStrategy();
-			}
+						.deserializeValue(userCodeLoader)
+						.getRestartStrategy();
 
-			log.info("Using restart strategy {} for {} ({}).", restartStrategy, jobGraph.getName(), jobGraph.getJobID());
+		final RestartStrategy restartStrategy = (restartStrategyConfiguration != null) ?
+				RestartStrategyFactory.createRestartStrategy(restartStrategyConfiguration) :
+				restartStrategyFactory.createRestartStrategy();
 
-			if (jobManagerMetricGroup != null) {
-				jobMetrics = jobManagerMetricGroup.addJob(jobGraph);
-			}
-			if (jobMetrics == null) {
-				jobMetrics = new UnregisteredMetricsGroup();
-			}
+		log.info("Using restart strategy {} for {} ({}).", restartStrategy, jobName, jid);
 
-			try {
-				checkpointRecoveryFactory = highAvailabilityServices.getCheckpointRecoveryFactory();
-			} catch (Exception e) {
-				log.error("Could not get the checkpoint recovery factory.", e);
-				throw new JobSubmissionException(jobGraph.getJobID(), "Could not get the checkpoint recovery factory.", e);
-			}
+		CheckpointRecoveryFactory checkpointRecoveryFactory;
+		try {
+			checkpointRecoveryFactory = highAvailabilityServices.getCheckpointRecoveryFactory();
+		} catch (Exception e) {
+			log.error("Could not create the access to highly-available checkpoint storage.", e);
+			throw new Exception("Could not create the access to highly-available checkpoint storage.", e);
+		}
 
-			try {
-				resourceManagerLeaderRetriever = highAvailabilityServices.getResourceManagerLeaderRetriever();
-			} catch (Exception e) {
-				log.error("Could not get the resource manager leader retriever.", e);
-				throw new JobSubmissionException(jobGraph.getJobID(),
+		try {
+			resourceManagerLeaderRetriever = highAvailabilityServices.getResourceManagerLeaderRetriever();
+		} catch (Exception e) {
+			log.error("Could not get the resource manager leader retriever.", e);
+			throw new JobSubmissionException(jobGraph.getJobID(),
 					"Could not get the resource manager leader retriever.", e);
-			}
-		} catch (Throwable t) {
-			log.error("Failed to initializing job {} ({})", jobGraph.getName(), jobGraph.getJobID(), t);
+		}
 
-			libraryCacheManager.unregisterJob(jobGraph.getJobID());
+		this.executionGraph = ExecutionGraphBuilder.buildGraph(
+				null,
+				jobGraph,
+				configuration,
+				executorService,
+				userCodeLoader,
+				checkpointRecoveryFactory,
+				savepointStore,
+				rpcAskTimeout,
+				restartStrategy,
+				jobMetricGroup,
+				-1,
+				log);
 
-			if (t instanceof JobSubmissionException) {
-				throw (JobSubmissionException) t;
-			}
-			else {
-				throw new JobSubmissionException(jobGraph.getJobID(), "Failed to initialize job " +
-					jobGraph.getName() + " (" + jobGraph.getJobID() + ")", t);
-			}
-		}
+		// TODO - temp fix
+		this.scheduler = new Scheduler(executorService);
 	}
 
+	//----------------------------------------------------------------------------------------------
+	// Lifecycle management
+	//----------------------------------------------------------------------------------------------
+
+
 	@Override
 	public void start() {
-		super.start();
+		throw new UnsupportedOperationException("Should never call start() without leader ID");
 	}
 
+	/**
+	 * Start the rpc service and begin to run the job.
+	 *
+	 * @param leaderSessionID The necessary leader id for running the job.
+	 */
+	public void start(final UUID leaderSessionID) {
+		if (LEADER_ID_UPDATER.compareAndSet(this, null, leaderSessionID)) {
+			super.start();
+
+			log.info("Starting JobManager for job {} ({})", jobGraph.getName(), jobGraph.getJobID());
+			getSelf().startJobExecution();
+		} else {
+			log.warn("Job already started with leaderId {}, ignoring this start request.", leaderSessionID);
+		}
+	}
+
+	/**
+	 * Suspend the job and shutdown all other services including rpc.
+	 */
 	@Override
 	public void shutDown() {
+		// make sure there is a graceful exit
+		getSelf().suspendExecution(new Exception("JobManager is shutting down."));
 		super.shutDown();
-
-		suspendJob(new Exception("JobManager is shutting down."));
-
-		disposeCommunicationWithResourceManager();
 	}
 
-
-
 	//----------------------------------------------------------------------------------------------
 	// RPC methods
 	//----------------------------------------------------------------------------------------------
 
-	/**
-	 * Start to run the job, runtime data structures like ExecutionGraph will be constructed now and checkpoint
-	 * being recovered. After this, we will begin to schedule the job.
-	 */
+	//-- job starting and stopping  -----------------------------------------------------------------
+
 	@RpcMethod
-	public void startJob(final UUID leaderSessionID) {
-		log.info("Starting job {} ({}) with leaderId {}.", jobGraph.getName(), jobGraph.getJobID(), leaderSessionID);
-
-		this.leaderSessionID = leaderSessionID;
-
-		if (executionGraph != null) {
-			executionGraph = new ExecutionGraph(
-				ExecutionContext$.MODULE$.fromExecutor(executionContext),
-				jobGraph.getJobID(),
-				jobGraph.getName(),
-				jobGraph.getJobConfiguration(),
-				jobGraph.getSerializedExecutionConfig(),
-				new FiniteDuration(timeout.getSize(), timeout.getUnit()),
-				restartStrategy,
-				jobGraph.getUserJarBlobKeys(),
-				jobGraph.getClasspaths(),
-				userCodeLoader,
-				jobMetrics);
-		}
-		else {
-			// TODO: update last active time in JobInfo
-		}
+	public void startJobExecution() {
+		log.info("Starting execution of job {} ({}) with leaderId {}.",
+				jobGraph.getName(), jobGraph.getJobID(), leaderSessionID);
 
 		try {
-			executionGraph.setScheduleMode(jobGraph.getScheduleMode());
-			executionGraph.setQueuedSchedulingAllowed(jobGraph.getAllowQueuedScheduling());
-
-			try {
-				executionGraph.setJsonPlan(JsonPlanGenerator.generatePlan(jobGraph));
-			} catch (Exception e) {
-				log.warn("Cannot create JSON plan for job {} ({})", jobGraph.getJobID(), jobGraph.getName(), e);
-				executionGraph.setJsonPlan("{}");
-			}
-
-			// initialize the vertices that have a master initialization hook
-			// file output formats create directories here, input formats create splits
-			if (log.isDebugEnabled()) {
-				log.debug("Running initialization on master for job {} ({}).", jobGraph.getJobID(), jobGraph.getName());
-			}
-			for (JobVertex vertex : jobGraph.getVertices()) {
-				final String executableClass = vertex.getInvokableClassName();
-				if (executableClass == null || executableClass.length() == 0) {
-					throw new JobExecutionException(jobGraph.getJobID(),
-						"The vertex " + vertex.getID() + " (" + vertex.getName() + ") has no invokable class.");
-				}
-				if (vertex.getParallelism() == ExecutionConfig.PARALLELISM_AUTO_MAX) {
-					vertex.setParallelism(scheduler.getTotalNumberOfSlots());
-				}
-
-				try {
-					vertex.initializeOnMaster(userCodeLoader);
-				} catch (Throwable t) {
-					throw new JobExecutionException(jobGraph.getJobID(),
-						"Cannot initialize task '" + vertex.getName() + "': " + t.getMessage(), t);
-				}
-			}
-
-			// topologically sort the job vertices and attach the graph to the existing one
-			final List<JobVertex> sortedTopology = jobGraph.getVerticesSortedTopologicallyFromSources();
-			if (log.isDebugEnabled()) {
-				log.debug("Adding {} vertices from job graph {} ({}).", sortedTopology.size(),
-					jobGraph.getJobID(), jobGraph.getName());
-			}
-			executionGraph.attachJobGraph(sortedTopology);
-
-			if (log.isDebugEnabled()) {
-				log.debug("Successfully created execution graph from job graph {} ({}).",
-					jobGraph.getJobID(), jobGraph.getName());
-			}
-
-			final JobSnapshottingSettings snapshotSettings = jobGraph.getSnapshotSettings();
-			if (snapshotSettings != null) {
-				List<ExecutionJobVertex> triggerVertices = getExecutionJobVertexWithId(
-					executionGraph, snapshotSettings.getVerticesToTrigger());
-
-				List<ExecutionJobVertex> ackVertices = getExecutionJobVertexWithId(
-					executionGraph, snapshotSettings.getVerticesToAcknowledge());
-
-				List<ExecutionJobVertex> confirmVertices = getExecutionJobVertexWithId(
-					executionGraph, snapshotSettings.getVerticesToConfirm());
-
-				CompletedCheckpointStore completedCheckpoints = checkpointRecoveryFactory.createCheckpointStore(
-					jobGraph.getJobID(), userCodeLoader);
-
-				CheckpointIDCounter checkpointIdCounter = checkpointRecoveryFactory.createCheckpointIDCounter(
-					jobGraph.getJobID());
-
-				// Checkpoint stats tracker
-				boolean isStatsDisabled = configuration.getBoolean(
-					ConfigConstants.JOB_MANAGER_WEB_CHECKPOINTS_DISABLE,
-					ConfigConstants.DEFAULT_JOB_MANAGER_WEB_CHECKPOINTS_DISABLE);
-
-				final CheckpointStatsTracker checkpointStatsTracker;
-				if (isStatsDisabled) {
-					checkpointStatsTracker = new DisabledCheckpointStatsTracker();
-				}
-				else {
-					int historySize = configuration.getInteger(
-						ConfigConstants.JOB_MANAGER_WEB_CHECKPOINTS_HISTORY_SIZE,
-						ConfigConstants.DEFAULT_JOB_MANAGER_WEB_CHECKPOINTS_HISTORY_SIZE);
-					checkpointStatsTracker = new SimpleCheckpointStatsTracker(historySize, ackVertices, jobMetrics);
-				}
-
-				executionGraph.enableSnapshotCheckpointing(
-					snapshotSettings.getCheckpointInterval(),
-					snapshotSettings.getCheckpointTimeout(),
-					snapshotSettings.getMinPauseBetweenCheckpoints(),
-					snapshotSettings.getMaxConcurrentCheckpoints(),
-					triggerVertices,
-					ackVertices,
-					confirmVertices,
-					checkpointIdCounter,
-					completedCheckpoints,
-					savepointStore,
-					checkpointStatsTracker);
-			}
-
-			// TODO: register this class to execution graph as job status change listeners
-
-			// TODO: register client as job / execution status change listeners if they are interested
-
-			/*
-			TODO: decide whether we should take the savepoint before recovery
-
-			if (isRecovery) {
-				// this is a recovery of a master failure (this master takes over)
-				executionGraph.restoreLatestCheckpointedState();
-			} else {
-				if (snapshotSettings != null) {
-					String savepointPath = snapshotSettings.getSavepointPath();
-					if (savepointPath != null) {
-						// got a savepoint
-						log.info("Starting job from savepoint {}.", savepointPath);
-
-						// load the savepoint as a checkpoint into the system
-						final CompletedCheckpoint savepoint = SavepointLoader.loadAndValidateSavepoint(
-							jobGraph.getJobID(), executionGraph.getAllVertices(), savepointStore, savepointPath);
-						executionGraph.getCheckpointCoordinator().getCheckpointStore().addCheckpoint(savepoint);
-
-						// Reset the checkpoint ID counter
-						long nextCheckpointId = savepoint.getCheckpointID() + 1;
-						log.info("Reset the checkpoint ID to " + nextCheckpointId);
-						executionGraph.getCheckpointCoordinator().getCheckpointIdCounter().setCount(nextCheckpointId);
-
-						executionGraph.restoreLatestCheckpointedState();
-					}
+			// register self as job status change listener
+			executionGraph.registerJobStatusListener(new JobStatusListener() {
+				@Override
+				public void jobStatusChanges(
+						final JobID jobId, final JobStatus newJobStatus, final long timestamp, final Throwable error)
+				{
+					// run in rpc thread to avoid concurrency
+					runAsync(new Runnable() {
+						@Override
+						public void run() {
+							jobStatusChanged(newJobStatus, timestamp, error);
+						}
+					});
 				}
-			}
-			*/
+			});
 
-			// job is good to go, try to locate resource manager's address
+			// job is ready to go, try to establish connection with resource manager
 			resourceManagerLeaderRetriever.start(new ResourceManagerLeaderListener());
 		} catch (Throwable t) {
+
+			// TODO - this should not result in a job failure, but another leader should take over
+			// TODO - either this master should retry the execution, or it should relinquish leadership / terminate
+
 			log.error("Failed to start job {} ({})", jobGraph.getName(), jobGraph.getJobID(), t);
 
 			executionGraph.fail(t);
-			executionGraph = null;
 
-			final Throwable rt;
+			final JobExecutionException rt;
 			if (t instanceof JobExecutionException) {
 				rt = (JobExecutionException) t;
-			}
-			else {
+			} else {
 				rt = new JobExecutionException(jobGraph.getJobID(),
-					"Failed to start job " + jobGraph.getJobID() + " (" + jobGraph.getName() + ")", t);
+						"Failed to start job " + jobGraph.getJobID() + " (" + jobGraph.getName() + ")", t);
 			}
 
 			// TODO: notify client about this failure
@@ -488,34 +335,51 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 		executionContext.execute(new Runnable() {
 			@Override
 			public void run() {
-				if (executionGraph != null) {
-					try {
-						executionGraph.scheduleForExecution(scheduler);
-					} catch (Throwable t) {
-						executionGraph.fail(t);
-					}
+				try {
+					executionGraph.scheduleForExecution(scheduler);
+				} catch (Throwable t) {
+					executionGraph.fail(t);
 				}
 			}
 		});
 	}
 
 	/**
-	 * Suspending job, all the running tasks will be cancelled, and runtime data will be cleared.
+	 * Suspending job, all the running tasks will be cancelled, and communication with other components
+	 * will be disposed.
+	 *
+	 * <p>Mostly job is suspended because of the leadership has been revoked, one can be restart this job by
+	 * calling the {@link #start(UUID)} method once we take the leadership back again.
 	 *
 	 * @param cause The reason of why this job been suspended.
 	 */
 	@RpcMethod
-	public void suspendJob(final Throwable cause) {
+	public void suspendExecution(final Throwable cause) {
+		if (leaderSessionID == null) {
+			log.debug("Job has already been suspended or shutdown.");
+			return;
+		}
+
+		// receive no more messages until started again, should be called before we clear self leader id
+		((StartStoppable) getSelf()).stop();
+
 		leaderSessionID = null;
+		executionGraph.suspend(cause);
 
-		if (executionGraph != null) {
-			executionGraph.suspend(cause);
-			executionGraph = null;
+		// disconnect from resource manager:
+		try {
+			resourceManagerLeaderRetriever.stop();
+		} catch (Exception e) {
+			log.warn("Failed to stop resource manager leader retriever when suspending.");
 		}
+		closeResourceManagerConnection();
+
+		// TODO: disconnect from all registered task managers
 
-		disposeCommunicationWithResourceManager();
 	}
 
+	//----------------------------------------------------------------------------------------------
+
 	/**
 	 * Updates the task execution state for a given task.
 	 *
@@ -523,24 +387,38 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 	 * @return Acknowledge the task execution state update
 	 */
 	@RpcMethod
-	public Acknowledge updateTaskExecutionState(final TaskExecutionState taskExecutionState) throws ExecutionGraphException {
+	public Acknowledge updateTaskExecutionState(
+			final UUID leaderSessionID,
+			final TaskExecutionState taskExecutionState) throws Exception
+	{
 		if (taskExecutionState == null) {
 			throw new NullPointerException("TaskExecutionState must not be null.");
 		}
 
+		if (!this.leaderSessionID.equals(leaderSessionID)) {
+			throw new Exception("Leader id not match, expected: " + this.leaderSessionID
+					+ ", actual: " + leaderSessionID);
+		}
+
 		if (executionGraph.updateState(taskExecutionState)) {
 			return Acknowledge.get();
 		} else {
 			throw new ExecutionGraphException("The execution attempt " +
-				taskExecutionState.getID() + " was not found.");
+					taskExecutionState.getID() + " was not found.");
 		}
 	}
 
 	@RpcMethod
 	public SerializedInputSplit requestNextInputSplit(
-		final JobVertexID vertexID,
-		final ExecutionAttemptID executionAttempt) throws Exception
+			final UUID leaderSessionID,
+			final JobVertexID vertexID,
+			final ExecutionAttemptID executionAttempt) throws Exception
 	{
+		if (!this.leaderSessionID.equals(leaderSessionID)) {
+			throw new Exception("Leader id not match, expected: " + this.leaderSessionID
+					+ ", actual: " + leaderSessionID);
+		}
+
 		final Execution execution = executionGraph.getRegisteredExecutions().get(executionAttempt);
 		if (execution == null) {
 			// can happen when JobManager had already unregistered this execution upon on task failure,
@@ -579,7 +457,7 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 		} catch (Exception ex) {
 			log.error("Could not serialize the next input split of class {}.", nextInputSplit.getClass(), ex);
 			IOException reason = new IOException("Could not serialize the next input split of class " +
-				nextInputSplit.getClass() + ".", ex);
+					nextInputSplit.getClass() + ".", ex);
 			vertex.fail(reason);
 			throw reason;
 		}
@@ -587,17 +465,31 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 
 	@RpcMethod
 	public PartitionState requestPartitionState(
-		final ResultPartitionID partitionId,
-		final ExecutionAttemptID taskExecutionId,
-		final IntermediateDataSetID taskResultId)
+			final UUID leaderSessionID,
+			final ResultPartitionID partitionId,
+			final ExecutionAttemptID taskExecutionId,
+			final IntermediateDataSetID taskResultId) throws Exception
 	{
+		if (!this.leaderSessionID.equals(leaderSessionID)) {
+			throw new Exception("Leader id not match, expected: " + this.leaderSessionID
+					+ ", actual: " + leaderSessionID);
+		}
+
 		final Execution execution = executionGraph.getRegisteredExecutions().get(partitionId.getProducerId());
 		final ExecutionState state = execution != null ? execution.getState() : null;
 		return new PartitionState(taskResultId, partitionId.getPartitionId(), state);
 	}
 
 	@RpcMethod
-	public Acknowledge scheduleOrUpdateConsumers(final ResultPartitionID partitionID) {
+	public Acknowledge scheduleOrUpdateConsumers(
+			final UUID leaderSessionID,
+			final ResultPartitionID partitionID) throws Exception
+	{
+		if (!this.leaderSessionID.equals(leaderSessionID)) {
+			throw new Exception("Leader id not match, expected: " + this.leaderSessionID
+					+ ", actual: " + leaderSessionID);
+		}
+
 		executionGraph.scheduleOrUpdateConsumers(partitionID);
 		return Acknowledge.get();
 	}
@@ -609,223 +501,153 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 
 	@RpcMethod
 	public void acknowledgeCheckpoint(
-		JobID jobID,
-		ExecutionAttemptID executionAttemptID,
-		long checkpointID,
-		CheckpointStateHandles checkpointStateHandles,
-		long synchronousDurationMillis,
-		long asynchronousDurationMillis,
-		long bytesBufferedInAlignment,
-		long alignmentDurationNanos) {
-		throw new UnsupportedOperationException();
-	}
-
-	@RpcMethod
-	public void declineCheckpoint(
-		JobID jobID,
-		ExecutionAttemptID executionAttemptID,
-		long checkpointID,
-		long checkpointTimestamp) {
-		throw new UnsupportedOperationException();
-	}
-
-	@RpcMethod
-	public void resourceRemoved(final ResourceID resourceId, final String message) {
-		// TODO: remove resource from slot pool
-	}
-
-	@RpcMethod
-	public void acknowledgeCheckpoint(final AcknowledgeCheckpoint acknowledge) {
-		if (executionGraph != null) {
-			final CheckpointCoordinator checkpointCoordinator = executionGraph.getCheckpointCoordinator();
-			if (checkpointCoordinator != null) {
-				getRpcService().execute(new Runnable() {
-					@Override
-					public void run() {
-						try {
-							if (!checkpointCoordinator.receiveAcknowledgeMessage(acknowledge)) {
-								log.info("Received message for non-existing checkpoint {}.",
+			final JobID jobID,
+			final ExecutionAttemptID executionAttemptID,
+			final long checkpointID,
+			final CheckpointStateHandles checkpointStateHandles,
+			final long synchronousDurationMillis,
+			final long asynchronousDurationMillis,
+			final long bytesBufferedInAlignment,
+			final long alignmentDurationNanos)
+	{
+		final AcknowledgeCheckpoint acknowledge = new AcknowledgeCheckpoint(
+				jobID,
+				executionAttemptID,
+				checkpointID,
+				checkpointStateHandles,
+				synchronousDurationMillis,
+				asynchronousDurationMillis,
+				bytesBufferedInAlignment,
+				alignmentDurationNanos);
+		final CheckpointCoordinator checkpointCoordinator = executionGraph.getCheckpointCoordinator();
+
+		if (checkpointCoordinator != null) {
+			getRpcService().execute(new Runnable() {
+				@Override
+				public void run() {
+					try {
+						if (!checkpointCoordinator.receiveAcknowledgeMessage(acknowledge)) {
+							log.info("Received message for non-existing checkpoint {}.",
 									acknowledge.getCheckpointId());
-							}
-						} catch (Exception e) {
-							log.error("Error in CheckpointCoordinator while processing {}", acknowledge, e);
 						}
+					} catch (Exception e) {
+						log.error("Error in CheckpointCoordinator while processing {}", acknowledge, e);
 					}
-				});
-			}
-			else {
-				log.error("Received AcknowledgeCheckpoint message for job {} with no CheckpointCoordinator",
-					jobGraph.getJobID());
-			}
+				}
+			});
 		} else {
-			log.error("Received AcknowledgeCheckpoint for unavailable job {}", jobGraph.getJobID());
+			log.error("Received AcknowledgeCheckpoint message for job {} with no CheckpointCoordinator",
+					jobGraph.getJobID());
 		}
 	}
 
 	@RpcMethod
-	public void declineCheckpoint(final DeclineCheckpoint decline) {
-		if (executionGraph != null) {
-			final CheckpointCoordinator checkpointCoordinator = executionGraph.getCheckpointCoordinator();
-			if (checkpointCoordinator != null) {
-				getRpcService().execute(new Runnable() {
-					@Override
-					public void run() {
-						try {
-							if (!checkpointCoordinator.receiveDeclineMessage(decline)) {
-								log.info("Received message for non-existing checkpoint {}.", decline.getCheckpointId());
-							}
-						} catch (Exception e) {
-							log.error("Error in CheckpointCoordinator while processing {}", decline, e);
+	public void declineCheckpoint(
+			final JobID jobID,
+			final ExecutionAttemptID executionAttemptID,
+			final long checkpointID,
+			final long checkpointTimestamp)
+	{
+		final DeclineCheckpoint decline = new DeclineCheckpoint(
+				jobID, executionAttemptID, checkpointID, checkpointTimestamp);
+		final CheckpointCoordinator checkpointCoordinator = executionGraph.getCheckpointCoordinator();
+
+		if (checkpointCoordinator != null) {
+			getRpcService().execute(new Runnable() {
+				@Override
+				public void run() {
+					try {
+						if (!checkpointCoordinator.receiveDeclineMessage(decline)) {
+							log.info("Received message for non-existing checkpoint {}.", decline.getCheckpointId());
 						}
+					} catch (Exception e) {
+						log.error("Error in CheckpointCoordinator while processing {}", decline, e);
 					}
-				});
-			} else {
-				log.error("Received DeclineCheckpoint message for job {} with no CheckpointCoordinator",
-					jobGraph.getJobID());
-			}
+				}
+			});
 		} else {
-			log.error("Received AcknowledgeCheckpoint for unavailable job {}", jobGraph.getJobID());
+			log.error("Received DeclineCheckpoint message for job {} with no CheckpointCoordinator",
+					jobGraph.getJobID());
 		}
 	}
 
 	@RpcMethod
 	public KvStateLocation lookupKvStateLocation(final String registrationName) throws Exception {
-		if (executionGraph != null) {
-			if (log.isDebugEnabled()) {
-				log.debug("Lookup key-value state for job {} with registration " +
+		if (log.isDebugEnabled()) {
+			log.debug("Lookup key-value state for job {} with registration " +
 					"name {}.", jobGraph.getJobID(), registrationName);
-			}
+		}
 
-			final KvStateLocationRegistry registry = executionGraph.getKvStateLocationRegistry();
-			final KvStateLocation location = registry.getKvStateLocation(registrationName);
-			if (location != null) {
-				return location;
-			} else {
-				throw new UnknownKvStateLocation(registrationName);
-			}
+		final KvStateLocationRegistry registry = executionGraph.getKvStateLocationRegistry();
+		final KvStateLocation location = registry.getKvStateLocation(registrationName);
+		if (location != null) {
+			return location;
 		} else {
-			throw new IllegalStateException("Received lookup KvState location request for unavailable job " +
-				jobGraph.getJobID());
+			throw new UnknownKvStateLocation(registrationName);
 		}
 	}
 
 	@RpcMethod
 	public void notifyKvStateRegistered(
-		final JobVertexID jobVertexId,
-		final KeyGroupRange keyGroupRange,
-		final String registrationName,
-		final KvStateID kvStateId,
-		final KvStateServerAddress kvStateServerAddress)
+			final JobVertexID jobVertexId,
+			final KeyGroupRange keyGroupRange,
+			final String registrationName,
+			final KvStateID kvStateId,
+			final KvStateServerAddress kvStateServerAddress)
 	{
-		if (executionGraph != null) {
-			if (log.isDebugEnabled()) {
-				log.debug("Key value state registered for job {} under name {}.",
+		if (log.isDebugEnabled()) {
+			log.debug("Key value state registered for job {} under name {}.",
 					jobGraph.getJobID(), registrationName);
-			}
-			try {
-				executionGraph.getKvStateLocationRegistry().notifyKvStateRegistered(
-					jobVertexId, keyGroupRange, registrationName, kvStateId, kvStateServerAddress
-				);
-			} catch (Exception e) {
-				log.error("Failed to notify KvStateRegistry about registration {}.", registrationName);
-			}
-		} else {
-			log.error("Received notify KvState registered request for unavailable job " + jobGraph.getJobID());
+		}
+
+		try {
+			executionGraph.getKvStateLocationRegistry().notifyKvStateRegistered(
+					jobVertexId, keyGroupRange, registrationName, kvStateId, kvStateServerAddress);
+		} catch (Exception e) {
+			log.error("Failed to notify KvStateRegistry about registration {}.", registrationName);
 		}
 	}
 
 	@RpcMethod
 	public void notifyKvStateUnregistered(
-		JobVertexID jobVertexId,
-		KeyGroupRange keyGroupRange,
-		String registrationName)
+			JobVertexID jobVertexId,
+			KeyGroupRange keyGroupRange,
+			String registrationName)
 	{
-		if (executionGraph != null) {
-			if (log.isDebugEnabled()) {
-				log.debug("Key value state unregistered for job {} under name {}.",
+		if (log.isDebugEnabled()) {
+			log.debug("Key value state unregistered for job {} under name {}.",
 					jobGraph.getJobID(), registrationName);
-			}
-			try {
-				executionGraph.getKvStateLocationRegistry().notifyKvStateUnregistered(
-					jobVertexId, keyGroupRange, registrationName
-				);
-			} catch (Exception e) {
-				log.error("Failed to notify KvStateRegistry about registration {}.", registrationName);
-			}
-		} else {
-			log.error("Received notify KvState unregistered request for unavailable job " + jobGraph.getJobID());
 		}
-	}
 
-	@RpcMethod
-	public Future<TriggerSavepointResponse> triggerSavepoint() throws Exception {
-		if (executionGraph != null) {
-			final CheckpointCoordinator checkpointCoordinator = executionGraph.getCheckpointCoordinator();
-			if (checkpointCoordinator != null) {
-				try {
-					Future<String> savepointFuture = new FlinkFuture<>(
-						checkpointCoordinator.triggerSavepoint(System.currentTimeMillis()));
-
-					return savepointFuture.handleAsync(new BiFunction<String, Throwable, TriggerSavepointResponse>() {
-						@Override
-						public TriggerSavepointResponse apply(String savepointPath, Throwable throwable) {
-							if (throwable == null) {
-								return new TriggerSavepointResponse.Success(jobGraph.getJobID(), savepointPath);
-							}
-							else {
-								return new TriggerSavepointResponse.Failure(jobGraph.getJobID(),
-									new Exception("Failed to complete savepoint", throwable));
-							}
-						}
-					}, getMainThreadExecutor());
-
-				} catch (Exception e) {
-					FlinkCompletableFuture<TriggerSavepointResponse> future = new FlinkCompletableFuture<>();
-					future.complete(new TriggerSavepointResponse.Failure(jobGraph.getJobID(),
-						new Exception("Failed to trigger savepoint", e)));
-					return future;
-				}
-			} else {
-				FlinkCompletableFuture<TriggerSavepointResponse> future = new FlinkCompletableFuture<>();
-				future.complete(new TriggerSavepointResponse.Failure(jobGraph.getJobID(),
-					new IllegalStateException("Checkpointing disabled. You can enable it via the execution " +
-						"environment of your job.")));
-				return future;
-			}
-		} else {
-			FlinkCompletableFuture<TriggerSavepointResponse> future = new FlinkCompletableFuture<>();
-			future.complete(new TriggerSavepointResponse.Failure(jobGraph.getJobID(),
-				new IllegalArgumentException("Received trigger savepoint request for unavailable job " +
-					jobGraph.getJobID())));
-			return future;
+		try {
+			executionGraph.getKvStateLocationRegistry().notifyKvStateUnregistered(
+					jobVertexId, keyGroupRange, registrationName);
+		} catch (Exception e) {
+			log.error("Failed to notify KvStateRegistry about registration {}.", registrationName);
 		}
 	}
 
 	@RpcMethod
-	public DisposeSavepointResponse disposeSavepoint(final String savepointPath) {
-		try {
-			log.info("Disposing savepoint at {}.", savepointPath);
-
-			// check whether the savepoint exists
-			savepointStore.loadSavepoint(savepointPath);
+	public Future<String> triggerSavepoint(final UUID leaderSessionID) throws Exception {
+		if (!this.leaderSessionID.equals(leaderSessionID)) {
+			throw new Exception("Leader id not match, expected: " + this.leaderSessionID
+					+ ", actual: " + leaderSessionID);
+		}
 
-			savepointStore.disposeSavepoint(savepointPath);
-			return new DisposeSavepointResponse.Success();
-		} catch (Exception e) {
-			log.error("Failed to dispose savepoint at {}.", savepointPath, e);
-			return new DisposeSavepointResponse.Failure(e);
+		final CheckpointCoordinator checkpointCoordinator = executionGraph.getCheckpointCoordinator();
+		if (checkpointCoordinator != null) {
+			return new FlinkFuture<>(checkpointCoordinator.triggerSavepoint(System.currentTimeMillis()));
+		} else {
+			throw new IllegalStateException("Checkpointing disabled. You can enable it via the execution " +
+					"environment of your job.");
 		}
 	}
 
 	@RpcMethod
 	public ClassloadingProps requestClassloadingProps() throws Exception {
-		if (executionGraph != null) {
-			return new ClassloadingProps(libraryCacheManager.getBlobServerPort(),
+		return new ClassloadingProps(libraryCacheManager.getBlobServerPort(),
 				executionGraph.getRequiredJarFiles(),
 				executionGraph.getRequiredClasspaths());
-		} else {
-			throw new Exception("Received classloading props request for unavailable job " + jobGraph.getJobID());
-		}
 	}
 
 	//----------------------------------------------------------------------------------------------
@@ -838,12 +660,11 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 			public void run() {
 				log.error("Fatal error occurred on JobManager, cause: {}", cause.getMessage(), cause);
 				shutDown();
-				jobCompletionActions.onFatalError(cause);
+				errorHandler.onFatalError(cause);
 			}
 		});
 	}
 
-	// TODO - wrap this as StatusListenerMessenger's callback with rpc main thread
 	private void jobStatusChanged(final JobStatus newJobStatus, long timestamp, final Throwable error) {
 		final JobID jobID = executionGraph.getJobID();
 		final String jobName = executionGraph.getJobName();
@@ -871,36 +692,33 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 			if (newJobStatus == JobStatus.FINISHED) {
 				try {
 					final Map<String, SerializedValue<Object>> accumulatorResults =
-						executionGraph.getAccumulatorsSerialized();
+							executionGraph.getAccumulatorsSerialized();
 					final SerializedJobExecutionResult result = new SerializedJobExecutionResult(
-						jobID, 0, accumulatorResults // TODO get correct job duration
+							jobID, 0, accumulatorResults // TODO get correct job duration
 					);
 					jobCompletionActions.jobFinished(result.toJobExecutionResult(userCodeLoader));
 				} catch (Exception e) {
 					log.error("Cannot fetch final accumulators for job {} ({})", jobName, jobID, e);
 					final JobExecutionException exception = new JobExecutionException(
-						jobID, "Failed to retrieve accumulator results.", e);
+							jobID, "Failed to retrieve accumulator results.", e);
 					// TODO should we also notify client?
 					jobCompletionActions.jobFailed(exception);
 				}
-			}
-			else if (newJobStatus == JobStatus.CANCELED) {
+			} else if (newJobStatus == JobStatus.CANCELED) {
 				final Throwable unpackedError = SerializedThrowable.get(error, userCodeLoader);
 				final JobExecutionException exception = new JobExecutionException(
-					jobID, "Job was cancelled.", unpackedError);
+						jobID, "Job was cancelled.", unpackedError);
 				// TODO should we also notify client?
 				jobCompletionActions.jobFailed(exception);
-			}
-			else if (newJobStatus == JobStatus.FAILED) {
+			} else if (newJobStatus == JobStatus.FAILED) {
 				final Throwable unpackedError = SerializedThrowable.get(error, userCodeLoader);
 				final JobExecutionException exception = new JobExecutionException(
-					jobID, "Job execution failed.", unpackedError);
+						jobID, "Job execution failed.", unpackedError);
 				// TODO should we also notify client?
 				jobCompletionActions.jobFailed(exception);
-			}
-			else {
+			} else {
 				final JobExecutionException exception = new JobExecutionException(
-					jobID, newJobStatus + " is not a terminal state.");
+						jobID, newJobStatus + " is not a terminal state.");
 				// TODO should we also notify client?
 				jobCompletionActions.jobFailed(exception);
 				throw new RuntimeException(exception);
@@ -909,7 +727,7 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 	}
 
 	private void notifyOfNewResourceManagerLeader(
-		final String resourceManagerAddress, final UUID resourceManagerLeaderId)
+			final String resourceManagerAddress, final UUID resourceManagerLeaderId)
 	{
 		// IMPORTANT: executed by main thread to avoid concurrence
 		runAsync(new Runnable() {
@@ -918,17 +736,15 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 				if (resourceManagerConnection != null) {
 					if (resourceManagerAddress != null) {
 						if (resourceManagerAddress.equals(resourceManagerConnection.getTargetAddress())
-							&& resourceManagerLeaderId.equals(resourceManagerConnection.getTargetLeaderId()))
-						{
+								&& resourceManagerLeaderId.equals(resourceManagerConnection.getTargetLeaderId())) {
 							// both address and leader id are not changed, we can keep the old connection
 							return;
 						}
 						log.info("ResourceManager leader changed from {} to {}. Registering at new leader.",
-							resourceManagerConnection.getTargetAddress(), resourceManagerAddress);
-					}
-					else {
+								resourceManagerConnection.getTargetAddress(), resourceManagerAddress);
+					} else {
 						log.info("Current ResourceManager {} lost leader status. Waiting for new ResourceManager leader.",
-							resourceManagerConnection.getTargetAddress());
+								resourceManagerConnection.getTargetAddress());
 					}
 				}
 
@@ -937,8 +753,8 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 				if (resourceManagerAddress != null) {
 					log.info("Attempting to register at ResourceManager {}", resourceManagerAddress);
 					resourceManagerConnection = new ResourceManagerConnection(
-						log, jobGraph.getJobID(), leaderSessionID,
-						resourceManagerAddress, resourceManagerLeaderId, executionContext);
+							log, jobGraph.getJobID(), leaderSessionID,
+							resourceManagerAddress, resourceManagerLeaderId, executionContext);
 					resourceManagerConnection.start();
 				}
 			}
@@ -952,26 +768,14 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 				// TODO - add tests for comment in https://github.com/apache/flink/pull/2565
 				// verify the response with current connection
 				if (resourceManagerConnection != null
-					&& resourceManagerConnection.getTargetLeaderId().equals(success.getResourceManagerLeaderId())) {
+						&& resourceManagerConnection.getTargetLeaderId().equals(success.getResourceManagerLeaderId())) {
 					log.info("JobManager successfully registered at ResourceManager, leader id: {}.",
-						success.getResourceManagerLeaderId());
+							success.getResourceManagerLeaderId());
 				}
 			}
 		});
 	}
 
-	private void disposeCommunicationWithResourceManager() {
-		// 1. stop the leader retriever so we will not receiving updates anymore
-		try {
-			resourceManagerLeaderRetriever.stop();
-		} catch (Exception e) {
-			log.warn("Failed to stop resource manager leader retriever.");
-		}
-
-		// 2. close current connection with ResourceManager if exists
-		closeResourceManagerConnection();
-	}
-
 	private void closeResourceManagerConnection() {
 		if (resourceManagerConnection != null) {
 			resourceManagerConnection.close();
@@ -980,34 +784,6 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 	}
 
 	//----------------------------------------------------------------------------------------------
-	// Helper methods
-	//----------------------------------------------------------------------------------------------
-
-	/**
-	 * Converts JobVertexIDs to corresponding ExecutionJobVertexes
-	 *
-	 * @param executionGraph The execution graph that holds the relationship
-	 * @param vertexIDs      The vertexIDs need to be converted
-	 * @return The corresponding ExecutionJobVertexes
-	 * @throws JobExecutionException
-	 */
-	private static List<ExecutionJobVertex> getExecutionJobVertexWithId(
-		final ExecutionGraph executionGraph, final List<JobVertexID> vertexIDs)
-		throws JobExecutionException
-	{
-		final List<ExecutionJobVertex> ret = new ArrayList<>(vertexIDs.size());
-		for (JobVertexID vertexID : vertexIDs) {
-			final ExecutionJobVertex executionJobVertex = executionGraph.getJobVertex(vertexID);
-			if (executionJobVertex == null) {
-				throw new JobExecutionException(executionGraph.getJobID(),
-					"The snapshot checkpointing settings refer to non-existent vertex " + vertexID);
-			}
-			ret.add(executionJobVertex);
-		}
-		return ret;
-	}
-
-	//----------------------------------------------------------------------------------------------
 	// Utility classes
 	//----------------------------------------------------------------------------------------------
 
@@ -1024,19 +800,19 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 	}
 
 	private class ResourceManagerConnection
-		extends RegisteredRpcConnection<ResourceManagerGateway, JobMasterRegistrationSuccess>
+			extends RegisteredRpcConnection<ResourceManagerGateway, JobMasterRegistrationSuccess>
 	{
 		private final JobID jobID;
 
 		private final UUID jobManagerLeaderID;
 
 		ResourceManagerConnection(
-			final Logger log,
-			final JobID jobID,
-			final UUID jobManagerLeaderID,
-			final String resourceManagerAddress,
-			final UUID resourceManagerLeaderID,
-			final Executor executor)
+				final Logger log,
+				final JobID jobID,
+				final UUID jobManagerLeaderID,
+				final String resourceManagerAddress,
+				final UUID resourceManagerLeaderID,
+				final Executor executor)
 		{
 			super(log, resourceManagerAddress, resourceManagerLeaderID, executor);
 			this.jobID = checkNotNull(jobID);
@@ -1046,12 +822,12 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 		@Override
 		protected RetryingRegistration<ResourceManagerGateway, JobMasterRegistrationSuccess> generateRegistration() {
 			return new RetryingRegistration<ResourceManagerGateway, JobMasterRegistrationSuccess>(
-				log, getRpcService(), "ResourceManager", ResourceManagerGateway.class,
-				getTargetAddress(), getTargetLeaderId())
+					log, getRpcService(), "ResourceManager", ResourceManagerGateway.class,
+					getTargetAddress(), getTargetLeaderId())
 			{
 				@Override
 				protected Future<RegistrationResponse> invokeRegistration(ResourceManagerGateway gateway, UUID leaderId,
-					long timeoutMillis) throws Exception
+						long timeoutMillis) throws Exception
 				{
 					Time timeout = Time.milliseconds(timeoutMillis);
 					return gateway.registerJobMaster(leaderId, jobManagerLeaderID, getAddress(), jobID, timeout);

http://git-wip-us.apache.org/repos/asf/flink/blob/34fef475/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterGateway.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterGateway.java
index 4b51258..fbe3f74 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterGateway.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterGateway.java
@@ -18,6 +18,8 @@
 
 package org.apache.flink.runtime.jobmaster;
 
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.checkpoint.CheckpointCoordinatorGateway;
 import org.apache.flink.runtime.clusterframework.types.ResourceID;
 import org.apache.flink.runtime.concurrent.Future;
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
@@ -29,15 +31,11 @@ import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
 import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.jobmaster.message.ClassloadingProps;
-import org.apache.flink.runtime.jobmaster.message.DisposeSavepointResponse;
-import org.apache.flink.runtime.jobmaster.message.NextInputSplit;
-import org.apache.flink.runtime.jobmaster.message.TriggerSavepointResponse;
-import org.apache.flink.runtime.messages.checkpoint.AcknowledgeCheckpoint;
-import org.apache.flink.runtime.messages.checkpoint.DeclineCheckpoint;
+import org.apache.flink.runtime.messages.Acknowledge;
 import org.apache.flink.runtime.query.KvStateID;
 import org.apache.flink.runtime.query.KvStateLocation;
 import org.apache.flink.runtime.query.KvStateServerAddress;
-import org.apache.flink.runtime.rpc.RpcGateway;
+import org.apache.flink.runtime.rpc.RpcTimeout;
 import org.apache.flink.runtime.state.KeyGroupRange;
 import org.apache.flink.runtime.state.KvState;
 import org.apache.flink.runtime.taskmanager.TaskExecutionState;
@@ -49,52 +47,56 @@ import java.util.UUID;
  */
 public interface JobMasterGateway extends CheckpointCoordinatorGateway {
 
-	/**
-	 * Starting the job under the given leader session ID.
-	 */
-	void startJob(final UUID leaderSessionID);
+	// ------------------------------------------------------------------------
+	//  Job start and stop methods
+	// ------------------------------------------------------------------------
 
-	/**
-	 * Suspending job, all the running tasks will be cancelled, and runtime status will be cleared.
-	 * Should re-submit the job before restarting it.
-	 *
-	 * @param cause The reason of why this job been suspended.
-	 */
-	void suspendJob(final Throwable cause);
+	void startJobExecution();
+
+	void suspendExecution(Throwable cause);
+
+	// ------------------------------------------------------------------------
 
 	/**
 	 * Updates the task execution state for a given task.
 	 *
+	 * @param leaderSessionID    The leader id of JobManager
 	 * @param taskExecutionState New task execution state for a given task
 	 * @return Future flag of the task execution state update result
 	 */
-	Future<Acknowledge> updateTaskExecutionState(TaskExecutionState taskExecutionState);
+	Future<Acknowledge> updateTaskExecutionState(
+			final UUID leaderSessionID,
+			final TaskExecutionState taskExecutionState);
 
 	/**
 	 * Requesting next input split for the {@link ExecutionJobVertex}. The next input split is sent back to the sender
 	 * as a {@link SerializedInputSplit} message.
 	 *
+	 * @param leaderSessionID  The leader id of JobManager
 	 * @param vertexID         The job vertex id
 	 * @param executionAttempt The execution attempt id
 	 * @return The future of the input split. If there is no further input split, will return an empty object.
 	 */
 	Future<SerializedInputSplit> requestNextInputSplit(
-		final JobVertexID vertexID,
-		final ExecutionAttemptID executionAttempt);
+			final UUID leaderSessionID,
+			final JobVertexID vertexID,
+			final ExecutionAttemptID executionAttempt);
 
 	/**
 	 * Requests the current state of the partition.
 	 * The state of a partition is currently bound to the state of the producing execution.
 	 *
+	 * @param leaderSessionID The leader id of JobManager
 	 * @param partitionId     The partition ID of the partition to request the state of.
 	 * @param taskExecutionId The execution attempt ID of the task requesting the partition state.
 	 * @param taskResultId    The input gate ID of the task requesting the partition state.
 	 * @return The future of the partition state
 	 */
 	Future<PartitionState> requestPartitionState(
-		final ResultPartitionID partitionId,
-		final ExecutionAttemptID taskExecutionId,
-		final IntermediateDataSetID taskResultId);
+			final UUID leaderSessionID,
+			final ResultPartitionID partitionId,
+			final ExecutionAttemptID taskExecutionId,
+			final IntermediateDataSetID taskResultId);
 
 	/**
 	 * Notifies the JobManager about available data for a produced partition.
@@ -105,11 +107,15 @@ public interface JobMasterGateway extends CheckpointCoordinatorGateway {
 	 * <p>
 	 * The JobManager then can decide when to schedule the partition consumers of the given session.
 	 *
-	 * @param partitionID The partition which has already produced data
-	 * @param timeout before the rpc call fails
+	 * @param leaderSessionID The leader id of JobManager
+	 * @param partitionID     The partition which has already produced data
+	 * @param timeout         before the rpc call fails
 	 * @return Future acknowledge of the schedule or update operation
 	 */
-	Future<Acknowledge> scheduleOrUpdateConsumers(final ResultPartitionID partitionID, @RpcTimeout final Time timeout);
+	Future<Acknowledge> scheduleOrUpdateConsumers(
+			final UUID leaderSessionID,
+			final ResultPartitionID partitionID,
+			@RpcTimeout final Time timeout);
 
 	/**
 	 * Disconnects the given {@link org.apache.flink.runtime.taskexecutor.TaskExecutor} from the
@@ -118,31 +124,6 @@ public interface JobMasterGateway extends CheckpointCoordinatorGateway {
 	 * @param resourceID identifying the TaskManager to disconnect
 	 */
 	void disconnectTaskManager(ResourceID resourceID);
-	void scheduleOrUpdateConsumers(final ResultPartitionID partitionID);
-
-	/**
-	 * Notifies the JobManager about the removal of a resource.
-	 *
-	 * @param resourceId The ID under which the resource is registered.
-	 * @param message    Optional message with details, for logging and debugging.
-	 */
-
-	void resourceRemoved(final ResourceID resourceId, final String message);
-
-	/**
-	 * Notifies the JobManager that the checkpoint of an individual task is completed.
-	 *
-	 * @param acknowledge The acknowledge message of the checkpoint
-	 */
-	void acknowledgeCheckpoint(final AcknowledgeCheckpoint acknowledge);
-
-	/**
-	 * Notifies the JobManager that a checkpoint request could not be heeded.
-	 * This can happen if a Task is already in RUNNING state but is internally not yet ready to perform checkpoints.
-	 *
-	 * @param decline The decline message of the checkpoint
-	 */
-	void declineCheckpoint(final DeclineCheckpoint decline);
 
 	/**
 	 * Requests a {@link KvStateLocation} for the specified {@link KvState} registration name.
@@ -150,7 +131,7 @@ public interface JobMasterGateway extends CheckpointCoordinatorGateway {
 	 * @param registrationName Name under which the KvState has been registered.
 	 * @return Future of the requested {@link KvState} location
 	 */
-	Future<KvStateLocation> lookupKvStateLocation(final String registrationName) throws Exception;
+	Future<KvStateLocation> lookupKvStateLocation(final String registrationName);
 
 	/**
 	 * @param jobVertexId          JobVertexID the KvState instance belongs to.
@@ -160,11 +141,11 @@ public interface JobMasterGateway extends CheckpointCoordinatorGateway {
 	 * @param kvStateServerAddress Server address where to find the KvState instance.
 	 */
 	void notifyKvStateRegistered(
-		final JobVertexID jobVertexId,
-		final KeyGroupRange keyGroupRange,
-		final String registrationName,
-		final KvStateID kvStateId,
-		final KvStateServerAddress kvStateServerAddress);
+			final JobVertexID jobVertexId,
+			final KeyGroupRange keyGroupRange,
+			final String registrationName,
+			final KvStateID kvStateId,
+			final KvStateServerAddress kvStateServerAddress);
 
 	/**
 	 * @param jobVertexId      JobVertexID the KvState instance belongs to.
@@ -172,24 +153,17 @@ public interface JobMasterGateway extends CheckpointCoordinatorGateway {
 	 * @param registrationName Name under which the KvState has been registered.
 	 */
 	void notifyKvStateUnregistered(
-		JobVertexID jobVertexId,
-		KeyGroupRange keyGroupRange,
-		String registrationName);
+			JobVertexID jobVertexId,
+			KeyGroupRange keyGroupRange,
+			String registrationName);
 
 	/**
 	 * Notifies the JobManager to trigger a savepoint for this job.
 	 *
-	 * @return Future of the savepoint trigger response.
-	 */
-	Future<TriggerSavepointResponse> triggerSavepoint();
-
-	/**
-	 * Notifies the Jobmanager to dispose specified savepoint.
-	 *
-	 * @param savepointPath The path of the savepoint.
-	 * @return The future of the savepoint disponse response.
+	 * @param leaderSessionID The leader id of JobManager
+	 * @return The savepoint path
 	 */
-	Future<DisposeSavepointResponse> disposeSavepoint(final String savepointPath);
+	Future<String> triggerSavepoint(final UUID leaderSessionID);
 
 	/**
 	 * Request the classloading props of this job.

http://git-wip-us.apache.org/repos/asf/flink/blob/34fef475/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/MiniClusterJobDispatcher.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/MiniClusterJobDispatcher.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/MiniClusterJobDispatcher.java
index e8fb5bb..019ccfe 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/MiniClusterJobDispatcher.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/MiniClusterJobDispatcher.java
@@ -25,6 +25,8 @@ import org.apache.flink.runtime.client.JobExecutionException;
 import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
 import org.apache.flink.runtime.jobgraph.JobGraph;
 import org.apache.flink.runtime.jobmanager.OnCompletionActions;
+import org.apache.flink.runtime.metrics.MetricRegistry;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
 import org.apache.flink.runtime.rpc.RpcService;
 
 import org.slf4j.Logger;
@@ -62,6 +64,9 @@ public class MiniClusterJobDispatcher {
 	/** al the services that the JobManager needs, such as BLOB service, factories, etc */
 	private final JobManagerServices jobManagerServices;
 
+	/** Registry for all metrics in the mini cluster */
+	private final MetricRegistry metricRegistry;
+
 	/** The number of JobManagers to launch (more than one simulates a high-availability setup) */
 	private final int numJobManagers;
 
@@ -86,8 +91,9 @@ public class MiniClusterJobDispatcher {
 	public MiniClusterJobDispatcher(
 			Configuration config,
 			RpcService rpcService,
-			HighAvailabilityServices haServices) throws Exception {
-		this(config, rpcService, haServices, 1);
+			HighAvailabilityServices haServices,
+			MetricRegistry metricRegistry) throws Exception {
+		this(config, rpcService, haServices, metricRegistry, 1);
 	}
 
 	/**
@@ -106,16 +112,18 @@ public class MiniClusterJobDispatcher {
 			Configuration config,
 			RpcService rpcService,
 			HighAvailabilityServices haServices,
+			MetricRegistry metricRegistry,
 			int numJobManagers) throws Exception {
 
 		checkArgument(numJobManagers >= 1);
 		this.configuration = checkNotNull(config);
 		this.rpcService = checkNotNull(rpcService);
 		this.haServices = checkNotNull(haServices);
+		this.metricRegistry = checkNotNull(metricRegistry);
 		this.numJobManagers = numJobManagers;
 
 		LOG.info("Creating JobMaster services");
-		this.jobManagerServices = JobManagerServices.fromConfiguration(config);
+		this.jobManagerServices = JobManagerServices.fromConfiguration(config, haServices);
 	}
 
 	// ------------------------------------------------------------------------
@@ -140,9 +148,8 @@ public class MiniClusterJobDispatcher {
 				if (runners != null) {
 					this.runners = null;
 
-					Exception shutdownException = new Exception("The MiniCluster is shutting down");
 					for (JobManagerRunner runner : runners) {
-						runner.shutdown(shutdownException);
+						runner.shutdown();
 					}
 				}
 			}
@@ -171,9 +178,9 @@ public class MiniClusterJobDispatcher {
 			checkState(!shutdown, "mini cluster is shut down");
 			checkState(runners == null, "mini cluster can only execute one job at a time");
 
-			OnCompletionActions onJobCompletion = new DetachedFinalizer(numJobManagers);
+			DetachedFinalizer finalizer = new DetachedFinalizer(numJobManagers);
 
-			this.runners = startJobRunners(job, onJobCompletion);
+			this.runners = startJobRunners(job, finalizer, finalizer);
 		}
 	}
 
@@ -191,17 +198,17 @@ public class MiniClusterJobDispatcher {
 		checkNotNull(job);
 		
 		LOG.info("Received job for blocking execution {} ({})", job.getName(), job.getJobID());
-		final BlockingJobSync onJobCompletion = new BlockingJobSync(job.getJobID(), numJobManagers);
+		final BlockingJobSync sync = new BlockingJobSync(job.getJobID(), numJobManagers);
 
 		synchronized (lock) {
 			checkState(!shutdown, "mini cluster is shut down");
 			checkState(runners == null, "mini cluster can only execute one job at a time");
 
-			this.runners = startJobRunners(job, onJobCompletion);
+			this.runners = startJobRunners(job, sync, sync);
 		}
 
 		try {
-			return onJobCompletion.getResult();
+			return sync.getResult();
 		}
 		finally {
 			// always clear the status for the next job
@@ -209,24 +216,26 @@ public class MiniClusterJobDispatcher {
 		}
 	}
 
-	private JobManagerRunner[] startJobRunners(JobGraph job, OnCompletionActions onCompletion) throws JobExecutionException {
+	private JobManagerRunner[] startJobRunners(
+			JobGraph job,
+			OnCompletionActions onCompletion,
+			FatalErrorHandler errorHandler) throws JobExecutionException {
 		LOG.info("Starting {} JobMaster(s) for job {} ({})", numJobManagers, job.getName(), job.getJobID());
 
 		JobManagerRunner[] runners = new JobManagerRunner[numJobManagers];
 		for (int i = 0; i < numJobManagers; i++) {
 			try {
 				runners[i] = new JobManagerRunner(job, configuration,
-						rpcService, haServices, jobManagerServices, onCompletion);
+						rpcService, haServices, jobManagerServices, metricRegistry, 
+						onCompletion, errorHandler);
 				runners[i].start();
 			}
 			catch (Throwable t) {
 				// shut down all the ones so far
-				Exception shutdownCause = new Exception("Failed to properly start all mini cluster JobManagers", t);
-
 				for (int k = 0; k <= i; k++) {
 					try {
 						if (runners[i] != null) {
-							runners[i].shutdown(shutdownCause);
+							runners[i].shutdown();
 						}
 					} catch (Throwable ignored) {
 						// silent shutdown
@@ -244,15 +253,15 @@ public class MiniClusterJobDispatcher {
 	//  test methods to simulate job master failures
 	// ------------------------------------------------------------------------
 
-	public void killJobMaster(int which) {
-		checkArgument(which >= 0 && which < numJobManagers, "no such job master");
-		checkState(!shutdown, "mini cluster is shut down");
-
-		JobManagerRunner[] runners = this.runners;
-		checkState(runners != null, "mini cluster it not executing a job right now");
-
-		runners[which].shutdown(new Throwable("kill JobManager"));
-	}
+//	public void killJobMaster(int which) {
+//		checkArgument(which >= 0 && which < numJobManagers, "no such job master");
+//		checkState(!shutdown, "mini cluster is shut down");
+//
+//		JobManagerRunner[] runners = this.runners;
+//		checkState(runners != null, "mini cluster it not executing a job right now");
+//
+//		runners[which].shutdown(new Throwable("kill JobManager"));
+//	}
 
 	// ------------------------------------------------------------------------
 	//  utility classes
@@ -263,7 +272,7 @@ public class MiniClusterJobDispatcher {
 	 * In the case of a high-availability test setup, there may be multiple runners.
 	 * After that, it marks the mini cluster as ready to receive new jobs.
 	 */
-	private class DetachedFinalizer implements OnCompletionActions {
+	private class DetachedFinalizer implements OnCompletionActions, FatalErrorHandler {
 
 		private final AtomicInteger numJobManagersToWaitFor;
 
@@ -308,7 +317,7 @@ public class MiniClusterJobDispatcher {
 	 * That way it is guaranteed that after the blocking job submit call returns,
 	 * the dispatcher is immediately free to accept another job.
 	 */
-	private static class BlockingJobSync implements OnCompletionActions {
+	private static class BlockingJobSync implements OnCompletionActions, FatalErrorHandler {
 
 		private final JobID jobId;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/34fef475/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/message/DisposeSavepointResponse.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/message/DisposeSavepointResponse.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/message/DisposeSavepointResponse.java
deleted file mode 100644
index 42bfc71..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/message/DisposeSavepointResponse.java
+++ /dev/null
@@ -1,49 +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.jobmaster.message;
-
-import java.io.Serializable;
-
-/**
- * The response of the dispose savepoint request to JobManager.
- */
-public abstract class DisposeSavepointResponse implements Serializable {
-
-	private static final long serialVersionUID = 6008792963949369567L;
-
-	public static class Success extends DisposeSavepointResponse implements Serializable {
-
-		private static final long serialVersionUID = 1572462960008711415L;
-	}
-
-	public static class Failure extends DisposeSavepointResponse implements Serializable {
-
-		private static final long serialVersionUID = -7505308325483022458L;
-
-		private final Throwable cause;
-
-		public Failure(final Throwable cause) {
-			this.cause = cause;
-		}
-
-		public Throwable getCause() {
-			return cause;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/34fef475/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/message/TriggerSavepointResponse.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/message/TriggerSavepointResponse.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/message/TriggerSavepointResponse.java
deleted file mode 100644
index 0b0edc5..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/message/TriggerSavepointResponse.java
+++ /dev/null
@@ -1,74 +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.jobmaster.message;
-
-import org.apache.flink.api.common.JobID;
-
-import java.io.Serializable;
-
-/**
- * The response of the trigger savepoint request to JobManager.
- */
-public abstract class TriggerSavepointResponse implements Serializable {
-
-	private static final long serialVersionUID = 3139327824611807707L;
-
-	private final JobID jobID;
-
-	public JobID getJobID() {
-		return jobID;
-	}
-
-	public TriggerSavepointResponse(final JobID jobID) {
-		this.jobID = jobID;
-	}
-
-	public static class Success extends TriggerSavepointResponse implements Serializable {
-
-		private static final long serialVersionUID = -1100637460388881776L;
-
-		private final String savepointPath;
-
-		public Success(final JobID jobID, final String savepointPath) {
-			super(jobID);
-			this.savepointPath = savepointPath;
-		}
-
-		public String getSavepointPath() {
-			return savepointPath;
-		}
-	}
-
-	public static class Failure extends TriggerSavepointResponse implements Serializable {
-
-		private static final long serialVersionUID = -1668479003490615139L;
-
-		private final Throwable cause;
-
-		public Failure(final JobID jobID, final Throwable cause) {
-			super(jobID);
-			this.cause = cause;
-		}
-
-		public Throwable getCause() {
-			return cause;
-		}
-	}
-}
-

http://git-wip-us.apache.org/repos/asf/flink/blob/34fef475/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcService.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcService.java
index 2052f98..4b9100a 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcService.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcService.java
@@ -33,8 +33,8 @@ import java.util.concurrent.TimeUnit;
 public interface RpcService {
 
 	/**
-	 * Return the address under which the rpc service can be reached. If the rpc service cannot be
-	 * contacted remotely, then it will return an empty string.
+	 * Return the hostname or host address under which the rpc service can be reached.
+	 * If the rpc service cannot be contacted remotely, then it will return an empty string.
 	 *
 	 * @return Address of the rpc service or empty string if local rpc service
 	 */

http://git-wip-us.apache.org/repos/asf/flink/blob/34fef475/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/JobManagerConnection.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/JobManagerConnection.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/JobManagerConnection.java
index ef62ef1..6fcd082 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/JobManagerConnection.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/JobManagerConnection.java
@@ -26,11 +26,16 @@ import org.apache.flink.runtime.taskmanager.CheckpointResponder;
 import org.apache.flink.runtime.taskmanager.TaskManagerActions;
 import org.apache.flink.util.Preconditions;
 
+import java.util.UUID;
+
 /**
  * Container class for JobManager specific communication utils used by the {@link TaskExecutor}.
  */
 public class JobManagerConnection {
 
+	// Job master leader session id
+	private final UUID jobMasterLeaderId;
+
 	// Gateway to the job master
 	private final JobMasterGateway jobMasterGateway;
 
@@ -50,13 +55,15 @@ public class JobManagerConnection {
 	private final PartitionStateChecker partitionStateChecker;
 
 	public JobManagerConnection(
-		JobMasterGateway jobMasterGateway,
-		TaskManagerActions taskManagerActions,
-		CheckpointResponder checkpointResponder,
-		LibraryCacheManager libraryCacheManager,
-		ResultPartitionConsumableNotifier resultPartitionConsumableNotifier,
-		PartitionStateChecker partitionStateChecker) {
-
+			UUID jobMasterLeaderId,
+			JobMasterGateway jobMasterGateway,
+			TaskManagerActions taskManagerActions,
+			CheckpointResponder checkpointResponder,
+			LibraryCacheManager libraryCacheManager,
+			ResultPartitionConsumableNotifier resultPartitionConsumableNotifier,
+			PartitionStateChecker partitionStateChecker)
+	{
+		this.jobMasterLeaderId = Preconditions.checkNotNull(jobMasterLeaderId);
 		this.jobMasterGateway = Preconditions.checkNotNull(jobMasterGateway);
 		this.taskManagerActions = Preconditions.checkNotNull(taskManagerActions);
 		this.checkpointResponder = Preconditions.checkNotNull(checkpointResponder);
@@ -65,6 +72,10 @@ public class JobManagerConnection {
 		this.partitionStateChecker = Preconditions.checkNotNull(partitionStateChecker);
 	}
 
+	public UUID getJobMasterLeaderId() {
+		return jobMasterLeaderId;
+	}
+
 	public JobMasterGateway getJobManagerGateway() {
 		return jobMasterGateway;
 	}


[5/8] flink git commit: [FLINK-4735] [cluster management] Implements some job execution related RPC calls on the JobManager

Posted by se...@apache.org.
[FLINK-4735] [cluster management] Implements some job execution related RPC calls on the JobManager


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

Branch: refs/heads/flip-6
Commit: 21b9f16bb09785f72a7592925d3bb50160636797
Parents: 35a44da
Author: Kurt Young <yk...@gmail.com>
Authored: Tue Oct 4 23:00:22 2016 +0800
Committer: Stephan Ewen <se...@apache.org>
Committed: Thu Oct 13 16:25:49 2016 +0200

----------------------------------------------------------------------
 .../flink/runtime/jobmaster/JobMaster.java      | 246 +++++++++++++++++--
 .../runtime/jobmaster/JobMasterGateway.java     |  93 ++++++-
 .../jobmaster/message/ClassloadingProps.java    |  68 +++++
 .../message/DisposeSavepointResponse.java       |  49 ++++
 .../message/TriggerSavepointResponse.java       |  74 ++++++
 5 files changed, 507 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/21b9f16b/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 8f3a342..3b8fc97 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
@@ -28,6 +28,7 @@ import org.apache.flink.core.io.InputSplit;
 import org.apache.flink.core.io.InputSplitAssigner;
 import org.apache.flink.metrics.MetricGroup;
 import org.apache.flink.metrics.groups.UnregisteredMetricsGroup;
+import org.apache.flink.runtime.checkpoint.CheckpointCoordinator;
 import org.apache.flink.runtime.checkpoint.CheckpointIDCounter;
 import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory;
 import org.apache.flink.runtime.checkpoint.CompletedCheckpointStore;
@@ -39,8 +40,11 @@ import org.apache.flink.runtime.client.JobExecutionException;
 import org.apache.flink.runtime.client.JobSubmissionException;
 import org.apache.flink.runtime.client.SerializedJobExecutionResult;
 import org.apache.flink.runtime.clusterframework.types.ResourceID;
-import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.concurrent.BiFunction;
 import org.apache.flink.runtime.concurrent.Future;
+import org.apache.flink.runtime.concurrent.impl.FlinkCompletableFuture;
+import org.apache.flink.runtime.concurrent.impl.FlinkFuture;
+import org.apache.flink.runtime.execution.ExecutionState;
 import org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager;
 import org.apache.flink.runtime.executiongraph.Execution;
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
@@ -61,10 +65,20 @@ import org.apache.flink.runtime.jobgraph.jsonplan.JsonPlanGenerator;
 import org.apache.flink.runtime.jobgraph.tasks.JobSnapshottingSettings;
 import org.apache.flink.runtime.jobmanager.OnCompletionActions;
 import org.apache.flink.runtime.jobmanager.scheduler.Scheduler;
+import org.apache.flink.runtime.jobmaster.message.ClassloadingProps;
+import org.apache.flink.runtime.jobmaster.message.DisposeSavepointResponse;
+import org.apache.flink.runtime.jobmaster.message.NextInputSplit;
+import org.apache.flink.runtime.jobmaster.message.TriggerSavepointResponse;
 import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener;
 import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
-import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.messages.checkpoint.AcknowledgeCheckpoint;
+import org.apache.flink.runtime.messages.checkpoint.DeclineCheckpoint;
 import org.apache.flink.runtime.metrics.groups.JobManagerMetricGroup;
+import org.apache.flink.runtime.query.KvStateID;
+import org.apache.flink.runtime.query.KvStateLocation;
+import org.apache.flink.runtime.query.KvStateLocationRegistry;
+import org.apache.flink.runtime.query.KvStateServerAddress;
+import org.apache.flink.runtime.query.UnknownKvStateLocation;
 import org.apache.flink.runtime.registration.RegisteredRpcConnection;
 import org.apache.flink.runtime.registration.RegistrationResponse;
 import org.apache.flink.runtime.registration.RetryingRegistration;
@@ -72,7 +86,7 @@ import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway;
 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.state.CheckpointStateHandles;
+import org.apache.flink.runtime.state.KeyGroupRange;
 import org.apache.flink.runtime.taskmanager.TaskExecutionState;
 import org.apache.flink.runtime.util.SerializedThrowable;
 import org.apache.flink.util.InstantiationUtil;
@@ -520,22 +534,6 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 			throw new ExecutionGraphException("The execution attempt " +
 				taskExecutionState.getID() + " was not found.");
 		}
-
-	}
-
-	//----------------------------------------------------------------------------------------------\u2028
-	// Internal methods\u2028
-	// ----------------------------------------------------------------------------------------------\u2028\u2028
-
-	private void handleFatalError(final Throwable cause) {
-		runAsync(new Runnable() {
-			@Override
-			public void run() {
-				log.error("Fatal error occurred on JobManager, cause: {}", cause.getMessage(), cause);
-				shutDown();
-				jobCompletionActions.onFatalError(cause);
-			}
-		});
 	}
 
 	@RpcMethod
@@ -631,10 +629,220 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 		throw new UnsupportedOperationException();
 	}
 
+	@RpcMethod
+	public void resourceRemoved(final ResourceID resourceId, final String message) {
+		// TODO: remove resource from slot pool
+	}
+
+	@RpcMethod
+	public void acknowledgeCheckpoint(final AcknowledgeCheckpoint acknowledge) {
+		if (executionGraph != null) {
+			final CheckpointCoordinator checkpointCoordinator = executionGraph.getCheckpointCoordinator();
+			if (checkpointCoordinator != null) {
+				getRpcService().execute(new Runnable() {
+					@Override
+					public void run() {
+						try {
+							if (!checkpointCoordinator.receiveAcknowledgeMessage(acknowledge)) {
+								log.info("Received message for non-existing checkpoint {}.",
+									acknowledge.getCheckpointId());
+							}
+						} catch (Exception e) {
+							log.error("Error in CheckpointCoordinator while processing {}", acknowledge, e);
+						}
+					}
+				});
+			}
+			else {
+				log.error("Received AcknowledgeCheckpoint message for job {} with no CheckpointCoordinator",
+					jobGraph.getJobID());
+			}
+		} else {
+			log.error("Received AcknowledgeCheckpoint for unavailable job {}", jobGraph.getJobID());
+		}
+	}
+
+	@RpcMethod
+	public void declineCheckpoint(final DeclineCheckpoint decline) {
+		if (executionGraph != null) {
+			final CheckpointCoordinator checkpointCoordinator = executionGraph.getCheckpointCoordinator();
+			if (checkpointCoordinator != null) {
+				getRpcService().execute(new Runnable() {
+					@Override
+					public void run() {
+						try {
+							if (!checkpointCoordinator.receiveDeclineMessage(decline)) {
+								log.info("Received message for non-existing checkpoint {}.", decline.getCheckpointId());
+							}
+						} catch (Exception e) {
+							log.error("Error in CheckpointCoordinator while processing {}", decline, e);
+						}
+					}
+				});
+			} else {
+				log.error("Received DeclineCheckpoint message for job {} with no CheckpointCoordinator",
+					jobGraph.getJobID());
+			}
+		} else {
+			log.error("Received AcknowledgeCheckpoint for unavailable job {}", jobGraph.getJobID());
+		}
+	}
+
+	@RpcMethod
+	public KvStateLocation lookupKvStateLocation(final String registrationName) throws Exception {
+		if (executionGraph != null) {
+			if (log.isDebugEnabled()) {
+				log.debug("Lookup key-value state for job {} with registration " +
+					"name {}.", jobGraph.getJobID(), registrationName);
+			}
+
+			final KvStateLocationRegistry registry = executionGraph.getKvStateLocationRegistry();
+			final KvStateLocation location = registry.getKvStateLocation(registrationName);
+			if (location != null) {
+				return location;
+			} else {
+				throw new UnknownKvStateLocation(registrationName);
+			}
+		} else {
+			throw new IllegalStateException("Received lookup KvState location request for unavailable job " +
+				jobGraph.getJobID());
+		}
+	}
+
+	@RpcMethod
+	public void notifyKvStateRegistered(
+		final JobVertexID jobVertexId,
+		final KeyGroupRange keyGroupRange,
+		final String registrationName,
+		final KvStateID kvStateId,
+		final KvStateServerAddress kvStateServerAddress)
+	{
+		if (executionGraph != null) {
+			if (log.isDebugEnabled()) {
+				log.debug("Key value state registered for job {} under name {}.",
+					jobGraph.getJobID(), registrationName);
+			}
+			try {
+				executionGraph.getKvStateLocationRegistry().notifyKvStateRegistered(
+					jobVertexId, keyGroupRange, registrationName, kvStateId, kvStateServerAddress
+				);
+			} catch (Exception e) {
+				log.error("Failed to notify KvStateRegistry about registration {}.", registrationName);
+			}
+		} else {
+			log.error("Received notify KvState registered request for unavailable job " + jobGraph.getJobID());
+		}
+	}
+
+	@RpcMethod
+	public void notifyKvStateUnregistered(
+		JobVertexID jobVertexId,
+		KeyGroupRange keyGroupRange,
+		String registrationName)
+	{
+		if (executionGraph != null) {
+			if (log.isDebugEnabled()) {
+				log.debug("Key value state unregistered for job {} under name {}.",
+					jobGraph.getJobID(), registrationName);
+			}
+			try {
+				executionGraph.getKvStateLocationRegistry().notifyKvStateUnregistered(
+					jobVertexId, keyGroupRange, registrationName
+				);
+			} catch (Exception e) {
+				log.error("Failed to notify KvStateRegistry about registration {}.", registrationName);
+			}
+		} else {
+			log.error("Received notify KvState unregistered request for unavailable job " + jobGraph.getJobID());
+		}
+	}
+
+	@RpcMethod
+	public Future<TriggerSavepointResponse> triggerSavepoint() throws Exception {
+		if (executionGraph != null) {
+			final CheckpointCoordinator checkpointCoordinator = executionGraph.getCheckpointCoordinator();
+			if (checkpointCoordinator != null) {
+				try {
+					Future<String> savepointFuture = new FlinkFuture<>(
+						checkpointCoordinator.triggerSavepoint(System.currentTimeMillis()));
+
+					return savepointFuture.handleAsync(new BiFunction<String, Throwable, TriggerSavepointResponse>() {
+						@Override
+						public TriggerSavepointResponse apply(String savepointPath, Throwable throwable) {
+							if (throwable == null) {
+								return new TriggerSavepointResponse.Success(jobGraph.getJobID(), savepointPath);
+							}
+							else {
+								return new TriggerSavepointResponse.Failure(jobGraph.getJobID(),
+									new Exception("Failed to complete savepoint", throwable));
+							}
+						}
+					}, getMainThreadExecutor());
+
+				} catch (Exception e) {
+					FlinkCompletableFuture<TriggerSavepointResponse> future = new FlinkCompletableFuture<>();
+					future.complete(new TriggerSavepointResponse.Failure(jobGraph.getJobID(),
+						new Exception("Failed to trigger savepoint", e)));
+					return future;
+				}
+			} else {
+				FlinkCompletableFuture<TriggerSavepointResponse> future = new FlinkCompletableFuture<>();
+				future.complete(new TriggerSavepointResponse.Failure(jobGraph.getJobID(),
+					new IllegalStateException("Checkpointing disabled. You can enable it via the execution " +
+						"environment of your job.")));
+				return future;
+			}
+		} else {
+			FlinkCompletableFuture<TriggerSavepointResponse> future = new FlinkCompletableFuture<>();
+			future.complete(new TriggerSavepointResponse.Failure(jobGraph.getJobID(),
+				new IllegalArgumentException("Received trigger savepoint request for unavailable job " +
+					jobGraph.getJobID())));
+			return future;
+		}
+	}
+
+	@RpcMethod
+	public DisposeSavepointResponse disposeSavepoint(final String savepointPath) {
+		try {
+			log.info("Disposing savepoint at {}.", savepointPath);
+
+			// check whether the savepoint exists
+			savepointStore.loadSavepoint(savepointPath);
+
+			savepointStore.disposeSavepoint(savepointPath);
+			return new DisposeSavepointResponse.Success();
+		} catch (Exception e) {
+			log.error("Failed to dispose savepoint at {}.", savepointPath, e);
+			return new DisposeSavepointResponse.Failure(e);
+		}
+	}
+
+	@RpcMethod
+	public ClassloadingProps requestClassloadingProps() throws Exception {
+		if (executionGraph != null) {
+			return new ClassloadingProps(libraryCacheManager.getBlobServerPort(),
+				executionGraph.getRequiredJarFiles(),
+				executionGraph.getRequiredClasspaths());
+		} else {
+			throw new Exception("Received classloading props request for unavailable job " + jobGraph.getJobID());
+		}
+	}
+
 	//----------------------------------------------------------------------------------------------
 	// Internal methods
 	//----------------------------------------------------------------------------------------------
 
+	private void handleFatalError(final Throwable cause) {
+		runAsync(new Runnable() {
+			@Override
+			public void run() {
+				log.error("Fatal error occurred on JobManager, cause: {}", cause.getMessage(), cause);
+				shutDown();
+				jobCompletionActions.onFatalError(cause);
+			}
+		});
+	}
+
 	// TODO - wrap this as StatusListenerMessenger's callback with rpc main thread
 	private void jobStatusChanged(final JobStatus newJobStatus, long timestamp, final Throwable error) {
 		final JobID jobID = executionGraph.getJobID();

http://git-wip-us.apache.org/repos/asf/flink/blob/21b9f16b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterGateway.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterGateway.java
index e3e57d4..4b51258 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterGateway.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterGateway.java
@@ -18,8 +18,6 @@
 
 package org.apache.flink.runtime.jobmaster;
 
-import org.apache.flink.api.common.time.Time;
-import org.apache.flink.runtime.checkpoint.CheckpointCoordinatorGateway;
 import org.apache.flink.runtime.clusterframework.types.ResourceID;
 import org.apache.flink.runtime.concurrent.Future;
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
@@ -30,8 +28,18 @@ import org.apache.flink.runtime.io.network.partition.ResultPartition;
 import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
 import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
-import org.apache.flink.runtime.messages.Acknowledge;
-import org.apache.flink.runtime.rpc.RpcTimeout;
+import org.apache.flink.runtime.jobmaster.message.ClassloadingProps;
+import org.apache.flink.runtime.jobmaster.message.DisposeSavepointResponse;
+import org.apache.flink.runtime.jobmaster.message.NextInputSplit;
+import org.apache.flink.runtime.jobmaster.message.TriggerSavepointResponse;
+import org.apache.flink.runtime.messages.checkpoint.AcknowledgeCheckpoint;
+import org.apache.flink.runtime.messages.checkpoint.DeclineCheckpoint;
+import org.apache.flink.runtime.query.KvStateID;
+import org.apache.flink.runtime.query.KvStateLocation;
+import org.apache.flink.runtime.query.KvStateServerAddress;
+import org.apache.flink.runtime.rpc.RpcGateway;
+import org.apache.flink.runtime.state.KeyGroupRange;
+import org.apache.flink.runtime.state.KvState;
 import org.apache.flink.runtime.taskmanager.TaskExecutionState;
 
 import java.util.UUID;
@@ -110,4 +118,81 @@ public interface JobMasterGateway extends CheckpointCoordinatorGateway {
 	 * @param resourceID identifying the TaskManager to disconnect
 	 */
 	void disconnectTaskManager(ResourceID resourceID);
+	void scheduleOrUpdateConsumers(final ResultPartitionID partitionID);
+
+	/**
+	 * Notifies the JobManager about the removal of a resource.
+	 *
+	 * @param resourceId The ID under which the resource is registered.
+	 * @param message    Optional message with details, for logging and debugging.
+	 */
+
+	void resourceRemoved(final ResourceID resourceId, final String message);
+
+	/**
+	 * Notifies the JobManager that the checkpoint of an individual task is completed.
+	 *
+	 * @param acknowledge The acknowledge message of the checkpoint
+	 */
+	void acknowledgeCheckpoint(final AcknowledgeCheckpoint acknowledge);
+
+	/**
+	 * Notifies the JobManager that a checkpoint request could not be heeded.
+	 * This can happen if a Task is already in RUNNING state but is internally not yet ready to perform checkpoints.
+	 *
+	 * @param decline The decline message of the checkpoint
+	 */
+	void declineCheckpoint(final DeclineCheckpoint decline);
+
+	/**
+	 * Requests a {@link KvStateLocation} for the specified {@link KvState} registration name.
+	 *
+	 * @param registrationName Name under which the KvState has been registered.
+	 * @return Future of the requested {@link KvState} location
+	 */
+	Future<KvStateLocation> lookupKvStateLocation(final String registrationName) throws Exception;
+
+	/**
+	 * @param jobVertexId          JobVertexID the KvState instance belongs to.
+	 * @param keyGroupRange        Key group range the KvState instance belongs to.
+	 * @param registrationName     Name under which the KvState has been registered.
+	 * @param kvStateId            ID of the registered KvState instance.
+	 * @param kvStateServerAddress Server address where to find the KvState instance.
+	 */
+	void notifyKvStateRegistered(
+		final JobVertexID jobVertexId,
+		final KeyGroupRange keyGroupRange,
+		final String registrationName,
+		final KvStateID kvStateId,
+		final KvStateServerAddress kvStateServerAddress);
+
+	/**
+	 * @param jobVertexId      JobVertexID the KvState instance belongs to.
+	 * @param keyGroupRange    Key group index the KvState instance belongs to.
+	 * @param registrationName Name under which the KvState has been registered.
+	 */
+	void notifyKvStateUnregistered(
+		JobVertexID jobVertexId,
+		KeyGroupRange keyGroupRange,
+		String registrationName);
+
+	/**
+	 * Notifies the JobManager to trigger a savepoint for this job.
+	 *
+	 * @return Future of the savepoint trigger response.
+	 */
+	Future<TriggerSavepointResponse> triggerSavepoint();
+
+	/**
+	 * Notifies the Jobmanager to dispose specified savepoint.
+	 *
+	 * @param savepointPath The path of the savepoint.
+	 * @return The future of the savepoint disponse response.
+	 */
+	Future<DisposeSavepointResponse> disposeSavepoint(final String savepointPath);
+
+	/**
+	 * Request the classloading props of this job.
+	 */
+	Future<ClassloadingProps> requestClassloadingProps();
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/21b9f16b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/message/ClassloadingProps.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/message/ClassloadingProps.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/message/ClassloadingProps.java
new file mode 100644
index 0000000..2d670b4
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/message/ClassloadingProps.java
@@ -0,0 +1,68 @@
+/*
+ * 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.jobmaster.message;
+
+import org.apache.flink.runtime.blob.BlobKey;
+
+import java.io.Serializable;
+import java.net.URL;
+import java.util.List;
+
+/**
+ * The response of classloading props request to JobManager.
+ */
+public class ClassloadingProps implements Serializable {
+
+	private static final long serialVersionUID = -3282341310808511823L;
+
+	private final int blobManagerPort;
+
+	private final List<BlobKey> requiredJarFiles;
+
+	private final List<URL> requiredClasspaths;
+
+	/**
+	 * Constructor of ClassloadingProps.
+	 *
+	 * @param blobManagerPort    The port of the blobManager
+	 * @param requiredJarFiles   The blob keys of the required jar files
+	 * @param requiredClasspaths The urls of the required classpaths
+	 */
+	public ClassloadingProps(
+		final int blobManagerPort,
+		final List<BlobKey> requiredJarFiles,
+		final List<URL> requiredClasspaths)
+	{
+		this.blobManagerPort = blobManagerPort;
+		this.requiredJarFiles = requiredJarFiles;
+		this.requiredClasspaths = requiredClasspaths;
+	}
+
+	public int getBlobManagerPort() {
+		return blobManagerPort;
+	}
+
+	public List<BlobKey> getRequiredJarFiles() {
+		return requiredJarFiles;
+	}
+
+	public List<URL> getRequiredClasspaths() {
+		return requiredClasspaths;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/21b9f16b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/message/DisposeSavepointResponse.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/message/DisposeSavepointResponse.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/message/DisposeSavepointResponse.java
new file mode 100644
index 0000000..42bfc71
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/message/DisposeSavepointResponse.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.jobmaster.message;
+
+import java.io.Serializable;
+
+/**
+ * The response of the dispose savepoint request to JobManager.
+ */
+public abstract class DisposeSavepointResponse implements Serializable {
+
+	private static final long serialVersionUID = 6008792963949369567L;
+
+	public static class Success extends DisposeSavepointResponse implements Serializable {
+
+		private static final long serialVersionUID = 1572462960008711415L;
+	}
+
+	public static class Failure extends DisposeSavepointResponse implements Serializable {
+
+		private static final long serialVersionUID = -7505308325483022458L;
+
+		private final Throwable cause;
+
+		public Failure(final Throwable cause) {
+			this.cause = cause;
+		}
+
+		public Throwable getCause() {
+			return cause;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/21b9f16b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/message/TriggerSavepointResponse.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/message/TriggerSavepointResponse.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/message/TriggerSavepointResponse.java
new file mode 100644
index 0000000..0b0edc5
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/message/TriggerSavepointResponse.java
@@ -0,0 +1,74 @@
+/*
+ * 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.jobmaster.message;
+
+import org.apache.flink.api.common.JobID;
+
+import java.io.Serializable;
+
+/**
+ * The response of the trigger savepoint request to JobManager.
+ */
+public abstract class TriggerSavepointResponse implements Serializable {
+
+	private static final long serialVersionUID = 3139327824611807707L;
+
+	private final JobID jobID;
+
+	public JobID getJobID() {
+		return jobID;
+	}
+
+	public TriggerSavepointResponse(final JobID jobID) {
+		this.jobID = jobID;
+	}
+
+	public static class Success extends TriggerSavepointResponse implements Serializable {
+
+		private static final long serialVersionUID = -1100637460388881776L;
+
+		private final String savepointPath;
+
+		public Success(final JobID jobID, final String savepointPath) {
+			super(jobID);
+			this.savepointPath = savepointPath;
+		}
+
+		public String getSavepointPath() {
+			return savepointPath;
+		}
+	}
+
+	public static class Failure extends TriggerSavepointResponse implements Serializable {
+
+		private static final long serialVersionUID = -1668479003490615139L;
+
+		private final Throwable cause;
+
+		public Failure(final JobID jobID, final Throwable cause) {
+			super(jobID);
+			this.cause = cause;
+		}
+
+		public Throwable getCause() {
+			return cause;
+		}
+	}
+}
+