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/12/23 20:22:20 UTC

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

Repository: flink
Updated Branches:
  refs/heads/master 5b4e3d889 -> 235a16969


http://git-wip-us.apache.org/repos/asf/flink/blob/c6486067/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 b421ba6..69ebc83 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
@@ -27,11 +27,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 PartitionProducerStateChecker {
 
+	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);
 	}
 
@@ -41,6 +45,6 @@ public class RpcPartitionStateChecker implements PartitionProducerStateChecker {
 			IntermediateDataSetID resultId,
 			ResultPartitionID partitionId) {
 
-		return jobMasterGateway.requestPartitionState(jobId, resultId, partitionId);
+		return jobMasterGateway.requestPartitionState(jobMasterLeaderId, resultId, partitionId);
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/c6486067/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/c6486067/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 b0d0b55..da89940 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
@@ -667,6 +667,12 @@ 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)|"+
+      "(�s|micro|microsecond)|(ns|nano|nanosecond)"
+  }
+  
   /** Returns the protocol field for the URL of the remote actor system given the user configuration
     *
     * @param config instance containing the user provided configuration values

http://git-wip-us.apache.org/repos/asf/flink/blob/c6486067/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/c6486067/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 d812f6b..1a9818e 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;
@@ -57,13 +64,23 @@ public class JobManagerRunnerMockTest {
 
 	private LeaderElectionService leaderElectionService;
 
+	private SubmittedJobGraphStore submittedJobGraphStore;
+
 	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,16 +91,22 @@ public class JobManagerRunnerMockTest {
 
 		SubmittedJobGraphStore submittedJobGraphStore = mock(SubmittedJobGraphStore.class);
 
+		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));
 	}
 
@@ -91,25 +114,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);
 
@@ -126,13 +150,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
@@ -145,13 +170,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
@@ -163,39 +189,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/c6486067/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 
+}


[23/52] [abbrv] flink git commit: [FLINK-4987] Add RpcTaskManagerGateway implementation; Port AllocatedSlotsTest, AvailableSlotsTest and SlotPoolTest

Posted by se...@apache.org.
[FLINK-4987] Add RpcTaskManagerGateway implementation; Port AllocatedSlotsTest, AvailableSlotsTest and SlotPoolTest

The RpcTaskManagerGateway is the TaskManagerGateway of Flink's new RPC abstraction. It basically forwards all calls to the underlying TaskExecutorGateway.

Moreover, this PR enables the disabled tests AllocatedSlotsTest, AvailableSlotsTest and SlotPoolTest.

Add license header to RpcTaskManagerGateway

Fix ExecutionGraphMetricsTest


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

Branch: refs/heads/master
Commit: 522edae3c767ed178691a4b4f93e676fdf2898fb
Parents: a1ba9f1
Author: Till Rohrmann <tr...@apache.org>
Authored: Tue Nov 1 03:03:23 2016 +0100
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Dec 23 20:54:25 2016 +0100

----------------------------------------------------------------------
 .../flink/runtime/executiongraph/Execution.java |   1 +
 .../runtime/executiongraph/ExecutionVertex.java |   3 +-
 .../apache/flink/runtime/instance/SlotPool.java |  19 +-
 .../runtime/jobmanager/slots/AllocatedSlot.java |  31 +-
 .../flink/runtime/jobmaster/JobMaster.java      |  28 +-
 .../jobmaster/RpcTaskManagerGateway.java        | 142 +++++
 .../resourcemanager/ResourceManager.java        |   5 +
 .../taskexecutor/TaskExecutorGateway.java       |  12 +-
 .../ExecutionGraphMetricsTest.java              |   8 +-
 .../ExecutionVertexDeploymentTest.java          |   7 +
 .../runtime/instance/AllocatedSlotsTest.java    | 275 +++++----
 .../runtime/instance/AvailableSlotsTest.java    | 244 ++++----
 .../flink/runtime/instance/SlotPoolTest.java    | 608 ++++++++++---------
 .../runtime/minicluster/MiniClusterITCase.java  |   4 +-
 .../runtime/rpc/TestingSerialRpcService.java    |   5 +-
 15 files changed, 783 insertions(+), 609 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/522edae3/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java
index 16aebce..18a4445 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java
@@ -298,6 +298,7 @@ public class Execution implements AccessExecution, Archiveable<ArchivedExecution
 			});
 
 			// if tasks have to scheduled immediately check that the task has been deployed
+			// TODO: This might be problematic if the future is not completed right away
 			if (!queued) {
 				if (!deploymentFuture.isDone()) {
 					markFailed(new IllegalArgumentException("The slot allocation future has not been completed yet."));

http://git-wip-us.apache.org/repos/asf/flink/blob/522edae3/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java
index a1c4d8b..3b5a6cc 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java
@@ -22,7 +22,6 @@ import org.apache.flink.api.common.Archiveable;
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.api.common.time.Time;
 import org.apache.flink.runtime.JobException;
-import org.apache.flink.runtime.clusterframework.types.AllocationID;
 import org.apache.flink.runtime.deployment.InputChannelDeploymentDescriptor;
 import org.apache.flink.runtime.deployment.InputGateDeploymentDescriptor;
 import org.apache.flink.runtime.deployment.PartialInputChannelDeploymentDescriptor;
@@ -627,7 +626,7 @@ public class ExecutionVertex implements AccessExecutionVertex, Archiveable<Archi
 			serializedJobInformation,
 			serializedJobVertexInformation,
 			executionId,
-			new AllocationID(), // TODO: Obtain the proper allocation id from the slot
+			targetSlot.getAllocatedSlot().getSlotAllocationId(),
 			subTaskIndex,
 			attemptNumber,
 			targetSlot.getRoot().getSlotNumber(),

http://git-wip-us.apache.org/repos/asf/flink/blob/522edae3/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
index 5a3a321..05884d1 100644
--- 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
@@ -172,7 +172,7 @@ public class SlotPool extends RpcEndpoint<SlotPoolGateway> {
 	 *
 	 * @param jobManagerLeaderId The necessary leader id for running the job.
 	 */
-	public void start(UUID jobManagerLeaderId) {
+	public void start(UUID jobManagerLeaderId) throws Exception {
 		this.jobManagerLeaderId = jobManagerLeaderId;
 
 		// TODO - start should not throw an exception
@@ -295,7 +295,7 @@ public class SlotPool extends RpcEndpoint<SlotPoolGateway> {
 		pendingRequests.put(allocationID, new PendingRequest(allocationID, future, resources));
 
 		Future<RMSlotRequestReply> rmResponse = resourceManagerGateway.requestSlot(
-				resourceManagerLeaderId, jobManagerLeaderId,
+				jobManagerLeaderId, resourceManagerLeaderId,
 				new SlotRequest(jobId, allocationID, resources),
 				resourceManagerRequestsTimeout);
 
@@ -579,7 +579,7 @@ public class SlotPool extends RpcEndpoint<SlotPoolGateway> {
 	/**
 	 * Organize allocated slots from different points of view.
 	 */
-	private static class AllocatedSlots {
+	static class AllocatedSlots {
 
 		/** All allocated slots organized by TaskManager's id */
 		private final Map<ResourceID, Set<Slot>> allocatedSlotsByTaskManager;
@@ -650,7 +650,7 @@ public class SlotPool extends RpcEndpoint<SlotPoolGateway> {
 				Set<Slot> slotsForTM = allocatedSlotsByTaskManager.get(taskManagerId);
 				slotsForTM.remove(slot);
 				if (slotsForTM.isEmpty()) {
-					allocatedSlotsByTaskManager.get(taskManagerId);
+					allocatedSlotsByTaskManager.remove(taskManagerId);
 				}
 				return slot;
 			}
@@ -692,6 +692,15 @@ public class SlotPool extends RpcEndpoint<SlotPoolGateway> {
 		int size() {
 			return allocatedSlotsById.size();
 		}
+
+		@VisibleForTesting
+		Set<Slot> getSlotsForTaskManager(ResourceID resourceId) {
+			if (allocatedSlotsByTaskManager.containsKey(resourceId)) {
+				return allocatedSlotsByTaskManager.get(resourceId);
+			} else {
+				return Collections.emptySet();
+			}
+		}
 	}
 
 	// ------------------------------------------------------------------------
@@ -699,7 +708,7 @@ public class SlotPool extends RpcEndpoint<SlotPoolGateway> {
 	/**
 	 * Organize all available slots from different points of view.
 	 */
-	private static class AvailableSlots {
+	static class AvailableSlots {
 
 		/** All available slots organized by TaskManager */
 		private final HashMap<ResourceID, Set<AllocatedSlot>> availableSlotsByTaskManager;

http://git-wip-us.apache.org/repos/asf/flink/blob/522edae3/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/slots/AllocatedSlot.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/slots/AllocatedSlot.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/slots/AllocatedSlot.java
index f477c49..a0c608d 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/slots/AllocatedSlot.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/slots/AllocatedSlot.java
@@ -53,11 +53,8 @@ public class AllocatedSlot {
 	/** 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 TaskManagerGateway taskManagerGateway;
-
 	/** RPC gateway to call the TaskManager that holds this slot */
-	private final TaskExecutorGateway taskExecutorGateway;
+	private final TaskManagerGateway taskManagerGateway;
 
 	/** The number of the slot on the TaskManager to which slot belongs. Purely informational. */
 	private final int slotNumber;
@@ -69,32 +66,14 @@ public class AllocatedSlot {
 			JobID jobID,
 			TaskManagerLocation location,
 			int slotNumber,
-			ResourceProfile resourceProfile,
-			TaskManagerGateway taskManagerGateway)
-	{
+			ResourceProfile resourceProfile,		
+			TaskManagerGateway taskManagerGateway) {
 		this.slotAllocationId = checkNotNull(slotAllocationId);
 		this.jobID = checkNotNull(jobID);
 		this.taskManagerLocation = checkNotNull(location);
 		this.slotNumber = slotNumber;
 		this.resourceProfile = checkNotNull(resourceProfile);
 		this.taskManagerGateway = checkNotNull(taskManagerGateway);
-		this.taskExecutorGateway = null;
-	}
-
-	public AllocatedSlot(
-			AllocationID slotAllocationId,
-			JobID jobID,
-			TaskManagerLocation location,
-			int slotNumber,
-			ResourceProfile resourceProfile,
-			TaskExecutorGateway taskExecutorGateway) {
-		this.slotAllocationId = checkNotNull(slotAllocationId);
-		this.jobID = checkNotNull(jobID);
-		this.taskManagerLocation = checkNotNull(location);
-		this.slotNumber = slotNumber;
-		this.resourceProfile = checkNotNull(resourceProfile);
-		this.taskManagerGateway = null;
-		this.taskExecutorGateway = checkNotNull(taskExecutorGateway);
 	}
 
 	// ------------------------------------------------------------------------
@@ -166,10 +145,6 @@ public class AllocatedSlot {
 		return taskManagerGateway;
 	}
 
-	public TaskExecutorGateway getTaskExecutorGateway() {
-		return taskExecutorGateway;
-	}
-
 	// ------------------------------------------------------------------------
 
 	/**

http://git-wip-us.apache.org/repos/asf/flink/blob/522edae3/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 a620390..d6bcf2c 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
@@ -308,9 +308,15 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 
 		log.info("Starting execution of job {} ({})", jobGraph.getName(), jobGraph.getJobID());
 
-		// start the slot pool make sure the slot pool now accepts messages for this leader
-		log.debug("Staring SlotPool component");
-		slotPool.start(leaderSessionID);
+		try {
+			// start the slot pool make sure the slot pool now accepts messages for this leader
+			log.debug("Staring SlotPool component");
+			slotPool.start(leaderSessionID);
+		} catch (Exception e) {
+			log.error("Faild to start job {} ({})", jobGraph.getName(), jobGraph.getJobID(), e);
+
+			handleFatalError(new Exception("Could not start job execution: Failed to start the slot pool.", e));
+		}
 
 		try {
 			// job is ready to go, try to establish connection with resource manager
@@ -634,24 +640,30 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 			final Iterable<SlotOffer> slots,
 			final UUID leaderId) throws Exception {
 
-		validateLeaderSessionId(leaderSessionID);
+		validateLeaderSessionId(leaderId);
 
 		Tuple2<TaskManagerLocation, TaskExecutorGateway> taskManager = registeredTaskManagers.get(taskManagerId);
+
 		if (taskManager == null) {
 			throw new Exception("Unknown TaskManager " + taskManagerId);
 		}
 
 		final JobID jid = jobGraph.getJobID();
 		final TaskManagerLocation taskManagerLocation = taskManager.f0;
-		final TaskExecutorGateway taskManagerGateway = taskManager.f1;
+		final TaskExecutorGateway taskExecutorGateway = taskManager.f1;
 
 		final ArrayList<Tuple2<AllocatedSlot, SlotOffer>> slotsAndOffers = new ArrayList<>();
 
+		final RpcTaskManagerGateway rpcTaskManagerGateway = new RpcTaskManagerGateway(taskExecutorGateway, leaderId);
+
 		for (SlotOffer slotOffer : slots) {
 			final AllocatedSlot slot = new AllocatedSlot(
-					slotOffer.getAllocationId(), jid, taskManagerLocation,
-					slotOffer.getSlotIndex(), slotOffer.getResourceProfile(),
-					taskManagerGateway);
+				slotOffer.getAllocationId(),
+				jid,
+				taskManagerLocation,
+				slotOffer.getSlotIndex(),
+				slotOffer.getResourceProfile(),
+				rpcTaskManagerGateway);
 
 			slotsAndOffers.add(new Tuple2<>(slot, slotOffer));
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/522edae3/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/RpcTaskManagerGateway.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/RpcTaskManagerGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/RpcTaskManagerGateway.java
new file mode 100644
index 0000000..eba97d2
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/RpcTaskManagerGateway.java
@@ -0,0 +1,142 @@
+/*
+ * 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;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.blob.BlobKey;
+import org.apache.flink.runtime.clusterframework.ApplicationStatus;
+import org.apache.flink.runtime.concurrent.Future;
+import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.executiongraph.PartitionInfo;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.jobmanager.slots.TaskManagerGateway;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.messages.StackTrace;
+import org.apache.flink.runtime.messages.StackTraceSampleResponse;
+import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway;
+import org.apache.flink.util.Preconditions;
+
+import java.util.UUID;
+
+/**
+ * Implementation of the {@link TaskManagerGateway} for Flink's RPC system
+ */
+public class RpcTaskManagerGateway implements TaskManagerGateway {
+
+	private final TaskExecutorGateway taskExecutorGateway;
+
+	private final UUID leaderId;
+
+	public RpcTaskManagerGateway(TaskExecutorGateway taskExecutorGateway, UUID leaderId) {
+		this.taskExecutorGateway = Preconditions.checkNotNull(taskExecutorGateway);
+		this.leaderId = Preconditions.checkNotNull(leaderId);
+	}
+
+	@Override
+	public String getAddress() {
+		return taskExecutorGateway.getAddress();
+	}
+
+	@Override
+	public void disconnectFromJobManager(InstanceID instanceId, Exception cause) {
+//		taskExecutorGateway.disconnectFromJobManager(instanceId, cause);
+		throw new UnsupportedOperationException("Operation is not yet supported.");
+	}
+
+	@Override
+	public void stopCluster(ApplicationStatus applicationStatus, String message) {
+//		taskExecutorGateway.stopCluster(applicationStatus, message);
+		throw new UnsupportedOperationException("Operation is not yet supported.");
+	}
+
+	@Override
+	public Future<StackTrace> requestStackTrace(Time timeout) {
+//		return taskExecutorGateway.requestStackTrace(timeout);
+		throw new UnsupportedOperationException("Operation is not yet supported.");
+	}
+
+	@Override
+	public Future<StackTraceSampleResponse> requestStackTraceSample(
+			ExecutionAttemptID executionAttemptID,
+			int sampleId,
+			int numSamples,
+			Time delayBetweenSamples,
+			int maxStackTraceDepth,
+			Time timeout) {
+//		return taskExecutorGateway.requestStackTraceSample(
+//			executionAttemptID,
+//			sampleId,
+//			numSamples,
+//			delayBetweenSamples,
+//			maxStackTraceDepth,
+//			timeout);
+
+		throw new UnsupportedOperationException("Operation is not yet supported.");
+	}
+
+	@Override
+	public Future<Acknowledge> submitTask(TaskDeploymentDescriptor tdd, Time timeout) {
+		return taskExecutorGateway.submitTask(tdd, leaderId, timeout);
+	}
+
+	@Override
+	public Future<Acknowledge> stopTask(ExecutionAttemptID executionAttemptID, Time timeout) {
+		return taskExecutorGateway.stopTask(executionAttemptID, timeout);
+	}
+
+	@Override
+	public Future<Acknowledge> cancelTask(ExecutionAttemptID executionAttemptID, Time timeout) {
+		return taskExecutorGateway.cancelTask(executionAttemptID, timeout);
+	}
+
+	@Override
+	public Future<Acknowledge> updatePartitions(ExecutionAttemptID executionAttemptID, Iterable<PartitionInfo> partitionInfos, Time timeout) {
+		return taskExecutorGateway.updatePartitions(executionAttemptID, partitionInfos, timeout);
+	}
+
+	@Override
+	public void failPartition(ExecutionAttemptID executionAttemptID) {
+		taskExecutorGateway.failPartition(executionAttemptID);
+	}
+
+	@Override
+	public void notifyCheckpointComplete(ExecutionAttemptID executionAttemptID, JobID jobId, long checkpointId, long timestamp) {
+//		taskExecutorGateway.notifyCheckpointComplete(executionAttemptID, jobId, checkpointId, timestamp);
+		throw new UnsupportedOperationException("Operation is not yet supported.");
+	}
+
+	@Override
+	public void triggerCheckpoint(ExecutionAttemptID executionAttemptID, JobID jobId, long checkpointId, long timestamp) {
+//		taskExecutorGateway.triggerCheckpoint(executionAttemptID, jobId, checkpointId, timestamp);
+		throw new UnsupportedOperationException("Operation is not yet supported.");
+	}
+
+	@Override
+	public Future<BlobKey> requestTaskManagerLog(Time timeout) {
+//		return taskExecutorGateway.requestTaskManagerLog(timeout);
+		throw new UnsupportedOperationException("Operation is not yet supported.");
+	}
+
+	@Override
+	public Future<BlobKey> requestTaskManagerStdout(Time timeout) {
+//		return taskExecutorGateway.requestTaskManagerStdout(timeout);
+		throw new UnsupportedOperationException("Operation is not yet supported.");
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/522edae3/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java
index a81c214..145cc40 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java
@@ -394,6 +394,11 @@ public abstract class ResourceManager<WorkerType extends Serializable>
 			UUID resourceManagerLeaderID,
 			SlotRequest slotRequest) {
 
+		log.info("Request slot with profile {} for job {} with allocation id {}.",
+			slotRequest.getResourceProfile(),
+			slotRequest.getJobId(),
+			slotRequest.getAllocationId());
+
 		JobID jobId = slotRequest.getJobId();
 		JobManagerRegistration jobManagerRegistration = jobManagerRegistrations.get(jobId);
 

http://git-wip-us.apache.org/repos/asf/flink/blob/522edae3/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorGateway.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorGateway.java
index 1ffc407..ebd4c0c 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorGateway.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorGateway.java
@@ -75,9 +75,13 @@ public interface TaskExecutorGateway extends RpcGateway {
 	 *
 	 * @param executionAttemptID identifying the task
 	 * @param partitionInfos telling where the partition can be retrieved from
+	 * @param timeout for the update partitions operation
 	 * @return Future acknowledge if the partitions have been successfully updated
 	 */
-	Future<Acknowledge> updatePartitions(ExecutionAttemptID executionAttemptID, Iterable<PartitionInfo> partitionInfos);
+	Future<Acknowledge> updatePartitions(
+		ExecutionAttemptID executionAttemptID,
+		Iterable<PartitionInfo> partitionInfos,
+		@RpcTimeout Time timeout);
 
 	/**
 	 * Fail all intermediate result partitions of the given task.
@@ -112,15 +116,17 @@ public interface TaskExecutorGateway extends RpcGateway {
 	 * Stop the given task.
 	 *
 	 * @param executionAttemptID identifying the task
+	 * @param timeout for the stop operation
 	 * @return Future acknowledge if the task is successfully stopped
 	 */
-	Future<Acknowledge> stopTask(ExecutionAttemptID executionAttemptID);
+	Future<Acknowledge> stopTask(ExecutionAttemptID executionAttemptID, @RpcTimeout Time timeout);
 
 	/**
 	 * Cancel the given task.
 	 *
 	 * @param executionAttemptID identifying the task
+	 * @param timeout for the cancel operation
 	 * @return Future acknowledge if the task is successfully canceled
 	 */
-	Future<Acknowledge> cancelTask(ExecutionAttemptID executionAttemptID);
+	Future<Acknowledge> cancelTask(ExecutionAttemptID executionAttemptID, @RpcTimeout Time timeout);
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/522edae3/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphMetricsTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphMetricsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphMetricsTest.java
index 0f7e75f..41e47d2 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphMetricsTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphMetricsTest.java
@@ -29,6 +29,7 @@ import org.apache.flink.metrics.MetricGroup;
 import org.apache.flink.metrics.reporter.MetricReporter;
 import org.apache.flink.runtime.JobException;
 import org.apache.flink.runtime.blob.BlobKey;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
 import org.apache.flink.runtime.clusterframework.types.ResourceID;
 import org.apache.flink.runtime.concurrent.impl.FlinkCompletableFuture;
 import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor;
@@ -36,6 +37,7 @@ import org.apache.flink.runtime.execution.ExecutionState;
 import org.apache.flink.runtime.execution.SuppressRestartsException;
 import org.apache.flink.runtime.executiongraph.restart.RestartStrategy;
 import org.apache.flink.runtime.instance.Instance;
+import org.apache.flink.runtime.jobmanager.slots.AllocatedSlot;
 import org.apache.flink.runtime.jobmanager.slots.TaskManagerGateway;
 import org.apache.flink.runtime.messages.Acknowledge;
 import org.apache.flink.runtime.metrics.MetricRegistryConfiguration;
@@ -127,6 +129,9 @@ public class ExecutionGraphMetricsTest extends TestLogger {
 
 			Slot rootSlot = mock(Slot.class);
 
+			AllocatedSlot mockAllocatedSlot = mock(AllocatedSlot.class);
+			when(mockAllocatedSlot.getSlotAllocationId()).thenReturn(new AllocationID());
+
 			SimpleSlot simpleSlot = mock(SimpleSlot.class);
 			when(simpleSlot.isAlive()).thenReturn(true);
 			when(simpleSlot.getTaskManagerLocation()).thenReturn(taskManagerLocation);
@@ -134,6 +139,7 @@ public class ExecutionGraphMetricsTest extends TestLogger {
 			when(simpleSlot.getTaskManagerGateway()).thenReturn(taskManagerGateway);
 			when(simpleSlot.setExecutedVertex(Matchers.any(Execution.class))).thenReturn(true);
 			when(simpleSlot.getRoot()).thenReturn(rootSlot);
+			when(simpleSlot.getAllocatedSlot()).thenReturn(mockAllocatedSlot);
 
 			FlinkCompletableFuture<SimpleSlot> future = new FlinkCompletableFuture<>();
 			future.complete(simpleSlot);
@@ -175,7 +181,7 @@ public class ExecutionGraphMetricsTest extends TestLogger {
 	
 			// start execution
 			executionGraph.scheduleForExecution(scheduler);
-	
+
 			assertTrue(0L == restartingTime.getValue());
 	
 			List<ExecutionAttemptID> executionIDs = new ArrayList<>();

http://git-wip-us.apache.org/repos/asf/flink/blob/522edae3/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexDeploymentTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexDeploymentTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexDeploymentTest.java
index 8bc39a7..f3db5d8 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexDeploymentTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexDeploymentTest.java
@@ -20,6 +20,7 @@ package org.apache.flink.runtime.executiongraph;
 
 import org.apache.flink.api.common.time.Time;
 import org.apache.flink.runtime.akka.AkkaUtils;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
 import org.apache.flink.runtime.deployment.ResultPartitionDeploymentDescriptor;
 import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor;
 import org.apache.flink.runtime.execution.ExecutionState;
@@ -31,6 +32,7 @@ import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.jobgraph.ScheduleMode;
 import org.apache.flink.runtime.jobmanager.slots.ActorTaskManagerGateway;
+import org.apache.flink.runtime.jobmanager.slots.AllocatedSlot;
 import org.apache.flink.runtime.testingUtils.TestingUtils;
 import org.junit.Test;
 
@@ -357,10 +359,15 @@ public class ExecutionVertexDeploymentTest {
 		IntermediateResult result = new IntermediateResult(new IntermediateDataSetID(), jobVertex, 4, ResultPartitionType.PIPELINED);
 		ExecutionVertex vertex = new ExecutionVertex(jobVertex, 0, new IntermediateResult[]{result}, Time.minutes(1));
 
+		AllocatedSlot allocatedSlot = mock(AllocatedSlot.class);
+		when(allocatedSlot.getSlotAllocationId()).thenReturn(new AllocationID());
+
 		Slot root = mock(Slot.class);
 		when(root.getSlotNumber()).thenReturn(1);
 		SimpleSlot slot = mock(SimpleSlot.class);
 		when(slot.getRoot()).thenReturn(root);
+		when(slot.getAllocatedSlot()).thenReturn(allocatedSlot);
+		when(root.getAllocatedSlot()).thenReturn(allocatedSlot);
 
 		for (ScheduleMode mode : ScheduleMode.values()) {
 			vertex.getExecutionGraph().setScheduleMode(mode);

http://git-wip-us.apache.org/repos/asf/flink/blob/522edae3/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
index 33ed679..e654a99 100644
--- 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
@@ -1,135 +1,140 @@
-///*
-// * 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;
-//	}
-//}
+/*
+ * 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.apache.flink.runtime.jobmanager.slots.AllocatedSlot;
+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, allocation1);
+
+		allocatedSlots.add(slot1);
+
+		assertTrue(allocatedSlots.contains(slot1.getAllocatedSlot().getSlotAllocationId()));
+		assertTrue(allocatedSlots.containResource(resource1));
+
+		assertEquals(slot1, allocatedSlots.get(allocation1));
+		assertEquals(1, allocatedSlots.getSlotsForTaskManager(resource1).size());
+		assertEquals(1, allocatedSlots.size());
+
+		final AllocationID allocation2 = new AllocationID();
+		final Slot slot2 = createSlot(resource1, allocation2);
+
+		allocatedSlots.add(slot2);
+
+		assertTrue(allocatedSlots.contains(slot1.getAllocatedSlot().getSlotAllocationId()));
+		assertTrue(allocatedSlots.contains(slot2.getAllocatedSlot().getSlotAllocationId()));
+		assertTrue(allocatedSlots.containResource(resource1));
+
+		assertEquals(slot1, allocatedSlots.get(allocation1));
+		assertEquals(slot2, allocatedSlots.get(allocation2));
+		assertEquals(2, allocatedSlots.getSlotsForTaskManager(resource1).size());
+		assertEquals(2, allocatedSlots.size());
+
+		final AllocationID allocation3 = new AllocationID();
+		final ResourceID resource2 = new ResourceID("resource2");
+		final Slot slot3 = createSlot(resource2, allocation3);
+
+		allocatedSlots.add(slot3);
+
+		assertTrue(allocatedSlots.contains(slot1.getAllocatedSlot().getSlotAllocationId()));
+		assertTrue(allocatedSlots.contains(slot2.getAllocatedSlot().getSlotAllocationId()));
+		assertTrue(allocatedSlots.contains(slot3.getAllocatedSlot().getSlotAllocationId()));
+		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.getSlotsForTaskManager(resource1).size());
+		assertEquals(1, allocatedSlots.getSlotsForTaskManager(resource2).size());
+		assertEquals(3, allocatedSlots.size());
+
+		allocatedSlots.remove(slot2);
+
+		assertTrue(allocatedSlots.contains(slot1.getAllocatedSlot().getSlotAllocationId()));
+		assertFalse(allocatedSlots.contains(slot2.getAllocatedSlot().getSlotAllocationId()));
+		assertTrue(allocatedSlots.contains(slot3.getAllocatedSlot().getSlotAllocationId()));
+		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.getSlotsForTaskManager(resource1).size());
+		assertEquals(1, allocatedSlots.getSlotsForTaskManager(resource2).size());
+		assertEquals(2, allocatedSlots.size());
+
+		allocatedSlots.remove(slot1);
+
+		assertFalse(allocatedSlots.contains(slot1.getAllocatedSlot().getSlotAllocationId()));
+		assertFalse(allocatedSlots.contains(slot2.getAllocatedSlot().getSlotAllocationId()));
+		assertTrue(allocatedSlots.contains(slot3.getAllocatedSlot().getSlotAllocationId()));
+		assertFalse(allocatedSlots.containResource(resource1));
+		assertTrue(allocatedSlots.containResource(resource2));
+
+		assertNull(allocatedSlots.get(allocation1));
+		assertNull(allocatedSlots.get(allocation2));
+		assertEquals(slot3, allocatedSlots.get(allocation3));
+		assertEquals(0, allocatedSlots.getSlotsForTaskManager(resource1).size());
+		assertEquals(1, allocatedSlots.getSlotsForTaskManager(resource2).size());
+		assertEquals(1, allocatedSlots.size());
+
+		allocatedSlots.remove(slot3);
+
+		assertFalse(allocatedSlots.contains(slot1.getAllocatedSlot().getSlotAllocationId()));
+		assertFalse(allocatedSlots.contains(slot2.getAllocatedSlot().getSlotAllocationId()));
+		assertFalse(allocatedSlots.contains(slot3.getAllocatedSlot().getSlotAllocationId()));
+		assertFalse(allocatedSlots.containResource(resource1));
+		assertFalse(allocatedSlots.containResource(resource2));
+
+		assertNull(allocatedSlots.get(allocation1));
+		assertNull(allocatedSlots.get(allocation2));
+		assertNull(allocatedSlots.get(allocation3));
+		assertEquals(0, allocatedSlots.getSlotsForTaskManager(resource1).size());
+		assertEquals(0, allocatedSlots.getSlotsForTaskManager(resource2).size());
+		assertEquals(0, allocatedSlots.size());
+	}
+
+	private Slot createSlot(final ResourceID resourceId, final AllocationID allocationId) {
+		AllocatedSlot mockAllocatedSlot = mock(AllocatedSlot.class);
+		Slot slot = mock(Slot.class);
+		when(slot.getTaskManagerID()).thenReturn(resourceId);
+		when(slot.getAllocatedSlot()).thenReturn(mockAllocatedSlot);
+
+		when(mockAllocatedSlot.getSlotAllocationId()).thenReturn(allocationId);
+		return slot;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/522edae3/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
index 4d58a31..4ed88c4 100644
--- 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
@@ -1,123 +1,121 @@
-///*
-// * Licensed to the Apache Software Foundation (ASF) under one
-// * or more contributor license agreements.  See the NOTICE file
-// * distributed with this work for additional information
-// * regarding copyright ownership.  The ASF licenses this file
-// * to you under the Apache License, Version 2.0 (the
-// * "License"); you may not use this file except in compliance
-// * with the License.  You may obtain a copy of the License at
-// *
-// *     http://www.apache.org/licenses/LICENSE-2.0
-// *
-// * Unless required by applicable law or agreed to in writing, software
-// * distributed under the License is distributed on an "AS IS" BASIS,
-// * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-// * See the License for the specific language governing permissions and
-// * limitations under the License.
-// */
-//
-//package org.apache.flink.runtime.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));
-//	}
-//}
+/*
+ * 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.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.jobmanager.slots.AllocatedSlot;
+import org.apache.flink.runtime.jobmanager.slots.SlotAndLocality;
+import org.apache.flink.runtime.jobmanager.slots.TaskManagerGateway;
+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 AllocatedSlot slot1 = createAllocatedSlot(resource1);
+		final AllocatedSlot slot2 = createAllocatedSlot(resource1);
+		final AllocatedSlot slot3 = createAllocatedSlot(resource2);
+
+		availableSlots.add(slot1, 1L);
+		availableSlots.add(slot2, 2L);
+		availableSlots.add(slot3, 3L);
+
+		assertEquals(3, availableSlots.size());
+		assertTrue(availableSlots.contains(slot1.getSlotAllocationId()));
+		assertTrue(availableSlots.contains(slot2.getSlotAllocationId()));
+		assertTrue(availableSlots.contains(slot3.getSlotAllocationId()));
+		assertTrue(availableSlots.containsTaskManager(resource1));
+		assertTrue(availableSlots.containsTaskManager(resource2));
+
+		availableSlots.removeAllForTaskManager(resource1);
+
+		assertEquals(1, availableSlots.size());
+		assertFalse(availableSlots.contains(slot1.getSlotAllocationId()));
+		assertFalse(availableSlots.contains(slot2.getSlotAllocationId()));
+		assertTrue(availableSlots.contains(slot3.getSlotAllocationId()));
+		assertFalse(availableSlots.containsTaskManager(resource1));
+		assertTrue(availableSlots.containsTaskManager(resource2));
+
+		availableSlots.removeAllForTaskManager(resource2);
+
+		assertEquals(0, availableSlots.size());
+		assertFalse(availableSlots.contains(slot1.getSlotAllocationId()));
+		assertFalse(availableSlots.contains(slot2.getSlotAllocationId()));
+		assertFalse(availableSlots.contains(slot3.getSlotAllocationId()));
+		assertFalse(availableSlots.containsTaskManager(resource1));
+		assertFalse(availableSlots.containsTaskManager(resource2));
+	}
+
+	@Test
+	public void testPollFreeSlot() {
+		SlotPool.AvailableSlots availableSlots = new SlotPool.AvailableSlots();
+
+		final ResourceID resource1 = new ResourceID("resource1");
+		final AllocatedSlot slot1 = createAllocatedSlot(resource1);
+
+		availableSlots.add(slot1, 1L);
+
+		assertEquals(1, availableSlots.size());
+		assertTrue(availableSlots.contains(slot1.getSlotAllocationId()));
+		assertTrue(availableSlots.containsTaskManager(resource1));
+
+		assertNull(availableSlots.poll(DEFAULT_TESTING_BIG_PROFILE, null));
+
+		SlotAndLocality slotAndLocality = availableSlots.poll(DEFAULT_TESTING_PROFILE, null);
+		assertEquals(slot1, slotAndLocality.slot());
+		assertEquals(0, availableSlots.size());
+		assertFalse(availableSlots.contains(slot1.getSlotAllocationId()));
+		assertFalse(availableSlots.containsTaskManager(resource1));
+	}
+
+	static AllocatedSlot createAllocatedSlot(final ResourceID resourceId) {
+		TaskManagerLocation mockTaskManagerLocation = mock(TaskManagerLocation.class);
+		when(mockTaskManagerLocation.getResourceID()).thenReturn(resourceId);
+
+		TaskManagerGateway mockTaskManagerGateway = mock(TaskManagerGateway.class);
+
+		return new AllocatedSlot(
+			new AllocationID(),
+			new JobID(),
+			mockTaskManagerLocation,
+			0,
+			DEFAULT_TESTING_PROFILE,
+			mockTaskManagerGateway);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/522edae3/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
index cc1d194..5fa7af3 100644
--- 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
@@ -1,299 +1,309 @@
-///*
-// * 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.ExecutorService;
-//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 ExecutorService 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());
-//	}
-//
-//}
+/*
+ * 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.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.concurrent.BiFunction;
+import org.apache.flink.runtime.concurrent.Future;
+import org.apache.flink.runtime.jobmanager.scheduler.ScheduledUnit;
+import org.apache.flink.runtime.jobmanager.slots.AllocatedSlot;
+import org.apache.flink.runtime.jobmanager.slots.TaskManagerGateway;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway;
+import org.apache.flink.runtime.resourcemanager.SlotRequest;
+import org.apache.flink.runtime.rpc.MainThreadValidatorUtil;
+import org.apache.flink.runtime.rpc.RpcService;
+import org.apache.flink.runtime.rpc.TestingSerialRpcService;
+import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
+import org.apache.flink.util.TestLogger;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.ArgumentCaptor;
+
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.flink.runtime.instance.AvailableSlotsTest.DEFAULT_TESTING_PROFILE;
+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 RpcService rpcService;
+
+	private JobID jobId;
+
+	private MainThreadValidatorUtil mainThreadValidatorUtil;
+
+	private SlotPool slotPool;
+
+	private ResourceManagerGateway resourceManagerGateway;
+
+	@Before
+	public void setUp() throws Exception {
+
+		this.rpcService = new TestingSerialRpcService();
+		this.jobId = new JobID();
+		this.slotPool = new SlotPool(rpcService, jobId);
+
+		this.mainThreadValidatorUtil = new MainThreadValidatorUtil(slotPool);
+
+		mainThreadValidatorUtil.enterMainThread();
+
+		slotPool.start(UUID.randomUUID());
+
+		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.connectToResourceManager(UUID.randomUUID(), resourceManagerGateway);
+	}
+
+	@After
+	public void tearDown() throws Exception {
+		mainThreadValidatorUtil.exitMainThread();
+	}
+
+	@Test
+	public void testAllocateSimpleSlot() throws Exception {
+		ResourceID resourceID = new ResourceID("resource");
+		slotPool.registerTaskManager(resourceID);
+
+		ScheduledUnit task = mock(ScheduledUnit.class);
+		Future<SimpleSlot> future = slotPool.allocateSlot(task, DEFAULT_TESTING_PROFILE, null);
+		assertFalse(future.isDone());
+
+		ArgumentCaptor<SlotRequest> slotRequestArgumentCaptor = ArgumentCaptor.forClass(SlotRequest.class);
+		verify(resourceManagerGateway).requestSlot(any(UUID.class), any(UUID.class), slotRequestArgumentCaptor.capture(), any(Time.class));
+
+		final SlotRequest slotRequest = slotRequestArgumentCaptor.getValue();
+
+		AllocatedSlot allocatedSlot = createAllocatedSlot(resourceID, slotRequest.getAllocationId(), jobId, DEFAULT_TESTING_PROFILE);
+		assertTrue(slotPool.offerSlot(allocatedSlot));
+
+		SimpleSlot slot = future.get(1, TimeUnit.SECONDS);
+		assertTrue(future.isDone());
+		assertTrue(slot.isAlive());
+		assertEquals(resourceID, slot.getTaskManagerID());
+		assertEquals(jobId, slot.getJobID());
+		assertEquals(slotPool.getSlotOwner(), slot.getOwner());
+	}
+
+	@Test
+	public void testAllocateSlotWithoutResourceManager() throws Exception {
+		slotPool.disconnectResourceManager();
+		Future<SimpleSlot> future = slotPool.allocateSlot(mock(ScheduledUnit.class),DEFAULT_TESTING_PROFILE, null);
+		future.handleAsync(
+			new BiFunction<SimpleSlot, Throwable, Void>() {
+				@Override
+				public Void apply(SimpleSlot simpleSlot, Throwable throwable) {
+					assertNull(simpleSlot);
+					assertNotNull(throwable);
+					return null;
+				}
+			},
+			rpcService.getExecutor());
+		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.registerTaskManager(resourceID);
+
+		Future<SimpleSlot> future1 = slotPool.allocateSlot(mock(ScheduledUnit.class),DEFAULT_TESTING_PROFILE, null);
+		Future<SimpleSlot> future2 = slotPool.allocateSlot(mock(ScheduledUnit.class),DEFAULT_TESTING_PROFILE, null);
+
+		assertFalse(future1.isDone());
+		assertFalse(future2.isDone());
+
+		ArgumentCaptor<SlotRequest> slotRequestArgumentCaptor = ArgumentCaptor.forClass(SlotRequest.class);
+		verify(resourceManagerGateway, times(2))
+			.requestSlot(any(UUID.class), any(UUID.class), slotRequestArgumentCaptor.capture(), any(Time.class));
+
+		final List<SlotRequest> slotRequests = slotRequestArgumentCaptor.getAllValues();
+
+		AllocatedSlot allocatedSlot = createAllocatedSlot(resourceID, slotRequests.get(0).getAllocationId(), jobId, DEFAULT_TESTING_PROFILE);
+		assertTrue(slotPool.offerSlot(allocatedSlot));
+
+		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.registerTaskManager(resourceID);
+
+		Future<SimpleSlot> future1 = slotPool.allocateSlot(mock(ScheduledUnit.class),DEFAULT_TESTING_PROFILE, null);
+		assertFalse(future1.isDone());
+
+		ArgumentCaptor<SlotRequest> slotRequestArgumentCaptor = ArgumentCaptor.forClass(SlotRequest.class);
+		verify(resourceManagerGateway).requestSlot(any(UUID.class), any(UUID.class), slotRequestArgumentCaptor.capture(), any(Time.class));
+
+		final SlotRequest slotRequest = slotRequestArgumentCaptor.getValue();
+
+		AllocatedSlot allocatedSlot = createAllocatedSlot(resourceID, slotRequest.getAllocationId(), jobId, DEFAULT_TESTING_PROFILE);
+		assertTrue(slotPool.offerSlot(allocatedSlot));
+
+		SimpleSlot slot1 = future1.get(1, TimeUnit.SECONDS);
+		assertTrue(future1.isDone());
+
+		// return this slot to pool
+		slot1.releaseSlot();
+
+		Future<SimpleSlot> future2 = slotPool.allocateSlot(mock(ScheduledUnit.class),DEFAULT_TESTING_PROFILE, null);
+
+		// 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.registerTaskManager(resourceID);
+
+		Future<SimpleSlot> future = slotPool.allocateSlot(mock(ScheduledUnit.class),DEFAULT_TESTING_PROFILE, null);
+		assertFalse(future.isDone());
+
+		ArgumentCaptor<SlotRequest> slotRequestArgumentCaptor = ArgumentCaptor.forClass(SlotRequest.class);
+		verify(resourceManagerGateway).requestSlot(any(UUID.class), any(UUID.class), slotRequestArgumentCaptor.capture(), any(Time.class));
+
+		final SlotRequest slotRequest = slotRequestArgumentCaptor.getValue();
+
+		// slot from unregistered resource
+		AllocatedSlot invalid = createAllocatedSlot(new ResourceID("unregistered"), slotRequest.getAllocationId(), jobId, DEFAULT_TESTING_PROFILE);
+		assertFalse(slotPool.offerSlot(invalid));
+
+		AllocatedSlot notRequested = createAllocatedSlot(resourceID, new AllocationID(), jobId, DEFAULT_TESTING_PROFILE);
+
+		// we'll also accept non requested slots
+		assertTrue(slotPool.offerSlot(notRequested));
+
+		AllocatedSlot allocatedSlot = createAllocatedSlot(resourceID, slotRequest.getAllocationId(), jobId, DEFAULT_TESTING_PROFILE);
+
+		// accepted slot
+		assertTrue(slotPool.offerSlot(allocatedSlot));
+		SimpleSlot slot = future.get(1, TimeUnit.SECONDS);
+		assertTrue(future.isDone());
+		assertTrue(slot.isAlive());
+
+		// duplicated offer with using slot
+		assertTrue(slotPool.offerSlot(allocatedSlot));
+		assertTrue(future.isDone());
+		assertTrue(slot.isAlive());
+
+		// duplicated offer with free slot
+		slot.releaseSlot();
+		assertTrue(slot.isReleased());
+		assertTrue(slotPool.offerSlot(allocatedSlot));
+	}
+
+	@Test
+	public void testReleaseResource() throws Exception {
+		ResourceID resourceID = new ResourceID("resource");
+		slotPool.registerTaskManager(resourceID);
+
+		Future<SimpleSlot> future1 = slotPool.allocateSlot(mock(ScheduledUnit.class),DEFAULT_TESTING_PROFILE, null);
+
+		ArgumentCaptor<SlotRequest> slotRequestArgumentCaptor = ArgumentCaptor.forClass(SlotRequest.class);
+		verify(resourceManagerGateway).requestSlot(any(UUID.class), any(UUID.class), slotRequestArgumentCaptor.capture(), any(Time.class));
+
+		final SlotRequest slotRequest = slotRequestArgumentCaptor.getValue();
+
+		Future<SimpleSlot> future2 = slotPool.allocateSlot(mock(ScheduledUnit.class),DEFAULT_TESTING_PROFILE, null);
+
+		AllocatedSlot allocatedSlot = createAllocatedSlot(resourceID, slotRequest.getAllocationId(), jobId, DEFAULT_TESTING_PROFILE);
+		assertTrue(slotPool.offerSlot(allocatedSlot));
+
+		SimpleSlot slot1 = future1.get(1, TimeUnit.SECONDS);
+		assertTrue(future1.isDone());
+		assertFalse(future2.isDone());
+
+		slotPool.releaseTaskManager(resourceID);
+		assertTrue(slot1.isReleased());
+
+		// slot released and not usable, second allocation still not fulfilled
+		Thread.sleep(10);
+		assertFalse(future2.isDone());
+	}
+
+	static AllocatedSlot createAllocatedSlot(
+			final ResourceID resourceId,
+			final AllocationID allocationId,
+			final JobID jobId,
+			final ResourceProfile resourceProfile) {
+		TaskManagerLocation mockTaskManagerLocation = mock(TaskManagerLocation.class);
+		when(mockTaskManagerLocation.getResourceID()).thenReturn(resourceId);
+
+		TaskManagerGateway mockTaskManagerGateway = mock(TaskManagerGateway.class);
+
+		return new AllocatedSlot(
+			allocationId,
+			jobId,
+			mockTaskManagerLocation,
+			0,
+			resourceProfile,
+			mockTaskManagerGateway);
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/522edae3/flink-runtime/src/test/java/org/apache/flink/runtime/minicluster/MiniClusterITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/minicluster/MiniClusterITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/minicluster/MiniClusterITCase.java
index f5b3892..2cf2d4d 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/minicluster/MiniClusterITCase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/minicluster/MiniClusterITCase.java
@@ -45,7 +45,7 @@ public class MiniClusterITCase extends TestLogger {
 		executeJob(miniCluster);
 	}
 
-//	@Test
+	@Test
 	public void runJobWithMultipleRpcServices() throws Exception {
 		MiniClusterConfiguration cfg = new MiniClusterConfiguration();
 		cfg.setUseRpcServicePerComponent();
@@ -54,7 +54,7 @@ public class MiniClusterITCase extends TestLogger {
 		executeJob(miniCluster);
 	}
 
-//	@Test
+	@Test
 	public void runJobWithMultipleJobManagers() throws Exception {
 		MiniClusterConfiguration cfg = new MiniClusterConfiguration();
 		cfg.setNumJobManagers(3);

http://git-wip-us.apache.org/repos/asf/flink/blob/522edae3/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingSerialRpcService.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingSerialRpcService.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingSerialRpcService.java
index 88906a7..1d30ea4 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingSerialRpcService.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingSerialRpcService.java
@@ -18,7 +18,6 @@
 
 package org.apache.flink.runtime.rpc;
 
-import akka.dispatch.Futures;
 import org.apache.flink.api.common.time.Time;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.runtime.concurrent.CompletableFuture;
@@ -210,9 +209,9 @@ public class TestingSerialRpcService implements RpcService {
 				if (returnType.equals(Future.class)) {
 					try {
 						Object result = handleRpcInvocationSync(methodName, filteredArguments.f0, filteredArguments.f1, futureTimeout);
-						return Futures.successful(result);
+						return FlinkCompletableFuture.completed(result);
 					} catch (Throwable e) {
-						return Futures.failed(e);
+						return FlinkCompletableFuture.completedExceptionally(e);
 					}
 				} else {
 					return handleRpcInvocationSync(methodName, filteredArguments.f0, filteredArguments.f1, futureTimeout);


[17/52] [abbrv] flink git commit: [FLINK-4847] Let RpcEndpoint.start/shutDown throw exceptions

Posted by se...@apache.org.
[FLINK-4847] Let RpcEndpoint.start/shutDown throw exceptions

Allowing the RpcEndpoint.start/shutDown to throw exceptions will help to let rpc endpoints
to quickly fail without having to use a callback like the FatalErrorHandler.

This closes #2651.


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

Branch: refs/heads/master
Commit: a1934255421b97eefd579183e9c7199c43ad1a2c
Parents: 3aafa16
Author: Till Rohrmann <tr...@apache.org>
Authored: Mon Oct 17 16:22:16 2016 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Dec 23 20:54:24 2016 +0100

----------------------------------------------------------------------
 .../flink/runtime/jobmaster/JobManagerRunner.java     |  6 +++++-
 .../org/apache/flink/runtime/jobmaster/JobMaster.java | 14 ++++++++++----
 .../org/apache/flink/runtime/rpc/RpcEndpoint.java     |  8 ++++++--
 .../flink/runtime/taskexecutor/TaskExecutor.java      |  2 +-
 .../flink/runtime/taskexecutor/TaskManagerRunner.java |  2 +-
 .../flink/runtime/rpc/akka/AkkaRpcActorTest.java      |  7 +------
 .../runtime/rpc/akka/MainThreadValidationTest.java    |  2 +-
 .../runtime/rpc/akka/MessageSerializationTest.java    |  2 +-
 .../flink/runtime/taskexecutor/TaskExecutorTest.java  |  4 ++--
 9 files changed, 28 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/a1934255/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 3313d8a..9d8e004 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
@@ -374,7 +374,11 @@ public class JobManagerRunner implements LeaderContender, OnCompletionActions, F
 			// This will eventually be noticed, but can not be ruled out from the beginning.
 			if (leaderElectionService.hasLeadership()) {
 				if (jobRunning) {
-					jobManager.start(leaderSessionID);
+					try {
+						jobManager.start(leaderSessionID);
+					} catch (Exception e) {
+						onFatalError(new Exception("Could not start the job manager.", e));
+					}
 				} else {
 					log.info("Job {} ({}) already finished by others.", jobGraph.getName(), jobGraph.getJobID());
 					jobFinishedByOther();

http://git-wip-us.apache.org/repos/asf/flink/blob/a1934255/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 204cd80..c80cc51 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
@@ -267,7 +267,7 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 	 *
 	 * @param leaderSessionID The necessary leader id for running the job.
 	 */
-	public void start(final UUID leaderSessionID) {
+	public void start(final UUID leaderSessionID) throws Exception {
 		if (LEADER_ID_UPDATER.compareAndSet(this, null, leaderSessionID)) {
 			super.start();
 
@@ -283,7 +283,7 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 	 * Suspend the job and shutdown all other services including rpc.
 	 */
 	@Override
-	public void shutDown() {
+	public void shutDown() throws Exception {
 		// make sure there is a graceful exit
 		getSelf().suspendExecution(new Exception("JobManager is shutting down."));
 		super.shutDown();
@@ -382,7 +382,7 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 		try {
 			resourceManagerLeaderRetriever.stop();
 		} catch (Exception e) {
-			log.warn("Failed to stop resource manager leader retriever when suspending.");
+			log.warn("Failed to stop resource manager leader retriever when suspending.", e);
 		}
 		closeResourceManagerConnection();
 
@@ -761,7 +761,13 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 			@Override
 			public void run() {
 				log.error("Fatal error occurred on JobManager, cause: {}", cause.getMessage(), cause);
-				shutDown();
+
+				try {
+					shutDown();
+				} catch (Exception e) {
+					cause.addSuppressed(e);
+				}
+
 				errorHandler.onFatalError(cause);
 			}
 		});

http://git-wip-us.apache.org/repos/asf/flink/blob/a1934255/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java
index f93a2e2..b971b96 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java
@@ -110,8 +110,10 @@ public abstract class RpcEndpoint<C extends RpcGateway> {
 	 *
 	 * IMPORTANT: Whenever you override this method, call the parent implementation to enable
 	 * rpc processing. It is advised to make the parent call last.
+	 *
+	 * @throws Exception indicating that something went wrong while starting the RPC endpoint
 	 */
-	public void start() {
+	public void start() throws Exception {
 		((StartStoppable) self).start();
 	}
 
@@ -123,8 +125,10 @@ public abstract class RpcEndpoint<C extends RpcGateway> {
 	 * 
 	 * <p>This method can be overridden to add RPC endpoint specific shut down code.
 	 * The overridden method should always call the parent shut down method.
+	 *
+	 * @throws Exception indicating that the something went wrong while shutting the RPC endpoint down
 	 */
-	public void shutDown() {
+	public void shutDown() throws Exception {
 		rpcService.stopServer(self);
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/a1934255/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 679324b..8187fde 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
@@ -196,7 +196,7 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 	// ------------------------------------------------------------------------
 
 	@Override
-	public void start() {
+	public void start() throws Exception {
 		super.start();
 
 		// start by connecting to the ResourceManager

http://git-wip-us.apache.org/repos/asf/flink/blob/a1934255/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java
index f56d17c..7d9ee55 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java
@@ -107,7 +107,7 @@ public class TaskManagerRunner implements FatalErrorHandler {
 	//  Lifecycle management
 	// --------------------------------------------------------------------------------------------
 
-	public void start() {
+	public void start() throws Exception {
 		taskManager.start();
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/a1934255/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorTest.java
index ba8eb11..d2dbab7 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorTest.java
@@ -22,7 +22,6 @@ import akka.actor.ActorSystem;
 import org.apache.flink.api.common.time.Time;
 import org.apache.flink.runtime.akka.AkkaUtils;
 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.rpc.RpcEndpoint;
 import org.apache.flink.runtime.rpc.RpcGateway;
@@ -36,10 +35,6 @@ import org.junit.Test;
 
 import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutionException;
-import java.util.concurrent.ForkJoinPool;
-import java.util.concurrent.ThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
@@ -163,7 +158,7 @@ public class AkkaRpcActorTest extends TestLogger {
 	 * @throws InterruptedException
 	 */
 	@Test(timeout=1000)
-	public void testRpcEndpointTerminationFuture() throws ExecutionException, InterruptedException {
+	public void testRpcEndpointTerminationFuture() throws Exception {
 		final DummyRpcEndpoint rpcEndpoint = new DummyRpcEndpoint(akkaRpcService);
 		rpcEndpoint.start();
 

http://git-wip-us.apache.org/repos/asf/flink/blob/a1934255/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/MainThreadValidationTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/MainThreadValidationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/MainThreadValidationTest.java
index 9ec1f7e..9f134d8 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/MainThreadValidationTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/MainThreadValidationTest.java
@@ -34,7 +34,7 @@ import static org.junit.Assert.assertTrue;
 public class MainThreadValidationTest extends TestLogger {
 
 	@Test
-	public void failIfNotInMainThread() {
+	public void failIfNotInMainThread() throws Exception {
 		// test if assertions are activated. The test only works if assertions are loaded.
 		try {
 			assert false;

http://git-wip-us.apache.org/repos/asf/flink/blob/a1934255/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/MessageSerializationTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/MessageSerializationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/MessageSerializationTest.java
index 0d5dc28..d640a97 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/MessageSerializationTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/MessageSerializationTest.java
@@ -80,7 +80,7 @@ public class MessageSerializationTest extends TestLogger {
 	 * Tests that a local rpc call with a non serializable argument can be executed.
 	 */
 	@Test
-	public void testNonSerializableLocalMessageTransfer() throws InterruptedException, IOException {
+	public void testNonSerializableLocalMessageTransfer() throws Exception {
 		LinkedBlockingQueue<Object> linkedBlockingQueue = new LinkedBlockingQueue<>();
 		TestEndpoint testEndpoint = new TestEndpoint(akkaRpcService1, linkedBlockingQueue);
 		testEndpoint.start();

http://git-wip-us.apache.org/repos/asf/flink/blob/a1934255/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java
index 4d73a4b..638ec56 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java
@@ -354,7 +354,7 @@ public class TaskExecutorTest extends TestLogger {
 	 * the job leader, it will offer all reserved slots to the JobManager.
 	 */
 	@Test
-	public void testJobLeaderDetection() throws TestingFatalErrorHandler.TestingException, SlotAllocationException {
+	public void testJobLeaderDetection() throws Exception {
 		final JobID jobId = new JobID();
 
 		final TestingSerialRpcService rpc = new TestingSerialRpcService();
@@ -621,7 +621,7 @@ public class TaskExecutorTest extends TestLogger {
 	 */
 	@Ignore
 	@Test
-	public void testRejectAllocationRequestsForOutOfSyncSlots() throws SlotAllocationException {
+	public void testRejectAllocationRequestsForOutOfSyncSlots() throws Exception {
 		final ResourceID resourceID = ResourceID.generate();
 
 		final String address1 = "/resource/manager/address/one";


[44/52] [abbrv] flink git commit: [FLINK-4930] [client] [yarn] delete tmp file of job graph and refine finalizeCluster

Posted by se...@apache.org.
[FLINK-4930] [client] [yarn] delete tmp file of job graph and refine finalizeCluster


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

Branch: refs/heads/master
Commit: afa271756036c668924aac41f5786efc2118717e
Parents: 3695a8e
Author: shuai.xus <sh...@alibaba-inc.com>
Authored: Thu Nov 24 16:58:14 2016 +0800
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Dec 23 20:54:27 2016 +0100

----------------------------------------------------------------------
 .../org/apache/flink/yarn/AbstractYarnClusterDescriptor.java    | 5 +++--
 .../main/java/org/apache/flink/yarn/YarnClusterClientV2.java    | 2 +-
 2 files changed, 4 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/afa27175/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 b4c87b8..0d1239d 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
@@ -716,10 +716,11 @@ public abstract class AbstractYarnClusterDescriptor implements ClusterDescriptor
 		classPathBuilder.append("flink-conf.yaml").append(File.pathSeparator);
 
 		// write job graph to tmp file and add it to local resource 
-		// TODO: need refine ?
+		// TODO: server use user main method to generate job graph 
 		if (jobGraph != null) {
 			try {
-				File fp = new File("/tmp/jobgraph-" + appId.toString());
+				File fp = File.createTempFile(appId.toString(), null);
+				fp.deleteOnExit();
 				FileOutputStream input = new FileOutputStream(fp);
 				ObjectOutputStream obInput = new ObjectOutputStream(input);
 				obInput.writeObject(jobGraph);

http://git-wip-us.apache.org/repos/asf/flink/blob/afa27175/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterClientV2.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterClientV2.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterClientV2.java
index daa2c3b..e9c6636 100644
--- a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterClientV2.java
+++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterClientV2.java
@@ -143,7 +143,7 @@ public class YarnClusterClientV2 extends ClusterClient {
 
 	@Override
 	public void finalizeCluster() {
-		throw new UnsupportedOperationException("Not support finalizeCluster since Flip-6.");
+		// Do nothing
 	}
 
 	@Override


[06/52] [abbrv] flink git commit: [FLINK-4839] [cluster management] JobManager handle TaskManager's slot offering

Posted by se...@apache.org.
[FLINK-4839] [cluster management] JobManager handle TaskManager's slot offering

This closes #2647 #2643.


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

Branch: refs/heads/master
Commit: af924b489420b8d7163e6216c0efb05e3ab30514
Parents: a7ed9a5
Author: Kurt Young <yk...@gmail.com>
Authored: Mon Oct 17 18:15:26 2016 +0800
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Dec 23 20:54:23 2016 +0100

----------------------------------------------------------------------
 .../apache/flink/runtime/instance/SlotPool.java | 46 +++++++++---
 .../flink/runtime/jobmaster/JobMaster.java      | 50 ++++++++++++-
 .../runtime/jobmaster/JobMasterGateway.java     | 26 +++++--
 .../resourcemanager/ResourceManager.java        |  2 +-
 .../runtime/taskexecutor/TaskExecutor.java      | 34 +++++----
 .../runtime/taskexecutor/slot/SlotOffer.java    | 79 ++++++++++++++++++++
 .../runtime/taskexecutor/slot/TaskSlot.java     | 13 ++++
 .../taskexecutor/slot/TaskSlotTable.java        | 12 +--
 .../runtime/taskexecutor/TaskExecutorTest.java  | 16 +++-
 9 files changed, 231 insertions(+), 47 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/af924b48/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
index 7e7b21e..44df29b 100644
--- 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
@@ -267,7 +267,7 @@ public class SlotPool implements SlotOwner {
 	}
 
 	// ------------------------------------------------------------------------
-	//  Slot De-allocation
+	//  Slot releasing & offering
 	// ------------------------------------------------------------------------
 
 	/**
@@ -323,10 +323,6 @@ public class SlotPool implements SlotOwner {
 		return null;
 	}
 
-	// ------------------------------------------------------------------------
-	//  Slot Releasing
-	// ------------------------------------------------------------------------
-
 	/**
 	 * Release slot to TaskManager, called for finished tasks or canceled jobs.
 	 *
@@ -340,10 +336,6 @@ public class SlotPool implements SlotOwner {
 		}
 	}
 
-	// ------------------------------------------------------------------------
-	//  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
@@ -401,6 +393,39 @@ public class SlotPool implements SlotOwner {
 	}
 
 	// ------------------------------------------------------------------------
+	//  Error Handling
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Fail the specified allocation and release the corresponding slot if we have one.
+	 * This may triggered by JobManager when some slot allocation failed with timeout.
+	 * Or this could be triggered by TaskManager, when it finds out something went wrong with the slot,
+	 * and decided to take it back.
+	 *
+	 * @param allocationID Represents the allocation which should be failed
+	 * @param cause        The cause of the failure
+	 */
+	public void failAllocation(final AllocationID allocationID, final Exception cause) {
+		synchronized (lock) {
+			// 1. check whether the allocation still pending
+			Tuple2<SlotRequest, FlinkCompletableFuture<SlotDescriptor>> pendingRequest =
+					pendingRequests.get(allocationID);
+			if (pendingRequest != null) {
+				pendingRequest.f1.completeExceptionally(cause);
+				return;
+			}
+
+			// 2. check whether we have a free slot corresponding to this allocation id
+			// TODO: add allocation id to slot descriptor, so we can remove it by allocation id
+
+			// 3. check whether we have a in-use slot corresponding to this allocation id
+			// TODO: needs mechanism to release the in-use Slot but don't return it back to this pool
+
+			// TODO: add some unit tests when the previous two are ready, the allocation may failed at any phase
+		}
+	}
+
+	// ------------------------------------------------------------------------
 	//  Resource
 	// ------------------------------------------------------------------------
 
@@ -464,12 +489,13 @@ public class SlotPool implements SlotOwner {
 	 */
 	static class AllocatedSlots {
 
-		/** All allocated slots organized by TaskManager */
+		/** All allocated slots organized by TaskManager's id */
 		private final Map<ResourceID, Set<Slot>> allocatedSlotsByResource;
 
 		/** All allocated slots organized by Slot object */
 		private final Map<Slot, AllocationID> allocatedSlots;
 
+		/** All allocated slot descriptors organized by Slot object */
 		private final Map<Slot, SlotDescriptor> allocatedSlotsWithDescriptor;
 
 		/** All allocated slots organized by AllocationID */

http://git-wip-us.apache.org/repos/asf/flink/blob/af924b48/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 7bcfb3a..3c6bbd3 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
@@ -53,6 +53,7 @@ import org.apache.flink.runtime.executiongraph.restart.RestartStrategy;
 import org.apache.flink.runtime.executiongraph.restart.RestartStrategyFactory;
 import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
 import org.apache.flink.runtime.instance.Slot;
+import org.apache.flink.runtime.instance.SlotDescriptor;
 import org.apache.flink.runtime.instance.SlotPool;
 import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
 import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
@@ -85,6 +86,7 @@ import org.apache.flink.runtime.rpc.RpcService;
 import org.apache.flink.runtime.rpc.StartStoppable;
 import org.apache.flink.runtime.state.KeyGroupRange;
 import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway;
+import org.apache.flink.runtime.taskexecutor.slot.SlotOffer;
 import org.apache.flink.runtime.taskmanager.TaskExecutionState;
 import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
 import org.apache.flink.runtime.util.SerializedThrowable;
@@ -95,7 +97,9 @@ import org.slf4j.Logger;
 import javax.annotation.Nullable;
 import java.io.IOException;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.Map;
+import java.util.Set;
 import java.util.UUID;
 import java.util.concurrent.Callable;
 import java.util.concurrent.Executor;
@@ -663,13 +667,51 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 	}
 
 	@RpcMethod
-	public Iterable<AllocationID> offerSlots(final Iterable<AllocationID> slots, UUID leaderId) {
-		throw new UnsupportedOperationException("Has to be implemented.");
+	public Iterable<SlotOffer> offerSlots(final ResourceID taskManagerId,
+			final Iterable<SlotOffer> slots, final UUID leaderId) throws Exception
+	{
+		if (!this.leaderSessionID.equals(leaderId)) {
+			throw new Exception("Leader id not match, expected: " + this.leaderSessionID
+					+ ", actual: " + leaderId);
+		}
+
+		Tuple2<TaskManagerLocation, TaskExecutorGateway> taskManager = registeredTaskManagers.get(taskManagerId);
+		if (taskManager == null) {
+			throw new Exception("Unknown TaskManager " + taskManagerId);
+		}
+
+		final Set<SlotOffer> acceptedSlotOffers = new HashSet<>(4);
+		for (SlotOffer slotOffer : slots) {
+			final SlotDescriptor slotDescriptor = new SlotDescriptor(
+					jobGraph.getJobID(),
+					taskManager.f0,
+					slotOffer.getSlotIndex(),
+					slotOffer.getResourceProfile(),
+					null); // TODO: replace the actor gateway with the new rpc gateway, it's ready (taskManager.f1)
+			if (slotPool.offerSlot(slotOffer.getAllocationId(), slotDescriptor)) {
+				acceptedSlotOffers.add(slotOffer);
+			}
+		}
+
+		return acceptedSlotOffers;
 	}
 
 	@RpcMethod
-	public void failSlot(final AllocationID allocationId, UUID leaderId, Exception cause) {
-		throw new UnsupportedOperationException("Has to be implemented.");
+	public void failSlot(final ResourceID taskManagerId,
+			final AllocationID allocationId,
+			final UUID leaderId,
+			final Exception cause) throws Exception
+	{
+		if (!this.leaderSessionID.equals(leaderId)) {
+			throw new Exception("Leader id not match, expected: " + this.leaderSessionID
+					+ ", actual: " + leaderId);
+		}
+
+		if (!registeredTaskManagers.containsKey(taskManagerId)) {
+			throw new Exception("Unknown TaskManager " + taskManagerId);
+		}
+
+		slotPool.failAllocation(allocationId, cause);
 	}
 
 	@RpcMethod

http://git-wip-us.apache.org/repos/asf/flink/blob/af924b48/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 8925d94..2d7ebb9 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
@@ -40,6 +40,7 @@ import org.apache.flink.runtime.registration.RegistrationResponse;
 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.taskexecutor.slot.SlotOffer;
 import org.apache.flink.runtime.taskmanager.TaskExecutionState;
 import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
 
@@ -166,21 +167,30 @@ public interface JobMasterGateway extends CheckpointCoordinatorGateway {
 	/**
 	 * Offer the given slots to the job manager. The response contains the set of accepted slots.
 	 *
-	 * @param slots to offer to the job manager
-	 * @param leaderId identifying the job leader
-	 * @param timeout for the rpc call
+	 * @param taskManagerId identifying the task manager
+	 * @param slots         to offer to the job manager
+	 * @param leaderId      identifying the job leader
+	 * @param timeout       for the rpc call
 	 * @return Future set of accepted slots.
 	 */
-	Future<Iterable<AllocationID>> offerSlots(final Iterable<AllocationID> slots, UUID leaderId, @RpcTimeout final Time timeout);
+	Future<Iterable<SlotOffer>> offerSlots(
+			final ResourceID taskManagerId,
+			final Iterable<SlotOffer> slots,
+			final UUID leaderId,
+			@RpcTimeout final Time timeout);
 
 	/**
 	 * Fail the slot with the given allocation id and cause.
 	 *
-	 * @param allocationId identifying the slot to fail
-	 * @param leaderId identifying the job leader
-	 * @param cause of the failing
+	 * @param taskManagerId identifying the task manager
+	 * @param allocationId  identifying the slot to fail
+	 * @param leaderId      identifying the job leader
+	 * @param cause         of the failing
 	 */
-	void failSlot(final AllocationID allocationId, UUID leaderId, Exception cause);
+	void failSlot(final ResourceID taskManagerId,
+			final AllocationID allocationId,
+			final UUID leaderId,
+			final Exception cause);
 
 	/**
 	 * Register the task manager at the job manager.

http://git-wip-us.apache.org/repos/asf/flink/blob/af924b48/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java
index 3122804..f1a5073 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java
@@ -69,7 +69,7 @@ import static org.apache.flink.util.Preconditions.checkNotNull;
  * ResourceManager implementation. The resource manager is responsible for resource de-/allocation
  * and bookkeeping.
  *
- * It offers the following methods as part of its rpc interface to interact with the him remotely:
+ * It offers the following methods as part of its rpc interface to interact with him remotely:
  * <ul>
  *     <li>{@link #registerJobMaster(UUID, UUID, String, JobID)} registers a {@link JobMaster} at the resource manager</li>
  *     <li>{@link #requestSlot(UUID, UUID, SlotRequest)} requests a slot from the resource manager</li>

http://git-wip-us.apache.org/repos/asf/flink/blob/af924b48/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 5146e5b..679324b 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
@@ -72,6 +72,8 @@ import org.apache.flink.runtime.taskexecutor.rpc.RpcResultPartitionConsumableNot
 import org.apache.flink.runtime.taskexecutor.slot.SlotActions;
 import org.apache.flink.runtime.taskexecutor.slot.SlotNotActiveException;
 import org.apache.flink.runtime.taskexecutor.slot.SlotNotFoundException;
+import org.apache.flink.runtime.taskexecutor.slot.SlotOffer;
+import org.apache.flink.runtime.taskexecutor.slot.TaskSlot;
 import org.apache.flink.runtime.taskexecutor.slot.TaskSlotTable;
 import org.apache.flink.runtime.taskmanager.CheckpointResponder;
 import org.apache.flink.runtime.taskmanager.Task;
@@ -660,47 +662,49 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 
 				final JobMasterGateway jobMasterGateway = jobManagerConnection.getJobManagerGateway();
 
-				final Iterator<AllocationID> reservedSlotsIterator = taskSlotTable.getAllocatedSlots(jobId);
+				final Iterator<TaskSlot> reservedSlotsIterator = taskSlotTable.getAllocatedSlots(jobId);
 				final UUID leaderId = jobManagerConnection.getLeaderId();
 
-				final Collection<AllocationID> reservedSlots = new HashSet<>(2);
+				final Collection<SlotOffer> reservedSlots = new HashSet<>(2);
 
 				while (reservedSlotsIterator.hasNext()) {
-					reservedSlots.add(reservedSlotsIterator.next());
+					reservedSlots.add(reservedSlotsIterator.next().generateSlotOffer());
 				}
 
-				Future<Iterable<AllocationID>> acceptedSlotsFuture = jobMasterGateway.offerSlots(
+				Future<Iterable<SlotOffer>> acceptedSlotsFuture = jobMasterGateway.offerSlots(
+					getResourceID(),
 					reservedSlots,
 					leaderId,
 					taskManagerConfiguration.getTimeout());
 
-				acceptedSlotsFuture.thenAcceptAsync(new AcceptFunction<Iterable<AllocationID>>() {
+				acceptedSlotsFuture.thenAcceptAsync(new AcceptFunction<Iterable<SlotOffer>>() {
 					@Override
-					public void accept(Iterable<AllocationID> acceptedSlots) {
+					public void accept(Iterable<SlotOffer> acceptedSlots) {
 						// check if the response is still valid
 						if (isJobManagerConnectionValid(jobId, leaderId)) {
 							// mark accepted slots active
-							for (AllocationID acceptedSlot: acceptedSlots) {
+							for (SlotOffer acceptedSlot: acceptedSlots) {
 								try {
-									if (!taskSlotTable.markSlotActive(acceptedSlot)) {
+									if (!taskSlotTable.markSlotActive(acceptedSlot.getAllocationId())) {
 										// the slot is either free or releasing at the moment
 										final String message = "Could not mark slot " + jobId + " active.";
 										log.debug(message);
-										jobMasterGateway.failSlot(acceptedSlot, leaderId, new Exception(message));
+										jobMasterGateway.failSlot(getResourceID(), acceptedSlot.getAllocationId(),
+												leaderId, new Exception(message));
 									}
 
 									// remove the assigned slots so that we can free the left overs
 									reservedSlots.remove(acceptedSlot);
 								} catch (SlotNotFoundException e) {
 									log.debug("Could not mark slot {} active.", acceptedSlot,  e);
-									jobMasterGateway.failSlot(acceptedSlot, leaderId, e);
+									jobMasterGateway.failSlot(getResourceID(), acceptedSlot.getAllocationId(), leaderId, e);
 								}
 							}
 
 							final Exception e = new Exception("The slot was rejected by the JobManager.");
 
-							for (AllocationID rejectedSlot: reservedSlots) {
-								freeSlot(rejectedSlot, e);
+							for (SlotOffer rejectedSlot: reservedSlots) {
+								freeSlot(rejectedSlot.getAllocationId(), e);
 							}
 						} else {
 							// discard the response since there is a new leader for the job
@@ -718,8 +722,8 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 							offerSlotsToJobManager(jobId);
 						} else {
 							// We encountered an exception. Free the slots and return them to the RM.
-							for (AllocationID reservedSlot: reservedSlots) {
-								freeSlot(reservedSlot, throwable);
+							for (SlotOffer reservedSlot: reservedSlots) {
+								freeSlot(reservedSlot.getAllocationId(), throwable);
 							}
 						}
 
@@ -870,7 +874,7 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 
 	private void unregisterTaskAndNotifyFinalState(
 			final UUID jobMasterLeaderId,
-			final JobMasterGateway jobMasterGateway,		
+			final JobMasterGateway jobMasterGateway,
 			final ExecutionAttemptID executionAttemptID) {
 
 		Task task = taskSlotTable.removeTask(executionAttemptID);

http://git-wip-us.apache.org/repos/asf/flink/blob/af924b48/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/slot/SlotOffer.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/slot/SlotOffer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/slot/SlotOffer.java
new file mode 100644
index 0000000..f8d7e6c
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/slot/SlotOffer.java
@@ -0,0 +1,79 @@
+/*
+ * 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.taskexecutor.slot;
+
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.util.Preconditions;
+
+import java.io.Serializable;
+
+/**
+ * Describe the slot offering to job manager provided by task manager.
+ */
+public class SlotOffer implements Serializable {
+
+	private static final long serialVersionUID = -7067814231108250971L;
+
+	/** Allocation id of this slot, this would be the only identifier for this slot offer */
+	private AllocationID allocationId;
+
+	/** Index of the offered slot */
+	private final int slotIndex;
+
+	/** The resource profile of the offered slot */
+	private final ResourceProfile resourceProfile;
+
+	public SlotOffer(final AllocationID allocationID, final int index, final ResourceProfile resourceProfile) {
+		Preconditions.checkArgument(0 <= index, "The index must be greater than 0.");
+		this.allocationId = Preconditions.checkNotNull(allocationID);
+		this.slotIndex = index;
+		this.resourceProfile = Preconditions.checkNotNull(resourceProfile);
+	}
+
+	public AllocationID getAllocationId() {
+		return allocationId;
+	}
+
+	public int getSlotIndex() {
+		return slotIndex;
+	}
+
+	public ResourceProfile getResourceProfile() {
+		return resourceProfile;
+	}
+
+	@Override
+	public boolean equals(Object o) {
+		if (this == o) {
+			return true;
+		}
+		if (o == null || getClass() != o.getClass()) {
+			return false;
+		}
+
+		SlotOffer slotOffer = (SlotOffer) o;
+		return allocationId.equals(slotOffer.allocationId);
+	}
+
+	@Override
+	public int hashCode() {
+		return allocationId.hashCode();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/af924b48/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlot.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlot.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlot.java
index 0942772..e12c15b 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlot.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlot.java
@@ -286,4 +286,17 @@ public class TaskSlot {
 		state = TaskSlotState.RELEASING;
 		return true;
 	}
+
+	/**
+	 * Generate the slot offer from this TaskSlot.
+	 *
+	 * @return The sot offer which this task slot can provide
+	 */
+	public SlotOffer generateSlotOffer() {
+		Preconditions.checkState(TaskSlotState.ACTIVE == state || TaskSlotState.ALLOCATED == state,
+				"The task slot is not in state active or allocated.");
+		Preconditions.checkState(allocationId != null, "The task slot are not allocated");
+
+		return new SlotOffer(allocationId, index, resourceProfile);
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/af924b48/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotTable.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotTable.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotTable.java
index 88123b4..88b83a0 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotTable.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotTable.java
@@ -70,7 +70,7 @@ public class TaskSlotTable implements TimeoutListener<AllocationID> {
 
 	/** Interface for slot actions, such as freeing them or timing them out */
 	private SlotActions slotActions;
-	
+
 	/** Whether the table has been started */
 	private boolean started;
 
@@ -250,7 +250,7 @@ public class TaskSlotTable implements TimeoutListener<AllocationID> {
 	 */
 	public int freeSlot(AllocationID allocationId, Throwable cause) throws SlotNotFoundException {
 		checkInit();
-		
+
 		if (LOG.isDebugEnabled()) {
 			LOG.debug("Free slot {}.", allocationId, cause);
 		} else {
@@ -370,13 +370,13 @@ public class TaskSlotTable implements TimeoutListener<AllocationID> {
 	}
 
 	/**
-	 * Return an iterator of allocated slots (their allocation ids) for the given job id.
+	 * Return an iterator of allocated slots for the given job id.
 	 *
 	 * @param jobId for which to return the allocated slots
-	 * @return Iterator of allocation ids of allocated slots.
+	 * @return Iterator of allocated slots.
 	 */
-	public Iterator<AllocationID> getAllocatedSlots(JobID jobId) {
-		return new AllocationIDIterator(jobId, TaskSlotState.ALLOCATED);
+	public Iterator<TaskSlot> getAllocatedSlots(JobID jobId) {
+		return new TaskSlotIterator(jobId, TaskSlotState.ALLOCATED);
 	}
 
 	/**

http://git-wip-us.apache.org/repos/asf/flink/blob/af924b48/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java
index 55cc142..4d73a4b 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java
@@ -65,6 +65,7 @@ import org.apache.flink.runtime.resourcemanager.messages.taskexecutor.TMSlotRequ
 import org.apache.flink.runtime.resourcemanager.messages.taskexecutor.TMSlotRequestReply;
 import org.apache.flink.runtime.rpc.FatalErrorHandler;
 import org.apache.flink.runtime.rpc.TestingSerialRpcService;
+import org.apache.flink.runtime.taskexecutor.slot.SlotOffer;
 import org.apache.flink.runtime.taskexecutor.slot.TaskSlotTable;
 import org.apache.flink.runtime.taskexecutor.exceptions.SlotAllocationException;
 import org.apache.flink.runtime.taskexecutor.slot.TimerService;
@@ -406,6 +407,7 @@ public class TaskExecutorTest extends TestLogger {
 
 		final AllocationID allocationId = new AllocationID();
 		final SlotID slotId = new SlotID(resourceId, 0);
+		final SlotOffer slotOffer = new SlotOffer(allocationId, 0, ResourceProfile.UNKNOWN);
 
 		try {
 			TaskExecutor taskManager = new TaskExecutor(
@@ -440,7 +442,11 @@ public class TaskExecutorTest extends TestLogger {
 			jobManagerLeaderRetrievalService.notifyListener(jobManagerAddress, jobManagerLeaderId);
 
 			// the job leader should get the allocation id offered
-			verify(jobMasterGateway).offerSlots((Iterable<AllocationID>)Matchers.argThat(contains(allocationId)), eq(jobManagerLeaderId), any(Time.class));
+			verify(jobMasterGateway).offerSlots(
+					any(ResourceID.class),
+					(Iterable<SlotOffer>)Matchers.argThat(contains(slotOffer)),
+					eq(jobManagerLeaderId),
+					any(Time.class));
 		} finally {
 			// check if a concurrent error occurred
 			testingFatalErrorHandler.rethrowException();
@@ -496,6 +502,9 @@ public class TaskExecutorTest extends TestLogger {
 		final AllocationID allocationId1 = new AllocationID();
 		final AllocationID allocationId2 = new AllocationID();
 
+		final SlotOffer offer1 = new SlotOffer(allocationId1, 0, ResourceProfile.UNKNOWN);
+		final SlotOffer offer2 = new SlotOffer(allocationId2, 0, ResourceProfile.UNKNOWN);
+
 		final JobMasterGateway jobMasterGateway = mock(JobMasterGateway.class);
 
 		when(jobMasterGateway.registerTaskManager(
@@ -506,8 +515,9 @@ public class TaskExecutorTest extends TestLogger {
 		)).thenReturn(FlinkCompletableFuture.<RegistrationResponse>completed(new JMTMRegistrationSuccess(jmResourceId, blobPort)));
 		when(jobMasterGateway.getAddress()).thenReturn(jobManagerAddress);
 
-		when(jobMasterGateway.offerSlots(any(Iterable.class), eq(jobManagerLeaderId), any(Time.class)))
-			.thenReturn(FlinkCompletableFuture.completed((Iterable<AllocationID>)Collections.singleton(allocationId1)));
+		when(jobMasterGateway.offerSlots(
+				any(ResourceID.class), any(Iterable.class), eq(jobManagerLeaderId), any(Time.class)))
+			.thenReturn(FlinkCompletableFuture.completed((Iterable<SlotOffer>)Collections.singleton(offer1)));
 
 		rpc.registerGateway(resourceManagerAddress, resourceManagerGateway);
 		rpc.registerGateway(jobManagerAddress, jobMasterGateway);


[33/52] [abbrv] flink git commit: [FLINK-5140] [JobManager] SlotPool accepts allocation requests while ResourceManager is not connected

Posted by se...@apache.org.
[FLINK-5140] [JobManager] SlotPool accepts allocation requests while ResourceManager is not connected

The requests are kept for a certain time and fulfilled once the ResourceManager is connected.
If no ResourceManager is connected in time, the allocation requests are failed.


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

Branch: refs/heads/master
Commit: fbfb234e05edd3918c687c78fc030665e97ce7fb
Parents: 3b5bdc3
Author: Stephan Ewen <se...@apache.org>
Authored: Fri Dec 2 16:17:11 2016 +0100
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Dec 23 20:54:26 2016 +0100

----------------------------------------------------------------------
 .../apache/flink/runtime/instance/SlotPool.java |  76 ++++++++++++--
 .../flink/runtime/instance/SlotPoolRpcTest.java | 101 +++++++++++++++++++
 .../flink/runtime/instance/SlotPoolTest.java    |  27 -----
 3 files changed, 166 insertions(+), 38 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/fbfb234e/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
index 05884d1..2a9aca7 100644
--- 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
@@ -93,8 +93,6 @@ public class SlotPool extends RpcEndpoint<SlotPoolGateway> {
 
 	// ------------------------------------------------------------------------
 
-	private final Object lock = new Object();
-
 	private final JobID jobId;
 
 	private final ProviderAndOwner providerAndOwner;
@@ -111,6 +109,9 @@ public class SlotPool extends RpcEndpoint<SlotPoolGateway> {
 	/** All pending requests waiting for slots */
 	private final HashMap<AllocationID, PendingRequest> pendingRequests;
 
+	/** The requests that are waiting for the resource manager to be connected */
+	private final HashMap<AllocationID, PendingRequest> waitingForResourceManager;
+
 	/** Timeout for request calls to the ResourceManager */
 	private final Time resourceManagerRequestsTimeout;
 
@@ -154,6 +155,7 @@ public class SlotPool extends RpcEndpoint<SlotPoolGateway> {
 		this.allocatedSlots = new AllocatedSlots();
 		this.availableSlots = new AvailableSlots();
 		this.pendingRequests = new HashMap<>();
+		this.waitingForResourceManager = new HashMap<>();
 
 		this.providerAndOwner = new ProviderAndOwner(getSelf(), slotRequestTimeout);
 	}
@@ -239,6 +241,14 @@ public class SlotPool extends RpcEndpoint<SlotPoolGateway> {
 	public void connectToResourceManager(UUID resourceManagerLeaderId, ResourceManagerGateway resourceManagerGateway) {
 		this.resourceManagerLeaderId = checkNotNull(resourceManagerLeaderId);
 		this.resourceManagerGateway = checkNotNull(resourceManagerGateway);
+
+		// work on all slots waiting for this connection
+		for (PendingRequest pending : waitingForResourceManager.values()) {
+			requestSlotFromResourceManager(pending.allocationID(), pending.future(), pending.resourceProfile());
+		}
+
+		// all sent off
+		waitingForResourceManager.clear();
 	}
 
 	@RpcMethod
@@ -279,16 +289,27 @@ public class SlotPool extends RpcEndpoint<SlotPoolGateway> {
 			return FlinkCompletableFuture.completed(slot);
 		}
 
-		// (2) no slot available, and no resource manager connection
+		// the request will be completed by a future
+		final AllocationID allocationID = new AllocationID();
+		final FlinkCompletableFuture<SimpleSlot> future = new FlinkCompletableFuture<>();
+
+		// (2) need to request a slot
+
 		if (resourceManagerGateway == null) {
-			return FlinkCompletableFuture.completedExceptionally(
-					new NoResourceAvailableException("not connected to ResourceManager and no slot available"));
-			
+			// no slot available, and no resource manager connection
+			stashRequestWaitingForResourceManager(allocationID, resources, future);
+		} else {
+			// we have a resource manager connection, so let's ask it for more resources
+			requestSlotFromResourceManager(allocationID, future, resources);
 		}
 
-		// (3) we have a resource manager connection, so let's ask it for more resources
-		final FlinkCompletableFuture<SimpleSlot> future = new FlinkCompletableFuture<>();
-		final AllocationID allocationID = new AllocationID();
+		return future;
+	}
+
+	private void requestSlotFromResourceManager(
+			final AllocationID allocationID,
+			final FlinkCompletableFuture<SimpleSlot> future,
+			final ResourceProfile resources) {
 
 		LOG.info("Requesting slot with profile {} from resource manager (request = {}).", resources, allocationID);
 
@@ -333,8 +354,6 @@ public class SlotPool extends RpcEndpoint<SlotPoolGateway> {
 				return null;
 			}
 		}, getMainThreadExecutor());
-
-		return future;
 	}
 
 	private void slotRequestToResourceManagerSuccess(final AllocationID allocationID) {
@@ -363,6 +382,32 @@ public class SlotPool extends RpcEndpoint<SlotPoolGateway> {
 		}
 	}
 
+	private void stashRequestWaitingForResourceManager(
+			final AllocationID allocationID,
+			final ResourceProfile resources,
+			final FlinkCompletableFuture<SimpleSlot> future) {
+
+		LOG.info("Cannot serve slot request, no ResourceManager connected. " +
+				"Adding as pending request {}",  allocationID);
+
+		waitingForResourceManager.put(allocationID, new PendingRequest(allocationID, future, resources));
+
+		scheduleRunAsync(new Runnable() {
+			@Override
+			public void run() {
+				checkTimeoutRequestWaitingForResourceManager(allocationID);
+			}
+		}, resourceManagerRequestsTimeout);
+	}
+
+	private void checkTimeoutRequestWaitingForResourceManager(AllocationID allocationID) {
+		PendingRequest request = waitingForResourceManager.remove(allocationID);
+		if (request != null && !request.future().isDone()) {
+			request.future().completeExceptionally(new NoResourceAvailableException(
+					"No slot available and no connection to Resource Manager established."));
+		}
+	}
+
 	// ------------------------------------------------------------------------
 	//  Slot releasing & offering
 	// ------------------------------------------------------------------------
@@ -407,6 +452,7 @@ public class SlotPool extends RpcEndpoint<SlotPoolGateway> {
 	private PendingRequest pollMatchingPendingRequest(final AllocatedSlot slot) {
 		final ResourceProfile slotResources = slot.getResourceProfile();
 
+		// try the requests sent to the resource manager first
 		for (PendingRequest request : pendingRequests.values()) {
 			if (slotResources.isMatching(request.resourceProfile())) {
 				pendingRequests.remove(request.allocationID());
@@ -414,6 +460,14 @@ public class SlotPool extends RpcEndpoint<SlotPoolGateway> {
 			}
 		}
 
+		// try the requests waiting for a resource manager connection next
+		for (PendingRequest request : waitingForResourceManager.values()) {
+			if (slotResources.isMatching(request.resourceProfile())) {
+				waitingForResourceManager.remove(request.allocationID());
+				return request;
+			}
+		}
+
 		// no request pending, or no request matches
 		return null;
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/fbfb234e/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SlotPoolRpcTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SlotPoolRpcTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SlotPoolRpcTest.java
new file mode 100644
index 0000000..89fd22f
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SlotPoolRpcTest.java
@@ -0,0 +1,101 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.instance;
+
+import akka.actor.ActorSystem;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.akka.AkkaUtils;
+import org.apache.flink.runtime.concurrent.Future;
+import org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException;
+import org.apache.flink.runtime.jobmanager.scheduler.ScheduledUnit;
+import org.apache.flink.runtime.rpc.RpcService;
+import org.apache.flink.runtime.rpc.akka.AkkaRpcService;
+import org.apache.flink.runtime.util.clock.SystemClock;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.util.UUID;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+import static org.apache.flink.runtime.instance.AvailableSlotsTest.DEFAULT_TESTING_PROFILE;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+import static org.mockito.Mockito.mock;
+
+/**
+ * Tests for the SlotPool using a proper RPC setup.
+ */
+public class SlotPoolRpcTest {
+
+	private static RpcService rpcService;
+
+	// ------------------------------------------------------------------------
+	//  setup
+	// ------------------------------------------------------------------------
+
+	@BeforeClass
+	public static void setup() {
+		ActorSystem actorSystem = AkkaUtils.createLocalActorSystem(new Configuration());
+		rpcService = new AkkaRpcService(actorSystem, Time.seconds(10));
+	}
+
+	@AfterClass
+	public static  void shutdown() {
+		rpcService.stopService();
+	}
+
+	// ------------------------------------------------------------------------
+	//  tests
+	// ------------------------------------------------------------------------
+
+	@Test
+	public void testSlotAllocationNoResourceManager() throws Exception {
+		final JobID jid = new JobID();
+		
+		final SlotPool pool = new SlotPool(
+				rpcService, jid,
+				SystemClock.getInstance(),
+				Time.days(1), Time.days(1),
+				Time.milliseconds(100) // this is the timeout for the request tested here
+		);
+		pool.start(UUID.randomUUID());
+
+		Future<SimpleSlot> future = pool.allocateSlot(mock(ScheduledUnit.class), DEFAULT_TESTING_PROFILE, null);
+
+		try {
+			future.get(4, TimeUnit.SECONDS);
+			fail("We expected a ExecutionException.");
+		}
+		catch (ExecutionException e) {
+			assertEquals(NoResourceAvailableException.class, e.getCause().getClass());
+		}
+		catch (TimeoutException e) {
+			fail("future timed out rather than being failed");
+		}
+		catch (Exception e) {
+			fail("wrong exception: " + e);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/fbfb234e/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
index 5fa7af3..97457e1 100644
--- 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
@@ -23,7 +23,6 @@ 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.clusterframework.types.ResourceProfile;
-import org.apache.flink.runtime.concurrent.BiFunction;
 import org.apache.flink.runtime.concurrent.Future;
 import org.apache.flink.runtime.jobmanager.scheduler.ScheduledUnit;
 import org.apache.flink.runtime.jobmanager.slots.AllocatedSlot;
@@ -42,17 +41,13 @@ import org.mockito.ArgumentCaptor;
 
 import java.util.List;
 import java.util.UUID;
-import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeUnit;
 
 import static org.apache.flink.runtime.instance.AvailableSlotsTest.DEFAULT_TESTING_PROFILE;
 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;
@@ -123,28 +118,6 @@ public class SlotPoolTest extends TestLogger {
 	}
 
 	@Test
-	public void testAllocateSlotWithoutResourceManager() throws Exception {
-		slotPool.disconnectResourceManager();
-		Future<SimpleSlot> future = slotPool.allocateSlot(mock(ScheduledUnit.class),DEFAULT_TESTING_PROFILE, null);
-		future.handleAsync(
-			new BiFunction<SimpleSlot, Throwable, Void>() {
-				@Override
-				public Void apply(SimpleSlot simpleSlot, Throwable throwable) {
-					assertNull(simpleSlot);
-					assertNotNull(throwable);
-					return null;
-				}
-			},
-			rpcService.getExecutor());
-		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.registerTaskManager(resourceID);


[35/52] [abbrv] flink git commit: [FLINK-5141] [streaming api] Implement LocalStreamEnvironment for new mini cluster.

Posted by se...@apache.org.
[FLINK-5141] [streaming api] Implement LocalStreamEnvironment for new mini cluster.

This closes #2877


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

Branch: refs/heads/master
Commit: 73d27d7cd7680858d111c6442a2aa6b6b94c7cef
Parents: 4d3a3ee
Author: biao.liub <bi...@alibaba-inc.com>
Authored: Wed Nov 23 17:02:11 2016 +0800
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Dec 23 20:54:26 2016 +0100

----------------------------------------------------------------------
 .../Flip6LocalStreamEnvironment.java            | 128 +++++++++++++++++++
 1 file changed, 128 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/73d27d7c/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/Flip6LocalStreamEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/Flip6LocalStreamEnvironment.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/Flip6LocalStreamEnvironment.java
new file mode 100644
index 0000000..a0c128e
--- /dev/null
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/Flip6LocalStreamEnvironment.java
@@ -0,0 +1,128 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.api.environment;
+
+import org.apache.flink.annotation.Public;
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.InvalidProgramException;
+import org.apache.flink.api.common.JobExecutionResult;
+import org.apache.flink.api.common.restartstrategy.RestartStrategies;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.minicluster.MiniCluster;
+import org.apache.flink.runtime.minicluster.MiniClusterConfiguration;
+import org.apache.flink.streaming.api.graph.StreamGraph;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The Flip6LocalStreamEnvironment is a StreamExecutionEnvironment that runs the program locally,
+ * multi-threaded, in the JVM where the environment is instantiated. It spawns an embedded
+ * Flink cluster in the background and executes the program on that cluster.
+ *
+ * <p>When this environment is instantiated, it uses a default parallelism of {@code 1}. The default
+ * parallelism can be set via {@link #setParallelism(int)}.
+ */
+@Public
+public class Flip6LocalStreamEnvironment extends StreamExecutionEnvironment {
+
+	private static final Logger LOG = LoggerFactory.getLogger(Flip6LocalStreamEnvironment.class);
+
+	/** The configuration to use for the mini cluster */
+	private final Configuration conf;
+
+	/**
+	 * Creates a new mini cluster stream environment that uses the default configuration.
+	 */
+	public Flip6LocalStreamEnvironment() {
+		this(null);
+	}
+
+	/**
+	 * Creates a new mini cluster stream environment that configures its local executor with the given configuration.
+	 *
+	 * @param config The configuration used to configure the local executor.
+	 */
+	public Flip6LocalStreamEnvironment(Configuration config) {
+		if (!ExecutionEnvironment.areExplicitEnvironmentsAllowed()) {
+			throw new InvalidProgramException(
+					"The Flip6LocalStreamEnvironment cannot be used when submitting a program through a client, " +
+							"or running in a TestEnvironment context.");
+		}
+		
+		this.conf = config == null ? new Configuration() : config;
+	}
+
+	/**
+	 * Executes the JobGraph of the on a mini cluster of CLusterUtil with a user
+	 * specified name.
+	 * 
+	 * @param jobName
+	 *            name of the job
+	 * @return The result of the job execution, containing elapsed time and accumulators.
+	 */
+	@Override
+	public JobExecutionResult execute(String jobName) throws Exception {
+		// transform the streaming program into a JobGraph
+		StreamGraph streamGraph = getStreamGraph();
+		streamGraph.setJobName(jobName);
+
+		JobGraph jobGraph = streamGraph.getJobGraph();
+
+		jobGraph.setAllowQueuedScheduling(true);
+
+		// As jira FLINK-5140 described,
+		// we have to set restart strategy to handle NoResourceAvailableException.
+		ExecutionConfig executionConfig = new ExecutionConfig();
+		executionConfig.setRestartStrategy(
+			RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, 1000));
+		jobGraph.setExecutionConfig(executionConfig);
+
+		Configuration configuration = new Configuration();
+		configuration.addAll(jobGraph.getJobConfiguration());
+		configuration.setLong(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, -1L);
+
+		// add (and override) the settings with what the user defined
+		configuration.addAll(this.conf);
+
+		MiniClusterConfiguration cfg = new MiniClusterConfiguration(configuration);
+
+		// Currently we do not reuse slot anymore, so we need to sum all parallelism of vertices up.
+		int slotsCount = 0;
+		for (JobVertex jobVertex : jobGraph.getVertices()) {
+			slotsCount += jobVertex.getParallelism();
+		}
+		cfg.setNumTaskManagerSlots(slotsCount);
+
+		if (LOG.isInfoEnabled()) {
+			LOG.info("Running job on local embedded Flink mini cluster");
+		}
+
+		MiniCluster miniCluster = new MiniCluster(cfg);
+		try {
+			miniCluster.start();
+			return miniCluster.runJobBlocking(jobGraph);
+		} finally {
+			transformations.clear();
+			miniCluster.shutdown();
+		}
+	}
+}


[37/52] [abbrv] flink git commit: [FLINK-5076] Shutting down TM when shutting down mini cluster.

Posted by se...@apache.org.
[FLINK-5076] Shutting down TM when shutting down mini cluster.

This closes #2817.


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

Branch: refs/heads/master
Commit: c424900820e923089dc1c489bd02efa874dd39b6
Parents: 1da7924
Author: biao.liub <bi...@alibaba-inc.com>
Authored: Wed Nov 16 17:54:48 2016 +0800
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Dec 23 20:54:26 2016 +0100

----------------------------------------------------------------------
 .../apache/flink/runtime/minicluster/MiniCluster.java  | 13 +++++++++++++
 1 file changed, 13 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/c4249008/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java
index 611d4c4..3ede5b5 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java
@@ -312,6 +312,19 @@ public class MiniCluster {
 			resourceManagerRunners = null;
 		}
 
+		if (taskManagerRunners != null) {
+			for (TaskManagerRunner tm : taskManagerRunners) {
+				if (tm != null) {
+					try {
+						tm.shutDown(null);
+					} catch (Throwable t) {
+						exception = firstOrSuppressed(t, exception);
+					}
+				}
+			}
+			taskManagerRunners = null;
+		}
+
 		// shut down the RpcServices
 		exception = shutDownRpc(commonRpcService, exception);
 		exception = shutDownRpcs(jobManagerRpcServices, exception);


[50/52] [abbrv] flink git commit: [FLINK-4928] [yarn] Implement FLIP-6 YARN Application Master Runner

Posted by se...@apache.org.
[FLINK-4928] [yarn] Implement FLIP-6 YARN Application Master Runner


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

Branch: refs/heads/master
Commit: e28b116cb59f53e2a4ec263789a01114f9c393f0
Parents: cbfb807
Author: shuai.xus <sh...@alibaba-inc.com>
Authored: Thu Nov 3 16:24:47 2016 +0800
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Dec 23 20:54:27 2016 +0100

----------------------------------------------------------------------
 .../resourcemanager/ResourceManager.java        |   2 +-
 .../apache/flink/yarn/YarnResourceManager.java  | 552 +++++++++++++++++++
 2 files changed, 553 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/e28b116c/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java
index 76b4a86..3bcbfda 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java
@@ -637,7 +637,7 @@ public abstract class ResourceManager<WorkerType extends Serializable>
 	 *
 	 * @param t The exception describing the fatal error
 	 */
-	void onFatalErrorAsync(final Throwable t) {
+	protected void onFatalErrorAsync(final Throwable t) {
 		runAsync(new Runnable() {
 			@Override
 			public void run() {

http://git-wip-us.apache.org/repos/asf/flink/blob/e28b116c/flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManager.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManager.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManager.java
new file mode 100644
index 0000000..6280bdf
--- /dev/null
+++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManager.java
@@ -0,0 +1,552 @@
+/*
+ * 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.yarn;
+
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.clusterframework.ApplicationStatus;
+import org.apache.flink.runtime.clusterframework.BootstrapTools;
+import org.apache.flink.runtime.clusterframework.ContaineredTaskManagerParameters;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
+import org.apache.flink.runtime.metrics.MetricRegistry;
+import org.apache.flink.runtime.resourcemanager.JobLeaderIdService;
+import org.apache.flink.runtime.resourcemanager.ResourceManager;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerConfiguration;
+import org.apache.flink.runtime.resourcemanager.exceptions.ResourceManagerException;
+import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManagerFactory;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+import org.apache.flink.runtime.rpc.RpcService;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.DataOutputBuffer;
+import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.yarn.api.ApplicationConstants;
+import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
+import org.apache.hadoop.yarn.api.records.Priority;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.ContainerStatus;
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
+import org.apache.hadoop.yarn.api.records.NodeReport;
+import org.apache.hadoop.yarn.api.records.LocalResource;
+import org.apache.hadoop.yarn.client.api.AMRMClient;
+import org.apache.hadoop.yarn.client.api.NMClient;
+import org.apache.hadoop.yarn.client.api.async.AMRMClientAsync;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.util.Records;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.concurrent.duration.FiniteDuration;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Collections;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.flink.yarn.YarnConfigKeys.ENV_FLINK_CLASSPATH;
+
+/**
+ * The yarn implementation of the resource manager. Used when the system is started
+ * via the resource framework YARN.
+ */
+public class YarnResourceManager extends ResourceManager<ResourceID> implements AMRMClientAsync.CallbackHandler {
+	protected final Logger LOG = LoggerFactory.getLogger(getClass());
+
+	/** The process environment variables */
+	private final Map<String, String> ENV;
+
+	/** The heartbeat interval while the resource master is waiting for containers */
+	private static final int FAST_YARN_HEARTBEAT_INTERVAL_MS = 500;
+
+	/** The default heartbeat interval during regular operation */
+	private static final int DEFAULT_YARN_HEARTBEAT_INTERVAL_MS = 5000;
+
+	/** The maximum time that TaskExecutors may be waiting to register at the ResourceManager before they quit */
+	private static final FiniteDuration TASKEXECUTOR_REGISTRATION_TIMEOUT = new FiniteDuration(5, TimeUnit.MINUTES);
+
+	/** Environment variable name of the final container id used by the YarnResourceManager.
+	 * Container ID generation may vary across Hadoop versions. */
+	final static String ENV_FLINK_CONTAINER_ID = "_FLINK_CONTAINER_ID";
+	
+	/** Environment variable name of the hostname used by the Yarn.
+	 * TaskExecutor use this host name to start port. */
+	final static String ENV_FLINK_NODE_ID = "_FLINK_NODE_ID";
+
+	/** Default heartbeat interval between this resource manager and the YARN ResourceManager */
+	private final int yarnHeartbeatIntervalMillis;
+
+	private final Configuration flinkConfig;
+
+	private final YarnConfiguration yarnConfig;
+
+	/** Client to communicate with the Resource Manager (YARN's master) */
+	private AMRMClientAsync<AMRMClient.ContainerRequest> resourceManagerClient;
+
+	/** Client to communicate with the Node manager and launch TaskExecutor processes */
+	private NMClient nodeManagerClient;
+
+	/** The number of containers requested, but not yet granted */
+	private int numPendingContainerRequests;
+
+	public YarnResourceManager(
+			Configuration flinkConfig,
+			Map<String, String> env,
+			RpcService rpcService,
+			ResourceManagerConfiguration resourceManagerConfiguration,
+			HighAvailabilityServices highAvailabilityServices,
+			SlotManagerFactory slotManagerFactory,
+			MetricRegistry metricRegistry,
+			JobLeaderIdService jobLeaderIdService,
+			FatalErrorHandler fatalErrorHandler) {
+		super(
+			rpcService,
+			resourceManagerConfiguration,
+			highAvailabilityServices,
+			slotManagerFactory,
+			metricRegistry,
+			jobLeaderIdService,
+			fatalErrorHandler);
+		this.flinkConfig  = flinkConfig;
+		this.yarnConfig = new YarnConfiguration();
+		this.ENV = env;
+		final int yarnHeartbeatIntervalMS = flinkConfig.getInteger(
+				ConfigConstants.YARN_HEARTBEAT_DELAY_SECONDS, DEFAULT_YARN_HEARTBEAT_INTERVAL_MS / 1000) * 1000;
+
+		final long yarnExpiryIntervalMS = yarnConfig.getLong(
+				YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS,
+				YarnConfiguration.DEFAULT_RM_AM_EXPIRY_INTERVAL_MS);
+
+		if (yarnHeartbeatIntervalMS >= yarnExpiryIntervalMS) {
+			log.warn("The heartbeat interval of the Flink Application master ({}) is greater " +
+					"than YARN's expiry interval ({}). The application is likely to be killed by YARN.",
+					yarnHeartbeatIntervalMS, yarnExpiryIntervalMS);
+		}
+		yarnHeartbeatIntervalMillis = yarnHeartbeatIntervalMS;
+		numPendingContainerRequests = 0;
+	}
+
+	@Override
+	protected void initialize() throws ResourceManagerException {
+		resourceManagerClient = AMRMClientAsync.createAMRMClientAsync(yarnHeartbeatIntervalMillis, this);
+		resourceManagerClient.init(yarnConfig);
+		resourceManagerClient.start();
+		try {
+			//TODO: change akka address to tcp host and port, the getAddress() interface should return a standard tcp address
+			Tuple2<String, Integer> hostPort = parseHostPort(getAddress());
+			//TODO: the third paramter should be the webmonitor address
+			resourceManagerClient.registerApplicationMaster(hostPort.f0, hostPort.f1, getAddress());
+		} catch (Exception e) {
+			LOG.info("registerApplicationMaster fail", e);
+		}
+
+		// create the client to communicate with the node managers
+		nodeManagerClient = NMClient.createNMClient();
+		nodeManagerClient.init(yarnConfig);
+		nodeManagerClient.start();
+		nodeManagerClient.cleanupRunningContainersOnStop(true);
+	}
+
+	@Override
+	public void shutDown() throws Exception {
+		// shut down all components
+		if (resourceManagerClient != null) {
+			try {
+				resourceManagerClient.stop();
+			} catch (Throwable t) {
+				LOG.error("Could not cleanly shut down the Asynchronous Resource Manager Client", t);
+			}
+		}
+		if (nodeManagerClient != null) {
+			try {
+				nodeManagerClient.stop();
+			} catch (Throwable t) {
+				LOG.error("Could not cleanly shut down the Node Manager Client", t);
+			}
+		}
+		super.shutDown();
+	}
+
+	@Override
+	protected void shutDownApplication(ApplicationStatus finalStatus, String optionalDiagnostics) {
+
+		// first, de-register from YARN
+		FinalApplicationStatus yarnStatus = getYarnStatus(finalStatus);
+		LOG.info("Unregistering application from the YARN Resource Manager");
+		try {
+			resourceManagerClient.unregisterApplicationMaster(yarnStatus, optionalDiagnostics, "");
+		} catch (Throwable t) {
+			LOG.error("Could not unregister the application master.", t);
+		}
+	}
+
+	@Override
+	public void startNewWorker(ResourceProfile resourceProfile) {
+		// Priority for worker containers - priorities are intra-application
+		//TODO: set priority according to the resource allocated
+		Priority priority = Priority.newInstance(0);
+		int mem = resourceProfile.getMemoryInMB() <= Integer.MAX_VALUE ? (int)resourceProfile.getMemoryInMB() : Integer.MAX_VALUE;
+		if (mem < 0) {
+			mem = 1024;
+		}
+		int vcore = resourceProfile.getCpuCores() < 1 ? 1 : (int)resourceProfile.getCpuCores() + 1;
+		Resource capability = Resource.newInstance(mem , vcore);
+		requestYarnContainer(capability, priority);
+	}
+
+	@Override
+	protected ResourceID workerStarted(ResourceID resourceID) {
+		return resourceID;
+	}
+
+	// AMRMClientAsync CallbackHandler methods
+	@Override
+	public float getProgress() {
+		// Temporarily need not record the total size of asked and allocated containers
+		return 1;
+	}
+
+	@Override
+	public void onContainersCompleted(List<ContainerStatus> list) {
+		for (ContainerStatus container : list) {
+			if (container.getExitStatus() < 0) {
+				notifyWorkerFailed(new ResourceID(container.getContainerId().toString()), container.getDiagnostics());
+				// TODO: notice job master slot fail
+			}
+		}
+	}
+
+	@Override
+	public void onContainersAllocated(List<Container> containers) {
+		for (Container container : containers) {
+			numPendingContainerRequests = Math.max(0, numPendingContainerRequests - 1);
+			LOG.info("Received new container: {} - Remaining pending container requests: {}",
+					container.getId(), numPendingContainerRequests);
+			try {
+				/** Context information used to start a TaskExecutor Java process */
+				ContainerLaunchContext taskExecutorLaunchContext =
+						createTaskExecutorLaunchContext(container.getResource(), container.getId().toString(), container.getNodeId().getHost());
+				nodeManagerClient.startContainer(container, taskExecutorLaunchContext);
+			}
+			catch (Throwable t) {
+				// failed to launch the container, will release the failed one and ask for a new one
+				LOG.error("Could not start TaskManager in container " + container, t);
+				resourceManagerClient.releaseAssignedContainer(container.getId());
+				requestYarnContainer(container.getResource(), container.getPriority());
+			}
+		}
+		if (numPendingContainerRequests <= 0) {
+			resourceManagerClient.setHeartbeatInterval(yarnHeartbeatIntervalMillis);
+		}
+	}
+
+	@Override
+	public void onShutdownRequest() {
+		// Nothing to do
+	}
+
+	@Override
+	public void onNodesUpdated(List<NodeReport> list) {
+		// We are not interested in node updates
+	}
+
+	@Override
+	public void onError(Throwable error) {
+		onFatalErrorAsync(error);
+	}
+
+	//Utility methods
+	/**
+	 * Converts a Flink application status enum to a YARN application status enum.
+	 * @param status The Flink application status.
+	 * @return The corresponding YARN application status.
+	 */
+	private FinalApplicationStatus getYarnStatus(ApplicationStatus status) {
+		if (status == null) {
+			return FinalApplicationStatus.UNDEFINED;
+		}
+		else {
+			switch (status) {
+				case SUCCEEDED:
+					return FinalApplicationStatus.SUCCEEDED;
+				case FAILED:
+					return FinalApplicationStatus.FAILED;
+				case CANCELED:
+					return FinalApplicationStatus.KILLED;
+				default:
+					return FinalApplicationStatus.UNDEFINED;
+			}
+		}
+	}
+
+	// parse the host and port from akka address, 
+	// the akka address is like akka.tcp://flink@100.81.153.180:49712/user/$a
+	private static Tuple2<String, Integer> parseHostPort(String address) {
+		String[] hostPort = address.split("@")[1].split(":");
+		String host = hostPort[0];
+		String port = hostPort[1].split("/")[0];
+		return new Tuple2(host, Integer.valueOf(port));
+	}
+
+	private void requestYarnContainer(Resource resource, Priority priority) {
+		resourceManagerClient.addContainerRequest(
+				new AMRMClient.ContainerRequest(resource, null, null, priority));
+		// make sure we transmit the request fast and receive fast news of granted allocations
+		resourceManagerClient.setHeartbeatInterval(FAST_YARN_HEARTBEAT_INTERVAL_MS);
+
+		numPendingContainerRequests++;
+		LOG.info("Requesting new TaskManager container pending requests: {}",
+				numPendingContainerRequests);
+	}
+
+	private ContainerLaunchContext createTaskExecutorLaunchContext(Resource resource, String containerId, String host)
+			throws Exception {
+		// init the ContainerLaunchContext
+		final String currDir = ENV.get(ApplicationConstants.Environment.PWD.key());
+
+		final ContaineredTaskManagerParameters taskManagerParameters =
+				ContaineredTaskManagerParameters.create(flinkConfig, resource.getMemory(), 1);
+
+		LOG.info("TaskExecutor{} will be started with container size {} MB, JVM heap size {} MB, " +
+				"JVM direct memory limit {} MB",
+				containerId,
+				taskManagerParameters.taskManagerTotalMemoryMB(),
+				taskManagerParameters.taskManagerHeapSizeMB(),
+				taskManagerParameters.taskManagerDirectMemoryLimitMB());
+		final Configuration taskManagerConfig = BootstrapTools.generateTaskManagerConfiguration(
+				flinkConfig, "", 0, 1, TASKEXECUTOR_REGISTRATION_TIMEOUT);
+		LOG.debug("TaskManager configuration: {}", taskManagerConfig);
+
+		ContainerLaunchContext taskExecutorLaunchContext = createTaskExecutorContext(
+				flinkConfig, yarnConfig, ENV,
+				taskManagerParameters, taskManagerConfig,
+				currDir, YarnTaskExecutorRunner.class, LOG);
+
+		// set a special environment variable to uniquely identify this container
+		taskExecutorLaunchContext.getEnvironment()
+				.put(ENV_FLINK_CONTAINER_ID, containerId);
+		taskExecutorLaunchContext.getEnvironment()
+				.put(ENV_FLINK_NODE_ID, host);
+		return taskExecutorLaunchContext;
+	}
+
+
+	/**
+	 * Creates the launch context, which describes how to bring up a TaskExecutor process in
+	 * an allocated YARN container.
+	 *
+	 * <p>This code is extremely YARN specific and registers all the resources that the TaskExecutor
+	 * needs (such as JAR file, config file, ...) and all environment variables in a YARN
+	 * container launch context. The launch context then ensures that those resources will be
+	 * copied into the containers transient working directory.
+	 *
+	 * @param flinkConfig
+	 *		 The Flink configuration object.
+	 * @param yarnConfig
+	 *		 The YARN configuration object.
+	 * @param env
+	 *		 The environment variables.
+	 * @param tmParams
+	 *		 The TaskExecutor container memory parameters.
+	 * @param taskManagerConfig
+	 *		 The configuration for the TaskExecutors.
+	 * @param workingDirectory
+	 *		 The current application master container's working directory.
+	 * @param taskManagerMainClass
+	 *		 The class with the main method.
+	 * @param log
+	 *		 The logger.
+	 *
+	 * @return The launch context for the TaskManager processes.
+	 *
+	 * @throws Exception Thrown if teh launch context could not be created, for example if
+	 *				   the resources could not be copied.
+	 */
+	private static ContainerLaunchContext createTaskExecutorContext(
+			Configuration flinkConfig,
+			YarnConfiguration yarnConfig,
+			Map<String, String> env,
+			ContaineredTaskManagerParameters tmParams,
+			Configuration taskManagerConfig,
+			String workingDirectory,
+			Class<?> taskManagerMainClass,
+			Logger log) throws Exception {
+
+		// get and validate all relevant variables
+
+		String remoteFlinkJarPath = env.get(YarnConfigKeys.FLINK_JAR_PATH);
+		
+		String appId = env.get(YarnConfigKeys.ENV_APP_ID);
+
+		String clientHomeDir = env.get(YarnConfigKeys.ENV_CLIENT_HOME_DIR);
+
+		String shipListString = env.get(YarnConfigKeys.ENV_CLIENT_SHIP_FILES);
+
+		String yarnClientUsername = env.get(YarnConfigKeys.ENV_HADOOP_USER_NAME);
+
+		final String remoteKeytabPath = env.get(YarnConfigKeys.KEYTAB_PATH);
+		log.info("TM:remote keytab path obtained {}", remoteKeytabPath);
+
+		final String remoteKeytabPrincipal = env.get(YarnConfigKeys.KEYTAB_PRINCIPAL);
+		log.info("TM:remote keytab principal obtained {}", remoteKeytabPrincipal);
+
+		final String remoteYarnConfPath = env.get(YarnConfigKeys.ENV_YARN_SITE_XML_PATH);
+		log.info("TM:remote yarn conf path obtained {}", remoteYarnConfPath);
+
+		final String remoteKrb5Path = env.get(YarnConfigKeys.ENV_KRB5_PATH);
+		log.info("TM:remote krb5 path obtained {}", remoteKrb5Path);
+
+		String classPathString = env.get(YarnConfigKeys.ENV_FLINK_CLASSPATH);
+
+		// obtain a handle to the file system used by YARN
+		final org.apache.hadoop.fs.FileSystem yarnFileSystem;
+		try {
+			yarnFileSystem = org.apache.hadoop.fs.FileSystem.get(yarnConfig);
+		} catch (IOException e) {
+			throw new Exception("Could not access YARN's default file system", e);
+		}
+
+		//register keytab
+		LocalResource keytabResource = null;
+		if(remoteKeytabPath != null) {
+			log.info("Adding keytab {} to the AM container local resource bucket", remoteKeytabPath);
+			keytabResource = Records.newRecord(LocalResource.class);
+			Path keytabPath = new Path(remoteKeytabPath);
+			Utils.registerLocalResource(yarnFileSystem, keytabPath, keytabResource);
+		}
+
+		//To support Yarn Secure Integration Test Scenario
+		LocalResource yarnConfResource = null;
+		LocalResource krb5ConfResource = null;
+		boolean hasKrb5 = false;
+		if(remoteYarnConfPath != null && remoteKrb5Path != null) {
+			log.info("TM:Adding remoteYarnConfPath {} to the container local resource bucket", remoteYarnConfPath);
+			yarnConfResource = Records.newRecord(LocalResource.class);
+			Path yarnConfPath = new Path(remoteYarnConfPath);
+			Utils.registerLocalResource(yarnFileSystem, yarnConfPath, yarnConfResource);
+
+			log.info("TM:Adding remoteKrb5Path {} to the container local resource bucket", remoteKrb5Path);
+			krb5ConfResource = Records.newRecord(LocalResource.class);
+			Path krb5ConfPath = new Path(remoteKrb5Path);
+			Utils.registerLocalResource(yarnFileSystem, krb5ConfPath, krb5ConfResource);
+
+			hasKrb5 = true;
+		}
+
+		// register Flink Jar with remote HDFS
+		LocalResource flinkJar = Records.newRecord(LocalResource.class);
+		{
+			Path remoteJarPath = new Path(remoteFlinkJarPath);
+			Utils.registerLocalResource(yarnFileSystem, remoteJarPath, flinkJar);
+		}
+
+		// register conf with local fs
+		LocalResource flinkConf = Records.newRecord(LocalResource.class);
+		{
+			// write the TaskManager configuration to a local file
+			final File taskManagerConfigFile =
+					new File(workingDirectory, UUID.randomUUID() + "-taskmanager-conf.yaml");
+			log.debug("Writing TaskManager configuration to {}", taskManagerConfigFile.getAbsolutePath());
+			BootstrapTools.writeConfiguration(taskManagerConfig, taskManagerConfigFile);
+
+			Utils.setupLocalResource(yarnFileSystem, appId,
+					new Path(taskManagerConfigFile.toURI()), flinkConf, new Path(clientHomeDir));
+
+			log.info("Prepared local resource for modified yaml: {}", flinkConf);
+		}
+
+		Map<String, LocalResource> taskManagerLocalResources = new HashMap<>();
+		taskManagerLocalResources.put("flink.jar", flinkJar);
+		taskManagerLocalResources.put("flink-conf.yaml", flinkConf);
+
+		//To support Yarn Secure Integration Test Scenario
+		if(yarnConfResource != null && krb5ConfResource != null) {
+			taskManagerLocalResources.put(Utils.YARN_SITE_FILE_NAME, yarnConfResource);
+			taskManagerLocalResources.put(Utils.KRB5_FILE_NAME, krb5ConfResource);
+		}
+
+		if(keytabResource != null) {
+			taskManagerLocalResources.put(Utils.KEYTAB_FILE_NAME, keytabResource);
+		}
+
+		// prepare additional files to be shipped
+		for (String pathStr : shipListString.split(",")) {
+			if (!pathStr.isEmpty()) {
+				LocalResource resource = Records.newRecord(LocalResource.class);
+				Path path = new Path(pathStr);
+				Utils.registerLocalResource(yarnFileSystem, path, resource);
+				taskManagerLocalResources.put(path.getName(), resource);
+			}
+		}
+
+		// now that all resources are prepared, we can create the launch context
+
+		log.info("Creating container launch context for TaskManagers");
+
+		boolean hasLogback = new File(workingDirectory, "logback.xml").exists();
+		boolean hasLog4j = new File(workingDirectory, "log4j.properties").exists();
+
+		String launchCommand = BootstrapTools.getTaskManagerShellCommand(
+				flinkConfig, tmParams, ".", ApplicationConstants.LOG_DIR_EXPANSION_VAR,
+				hasLogback, hasLog4j, hasKrb5, taskManagerMainClass);
+
+		log.info("Starting TaskManagers with command: " + launchCommand);
+
+		ContainerLaunchContext ctx = Records.newRecord(ContainerLaunchContext.class);
+		ctx.setCommands(Collections.singletonList(launchCommand));
+		ctx.setLocalResources(taskManagerLocalResources);
+
+		Map<String, String> containerEnv = new HashMap<>();
+		containerEnv.putAll(tmParams.taskManagerEnv());
+
+		// add YARN classpath, etc to the container environment
+		containerEnv.put(ENV_FLINK_CLASSPATH, classPathString);
+		Utils.setupYarnClassPath(yarnConfig, containerEnv);
+
+		containerEnv.put(YarnConfigKeys.ENV_HADOOP_USER_NAME, UserGroupInformation.getCurrentUser().getUserName());
+
+		if(remoteKeytabPath != null && remoteKeytabPrincipal != null) {
+			containerEnv.put(YarnConfigKeys.KEYTAB_PATH, remoteKeytabPath);
+			containerEnv.put(YarnConfigKeys.KEYTAB_PRINCIPAL, remoteKeytabPrincipal);
+		}
+
+		ctx.setEnvironment(containerEnv);
+
+		try (DataOutputBuffer dob = new DataOutputBuffer()) {
+			log.debug("Adding security tokens to Task Executor Container launch Context....");
+			UserGroupInformation user = UserGroupInformation.getCurrentUser();
+			Credentials credentials = user.getCredentials();
+			credentials.writeTokenStorageToStream(dob);
+			ByteBuffer securityTokens = ByteBuffer.wrap(dob.getData(), 0, dob.getLength());
+			ctx.setTokens(securityTokens);
+		}
+		catch (Throwable t) {
+			log.error("Getting current user info failed when trying to launch the container", t);
+		}
+
+		return ctx;
+	}
+}


[10/52] [abbrv] flink git commit: [hotfix] Treat taskManager's rpc address and location separately

Posted by se...@apache.org.
[hotfix] Treat taskManager's rpc address and location separately


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

Branch: refs/heads/master
Commit: a7ed9a5e3876c538deae147217c5443a287e98d5
Parents: bc68236
Author: Kurt Young <yk...@gmail.com>
Authored: Mon Oct 17 09:38:46 2016 +0800
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Dec 23 20:54:23 2016 +0100

----------------------------------------------------------------------
 .../flink/runtime/jobmaster/JobMaster.java      |  7 ++--
 .../runtime/jobmaster/JobMasterGateway.java     |  8 +++--
 .../runtime/taskexecutor/JobLeaderService.java  | 37 +++++++++++++-------
 .../runtime/taskexecutor/TaskExecutor.java      |  2 +-
 .../runtime/taskexecutor/TaskExecutorTest.java  | 14 ++++----
 5 files changed, 43 insertions(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/a7ed9a5e/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 1fb5474..7bcfb3a 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
@@ -674,13 +674,14 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 
 	@RpcMethod
 	public Future<RegistrationResponse> registerTaskManager(
+			final String taskManagerRpcAddress,
 			final TaskManagerLocation taskManagerLocation,
 			final UUID leaderId) throws Exception
 	{
 		if (!JobMaster.this.leaderSessionID.equals(leaderId)) {
 			log.warn("Discard registration from TaskExecutor {} at ({}) because the expected " +
 							"leader session ID {} did not equal the received leader session ID {}.",
-					taskManagerLocation.getResourceID(), taskManagerLocation.addressString(),
+					taskManagerLocation.getResourceID(), taskManagerRpcAddress,
 					JobMaster.this.leaderSessionID, leaderId);
 			throw new Exception("Leader id not match, expected: " + JobMaster.this.leaderSessionID
 					+ ", actual: " + leaderId);
@@ -696,7 +697,7 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 			return getRpcService().execute(new Callable<TaskExecutorGateway>() {
 				@Override
 				public TaskExecutorGateway call() throws Exception {
-					return getRpcService().connect(taskManagerLocation.addressString(), TaskExecutorGateway.class)
+					return getRpcService().connect(taskManagerRpcAddress, TaskExecutorGateway.class)
 							.get(rpcTimeout.getSize(), rpcTimeout.getUnit());
 				}
 			}).handleAsync(new BiFunction<TaskExecutorGateway, Throwable, RegistrationResponse>() {
@@ -709,7 +710,7 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 					if (!JobMaster.this.leaderSessionID.equals(leaderId)) {
 						log.warn("Discard registration from TaskExecutor {} at ({}) because the expected " +
 										"leader session ID {} did not equal the received leader session ID {}.",
-								taskManagerLocation.getResourceID(), taskManagerLocation.addressString(),
+								taskManagerId, taskManagerRpcAddress,
 								JobMaster.this.leaderSessionID, leaderId);
 						return new RegistrationResponse.Decline("Invalid leader session id");
 					}

http://git-wip-us.apache.org/repos/asf/flink/blob/a7ed9a5e/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 508e70a..8925d94 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
@@ -185,12 +185,14 @@ public interface JobMasterGateway extends CheckpointCoordinatorGateway {
 	/**
 	 * Register the task manager at the job manager.
 	 *
-	 * @param taskManagerLocation location of the task manager
-	 * @param leaderId            identifying the job leader
-	 * @param timeout             for the rpc call
+	 * @param taskManagerRpcAddress the rpc address of the task manager
+	 * @param taskManagerLocation   location of the task manager
+	 * @param leaderId              identifying the job leader
+	 * @param timeout               for the rpc call
 	 * @return Future registration response indicating whether the registration was successful or not
 	 */
 	Future<RegistrationResponse> registerTaskManager(
+			final String taskManagerRpcAddress,
 			final TaskManagerLocation taskManagerLocation,
 			final UUID leaderId,
 			@RpcTimeout final Time timeout);

http://git-wip-us.apache.org/repos/asf/flink/blob/a7ed9a5e/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/JobLeaderService.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/JobLeaderService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/JobLeaderService.java
index 14d36ab..93c7bb7 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/JobLeaderService.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/JobLeaderService.java
@@ -62,6 +62,9 @@ public class JobLeaderService {
 	/** Internal state of the service */
 	private volatile JobLeaderService.State state;
 
+	/** Address of the owner of this service. This address is used for the job manager connection */
+	private String ownerAddress;
+
 	/** Rpc service to use for establishing connections */
 	private RpcService rpcService;
 
@@ -78,6 +81,7 @@ public class JobLeaderService {
 
 		state = JobLeaderService.State.CREATED;
 
+		ownerAddress = null;
 		rpcService = null;
 		highAvailabilityServices = null;
 		jobLeaderListener = null;
@@ -90,20 +94,23 @@ public class JobLeaderService {
 	/**
 	 * Start the job leader service with the given services.
 	 *
+ 	 * @param initialOwnerAddress to be used for establishing connections (source address)
 	 * @param initialRpcService to be used to create rpc connections
 	 * @param initialHighAvailabilityServices to create leader retrieval services for the different jobs
 	 * @param initialJobLeaderListener listening for job leader changes
 	 */
 	public void start(
-		final RpcService initialRpcService,
-		final HighAvailabilityServices initialHighAvailabilityServices,
-		final JobLeaderListener initialJobLeaderListener) {
+			final String initialOwnerAddress,
+			final RpcService initialRpcService,
+			final HighAvailabilityServices initialHighAvailabilityServices,
+			final JobLeaderListener initialJobLeaderListener) {
 
 		if (JobLeaderService.State.CREATED != state) {
 			throw new IllegalStateException("The service has already been started.");
 		} else {
 			LOG.info("Start job leader service.");
 
+			this.ownerAddress = Preconditions.checkNotNull(initialOwnerAddress);
 			this.rpcService = Preconditions.checkNotNull(initialRpcService);
 			this.highAvailabilityServices = Preconditions.checkNotNull(initialHighAvailabilityServices);
 			this.jobLeaderListener = Preconditions.checkNotNull(initialJobLeaderListener);
@@ -310,6 +317,7 @@ public class JobLeaderService {
 						JobMasterGateway.class,
 						getTargetAddress(),
 						getTargetLeaderId(),
+						ownerAddress,
 						ownLocation);
 			}
 
@@ -345,19 +353,23 @@ public class JobLeaderService {
 			extends RetryingRegistration<JobMasterGateway, JMTMRegistrationSuccess>
 	{
 
+		private final String taskManagerRpcAddress;
+
 		private final TaskManagerLocation taskManagerLocation;
 
 		JobManagerRetryingRegistration(
-			Logger log,
-			RpcService rpcService,
-			String targetName,
-			Class<JobMasterGateway> targetType,
-			String targetAddress,
-			UUID leaderId,
-			TaskManagerLocation taskManagerLocation) {
-
+				Logger log,
+				RpcService rpcService,
+				String targetName,
+				Class<JobMasterGateway> targetType,
+				String targetAddress,
+				UUID leaderId,
+				String taskManagerRpcAddress,
+				TaskManagerLocation taskManagerLocation)
+		{
 			super(log, rpcService, targetName, targetType, targetAddress, leaderId);
 
+			this.taskManagerRpcAddress = taskManagerRpcAddress;
 			this.taskManagerLocation = Preconditions.checkNotNull(taskManagerLocation);
 		}
 
@@ -365,7 +377,8 @@ public class JobLeaderService {
 		protected Future<RegistrationResponse> invokeRegistration(
 				JobMasterGateway gateway, UUID leaderId, long timeoutMillis) throws Exception
 		{
-			return gateway.registerTaskManager(taskManagerLocation, leaderId, Time.milliseconds(timeoutMillis));
+			return gateway.registerTaskManager(taskManagerRpcAddress, taskManagerLocation,
+					leaderId, Time.milliseconds(timeoutMillis));
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/a7ed9a5e/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 1b1c02b..5146e5b 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
@@ -208,7 +208,7 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 		taskSlotTable.start(new SlotActionsImpl());
 
 		// start the job leader service
-		jobLeaderService.start(getRpcService(), haServices, new JobLeaderListenerImpl());
+		jobLeaderService.start(getAddress(), getRpcService(), haServices, new JobLeaderListenerImpl());
 	}
 
 	/**

http://git-wip-us.apache.org/repos/asf/flink/blob/a7ed9a5e/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java
index a8da4fd..55cc142 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java
@@ -394,9 +394,10 @@ public class TaskExecutorTest extends TestLogger {
 		final JobMasterGateway jobMasterGateway = mock(JobMasterGateway.class);
 
 		when(jobMasterGateway.registerTaskManager(
-			eq(taskManagerLocation),
-			eq(jobManagerLeaderId),
-			any(Time.class)
+				any(String.class),
+				eq(taskManagerLocation),
+				eq(jobManagerLeaderId),
+				any(Time.class)
 		)).thenReturn(FlinkCompletableFuture.<RegistrationResponse>completed(new JMTMRegistrationSuccess(jmResourceId, blobPort)));
 		when(jobMasterGateway.getAddress()).thenReturn(jobManagerAddress);
 
@@ -498,9 +499,10 @@ public class TaskExecutorTest extends TestLogger {
 		final JobMasterGateway jobMasterGateway = mock(JobMasterGateway.class);
 
 		when(jobMasterGateway.registerTaskManager(
-			eq(taskManagerLocation),
-			eq(jobManagerLeaderId),
-			any(Time.class)
+				any(String.class),
+				eq(taskManagerLocation),
+				eq(jobManagerLeaderId),
+				any(Time.class)
 		)).thenReturn(FlinkCompletableFuture.<RegistrationResponse>completed(new JMTMRegistrationSuccess(jmResourceId, blobPort)));
 		when(jobMasterGateway.getAddress()).thenReturn(jobManagerAddress);
 


[07/52] [abbrv] flink git commit: [FLINK-4836] [cluster management] Add flink mini cluster (part 1)

Posted by se...@apache.org.
[FLINK-4836] [cluster management] Add flink mini cluster (part 1)

This implements
  - mini cluster configuration
  - startup / shutdown of common services (rpc, ha)
  - startup / shutdown of JobManager and Dispatcher


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

Branch: refs/heads/master
Commit: 106cb9e3ddb8ad019db99bc746e77c2ef48cb5e4
Parents: 9615f15
Author: Stephan Ewen <se...@apache.org>
Authored: Sat Oct 15 00:25:41 2016 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Dec 23 20:54:23 2016 +0100

----------------------------------------------------------------------
 .../org/apache/flink/util/ExceptionUtils.java   |  12 +-
 .../HighAvailabilityServicesUtils.java          |  17 +
 .../highavailability/ZookeeperHaServices.java   |   2 +-
 .../runtime/jobmaster/JobManagerRunner.java     |   1 -
 .../jobmaster/MiniClusterJobDispatcher.java     | 394 -----------------
 .../flink/runtime/minicluster/MiniCluster.java  | 406 ++++++++++++++++++
 .../minicluster/MiniClusterConfiguration.java   | 147 +++++++
 .../minicluster/MiniClusterJobDispatcher.java   | 418 +++++++++++++++++++
 .../resourcemanager/ResourceManager.java        |   2 +-
 .../runtime/taskexecutor/JobLeaderService.java  |   3 +-
 .../TestingHighAvailabilityServices.java        |   2 +-
 .../runtime/minicluster/MiniClusterITCase.java  |  79 ++++
 12 files changed, 1075 insertions(+), 408 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/106cb9e3/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 d1357a8..1069f2d 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
@@ -110,8 +110,6 @@ public final class ExceptionUtils {
 		}
 	}
 
-
-
 	/**
 	 * Throws the given {@code Throwable} in scenarios where the signatures do not allow you to
 	 * throw an arbitrary Throwable. Errors and RuntimeExceptions are thrown directly, other exceptions
@@ -214,10 +212,8 @@ public final class ExceptionUtils {
 		}
 	}
 
-	/**
-	 * Private constructor to prevent instantiation.
-	 */
-	private ExceptionUtils() {
-		throw new RuntimeException();
-	}
+	// ------------------------------------------------------------------------
+
+	/** Private constructor to prevent instantiation. */
+	private ExceptionUtils() {}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/106cb9e3/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServicesUtils.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServicesUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServicesUtils.java
index f3da847..9113309 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServicesUtils.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServicesUtils.java
@@ -24,6 +24,23 @@ import org.apache.flink.runtime.util.LeaderRetrievalUtils;
 
 public class HighAvailabilityServicesUtils {
 
+	public static HighAvailabilityServices createAvailableOrEmbeddedServices(Configuration config) throws Exception {
+		HighAvailabilityMode highAvailabilityMode = LeaderRetrievalUtils.getRecoveryMode(config);
+
+		switch (highAvailabilityMode) {
+			case NONE:
+				return new EmbeddedNonHaServices();
+
+			case ZOOKEEPER:
+				throw new UnsupportedOperationException("ZooKeeper high availability services " +
+						"have not been implemented yet.");
+
+			default:
+				throw new Exception("High availability mode " + highAvailabilityMode + " is not supported.");
+		}
+	}
+	
+	
 	public static HighAvailabilityServices createHighAvailabilityServices(Configuration configuration) throws Exception {
 		HighAvailabilityMode highAvailabilityMode = LeaderRetrievalUtils.getRecoveryMode(configuration);
 

http://git-wip-us.apache.org/repos/asf/flink/blob/106cb9e3/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 be19c60..e38840b 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
@@ -113,7 +113,7 @@ public class ZookeeperHaServices implements HighAvailabilityServices {
 	}
 
 	@Override
-	public LeaderRetrievalService getJobManagerLeaderRetriever(JobID jobID, String defaultAddress) throws Exception {
+	public LeaderRetrievalService getJobManagerLeaderRetriever(JobID jobID) throws Exception {
 		return ZooKeeperUtils.createLeaderRetrievalService(client, configuration, getPathForJobManager(jobID));
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/106cb9e3/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 74ca6f3..3313d8a 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
@@ -289,7 +289,6 @@ public class JobManagerRunner implements LeaderContender, OnCompletionActions, F
 	@Override
 	public void jobFinishedByOther() {
 		try {
-			unregisterJobFromHighAvailability();
 			shutdownInternally();
 		}
 		finally {

http://git-wip-us.apache.org/repos/asf/flink/blob/106cb9e3/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
deleted file mode 100644
index 019ccfe..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/MiniClusterJobDispatcher.java
+++ /dev/null
@@ -1,394 +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;
-
-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.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;
-import org.slf4j.LoggerFactory;
-
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import static org.apache.flink.util.Preconditions.checkArgument;
-import static org.apache.flink.util.Preconditions.checkNotNull;
-import static org.apache.flink.util.Preconditions.checkState;
-
-/**
- * The dispatcher that runs in the mini cluster, waits for jobs, and starts job masters
- * upon receiving jobs.
- */
-public class MiniClusterJobDispatcher {
-
-	private static final Logger LOG = LoggerFactory.getLogger(MiniClusterJobDispatcher.class);
-
-	// ------------------------------------------------------------------------
-
-	/** lock to ensure that this dispatcher executes only one job at a time */
-	private final Object lock = new Object();
-
-	/** the configuration with which the mini cluster was started */
-	private final Configuration configuration;
-
-	/** the RPC service to use by the job managers */
-	private final RpcService rpcService;
-
-	/** services for discovery, leader election, and recovery */
-	private final HighAvailabilityServices haServices;
-
-	/** 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;
-
-	/** The runner for the job and master. non-null if a job is currently running */
-	private volatile JobManagerRunner[] runners;
-
-	/** flag marking the dispatcher as hut down */
-	private volatile boolean shutdown;
-
-
-	/**
-	 * Starts a mini cluster job dispatcher.
-	 * 
-	 * <p>The dispatcher kicks off one JobManager per job, a behavior similar to a
-	 * non-highly-available setup.
-	 * 
-	 * @param config The configuration of the mini cluster
-	 * @param haServices Access to the discovery, leader election, and recovery services
-	 * 
-	 * @throws Exception Thrown, if the services for the JobMaster could not be started.
-	 */
-	public MiniClusterJobDispatcher(
-			Configuration config,
-			RpcService rpcService,
-			HighAvailabilityServices haServices,
-			MetricRegistry metricRegistry) throws Exception {
-		this(config, rpcService, haServices, metricRegistry, 1);
-	}
-
-	/**
-	 * Starts a mini cluster job dispatcher.
-	 *
-	 * <p>The dispatcher may kick off more than one JobManager per job, thus simulating
-	 * a highly-available setup.
-	 * 
-	 * @param config The configuration of the mini cluster
-	 * @param haServices Access to the discovery, leader election, and recovery services
-	 * @param numJobManagers The number of JobMasters to start for each job.
-	 * 
-	 * @throws Exception Thrown, if the services for the JobMaster could not be started.
-	 */
-	public 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, haServices);
-	}
-
-	// ------------------------------------------------------------------------
-	//  life cycle
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Shuts down the mini cluster dispatcher. If a job is currently running, that job will be
-	 * terminally failed.
-	 */
-	public void shutdown() {
-		synchronized (lock) {
-			if (!shutdown) {
-				shutdown = true;
-
-				LOG.info("Shutting down the dispatcher");
-
-				// in this shutdown code we copy the references to the stack first,
-				// to avoid concurrent modification
-
-				JobManagerRunner[] runners = this.runners;
-				if (runners != null) {
-					this.runners = null;
-
-					for (JobManagerRunner runner : runners) {
-						runner.shutdown();
-					}
-				}
-			}
-		}
-	}
-
-	// ------------------------------------------------------------------------
-	//  submitting jobs
-	// ------------------------------------------------------------------------
-
-	/**
-	 * This method executes a job in detached mode. The method returns immediately after the job
-	 * has been added to the
-	 *
-	 * @param job  The Flink job to execute
-	 *
-	 * @throws JobExecutionException Thrown if anything went amiss during initial job launch,
-	 *         or if the job terminally failed.
-	 */
-	public void runDetached(JobGraph job) throws JobExecutionException {
-		checkNotNull(job);
-
-		LOG.info("Received job for detached execution {} ({})", job.getName(), job.getJobID());
-
-		synchronized (lock) {
-			checkState(!shutdown, "mini cluster is shut down");
-			checkState(runners == null, "mini cluster can only execute one job at a time");
-
-			DetachedFinalizer finalizer = new DetachedFinalizer(numJobManagers);
-
-			this.runners = startJobRunners(job, finalizer, finalizer);
-		}
-	}
-
-	/**
-	 * This method runs a job in blocking mode. The method returns only after the job
-	 * completed successfully, or after it failed terminally.
-	 *
-	 * @param job  The Flink job to execute 
-	 * @return The result of the job execution
-	 *
-	 * @throws JobExecutionException Thrown if anything went amiss during initial job lauch,
-	 *         or if the job terminally failed.
-	 */
-	public JobExecutionResult runJobBlocking(JobGraph job) throws JobExecutionException, InterruptedException {
-		checkNotNull(job);
-		
-		LOG.info("Received job for blocking execution {} ({})", job.getName(), job.getJobID());
-		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, sync, sync);
-		}
-
-		try {
-			return sync.getResult();
-		}
-		finally {
-			// always clear the status for the next job
-			runners = null;
-		}
-	}
-
-	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, metricRegistry, 
-						onCompletion, errorHandler);
-				runners[i].start();
-			}
-			catch (Throwable t) {
-				// shut down all the ones so far
-				for (int k = 0; k <= i; k++) {
-					try {
-						if (runners[i] != null) {
-							runners[i].shutdown();
-						}
-					} catch (Throwable ignored) {
-						// silent shutdown
-					}
-				}
-
-				throw new JobExecutionException(job.getJobID(), "Could not start the JobManager(s) for the job", t);
-			}
-		}
-
-		return runners;
-	}
-
-	// ------------------------------------------------------------------------
-	//  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"));
-//	}
-
-	// ------------------------------------------------------------------------
-	//  utility classes
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Simple class that waits for all runners to have reported that they are done.
-	 * 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, FatalErrorHandler {
-
-		private final AtomicInteger numJobManagersToWaitFor;
-
-		private DetachedFinalizer(int numJobManagersToWaitFor) {
-			this.numJobManagersToWaitFor = new AtomicInteger(numJobManagersToWaitFor);
-		}
-
-		@Override
-		public void jobFinished(JobExecutionResult result) {
-			decrementCheckAndCleanup();
-		}
-
-		@Override
-		public void jobFailed(Throwable cause) {
-			decrementCheckAndCleanup();
-		}
-
-		@Override
-		public void jobFinishedByOther() {
-			decrementCheckAndCleanup();
-		}
-
-		@Override
-		public void onFatalError(Throwable exception) {
-			decrementCheckAndCleanup();
-		}
-
-		private void decrementCheckAndCleanup() {
-			if (numJobManagersToWaitFor.decrementAndGet() == 0) {
-				MiniClusterJobDispatcher.this.runners = null;
-			}
-		}
-	}
-
-	// ------------------------------------------------------------------------
-
-	/**
-	 * This class is used to sync on blocking jobs across multiple runners.
-	 * Only after all runners reported back that they are finished, the
-	 * result will be released.
-	 * 
-	 * 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, FatalErrorHandler {
-
-		private final JobID jobId;
-
-		private final CountDownLatch jobMastersToWaitFor;
-
-		private volatile Throwable jobException;
-
-		private volatile Throwable runnerException;
-
-		private volatile JobExecutionResult result;
-		
-		BlockingJobSync(JobID jobId, int numJobMastersToWaitFor) {
-			this.jobId = jobId;
-			this.jobMastersToWaitFor = new CountDownLatch(numJobMastersToWaitFor);
-		}
-
-		@Override
-		public void jobFinished(JobExecutionResult jobResult) {
-			this.result = jobResult;
-			jobMastersToWaitFor.countDown();
-		}
-
-		@Override
-		public void jobFailed(Throwable cause) {
-			jobException = cause;
-			jobMastersToWaitFor.countDown();
-		}
-
-		@Override
-		public void jobFinishedByOther() {
-			this.jobMastersToWaitFor.countDown();
-		}
-
-		@Override
-		public void onFatalError(Throwable exception) {
-			if (runnerException == null) {
-				runnerException = exception;
-			}
-		}
-
-		public JobExecutionResult getResult() throws JobExecutionException, InterruptedException {
-			jobMastersToWaitFor.await();
-
-			final Throwable jobFailureCause = this.jobException;
-			final Throwable runnerException = this.runnerException;
-			final JobExecutionResult result = this.result;
-
-			// (1) we check if the job terminated with an exception
-			// (2) we check whether the job completed successfully
-			// (3) we check if we have exceptions from the JobManagers. the job may still have
-			//     completed successfully in that case, if multiple JobMasters were running
-			//     and other took over. only if all encounter a fatal error, the job cannot finish
-
-			if (jobFailureCause != null) {
-				if (jobFailureCause instanceof JobExecutionException) {
-					throw (JobExecutionException) jobFailureCause;
-				}
-				else {
-					throw new JobExecutionException(jobId, "The job execution failed", jobFailureCause);
-				}
-			}
-			else if (result != null) {
-				return result;
-			}
-			else if (runnerException != null) {
-				throw new JobExecutionException(jobId,
-						"The job execution failed because all JobManagers encountered fatal errors", runnerException);
-			}
-			else {
-				throw new IllegalStateException("Bug: Job finished with neither error nor result.");
-			}
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/106cb9e3/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java
new file mode 100644
index 0000000..1ee38e0
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java
@@ -0,0 +1,406 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.minicluster;
+
+import akka.actor.ActorSystem;
+
+import org.apache.flink.api.common.JobExecutionResult;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.UnmodifiableConfiguration;
+import org.apache.flink.runtime.akka.AkkaUtils;
+import org.apache.flink.runtime.client.JobExecutionException;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.metrics.MetricRegistry;
+import org.apache.flink.runtime.metrics.MetricRegistryConfiguration;
+import org.apache.flink.runtime.rpc.RpcService;
+import org.apache.flink.runtime.rpc.akka.AkkaRpcService;
+import org.apache.flink.util.ExceptionUtils;
+
+import scala.Option;
+import scala.Tuple2;
+
+import javax.annotation.concurrent.GuardedBy;
+
+import static org.apache.flink.util.ExceptionUtils.firstOrSuppressed;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+
+public class MiniCluster {
+
+	/** The lock to guard startup / shutdown / manipulation methods */
+	private final Object lock = new Object();
+
+	/** The configuration for this mini cluster */
+	private final MiniClusterConfiguration config;
+
+	@GuardedBy("lock")
+	private MetricRegistry metricRegistry;
+
+	@GuardedBy("lock")
+	private RpcService commonRpcService;
+
+	@GuardedBy("lock")
+	private RpcService[] jobManagerRpcServices;
+
+	@GuardedBy("lock")
+	private RpcService[] taskManagerRpcServices;
+
+	@GuardedBy("lock")
+	private HighAvailabilityServices haServices;
+
+	@GuardedBy("lock")
+	private MiniClusterJobDispatcher jobDispatcher;
+
+	/** Flag marking the mini cluster as started/running */
+	@GuardedBy("lock")
+	private boolean running;
+
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Creates a new mini cluster with the default configuration:
+	 * <ul>
+	 *     <li>One JobManager</li>
+	 *     <li>One TaskManager</li>
+	 *     <li>One task slot in the TaskManager</li>
+	 *     <li>All components share the same RPC subsystem (minimizes communication overhead)</li>
+	 * </ul>
+	 */
+	public MiniCluster() {
+		this(new MiniClusterConfiguration());
+	}
+
+	/**
+	 * 
+	 * @param config The configuration for the mini cluster
+	 */
+	public MiniCluster(MiniClusterConfiguration config) {
+		this.config = checkNotNull(config, "config may not be null");
+	}
+
+	/**
+	 * Creates a mini cluster based on the given configuration.
+	 * 
+	 * @deprecated Use {@link #MiniCluster(MiniClusterConfiguration)} instead. 
+	 * @see #MiniCluster(MiniClusterConfiguration)
+	 */
+	@Deprecated
+	public MiniCluster(Configuration config) {
+		this(createConfig(config, true));
+	}
+
+	/**
+	 * Creates a mini cluster based on the given configuration, starting one or more
+	 * RPC services, depending on the given flag.
+	 *
+	 * @deprecated Use {@link #MiniCluster(MiniClusterConfiguration)} instead. 
+	 * @see #MiniCluster(MiniClusterConfiguration)
+	 */
+	@Deprecated
+	public MiniCluster(Configuration config, boolean singleRpcService) {
+		this(createConfig(config, singleRpcService));
+	}
+
+	// ------------------------------------------------------------------------
+	//  life cycle
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Checks if the mini cluster was started and is running.
+	 */
+	public boolean isRunning() {
+		return running;
+	}
+
+	/**
+	 * Starts the mini cluster, based on the configured properties.
+	 * 
+	 * @throws Exception This method passes on any exception that occurs during the startup of
+	 *                   the mini cluster.
+	 */
+	public void start() throws Exception {
+		synchronized (lock) {
+			checkState(!running, "FlinkMiniCluster is already running");
+
+			final Configuration configuration = new UnmodifiableConfiguration(config.getConfiguration());
+			final Time rpcTimeout = config.getRpcTimeout();
+			final int numJobManagers = config.getNumJobManagers();
+			final int numTaskManagers = config.getNumTaskManagers();
+			final boolean singleRpc = config.getUseSingleRpcSystem();
+
+			try {
+				metricRegistry = createMetricRegistry(configuration);
+
+				RpcService[] jobManagerRpcServices = new RpcService[numJobManagers];
+				RpcService[] taskManagerRpcServices = new RpcService[numTaskManagers];
+
+				// bring up all the RPC services
+				if (singleRpc) {
+					// one common RPC for all
+					commonRpcService = createRpcService(configuration, rpcTimeout, false, null);
+
+					// set that same RPC service for all JobManagers and TaskManagers
+					for (int i = 0; i < numJobManagers; i++) {
+						jobManagerRpcServices[i] = commonRpcService;
+					}
+					for (int i = 0; i < numTaskManagers; i++) {
+						taskManagerRpcServices[i] = commonRpcService;
+					}
+				}
+				else {
+					// start a new service per component, possibly with custom bind addresses
+					final String jobManagerBindAddress = config.getJobManagerBindAddress();
+					final String taskManagerBindAddress = config.getTaskManagerBindAddress();
+
+					for (int i = 0; i < numJobManagers; i++) {
+						jobManagerRpcServices[i] = createRpcService(
+								configuration, rpcTimeout, true, jobManagerBindAddress);
+					}
+
+					for (int i = 0; i < numTaskManagers; i++) {
+						taskManagerRpcServices[i] = createRpcService(
+								configuration, rpcTimeout, true, taskManagerBindAddress);
+					}
+
+					this.jobManagerRpcServices = jobManagerRpcServices;
+					this.taskManagerRpcServices = taskManagerRpcServices;
+				}
+
+				// create the high-availability services
+				haServices = HighAvailabilityServicesUtils.createAvailableOrEmbeddedServices(configuration);
+
+				// bring up the dispatcher that launches JobManagers when jobs submitted
+				jobDispatcher = new MiniClusterJobDispatcher(
+						configuration, haServices, metricRegistry, numJobManagers, jobManagerRpcServices);
+			}
+			catch (Exception e) {
+				// cleanup everything
+				try {
+					shutdownInternally();
+				} catch (Exception ee) {
+					e.addSuppressed(ee);
+				}
+				throw e;
+			}
+
+			// now officially mark this as running
+			running = true;
+		}
+	}
+
+	/**
+	 * Shuts down the mini cluster, failing all currently executing jobs.
+	 * The mini cluster can be started again by calling the {@link #start()} method again.
+	 * 
+	 * <p>This method shuts down all started services and components,
+	 * even if an exception occurs in the process of shutting down some component. 
+	 * 
+	 * @throws Exception Thrown, if the shutdown did not complete cleanly.
+	 */
+	public void shutdown() throws Exception {
+		synchronized (lock) {
+			if (running) {
+				try {
+					shutdownInternally();
+				} finally {
+					running = false;
+				}
+			}
+		}
+	}
+
+	@GuardedBy("lock")
+	private void shutdownInternally() throws Exception {
+		// this should always be called under the lock
+		assert Thread.holdsLock(lock);
+
+		// collect the first exception, but continue and add all successive
+		// exceptions as suppressed
+		Throwable exception = null;
+
+		// cancel all jobs and shut down the job dispatcher
+		if (jobDispatcher != null) {
+			try {
+				jobDispatcher.shutdown();
+			} catch (Exception e) {
+				exception = firstOrSuppressed(e, exception);
+			}
+			jobDispatcher = null;
+		}
+
+		// shut down high-availability services
+		if (haServices != null) {
+			try {
+				haServices.shutdown();
+			} catch (Exception e) {
+				exception = firstOrSuppressed(e, exception);
+			}
+			haServices = null;
+		}
+
+		// shut down the RpcServices
+		if (commonRpcService != null) {
+			exception = shutDownRpc(commonRpcService, exception);
+			commonRpcService = null;
+		}
+		if (jobManagerRpcServices != null) {
+			for (RpcService service : jobManagerRpcServices) {
+				exception = shutDownRpc(service, exception);
+			}
+			jobManagerRpcServices = null;
+		}
+		if (taskManagerRpcServices != null) {
+			for (RpcService service : taskManagerRpcServices) {
+				exception = shutDownRpc(service, exception);
+			}
+			taskManagerRpcServices = null;
+		}
+
+		// metrics shutdown
+		if (metricRegistry != null) {
+			metricRegistry.shutdown();
+			metricRegistry = null;
+		}
+
+		// if anything went wrong, throw the first error with all the additional suppressed exceptions
+		if (exception != null) {
+			ExceptionUtils.rethrowException(exception, "Error while shutting down mini cluster");
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  running jobs
+	// ------------------------------------------------------------------------
+
+	/**
+	 * This method executes a job in detached mode. The method returns immediately after the job
+	 * has been added to the
+	 *
+	 * @param job  The Flink job to execute
+	 *
+	 * @throws JobExecutionException Thrown if anything went amiss during initial job launch,
+	 *         or if the job terminally failed.
+	 */
+	public void runDetached(JobGraph job) throws JobExecutionException {
+		checkNotNull(job, "job is null");
+
+		synchronized (lock) {
+			checkState(running, "mini cluster is not running");
+			jobDispatcher.runDetached(job);
+		}
+	}
+
+	/**
+	 * This method runs a job in blocking mode. The method returns only after the job
+	 * completed successfully, or after it failed terminally.
+	 *
+	 * @param job  The Flink job to execute 
+	 * @return The result of the job execution
+	 *
+	 * @throws JobExecutionException Thrown if anything went amiss during initial job launch,
+	 *         or if the job terminally failed.
+	 */
+	public JobExecutionResult runJobBlocking(JobGraph job) throws JobExecutionException, InterruptedException {
+		checkNotNull(job, "job is null");
+
+		MiniClusterJobDispatcher dispatcher;
+		synchronized (lock) {
+			checkState(running, "mini cluster is not running");
+			dispatcher = this.jobDispatcher;
+		}
+
+		return dispatcher.runJobBlocking(job);
+	}
+
+	// ------------------------------------------------------------------------
+	//  factories - can be overridden by subclasses to alter behavior
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Factory method to create the metric registry for the mini cluster
+	 * 
+	 * @param config The configuration of the mini cluster
+	 */
+	protected MetricRegistry createMetricRegistry(Configuration config) {
+		return new MetricRegistry(MetricRegistryConfiguration.fromConfiguration(config));
+	}
+
+	/**
+	 * Factory method to instantiate the RPC service.
+	 * 
+	 * @param config
+	 *            The configuration of the mini cluster
+	 * @param askTimeout
+	 *            The default RPC timeout for asynchronous "ask" requests.
+	 * @param remoteEnabled
+	 *            True, if the RPC service should be reachable from other (remote) RPC services.
+	 * @param bindAddress
+	 *            The address to bind the RPC service to. Only relevant when "remoteEnabled" is true.
+	 * 
+	 * @return The instantiated RPC service
+	 */
+	protected RpcService createRpcService(
+			Configuration config,
+			Time askTimeout,
+			boolean remoteEnabled,
+			String bindAddress) {
+
+		ActorSystem actorSystem;
+		if (remoteEnabled) {
+			Tuple2<String, Object> remoteSettings = new Tuple2<String, Object>(bindAddress, 0);
+			actorSystem = AkkaUtils.createActorSystem(config, Option.apply(remoteSettings));
+		} else {
+			actorSystem = AkkaUtils.createLocalActorSystem(config);
+		}
+
+		return new AkkaRpcService(actorSystem, askTimeout);
+	}
+
+	// ------------------------------------------------------------------------
+	//  miscellaneous utilities
+	// ------------------------------------------------------------------------
+
+	private static Throwable shutDownRpc(RpcService rpcService, Throwable priorException) {
+		try {
+			if (rpcService != null) {
+				rpcService.stopService();
+			}
+			return priorException;
+		}
+		catch (Throwable t) {
+			return firstOrSuppressed(t, priorException);
+		}
+	}
+
+	private static MiniClusterConfiguration createConfig(Configuration cfg, boolean singleActorSystem) {
+		MiniClusterConfiguration config = cfg == null ?
+				new MiniClusterConfiguration() :
+				new MiniClusterConfiguration(cfg);
+
+		if (!singleActorSystem) {
+			config.setUseRpcServicePerComponent();
+		}
+
+		return config;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/106cb9e3/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniClusterConfiguration.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniClusterConfiguration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniClusterConfiguration.java
new file mode 100644
index 0000000..a8d7b10
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniClusterConfiguration.java
@@ -0,0 +1,147 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.minicluster;
+
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.akka.AkkaUtils;
+import scala.concurrent.duration.FiniteDuration;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+public class MiniClusterConfiguration {
+
+	private final Configuration config;
+
+	private boolean singleRpcService = true;
+
+	private int numJobManagers = 1;
+
+	private int numTaskManagers = 1;
+
+	private String commonBindAddress;
+
+	// ------------------------------------------------------------------------
+	//  Construction
+	// ------------------------------------------------------------------------
+
+	public MiniClusterConfiguration() {
+		this.config = new Configuration();
+	}
+
+	public MiniClusterConfiguration(Configuration config) {
+		checkNotNull(config);
+		this.config = new Configuration(config);
+	}
+
+	// ------------------------------------------------------------------------
+	//  setters
+	// ------------------------------------------------------------------------
+
+	public void addConfiguration(Configuration config) {
+		checkNotNull(config, "configuration must not be null");
+		this.config.addAll(config);
+	}
+
+	public void setUseSingleRpcService() {
+		this.singleRpcService = true;
+	}
+
+	public void setUseRpcServicePerComponent() {
+		this.singleRpcService = false;
+	}
+
+	public void setNumJobManagers(int numJobManagers) {
+		checkArgument(numJobManagers >= 1, "must have at least one JobManager");
+		this.numJobManagers = numJobManagers;
+	}
+
+	public void setNumTaskManagers(int numTaskManagers) {
+		checkArgument(numTaskManagers >= 1, "must have at least one TaskManager");
+		this.numTaskManagers = numTaskManagers;
+	}
+
+	public void setNumTaskManagerSlots(int numTaskSlots) {
+		checkArgument(numTaskSlots >= 1, "must have at least one task slot per TaskManager");
+		this.config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, numTaskSlots);
+	}
+
+	public void setCommonRpcBindAddress(String bindAddress) {
+		checkNotNull(bindAddress, "bind address must not be null");
+		this.commonBindAddress = bindAddress;
+	}
+
+	// ------------------------------------------------------------------------
+	//  getters
+	// ------------------------------------------------------------------------
+
+	public Configuration getConfiguration() {
+		return config;
+	}
+
+	public boolean getUseSingleRpcSystem() {
+		return singleRpcService;
+	}
+
+	public int getNumJobManagers() {
+		return numJobManagers;
+	}
+
+	public int getNumTaskManagers() {
+		return numTaskManagers;
+	}
+
+	public int getNumSlotsPerTaskManager() {
+		return config.getInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 1);
+	}
+
+	public String getJobManagerBindAddress() {
+		return commonBindAddress != null ?
+				commonBindAddress :
+				config.getString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, "localhost");
+	}
+
+	public String getTaskManagerBindAddress() {
+		return commonBindAddress != null ?
+				commonBindAddress :
+				config.getString(ConfigConstants.TASK_MANAGER_HOSTNAME_KEY, "localhost");
+	}
+
+	public Time getRpcTimeout() {
+		FiniteDuration duration = AkkaUtils.getTimeout(config);
+		return Time.of(duration.length(), duration.unit());
+	}
+
+	// ------------------------------------------------------------------------
+	//  utils
+	// ------------------------------------------------------------------------
+
+	@Override
+	public String toString() {
+		return "MiniClusterConfiguration{" +
+				"singleRpcService=" + singleRpcService +
+				", numJobManagers=" + numJobManagers +
+				", numTaskManagers=" + numTaskManagers +
+				", commonBindAddress='" + commonBindAddress + '\'' +
+				", config=" + config +
+				'}';
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/106cb9e3/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniClusterJobDispatcher.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniClusterJobDispatcher.java b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniClusterJobDispatcher.java
new file mode 100644
index 0000000..d99eff6
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniClusterJobDispatcher.java
@@ -0,0 +1,418 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.minicluster;
+
+import org.apache.flink.api.common.JobExecutionResult;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.configuration.Configuration;
+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.jobmaster.JobManagerRunner;
+import org.apache.flink.runtime.jobmaster.JobManagerServices;
+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;
+import org.slf4j.LoggerFactory;
+
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * The dispatcher that runs in the mini cluster, waits for jobs, and starts job masters
+ * upon receiving jobs.
+ */
+public class MiniClusterJobDispatcher {
+
+	private static final Logger LOG = LoggerFactory.getLogger(MiniClusterJobDispatcher.class);
+
+	// ------------------------------------------------------------------------
+
+	/** lock to ensure that this dispatcher executes only one job at a time */
+	private final Object lock = new Object();
+
+	/** the configuration with which the mini cluster was started */
+	private final Configuration configuration;
+
+	/** the RPC services to use by the job managers */
+	private final RpcService[] rpcServices;
+
+	/** services for discovery, leader election, and recovery */
+	private final HighAvailabilityServices haServices;
+
+	/** 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;
+
+	/** The runner for the job and master. non-null if a job is currently running */
+	private volatile JobManagerRunner[] runners;
+
+	/** flag marking the dispatcher as hut down */
+	private volatile boolean shutdown;
+
+
+	/**
+	 * Starts a mini cluster job dispatcher.
+	 * 
+	 * <p>The dispatcher kicks off one JobManager per job, a behavior similar to a
+	 * non-highly-available setup.
+	 * 
+	 * @param config The configuration of the mini cluster
+	 * @param haServices Access to the discovery, leader election, and recovery services
+	 * 
+	 * @throws Exception Thrown, if the services for the JobMaster could not be started.
+	 */
+	public MiniClusterJobDispatcher(
+			Configuration config,
+			RpcService rpcService,
+			HighAvailabilityServices haServices,
+			MetricRegistry metricRegistry) throws Exception {
+		this(config, haServices, metricRegistry, 1, new RpcService[] { rpcService });
+	}
+
+	/**
+	 * Starts a mini cluster job dispatcher.
+	 *
+	 * <p>The dispatcher may kick off more than one JobManager per job, thus simulating
+	 * a highly-available setup.
+	 * 
+	 * @param config The configuration of the mini cluster
+	 * @param haServices Access to the discovery, leader election, and recovery services
+	 * @param numJobManagers The number of JobMasters to start for each job.
+	 * 
+	 * @throws Exception Thrown, if the services for the JobMaster could not be started.
+	 */
+	public MiniClusterJobDispatcher(
+			Configuration config,
+			HighAvailabilityServices haServices,
+			MetricRegistry metricRegistry,
+			int numJobManagers,
+			RpcService[] rpcServices) throws Exception {
+		
+		checkArgument(numJobManagers >= 1);
+		checkArgument(rpcServices.length == numJobManagers);
+		
+		this.configuration = checkNotNull(config);
+		this.rpcServices = rpcServices;
+		this.haServices = checkNotNull(haServices);
+		this.metricRegistry = checkNotNull(metricRegistry);
+		this.numJobManagers = numJobManagers;
+
+		LOG.info("Creating JobMaster services");
+		this.jobManagerServices = JobManagerServices.fromConfiguration(config, haServices);
+	}
+
+	// ------------------------------------------------------------------------
+	//  life cycle
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Shuts down the mini cluster dispatcher. If a job is currently running, that job will be
+	 * terminally failed.
+	 */
+	public void shutdown() {
+		synchronized (lock) {
+			if (!shutdown) {
+				shutdown = true;
+
+				LOG.info("Shutting down the dispatcher");
+
+				// in this shutdown code we copy the references to the stack first,
+				// to avoid concurrent modification
+
+				JobManagerRunner[] runners = this.runners;
+				if (runners != null) {
+					this.runners = null;
+
+					for (JobManagerRunner runner : runners) {
+						runner.shutdown();
+					}
+				}
+			}
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  submitting jobs
+	// ------------------------------------------------------------------------
+
+	/**
+	 * This method executes a job in detached mode. The method returns immediately after the job
+	 * has been added to the
+	 *
+	 * @param job  The Flink job to execute
+	 *
+	 * @throws JobExecutionException Thrown if anything went amiss during initial job launch,
+	 *         or if the job terminally failed.
+	 */
+	public void runDetached(JobGraph job) throws JobExecutionException {
+		checkNotNull(job);
+
+		LOG.info("Received job for detached execution {} ({})", job.getName(), job.getJobID());
+
+		synchronized (lock) {
+			checkState(!shutdown, "mini cluster is shut down");
+			checkState(runners == null, "mini cluster can only execute one job at a time");
+
+			DetachedFinalizer finalizer = new DetachedFinalizer(numJobManagers);
+
+			this.runners = startJobRunners(job, finalizer, finalizer);
+		}
+	}
+
+	/**
+	 * This method runs a job in blocking mode. The method returns only after the job
+	 * completed successfully, or after it failed terminally.
+	 *
+	 * @param job  The Flink job to execute 
+	 * @return The result of the job execution
+	 *
+	 * @throws JobExecutionException Thrown if anything went amiss during initial job launch,
+	 *         or if the job terminally failed.
+	 */
+	public JobExecutionResult runJobBlocking(JobGraph job) throws JobExecutionException, InterruptedException {
+		checkNotNull(job);
+		
+		LOG.info("Received job for blocking execution {} ({})", job.getName(), job.getJobID());
+		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, sync, sync);
+		}
+
+		try {
+			return sync.getResult();
+		}
+		finally {
+			// always clear the status for the next job
+			runners = null;
+		}
+	}
+
+	private JobManagerRunner[] startJobRunners(
+			JobGraph job,
+			OnCompletionActions onCompletion,
+			FatalErrorHandler errorHandler) throws JobExecutionException {
+
+		LOG.info("Starting {} JobMaster(s) for job {} ({})", numJobManagers, job.getName(), job.getJobID());
+
+		// we first need to mark the job as running in the HA services, so that the
+		// JobManager leader will recognize that it as work to do
+		try {
+			haServices.getRunningJobsRegistry().setJobRunning(job.getJobID());
+		}
+		catch (Throwable t) {
+			throw new JobExecutionException(job.getJobID(),
+					"Could not register the job at the high-availability services", t);
+		}
+
+		// start all JobManagers
+		JobManagerRunner[] runners = new JobManagerRunner[numJobManagers];
+		for (int i = 0; i < numJobManagers; i++) {
+			try {
+				runners[i] = new JobManagerRunner(job, configuration,
+						rpcServices[i], haServices, jobManagerServices, metricRegistry, 
+						onCompletion, errorHandler);
+				runners[i].start();
+			}
+			catch (Throwable t) {
+				// shut down all the ones so far
+				for (int k = 0; k <= i; k++) {
+					try {
+						if (runners[i] != null) {
+							runners[i].shutdown();
+						}
+					} catch (Throwable ignored) {
+						// silent shutdown
+					}
+				}
+
+				// un-register the job from the high.availability services
+				try {
+					haServices.getRunningJobsRegistry().setJobFinished(job.getJobID());
+				}
+				catch (Throwable tt) {
+					LOG.warn("Could not properly unregister job from high-availability services", tt);
+				}
+
+				throw new JobExecutionException(job.getJobID(), "Could not start the JobManager(s) for the job", t);
+			}
+		}
+
+		return runners;
+	}
+
+	// ------------------------------------------------------------------------
+	//  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"));
+//	}
+
+	// ------------------------------------------------------------------------
+	//  utility classes
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Simple class that waits for all runners to have reported that they are done.
+	 * 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, FatalErrorHandler {
+
+		private final AtomicInteger numJobManagersToWaitFor;
+
+		private DetachedFinalizer(int numJobManagersToWaitFor) {
+			this.numJobManagersToWaitFor = new AtomicInteger(numJobManagersToWaitFor);
+		}
+
+		@Override
+		public void jobFinished(JobExecutionResult result) {
+			decrementCheckAndCleanup();
+		}
+
+		@Override
+		public void jobFailed(Throwable cause) {
+			decrementCheckAndCleanup();
+		}
+
+		@Override
+		public void jobFinishedByOther() {
+			decrementCheckAndCleanup();
+		}
+
+		@Override
+		public void onFatalError(Throwable exception) {
+			decrementCheckAndCleanup();
+		}
+
+		private void decrementCheckAndCleanup() {
+			if (numJobManagersToWaitFor.decrementAndGet() == 0) {
+				MiniClusterJobDispatcher.this.runners = null;
+			}
+		}
+	}
+
+	// ------------------------------------------------------------------------
+
+	/**
+	 * This class is used to sync on blocking jobs across multiple runners.
+	 * Only after all runners reported back that they are finished, the
+	 * result will be released.
+	 * 
+	 * 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, FatalErrorHandler {
+
+		private final JobID jobId;
+
+		private final CountDownLatch jobMastersToWaitFor;
+
+		private volatile Throwable jobException;
+
+		private volatile Throwable runnerException;
+
+		private volatile JobExecutionResult result;
+		
+		BlockingJobSync(JobID jobId, int numJobMastersToWaitFor) {
+			this.jobId = jobId;
+			this.jobMastersToWaitFor = new CountDownLatch(numJobMastersToWaitFor);
+		}
+
+		@Override
+		public void jobFinished(JobExecutionResult jobResult) {
+			this.result = jobResult;
+			jobMastersToWaitFor.countDown();
+		}
+
+		@Override
+		public void jobFailed(Throwable cause) {
+			jobException = cause;
+			jobMastersToWaitFor.countDown();
+		}
+
+		@Override
+		public void jobFinishedByOther() {
+			this.jobMastersToWaitFor.countDown();
+		}
+
+		@Override
+		public void onFatalError(Throwable exception) {
+			if (runnerException == null) {
+				runnerException = exception;
+			}
+		}
+
+		public JobExecutionResult getResult() throws JobExecutionException, InterruptedException {
+			jobMastersToWaitFor.await();
+
+			final Throwable jobFailureCause = this.jobException;
+			final Throwable runnerException = this.runnerException;
+			final JobExecutionResult result = this.result;
+
+			// (1) we check if the job terminated with an exception
+			// (2) we check whether the job completed successfully
+			// (3) we check if we have exceptions from the JobManagers. the job may still have
+			//     completed successfully in that case, if multiple JobMasters were running
+			//     and other took over. only if all encounter a fatal error, the job cannot finish
+
+			if (jobFailureCause != null) {
+				if (jobFailureCause instanceof JobExecutionException) {
+					throw (JobExecutionException) jobFailureCause;
+				}
+				else {
+					throw new JobExecutionException(jobId, "The job execution failed", jobFailureCause);
+				}
+			}
+			else if (result != null) {
+				return result;
+			}
+			else if (runnerException != null) {
+				throw new JobExecutionException(jobId,
+						"The job execution failed because all JobManagers encountered fatal errors", runnerException);
+			}
+			else {
+				throw new IllegalStateException("Bug: Job finished with neither error nor result.");
+			}
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/106cb9e3/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java
index 6f6d525..3122804 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java
@@ -188,7 +188,7 @@ public abstract class ResourceManager<WorkerType extends Serializable>
 		} else {
 			try {
 				LeaderRetrievalService jobMasterLeaderRetriever =
-					highAvailabilityServices.getJobManagerLeaderRetriever(jobID, jobMasterAddress);
+					highAvailabilityServices.getJobManagerLeaderRetriever(jobID);
 				jobIdLeaderListener = new JobIdLeaderListener(jobID, jobMasterLeaderRetriever);
 			} catch (Exception e) {
 				log.warn("Failed to start JobMasterLeaderRetriever for JobID {}", jobID);

http://git-wip-us.apache.org/repos/asf/flink/blob/106cb9e3/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/JobLeaderService.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/JobLeaderService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/JobLeaderService.java
index 9e71349..e7f52e2 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/JobLeaderService.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/JobLeaderService.java
@@ -191,8 +191,7 @@ public class JobLeaderService {
 		LOG.info("Add job {} for job leader monitoring.", jobId);
 
 		final LeaderRetrievalService leaderRetrievalService = highAvailabilityServices.getJobManagerLeaderRetriever(
-			jobId,
-			defaultTargetAddress);
+			jobId);
 
 		JobLeaderService.JobManagerLeaderListener jobManagerLeaderListener = new JobManagerLeaderListener(jobId);
 

http://git-wip-us.apache.org/repos/asf/flink/blob/106cb9e3/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 3e88e8c..877812b 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
@@ -91,7 +91,7 @@ public class TestingHighAvailabilityServices implements HighAvailabilityServices
 	}
 
 	@Override
-	public LeaderRetrievalService getJobManagerLeaderRetriever(JobID jobID, String defaultJobManagerAddress) throws Exception {
+	public LeaderRetrievalService getJobManagerLeaderRetriever(JobID jobID) throws Exception {
 		LeaderRetrievalService service = this.jobMasterLeaderRetrievers.get(jobID);
 		if (service != null) {
 			return service;

http://git-wip-us.apache.org/repos/asf/flink/blob/106cb9e3/flink-runtime/src/test/java/org/apache/flink/runtime/minicluster/MiniClusterITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/minicluster/MiniClusterITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/minicluster/MiniClusterITCase.java
new file mode 100644
index 0000000..dd43337
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/minicluster/MiniClusterITCase.java
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.minicluster;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.testtasks.NoOpInvokable;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Test;
+
+/**
+ * Integration test cases for the {@link MiniCluster}.
+ */
+public class MiniClusterITCase extends TestLogger {
+
+//	@Test
+	public void runJobWithSingleRpcService() throws Exception {
+		MiniClusterConfiguration cfg = new MiniClusterConfiguration();
+
+		// should be the default, but set anyways to make sure the test
+		// stays valid when the default changes
+		cfg.setUseSingleRpcService();
+
+		MiniCluster miniCluster = new MiniCluster(cfg);
+		executeJob(miniCluster);
+	}
+
+//	@Test
+	public void runJobWithMultipleRpcServices() throws Exception {
+		MiniClusterConfiguration cfg = new MiniClusterConfiguration();
+		cfg.setUseRpcServicePerComponent();
+
+		MiniCluster miniCluster = new MiniCluster(cfg);
+		executeJob(miniCluster);
+	}
+
+//	@Test
+	public void runJobWithMultipleJobManagers() throws Exception {
+		MiniClusterConfiguration cfg = new MiniClusterConfiguration();
+		cfg.setNumJobManagers(3);
+
+		MiniCluster miniCluster = new MiniCluster(cfg);
+		executeJob(miniCluster);
+	}
+
+	private static void executeJob(MiniCluster miniCluster) throws Exception {
+		miniCluster.start();
+
+		JobGraph job = getSimpleJob();
+		miniCluster.runJobBlocking(job);
+	}
+
+	private static JobGraph getSimpleJob() {
+		JobVertex task = new JobVertex("Test task");
+		task.setParallelism(1);
+		task.setMaxParallelism(1);
+		task.setInvokableClass(NoOpInvokable.class);
+
+		return new JobGraph(new JobID(), "Test Job", task);
+	}
+}


[27/52] [abbrv] flink git commit: [FLINK-4834] [cluster management] Add LeaderIdMismatchException at a common way to report RPC calls rejected due to outdated leader status.

Posted by se...@apache.org.
[FLINK-4834] [cluster management] Add LeaderIdMismatchException at a common way to report RPC calls rejected due to outdated leader status.


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

Branch: refs/heads/master
Commit: 91f1d098fea09506910a30d2a51517b015613bf4
Parents: 8734100
Author: Stephan Ewen <se...@apache.org>
Authored: Mon Oct 17 20:20:26 2016 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Dec 23 20:54:25 2016 +0100

----------------------------------------------------------------------
 .../LeaderIdMismatchException.java              | 47 ++++++++++++++++++++
 1 file changed, 47 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/91f1d098/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/LeaderIdMismatchException.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/LeaderIdMismatchException.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/LeaderIdMismatchException.java
new file mode 100644
index 0000000..5caf1b2
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/LeaderIdMismatchException.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.highavailability;
+
+import java.util.UUID;
+
+/**
+ * An exception thrown when the leader session id attached to a message does not match
+ * the leader session id that the receiver expects.
+ */
+public class LeaderIdMismatchException extends Exception {
+
+	private static final long serialVersionUID = 1L;
+
+	private final UUID expected;
+	private final UUID actual;
+
+	public LeaderIdMismatchException(UUID expected, UUID actual) {
+		super("Leader session ID mismatch: expected=" + expected + ", actual=" + actual);
+		this.expected = expected;
+		this.actual = actual;
+	}
+
+	public UUID getExpected() {
+		return expected;
+	}
+
+	public UUID getActual() {
+		return actual;
+	}
+}


[24/52] [abbrv] flink git commit: [FLINK-4958] [tm] Send slot report to RM when registering

Posted by se...@apache.org.
[FLINK-4958] [tm] Send slot report to RM when registering

Fix failing test cases


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

Branch: refs/heads/master
Commit: 873410010df4be494f3573c4adfc2cbbc3ad5d0b
Parents: 5776235
Author: Till Rohrmann <tr...@apache.org>
Authored: Fri Oct 28 15:04:00 2016 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Dec 23 20:54:25 2016 +0100

----------------------------------------------------------------------
 .../runtime/taskexecutor/TaskExecutor.java      |  16 +-
 ...TaskExecutorToResourceManagerConnection.java |  48 +++--
 .../taskexecutor/slot/TaskSlotTable.java        |  31 ++++
 .../taskexecutor/TaskExecutorITCase.java        | 183 +++++++++++++++++++
 .../runtime/taskexecutor/TaskExecutorTest.java  |  16 +-
 5 files changed, 277 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/87341001/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 c94113c..f11cb98 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
@@ -626,10 +626,14 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 			resourceManagerConnection =
 				new TaskExecutorToResourceManagerConnection(
 					log,
-					this,
+					getRpcService(),
+					getAddress(),
+					getResourceID(),
+					taskSlotTable.createSlotReport(getResourceID()),
 					newLeaderAddress,
 					newLeaderId,
-					getMainThreadExecutor());
+					getMainThreadExecutor(),
+					new ForwardingFatalErrorHandler());
 			resourceManagerConnection.start();
 		}
 	}
@@ -1054,6 +1058,14 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 		}
 	}
 
+	private final class ForwardingFatalErrorHandler implements FatalErrorHandler {
+
+		@Override
+		public void onFatalError(Throwable exception) {
+			onFatalErrorAsync(exception);
+		}
+	}
+
 	private final class TaskManagerActionsImpl implements TaskManagerActions {
 		private final UUID jobMasterLeaderId;
 		private final JobMasterGateway jobMasterGateway;

http://git-wip-us.apache.org/repos/asf/flink/blob/87341001/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorToResourceManagerConnection.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorToResourceManagerConnection.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorToResourceManagerConnection.java
index 53f030e..6e3e39b 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorToResourceManagerConnection.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorToResourceManagerConnection.java
@@ -22,12 +22,14 @@ import org.apache.flink.api.common.time.Time;
 import org.apache.flink.runtime.clusterframework.types.ResourceID;
 import org.apache.flink.runtime.instance.InstanceID;
 import org.apache.flink.runtime.registration.RegisteredRpcConnection;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
 import org.apache.flink.runtime.rpc.RpcService;
 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.concurrent.Future;
 
+import org.apache.flink.util.Preconditions;
 import org.slf4j.Logger;
 
 import java.util.UUID;
@@ -41,29 +43,49 @@ import static org.apache.flink.util.Preconditions.checkNotNull;
 public class TaskExecutorToResourceManagerConnection
 		extends RegisteredRpcConnection<ResourceManagerGateway, TaskExecutorRegistrationSuccess> {
 
-	/** the TaskExecutor whose connection to the ResourceManager this represents */
-	private final TaskExecutor taskExecutor;
+	private final RpcService rpcService;
+
+	private final String taskManagerAddress;
+
+	private final ResourceID taskManagerResourceId;
+
+	private final SlotReport slotReport;
+
+	private final FatalErrorHandler fatalErrorHandler;
 
 	private InstanceID registrationId;
 
 	public TaskExecutorToResourceManagerConnection(
 			Logger log,
-			TaskExecutor taskExecutor,
+			RpcService rpcService,
+			String taskManagerAddress,
+			ResourceID taskManagerResourceId,
+			SlotReport slotReport,
 			String resourceManagerAddress,
 			UUID resourceManagerLeaderId,
-			Executor executor) {
+			Executor executor,
+			FatalErrorHandler fatalErrorHandler) {
 
 		super(log, resourceManagerAddress, resourceManagerLeaderId, executor);
-		this.taskExecutor = checkNotNull(taskExecutor);
+
+		this.rpcService = Preconditions.checkNotNull(rpcService);
+		this.taskManagerAddress = Preconditions.checkNotNull(taskManagerAddress);
+		this.taskManagerResourceId = Preconditions.checkNotNull(taskManagerResourceId);
+		this.slotReport = Preconditions.checkNotNull(slotReport);
+		this.fatalErrorHandler = Preconditions.checkNotNull(fatalErrorHandler);
 	}
 
 
 	@Override
 	protected RetryingRegistration<ResourceManagerGateway, TaskExecutorRegistrationSuccess> generateRegistration() {
 		return new TaskExecutorToResourceManagerConnection.ResourceManagerRegistration(
-			log, taskExecutor.getRpcService(),
-			getTargetAddress(), getTargetLeaderId(),
-			taskExecutor.getAddress(),taskExecutor.getResourceID());
+			log,
+			rpcService,
+			getTargetAddress(),
+			getTargetLeaderId(),
+			taskManagerAddress,
+			taskManagerResourceId,
+			slotReport);
 	}
 
 	@Override
@@ -78,7 +100,7 @@ public class TaskExecutorToResourceManagerConnection
 	protected void onRegistrationFailure(Throwable failure) {
 		log.info("Failed to register at resource manager {}.", getTargetAddress(), failure);
 
-		taskExecutor.onFatalErrorAsync(failure);
+		fatalErrorHandler.onFatalError(failure);
 	}
 
 	/**
@@ -100,17 +122,21 @@ public class TaskExecutorToResourceManagerConnection
 		
 		private final ResourceID resourceID;
 
+		private final SlotReport slotReport;
+
 		ResourceManagerRegistration(
 				Logger log,
 				RpcService rpcService,
 				String targetAddress,
 				UUID leaderId,
 				String taskExecutorAddress,
-				ResourceID resourceID) {
+				ResourceID resourceID,
+				SlotReport slotReport) {
 
 			super(log, rpcService, "ResourceManager", ResourceManagerGateway.class, targetAddress, leaderId);
 			this.taskExecutorAddress = checkNotNull(taskExecutorAddress);
 			this.resourceID = checkNotNull(resourceID);
+			this.slotReport = checkNotNull(slotReport);
 		}
 
 		@Override
@@ -118,7 +144,7 @@ public class TaskExecutorToResourceManagerConnection
 				ResourceManagerGateway resourceManager, UUID leaderId, long timeoutMillis) throws Exception {
 
 			Time timeout = Time.milliseconds(timeoutMillis);
-			return resourceManager.registerTaskExecutor(leaderId, taskExecutorAddress, resourceID, new SlotReport(), timeout);
+			return resourceManager.registerTaskExecutor(leaderId, taskExecutorAddress, resourceID, slotReport, timeout);
 		}
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/87341001/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotTable.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotTable.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotTable.java
index 88b83a0..081d8f8 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotTable.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotTable.java
@@ -21,8 +21,12 @@ package org.apache.flink.runtime.taskexecutor.slot;
 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.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.taskexecutor.SlotReport;
+import org.apache.flink.runtime.taskexecutor.SlotStatus;
 import org.apache.flink.runtime.taskmanager.Task;
 import org.apache.flink.util.Preconditions;
 import org.slf4j.Logger;
@@ -127,6 +131,33 @@ public class TaskSlotTable implements TimeoutListener<AllocationID> {
 	}
 
 	// ---------------------------------------------------------------------
+	// Slot report methods
+	// ---------------------------------------------------------------------
+
+	public SlotReport createSlotReport(ResourceID resourceId) {
+		final int numberSlots = taskSlots.size();
+
+		List<SlotStatus> slotStatuses = Arrays.asList(new SlotStatus[numberSlots]);
+
+		for (int i = 0; i < numberSlots; i++) {
+			TaskSlot taskSlot = taskSlots.get(i);
+			SlotID slotId = new SlotID(resourceId, taskSlot.getIndex());
+
+			SlotStatus slotStatus = new SlotStatus(
+				slotId,
+				taskSlot.getResourceProfile(),
+				taskSlot.getJobId(),
+				taskSlot.getAllocationId());
+
+			slotStatuses.set(i, slotStatus);
+		}
+
+		final SlotReport slotReport = new SlotReport(slotStatuses);
+
+		return slotReport;
+	}
+
+	// ---------------------------------------------------------------------
 	// Slot methods
 	// ---------------------------------------------------------------------
 

http://git-wip-us.apache.org/repos/asf/flink/blob/87341001/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorITCase.java
new file mode 100644
index 0000000..050db44
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorITCase.java
@@ -0,0 +1,183 @@
+/*
+ * 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.taskexecutor;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.configuration.Configuration;
+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.ResourceProfile;
+import org.apache.flink.runtime.concurrent.Future;
+import org.apache.flink.runtime.concurrent.impl.FlinkCompletableFuture;
+import org.apache.flink.runtime.filecache.FileCache;
+import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices;
+import org.apache.flink.runtime.io.disk.iomanager.IOManager;
+import org.apache.flink.runtime.io.network.NetworkEnvironment;
+import org.apache.flink.runtime.jobmaster.JMTMRegistrationSuccess;
+import org.apache.flink.runtime.jobmaster.JobMasterGateway;
+import org.apache.flink.runtime.jobmaster.JobMasterRegistrationSuccess;
+import org.apache.flink.runtime.leaderelection.TestingLeaderElectionService;
+import org.apache.flink.runtime.leaderelection.TestingLeaderRetrievalService;
+import org.apache.flink.runtime.memory.MemoryManager;
+import org.apache.flink.runtime.metrics.MetricRegistry;
+import org.apache.flink.runtime.metrics.groups.TaskManagerMetricGroup;
+import org.apache.flink.runtime.registration.RegistrationResponse;
+import org.apache.flink.runtime.resourcemanager.JobLeaderIdService;
+import org.apache.flink.runtime.resourcemanager.ResourceManager;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerConfiguration;
+import org.apache.flink.runtime.resourcemanager.SlotRequest;
+import org.apache.flink.runtime.resourcemanager.StandaloneResourceManager;
+import org.apache.flink.runtime.resourcemanager.slotmanager.DefaultSlotManager;
+import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManagerFactory;
+import org.apache.flink.runtime.rpc.TestingSerialRpcService;
+import org.apache.flink.runtime.taskexecutor.slot.SlotOffer;
+import org.apache.flink.runtime.taskexecutor.slot.TaskSlotTable;
+import org.apache.flink.runtime.taskexecutor.slot.TimerService;
+import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
+import org.apache.flink.runtime.util.TestingFatalErrorHandler;
+import org.hamcrest.Matchers;
+import org.junit.Test;
+
+import java.net.InetAddress;
+import java.util.Arrays;
+import java.util.UUID;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.argThat;
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+public class TaskExecutorITCase {
+
+	@Test
+	public void testSlotAllocation() throws Exception {
+		TestingFatalErrorHandler testingFatalErrorHandler = new TestingFatalErrorHandler();
+		TestingHighAvailabilityServices testingHAServices = new TestingHighAvailabilityServices();
+		final Configuration configuration = new Configuration();
+		final ScheduledExecutorService scheduledExecutorService = new ScheduledThreadPoolExecutor(1);
+		final ResourceID taskManagerResourceId = new ResourceID("foobar");
+		final UUID rmLeaderId = UUID.randomUUID();
+		final TestingLeaderElectionService rmLeaderElectionService = new TestingLeaderElectionService();
+		final TestingLeaderRetrievalService rmLeaderRetrievalService = new TestingLeaderRetrievalService();
+		final String rmAddress = "rm";
+		final String jmAddress = "jm";
+		final UUID jmLeaderId = UUID.randomUUID();
+		final JobID jobId = new JobID();
+		final ResourceProfile resourceProfile = new ResourceProfile(1.0, 1L);
+
+		testingHAServices.setResourceManagerLeaderElectionService(rmLeaderElectionService);
+		testingHAServices.setResourceManagerLeaderRetriever(rmLeaderRetrievalService);
+		testingHAServices.setJobMasterLeaderRetriever(jobId, new TestingLeaderRetrievalService(jmAddress, jmLeaderId));
+
+		TestingSerialRpcService rpcService = new TestingSerialRpcService();
+		ResourceManagerConfiguration resourceManagerConfiguration = new ResourceManagerConfiguration(Time.milliseconds(500L), Time.milliseconds(500L));
+		SlotManagerFactory slotManagerFactory = new DefaultSlotManager.Factory();
+		JobLeaderIdService jobLeaderIdService = new JobLeaderIdService(testingHAServices);
+		MetricRegistry metricRegistry = mock(MetricRegistry.class);
+
+		final TaskManagerConfiguration taskManagerConfiguration = TaskManagerConfiguration.fromConfiguration(configuration);
+		final TaskManagerLocation taskManagerLocation = new TaskManagerLocation(taskManagerResourceId, InetAddress.getLocalHost(), 1234);
+		final MemoryManager memoryManager = mock(MemoryManager.class);
+		final IOManager ioManager = mock(IOManager.class);
+		final NetworkEnvironment networkEnvironment = mock(NetworkEnvironment.class);
+		final TaskManagerMetricGroup taskManagerMetricGroup = mock(TaskManagerMetricGroup.class);
+		final BroadcastVariableManager broadcastVariableManager = mock(BroadcastVariableManager.class);
+		final FileCache fileCache = mock(FileCache.class);
+		final TaskSlotTable taskSlotTable = new TaskSlotTable(Arrays.asList(resourceProfile), new TimerService<AllocationID>(scheduledExecutorService));
+		final JobManagerTable jobManagerTable = new JobManagerTable();
+		final JobLeaderService jobLeaderService = new JobLeaderService(taskManagerLocation);
+
+		ResourceManager<ResourceID> resourceManager = new StandaloneResourceManager(
+			rpcService,
+			resourceManagerConfiguration,
+			testingHAServices,
+			slotManagerFactory,
+			metricRegistry,
+			jobLeaderIdService,
+			testingFatalErrorHandler);
+
+		TaskExecutor taskExecutor = new TaskExecutor(
+			taskManagerConfiguration,
+			taskManagerLocation,
+			rpcService,
+			memoryManager,
+			ioManager,
+			networkEnvironment,
+			testingHAServices,
+			metricRegistry,
+			taskManagerMetricGroup,
+			broadcastVariableManager,
+			fileCache,
+			taskSlotTable,
+			jobManagerTable,
+			jobLeaderService,
+			testingFatalErrorHandler);
+
+		JobMasterGateway jmGateway = mock(JobMasterGateway.class);
+
+		when(jmGateway.registerTaskManager(any(String.class), any(TaskManagerLocation.class), eq(jmLeaderId), any(Time.class)))
+			.thenReturn(FlinkCompletableFuture.<RegistrationResponse>completed(new JMTMRegistrationSuccess(taskManagerResourceId, 1234)));
+		when(jmGateway.getAddress()).thenReturn(jmAddress);
+
+
+		rpcService.registerGateway(rmAddress, resourceManager.getSelf());
+		rpcService.registerGateway(jmAddress, jmGateway);
+
+		final AllocationID allocationId = new AllocationID();
+		final SlotRequest slotRequest = new SlotRequest(jobId, allocationId, resourceProfile);
+		final SlotOffer slotOffer = new SlotOffer(allocationId, 0, resourceProfile);
+
+		try {
+			resourceManager.start();
+			taskExecutor.start();
+
+			// notify the RM that it is the leader
+			rmLeaderElectionService.isLeader(rmLeaderId);
+
+			// notify the TM about the new RM leader
+			rmLeaderRetrievalService.notifyListener(rmAddress, rmLeaderId);
+
+			Future<RegistrationResponse> registrationResponseFuture = resourceManager.registerJobManager(rmLeaderId, jmLeaderId, jmAddress, jobId);
+
+			RegistrationResponse registrationResponse = registrationResponseFuture.get();
+
+			assertTrue(registrationResponse instanceof JobMasterRegistrationSuccess);
+
+			resourceManager.requestSlot(jmLeaderId, rmLeaderId, slotRequest);
+
+			verify(jmGateway).offerSlots(
+				eq(taskManagerResourceId),
+				(Iterable<SlotOffer>)argThat(Matchers.contains(slotOffer)),
+				eq(jmLeaderId), any(Time.class));
+		} finally {
+			if (testingFatalErrorHandler.hasExceptionOccurred()) {
+				testingFatalErrorHandler.rethrowError();
+			}
+		}
+
+
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/87341001/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java
index 1ef7140..2af97b5 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java
@@ -119,6 +119,10 @@ public class TaskExecutorTest extends TestLogger {
 
 			NonHaServices haServices = new NonHaServices(resourceManagerAddress);
 
+			final TaskSlotTable taskSlotTable = mock(TaskSlotTable.class);
+			final SlotReport slotReport = new SlotReport();
+			when(taskSlotTable.createSlotReport(any(ResourceID.class))).thenReturn(slotReport);
+
 			TaskExecutor taskManager = new TaskExecutor(
 				taskManagerServicesConfiguration,
 				taskManagerLocation,
@@ -131,7 +135,7 @@ public class TaskExecutorTest extends TestLogger {
 				mock(TaskManagerMetricGroup.class),
 				mock(BroadcastVariableManager.class),
 				mock(FileCache.class),
-				mock(TaskSlotTable.class),
+				taskSlotTable,
 				mock(JobManagerTable.class),
 				mock(JobLeaderService.class),
 				mock(FatalErrorHandler.class));
@@ -140,7 +144,7 @@ public class TaskExecutorTest extends TestLogger {
 			String taskManagerAddress = taskManager.getAddress();
 
 			verify(rmGateway).registerTaskExecutor(
-					any(UUID.class), eq(taskManagerAddress), eq(resourceID), any(SlotReport.class), any(Time.class));
+					any(UUID.class), eq(taskManagerAddress), eq(resourceID), eq(slotReport), any(Time.class));
 		}
 		finally {
 			rpc.stopService();
@@ -178,6 +182,10 @@ public class TaskExecutorTest extends TestLogger {
 			when(taskManagerLocation.getResourceID()).thenReturn(resourceID);
 			when(taskManagerLocation.getHostname()).thenReturn("foobar");
 
+			final TaskSlotTable taskSlotTable = mock(TaskSlotTable.class);
+			final SlotReport slotReport = new SlotReport();
+			when(taskSlotTable.createSlotReport(any(ResourceID.class))).thenReturn(slotReport);
+
 			TaskExecutor taskManager = new TaskExecutor(
 				taskManagerServicesConfiguration,
 				taskManagerLocation,
@@ -190,7 +198,7 @@ public class TaskExecutorTest extends TestLogger {
 				mock(TaskManagerMetricGroup.class),
 				mock(BroadcastVariableManager.class),
 				mock(FileCache.class),
-				mock(TaskSlotTable.class),
+				taskSlotTable,
 				mock(JobManagerTable.class),
 				mock(JobLeaderService.class),
 				mock(FatalErrorHandler.class));
@@ -215,7 +223,7 @@ public class TaskExecutorTest extends TestLogger {
 			testLeaderService.notifyListener(address2, leaderId2);
 
 			verify(rmGateway2).registerTaskExecutor(
-					eq(leaderId2), eq(taskManagerAddress), eq(resourceID), any(SlotReport.class), any(Time.class));
+					eq(leaderId2), eq(taskManagerAddress), eq(resourceID), eq(slotReport), any(Time.class));
 			assertNotNull(taskManager.getResourceManagerConnection());
 		}
 		finally {


[38/52] [abbrv] flink git commit: [FLINK-5170] [akka] Extend AkkaUtils.getAkkaConfig methods to properly work with Java

Posted by se...@apache.org.
[FLINK-5170] [akka] Extend AkkaUtils.getAkkaConfig methods to properly work with Java


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

Branch: refs/heads/master
Commit: 1da7924187c7dc9dc003a77ad0b2752a24e42cb3
Parents: 9ad9fa9
Author: Till Rohrmann <tr...@apache.org>
Authored: Mon Nov 28 12:10:43 2016 +0100
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Dec 23 20:54:26 2016 +0100

----------------------------------------------------------------------
 .../flink/runtime/rpc/RpcServiceUtils.java      | 10 +++++++++-
 .../apache/flink/runtime/akka/AkkaUtils.scala   | 20 +++++++++++++++++++-
 2 files changed, 28 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/1da79241/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcServiceUtils.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcServiceUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcServiceUtils.java
index d40e336..1ac54ac 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcServiceUtils.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcServiceUtils.java
@@ -50,7 +50,15 @@ public class RpcServiceUtils {
 		final ActorSystem actorSystem;
 
 		try {
-			Config akkaConfig = AkkaUtils.getAkkaConfig(configuration, hostname, port);
+			Config akkaConfig;
+
+			if (hostname != null && !hostname.isEmpty()) {
+				// remote akka config
+				akkaConfig = AkkaUtils.getAkkaConfig(configuration, hostname, port);
+			} else {
+				// local akka config
+				akkaConfig = AkkaUtils.getAkkaConfig(configuration);
+			}
 
 			LOG.debug("Using akka configuration \n {}.", akkaConfig);
 

http://git-wip-us.apache.org/repos/asf/flink/blob/1da79241/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 4fe39b6..d94eb7a 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
@@ -114,8 +114,26 @@ object AkkaUtils {
     createActorSystem(getDefaultAkkaConfig)
   }
 
+  /**
+    * Return a remote Akka config for the given configuration values.
+    *
+    * @param configuration containing the user provided configuration values
+    * @param hostname to bind against. If null, then the loopback interface is used
+    * @param port to bind against
+    * @return A remote Akka config
+    */
   def getAkkaConfig(configuration: Configuration, hostname: String, port: Int): Config = {
-    getAkkaConfig(configuration, if (hostname != null) Some((hostname, port)) else None)
+    getAkkaConfig(configuration, Some((hostname, port)))
+  }
+
+  /**
+    * Return a local Akka config for the given configuration values.
+    *
+    * @param configuration containing the user provided configuration values
+    * @return A local Akka config
+    */
+  def getAkkaConfig(configuration: Configuration): Config = {
+    getAkkaConfig(configuration, None)
   }
 
   /**


[19/52] [abbrv] flink git commit: [FLINK-4836] [cluster management] Start ResourceManager and TaskManager services in MiniCluster

Posted by se...@apache.org.
[FLINK-4836] [cluster management] Start ResourceManager and TaskManager services in MiniCluster


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

Branch: refs/heads/master
Commit: 49a296899dacd5701b8790c3b1ccd72c56fdd1ea
Parents: a685c75
Author: Till Rohrmann <tr...@apache.org>
Authored: Mon Oct 17 15:16:59 2016 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Dec 23 20:54:24 2016 +0100

----------------------------------------------------------------------
 .../flink/runtime/minicluster/MiniCluster.java  | 58 +++++++++++++++++++-
 .../minicluster/MiniClusterConfiguration.java   | 20 ++++++-
 .../minicluster/MiniClusterJobDispatcher.java   |  2 +-
 .../runtime/taskexecutor/TaskManagerRunner.java |  3 +-
 .../runtime/minicluster/MiniClusterITCase.java  |  2 +-
 5 files changed, 78 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/49a29689/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java
index d85234d..1ffcd12 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java
@@ -26,6 +26,7 @@ import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.UnmodifiableConfiguration;
 import org.apache.flink.runtime.akka.AkkaUtils;
 import org.apache.flink.runtime.client.JobExecutionException;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
 import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
 import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils;
 import org.apache.flink.runtime.jobgraph.JobGraph;
@@ -33,10 +34,13 @@ import org.apache.flink.runtime.metrics.MetricRegistry;
 import org.apache.flink.runtime.metrics.MetricRegistryConfiguration;
 import org.apache.flink.runtime.rpc.RpcService;
 import org.apache.flink.runtime.rpc.akka.AkkaRpcService;
+import org.apache.flink.runtime.taskexecutor.TaskManagerRunner;
 import org.apache.flink.util.ExceptionUtils;
 
 import javax.annotation.concurrent.GuardedBy;
 
+import java.util.UUID;
+
 import static org.apache.flink.util.ExceptionUtils.firstOrSuppressed;
 import static org.apache.flink.util.Preconditions.checkNotNull;
 import static org.apache.flink.util.Preconditions.checkState;
@@ -63,9 +67,15 @@ public class MiniCluster {
 	private RpcService[] taskManagerRpcServices;
 
 	@GuardedBy("lock")
+	private RpcService[] resourceManagerRpcServices;
+
+	@GuardedBy("lock")
 	private HighAvailabilityServices haServices;
 
 	@GuardedBy("lock")
+	private TaskManagerRunner[] taskManagerRunners;
+
+	@GuardedBy("lock")
 	private MiniClusterJobDispatcher jobDispatcher;
 
 	/** Flag marking the mini cluster as started/running */
@@ -143,6 +153,7 @@ public class MiniCluster {
 			final Time rpcTimeout = config.getRpcTimeout();
 			final int numJobManagers = config.getNumJobManagers();
 			final int numTaskManagers = config.getNumTaskManagers();
+			final int numResourceManagers = config.getNumResourceManagers();
 			final boolean singleRpc = config.getUseSingleRpcSystem();
 
 			try {
@@ -150,6 +161,7 @@ public class MiniCluster {
 
 				RpcService[] jobManagerRpcServices = new RpcService[numJobManagers];
 				RpcService[] taskManagerRpcServices = new RpcService[numTaskManagers];
+				RpcService[] resourceManagerRpcServices = new RpcService[numResourceManagers];
 
 				// bring up all the RPC services
 				if (singleRpc) {
@@ -163,11 +175,19 @@ public class MiniCluster {
 					for (int i = 0; i < numTaskManagers; i++) {
 						taskManagerRpcServices[i] = commonRpcService;
 					}
+					for (int i = 0; i < numResourceManagers; i++) {
+						resourceManagerRpcServices[i] = commonRpcService;
+					}
+
+					this.resourceManagerRpcServices = null;
+					this.jobManagerRpcServices = null;
+					this.taskManagerRpcServices = null;
 				}
 				else {
 					// start a new service per component, possibly with custom bind addresses
 					final String jobManagerBindAddress = config.getJobManagerBindAddress();
 					final String taskManagerBindAddress = config.getTaskManagerBindAddress();
+					final String resourceManagerBindAddress = config.getResourceManagerBindAddress();
 
 					for (int i = 0; i < numJobManagers; i++) {
 						jobManagerRpcServices[i] = createRpcService(
@@ -179,13 +199,23 @@ public class MiniCluster {
 								configuration, rpcTimeout, true, taskManagerBindAddress);
 					}
 
+					for (int i = 0; i < numResourceManagers; i++) {
+						resourceManagerRpcServices[i] = createRpcService(
+								configuration, rpcTimeout, true, resourceManagerBindAddress);
+					}
+
 					this.jobManagerRpcServices = jobManagerRpcServices;
 					this.taskManagerRpcServices = taskManagerRpcServices;
+					this.resourceManagerRpcServices = resourceManagerRpcServices;
 				}
 
 				// create the high-availability services
 				haServices = HighAvailabilityServicesUtils.createAvailableOrEmbeddedServices(configuration);
 
+				// bring up the task managers for the mini cluster
+				taskManagerRunners = startTaskManagers(
+						configuration, haServices, metricRegistry, numTaskManagers, taskManagerRpcServices);
+
 				// bring up the dispatcher that launches JobManagers when jobs submitted
 				jobDispatcher = new MiniClusterJobDispatcher(
 						configuration, haServices, metricRegistry, numJobManagers, jobManagerRpcServices);
@@ -372,6 +402,28 @@ public class MiniCluster {
 		return new AkkaRpcService(actorSystem, askTimeout);
 	}
 
+	protected TaskManagerRunner[] startTaskManagers(
+			Configuration configuration,
+			HighAvailabilityServices haServices,
+			MetricRegistry metricRegistry,
+			int numTaskManagers,
+			RpcService[] taskManagerRpcServices) throws Exception {
+
+		final TaskManagerRunner[] taskManagerRunners = new TaskManagerRunner[numTaskManagers];
+
+		for (int i = 0; i < numTaskManagers; i++) {
+			taskManagerRunners[i] = new TaskManagerRunner(
+				configuration,
+				new ResourceID(UUID.randomUUID().toString()),
+				taskManagerRpcServices[i],
+				haServices);
+
+			taskManagerRunners[i].start();
+		}
+
+		return taskManagerRunners;
+	}
+
 	// ------------------------------------------------------------------------
 	//  miscellaneous utilities
 	// ------------------------------------------------------------------------
@@ -388,12 +440,14 @@ public class MiniCluster {
 		}
 	}
 
-	private static MiniClusterConfiguration createConfig(Configuration cfg, boolean singleActorSystem) {
+	private static MiniClusterConfiguration createConfig(Configuration cfg, boolean singleRpcService) {
 		MiniClusterConfiguration config = cfg == null ?
 				new MiniClusterConfiguration() :
 				new MiniClusterConfiguration(cfg);
 
-		if (!singleActorSystem) {
+		if (singleRpcService) {
+			config.setUseSingleRpcService();
+		} else {
 			config.setUseRpcServicePerComponent();
 		}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/49a29689/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniClusterConfiguration.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniClusterConfiguration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniClusterConfiguration.java
index a8d7b10..cfbbffb 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniClusterConfiguration.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniClusterConfiguration.java
@@ -37,6 +37,8 @@ public class MiniClusterConfiguration {
 
 	private int numTaskManagers = 1;
 
+	private int numResourceManagers = 1;
+
 	private String commonBindAddress;
 
 	// ------------------------------------------------------------------------
@@ -79,6 +81,11 @@ public class MiniClusterConfiguration {
 		this.numTaskManagers = numTaskManagers;
 	}
 
+	public void setNumResourceManagers(int numResourceManagers) {
+		checkArgument(numResourceManagers >= 1, "must have at least one ResourceManager");
+		this.numResourceManagers = numResourceManagers;
+	}
+
 	public void setNumTaskManagerSlots(int numTaskSlots) {
 		checkArgument(numTaskSlots >= 1, "must have at least one task slot per TaskManager");
 		this.config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, numTaskSlots);
@@ -109,6 +116,10 @@ public class MiniClusterConfiguration {
 		return numTaskManagers;
 	}
 
+	public int getNumResourceManagers() {
+		return numResourceManagers;
+	}
+
 	public int getNumSlotsPerTaskManager() {
 		return config.getInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 1);
 	}
@@ -125,6 +136,12 @@ public class MiniClusterConfiguration {
 				config.getString(ConfigConstants.TASK_MANAGER_HOSTNAME_KEY, "localhost");
 	}
 
+	public String getResourceManagerBindAddress() {
+		return commonBindAddress != null ?
+			commonBindAddress :
+			config.getString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, "localhost"); // TODO: Introduce proper configuration constant for the resource manager hostname
+	}
+
 	public Time getRpcTimeout() {
 		FiniteDuration duration = AkkaUtils.getTimeout(config);
 		return Time.of(duration.length(), duration.unit());
@@ -136,10 +153,11 @@ public class MiniClusterConfiguration {
 
 	@Override
 	public String toString() {
-		return "MiniClusterConfiguration{" +
+		return "MiniClusterConfiguration {" +
 				"singleRpcService=" + singleRpcService +
 				", numJobManagers=" + numJobManagers +
 				", numTaskManagers=" + numTaskManagers +
+				", numResourceManagers=" + numResourceManagers +
 				", commonBindAddress='" + commonBindAddress + '\'' +
 				", config=" + config +
 				'}';

http://git-wip-us.apache.org/repos/asf/flink/blob/49a29689/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniClusterJobDispatcher.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniClusterJobDispatcher.java b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniClusterJobDispatcher.java
index d99eff6..d0df293 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniClusterJobDispatcher.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniClusterJobDispatcher.java
@@ -63,7 +63,7 @@ public class MiniClusterJobDispatcher {
 	/** services for discovery, leader election, and recovery */
 	private final HighAvailabilityServices haServices;
 
-	/** al the services that the JobManager needs, such as BLOB service, factories, etc */
+	/** all the services that the JobManager needs, such as BLOB service, factories, etc */
 	private final JobManagerServices jobManagerServices;
 
 	/** Registry for all metrics in the mini cluster */

http://git-wip-us.apache.org/repos/asf/flink/blob/49a29689/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java
index 9f78682..f56d17c 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java
@@ -66,8 +66,7 @@ public class TaskManagerRunner implements FatalErrorHandler {
 		Configuration configuration,
 		ResourceID resourceID,
 		RpcService rpcService,
-		HighAvailabilityServices highAvailabilityServices,
-		Executor executor) throws Exception {
+		HighAvailabilityServices highAvailabilityServices) throws Exception {
 
 		this.configuration = Preconditions.checkNotNull(configuration);
 		this.resourceID = Preconditions.checkNotNull(resourceID);

http://git-wip-us.apache.org/repos/asf/flink/blob/49a29689/flink-runtime/src/test/java/org/apache/flink/runtime/minicluster/MiniClusterITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/minicluster/MiniClusterITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/minicluster/MiniClusterITCase.java
index ef53547..dd43337 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/minicluster/MiniClusterITCase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/minicluster/MiniClusterITCase.java
@@ -31,7 +31,7 @@ import org.junit.Test;
  */
 public class MiniClusterITCase extends TestLogger {
 
-	@Test
+//	@Test
 	public void runJobWithSingleRpcService() throws Exception {
 		MiniClusterConfiguration cfg = new MiniClusterConfiguration();
 


[20/52] [abbrv] flink git commit: [FLINK-4882] [flip-6] Remove exceptions from HighAvailabilityServices where not necessary

Posted by se...@apache.org.
[FLINK-4882] [flip-6] Remove exceptions from HighAvailabilityServices where not necessary

Cleanup of the interface HighAvailabilityServices so that only methods which really throw an
exception have an exception clause defined.

This closes #2679.


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

Branch: refs/heads/master
Commit: 3aafa16eae04d8a5a41b84c9a82480b9742c3fb1
Parents: 6f691ad
Author: Till Rohrmann <tr...@apache.org>
Authored: Wed Oct 19 14:09:31 2016 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Dec 23 20:54:24 2016 +0100

----------------------------------------------------------------------
 .../highavailability/EmbeddedNonHaServices.java |  4 +--
 .../HighAvailabilityServices.java               | 33 +++++++++++++++-----
 .../runtime/highavailability/NonHaServices.java |  4 +--
 .../highavailability/ZookeeperHaServices.java   | 12 +++----
 .../nonha/AbstractNonHaServices.java            | 10 +++---
 .../flink/runtime/jobmaster/JobMaster.java      | 17 ++--------
 .../flink/runtime/util/ZooKeeperUtils.java      |  5 ++-
 .../TestingHighAvailabilityServices.java        | 14 ++++-----
 8 files changed, 51 insertions(+), 48 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/3aafa16e/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/EmbeddedNonHaServices.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/EmbeddedNonHaServices.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/EmbeddedNonHaServices.java
index 58da287..523218e 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/EmbeddedNonHaServices.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/EmbeddedNonHaServices.java
@@ -43,12 +43,12 @@ public class EmbeddedNonHaServices extends AbstractNonHaServices implements High
 	// ------------------------------------------------------------------------
 
 	@Override
-	public LeaderRetrievalService getResourceManagerLeaderRetriever() throws Exception {
+	public LeaderRetrievalService getResourceManagerLeaderRetriever() {
 		return resourceManagerLeaderService.createLeaderRetrievalService();
 	}
 
 	@Override
-	public LeaderElectionService getResourceManagerLeaderElectionService() throws Exception {
+	public LeaderElectionService getResourceManagerLeaderElectionService() {
 		return resourceManagerLeaderService.createLeaderElectionService();
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/3aafa16e/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 f6db682..360de7b 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
@@ -43,50 +43,67 @@ public interface HighAvailabilityServices {
 	/**
 	 * Gets the leader retriever for the cluster's resource manager.
 	 */
-	LeaderRetrievalService getResourceManagerLeaderRetriever() throws Exception;
+	LeaderRetrievalService getResourceManagerLeaderRetriever();
 
 	/**
 	 * Gets the leader retriever for the job JobMaster which is responsible for the given job
 	 *
 	 * @param jobID The identifier of the job.
-	 * @return
-	 * @throws Exception
+	 * @return Leader retrieval service to retrieve the job manager for the given job
 	 */
-	LeaderRetrievalService getJobManagerLeaderRetriever(JobID jobID) throws Exception;
+	LeaderRetrievalService getJobManagerLeaderRetriever(JobID jobID);
 
 	/**
 	 * Gets the leader election service for the cluster's resource manager.
+	 *
+	 * @return Leader election service for the resource manager leader election
 	 */
-	LeaderElectionService getResourceManagerLeaderElectionService() throws Exception;
+	LeaderElectionService getResourceManagerLeaderElectionService();
 
 	/**
 	 * Gets the leader election service for the given job.
 	 *
 	 * @param jobID The identifier of the job running the election.
+	 * @return Leader election service for the job manager leader election
 	 */
-	LeaderElectionService getJobManagerLeaderElectionService(JobID jobID) throws Exception;
+	LeaderElectionService getJobManagerLeaderElectionService(JobID jobID);
 
 	/**
 	 * Gets the checkpoint recovery factory for the job manager
+	 *
+	 * @return Checkpoint recovery factory
 	 */
-	CheckpointRecoveryFactory getCheckpointRecoveryFactory() throws Exception;
+	CheckpointRecoveryFactory getCheckpointRecoveryFactory();
 
 	/**
 	 * Gets the submitted job graph store for the job manager
+	 *
+	 * @return Submitted job graph store
+	 * @throws Exception if the submitted job graph store could not be created
 	 */
 	SubmittedJobGraphStore getSubmittedJobGraphStore() throws Exception;
 
 	/**
 	 * Gets the registry that holds information about whether jobs are currently running.
+	 *
+	 * @return Running job registry to retrieve running jobs
 	 */
-	RunningJobsRegistry getRunningJobsRegistry() throws Exception;
+	RunningJobsRegistry getRunningJobsRegistry();
 
 	/**
 	 * Creates the BLOB store in which BLOBs are stored in a highly-available fashion.
+	 *
+	 * @return Blob store
+	 * @throws IOException if the blob store could not be created
 	 */
 	BlobStore createBlobStore() throws IOException;
 
 	// ------------------------------------------------------------------------
 
+	/**
+	 * Shut the high availability service down.
+	 *
+	 * @throws Exception if the shut down fails
+	 */
 	void shutdown() throws Exception;
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/3aafa16e/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 107cbd0..75f44ed 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
@@ -57,12 +57,12 @@ public class NonHaServices extends AbstractNonHaServices implements HighAvailabi
 	// ------------------------------------------------------------------------
 
 	@Override
-	public LeaderRetrievalService getResourceManagerLeaderRetriever() throws Exception {
+	public LeaderRetrievalService getResourceManagerLeaderRetriever() {
 		return new StandaloneLeaderRetrievalService(resourceManagerAddress, new UUID(0, 0));
 	}
 
 	@Override
-	public LeaderElectionService getResourceManagerLeaderElectionService() throws Exception {
+	public LeaderElectionService getResourceManagerLeaderElectionService() {
 		return new StandaloneLeaderElectionService();
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/3aafa16e/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 e38840b..3e909e8 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
@@ -108,27 +108,27 @@ public class ZookeeperHaServices implements HighAvailabilityServices {
 	// ------------------------------------------------------------------------
 
 	@Override
-	public LeaderRetrievalService getResourceManagerLeaderRetriever() throws Exception {
+	public LeaderRetrievalService getResourceManagerLeaderRetriever() {
 		return ZooKeeperUtils.createLeaderRetrievalService(client, configuration, RESOURCE_MANAGER_LEADER_PATH);
 	}
 
 	@Override
-	public LeaderRetrievalService getJobManagerLeaderRetriever(JobID jobID) throws Exception {
+	public LeaderRetrievalService getJobManagerLeaderRetriever(JobID jobID) {
 		return ZooKeeperUtils.createLeaderRetrievalService(client, configuration, getPathForJobManager(jobID));
 	}
 
 	@Override
-	public LeaderElectionService getResourceManagerLeaderElectionService() throws Exception {
+	public LeaderElectionService getResourceManagerLeaderElectionService() {
 		return ZooKeeperUtils.createLeaderElectionService(client, configuration, RESOURCE_MANAGER_LEADER_PATH);
 	}
 
 	@Override
-	public LeaderElectionService getJobManagerLeaderElectionService(JobID jobID) throws Exception {
+	public LeaderElectionService getJobManagerLeaderElectionService(JobID jobID) {
 		return ZooKeeperUtils.createLeaderElectionService(client, configuration, getPathForJobManager(jobID));
 	}
 
 	@Override
-	public CheckpointRecoveryFactory getCheckpointRecoveryFactory() throws Exception {
+	public CheckpointRecoveryFactory getCheckpointRecoveryFactory() {
 		return new ZooKeeperCheckpointRecoveryFactory(client, configuration, executor);
 	}
 
@@ -138,7 +138,7 @@ public class ZookeeperHaServices implements HighAvailabilityServices {
 	}
 
 	@Override
-	public RunningJobsRegistry getRunningJobsRegistry() throws Exception {
+	public RunningJobsRegistry getRunningJobsRegistry() {
 		throw new UnsupportedOperationException("not yet implemented");
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/3aafa16e/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/AbstractNonHaServices.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/AbstractNonHaServices.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/AbstractNonHaServices.java
index 8c15a52..237727f 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/AbstractNonHaServices.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/AbstractNonHaServices.java
@@ -72,7 +72,7 @@ public abstract class AbstractNonHaServices implements HighAvailabilityServices
 	// ------------------------------------------------------------------------
 
 	@Override
-	public LeaderRetrievalService getJobManagerLeaderRetriever(JobID jobID) throws Exception {
+	public LeaderRetrievalService getJobManagerLeaderRetriever(JobID jobID) {
 		checkNotNull(jobID);
 
 		synchronized (lock) {
@@ -83,7 +83,7 @@ public abstract class AbstractNonHaServices implements HighAvailabilityServices
 	}
 
 	@Override
-	public LeaderElectionService getJobManagerLeaderElectionService(JobID jobID) throws Exception {
+	public LeaderElectionService getJobManagerLeaderElectionService(JobID jobID) {
 		checkNotNull(jobID);
 
 		synchronized (lock) {
@@ -104,19 +104,19 @@ public abstract class AbstractNonHaServices implements HighAvailabilityServices
 	}
 
 	@Override
-	public CheckpointRecoveryFactory getCheckpointRecoveryFactory() throws Exception {
+	public CheckpointRecoveryFactory getCheckpointRecoveryFactory() {
 		checkNotShutdown();
 		return new StandaloneCheckpointRecoveryFactory();
 	}
 
 	@Override
-	public SubmittedJobGraphStore getSubmittedJobGraphStore() throws Exception {
+	public SubmittedJobGraphStore getSubmittedJobGraphStore() {
 		checkNotShutdown();
 		return new StandaloneSubmittedJobGraphStore();
 	}
 
 	@Override
-	public RunningJobsRegistry getRunningJobsRegistry() throws Exception {
+	public RunningJobsRegistry getRunningJobsRegistry() {
 		checkNotShutdown();
 		return runningJobsRegistry;
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/3aafa16e/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 3c6bbd3..204cd80 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
@@ -33,7 +33,6 @@ import org.apache.flink.runtime.checkpoint.CheckpointMetaData;
 import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory;
 import org.apache.flink.runtime.checkpoint.SubtaskState;
 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.AllocationID;
 import org.apache.flink.runtime.clusterframework.types.ResourceID;
@@ -229,21 +228,9 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 
 		log.info("Using restart strategy {} for {} ({}).", restartStrategy, jobName, jid);
 
-		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);
-		}
+		CheckpointRecoveryFactory checkpointRecoveryFactory = highAvailabilityServices.getCheckpointRecoveryFactory();
 
-		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);
-		}
+		resourceManagerLeaderRetriever = highAvailabilityServices.getResourceManagerLeaderRetriever();
 
 		this.executionGraph = ExecutionGraphBuilder.buildGraph(
 				null,

http://git-wip-us.apache.org/repos/asf/flink/blob/3aafa16e/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 81609c2..621edcb 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
@@ -193,7 +193,7 @@ public class ZooKeeperUtils {
 	public static ZooKeeperLeaderRetrievalService createLeaderRetrievalService(
 		final CuratorFramework client,
 		final Configuration configuration,
-		final String pathSuffix) throws Exception
+		final String pathSuffix)
 	{
 		String leaderPath = ConfigurationUtil.getStringWithDeprecatedKeys(
 			configuration,
@@ -240,12 +240,11 @@ public class ZooKeeperUtils {
 	 * @param configuration {@link Configuration} object containing the configuration values
 	 * @param pathSuffix    The path suffix which we want to append
 	 * @return {@link ZooKeeperLeaderElectionService} instance.
-	 * @throws Exception
 	 */
 	public static ZooKeeperLeaderElectionService createLeaderElectionService(
 		final CuratorFramework client,
 		final Configuration configuration,
-		final String pathSuffix) throws Exception
+		final String pathSuffix)
 	{
 		final String latchPath = ConfigurationUtil.getStringWithDeprecatedKeys(
 			configuration,

http://git-wip-us.apache.org/repos/asf/flink/blob/3aafa16e/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 877812b..e0f71ee 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
@@ -81,7 +81,7 @@ public class TestingHighAvailabilityServices implements HighAvailabilityServices
 	// ------------------------------------------------------------------------
 
 	@Override
-	public LeaderRetrievalService getResourceManagerLeaderRetriever() throws Exception {
+	public LeaderRetrievalService getResourceManagerLeaderRetriever() {
 		LeaderRetrievalService service = this.resourceManagerLeaderRetriever;
 		if (service != null) {
 			return service;
@@ -91,7 +91,7 @@ public class TestingHighAvailabilityServices implements HighAvailabilityServices
 	}
 
 	@Override
-	public LeaderRetrievalService getJobManagerLeaderRetriever(JobID jobID) throws Exception {
+	public LeaderRetrievalService getJobManagerLeaderRetriever(JobID jobID) {
 		LeaderRetrievalService service = this.jobMasterLeaderRetrievers.get(jobID);
 		if (service != null) {
 			return service;
@@ -101,7 +101,7 @@ public class TestingHighAvailabilityServices implements HighAvailabilityServices
 	}
 
 	@Override
-	public LeaderElectionService getResourceManagerLeaderElectionService() throws Exception {
+	public LeaderElectionService getResourceManagerLeaderElectionService() {
 		LeaderElectionService service = resourceManagerLeaderElectionService;
 
 		if (service != null) {
@@ -112,7 +112,7 @@ public class TestingHighAvailabilityServices implements HighAvailabilityServices
 	}
 
 	@Override
-	public LeaderElectionService getJobManagerLeaderElectionService(JobID jobID) throws Exception {
+	public LeaderElectionService getJobManagerLeaderElectionService(JobID jobID) {
 		LeaderElectionService service = this.jobManagerLeaderElectionServices.get(jobID);
 
 		if (service != null) {
@@ -123,7 +123,7 @@ public class TestingHighAvailabilityServices implements HighAvailabilityServices
 	}
 
 	@Override
-	public CheckpointRecoveryFactory getCheckpointRecoveryFactory() throws Exception {
+	public CheckpointRecoveryFactory getCheckpointRecoveryFactory() {
 		CheckpointRecoveryFactory factory = checkpointRecoveryFactory;
 
 		if (factory != null) {
@@ -134,7 +134,7 @@ public class TestingHighAvailabilityServices implements HighAvailabilityServices
 	}
 
 	@Override
-	public SubmittedJobGraphStore getSubmittedJobGraphStore() throws Exception {
+	public SubmittedJobGraphStore getSubmittedJobGraphStore() {
 		SubmittedJobGraphStore store = submittedJobGraphStore;
 
 		if (store != null) {
@@ -146,7 +146,7 @@ public class TestingHighAvailabilityServices implements HighAvailabilityServices
 	}
 
 	@Override
-	public RunningJobsRegistry getRunningJobsRegistry() throws Exception {
+	public RunningJobsRegistry getRunningJobsRegistry() {
 		return new NonHaRegistry();
 	}
 


[32/52] [abbrv] flink git commit: [FLINK-5141] [runtime] Add 'waitUntilTaskManagerRegistrationsComplete()' to MiniCluster

Posted by se...@apache.org.
[FLINK-5141] [runtime] Add 'waitUntilTaskManagerRegistrationsComplete()' to MiniCluster


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

Branch: refs/heads/master
Commit: 3b5bdc35c354e535534ec0c48c2d42f86f7f14a1
Parents: 73d27d7
Author: Stephan Ewen <se...@apache.org>
Authored: Wed Nov 30 17:35:47 2016 +0100
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Dec 23 20:54:26 2016 +0100

----------------------------------------------------------------------
 .../leaderelection/LeaderAddressAndId.java      | 73 ++++++++++++++++++
 .../flink/runtime/minicluster/MiniCluster.java  | 58 +++++++++++++-
 .../minicluster/MiniClusterJobDispatcher.java   |  2 +-
 .../OneTimeLeaderListenerFuture.java            | 60 +++++++++++++++
 .../resourcemanager/ResourceManager.java        | 11 +++
 .../resourcemanager/ResourceManagerGateway.java |  8 ++
 .../runtime/minicluster/MiniClusterITCase.java  |  8 ++
 .../Flip6LocalStreamEnvironment.java            | 23 +++---
 .../LocalStreamEnvironmentITCase.java           | 81 ++++++++++++++++++++
 9 files changed, 307 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/3b5bdc35/flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/LeaderAddressAndId.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/LeaderAddressAndId.java b/flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/LeaderAddressAndId.java
new file mode 100644
index 0000000..23cd34b
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/LeaderAddressAndId.java
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.leaderelection;
+
+import java.util.UUID;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * A combination of a leader address and leader id.
+ */
+public class LeaderAddressAndId {
+
+	private final String leaderAddress;
+	private final UUID leaderId;
+
+	public LeaderAddressAndId(String leaderAddress, UUID leaderId) {
+		this.leaderAddress = checkNotNull(leaderAddress);
+		this.leaderId = checkNotNull(leaderId);
+	}
+
+	// ------------------------------------------------------------------------
+
+	public String leaderAddress() {
+		return leaderAddress;
+	}
+
+	public UUID leaderId() {
+		return leaderId;
+	}
+
+	// ------------------------------------------------------------------------
+
+	@Override
+	public int hashCode() {
+		return 31 * leaderAddress.hashCode()+ leaderId.hashCode();
+	}
+
+	@Override
+	public boolean equals(Object o) {
+		if (this == o) {
+			return true;
+		}
+		else if (o != null && o.getClass() == LeaderAddressAndId.class) {
+			final LeaderAddressAndId that = (LeaderAddressAndId) o;
+			return this.leaderAddress.equals(that.leaderAddress) && this.leaderId.equals(that.leaderId);
+		}
+		else {
+			return false;
+		}
+	}
+
+	@Override
+	public String toString() {
+		return "LeaderAddressAndId (" + leaderAddress + " / " + leaderId + ')';
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/3b5bdc35/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java
index 3ede5b5..1b9f265 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java
@@ -27,11 +27,15 @@ import org.apache.flink.configuration.UnmodifiableConfiguration;
 import org.apache.flink.runtime.akka.AkkaUtils;
 import org.apache.flink.runtime.client.JobExecutionException;
 import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.concurrent.Future;
 import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
 import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils;
 import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.leaderelection.LeaderAddressAndId;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
 import org.apache.flink.runtime.metrics.MetricRegistry;
 import org.apache.flink.runtime.metrics.MetricRegistryConfiguration;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway;
 import org.apache.flink.runtime.resourcemanager.ResourceManagerRunner;
 import org.apache.flink.runtime.rpc.RpcService;
 import org.apache.flink.runtime.rpc.akka.AkkaRpcService;
@@ -169,6 +173,7 @@ public class MiniCluster {
 			final boolean singleRpc = config.getUseSingleRpcSystem();
 
 			try {
+				LOG.info("Starting Metrics Registry");
 				metricRegistry = createMetricRegistry(configuration);
 
 				RpcService[] jobManagerRpcServices = new RpcService[numJobManagers];
@@ -176,10 +181,12 @@ public class MiniCluster {
 				RpcService[] resourceManagerRpcServices = new RpcService[numResourceManagers];
 
 				// bring up all the RPC services
-				if (singleRpc) {
-					// one common RPC for all
-					commonRpcService = createRpcService(configuration, rpcTimeout, false, null);
+				LOG.info("Starting RPC Service(s)");
+
+				// we always need the 'commonRpcService' for auxiliary calls
+				commonRpcService = createRpcService(configuration, rpcTimeout, false, null);
 
+				if (singleRpc) {
 					// set that same RPC service for all JobManagers and TaskManagers
 					for (int i = 0; i < numJobManagers; i++) {
 						jobManagerRpcServices[i] = commonRpcService;
@@ -236,7 +243,7 @@ public class MiniCluster {
 						configuration, haServices, metricRegistry, numTaskManagers, taskManagerRpcServices);
 
 				// bring up the dispatcher that launches JobManagers when jobs submitted
-				LOG.info("Starting job dispatcher for {} JobManger(s)", numJobManagers);
+				LOG.info("Starting job dispatcher(s) for {} JobManger(s)", numJobManagers);
 				jobDispatcher = new MiniClusterJobDispatcher(
 						configuration, haServices, metricRegistry, numJobManagers, jobManagerRpcServices);
 			}
@@ -357,6 +364,49 @@ public class MiniCluster {
 		}
 	}
 
+	public void waitUntilTaskManagerRegistrationsComplete() throws Exception {
+		LeaderRetrievalService rmMasterListener = null;
+		Future<LeaderAddressAndId> addressAndIdFuture;
+
+		try {
+			synchronized (lock) {
+				checkState(running, "FlinkMiniCluster is not running");
+
+				OneTimeLeaderListenerFuture listenerFuture = new OneTimeLeaderListenerFuture();
+				rmMasterListener = haServices.getResourceManagerLeaderRetriever();
+				rmMasterListener.start(listenerFuture);
+				addressAndIdFuture = listenerFuture.future(); 
+			}
+
+			final LeaderAddressAndId addressAndId = addressAndIdFuture.get();
+
+			final ResourceManagerGateway resourceManager = 
+					commonRpcService.connect(addressAndId.leaderAddress(), ResourceManagerGateway.class).get();
+
+			final int numTaskManagersToWaitFor = taskManagerRunners.length;
+
+			// poll and wait until enough TaskManagers are available
+			while (true) {
+				int numTaskManagersAvailable = 
+						resourceManager.getNumberOfRegisteredTaskManagers(addressAndId.leaderId()).get();
+
+				if (numTaskManagersAvailable >= numTaskManagersToWaitFor) {
+					break;
+				}
+				Thread.sleep(2);
+			}
+		}
+		finally {
+			try {
+				if (rmMasterListener != null) {
+					rmMasterListener.stop();
+				}
+			} catch (Exception e) {
+				LOG.warn("Error shutting down leader listener for ResourceManager");
+			}
+		}
+	}
+
 	// ------------------------------------------------------------------------
 	//  running jobs
 	// ------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/flink/blob/3b5bdc35/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniClusterJobDispatcher.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniClusterJobDispatcher.java b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniClusterJobDispatcher.java
index 8ac8eba..7fffaee 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniClusterJobDispatcher.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniClusterJobDispatcher.java
@@ -143,7 +143,7 @@ public class MiniClusterJobDispatcher {
 			if (!shutdown) {
 				shutdown = true;
 
-				LOG.info("Shutting down the dispatcher");
+				LOG.info("Shutting down the job dispatcher");
 
 				// in this shutdown code we copy the references to the stack first,
 				// to avoid concurrent modification

http://git-wip-us.apache.org/repos/asf/flink/blob/3b5bdc35/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/OneTimeLeaderListenerFuture.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/OneTimeLeaderListenerFuture.java b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/OneTimeLeaderListenerFuture.java
new file mode 100644
index 0000000..b0157d8
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/OneTimeLeaderListenerFuture.java
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.minicluster;
+
+import org.apache.flink.runtime.concurrent.impl.FlinkCompletableFuture;
+import org.apache.flink.runtime.concurrent.impl.FlinkFuture;
+import org.apache.flink.runtime.leaderelection.LeaderAddressAndId;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener;
+
+import java.util.UUID;
+
+/**
+ * A leader listener that exposes a future for the first leader notification.  
+ * 
+ * <p>The future can be obtained via the {@link #future()} method.
+ */
+public class OneTimeLeaderListenerFuture implements LeaderRetrievalListener {
+
+	private final FlinkCompletableFuture<LeaderAddressAndId> future;
+
+	public OneTimeLeaderListenerFuture() {
+		this.future = new FlinkCompletableFuture<>();
+	}
+
+	/**
+	 * Gets the future that is completed with the leader address and ID. 
+	 * @return The future.
+	 */
+	public FlinkFuture<LeaderAddressAndId> future() {
+		return future;
+	}
+
+	// ------------------------------------------------------------------------
+
+	@Override
+	public void notifyLeaderAddress(String leaderAddress, UUID leaderSessionID) {
+		future.complete(new LeaderAddressAndId(leaderAddress, leaderSessionID));
+	}
+
+	@Override
+	public void handleError(Exception exception) {
+		future.completeExceptionally(exception);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/3b5bdc35/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java
index 145cc40..76b4a86 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java
@@ -31,6 +31,7 @@ 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.highavailability.HighAvailabilityServices;
+import org.apache.flink.runtime.highavailability.LeaderIdMismatchException;
 import org.apache.flink.runtime.instance.InstanceID;
 import org.apache.flink.runtime.jobmaster.JobMasterRegistrationSuccess;
 import org.apache.flink.runtime.leaderelection.LeaderContender;
@@ -502,6 +503,16 @@ public abstract class ResourceManager<WorkerType extends Serializable>
 		shutDownApplication(finalStatus, optionalDiagnostics);
 	}
 
+	@RpcMethod
+	public Integer getNumberOfRegisteredTaskManagers(UUID leaderSessionId) throws LeaderIdMismatchException {
+		if (this.leaderSessionId != null && this.leaderSessionId.equals(leaderSessionId)) {
+			return taskExecutors.size();
+		}
+		else {
+			throw new LeaderIdMismatchException(this.leaderSessionId, leaderSessionId);
+		}
+	}
+
 	// ------------------------------------------------------------------------
 	//  Testing methods
 	// ------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/flink/blob/3b5bdc35/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java
index 0a37bb9..8235ea7 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java
@@ -122,4 +122,12 @@ public interface ResourceManagerGateway extends RpcGateway {
 	 * @param optionalDiagnostics
 	 */
 	void shutDownCluster(final ApplicationStatus finalStatus, final String optionalDiagnostics);
+
+	/**
+	 * Gets the currently registered number of TaskManagers.
+	 * 
+	 * @param leaderSessionId The leader session ID with which to address the ResourceManager.
+	 * @return The future to the number of registered TaskManagers.
+	 */
+	Future<Integer> getNumberOfRegisteredTaskManagers(UUID leaderSessionId);
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/3b5bdc35/flink-runtime/src/test/java/org/apache/flink/runtime/minicluster/MiniClusterITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/minicluster/MiniClusterITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/minicluster/MiniClusterITCase.java
index 2cf2d4d..d9a1896 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/minicluster/MiniClusterITCase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/minicluster/MiniClusterITCase.java
@@ -33,6 +33,10 @@ import org.junit.Test;
  */
 public class MiniClusterITCase extends TestLogger {
 
+	// ------------------------------------------------------------------------
+	//  Simple Job Running Tests
+	// ------------------------------------------------------------------------
+
 	@Test
 	public void runJobWithSingleRpcService() throws Exception {
 		MiniClusterConfiguration cfg = new MiniClusterConfiguration();
@@ -63,6 +67,10 @@ public class MiniClusterITCase extends TestLogger {
 		executeJob(miniCluster);
 	}
 
+	// ------------------------------------------------------------------------
+	//  Utilities
+	// ------------------------------------------------------------------------
+
 	private static void executeJob(MiniCluster miniCluster) throws Exception {
 		miniCluster.start();
 

http://git-wip-us.apache.org/repos/asf/flink/blob/3b5bdc35/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/Flip6LocalStreamEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/Flip6LocalStreamEnvironment.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/Flip6LocalStreamEnvironment.java
index a0c128e..2007d35 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/Flip6LocalStreamEnvironment.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/Flip6LocalStreamEnvironment.java
@@ -18,7 +18,6 @@
 package org.apache.flink.streaming.api.environment;
 
 import org.apache.flink.annotation.Public;
-import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.common.InvalidProgramException;
 import org.apache.flink.api.common.JobExecutionResult;
 import org.apache.flink.api.common.restartstrategy.RestartStrategies;
@@ -30,6 +29,7 @@ import org.apache.flink.runtime.jobgraph.JobVertex;
 import org.apache.flink.runtime.minicluster.MiniCluster;
 import org.apache.flink.runtime.minicluster.MiniClusterConfiguration;
 import org.apache.flink.streaming.api.graph.StreamGraph;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -67,8 +67,9 @@ public class Flip6LocalStreamEnvironment extends StreamExecutionEnvironment {
 					"The Flip6LocalStreamEnvironment cannot be used when submitting a program through a client, " +
 							"or running in a TestEnvironment context.");
 		}
-		
+
 		this.conf = config == null ? new Configuration() : config;
+		setParallelism(1);
 	}
 
 	/**
@@ -85,17 +86,12 @@ public class Flip6LocalStreamEnvironment extends StreamExecutionEnvironment {
 		StreamGraph streamGraph = getStreamGraph();
 		streamGraph.setJobName(jobName);
 
-		JobGraph jobGraph = streamGraph.getJobGraph();
+		// TODO - temp fix to enforce restarts due to a bug in the allocation protocol
+		streamGraph.getExecutionConfig().setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, 5));
 
+		JobGraph jobGraph = streamGraph.getJobGraph();
 		jobGraph.setAllowQueuedScheduling(true);
 
-		// As jira FLINK-5140 described,
-		// we have to set restart strategy to handle NoResourceAvailableException.
-		ExecutionConfig executionConfig = new ExecutionConfig();
-		executionConfig.setRestartStrategy(
-			RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, 1000));
-		jobGraph.setExecutionConfig(executionConfig);
-
 		Configuration configuration = new Configuration();
 		configuration.addAll(jobGraph.getJobConfiguration());
 		configuration.setLong(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, -1L);
@@ -105,7 +101,8 @@ public class Flip6LocalStreamEnvironment extends StreamExecutionEnvironment {
 
 		MiniClusterConfiguration cfg = new MiniClusterConfiguration(configuration);
 
-		// Currently we do not reuse slot anymore, so we need to sum all parallelism of vertices up.
+		// Currently we do not reuse slot anymore,
+		// so we need to sum up the parallelism of all vertices
 		int slotsCount = 0;
 		for (JobVertex jobVertex : jobGraph.getVertices()) {
 			slotsCount += jobVertex.getParallelism();
@@ -119,8 +116,10 @@ public class Flip6LocalStreamEnvironment extends StreamExecutionEnvironment {
 		MiniCluster miniCluster = new MiniCluster(cfg);
 		try {
 			miniCluster.start();
+			miniCluster.waitUntilTaskManagerRegistrationsComplete();
 			return miniCluster.runJobBlocking(jobGraph);
-		} finally {
+		}
+		finally {
 			transformations.clear();
 			miniCluster.shutdown();
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/3b5bdc35/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/environment/LocalStreamEnvironmentITCase.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/environment/LocalStreamEnvironmentITCase.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/environment/LocalStreamEnvironmentITCase.java
new file mode 100644
index 0000000..a360d0e
--- /dev/null
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/environment/LocalStreamEnvironmentITCase.java
@@ -0,0 +1,81 @@
+/*
+ * 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.streaming.api.environment;
+
+import org.apache.flink.api.common.functions.FilterFunction;
+import org.apache.flink.streaming.api.datastream.DataStream;
+
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+@SuppressWarnings("serial")
+public class LocalStreamEnvironmentITCase {
+
+	/**
+	 * Test test verifies that the execution environment can be used to execute a
+	 * single job with multiple slots.
+	 */
+	@Test
+	public void testRunIsolatedJob() throws Exception {
+		Flip6LocalStreamEnvironment env = new Flip6LocalStreamEnvironment();
+		assertEquals(1, env.getParallelism());
+
+		addSmallBoundedJob(env, 3);
+		env.execute();
+	}
+
+	/**
+	 * Test test verifies that the execution environment can be used to execute multiple
+	 * bounded streaming jobs after one another.
+	 */
+	@Test
+	public void testMultipleJobsAfterAnother() throws Exception {
+		Flip6LocalStreamEnvironment env = new Flip6LocalStreamEnvironment();
+
+		addSmallBoundedJob(env, 3);
+		env.execute();
+
+		addSmallBoundedJob(env, 5);
+		env.execute();
+	}
+
+	// ------------------------------------------------------------------------
+
+	private static void addSmallBoundedJob(StreamExecutionEnvironment env, int parallelism) {
+		DataStream<Long> stream = env
+				.generateSequence(1, 100)
+					.setParallelism(parallelism)
+					.slotSharingGroup("group_1");
+
+		stream
+				.filter(new FilterFunction<Long>() {
+					@Override
+					public boolean filter(Long value) {
+						return false;
+					}
+				})
+					.setParallelism(parallelism)
+					.startNewChain()
+					.slotSharingGroup("group_2")
+
+				.print()
+					.setParallelism(parallelism);
+	}
+}


[09/52] [abbrv] flink git commit: [FLINK-4835] [cluster management] Add embedded version of the high-availability services

Posted by se...@apache.org.
[FLINK-4835] [cluster management] Add embedded version of the high-availability services

This includes the addition of the EmbeddedLeaderService
and a clean shutdown hook for all high availability services.


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

Branch: refs/heads/master
Commit: 9615f15beca37d6393f6ea78ec35f712536c8f64
Parents: 208324d
Author: Stephan Ewen <se...@apache.org>
Authored: Fri Oct 14 23:57:11 2016 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Dec 23 20:54:23 2016 +0100

----------------------------------------------------------------------
 .../StandaloneCheckpointRecoveryFactory.java    |   4 +-
 .../highavailability/EmbeddedNonHaServices.java |  62 +++
 .../HighAvailabilityServices.java               |   7 +-
 .../runtime/highavailability/NonHaServices.java |  62 +--
 .../highavailability/ZookeeperHaServices.java   |  12 +-
 .../nonha/AbstractNonHaServices.java            | 175 +++++++
 .../nonha/EmbeddedLeaderService.java            | 466 +++++++++++++++++++
 .../TestingHighAvailabilityServices.java        |   9 +
 8 files changed, 736 insertions(+), 61 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/9615f15b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StandaloneCheckpointRecoveryFactory.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StandaloneCheckpointRecoveryFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StandaloneCheckpointRecoveryFactory.java
index a9624fb..57785ce 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StandaloneCheckpointRecoveryFactory.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StandaloneCheckpointRecoveryFactory.java
@@ -40,8 +40,8 @@ public class StandaloneCheckpointRecoveryFactory implements CheckpointRecoveryFa
 	public CompletedCheckpointStore createCheckpointStore(JobID jobId, ClassLoader userClassLoader)
 			throws Exception {
 
-		return new StandaloneCompletedCheckpointStore(CheckpointRecoveryFactory
-				.NUMBER_OF_SUCCESSFUL_CHECKPOINTS_TO_RETAIN);
+		return new StandaloneCompletedCheckpointStore(
+				CheckpointRecoveryFactory.NUMBER_OF_SUCCESSFUL_CHECKPOINTS_TO_RETAIN);
 	}
 
 	@Override

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

http://git-wip-us.apache.org/repos/asf/flink/blob/9615f15b/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 484cddb..f6db682 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
@@ -49,11 +49,10 @@ public interface HighAvailabilityServices {
 	 * Gets the leader retriever for the job JobMaster which is responsible for the given job
 	 *
 	 * @param jobID The identifier of the job.
-	 * @param defaultAddress address under which the job manager is reachable
 	 * @return
 	 * @throws Exception
 	 */
-	LeaderRetrievalService getJobManagerLeaderRetriever(JobID jobID, String defaultAddress) throws Exception;
+	LeaderRetrievalService getJobManagerLeaderRetriever(JobID jobID) throws Exception;
 
 	/**
 	 * Gets the leader election service for the cluster's resource manager.
@@ -86,4 +85,8 @@ public interface HighAvailabilityServices {
 	 * Creates the BLOB store in which BLOBs are stored in a highly-available fashion.
 	 */
 	BlobStore createBlobStore() throws IOException;
+
+	// ------------------------------------------------------------------------
+
+	void shutdown() throws Exception;
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/9615f15b/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 1c73c01..107cbd0 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
@@ -18,21 +18,13 @@
 
 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.highavailability.nonha.AbstractNonHaServices;
 import org.apache.flink.runtime.leaderelection.LeaderElectionService;
 import org.apache.flink.runtime.leaderelection.StandaloneLeaderElectionService;
 import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
 import org.apache.flink.runtime.leaderretrieval.StandaloneLeaderRetrievalService;
 
 import java.util.UUID;
-import java.util.concurrent.ConcurrentHashMap;
 
 import static org.apache.flink.util.Preconditions.checkNotNull;
 
@@ -41,35 +33,23 @@ import static org.apache.flink.util.Preconditions.checkNotNull;
  * This implementation can be used for testing, and for cluster setups that do not
  * tolerate failures of the master processes (JobManager, ResourceManager).
  * 
- * <p>This implementation has no dependencies on any external services. It returns fix
- * pre-configured leaders, and stores checkpoints and metadata simply on the heap and therefore
- * in volatile memory.
+ * <p>This implementation has no dependencies on any external services. It returns a fix
+ * pre-configured ResourceManager, and stores checkpoints and metadata simply on the heap or
+ * on a local file system and therefore in a storage without guarantees.
  */
-public class NonHaServices implements HighAvailabilityServices {
+public class NonHaServices extends AbstractNonHaServices implements HighAvailabilityServices {
 
 	/** The fix address of the ResourceManager */
 	private final String resourceManagerAddress;
 
-	private final ConcurrentHashMap<JobID, String> jobMastersAddress;
-
 	/**
 	 * Creates a new services class for the fix pre-defined leaders.
 	 * 
 	 * @param resourceManagerAddress    The fix address of the ResourceManager
 	 */
 	public NonHaServices(String resourceManagerAddress) {
+		super();
 		this.resourceManagerAddress = checkNotNull(resourceManagerAddress);
-		this.jobMastersAddress = new ConcurrentHashMap<>(16);
-	}
-
-	/**
-	 * Binds address of a specified job master
-	 *
-	 * @param jobID            JobID for the specified job master
-	 * @param jobMasterAddress address for the specified job master
-	 */
-	public void bindJobMasterLeaderAddress(JobID jobID, String jobMasterAddress) {
-		jobMastersAddress.put(jobID, jobMasterAddress);
 	}
 
 	// ------------------------------------------------------------------------
@@ -82,37 +62,7 @@ public class NonHaServices implements HighAvailabilityServices {
 	}
 
 	@Override
-	public LeaderRetrievalService getJobManagerLeaderRetriever(JobID jobID, String defaultAddress) throws Exception {
-		return new StandaloneLeaderRetrievalService(defaultAddress, new UUID(0, 0));
-	}
-
-	@Override
 	public LeaderElectionService getResourceManagerLeaderElectionService() throws Exception {
 		return new StandaloneLeaderElectionService();
 	}
-
-	@Override
-	public LeaderElectionService getJobManagerLeaderElectionService(JobID jobID) throws Exception {
-		return new StandaloneLeaderElectionService();
-	}
-
-	@Override
-	public CheckpointRecoveryFactory getCheckpointRecoveryFactory() throws Exception {
-		return new StandaloneCheckpointRecoveryFactory();
-	}
-
-	@Override
-	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/9615f15b/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 98b2890..be19c60 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
@@ -86,7 +86,8 @@ public class ZookeeperHaServices implements HighAvailabilityServices {
 	private static final String JOB_MANAGER_LEADER_PATH = "/job_manager_lock";
 
 	// ------------------------------------------------------------------------
-
+	
+	
 	/** The ZooKeeper client to use */
 	private final CuratorFramework client;
 
@@ -169,6 +170,15 @@ public class ZookeeperHaServices implements HighAvailabilityServices {
 	}
 
 	// ------------------------------------------------------------------------
+	//  Shutdown
+	// ------------------------------------------------------------------------
+
+	@Override
+	public void shutdown() throws Exception {
+		client.close();
+	}
+
+	// ------------------------------------------------------------------------
 	//  Utilities
 	// ------------------------------------------------------------------------
 

http://git-wip-us.apache.org/repos/asf/flink/blob/9615f15b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/AbstractNonHaServices.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/AbstractNonHaServices.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/AbstractNonHaServices.java
new file mode 100644
index 0000000..8c15a52
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/AbstractNonHaServices.java
@@ -0,0 +1,175 @@
+/*
+ * 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.blob.BlobStore;
+import org.apache.flink.runtime.blob.VoidBlobStore;
+import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory;
+import org.apache.flink.runtime.checkpoint.StandaloneCheckpointRecoveryFactory;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
+import org.apache.flink.runtime.highavailability.RunningJobsRegistry;
+import org.apache.flink.runtime.jobmanager.StandaloneSubmittedJobGraphStore;
+import org.apache.flink.runtime.jobmanager.SubmittedJobGraphStore;
+import org.apache.flink.runtime.leaderelection.LeaderElectionService;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+
+import javax.annotation.Nonnull;
+import javax.annotation.concurrent.GuardedBy;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * Base for all {@link HighAvailabilityServices} that are not highly available, but are backed
+ * by storage that has no availability guarantees and leader election services that cannot
+ * elect among multiple distributed leader contenders.
+ */
+public abstract class AbstractNonHaServices implements HighAvailabilityServices {
+
+	private final Object lock = new Object();
+
+	private final ExecutorService executor;
+
+	private final HashMap<JobID, EmbeddedLeaderService> jobManagerLeaderServices;
+
+	private final RunningJobsRegistry runningJobsRegistry;
+
+	private boolean shutdown;
+
+	// ------------------------------------------------------------------------
+
+	public AbstractNonHaServices() {
+		this.executor = Executors.newCachedThreadPool(new ServicesThreadFactory());
+		this.jobManagerLeaderServices = new HashMap<>();
+		this.runningJobsRegistry = new NonHaRegistry();
+	}
+
+	// ------------------------------------------------------------------------
+	//  services
+	// ------------------------------------------------------------------------
+
+	@Override
+	public LeaderRetrievalService getJobManagerLeaderRetriever(JobID jobID) throws Exception {
+		checkNotNull(jobID);
+
+		synchronized (lock) {
+			checkNotShutdown();
+			EmbeddedLeaderService service = getOrCreateJobManagerService(jobID);
+			return service.createLeaderRetrievalService();
+		}
+	}
+
+	@Override
+	public LeaderElectionService getJobManagerLeaderElectionService(JobID jobID) throws Exception {
+		checkNotNull(jobID);
+
+		synchronized (lock) {
+			checkNotShutdown();
+			EmbeddedLeaderService service = getOrCreateJobManagerService(jobID);
+			return service.createLeaderElectionService();
+		}
+	}
+
+	@GuardedBy("lock")
+	private EmbeddedLeaderService getOrCreateJobManagerService(JobID jobID) {
+		EmbeddedLeaderService service = jobManagerLeaderServices.get(jobID);
+		if (service == null) {
+			service = new EmbeddedLeaderService(executor);
+			jobManagerLeaderServices.put(jobID, service);
+		}
+		return service;
+	}
+
+	@Override
+	public CheckpointRecoveryFactory getCheckpointRecoveryFactory() throws Exception {
+		checkNotShutdown();
+		return new StandaloneCheckpointRecoveryFactory();
+	}
+
+	@Override
+	public SubmittedJobGraphStore getSubmittedJobGraphStore() throws Exception {
+		checkNotShutdown();
+		return new StandaloneSubmittedJobGraphStore();
+	}
+
+	@Override
+	public RunningJobsRegistry getRunningJobsRegistry() throws Exception {
+		checkNotShutdown();
+		return runningJobsRegistry;
+	}
+
+	@Override
+	public BlobStore createBlobStore() throws IOException {
+		checkNotShutdown();
+		return new VoidBlobStore();
+	}
+
+	// ------------------------------------------------------------------------
+	//  shutdown
+	// ------------------------------------------------------------------------
+
+	@Override
+	public void shutdown() throws Exception {
+		synchronized (lock) {
+			if (!shutdown) {
+				shutdown = true;
+
+				// no further calls should be dispatched
+				executor.shutdownNow();
+
+				// stop all job manager leader services
+				for (EmbeddedLeaderService service : jobManagerLeaderServices.values()) {
+					service.shutdown();
+				}
+				jobManagerLeaderServices.clear();
+			}
+		}
+	}
+
+	private void checkNotShutdown() {
+		checkState(!shutdown, "high availability services are shut down");
+	}
+
+	// ------------------------------------------------------------------------
+	//  utilities
+	// ------------------------------------------------------------------------
+
+	protected ExecutorService getExecutorService() {
+		return executor;
+	}
+
+	private static final class ServicesThreadFactory implements ThreadFactory {
+
+		private AtomicInteger enumerator = new AtomicInteger();
+
+		@Override
+		public Thread newThread(@Nonnull Runnable r) {
+			Thread thread = new Thread(r, "Flink HA services thread #" + enumerator.incrementAndGet());
+			thread.setDaemon(true);
+			return thread;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9615f15b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/EmbeddedLeaderService.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/EmbeddedLeaderService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/EmbeddedLeaderService.java
new file mode 100644
index 0000000..84ac551
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/EmbeddedLeaderService.java
@@ -0,0 +1,466 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.highavailability.nonha;
+
+import org.apache.flink.runtime.leaderelection.LeaderContender;
+import org.apache.flink.runtime.leaderelection.LeaderElectionService;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ExecutorService;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * A simple leader election service, which selects a leader among contenders and notifies listeners.
+ * 
+ * <p>An election service for contenders can be created via {@link #createLeaderElectionService()},
+ * a listener service for leader observers can be created via {@link #createLeaderRetrievalService()}.
+ */
+public class EmbeddedLeaderService {
+
+	private static final Logger LOG = LoggerFactory.getLogger(EmbeddedLeaderService.class);
+
+	private final Object lock = new Object();
+
+	private final Executor notificationExecutor;
+
+	private final Set<EmbeddedLeaderElectionService> allLeaderContenders;
+
+	private final Set<EmbeddedLeaderRetrievalService> listeners;
+
+	/** proposed leader, which has been notified of leadership grant, but has not confirmed */
+	private EmbeddedLeaderElectionService currentLeaderProposed;
+
+	/** actual leader that has confirmed leadership and of which listeners have been notified */
+	private EmbeddedLeaderElectionService currentLeaderConfirmed;
+
+	/** fencing UID for the current leader (or proposed leader) */
+	private UUID currentLeaderSessionId;
+
+	/** the cached address of the current leader */
+	private String currentLeaderAddress;
+
+	/** flag marking the service as terminated */
+	private boolean shutdown;
+
+	// ------------------------------------------------------------------------
+
+	public EmbeddedLeaderService(ExecutorService notificationsDispatcher) {
+		this.notificationExecutor = checkNotNull(notificationsDispatcher);
+		this.allLeaderContenders = new HashSet<>();
+		this.listeners = new HashSet<>();
+	}
+
+	// ------------------------------------------------------------------------
+	//  shutdown and errors
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Shuts down this leader election service.
+	 * 
+	 * <p>This method does not perform a clean revocation of the leader status and
+	 * no notification to any leader listeners. It simply notifies all contenders
+	 * and listeners that the service is no longer available.
+	 */
+	public void shutdown() {
+		synchronized (lock) {
+			shutdownInternally(new Exception("Leader election service is shutting down"));
+		}
+	}
+
+	private void fatalError(Throwable error) {
+		LOG.error("Embedded leader election service encountered a fatal error. Shutting down service.", error);
+
+		synchronized (lock) {
+			shutdownInternally(new Exception("Leader election service is shutting down after a fatal error", error));
+		}
+	}
+
+	@GuardedBy("lock")
+	private void shutdownInternally(Exception exceptionForHandlers) {
+		assert Thread.holdsLock(lock);
+
+		if (!shutdown) {
+			// clear all leader status
+			currentLeaderProposed = null;
+			currentLeaderConfirmed = null;
+			currentLeaderSessionId = null;
+			currentLeaderAddress = null;
+
+			// fail all registered listeners
+			for (EmbeddedLeaderElectionService service : allLeaderContenders) {
+				service.shutdown(exceptionForHandlers);
+			}
+			allLeaderContenders.clear();
+
+			// fail all registered listeners
+			for (EmbeddedLeaderRetrievalService service : listeners) {
+				service.shutdown(exceptionForHandlers);
+			}
+			listeners.clear();
+
+			shutdown = true;
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  creating contenders and listeners
+	// ------------------------------------------------------------------------
+
+	public LeaderElectionService createLeaderElectionService() {
+		checkState(!shutdown, "leader election service is shut down");
+		return new EmbeddedLeaderElectionService();
+	}
+
+	public LeaderRetrievalService createLeaderRetrievalService() {
+		checkState(!shutdown, "leader election service is shut down");
+		return new EmbeddedLeaderRetrievalService();
+	}
+
+	// ------------------------------------------------------------------------
+	//  adding and removing contenders & listeners
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Callback from leader contenders when they start their service.
+	 */
+	void addContender(EmbeddedLeaderElectionService service, LeaderContender contender) {
+		synchronized (lock) {
+			checkState(!shutdown, "leader election service is shut down");
+			checkState(!service.running, "leader election service is already started");
+
+			try {
+				if (!allLeaderContenders.add(service)) {
+					throw new IllegalStateException("leader election service was added to this service multiple times");
+				}
+
+				service.contender = contender;
+				service.running = true;
+
+				updateLeader();
+			}
+			catch (Throwable t) {
+				fatalError(t);
+			}
+		}
+	}
+
+	/**
+	 * Callback from leader contenders when they stop their service.
+	 */
+	void removeContender(EmbeddedLeaderElectionService service) {
+		synchronized (lock) {
+			// if the service was not even started, simply do nothing
+			if (!service.running || shutdown) {
+				return;
+			}
+
+			try {
+				if (!allLeaderContenders.remove(service)) {
+					throw new IllegalStateException("leader election service does not belong to this service");
+				}
+
+				// stop the service
+				service.contender = null;
+				service.running = false;
+				service.isLeader = false;
+
+				// if that was the current leader, unset its status
+				if (currentLeaderConfirmed == service) {
+					currentLeaderConfirmed = null;
+					currentLeaderSessionId = null;
+					currentLeaderAddress = null;
+				}
+				if (currentLeaderProposed == service) {
+					currentLeaderProposed = null;
+					currentLeaderSessionId = null;
+				}
+
+				updateLeader();
+			}
+			catch (Throwable t) {
+				fatalError(t);
+			}
+		}
+	}
+
+	/**
+	 * Callback from leader contenders when they confirm a leader grant
+	 */
+	void confirmLeader(final EmbeddedLeaderElectionService service, final UUID leaderSessionId) {
+		synchronized (lock) {
+			// if the service was shut down in the meantime, ignore this confirmation
+			if (!service.running || shutdown) {
+				return;
+			}
+
+			try {
+				// check if the confirmation is for the same grant, or whether it is a stale grant 
+				if (service == currentLeaderProposed && currentLeaderSessionId.equals(leaderSessionId)) {
+					final String address = service.contender.getAddress();
+					LOG.info("Received confirmation of leadership for leader {} / session={}", address, leaderSessionId);
+
+					// mark leadership
+					currentLeaderConfirmed = service;
+					currentLeaderAddress = address;
+					currentLeaderProposed = null;
+					service.isLeader = true;
+
+					// notify all listeners
+					for (EmbeddedLeaderRetrievalService listener : listeners) {
+						notificationExecutor.execute(
+								new NotifyOfLeaderCall(address, leaderSessionId, listener.listener, LOG));
+					}
+				}
+				else {
+					LOG.debug("Received confirmation of leadership for a stale leadership grant. Ignoring.");
+				}
+			}
+			catch (Throwable t) {
+				fatalError(t);
+			}
+		}
+	}
+
+	@GuardedBy("lock")
+	private void updateLeader() {
+		// this must be called under the lock
+		assert Thread.holdsLock(lock);
+
+		if (currentLeaderConfirmed == null && currentLeaderProposed == null) {
+			// we need a new leader
+			if (allLeaderContenders.isEmpty()) {
+				// no new leader available, tell everyone that there is no leader currently
+				for (EmbeddedLeaderRetrievalService listener : listeners) {
+					notificationExecutor.execute(
+							new NotifyOfLeaderCall(null, null, listener.listener, LOG));
+				}
+			}
+			else {
+				// propose a leader and ask it
+				final UUID leaderSessionId = UUID.randomUUID();
+				EmbeddedLeaderElectionService leaderService = allLeaderContenders.iterator().next();
+
+				currentLeaderSessionId = leaderSessionId;
+				currentLeaderProposed = leaderService;
+
+				notificationExecutor.execute(
+						new GrantLeadershipCall(leaderService.contender, leaderSessionId, LOG));
+			}
+		}
+	}
+
+	void addListener(EmbeddedLeaderRetrievalService service, LeaderRetrievalListener listener) {
+		synchronized (lock) {
+			checkState(!shutdown, "leader election service is shut down");
+			checkState(!service.running, "leader retrieval service is already started");
+
+			try {
+				if (!listeners.add(service)) {
+					throw new IllegalStateException("leader retrieval service was added to this service multiple times");
+				}
+
+				service.listener = listener;
+				service.running = true;
+
+				// if we already have a leader, immediately notify this new listener
+				if (currentLeaderConfirmed != null) {
+					notificationExecutor.execute(
+							new NotifyOfLeaderCall(currentLeaderAddress, currentLeaderSessionId, listener, LOG));
+				}
+			}
+			catch (Throwable t) {
+				fatalError(t);
+			}
+		}
+	}
+
+	void removeListener(EmbeddedLeaderRetrievalService service) {
+		synchronized (lock) {
+			// if the service was not even started, simply do nothing
+			if (!service.running || shutdown) {
+				return;
+			}
+
+			try {
+				if (!listeners.remove(service)) {
+					throw new IllegalStateException("leader retrieval service does not belong to this service");
+				}
+
+				// stop the service
+				service.listener = null;
+				service.running = false;
+			}
+			catch (Throwable t) {
+				fatalError(t);
+			}
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  election and retrieval service implementations 
+	// ------------------------------------------------------------------------
+
+	private class EmbeddedLeaderElectionService implements LeaderElectionService {
+
+		volatile LeaderContender contender;
+
+		volatile boolean isLeader;
+
+		volatile boolean running;
+
+		@Override
+		public void start(LeaderContender contender) throws Exception {
+			checkNotNull(contender);
+			addContender(this, contender);
+		}
+
+		@Override
+		public void stop() throws Exception {
+			removeContender(this);
+		}
+
+		@Override
+		public void confirmLeaderSessionID(UUID leaderSessionID) {
+			checkNotNull(leaderSessionID);
+			confirmLeader(this, leaderSessionID);
+		}
+
+		@Override
+		public boolean hasLeadership() {
+			return isLeader;
+		}
+
+		void shutdown(Exception cause) {
+			if (running) {
+				running = false;
+				isLeader = false;
+				contender.handleError(cause);
+				contender = null;
+			}
+		}
+	}
+
+	// ------------------------------------------------------------------------
+
+	private class EmbeddedLeaderRetrievalService implements LeaderRetrievalService {
+
+		volatile LeaderRetrievalListener listener;
+
+		volatile boolean running;
+
+		@Override
+		public void start(LeaderRetrievalListener listener) throws Exception {
+			checkNotNull(listener);
+			addListener(this, listener);
+		}
+
+		@Override
+		public void stop() throws Exception {
+			removeListener(this);
+		}
+
+		public void shutdown(Exception cause) {
+			if (running) {
+				running = false;
+				listener.handleError(cause);
+				listener = null;
+			}
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  asynchronous notifications
+	// ------------------------------------------------------------------------
+
+	private static class NotifyOfLeaderCall implements Runnable {
+
+		@Nullable
+		private final String address;       // null if leader revoked without new leader
+		@Nullable
+		private final UUID leaderSessionId; // null if leader revoked without new leader
+
+		private final LeaderRetrievalListener listener;
+		private final Logger logger;
+
+		NotifyOfLeaderCall(
+				@Nullable String address,
+				@Nullable UUID leaderSessionId,
+				LeaderRetrievalListener listener,
+				Logger logger) {
+
+			this.address = address;
+			this.leaderSessionId = leaderSessionId;
+			this.listener = checkNotNull(listener);
+			this.logger = checkNotNull(logger);
+		}
+
+		@Override
+		public void run() {
+			try {
+				listener.notifyLeaderAddress(address, leaderSessionId);
+			}
+			catch (Throwable t) {
+				logger.warn("Error notifying leader listener about new leader", t);
+				listener.handleError(t instanceof Exception ? (Exception) t : new Exception(t));
+			}
+		}
+	}
+
+	// ------------------------------------------------------------------------
+
+	private static class GrantLeadershipCall implements Runnable {
+
+		private final LeaderContender contender;
+		private final UUID leaderSessionId;
+		private final Logger logger;
+
+		GrantLeadershipCall(
+				LeaderContender contender,
+				UUID leaderSessionId,
+				Logger logger) {
+
+			this.contender = checkNotNull(contender);
+			this.leaderSessionId = checkNotNull(leaderSessionId);
+			this.logger = checkNotNull(logger);
+		}
+
+		@Override
+		public void run() {
+			try {
+				contender.grantLeadership(leaderSessionId);
+			}
+			catch (Throwable t) {
+				logger.warn("Error notifying leader listener about new leader", t);
+				contender.handleError(t instanceof Exception ? (Exception) t : new Exception(t));
+			}
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9615f15b/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 38e372d..3e88e8c 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
@@ -154,4 +154,13 @@ public class TestingHighAvailabilityServices implements HighAvailabilityServices
 	public BlobStore createBlobStore() throws IOException {
 		return new VoidBlobStore();
 	}
+
+	// ------------------------------------------------------------------------
+	//  Shutdown
+	// ------------------------------------------------------------------------
+
+	@Override
+	public void shutdown() throws Exception {
+		// nothing to do, since this should not shut down individual services, but cross service parts
+	}
 }


[36/52] [abbrv] flink git commit: [hotfix] Improve logging and thread characteristics for 'EmbeddedNonHaServices'

Posted by se...@apache.org.
[hotfix] Improve logging and thread characteristics for 'EmbeddedNonHaServices'


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

Branch: refs/heads/master
Commit: 10f7e8615dba79b0fc001a0d80c100bbfb9a6a28
Parents: 440e757
Author: Stephan Ewen <se...@apache.org>
Authored: Fri Dec 2 17:43:10 2016 +0100
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Dec 23 20:54:26 2016 +0100

----------------------------------------------------------------------
 .../runtime/highavailability/EmbeddedNonHaServices.java     | 7 +++++--
 .../highavailability/nonha/AbstractNonHaServices.java       | 9 +++++++--
 .../highavailability/nonha/EmbeddedLeaderService.java       | 5 ++++-
 .../src/test/resources/log4j-test.properties                | 2 +-
 4 files changed, 17 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/10f7e861/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/EmbeddedNonHaServices.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/EmbeddedNonHaServices.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/EmbeddedNonHaServices.java
index 523218e..b91cec1 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/EmbeddedNonHaServices.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/EmbeddedNonHaServices.java
@@ -56,7 +56,10 @@ public class EmbeddedNonHaServices extends AbstractNonHaServices implements High
 
 	@Override
 	public void shutdown() throws Exception {
-		super.shutdown();
-		resourceManagerLeaderService.shutdown();
+		try {
+			super.shutdown();
+		} finally {
+			resourceManagerLeaderService.shutdown();
+		}
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/10f7e861/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/AbstractNonHaServices.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/AbstractNonHaServices.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/AbstractNonHaServices.java
index 237727f..474faa8 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/AbstractNonHaServices.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/AbstractNonHaServices.java
@@ -55,7 +55,7 @@ public abstract class AbstractNonHaServices implements HighAvailabilityServices
 
 	private final HashMap<JobID, EmbeddedLeaderService> jobManagerLeaderServices;
 
-	private final RunningJobsRegistry runningJobsRegistry;
+	private final NonHaRegistry runningJobsRegistry;
 
 	private boolean shutdown;
 
@@ -167,8 +167,13 @@ public abstract class AbstractNonHaServices implements HighAvailabilityServices
 
 		@Override
 		public Thread newThread(@Nonnull Runnable r) {
-			Thread thread = new Thread(r, "Flink HA services thread #" + enumerator.incrementAndGet());
+			Thread thread = new Thread(r, "Flink HA Services Thread #" + enumerator.incrementAndGet());
+
+			// HA threads should have a very high priority, but not
+			// keep the JVM running by themselves
+			thread.setPriority(Thread.MAX_PRIORITY);
 			thread.setDaemon(true);
+
 			return thread;
 		}
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/10f7e861/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/EmbeddedLeaderService.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/EmbeddedLeaderService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/EmbeddedLeaderService.java
index 84ac551..9fad9be 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/EmbeddedLeaderService.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/EmbeddedLeaderService.java
@@ -225,7 +225,7 @@ public class EmbeddedLeaderService {
 				// check if the confirmation is for the same grant, or whether it is a stale grant 
 				if (service == currentLeaderProposed && currentLeaderSessionId.equals(leaderSessionId)) {
 					final String address = service.contender.getAddress();
-					LOG.info("Received confirmation of leadership for leader {} / session={}", address, leaderSessionId);
+					LOG.info("Received confirmation of leadership for leader {} , session={}", address, leaderSessionId);
 
 					// mark leadership
 					currentLeaderConfirmed = service;
@@ -271,6 +271,9 @@ public class EmbeddedLeaderService {
 				currentLeaderSessionId = leaderSessionId;
 				currentLeaderProposed = leaderService;
 
+				LOG.info("Proposing leadership to contender {} @ {}",
+						leaderService.contender, leaderService.contender.getAddress());
+
 				notificationExecutor.execute(
 						new GrantLeadershipCall(leaderService.contender, leaderSessionId, LOG));
 			}

http://git-wip-us.apache.org/repos/asf/flink/blob/10f7e861/flink-streaming-java/src/test/resources/log4j-test.properties
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/resources/log4j-test.properties b/flink-streaming-java/src/test/resources/log4j-test.properties
index 881dc06..e7cd3e0 100644
--- a/flink-streaming-java/src/test/resources/log4j-test.properties
+++ b/flink-streaming-java/src/test/resources/log4j-test.properties
@@ -24,4 +24,4 @@ log4j.appender.A1=org.apache.log4j.ConsoleAppender
 
 # A1 uses PatternLayout.
 log4j.appender.A1.layout=org.apache.log4j.PatternLayout
-log4j.appender.A1.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n
+log4j.appender.A1.layout.ConversionPattern=%-5r [%-38t] %-5p %-60c %x - %m%n


[47/52] [abbrv] flink git commit: [FLINK-4927] [yarn] refine YARN Resource manager according to till's comments

Posted by se...@apache.org.
[FLINK-4927] [yarn] refine YARN Resource manager according to till's comments


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

Branch: refs/heads/master
Commit: e2922add100338776db765a62deb02f556845cf9
Parents: 371997a
Author: shuai.xus <sh...@alibaba-inc.com>
Authored: Mon Dec 5 15:36:16 2016 +0800
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Dec 23 20:54:27 2016 +0100

----------------------------------------------------------------------
 .../apache/flink/yarn/YarnResourceManager.java  | 68 +++++++++++++++-----
 1 file changed, 51 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/e2922add/flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManager.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManager.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManager.java
index 6280bdf..9b9ea39 100644
--- a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManager.java
+++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManager.java
@@ -56,6 +56,7 @@ import org.apache.hadoop.yarn.util.Records;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import scala.concurrent.duration.FiniteDuration;
+import org.apache.flink.util.ExceptionUtils;
 
 import java.io.File;
 import java.io.IOException;
@@ -79,21 +80,24 @@ public class YarnResourceManager extends ResourceManager<ResourceID> implements
 	/** The process environment variables */
 	private final Map<String, String> ENV;
 
+	/** The default registration timeout for task executor in seconds. */
+	private final static int DEFAULT_TASK_MANAGER_REGISTRATION_DURATION = 300;
+
 	/** The heartbeat interval while the resource master is waiting for containers */
 	private static final int FAST_YARN_HEARTBEAT_INTERVAL_MS = 500;
 
 	/** The default heartbeat interval during regular operation */
 	private static final int DEFAULT_YARN_HEARTBEAT_INTERVAL_MS = 5000;
 
-	/** The maximum time that TaskExecutors may be waiting to register at the ResourceManager before they quit */
-	private static final FiniteDuration TASKEXECUTOR_REGISTRATION_TIMEOUT = new FiniteDuration(5, TimeUnit.MINUTES);
+	/** The default memory of task executor to allocate (in MB) */
+	private static final int DEFAULT_TSK_EXECUTOR_MEMORY_SIZE = 1024;
 
 	/** Environment variable name of the final container id used by the YarnResourceManager.
 	 * Container ID generation may vary across Hadoop versions. */
 	final static String ENV_FLINK_CONTAINER_ID = "_FLINK_CONTAINER_ID";
 	
-	/** Environment variable name of the hostname used by the Yarn.
-	 * TaskExecutor use this host name to start port. */
+	/** Environment variable name of the hostname given by the YARN.
+	 * In task executor we use the hostnames given by YARN consistently throughout akka */
 	final static String ENV_FLINK_NODE_ID = "_FLINK_NODE_ID";
 
 	/** Default heartbeat interval between this resource manager and the YARN ResourceManager */
@@ -112,6 +116,8 @@ public class YarnResourceManager extends ResourceManager<ResourceID> implements
 	/** The number of containers requested, but not yet granted */
 	private int numPendingContainerRequests;
 
+	final private Map<ResourceProfile, Integer> resourcePriorities = new HashMap<>();
+
 	public YarnResourceManager(
 			Configuration flinkConfig,
 			Map<String, String> env,
@@ -173,20 +179,28 @@ public class YarnResourceManager extends ResourceManager<ResourceID> implements
 	@Override
 	public void shutDown() throws Exception {
 		// shut down all components
+		Throwable firstException = null;
 		if (resourceManagerClient != null) {
 			try {
 				resourceManagerClient.stop();
 			} catch (Throwable t) {
-				LOG.error("Could not cleanly shut down the Asynchronous Resource Manager Client", t);
+				firstException = t;
 			}
 		}
 		if (nodeManagerClient != null) {
 			try {
 				nodeManagerClient.stop();
 			} catch (Throwable t) {
-				LOG.error("Could not cleanly shut down the Node Manager Client", t);
+				if (firstException == null) {
+					firstException = t;
+				} else {
+					firstException.addSuppressed(t);
+				}
 			}
 		}
+		if (firstException != null) {
+			ExceptionUtils.rethrowException(firstException, "Error while shutting down YARN resource manager");
+		}
 		super.shutDown();
 	}
 
@@ -207,13 +221,10 @@ public class YarnResourceManager extends ResourceManager<ResourceID> implements
 	public void startNewWorker(ResourceProfile resourceProfile) {
 		// Priority for worker containers - priorities are intra-application
 		//TODO: set priority according to the resource allocated
-		Priority priority = Priority.newInstance(0);
-		int mem = resourceProfile.getMemoryInMB() <= Integer.MAX_VALUE ? (int)resourceProfile.getMemoryInMB() : Integer.MAX_VALUE;
-		if (mem < 0) {
-			mem = 1024;
-		}
-		int vcore = resourceProfile.getCpuCores() < 1 ? 1 : (int)resourceProfile.getCpuCores() + 1;
-		Resource capability = Resource.newInstance(mem , vcore);
+		Priority priority = Priority.newInstance(generatePriority(resourceProfile));
+		int mem = resourceProfile.getMemoryInMB() < 0 ? DEFAULT_TSK_EXECUTOR_MEMORY_SIZE : (int)resourceProfile.getMemoryInMB();
+		int vcore = resourceProfile.getCpuCores() < 1 ? 1 : (int)resourceProfile.getCpuCores();
+		Resource capability = Resource.newInstance(mem, vcore);
 		requestYarnContainer(capability, priority);
 	}
 
@@ -234,7 +245,6 @@ public class YarnResourceManager extends ResourceManager<ResourceID> implements
 		for (ContainerStatus container : list) {
 			if (container.getExitStatus() < 0) {
 				notifyWorkerFailed(new ResourceID(container.getContainerId().toString()), container.getDiagnostics());
-				// TODO: notice job master slot fail
 			}
 		}
 	}
@@ -253,7 +263,7 @@ public class YarnResourceManager extends ResourceManager<ResourceID> implements
 			}
 			catch (Throwable t) {
 				// failed to launch the container, will release the failed one and ask for a new one
-				LOG.error("Could not start TaskManager in container " + container, t);
+				LOG.error("Could not start TaskManager in container {},", container, t);
 				resourceManagerClient.releaseAssignedContainer(container.getId());
 				requestYarnContainer(container.getResource(), container.getPriority());
 			}
@@ -265,7 +275,11 @@ public class YarnResourceManager extends ResourceManager<ResourceID> implements
 
 	@Override
 	public void onShutdownRequest() {
-		// Nothing to do
+		try {
+			shutDown();
+		} catch (Exception e) {
+			LOG.warn("Fail to shutdown the YARN resource manager.", e);
+		}
 	}
 
 	@Override
@@ -336,8 +350,11 @@ public class YarnResourceManager extends ResourceManager<ResourceID> implements
 				taskManagerParameters.taskManagerTotalMemoryMB(),
 				taskManagerParameters.taskManagerHeapSizeMB(),
 				taskManagerParameters.taskManagerDirectMemoryLimitMB());
+		int timeout = flinkConfig.getInteger(ConfigConstants.TASK_MANAGER_MAX_REGISTRATION_DURATION, 
+				DEFAULT_TASK_MANAGER_REGISTRATION_DURATION);
+		FiniteDuration teRegistrationTimeout = new FiniteDuration(timeout, TimeUnit.SECONDS);
 		final Configuration taskManagerConfig = BootstrapTools.generateTaskManagerConfiguration(
-				flinkConfig, "", 0, 1, TASKEXECUTOR_REGISTRATION_TIMEOUT);
+				flinkConfig, "", 0, 1, teRegistrationTimeout);
 		LOG.debug("TaskManager configuration: {}", taskManagerConfig);
 
 		ContainerLaunchContext taskExecutorLaunchContext = createTaskExecutorContext(
@@ -549,4 +566,21 @@ public class YarnResourceManager extends ResourceManager<ResourceID> implements
 
 		return ctx;
 	}
+	
+	/**
+	 * Generate priority by given resouce profile. 
+	 * Priority is only used for distinguishing request of different resource.
+	 * @param resourceProfile The resource profile of a request
+	 * @return The priority of this resource profile.
+	 */
+	private int generatePriority(ResourceProfile resourceProfile) {
+		if (resourcePriorities.containsKey(resourceProfile)) {
+			return resourcePriorities.get(resourceProfile);
+		} else {
+			int priority = resourcePriorities.size();
+			resourcePriorities.put(resourceProfile, priority);
+			return priority;
+		}
+	}
+
 }


[39/52] [abbrv] flink git commit: [FLINK-5093] Fix bug about throwing ConcurrentModificationException when stopping TimerService.

Posted by se...@apache.org.
[FLINK-5093] Fix bug about throwing ConcurrentModificationException when stopping TimerService.

[FLINK-5093] Remove useless import.

This closes #2828.


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

Branch: refs/heads/master
Commit: eefcbbded159ab5819fe2b09606f8a33b9150254
Parents: c424900
Author: biao.liub <bi...@alibaba-inc.com>
Authored: Fri Nov 18 18:15:37 2016 +0800
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Dec 23 20:54:26 2016 +0100

----------------------------------------------------------------------
 .../runtime/taskexecutor/slot/TimerService.java | 16 ++++-
 .../taskexecutor/slot/TimerServiceTest.java     | 68 ++++++++++++++++++++
 2 files changed, 81 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/eefcbbde/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/slot/TimerService.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/slot/TimerService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/slot/TimerService.java
index e28e801..14c9ab1 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/slot/TimerService.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/slot/TimerService.java
@@ -60,9 +60,7 @@ public class TimerService<K> {
 	}
 
 	public void stop() {
-		for (K key: timeouts.keySet()) {
-			unregisterTimeout(key);
-		}
+		unregisterAllTimeouts();
 
 		timeoutListener = null;
 
@@ -101,6 +99,18 @@ public class TimerService<K> {
 	}
 
 	/**
+	 * Unregister all timeouts.
+	 */
+	protected void unregisterAllTimeouts() {
+		for (Timeout<K> timeout : timeouts.values()) {
+			if (timeout != null) {
+				timeout.cancel();
+			}
+		}
+		timeouts.clear();
+	}
+
+	/**
 	 * Check whether the timeout for the given key and ticket is still valid (not yet unregistered
 	 * and not yet overwritten).
 	 *

http://git-wip-us.apache.org/repos/asf/flink/blob/eefcbbde/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/slot/TimerServiceTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/slot/TimerServiceTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/slot/TimerServiceTest.java
new file mode 100644
index 0000000..9dd5f39
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/slot/TimerServiceTest.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.taskexecutor.slot;
+
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.junit.Test;
+import org.mockito.internal.util.reflection.Whitebox;
+
+import java.util.Map;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+
+import static org.junit.Assert.*;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyLong;
+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 TimerServiceTest {
+	/**
+	 * Test all timeouts registered can be unregistered
+	 * @throws Exception
+   */
+	@Test
+	@SuppressWarnings("unchecked")
+	public void testUnregisterAllTimeouts() throws Exception {
+		// Prepare all instances.
+		ScheduledExecutorService scheduledExecutorService = mock(ScheduledExecutorService.class);
+		ScheduledFuture scheduledFuture = mock(ScheduledFuture.class);
+		when(scheduledExecutorService.schedule(any(Runnable.class), anyLong(), any(TimeUnit.class)))
+			.thenReturn(scheduledFuture);
+		TimerService<AllocationID> timerService = new TimerService<>(scheduledExecutorService);
+		TimeoutListener<AllocationID> listener = mock(TimeoutListener.class);
+
+		timerService.start(listener);
+
+		// Invoke register and unregister.
+		timerService.registerTimeout(new AllocationID(), 10, TimeUnit.SECONDS);
+		timerService.registerTimeout(new AllocationID(), 10, TimeUnit.SECONDS);
+
+		timerService.unregisterAllTimeouts();
+
+		// Verify.
+		Map<?, ?> timeouts = (Map<?, ?>) Whitebox.getInternalState(timerService, "timeouts");
+		assertTrue(timeouts.isEmpty());
+		verify(scheduledFuture, times(2)).cancel(true);
+	}
+
+}


[04/52] [abbrv] flink git commit: [hotfix] Add a DefaultSlotManager similar to the TestingSlotManager

Posted by se...@apache.org.
[hotfix] Add a DefaultSlotManager similar to the TestingSlotManager


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

Branch: refs/heads/master
Commit: a685c75166470d9abd32870265149f1dcbd2a2cf
Parents: aaf80a2
Author: Stephan Ewen <se...@apache.org>
Authored: Mon Oct 17 16:38:17 2016 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Dec 23 20:54:23 2016 +0100

----------------------------------------------------------------------
 .../ResourceManagerServices.java                |  2 +-
 .../slotmanager/DefaultSlotManager.java         | 69 ++++++++++++++++++++
 2 files changed, 70 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/a685c751/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerServices.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerServices.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerServices.java
index 16d0a7d..c524604 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerServices.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerServices.java
@@ -38,7 +38,7 @@ public interface ResourceManagerServices {
 	void allocateResource(ResourceProfile resourceProfile);
 
 	/**
-	 * Gets the async excutor which executes outside of the main thread of the ResourceManager
+	 * Gets the async executor which executes outside of the main thread of the ResourceManager
 	 */
 	Executor getAsyncExecutor();
 

http://git-wip-us.apache.org/repos/asf/flink/blob/a685c751/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/DefaultSlotManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/DefaultSlotManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/DefaultSlotManager.java
new file mode 100644
index 0000000..9508936
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/DefaultSlotManager.java
@@ -0,0 +1,69 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceSlot;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerServices;
+import org.apache.flink.runtime.resourcemanager.SlotRequest;
+
+import java.util.Iterator;
+import java.util.Map;
+
+/**
+ * A slot manager that answers requests with slots without any special logic. The first slot
+ * in the maps to match a request is chosen.
+ */
+public class DefaultSlotManager extends SlotManager {
+
+	public DefaultSlotManager(ResourceManagerServices rmServices) {
+		super(rmServices);
+	}
+
+	@Override
+	protected ResourceSlot chooseSlotToUse(SlotRequest request, Map<SlotID, ResourceSlot> freeSlots) {
+		final Iterator<ResourceSlot> slotIterator = freeSlots.values().iterator();
+		if (slotIterator.hasNext()) {
+			return slotIterator.next();
+		} else {
+			return null;
+		}
+	}
+
+	@Override
+	protected SlotRequest chooseRequestToFulfill(ResourceSlot offeredSlot, Map<AllocationID, SlotRequest> pendingRequests) {
+		final Iterator<SlotRequest> requestIterator = pendingRequests.values().iterator();
+		if (requestIterator.hasNext()) {
+			return requestIterator.next();
+		} else {
+			return null;
+		}
+	}
+
+	// ------------------------------------------------------------------------
+
+	public static class Factory implements SlotManagerFactory {
+
+		@Override
+		public SlotManager create(ResourceManagerServices rmServices) {
+			return new DefaultSlotManager(rmServices);
+		}
+	}
+}


[43/52] [abbrv] flink git commit: [FLINK-5254] [yarn] Implement YARN High-Availability Services

Posted by se...@apache.org.
[FLINK-5254] [yarn] Implement YARN High-Availability Services


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

Branch: refs/heads/master
Commit: 2a7dbda79a00863a511fcf64b339770d1d00f805
Parents: e2922ad
Author: Stephan Ewen <se...@apache.org>
Authored: Mon Dec 5 01:34:32 2016 +0100
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Dec 23 20:54:27 2016 +0100

----------------------------------------------------------------------
 .../flink/configuration/Configuration.java      |  12 +-
 .../FsNegativeRunningJobsRegistryTest.java      | 121 ++++++
 .../flink/runtime/fs/hdfs/HadoopFileSystem.java |  20 +-
 .../highavailability/EmbeddedNonHaServices.java |  15 +-
 .../FsNegativeRunningJobsRegistry.java          | 153 ++++++++
 .../HighAvailabilityServices.java               |  69 +++-
 .../runtime/highavailability/NonHaServices.java |  21 +-
 .../highavailability/ServicesThreadFactory.java |  40 ++
 .../highavailability/ZookeeperHaServices.java   |  17 +-
 .../SingleLeaderElectionService.java            | 384 +++++++++++++++++++
 .../nonha/AbstractNonHaServices.java            |  29 +-
 .../nonha/EmbeddedLeaderService.java            |   2 +-
 .../runtime/jobmanager/slots/AllocatedSlot.java |   1 -
 .../StandaloneLeaderRetrievalService.java       |   1 +
 .../flink/runtime/minicluster/MiniCluster.java  |   2 +-
 .../resourcemanager/JobLeaderIdService.java     |   2 +-
 .../resourcemanager/ResourceManagerRunner.java  |   3 +-
 .../flink/runtime/rpc/RpcServiceUtils.java      |  70 ++++
 .../FsNegativeRunningJobsRegistryTest.java      |  85 ++++
 .../TestingHighAvailabilityServices.java        |  14 +-
 .../SingleLeaderElectionServiceTest.java        | 226 +++++++++++
 flink-yarn/pom.xml                              |  21 +
 ...bstractYarnFlinkApplicationMasterRunner.java |  34 +-
 .../flink/yarn/YarnApplicationMasterRunner.java |   2 +-
 .../yarn/YarnFlinkApplicationMasterRunner.java  |  18 +-
 .../flink/yarn/YarnTaskExecutorRunner.java      |  13 +-
 .../yarn/configuration/YarnConfigOptions.java   |  49 +++
 .../AbstractYarnNonHaServices.java              | 105 +++++
 .../YarnHighAvailabilityServices.java           | 343 +++++++++++++++++
 .../YarnIntraNonHaMasterServices.java           | 188 +++++++++
 .../YarnPreConfiguredMasterNonHaServices.java   | 172 +++++++++
 .../YarnIntraNonHaMasterServicesTest.java       | 149 +++++++
 .../YarnPreConfiguredMasterHaServicesTest.java  | 234 +++++++++++
 33 files changed, 2537 insertions(+), 78 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/2a7dbda7/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 f15c669..8f23435 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
@@ -44,7 +44,7 @@ public class Configuration extends ExecutionConfig.GlobalJobParameters
 		implements IOReadableWritable, java.io.Serializable, Cloneable {
 
 	private static final long serialVersionUID = 1L;
-	
+
 	private static final byte TYPE_STRING = 0;
 	private static final byte TYPE_INT = 1;
 	private static final byte TYPE_LONG = 2;
@@ -52,14 +52,14 @@ public class Configuration extends ExecutionConfig.GlobalJobParameters
 	private static final byte TYPE_FLOAT = 4;
 	private static final byte TYPE_DOUBLE = 5;
 	private static final byte TYPE_BYTES = 6;
-	
+
 	/** The log object used for debugging. */
 	private static final Logger LOG = LoggerFactory.getLogger(Configuration.class);
-	
+
 
 	/** Stores the concrete key/value pairs of this configuration object. */
 	protected final HashMap<String, Object> confData;
-	
+
 	// --------------------------------------------------------------------------------------------
 
 	/**
@@ -639,12 +639,16 @@ public class Configuration extends ExecutionConfig.GlobalJobParameters
 		Object o = getRawValue(configOption.key());
 
 		if (o != null) {
+			// found a value for the current proper key
 			return o;
 		}
 		else if (configOption.hasDeprecatedKeys()) {
+			// try the deprecated keys
 			for (String deprecatedKey : configOption.deprecatedKeys()) {
 				Object oo = getRawValue(deprecatedKey);
 				if (oo != null) {
+					LOG.warn("Config uses deprecated configuration key '{}' instead of proper key '{}'", 
+							deprecatedKey, configOption.key());
 					return oo;
 				}
 			}

http://git-wip-us.apache.org/repos/asf/flink/blob/2a7dbda7/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/FsNegativeRunningJobsRegistryTest.java
----------------------------------------------------------------------
diff --git a/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/FsNegativeRunningJobsRegistryTest.java b/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/FsNegativeRunningJobsRegistryTest.java
new file mode 100644
index 0000000..40d75e8
--- /dev/null
+++ b/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/FsNegativeRunningJobsRegistryTest.java
@@ -0,0 +1,121 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.hdfstests;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.runtime.highavailability.FsNegativeRunningJobsRegistry;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.File;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+public class FsNegativeRunningJobsRegistryTest {
+
+	@ClassRule
+	public static final TemporaryFolder TEMP_DIR = new TemporaryFolder();
+
+	private static MiniDFSCluster HDFS_CLUSTER;
+
+	private static Path HDFS_ROOT_PATH;
+
+	// ------------------------------------------------------------------------
+	//  startup / shutdown
+	// ------------------------------------------------------------------------
+
+	@BeforeClass
+	public static void createHDFS() throws Exception {
+		final File tempDir = TEMP_DIR.newFolder();
+
+		Configuration hdConf = new Configuration();
+		hdConf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, tempDir.getAbsolutePath());
+
+		MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(hdConf);
+		HDFS_CLUSTER = builder.build();
+
+		HDFS_ROOT_PATH = new Path("hdfs://" + HDFS_CLUSTER.getURI().getHost() + ":"
+				+ HDFS_CLUSTER.getNameNodePort() + "/");
+	}
+
+	@AfterClass
+	public static void destroyHDFS() {
+		if (HDFS_CLUSTER != null) {
+			HDFS_CLUSTER.shutdown();
+		}
+		HDFS_CLUSTER = null;
+		HDFS_ROOT_PATH = null;
+	}
+
+	// ------------------------------------------------------------------------
+	//  Tests
+	// ------------------------------------------------------------------------
+	
+	@Test
+	public void testCreateAndSetFinished() throws Exception {
+		final Path workDir = new Path(HDFS_ROOT_PATH, "test-work-dir");
+		final JobID jid = new JobID();
+
+		FsNegativeRunningJobsRegistry registry = new FsNegativeRunningJobsRegistry(workDir);
+
+		// initially, without any call, the job is considered running
+		assertTrue(registry.isJobRunning(jid));
+
+		// repeated setting should not affect the status
+		registry.setJobRunning(jid);
+		assertTrue(registry.isJobRunning(jid));
+
+		// set the job to finished and validate
+		registry.setJobFinished(jid);
+		assertFalse(registry.isJobRunning(jid));
+
+		// another registry should pick this up
+		FsNegativeRunningJobsRegistry otherRegistry = new FsNegativeRunningJobsRegistry(workDir);
+		assertFalse(otherRegistry.isJobRunning(jid));
+	}
+
+	@Test
+	public void testSetFinishedAndRunning() throws Exception {
+		final Path workDir = new Path(HDFS_ROOT_PATH, "�nother_w�rk_direct�r�");
+		final JobID jid = new JobID();
+
+		FsNegativeRunningJobsRegistry registry = new FsNegativeRunningJobsRegistry(workDir);
+
+		// set the job to finished and validate
+		registry.setJobFinished(jid);
+		assertFalse(registry.isJobRunning(jid));
+
+		// set the job to back to running and validate
+		registry.setJobRunning(jid);
+		assertTrue(registry.isJobRunning(jid));
+
+		// another registry should pick this up
+		FsNegativeRunningJobsRegistry otherRegistry = new FsNegativeRunningJobsRegistry(workDir);
+		assertTrue(otherRegistry.isJobRunning(jid));
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/2a7dbda7/flink-runtime/src/main/java/org/apache/flink/runtime/fs/hdfs/HadoopFileSystem.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/fs/hdfs/HadoopFileSystem.java b/flink-runtime/src/main/java/org/apache/flink/runtime/fs/hdfs/HadoopFileSystem.java
index 0eab032..36dfa55 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/fs/hdfs/HadoopFileSystem.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/fs/hdfs/HadoopFileSystem.java
@@ -35,6 +35,8 @@ import java.lang.reflect.Method;
 import java.net.URI;
 import java.net.UnknownHostException;
 
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
 /**
  * Concrete implementation of the {@link FileSystem} base class for the Hadoop File System. The
  * class is a wrapper class which encapsulated the original Hadoop HDFS API.
@@ -60,7 +62,8 @@ public final class HadoopFileSystem extends FileSystem implements HadoopFileSyst
 
 
 	/**
-	 * Creates a new DistributedFileSystem object to access HDFS
+	 * Creates a new DistributedFileSystem object to access HDFS, based on a class name
+	 * and picking up the configuration from the class path or the Flink configuration.
 	 * 
 	 * @throws IOException
 	 *         throw if the required HDFS classes cannot be instantiated
@@ -76,6 +79,21 @@ public final class HadoopFileSystem extends FileSystem implements HadoopFileSyst
 		this.fs = instantiateFileSystem(fsClass);
 	}
 
+	/**
+	 * Creates a new DistributedFileSystem that uses the given Hadoop
+	 * {@link org.apache.hadoop.fs.FileSystem} under the hood.
+	 *
+	 * @param hadoopConfig The Hadoop configuration that the FileSystem is based on.
+	 * @param hadoopFileSystem The Hadoop FileSystem that will be used under the hood.
+	 */
+	public HadoopFileSystem(
+			org.apache.hadoop.conf.Configuration hadoopConfig,
+			org.apache.hadoop.fs.FileSystem hadoopFileSystem) {
+
+		this.conf = checkNotNull(hadoopConfig, "hadoopConfig");
+		this.fs = checkNotNull(hadoopFileSystem, "hadoopFileSystem");
+	}
+
 	private Class<? extends org.apache.hadoop.fs.FileSystem> getDefaultHDFSClass() throws IOException {
 		Class<? extends org.apache.hadoop.fs.FileSystem> fsClass = null;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/2a7dbda7/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/EmbeddedNonHaServices.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/EmbeddedNonHaServices.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/EmbeddedNonHaServices.java
index b91cec1..a417599 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/EmbeddedNonHaServices.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/EmbeddedNonHaServices.java
@@ -43,6 +43,12 @@ public class EmbeddedNonHaServices extends AbstractNonHaServices implements High
 	// ------------------------------------------------------------------------
 
 	@Override
+	public String getResourceManagerEndpointName() {
+		// dynamic actor name
+		return null;
+	}
+
+	@Override
 	public LeaderRetrievalService getResourceManagerLeaderRetriever() {
 		return resourceManagerLeaderService.createLeaderRetrievalService();
 	}
@@ -55,11 +61,16 @@ public class EmbeddedNonHaServices extends AbstractNonHaServices implements High
 	// ------------------------------------------------------------------------
 
 	@Override
-	public void shutdown() throws Exception {
+	public void close() throws Exception {
 		try {
-			super.shutdown();
+			super.close();
 		} finally {
 			resourceManagerLeaderService.shutdown();
 		}
 	}
+
+	@Override
+	public void closeAndCleanupAllData() throws Exception {
+		close();
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/2a7dbda7/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/FsNegativeRunningJobsRegistry.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/FsNegativeRunningJobsRegistry.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/FsNegativeRunningJobsRegistry.java
new file mode 100644
index 0000000..9d8b226
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/FsNegativeRunningJobsRegistry.java
@@ -0,0 +1,153 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.highavailability;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.core.fs.FSDataOutputStream;
+import org.apache.flink.core.fs.FileSystem;
+import org.apache.flink.core.fs.Path;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * This {@link RunningJobsRegistry} tracks the status jobs via marker files,
+ * marking finished jobs via marker files.
+ * 
+ * <p>The general contract is the following:
+ * <ul>
+ *     <li>Initially, a marker file does not exist (no one created it, yet), which means
+ *         the specific job is assumed to be running</li>
+ *     <li>The JobManager that finishes calls this service to create the marker file,
+ *         which marks the job as finished.</li>
+ *     <li>If a JobManager gains leadership at some point when shutdown is in progress,
+ *         it will see the marker file and realize that the job is finished.</li>
+ *     <li>The application framework is expected to clean the file once the application
+ *         is completely shut down. At that point, no JobManager will attempt to
+ *         start the job, even if it gains leadership.</li>
+ * </ul>
+ * 
+ * <p>It is especially tailored towards deployment modes like for example
+ * YARN, where HDFS is available as a persistent file system, and the YARN
+ * application's working directories on HDFS are automatically cleaned
+ * up after the application completed. 
+ */
+public class FsNegativeRunningJobsRegistry implements RunningJobsRegistry {
+
+	private static final String PREFIX = ".job_complete_";
+
+	private final FileSystem fileSystem;
+
+	private final Path basePath;
+
+	/**
+	 * Creates a new registry that writes to the FileSystem and working directory
+	 * denoted by the given path.
+	 * 
+	 * <p>The initialization will attempt to write to the given working directory, in
+	 * order to catch setup/configuration errors early.
+	 *
+	 * @param workingDirectory The working directory for files to track the job status.
+	 *
+	 * @throws IOException Thrown, if the specified directory cannot be accessed.
+	 */
+	public FsNegativeRunningJobsRegistry(Path workingDirectory) throws IOException {
+		this(workingDirectory.getFileSystem(), workingDirectory);
+	}
+
+	/**
+	 * Creates a new registry that writes its files to the given FileSystem at
+	 * the given working directory path.
+	 * 
+	 * <p>The initialization will attempt to write to the given working directory, in
+	 * order to catch setup/configuration errors early.
+	 *
+	 * @param fileSystem The FileSystem to use for the marker files.
+	 * @param workingDirectory The working directory for files to track the job status.
+	 *
+	 * @throws IOException Thrown, if the specified directory cannot be accessed.
+	 */
+	public FsNegativeRunningJobsRegistry(FileSystem fileSystem, Path workingDirectory) throws IOException {
+		this.fileSystem = checkNotNull(fileSystem, "fileSystem");
+		this.basePath = checkNotNull(workingDirectory, "workingDirectory");
+
+		// to be safe, attempt to write to the working directory, to
+		// catch problems early
+		final Path testFile = new Path(workingDirectory, ".registry_test");
+		try (FSDataOutputStream out = fileSystem.create(testFile, false)) {
+			out.write(42);
+		}
+		catch (IOException e) {
+			throw new IOException("Unable to write to working directory: " + workingDirectory, e);
+		}
+		finally {
+			fileSystem.delete(testFile, false);
+		}
+	}
+
+	// ------------------------------------------------------------------------
+
+	@Override
+	public void setJobRunning(JobID jobID) throws IOException {
+		checkNotNull(jobID, "jobID");
+		final Path filePath = createMarkerFilePath(jobID);
+
+		// delete the marker file, if it exists
+		try {
+			fileSystem.delete(filePath, false);
+		}
+		catch (FileNotFoundException e) {
+			// apparently job was already considered running
+		}
+	}
+
+	@Override
+	public void setJobFinished(JobID jobID) throws IOException {
+		checkNotNull(jobID, "jobID");
+		final Path filePath = createMarkerFilePath(jobID);
+
+		// create the file
+		// to avoid an exception if the job already exists, set overwrite=true
+		try (FSDataOutputStream out = fileSystem.create(filePath, true)) {
+			out.write(42);
+		}
+	}
+
+	@Override
+	public boolean isJobRunning(JobID jobID) throws IOException {
+		checkNotNull(jobID, "jobID");
+
+		// check for the existence of the file
+		try {
+			fileSystem.getFileStatus(createMarkerFilePath(jobID));
+			// file was found --> job is terminated
+			return false;
+		}
+		catch (FileNotFoundException e) {
+			// file does not exist, job is still running
+			return true;
+		}
+	}
+
+	private Path createMarkerFilePath(JobID jobId) {
+		return new Path(basePath, PREFIX + jobId.toString());
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/2a7dbda7/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 360de7b..4169204 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
@@ -26,19 +26,45 @@ import org.apache.flink.runtime.leaderelection.LeaderElectionService;
 import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
 
 import java.io.IOException;
+import java.util.UUID;
 
 /**
- * This class gives access to all services needed for
- *
+ * The HighAvailabilityServices give access to all services needed for a highly-available
+ * setup. In particular, the services provide access to highly available storage and
+ * registries, as well as distributed counters and leader election.
+ * 
  * <ul>
  *     <li>ResourceManager leader election and leader retrieval</li>
  *     <li>JobManager leader election and leader retrieval</li>
  *     <li>Persistence for checkpoint metadata</li>
  *     <li>Registering the latest completed checkpoint(s)</li>
- *     <li>Persistence for submitted job graph</li>
+ *     <li>Persistence for the BLOB store</li>
+ *     <li>Registry that marks a job's status</li>
+ *     <li>Naming of RPC endpoints</li>
  * </ul>
  */
-public interface HighAvailabilityServices {
+public interface HighAvailabilityServices extends AutoCloseable {
+
+	// ------------------------------------------------------------------------
+	//  Constants
+	// ------------------------------------------------------------------------
+
+	/**
+	 * This UUID should be used when no proper leader election happens, but a simple
+	 * pre-configured leader is used. That is for example the case in non-highly-available
+	 * standalone setups.
+	 */
+	UUID DEFAULT_LEADER_ID = new UUID(0, 0);
+
+	// ------------------------------------------------------------------------
+	//  Endpoint Naming
+	// ------------------------------------------------------------------------
+
+	String getResourceManagerEndpointName();
+
+	// ------------------------------------------------------------------------
+	//  Services
+	// ------------------------------------------------------------------------
 
 	/**
 	 * Gets the leader retriever for the cluster's resource manager.
@@ -88,7 +114,7 @@ public interface HighAvailabilityServices {
 	 *
 	 * @return Running job registry to retrieve running jobs
 	 */
-	RunningJobsRegistry getRunningJobsRegistry();
+	RunningJobsRegistry getRunningJobsRegistry() throws Exception;
 
 	/**
 	 * Creates the BLOB store in which BLOBs are stored in a highly-available fashion.
@@ -99,11 +125,38 @@ public interface HighAvailabilityServices {
 	BlobStore createBlobStore() throws IOException;
 
 	// ------------------------------------------------------------------------
+	//  Shutdown and Cleanup
+	// ------------------------------------------------------------------------
 
 	/**
-	 * Shut the high availability service down.
+	 * Closes the high availability services, releasing all resources.
+	 * 
+	 * <p>This method <b>does not delete or clean up</b> any data stored in external stores
+	 * (file systems, ZooKeeper, etc). Another instance of the high availability
+	 * services will be able to recover the job.
+	 * 
+	 * <p>If an exception occurs during closing services, this method will attempt to
+	 * continue closing other services and report exceptions only after all services
+	 * have been attempted to be closed.
 	 *
-	 * @throws Exception if the shut down fails
+	 * @throws Exception Thrown, if an exception occurred while closing these services.
+	 */
+	@Override
+	void close() throws Exception;
+
+	/**
+	 * Closes the high availability services (releasing all resources) and deletes
+	 * all data stored by these services in external stores.
+	 * 
+	 * <p>After this method was called, the any job or session that was managed by
+	 * these high availability services will be unrecoverable.
+	 * 
+	 * <p>If an exception occurs during cleanup, this method will attempt to
+	 * continue the cleanup and report exceptions only after all cleanup steps have
+	 * been attempted.
+	 * 
+	 * @throws Exception Thrown, if an exception occurred while closing these services
+	 *                   or cleaning up data stored by them.
 	 */
-	void shutdown() throws Exception;
+	void closeAndCleanupAllData() throws Exception;
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/2a7dbda7/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 75f44ed..d644fb9 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
@@ -18,14 +18,12 @@
 
 package org.apache.flink.runtime.highavailability;
 
+import org.apache.flink.runtime.highavailability.leaderelection.SingleLeaderElectionService;
 import org.apache.flink.runtime.highavailability.nonha.AbstractNonHaServices;
 import org.apache.flink.runtime.leaderelection.LeaderElectionService;
-import org.apache.flink.runtime.leaderelection.StandaloneLeaderElectionService;
 import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
 import org.apache.flink.runtime.leaderretrieval.StandaloneLeaderRetrievalService;
 
-import java.util.UUID;
-
 import static org.apache.flink.util.Preconditions.checkNotNull;
 
 /**
@@ -39,6 +37,9 @@ import static org.apache.flink.util.Preconditions.checkNotNull;
  */
 public class NonHaServices extends AbstractNonHaServices implements HighAvailabilityServices {
 
+	/** The constant name of the ResourceManager RPC endpoint */
+	private static final String RESOURCE_MANAGER_RPC_ENDPOINT_NAME = "resource_manager";
+
 	/** The fix address of the ResourceManager */
 	private final String resourceManagerAddress;
 
@@ -53,16 +54,26 @@ public class NonHaServices extends AbstractNonHaServices implements HighAvailabi
 	}
 
 	// ------------------------------------------------------------------------
+	//  Names
+	// ------------------------------------------------------------------------
+
+	@Override
+	public String getResourceManagerEndpointName() {
+		return RESOURCE_MANAGER_RPC_ENDPOINT_NAME;
+	}
+
+
+	// ------------------------------------------------------------------------
 	//  Services
 	// ------------------------------------------------------------------------
 
 	@Override
 	public LeaderRetrievalService getResourceManagerLeaderRetriever() {
-		return new StandaloneLeaderRetrievalService(resourceManagerAddress, new UUID(0, 0));
+		return new StandaloneLeaderRetrievalService(resourceManagerAddress, DEFAULT_LEADER_ID);
 	}
 
 	@Override
 	public LeaderElectionService getResourceManagerLeaderElectionService() {
-		return new StandaloneLeaderElectionService();
+		return new SingleLeaderElectionService(getExecutorService(), DEFAULT_LEADER_ID);
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/2a7dbda7/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/ServicesThreadFactory.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/ServicesThreadFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/ServicesThreadFactory.java
new file mode 100644
index 0000000..24667e4
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/ServicesThreadFactory.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.highavailability;
+
+import javax.annotation.Nonnull;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.atomic.AtomicInteger;
+
+public class ServicesThreadFactory implements ThreadFactory {
+
+	private AtomicInteger enumerator = new AtomicInteger();
+
+	@Override
+	public Thread newThread(@Nonnull Runnable r) {
+		Thread thread = new Thread(r, "Flink HA Services Thread #" + enumerator.incrementAndGet());
+
+		// HA threads should have a very high priority, but not
+		// keep the JVM running by themselves
+		thread.setPriority(Thread.MAX_PRIORITY);
+		thread.setDaemon(true);
+
+		return thread;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/2a7dbda7/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 3e909e8..25d21ef 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
@@ -108,6 +108,16 @@ public class ZookeeperHaServices implements HighAvailabilityServices {
 	// ------------------------------------------------------------------------
 
 	@Override
+	public String getResourceManagerEndpointName() {
+		// since the resource manager name must be dynamic, we return null here
+		return null;
+	}
+
+	// ------------------------------------------------------------------------
+	//  Services
+	// ------------------------------------------------------------------------
+
+	@Override
 	public LeaderRetrievalService getResourceManagerLeaderRetriever() {
 		return ZooKeeperUtils.createLeaderRetrievalService(client, configuration, RESOURCE_MANAGER_LEADER_PATH);
 	}
@@ -174,10 +184,15 @@ public class ZookeeperHaServices implements HighAvailabilityServices {
 	// ------------------------------------------------------------------------
 
 	@Override
-	public void shutdown() throws Exception {
+	public void close() throws Exception {
 		client.close();
 	}
 
+	@Override
+	public void closeAndCleanupAllData() throws Exception {
+		close();
+	}
+
 	// ------------------------------------------------------------------------
 	//  Utilities
 	// ------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/flink/blob/2a7dbda7/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/leaderelection/SingleLeaderElectionService.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/leaderelection/SingleLeaderElectionService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/leaderelection/SingleLeaderElectionService.java
new file mode 100644
index 0000000..26e3cbf
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/leaderelection/SingleLeaderElectionService.java
@@ -0,0 +1,384 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.highavailability.leaderelection;
+
+import org.apache.flink.runtime.leaderelection.LeaderContender;
+import org.apache.flink.runtime.leaderelection.LeaderElectionService;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+import java.util.HashSet;
+import java.util.UUID;
+import java.util.concurrent.Executor;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * An implementation of the {@link LeaderElectionService} interface that handles a single
+ * leader contender. When started, this service immediately grants the contender the leadership.
+ * 
+ * <p>The implementation accepts a single static leader session ID and is hence compatible with
+ * pre-configured single leader (no leader failover) setups.
+ * 
+ * <p>This implementation supports a series of leader listeners that receive notifications about
+ * the leader contender.
+ */
+public class SingleLeaderElectionService implements LeaderElectionService {
+
+	private static final Logger LOG = LoggerFactory.getLogger(SingleLeaderElectionService.class);
+
+	// ------------------------------------------------------------------------
+
+	/** lock for all operations on this instance */
+	private final Object lock = new Object();
+
+	/** The executor service that dispatches notifications */
+	private final Executor notificationExecutor;
+
+	/** The leader ID assigned to the immediate leader */
+	private final UUID leaderId;
+
+	@GuardedBy("lock")
+	private final HashSet<EmbeddedLeaderRetrievalService> listeners;
+
+	/** The currently proposed leader */
+	@GuardedBy("lock")
+	private volatile LeaderContender proposedLeader;
+
+	/** The confirmed leader */
+	@GuardedBy("lock")
+	private volatile LeaderContender leader;
+
+	/** The address of the confirmed leader */
+	@GuardedBy("lock")
+	private volatile String leaderAddress;
+
+	/** Flag marking this service as shutdown, meaning it cannot be started again */
+	@GuardedBy("lock")
+	private volatile boolean shutdown;
+
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Creates a new leader election service. The service assigns the given leader ID
+	 * to the leader contender.
+	 * 
+	 * @param leaderId The constant leader ID assigned to the leader.
+	 */
+	public SingleLeaderElectionService(Executor notificationsDispatcher, UUID leaderId) {
+		this.notificationExecutor = checkNotNull(notificationsDispatcher);
+		this.leaderId = checkNotNull(leaderId);
+		this.listeners = new HashSet<>();
+	}
+
+	// ------------------------------------------------------------------------
+	//  leader election service
+	// ------------------------------------------------------------------------
+
+	@Override
+	public void start(LeaderContender contender) throws Exception {
+		checkNotNull(contender, "contender");
+
+		synchronized (lock) {
+			checkState(!shutdown, "service is shut down");
+			checkState(proposedLeader == null, "service already started");
+
+			// directly grant leadership to the given contender
+			proposedLeader = contender;
+			notificationExecutor.execute(new GrantLeadershipCall(contender, leaderId));
+		}
+	}
+
+	@Override
+	public void stop() {
+		synchronized (lock) {
+			// notify all listeners that there is no leader
+			for (EmbeddedLeaderRetrievalService listener : listeners) {
+				notificationExecutor.execute(
+						new NotifyOfLeaderCall(null, null, listener.listener, LOG));
+			}
+
+			// if there was a leader, revoke its leadership
+			if (leader != null) {
+				try {
+					leader.revokeLeadership();
+				} catch (Throwable t) {
+					leader.handleError(t instanceof Exception ? (Exception) t : new Exception(t));
+				}
+			}
+
+			proposedLeader = null;
+			leader = null;
+			leaderAddress = null;
+		}
+	}
+
+	@Override
+	public void confirmLeaderSessionID(UUID leaderSessionID) {
+		checkNotNull(leaderSessionID, "leaderSessionID");
+		checkArgument(leaderSessionID.equals(leaderId), "confirmed wrong leader session id");
+
+		synchronized (lock) {
+			checkState(!shutdown, "service is shut down");
+			checkState(proposedLeader != null, "no leader proposed yet");
+			checkState(leader == null, "leader already confirmed");
+
+			// accept the confirmation
+			final String address = proposedLeader.getAddress();
+			leaderAddress = address;
+			leader = proposedLeader;
+
+			// notify all listeners
+			for (EmbeddedLeaderRetrievalService listener : listeners) {
+				notificationExecutor.execute(
+						new NotifyOfLeaderCall(address, leaderId, listener.listener, LOG));
+			}
+		}
+	}
+
+	@Override
+	public boolean hasLeadership() {
+		synchronized (lock) {
+			return leader != null;
+		}
+	}
+
+	void errorOnGrantLeadership(LeaderContender contender, Throwable error) {
+		LOG.warn("Error notifying leader listener about new leader", error);
+		contender.handleError(error instanceof Exception ? (Exception) error : new Exception(error));
+		
+		synchronized (lock) {
+			if (proposedLeader == contender) {
+				proposedLeader = null;
+				leader = null;
+			}
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  shutdown
+	// ------------------------------------------------------------------------
+
+	public boolean isShutdown() {
+		return shutdown;
+	}
+
+	public void shutdown() {
+		shutdownInternally(new Exception("The leader service is shutting down"));
+	}
+
+	private void shutdownInternally(Exception exceptionForHandlers) {
+		synchronized (lock) {
+			if (shutdown) {
+				return;
+			}
+
+			shutdown = true;
+
+			// fail the leader (if there is one)
+			if (leader != null) {
+				try {
+					leader.handleError(exceptionForHandlers);
+				} catch (Throwable ignored) {}
+			}
+
+			// clear all leader status
+			leader = null;
+			proposedLeader = null;
+			leaderAddress = null;
+
+			// fail all registered listeners
+			for (EmbeddedLeaderRetrievalService service : listeners) {
+				service.shutdown(exceptionForHandlers);
+			}
+			listeners.clear();
+		}
+	}
+
+	private void fatalError(Throwable error) {
+		LOG.error("Embedded leader election service encountered a fatal error. Shutting down service.", error);
+
+		shutdownInternally(new Exception("Leader election service is shutting down after a fatal error", error));
+	}
+
+	// ------------------------------------------------------------------------
+	//  leader listeners
+	// ------------------------------------------------------------------------
+
+	public LeaderRetrievalService createLeaderRetrievalService() {
+		checkState(!shutdown, "leader election service is shut down");
+		return new EmbeddedLeaderRetrievalService();
+	}
+
+	void addListener(EmbeddedLeaderRetrievalService service, LeaderRetrievalListener listener) {
+		synchronized (lock) {
+			checkState(!shutdown, "leader election service is shut down");
+			checkState(!service.running, "leader retrieval service is already started");
+
+			try {
+				if (!listeners.add(service)) {
+					throw new IllegalStateException("leader retrieval service was added to this service multiple times");
+				}
+
+				service.listener = listener;
+				service.running = true;
+
+				// if we already have a leader, immediately notify this new listener
+				if (leader != null) {
+					notificationExecutor.execute(
+							new NotifyOfLeaderCall(leaderAddress, leaderId, listener, LOG));
+				}
+			}
+			catch (Throwable t) {
+				fatalError(t);
+			}
+		}
+	}
+
+	void removeListener(EmbeddedLeaderRetrievalService service) {
+		synchronized (lock) {
+			// if the service was not even started, simply do nothing
+			if (!service.running || shutdown) {
+				return;
+			}
+
+			try {
+				if (!listeners.remove(service)) {
+					throw new IllegalStateException("leader retrieval service does not belong to this service");
+				}
+
+				// stop the service
+				service.listener = null;
+				service.running = false;
+			}
+			catch (Throwable t) {
+				fatalError(t);
+			}
+		}
+	}
+
+	// ------------------------------------------------------------------------
+
+	private class EmbeddedLeaderRetrievalService implements LeaderRetrievalService {
+
+		volatile LeaderRetrievalListener listener;
+
+		volatile boolean running;
+
+		@Override
+		public void start(LeaderRetrievalListener listener) throws Exception {
+			checkNotNull(listener);
+			addListener(this, listener);
+		}
+
+		@Override
+		public void stop() throws Exception {
+			removeListener(this);
+		}
+
+		void shutdown(Exception cause) {
+			if (running) {
+				final LeaderRetrievalListener lst = listener;
+				running = false;
+				listener = null;
+
+				try {
+					lst.handleError(cause);
+				} catch (Throwable ignored) {}
+			}
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  asynchronous notifications
+	// ------------------------------------------------------------------------
+
+	/**
+	 * This runnable informs a leader contender that it gained leadership.
+	 */
+	private class GrantLeadershipCall implements Runnable {
+
+		private final LeaderContender contender;
+		private final UUID leaderSessionId;
+
+		GrantLeadershipCall(LeaderContender contender, UUID leaderSessionId) {
+
+			this.contender = checkNotNull(contender);
+			this.leaderSessionId = checkNotNull(leaderSessionId);
+		}
+
+		@Override
+		public void run() {
+			try {
+				contender.grantLeadership(leaderSessionId);
+			}
+			catch (Throwable t) {
+				errorOnGrantLeadership(contender, t);
+			}
+		}
+	}
+
+	// ------------------------------------------------------------------------
+
+	/**
+	 * This runnable informs a leader listener of a new leader
+	 */
+	private static class NotifyOfLeaderCall implements Runnable {
+
+		@Nullable
+		private final String address;       // null if leader revoked without new leader
+		@Nullable
+		private final UUID leaderSessionId; // null if leader revoked without new leader
+
+		private final LeaderRetrievalListener listener;
+		private final Logger logger;
+
+		NotifyOfLeaderCall(
+				@Nullable String address,
+				@Nullable UUID leaderSessionId,
+				LeaderRetrievalListener listener,
+				Logger logger) {
+
+			this.address = address;
+			this.leaderSessionId = leaderSessionId;
+			this.listener = checkNotNull(listener);
+			this.logger = checkNotNull(logger);
+		}
+
+		@Override
+		public void run() {
+			try {
+				listener.notifyLeaderAddress(address, leaderSessionId);
+			}
+			catch (Throwable t) {
+				logger.warn("Error notifying leader listener about new leader", t);
+				listener.handleError(t instanceof Exception ? (Exception) t : new Exception(t));
+			}
+		}
+	}
+
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/2a7dbda7/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/AbstractNonHaServices.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/AbstractNonHaServices.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/AbstractNonHaServices.java
index 474faa8..b10e414 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/AbstractNonHaServices.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/AbstractNonHaServices.java
@@ -25,19 +25,17 @@ import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory;
 import org.apache.flink.runtime.checkpoint.StandaloneCheckpointRecoveryFactory;
 import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
 import org.apache.flink.runtime.highavailability.RunningJobsRegistry;
+import org.apache.flink.runtime.highavailability.ServicesThreadFactory;
 import org.apache.flink.runtime.jobmanager.StandaloneSubmittedJobGraphStore;
 import org.apache.flink.runtime.jobmanager.SubmittedJobGraphStore;
 import org.apache.flink.runtime.leaderelection.LeaderElectionService;
 import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
 
-import javax.annotation.Nonnull;
 import javax.annotation.concurrent.GuardedBy;
 import java.io.IOException;
 import java.util.HashMap;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
-import java.util.concurrent.ThreadFactory;
-import java.util.concurrent.atomic.AtomicInteger;
 
 import static org.apache.flink.util.Preconditions.checkNotNull;
 import static org.apache.flink.util.Preconditions.checkState;
@@ -132,7 +130,7 @@ public abstract class AbstractNonHaServices implements HighAvailabilityServices
 	// ------------------------------------------------------------------------
 
 	@Override
-	public void shutdown() throws Exception {
+	public void close() throws Exception {
 		synchronized (lock) {
 			if (!shutdown) {
 				shutdown = true;
@@ -149,6 +147,12 @@ public abstract class AbstractNonHaServices implements HighAvailabilityServices
 		}
 	}
 
+	@Override
+	public void closeAndCleanupAllData() throws Exception {
+		// this stores no data, so this method is the same as 'close()'
+		close();
+	}
+
 	private void checkNotShutdown() {
 		checkState(!shutdown, "high availability services are shut down");
 	}
@@ -160,21 +164,4 @@ public abstract class AbstractNonHaServices implements HighAvailabilityServices
 	protected ExecutorService getExecutorService() {
 		return executor;
 	}
-
-	private static final class ServicesThreadFactory implements ThreadFactory {
-
-		private AtomicInteger enumerator = new AtomicInteger();
-
-		@Override
-		public Thread newThread(@Nonnull Runnable r) {
-			Thread thread = new Thread(r, "Flink HA Services Thread #" + enumerator.incrementAndGet());
-
-			// HA threads should have a very high priority, but not
-			// keep the JVM running by themselves
-			thread.setPriority(Thread.MAX_PRIORITY);
-			thread.setDaemon(true);
-
-			return thread;
-		}
-	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/2a7dbda7/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/EmbeddedLeaderService.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/EmbeddedLeaderService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/EmbeddedLeaderService.java
index 9fad9be..d4eba26 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/EmbeddedLeaderService.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/EmbeddedLeaderService.java
@@ -461,7 +461,7 @@ public class EmbeddedLeaderService {
 				contender.grantLeadership(leaderSessionId);
 			}
 			catch (Throwable t) {
-				logger.warn("Error notifying leader listener about new leader", t);
+				logger.warn("Error granting leadership to contender", t);
 				contender.handleError(t instanceof Exception ? (Exception) t : new Exception(t));
 			}
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/2a7dbda7/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/slots/AllocatedSlot.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/slots/AllocatedSlot.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/slots/AllocatedSlot.java
index a0c608d..269a8f3 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/slots/AllocatedSlot.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/slots/AllocatedSlot.java
@@ -22,7 +22,6 @@ import org.apache.flink.api.common.JobID;
 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.taskexecutor.TaskExecutorGateway;
 import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
 
 import static org.apache.flink.util.Preconditions.checkNotNull;

http://git-wip-us.apache.org/repos/asf/flink/blob/2a7dbda7/flink-runtime/src/main/java/org/apache/flink/runtime/leaderretrieval/StandaloneLeaderRetrievalService.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/leaderretrieval/StandaloneLeaderRetrievalService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/leaderretrieval/StandaloneLeaderRetrievalService.java
index 16b163c..4ad4646 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/leaderretrieval/StandaloneLeaderRetrievalService.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/leaderretrieval/StandaloneLeaderRetrievalService.java
@@ -51,6 +51,7 @@ public class StandaloneLeaderRetrievalService implements LeaderRetrievalService
 	 *
 	 * @param leaderAddress The leader's pre-configured address
 	 */
+	@Deprecated
 	public StandaloneLeaderRetrievalService(String leaderAddress) {
 		this.leaderAddress = checkNotNull(leaderAddress);
 		this.leaderId = null;

http://git-wip-us.apache.org/repos/asf/flink/blob/2a7dbda7/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java
index 29a6e59..1933554 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java
@@ -345,7 +345,7 @@ public class MiniCluster {
 		// shut down high-availability services
 		if (haServices != null) {
 			try {
-				haServices.shutdown();
+				haServices.closeAndCleanupAllData();
 			} catch (Exception e) {
 				exception = firstOrSuppressed(e, exception);
 			}

http://git-wip-us.apache.org/repos/asf/flink/blob/2a7dbda7/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/JobLeaderIdService.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/JobLeaderIdService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/JobLeaderIdService.java
index 56e72c0..6c7e249 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/JobLeaderIdService.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/JobLeaderIdService.java
@@ -59,7 +59,7 @@ public class JobLeaderIdService {
 	/** Actions to call when the job leader changes */
 	private JobLeaderIdActions jobLeaderIdActions;
 
-	public JobLeaderIdService(HighAvailabilityServices highAvailabilityServices) {
+	public JobLeaderIdService(HighAvailabilityServices highAvailabilityServices) throws Exception {
 		this.highAvailabilityServices = Preconditions.checkNotNull(highAvailabilityServices);
 
 		this.runningJobsRegistry = highAvailabilityServices.getRunningJobsRegistry();

http://git-wip-us.apache.org/repos/asf/flink/blob/2a7dbda7/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerRunner.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerRunner.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerRunner.java
index 959b727..e0dee0b 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerRunner.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerRunner.java
@@ -21,7 +21,6 @@ package org.apache.flink.runtime.resourcemanager;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
 import org.apache.flink.runtime.metrics.MetricRegistry;
-import org.apache.flink.runtime.resourcemanager.exceptions.ConfigurationException;
 import org.apache.flink.runtime.resourcemanager.slotmanager.DefaultSlotManager;
 import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManagerFactory;
 import org.apache.flink.runtime.rpc.FatalErrorHandler;
@@ -46,7 +45,7 @@ public class ResourceManagerRunner implements FatalErrorHandler {
 			final Configuration configuration,
 			final RpcService rpcService,
 			final HighAvailabilityServices highAvailabilityServices,
-			final MetricRegistry metricRegistry) throws ConfigurationException {
+			final MetricRegistry metricRegistry) throws Exception {
 
 		Preconditions.checkNotNull(configuration);
 		Preconditions.checkNotNull(rpcService);

http://git-wip-us.apache.org/repos/asf/flink/blob/2a7dbda7/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcServiceUtils.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcServiceUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcServiceUtils.java
index 1ac54ac..018c3ed 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcServiceUtils.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcServiceUtils.java
@@ -21,19 +21,36 @@ package org.apache.flink.runtime.rpc;
 import akka.actor.ActorSystem;
 import com.typesafe.config.Config;
 import org.apache.flink.api.common.time.Time;
+import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.akka.AkkaUtils;
+import org.apache.flink.runtime.net.SSLUtils;
 import org.apache.flink.runtime.rpc.akka.AkkaRpcService;
 import org.apache.flink.util.NetUtils;
+
 import org.jboss.netty.channel.ChannelException;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
+import java.net.UnknownHostException;
 
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * These RPC utilities contain helper methods around RPC use, such as starting an RPC service,
+ * or constructing RPC addresses.
+ */
 public class RpcServiceUtils {
+
 	private static final Logger LOG = LoggerFactory.getLogger(RpcServiceUtils.class);
 
+	// ------------------------------------------------------------------------
+	//  RPC instantiation
+	// ------------------------------------------------------------------------
+
 	/**
 	 * Utility method to create RPC service from configuration and hostname, port.
 	 *
@@ -78,4 +95,57 @@ public class RpcServiceUtils {
 		final Time timeout = Time.milliseconds(AkkaUtils.getTimeout(configuration).toMillis());
 		return new AkkaRpcService(actorSystem, timeout);
 	}
+
+	// ------------------------------------------------------------------------
+	//  RPC endpoint addressing
+	// ------------------------------------------------------------------------
+
+	/**
+	 *
+	 * @param hostname     The hostname or address where the target RPC service is listening.
+	 * @param port         The port where the target RPC service is listening.
+	 * @param endpointName The name of the RPC endpoint.
+	 * @param config       Teh configuration from which to deduce further settings.
+	 *
+	 * @return The RPC URL of the specified RPC endpoint.
+	 */
+	public static String getRpcUrl(String hostname, int port, String endpointName, Configuration config)
+			throws UnknownHostException {
+
+		checkNotNull(config, "config is null");
+
+		final boolean sslEnabled = config.getBoolean(
+					ConfigConstants.AKKA_SSL_ENABLED,
+					ConfigConstants.DEFAULT_AKKA_SSL_ENABLED) &&
+				SSLUtils.getSSLEnabled(config);
+
+		return getRpcUrl(hostname, port, endpointName, sslEnabled);
+	}
+
+	/**
+	 * 
+	 * @param hostname     The hostname or address where the target RPC service is listening.
+	 * @param port         The port where the target RPC service is listening.
+	 * @param endpointName The name of the RPC endpoint.
+	 * @param secure       True, if security/encryption is enabled, false otherwise.
+	 * 
+	 * @return The RPC URL of the specified RPC endpoint.
+	 */
+	public static String getRpcUrl(String hostname, int port, String endpointName, boolean secure)
+			throws UnknownHostException {
+
+		checkNotNull(hostname, "hostname is null");
+		checkNotNull(endpointName, "endpointName is null");
+		checkArgument(port > 0 && port <= 65535, "port must be in [1, 65535]");
+
+		final String protocol = secure ? "akka.ssl.tcp" : "akka.tcp";
+		final String hostPort = NetUtils.hostAndPortToUrlString(hostname, port);
+
+		return String.format("%s://flink@%s/user/%s", protocol, hostPort, endpointName);
+	}
+
+	// ------------------------------------------------------------------------
+
+	/** This class is not meant to be instantiated */
+	private RpcServiceUtils() {}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/2a7dbda7/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/FsNegativeRunningJobsRegistryTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/FsNegativeRunningJobsRegistryTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/FsNegativeRunningJobsRegistryTest.java
new file mode 100644
index 0000000..f1ece0e
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/FsNegativeRunningJobsRegistryTest.java
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.highavailability;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.File;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+public class FsNegativeRunningJobsRegistryTest extends TestLogger {
+
+	@Rule
+	public final TemporaryFolder tempFolder = new TemporaryFolder();
+
+	@Test
+	public void testCreateAndSetFinished() throws Exception {
+		final File folder = tempFolder.newFolder();
+		final String uri = folder.toURI().toString();
+
+		final JobID jid = new JobID();
+
+		FsNegativeRunningJobsRegistry registry = new FsNegativeRunningJobsRegistry(new Path(uri));
+
+		// initially, without any call, the job is considered running
+		assertTrue(registry.isJobRunning(jid));
+
+		// repeated setting should not affect the status
+		registry.setJobRunning(jid);
+		assertTrue(registry.isJobRunning(jid));
+
+		// set the job to finished and validate
+		registry.setJobFinished(jid);
+		assertFalse(registry.isJobRunning(jid));
+
+		// another registry should pick this up
+		FsNegativeRunningJobsRegistry otherRegistry = new FsNegativeRunningJobsRegistry(new Path(uri));
+		assertFalse(otherRegistry.isJobRunning(jid));
+	}
+
+	@Test
+	public void testSetFinishedAndRunning() throws Exception {
+		final File folder = tempFolder.newFolder();
+		final String uri = folder.toURI().toString();
+
+		final JobID jid = new JobID();
+
+		FsNegativeRunningJobsRegistry registry = new FsNegativeRunningJobsRegistry(new Path(uri));
+
+		// set the job to finished and validate
+		registry.setJobFinished(jid);
+		assertFalse(registry.isJobRunning(jid));
+
+		// set the job to back to running and validate
+		registry.setJobRunning(jid);
+		assertTrue(registry.isJobRunning(jid));
+
+		// another registry should pick this up
+		FsNegativeRunningJobsRegistry otherRegistry = new FsNegativeRunningJobsRegistry(new Path(uri));
+		assertTrue(otherRegistry.isJobRunning(jid));
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/2a7dbda7/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 e0f71ee..3f9865c 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
@@ -155,12 +155,22 @@ public class TestingHighAvailabilityServices implements HighAvailabilityServices
 		return new VoidBlobStore();
 	}
 
+	@Override
+	public String getResourceManagerEndpointName() {
+		throw new UnsupportedOperationException();
+	}
+
 	// ------------------------------------------------------------------------
 	//  Shutdown
 	// ------------------------------------------------------------------------
 
 	@Override
-	public void shutdown() throws Exception {
-		// nothing to do, since this should not shut down individual services, but cross service parts
+	public void close() throws Exception {
+		// nothing to do
+	}
+
+	@Override
+	public void closeAndCleanupAllData() throws Exception {
+		// nothing to do
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/2a7dbda7/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/leaderelection/SingleLeaderElectionServiceTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/leaderelection/SingleLeaderElectionServiceTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/leaderelection/SingleLeaderElectionServiceTest.java
new file mode 100644
index 0000000..a9805a1
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/leaderelection/SingleLeaderElectionServiceTest.java
@@ -0,0 +1,226 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.highavailability.leaderelection;
+
+import org.apache.flink.runtime.concurrent.Executors;
+import org.apache.flink.runtime.leaderelection.LeaderContender;
+import org.apache.flink.runtime.leaderelection.LeaderElectionService;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+import org.apache.flink.util.StringUtils;
+
+import org.junit.Test;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+import java.util.Random;
+import java.util.UUID;
+import java.util.concurrent.Executor;
+
+import static org.junit.Assert.fail;
+import static org.mockito.Mockito.*;
+
+/**
+ * Tests for the {@link SingleLeaderElectionService}.
+ */
+public class SingleLeaderElectionServiceTest {
+
+	private static final Random RND = new Random();
+
+	private final Executor executor = Executors.directExecutor();
+
+	// ------------------------------------------------------------------------
+
+	@Test
+	public void testStartStopAssignLeadership() throws Exception {
+		final UUID uuid = UUID.randomUUID();
+		final SingleLeaderElectionService service = new SingleLeaderElectionService(executor, uuid);
+
+		final LeaderContender contender = mockContender(service);
+		final LeaderContender otherContender = mockContender(service);
+
+		service.start(contender);
+		verify(contender, times(1)).grantLeadership(uuid);
+
+		service.stop();
+		verify(contender, times(1)).revokeLeadership();
+
+		// start with a new contender - the old contender must not gain another leadership
+		service.start(otherContender);
+		verify(otherContender, times(1)).grantLeadership(uuid);
+
+		verify(contender, times(1)).grantLeadership(uuid);
+		verify(contender, times(1)).revokeLeadership();
+	}
+
+	@Test
+	public void testStopBeforeConfirmingLeadership() throws Exception {
+		final UUID uuid = UUID.randomUUID();
+		final SingleLeaderElectionService service = new SingleLeaderElectionService(executor, uuid);
+
+		final LeaderContender contender = mock(LeaderContender.class);
+
+		service.start(contender);
+		verify(contender, times(1)).grantLeadership(uuid);
+
+		service.stop();
+
+		// because the leadership was never confirmed, there is no "revoke" call
+		verifyNoMoreInteractions(contender);
+	}
+
+	@Test
+	public void testStartOnlyOnce() throws Exception {
+		final UUID uuid = UUID.randomUUID();
+		final SingleLeaderElectionService service = new SingleLeaderElectionService(executor, uuid);
+
+		final LeaderContender contender = mock(LeaderContender.class);
+		final LeaderContender otherContender = mock(LeaderContender.class);
+
+		service.start(contender);
+		verify(contender, times(1)).grantLeadership(uuid);
+
+		// should not be possible to start again this with another contender
+		try {
+			service.start(otherContender);
+			fail("should fail with an exception");
+		} catch (IllegalStateException e) {
+			// expected
+		}
+
+		// should not be possible to start this again with the same contender
+		try {
+			service.start(contender);
+			fail("should fail with an exception");
+		} catch (IllegalStateException e) {
+			// expected
+		}
+	}
+
+	@Test
+	public void testShutdown() throws Exception {
+		final UUID uuid = UUID.randomUUID();
+		final SingleLeaderElectionService service = new SingleLeaderElectionService(executor, uuid);
+
+		// create a leader contender and let it grab leadership
+		final LeaderContender contender = mockContender(service);
+		service.start(contender);
+		verify(contender, times(1)).grantLeadership(uuid);
+
+		// some leader listeners
+		final LeaderRetrievalListener listener1 = mock(LeaderRetrievalListener.class);
+		final LeaderRetrievalListener listener2 = mock(LeaderRetrievalListener.class);
+
+		LeaderRetrievalService listenerService1 = service.createLeaderRetrievalService();
+		LeaderRetrievalService listenerService2 = service.createLeaderRetrievalService();
+
+		listenerService1.start(listener1);
+		listenerService2.start(listener2);
+
+		// one listener stops
+		listenerService1.stop();
+
+		// shut down the service
+		service.shutdown();
+
+		// the leader contender and running listener should get error notifications
+		verify(contender, times(1)).handleError(any(Exception.class));
+		verify(listener2, times(1)).handleError(any(Exception.class));
+
+		// the stopped listener gets no notification
+		verify(listener1, times(0)).handleError(any(Exception.class));
+
+		// should not be possible to start again after shutdown
+		try {
+			service.start(contender);
+			fail("should fail with an exception");
+		} catch (IllegalStateException e) {
+			// expected
+		}
+
+		// no additional leadership grant
+		verify(contender, times(1)).grantLeadership(any(UUID.class));
+	}
+
+	@Test
+	public void testImmediateShutdown() throws Exception {
+		final UUID uuid = UUID.randomUUID();
+		final SingleLeaderElectionService service = new SingleLeaderElectionService(executor, uuid);
+		service.shutdown();
+
+		final LeaderContender contender = mock(LeaderContender.class);
+		
+		// should not be possible to start
+		try {
+			service.start(contender);
+			fail("should fail with an exception");
+		} catch (IllegalStateException e) {
+			// expected
+		}
+
+		// no additional leadership grant
+		verify(contender, times(0)).grantLeadership(any(UUID.class));
+	}
+
+//	@Test
+//	public void testNotifyListenersWhenLeaderElected() throws Exception {
+//		final UUID uuid = UUID.randomUUID();
+//		final SingleLeaderElectionService service = new SingleLeaderElectionService(executor, uuid);
+//
+//		final LeaderRetrievalListener listener1 = mock(LeaderRetrievalListener.class);
+//		final LeaderRetrievalListener listener2 = mock(LeaderRetrievalListener.class);
+//
+//		LeaderRetrievalService listenerService1 = service.createLeaderRetrievalService();
+//		LeaderRetrievalService listenerService2 = service.createLeaderRetrievalService();
+//
+//		listenerService1.start(listener1);
+//		listenerService1.start(listener2);
+//
+//		final LeaderContender contender = mockContender(service);
+//		service.start(contender);
+//
+//		veri
+//	}
+
+	// ------------------------------------------------------------------------
+	//  utilities
+	// ------------------------------------------------------------------------
+
+	private static LeaderContender mockContender(final LeaderElectionService service) {
+		String address = StringUtils.getRandomString(RND, 5, 10, 'a', 'z');
+		return mockContender(service, address);
+	}
+
+	private static LeaderContender mockContender(final LeaderElectionService service, final String address) {
+		LeaderContender mockContender = mock(LeaderContender.class);
+
+		when(mockContender.getAddress()).thenReturn(address);
+
+		doAnswer(new Answer<Void>() {
+				@Override
+				public Void answer(InvocationOnMock invocation) throws Throwable {
+					final UUID uuid = (UUID) invocation.getArguments()[0];
+					service.confirmLeaderSessionID(uuid);
+					return null;
+				}
+		}).when(mockContender).grantLeadership(any(UUID.class));
+
+		return mockContender;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/2a7dbda7/flink-yarn/pom.xml
----------------------------------------------------------------------
diff --git a/flink-yarn/pom.xml b/flink-yarn/pom.xml
index 18ffe13..f69e0e4 100644
--- a/flink-yarn/pom.xml
+++ b/flink-yarn/pom.xml
@@ -32,6 +32,9 @@ under the License.
 	<packaging>jar</packaging>
 
 	<dependencies>
+
+		<!-- core dependencies -->
+
 		<dependency>
 			<groupId>org.apache.flink</groupId>
 			<artifactId>flink-runtime_2.10</artifactId>
@@ -91,6 +94,8 @@ under the License.
 			<scope>test</scope>
 		</dependency>
 
+		<!-- test dependencies -->
+
 		<dependency>
 			<groupId>org.apache.flink</groupId>
 			<artifactId>flink-runtime_2.10</artifactId>
@@ -98,6 +103,22 @@ under the License.
 			<type>test-jar</type>
 			<scope>test</scope>
 		</dependency>
+
+		<dependency>
+			<groupId>org.apache.hadoop</groupId>
+			<artifactId>hadoop-hdfs</artifactId>
+			<scope>test</scope>
+			<type>test-jar</type>
+			<version>${hadoop.version}</version>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.hadoop</groupId>
+			<artifactId>hadoop-common</artifactId>
+			<scope>test</scope>
+			<type>test-jar</type>
+			<version>${hadoop.version}</version>
+		</dependency>
 	</dependencies>
 
 	<build>

http://git-wip-us.apache.org/repos/asf/flink/blob/2a7dbda7/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnFlinkApplicationMasterRunner.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnFlinkApplicationMasterRunner.java b/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnFlinkApplicationMasterRunner.java
index 923694e..1c8bad7 100644
--- a/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnFlinkApplicationMasterRunner.java
+++ b/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnFlinkApplicationMasterRunner.java
@@ -23,17 +23,19 @@ import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.GlobalConfiguration;
 import org.apache.flink.configuration.HighAvailabilityOptions;
 import org.apache.flink.runtime.clusterframework.BootstrapTools;
-import org.apache.flink.runtime.security.SecurityContext;
+import org.apache.flink.runtime.security.SecurityUtils;
 import org.apache.flink.util.Preconditions;
 import org.apache.flink.yarn.cli.FlinkYarnSessionCli;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.api.ApplicationConstants.Environment;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.File;
 import java.util.Map;
+import java.util.concurrent.Callable;
 
 /**
  * This class is the executable entry point for the YARN application master.
@@ -95,7 +97,18 @@ public abstract class AbstractYarnFlinkApplicationMasterRunner {
 			LOG.info("YARN daemon is running as: {} Yarn client user obtainer: {}",
 					currentUser.getShortUserName(), yarnClientUsername );
 
-			SecurityContext.SecurityConfiguration sc = new SecurityContext.SecurityConfiguration();
+			// Flink configuration
+			final Map<String, String> dynamicProperties =
+					FlinkYarnSessionCli.getDynamicProperties(ENV.get(YarnConfigKeys.ENV_DYNAMIC_PROPERTIES));
+			LOG.debug("YARN dynamic properties: {}", dynamicProperties);
+
+			final Configuration flinkConfig = createConfiguration(currDir, dynamicProperties);
+			if(keytabPath != null && remoteKeytabPrincipal != null) {
+				flinkConfig.setString(ConfigConstants.SECURITY_KEYTAB_KEY, keytabPath);
+				flinkConfig.setString(ConfigConstants.SECURITY_PRINCIPAL_KEY, remoteKeytabPrincipal);
+			}
+
+			SecurityUtils.SecurityConfiguration sc = new SecurityUtils.SecurityConfiguration(flinkConfig);
 
 			//To support Yarn Secure Integration Test Scenario
 			File krb5Conf = new File(currDir, Utils.KRB5_FILE_NAME);
@@ -108,18 +121,7 @@ public abstract class AbstractYarnFlinkApplicationMasterRunner {
 				sc.setHadoopConfiguration(conf);
 			}
 
-			// Flink configuration
-			final Map<String, String> dynamicProperties =
-					FlinkYarnSessionCli.getDynamicProperties(ENV.get(YarnConfigKeys.ENV_DYNAMIC_PROPERTIES));
-			LOG.debug("YARN dynamic properties: {}", dynamicProperties);
-
-			final Configuration flinkConfig = createConfiguration(currDir, dynamicProperties);
-			if(keytabPath != null && remoteKeytabPrincipal != null) {
-				flinkConfig.setString(ConfigConstants.SECURITY_KEYTAB_KEY, keytabPath);
-				flinkConfig.setString(ConfigConstants.SECURITY_PRINCIPAL_KEY, remoteKeytabPrincipal);
-			}
-
-			SecurityContext.install(sc.setFlinkConfiguration(flinkConfig));
+			SecurityUtils.install(sc);
 
 			// Note that we use the "appMasterHostname" given by YARN here, to make sure
 			// we use the hostnames given by YARN consistently throughout akka.
@@ -129,9 +131,9 @@ public abstract class AbstractYarnFlinkApplicationMasterRunner {
 					"ApplicationMaster hostname variable %s not set", Environment.NM_HOST.key());
 			LOG.info("YARN assigned hostname for application master: {}", appMasterHostname);
 
-			return SecurityContext.getInstalled().runSecured(new SecurityContext.FlinkSecuredRunner<Integer>() {
+			return SecurityUtils.getInstalledContext().runSecured(new Callable<Integer>() {
 				@Override
-				public Integer run() {
+				public Integer call() throws Exception {
 					return runApplicationMaster(flinkConfig);
 				}
 			});

http://git-wip-us.apache.org/repos/asf/flink/blob/2a7dbda7/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 1826d43..d1ef553 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
@@ -115,7 +115,7 @@ public class YarnApplicationMasterRunner {
 	 * @param args The command line arguments.
 	 */
 	public static void main(String[] args) {
-		EnvironmentInformation.logEnvironmentInfo(LOG, "YARN ApplicationMaster / JobManager", args);
+		EnvironmentInformation.logEnvironmentInfo(LOG, "YARN ApplicationMaster / ResourceManager / JobManager", args);
 		SignalHandler.register(LOG);
 		JvmShutdownSafeguard.installAsShutdownHook(LOG);
 

http://git-wip-us.apache.org/repos/asf/flink/blob/2a7dbda7/flink-yarn/src/main/java/org/apache/flink/yarn/YarnFlinkApplicationMasterRunner.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnFlinkApplicationMasterRunner.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnFlinkApplicationMasterRunner.java
index e58c77e..188d9ef 100644
--- a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnFlinkApplicationMasterRunner.java
+++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnFlinkApplicationMasterRunner.java
@@ -37,7 +37,6 @@ import org.apache.flink.runtime.metrics.MetricRegistryConfiguration;
 import org.apache.flink.runtime.resourcemanager.JobLeaderIdService;
 import org.apache.flink.runtime.resourcemanager.ResourceManager;
 import org.apache.flink.runtime.resourcemanager.ResourceManagerConfiguration;
-import org.apache.flink.runtime.resourcemanager.exceptions.ConfigurationException;
 import org.apache.flink.runtime.resourcemanager.slotmanager.DefaultSlotManager;
 import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManagerFactory;
 import org.apache.flink.runtime.rpc.FatalErrorHandler;
@@ -57,7 +56,12 @@ import java.io.FileInputStream;
 import java.io.ObjectInputStream;
 
 /**
- * This class is the executable entry point for the YARN application master.
+ * This class is the executable entry point for the YARN Application Master that
+ * executes a single Flink job and then shuts the YARN application down.
+ * 
+ * <p>The lifetime of the YARN application bound to that of the Flink job. Other
+ * YARN Application Master implementations are for example the YARN session.
+ * 
  * It starts actor system and the actors for {@link org.apache.flink.runtime.jobmaster.JobManagerRunner}
  * and {@link org.apache.flink.yarn.YarnResourceManager}.
  *
@@ -74,6 +78,8 @@ public class YarnFlinkApplicationMasterRunner extends AbstractYarnFlinkApplicati
 	/** The job graph file path */
 	private static final String JOB_GRAPH_FILE_PATH = "flink.jobgraph.path";
 
+	// ------------------------------------------------------------------------
+
 	/** The lock to guard startup / shutdown / manipulation methods */
 	private final Object lock = new Object();
 
@@ -105,7 +111,7 @@ public class YarnFlinkApplicationMasterRunner extends AbstractYarnFlinkApplicati
 	 * @param args The command line arguments.
 	 */
 	public static void main(String[] args) {
-		EnvironmentInformation.logEnvironmentInfo(LOG, "YARN ApplicationMaster runner", args);
+		EnvironmentInformation.logEnvironmentInfo(LOG, "YARN ApplicationMaster / ResourceManager / JobManager", args);
 		SignalHandler.register(LOG);
 		JvmShutdownSafeguard.installAsShutdownHook(LOG);
 
@@ -127,7 +133,9 @@ public class YarnFlinkApplicationMasterRunner extends AbstractYarnFlinkApplicati
 					ConfigConstants.DEFAULT_YARN_JOB_MANAGER_PORT);
 
 			synchronized (lock) {
+				LOG.info("Starting High Availability Services");
 				haServices = HighAvailabilityServicesUtils.createAvailableOrEmbeddedServices(config);
+				
 				metricRegistry = new MetricRegistry(MetricRegistryConfiguration.fromConfiguration(config));
 				commonRpcService = createRpcService(config, appMasterHostname, amPortRange);
 
@@ -176,7 +184,7 @@ public class YarnFlinkApplicationMasterRunner extends AbstractYarnFlinkApplicati
 		return new AkkaRpcService(actorSystem, Time.of(duration.length(), duration.unit()));
 	}
 
-	private ResourceManager createResourceManager(Configuration config) throws ConfigurationException {
+	private ResourceManager<?> createResourceManager(Configuration config) throws Exception {
 		final ResourceManagerConfiguration resourceManagerConfiguration = ResourceManagerConfiguration.fromConfiguration(config);
 		final SlotManagerFactory slotManagerFactory = new DefaultSlotManager.Factory();
 		final JobLeaderIdService jobLeaderIdService = new JobLeaderIdService(haServices);
@@ -242,7 +250,7 @@ public class YarnFlinkApplicationMasterRunner extends AbstractYarnFlinkApplicati
 			}
 			if (haServices != null) {
 				try {
-					haServices.shutdown();
+					haServices.close();
 				} catch (Throwable tt) {
 					LOG.warn("Failed to stop the HA service", tt);
 				}

http://git-wip-us.apache.org/repos/asf/flink/blob/2a7dbda7/flink-yarn/src/main/java/org/apache/flink/yarn/YarnTaskExecutorRunner.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnTaskExecutorRunner.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnTaskExecutorRunner.java
index d9912eb..dc8c604 100644
--- a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnTaskExecutorRunner.java
+++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnTaskExecutorRunner.java
@@ -28,7 +28,7 @@ import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils;
 import org.apache.flink.runtime.metrics.MetricRegistry;
 import org.apache.flink.runtime.metrics.MetricRegistryConfiguration;
 import org.apache.flink.runtime.rpc.RpcService;
-import org.apache.flink.runtime.security.SecurityContext;
+import org.apache.flink.runtime.security.SecurityUtils;
 import org.apache.flink.runtime.taskexecutor.TaskManagerRunner;
 import org.apache.flink.runtime.util.EnvironmentInformation;
 import org.apache.flink.runtime.util.JvmShutdownSafeguard;
@@ -42,6 +42,7 @@ import org.slf4j.LoggerFactory;
 
 import java.io.File;
 import java.util.Map;
+import java.util.concurrent.Callable;
 
 /**
  * This class is the executable entry point for running a TaskExecutor in a YARN container.
@@ -138,7 +139,7 @@ public class YarnTaskExecutorRunner {
 			LOG.info("YARN daemon is running as: {} Yarn client user obtainer: {}",
 					currentUser.getShortUserName(), yarnClientUsername);
 
-			SecurityContext.SecurityConfiguration sc = new SecurityContext.SecurityConfiguration();
+			SecurityUtils.SecurityConfiguration sc = new SecurityUtils.SecurityConfiguration(configuration);
 
 			//To support Yarn Secure Integration Test Scenario
 			File krb5Conf = new File(currDir, Utils.KRB5_FILE_NAME);
@@ -156,11 +157,11 @@ public class YarnTaskExecutorRunner {
 				configuration.setString(ConfigConstants.SECURITY_PRINCIPAL_KEY, remoteKeytabPrincipal);
 			}
 
-			SecurityContext.install(sc.setFlinkConfiguration(configuration));
+			SecurityUtils.install(sc);
 
-			return SecurityContext.getInstalled().runSecured(new SecurityContext.FlinkSecuredRunner<Integer>() {
+			return SecurityUtils.getInstalledContext().runSecured(new Callable<Integer>() {
 				@Override
-				public Integer run() {
+				public Integer call() throws Exception {
 					return runTaskExecutor(configuration);
 				}
 			});
@@ -240,7 +241,7 @@ public class YarnTaskExecutorRunner {
 			}
 			if (haServices != null) {
 				try {
-					haServices.shutdown();
+					haServices.close();
 				} catch (Throwable tt) {
 					LOG.warn("Failed to stop the HA service", tt);
 				}

http://git-wip-us.apache.org/repos/asf/flink/blob/2a7dbda7/flink-yarn/src/main/java/org/apache/flink/yarn/configuration/YarnConfigOptions.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/configuration/YarnConfigOptions.java b/flink-yarn/src/main/java/org/apache/flink/yarn/configuration/YarnConfigOptions.java
new file mode 100644
index 0000000..c3902d3
--- /dev/null
+++ b/flink-yarn/src/main/java/org/apache/flink/yarn/configuration/YarnConfigOptions.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.yarn.configuration;
+
+import org.apache.flink.configuration.ConfigOption;
+
+import static org.apache.flink.configuration.ConfigOptions.key;
+
+/**
+ * This class holds configuration constants used by Flink's YARN runners.
+ * These options are not expected to be ever configured by users explicitly. 
+ */
+public class YarnConfigOptions {
+
+	/**
+	 * The hostname or address where the application master RPC system is listening.
+	 */
+	public static final ConfigOption<String> APP_MASTER_RPC_ADDRESS =
+			key("yarn.appmaster.rpc.address")
+			.noDefaultValue();
+
+	/**
+	 * The port where the application master RPC system is listening.
+	 */
+	public static final ConfigOption<Integer> APP_MASTER_RPC_PORT =
+			key("yarn.appmaster.rpc.address")
+			.defaultValue(-1);
+
+	// ------------------------------------------------------------------------
+
+	/** This class is not meant to be instantiated */
+	private YarnConfigOptions() {}
+}


[48/52] [abbrv] flink git commit: [FLINK-5239] [distributed coordination] RPC service properly unpacks 'InvocationTargetExceptions'

Posted by se...@apache.org.
[FLINK-5239] [distributed coordination] RPC service properly unpacks 'InvocationTargetExceptions'


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

Branch: refs/heads/master
Commit: 95c68299a6e2ca478e100602ad2c24b05059ad21
Parents: 10f7e86
Author: Stephan Ewen <se...@apache.org>
Authored: Fri Dec 2 18:49:21 2016 +0100
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Dec 23 20:54:27 2016 +0100

----------------------------------------------------------------------
 .../flink/runtime/rpc/akka/AkkaRpcActor.java    | 16 +++-
 .../runtime/rpc/akka/AkkaRpcActorTest.java      | 89 +++++++++++++++++++-
 2 files changed, 102 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/95c68299/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActor.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActor.java
index fe6b23b..264ba96 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActor.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActor.java
@@ -44,6 +44,7 @@ import org.slf4j.LoggerFactory;
 import scala.concurrent.duration.FiniteDuration;
 
 import java.io.IOException;
+import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
 import java.util.concurrent.Callable;
 import java.util.concurrent.TimeUnit;
@@ -180,8 +181,19 @@ class AkkaRpcActor<C extends RpcGateway, T extends RpcEndpoint<C>> extends Untyp
 				if (rpcMethod.getReturnType().equals(Void.TYPE)) {
 					// No return value to send back
 					rpcMethod.invoke(rpcEndpoint, rpcInvocation.getArgs());
-				} else {
-					Object result = rpcMethod.invoke(rpcEndpoint, rpcInvocation.getArgs());
+				}
+				else {
+					final Object result;
+					try {
+						result = rpcMethod.invoke(rpcEndpoint, rpcInvocation.getArgs());
+					}
+					catch (InvocationTargetException e) {
+						LOG.trace("Reporting back error thrown in remote procedure {}", rpcMethod, e);
+
+						// tell the sender about the failure
+						getSender().tell(new Status.Failure(e.getTargetException()), getSelf());
+						return;
+					}
 
 					if (result instanceof Future) {
 						final Future<?> future = (Future<?>) result;

http://git-wip-us.apache.org/repos/asf/flink/blob/95c68299/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorTest.java
index 760e1a7..c73240c 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorTest.java
@@ -22,6 +22,7 @@ import akka.actor.ActorSystem;
 import org.apache.flink.api.common.time.Time;
 import org.apache.flink.runtime.akka.AkkaUtils;
 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.rpc.RpcEndpoint;
 import org.apache.flink.runtime.rpc.RpcGateway;
@@ -30,6 +31,7 @@ import org.apache.flink.runtime.rpc.RpcService;
 import org.apache.flink.runtime.rpc.akka.exceptions.AkkaRpcException;
 import org.apache.flink.runtime.rpc.exceptions.RpcConnectionException;
 import org.apache.flink.util.TestLogger;
+
 import org.hamcrest.core.Is;
 import org.junit.AfterClass;
 import org.junit.Test;
@@ -86,7 +88,7 @@ public class AkkaRpcActorTest extends TestLogger {
 		Future<DummyRpcGateway> futureRpcGateway = akkaRpcService.connect("foobar", DummyRpcGateway.class);
 
 		try {
-			DummyRpcGateway gateway = futureRpcGateway.get(timeout.getSize(), timeout.getUnit());
+			futureRpcGateway.get(timeout.getSize(), timeout.getUnit());
 
 			fail("The rpc connection resolution should have failed.");
 		} catch (ExecutionException exception) {
@@ -192,6 +194,48 @@ public class AkkaRpcActorTest extends TestLogger {
 		terminationFuture.get();
 	}
 
+	@Test
+	public void testExceptionPropagation() throws Exception {
+		ExceptionalEndpoint rpcEndpoint = new ExceptionalEndpoint(akkaRpcService);
+		rpcEndpoint.start();
+
+		ExceptionalGateway rpcGateway = rpcEndpoint.getSelf();
+		Future<Integer> result = rpcGateway.doStuff();
+
+		try {
+			result.get(timeout.getSize(), timeout.getUnit());
+			fail("this should fail with an exception");
+		}
+		catch (ExecutionException e) {
+			Throwable cause = e.getCause();
+			assertEquals(RuntimeException.class, cause.getClass());
+			assertEquals("my super specific test exception", cause.getMessage());
+		}
+	}
+
+	@Test
+	public void testExceptionPropagationFuturePiping() throws Exception {
+		ExceptionalFutureEndpoint rpcEndpoint = new ExceptionalFutureEndpoint(akkaRpcService);
+		rpcEndpoint.start();
+
+		ExceptionalGateway rpcGateway = rpcEndpoint.getSelf();
+		Future<Integer> result = rpcGateway.doStuff();
+
+		try {
+			result.get(timeout.getSize(), timeout.getUnit());
+			fail("this should fail with an exception");
+		}
+		catch (ExecutionException e) {
+			Throwable cause = e.getCause();
+			assertEquals(Exception.class, cause.getClass());
+			assertEquals("some test", cause.getMessage());
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  Test Actors and Interfaces
+	// ------------------------------------------------------------------------
+
 	private interface DummyRpcGateway extends RpcGateway {
 		Future<Integer> foobar();
 	}
@@ -218,4 +262,47 @@ public class AkkaRpcActorTest extends TestLogger {
 			_foobar = value;
 		}
 	}
+
+	// ------------------------------------------------------------------------
+
+	private interface ExceptionalGateway extends RpcGateway {
+		Future<Integer> doStuff();
+	}
+
+	private static class ExceptionalEndpoint extends RpcEndpoint<ExceptionalGateway> {
+
+		protected ExceptionalEndpoint(RpcService rpcService) {
+			super(rpcService);
+		}
+
+		@RpcMethod
+		public int doStuff() {
+			throw new RuntimeException("my super specific test exception");
+		}
+	}
+
+	private static class ExceptionalFutureEndpoint extends RpcEndpoint<ExceptionalGateway> {
+
+		protected ExceptionalFutureEndpoint(RpcService rpcService) {
+			super(rpcService);
+		}
+
+		@RpcMethod
+		public Future<Integer> doStuff() {
+			final FlinkCompletableFuture<Integer> future = new FlinkCompletableFuture<>();
+
+			// complete the future slightly in the, well, future...
+			new Thread() {
+				@Override
+				public void run() {
+					try {
+						Thread.sleep(10);
+					} catch (InterruptedException ignored) {}
+					future.completeExceptionally(new Exception("some test"));
+				}
+			}.start();
+
+			return future;
+		}
+	}
 }


[41/52] [abbrv] flink git commit: [FLINK-4929] [yarn] Implement FLIP-6 YARN TaskExecutor Runner

Posted by se...@apache.org.
[FLINK-4929] [yarn] Implement FLIP-6 YARN TaskExecutor Runner

Summary: Implement FLIP-6 YARN TaskExecutor Runner

Test Plan: NA

Reviewers: biao.liub

Differential Revision: http://phabricator.taobao.net/D6564


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

Branch: refs/heads/master
Commit: 371997a81a7035b68cf47e7bca64ce01e3ac36a6
Parents: 0113e5a
Author: shuai.xus <sh...@alibaba-inc.com>
Authored: Wed Nov 23 18:00:07 2016 +0800
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Dec 23 20:54:27 2016 +0100

----------------------------------------------------------------------
 .../runtime/taskexecutor/TaskManagerRunner.java |   6 +
 .../flink/yarn/YarnTaskExecutorRunner.java      | 257 +++++++++++++++++++
 2 files changed, 263 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/371997a8/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java
index 1145a46..3500f6d 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java
@@ -23,6 +23,7 @@ import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.akka.AkkaUtils;
 import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.concurrent.Future;
 import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
 import org.apache.flink.runtime.metrics.MetricRegistry;
 import org.apache.flink.runtime.metrics.groups.TaskManagerMetricGroup;
@@ -151,6 +152,11 @@ public class TaskManagerRunner implements FatalErrorHandler {
 		}
 	}
 
+	// export the termination future for caller to know it is terminated
+	public Future<Void> getTerminationFuture() {
+		return taskManager.getTerminationFuture();
+	}
+
 	// --------------------------------------------------------------------------------------------
 	//  FatalErrorHandler methods
 	// --------------------------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/flink/blob/371997a8/flink-yarn/src/main/java/org/apache/flink/yarn/YarnTaskExecutorRunner.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnTaskExecutorRunner.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnTaskExecutorRunner.java
new file mode 100644
index 0000000..d9912eb
--- /dev/null
+++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnTaskExecutorRunner.java
@@ -0,0 +1,257 @@
+/*
+ * 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.yarn;
+
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.GlobalConfiguration;
+import org.apache.flink.core.fs.FileSystem;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils;
+import org.apache.flink.runtime.metrics.MetricRegistry;
+import org.apache.flink.runtime.metrics.MetricRegistryConfiguration;
+import org.apache.flink.runtime.rpc.RpcService;
+import org.apache.flink.runtime.security.SecurityContext;
+import org.apache.flink.runtime.taskexecutor.TaskManagerRunner;
+import org.apache.flink.runtime.util.EnvironmentInformation;
+import org.apache.flink.runtime.util.JvmShutdownSafeguard;
+import org.apache.flink.runtime.util.SignalHandler;
+import org.apache.flink.util.Preconditions;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.yarn.api.ApplicationConstants.Environment;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.util.Map;
+
+/**
+ * This class is the executable entry point for running a TaskExecutor in a YARN container.
+ */
+public class YarnTaskExecutorRunner {
+
+	/** Logger */
+	protected static final Logger LOG = LoggerFactory.getLogger(YarnTaskExecutorRunner.class);
+
+	/** The process environment variables */
+	private static final Map<String, String> ENV = System.getenv();
+
+	/** The exit code returned if the initialization of the yarn task executor runner failed */
+	private static final int INIT_ERROR_EXIT_CODE = 31;
+
+	private MetricRegistry metricRegistry;
+
+	private HighAvailabilityServices haServices;
+
+	private RpcService taskExecutorRpcService;
+
+	private TaskManagerRunner taskManagerRunner;
+
+	// ------------------------------------------------------------------------
+	//  Program entry point
+	// ------------------------------------------------------------------------
+
+	/**
+	 * The entry point for the YARN task executor runner.
+	 *
+	 * @param args The command line arguments.
+	 */
+	public static void main(String[] args) {
+		EnvironmentInformation.logEnvironmentInfo(LOG, "YARN TaskExecutor runner", args);
+		SignalHandler.register(LOG);
+		JvmShutdownSafeguard.installAsShutdownHook(LOG);
+
+		// run and exit with the proper return code
+		int returnCode = new YarnTaskExecutorRunner().run(args);
+		System.exit(returnCode);
+	}
+
+	/**
+	 * The instance entry point for the YARN task executor. Obtains user group
+	 * information and calls the main work method {@link #runTaskExecutor(org.apache.flink.configuration.Configuration)} as a
+	 * privileged action.
+	 *
+	 * @param args The command line arguments.
+	 * @return The process exit code.
+	 */
+	protected int run(String[] args) {
+		try {
+			LOG.debug("All environment variables: {}", ENV);
+
+			final String yarnClientUsername = ENV.get(YarnConfigKeys.ENV_HADOOP_USER_NAME);
+			final String localDirs = ENV.get(Environment.LOCAL_DIRS.key());
+			LOG.info("Current working/local Directory: {}", localDirs);
+
+			final String currDir = ENV.get(Environment.PWD.key());
+			LOG.info("Current working Directory: {}", currDir);
+
+			final String remoteKeytabPath = ENV.get(YarnConfigKeys.KEYTAB_PATH);
+			LOG.info("TM: remote keytab path obtained {}", remoteKeytabPath);
+
+			final String remoteKeytabPrincipal = ENV.get(YarnConfigKeys.KEYTAB_PRINCIPAL);
+			LOG.info("TM: remote keytab principal obtained {}", remoteKeytabPrincipal);
+
+			final Configuration configuration = GlobalConfiguration.loadConfiguration(currDir);
+			FileSystem.setDefaultScheme(configuration);
+
+			// configure local directory
+			String flinkTempDirs = configuration.getString(ConfigConstants.TASK_MANAGER_TMP_DIR_KEY, null);
+			if (flinkTempDirs == null) {
+				LOG.info("Setting directories for temporary file " + localDirs);
+				configuration.setString(ConfigConstants.TASK_MANAGER_TMP_DIR_KEY, localDirs);
+			}
+			else {
+				LOG.info("Overriding YARN's temporary file directories with those " +
+						"specified in the Flink config: " + flinkTempDirs);
+			}
+
+			// tell akka to die in case of an error
+			configuration.setBoolean(ConfigConstants.AKKA_JVM_EXIT_ON_FATAL_ERROR, true);
+
+			String keytabPath = null;
+			if(remoteKeytabPath != null) {
+				File f = new File(currDir, Utils.KEYTAB_FILE_NAME);
+				keytabPath = f.getAbsolutePath();
+				LOG.info("keytab path: {}", keytabPath);
+			}
+
+			UserGroupInformation currentUser = UserGroupInformation.getCurrentUser();
+
+			LOG.info("YARN daemon is running as: {} Yarn client user obtainer: {}",
+					currentUser.getShortUserName(), yarnClientUsername);
+
+			SecurityContext.SecurityConfiguration sc = new SecurityContext.SecurityConfiguration();
+
+			//To support Yarn Secure Integration Test Scenario
+			File krb5Conf = new File(currDir, Utils.KRB5_FILE_NAME);
+			if(krb5Conf.exists() && krb5Conf.canRead()) {
+				String krb5Path = krb5Conf.getAbsolutePath();
+				LOG.info("KRB5 Conf: {}", krb5Path);
+				org.apache.hadoop.conf.Configuration conf = new org.apache.hadoop.conf.Configuration();
+				conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION, "kerberos");
+				conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHORIZATION, "true");
+				sc.setHadoopConfiguration(conf);
+			}
+
+			if(keytabPath != null && remoteKeytabPrincipal != null) {
+				configuration.setString(ConfigConstants.SECURITY_KEYTAB_KEY, keytabPath);
+				configuration.setString(ConfigConstants.SECURITY_PRINCIPAL_KEY, remoteKeytabPrincipal);
+			}
+
+			SecurityContext.install(sc.setFlinkConfiguration(configuration));
+
+			return SecurityContext.getInstalled().runSecured(new SecurityContext.FlinkSecuredRunner<Integer>() {
+				@Override
+				public Integer run() {
+					return runTaskExecutor(configuration);
+				}
+			});
+
+		}
+		catch (Throwable t) {
+			// make sure that everything whatever ends up in the log
+			LOG.error("YARN Application Master initialization failed", t);
+			return INIT_ERROR_EXIT_CODE;
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  Core work method
+	// ------------------------------------------------------------------------
+
+	/**
+	 * The main work method, must run as a privileged action.
+	 *
+	 * @return The return code for the Java process.
+	 */
+	protected int runTaskExecutor(Configuration config) {
+
+		try {
+			// ---- (1) create common services
+			// first get the ResouceId, resource id is the container id for yarn.
+			final String containerId = ENV.get(YarnFlinkResourceManager.ENV_FLINK_CONTAINER_ID);
+			Preconditions.checkArgument(containerId != null,
+					"ContainerId variable %s not set", YarnFlinkResourceManager.ENV_FLINK_CONTAINER_ID);
+			// use the hostname passed by job manager
+			final String taskExecutorHostname = ENV.get(YarnResourceManager.ENV_FLINK_NODE_ID);
+			if (taskExecutorHostname != null) {
+				config.setString(ConfigConstants.TASK_MANAGER_HOSTNAME_KEY, taskExecutorHostname);
+			}
+
+			ResourceID resourceID = new ResourceID(containerId);
+			LOG.info("YARN assigned resource id {} for the task executor.", resourceID.toString());
+
+			haServices = HighAvailabilityServicesUtils.createAvailableOrEmbeddedServices(config);
+			metricRegistry = new MetricRegistry(MetricRegistryConfiguration.fromConfiguration(config));
+
+			// ---- (2) init task manager runner -------
+			taskExecutorRpcService = TaskManagerRunner.createRpcService(config, haServices);
+			taskManagerRunner = new TaskManagerRunner(config, resourceID, taskExecutorRpcService, haServices, metricRegistry);
+
+			// ---- (3) start the task manager runner
+			taskManagerRunner.start();
+			LOG.debug("YARN task executor started");
+
+			taskManagerRunner.getTerminationFuture().get();
+			// everything started, we can wait until all is done or the process is killed
+			LOG.info("YARN task manager runner finished");
+			shutdown();
+		}
+		catch (Throwable t) {
+			// make sure that everything whatever ends up in the log
+			LOG.error("YARN task executor initialization failed", t);
+			shutdown();
+			return INIT_ERROR_EXIT_CODE;
+		}
+
+		return 0;
+	}
+
+	// ------------------------------------------------------------------------
+	//  Utilities
+	// ------------------------------------------------------------------------
+
+
+	protected void shutdown() {
+			if (taskExecutorRpcService != null) {
+				try {
+					taskExecutorRpcService.stopService();
+				} catch (Throwable tt) {
+					LOG.error("Error shutting down job master rpc service", tt);
+				}
+			}
+			if (haServices != null) {
+				try {
+					haServices.shutdown();
+				} catch (Throwable tt) {
+					LOG.warn("Failed to stop the HA service", tt);
+				}
+			}
+			if (metricRegistry != null) {
+				try {
+					metricRegistry.shutdown();
+				} catch (Throwable tt) {
+					LOG.warn("Failed to stop the metrics registry", tt);
+				}
+			}
+	}
+
+}


[42/52] [abbrv] flink git commit: [FLINK-5254] [yarn] Implement YARN High-Availability Services

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/2a7dbda7/flink-yarn/src/main/java/org/apache/flink/yarn/highavailability/AbstractYarnNonHaServices.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/highavailability/AbstractYarnNonHaServices.java b/flink-yarn/src/main/java/org/apache/flink/yarn/highavailability/AbstractYarnNonHaServices.java
new file mode 100644
index 0000000..7aa481f
--- /dev/null
+++ b/flink-yarn/src/main/java/org/apache/flink/yarn/highavailability/AbstractYarnNonHaServices.java
@@ -0,0 +1,105 @@
+/*
+ * 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.yarn.highavailability;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory;
+import org.apache.flink.runtime.checkpoint.StandaloneCheckpointRecoveryFactory;
+import org.apache.flink.runtime.highavailability.FsNegativeRunningJobsRegistry;
+import org.apache.flink.runtime.highavailability.RunningJobsRegistry;
+import org.apache.flink.runtime.jobmanager.SubmittedJobGraphStore;
+
+import java.io.IOException;
+
+/**
+ * Abstract base class for the high availability services for Flink YARN applications that support
+ * no master fail over.
+ *
+ * <p>Internally, these services put their recovery data into YARN's working directory,
+ * except for checkpoints, which are in the configured checkpoint directory. That way,
+ * checkpoints can be resumed with a new job/application, even if the complete YARN application
+ * is killed and cleaned up.
+ */
+public abstract class AbstractYarnNonHaServices extends YarnHighAvailabilityServices {
+
+	/** The constant name of the ResourceManager RPC endpoint */
+	protected static final String RESOURCE_MANAGER_RPC_ENDPOINT_NAME = "resource_manager";
+
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Creates new YARN high-availability services, configuring the file system and recovery
+	 * data directory based on the working directory in the given Hadoop configuration.
+	 *
+	 * <p>This class requires that the default Hadoop file system configured in the given
+	 * Hadoop configuration is an HDFS.
+	 *
+	 * @param config     The Flink configuration of this component / process.
+	 * @param hadoopConf The Hadoop configuration for the YARN cluster.
+	 *
+	 * @throws IOException Thrown, if the initialization of the Hadoop file system used by YARN fails.
+	 */
+	protected AbstractYarnNonHaServices(
+			Configuration config,
+			org.apache.hadoop.conf.Configuration hadoopConf) throws IOException {
+		super(config, hadoopConf);
+	}
+
+	// ------------------------------------------------------------------------
+	//  Names
+	// ------------------------------------------------------------------------
+
+	@Override
+	public String getResourceManagerEndpointName() {
+		return RESOURCE_MANAGER_RPC_ENDPOINT_NAME;
+	}
+
+	// ------------------------------------------------------------------------
+	//  Services
+	// ------------------------------------------------------------------------
+
+	@Override
+	public RunningJobsRegistry getRunningJobsRegistry() throws IOException {
+		enter();
+		try {
+			// IMPORTANT: The registry must NOT place its data in a directory that is
+			// cleaned up by these services.
+			return new FsNegativeRunningJobsRegistry(flinkFileSystem, workingDirectory);
+		}
+		finally {
+			exit();
+		}
+	}
+
+	@Override
+	public CheckpointRecoveryFactory getCheckpointRecoveryFactory() {
+		enter();
+		try {
+			return new StandaloneCheckpointRecoveryFactory();
+		}
+		finally {
+			exit();
+		}
+	}
+
+	@Override
+	public SubmittedJobGraphStore getSubmittedJobGraphStore() throws Exception {
+		throw new UnsupportedOperationException("These High-Availability Services do not support storing job graphs");
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/2a7dbda7/flink-yarn/src/main/java/org/apache/flink/yarn/highavailability/YarnHighAvailabilityServices.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/highavailability/YarnHighAvailabilityServices.java b/flink-yarn/src/main/java/org/apache/flink/yarn/highavailability/YarnHighAvailabilityServices.java
new file mode 100644
index 0000000..4c78726
--- /dev/null
+++ b/flink-yarn/src/main/java/org/apache/flink/yarn/highavailability/YarnHighAvailabilityServices.java
@@ -0,0 +1,343 @@
+/*
+ * 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.yarn.highavailability;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.IllegalConfigurationException;
+import org.apache.flink.core.fs.FileSystem;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.runtime.blob.BlobStore;
+import org.apache.flink.runtime.blob.FileSystemBlobStore;
+import org.apache.flink.runtime.fs.hdfs.HadoopFileSystem;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
+import org.apache.flink.runtime.jobmanager.HighAvailabilityMode;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.InstantiationUtil;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.URI;
+import java.util.concurrent.locks.ReentrantLock;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.ExceptionUtils.firstOrSuppressed;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * The basis of {@link HighAvailabilityServices} for YARN setups.
+ * These high-availability services auto-configure YARN's HDFS and the YARN application's
+ * working directory to be used to store job recovery data.
+ * 
+ * <p>Note for implementers: This class locks access to and creation of services,
+ * to make sure all services are properly shut down when shutting down this class.
+ * To participate in the checks, overriding methods should frame method body with
+ * calls to {@code enter()} and {@code exit()} as shown in the following pattern:
+ * 
+ * <pre>{@code
+ * public LeaderRetrievalService getResourceManagerLeaderRetriever() {
+ *     enter();
+ *     try {
+ *         CuratorClient client = getCuratorClient();
+ *         return ZooKeeperUtils.createLeaderRetrievalService(client, configuration, RESOURCE_MANAGER_LEADER_PATH);
+ *     } finally {
+ *         exit();
+ *     }
+ * }
+ * }</pre>
+ */
+public abstract class YarnHighAvailabilityServices implements HighAvailabilityServices {
+
+	/** The name of the sub directory in which Flink stores the recovery data */
+	public static final String FLINK_RECOVERY_DATA_DIR = "flink_recovery_data";
+
+	/** Logger for these services, shared with subclasses */
+	protected static final Logger LOG = LoggerFactory.getLogger(YarnHighAvailabilityServices.class);
+
+	// ------------------------------------------------------------------------
+
+	/** The lock that guards all accesses to methods in this class */
+	private final ReentrantLock lock;
+
+	/** The Flink FileSystem object that represent the HDFS used by YARN */
+	protected final FileSystem flinkFileSystem;
+
+	/** The Hadoop FileSystem object that represent the HDFS used by YARN */
+	protected final org.apache.hadoop.fs.FileSystem hadoopFileSystem;
+
+	/** The working directory of this YARN application.
+	 * This MUST NOT be deleted when the HA services clean up */
+	protected final Path workingDirectory;
+
+	/** The directory for HA persistent data. This should be deleted when the
+	 * HA services clean up */
+	protected final Path haDataDirectory;
+
+	/** Flag marking this instance as shut down */
+	private volatile boolean closed;
+
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Creates new YARN high-availability services, configuring the file system and recovery
+	 * data directory based on the working directory in the given Hadoop configuration.
+	 * 
+	 * <p>This class requires that the default Hadoop file system configured in the given
+	 * Hadoop configuration is an HDFS.
+	 * 
+	 * @param config     The Flink configuration of this component / process.
+	 * @param hadoopConf The Hadoop configuration for the YARN cluster.
+	 * 
+	 * @throws IOException Thrown, if the initialization of the Hadoop file system used by YARN fails.
+	 */
+	protected YarnHighAvailabilityServices(
+			Configuration config,
+			org.apache.hadoop.conf.Configuration hadoopConf) throws IOException {
+
+		checkNotNull(config);
+		checkNotNull(hadoopConf);
+
+		this.lock = new ReentrantLock();
+
+		// get and verify the YARN HDFS URI
+		final URI fsUri = org.apache.hadoop.fs.FileSystem.getDefaultUri(hadoopConf);
+		if (fsUri.getScheme() == null || !"hdfs".equals(fsUri.getScheme().toLowerCase())) {
+			throw new IOException("Invalid file system found for YarnHighAvailabilityServices: " +
+					"Expected 'hdfs', but found '" + fsUri.getScheme() + "'.");
+		}
+
+		// initialize the Hadoop File System
+		// we go through this special code path here to make sure we get no shared cached
+		// instance of the FileSystem
+		try {
+			final Class<? extends org.apache.hadoop.fs.FileSystem> fsClass =
+					org.apache.hadoop.fs.FileSystem.getFileSystemClass(fsUri.getScheme(), hadoopConf);
+
+			this.hadoopFileSystem = InstantiationUtil.instantiate(fsClass);
+			this.hadoopFileSystem.initialize(fsUri, hadoopConf);
+		}
+		catch (Exception e) {
+			throw new IOException("Cannot instantiate YARN's Hadoop file system for " + fsUri, e);
+		}
+
+		this.flinkFileSystem = new HadoopFileSystem(hadoopConf, hadoopFileSystem);
+
+		this.workingDirectory = new Path(hadoopFileSystem.getWorkingDirectory().toUri());
+		this.haDataDirectory = new Path(workingDirectory, FLINK_RECOVERY_DATA_DIR);
+
+		// test the file system, to make sure we fail fast if access does not work
+		try {
+			flinkFileSystem.mkdirs(haDataDirectory);
+		}
+		catch (Exception e) {
+			throw new IOException("Could not create the directory for recovery data in YARN's file system at '"
+					+ haDataDirectory + "'.", e);
+		}
+
+		LOG.info("Flink YARN application will store recovery data at {}", haDataDirectory);
+	}
+
+	// ------------------------------------------------------------------------
+	//  high availability services
+	// ------------------------------------------------------------------------
+
+	@Override
+	public BlobStore createBlobStore() throws IOException {
+		enter();
+		try {
+			return new FileSystemBlobStore(flinkFileSystem, haDataDirectory.toString());
+		} finally {
+			exit();
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  Shutdown
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Checks whether these services have been shut down.
+	 *
+	 * @return True, if this instance has been shut down, false if it still operational.
+	 */
+	public boolean isClosed() {
+		return closed;
+	}
+
+	@Override
+	public void close() throws Exception {
+		lock.lock();
+		try {
+			// close only once
+			if (closed) {
+				return;
+			}
+			closed = true;
+
+			// we do not propagate exceptions here, but only log them
+			try {
+				hadoopFileSystem.close();
+			} catch (Throwable t) {
+				LOG.warn("Error closing Hadoop FileSystem", t);
+			}
+		}
+		finally {
+			lock.unlock();
+		}
+	}
+
+	@Override
+	public void closeAndCleanupAllData() throws Exception {
+		lock.lock();
+		try {
+			checkState(!closed, "YarnHighAvailabilityServices are already closed");
+
+			// we remember exceptions only, then continue cleanup, and re-throw at the end
+			Throwable exception = null;
+
+			// first, we delete all data in Flink's data directory
+			try {
+				flinkFileSystem.delete(haDataDirectory, true);
+			}
+			catch (Throwable t) {
+				exception = t;
+			}
+
+			// now we actually close the services
+			try {
+				close();
+			}
+			catch (Throwable t) {
+				exception = firstOrSuppressed(t, exception);
+			}
+
+			// if some exception occurred, rethrow
+			if (exception != null) {
+				ExceptionUtils.rethrowException(exception, exception.getMessage());
+			}
+		}
+		finally {
+			lock.unlock();
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  Utilities
+	// ------------------------------------------------------------------------
+
+	/**
+	 * To be called at the beginning of every method that creates an HA service. Acquires the lock
+	 * and check whether this HighAvailabilityServices instance is shut down.
+	 */
+	void enter() {
+		if (!enterUnlessClosed()) {
+			throw new IllegalStateException("closed");
+		}
+	}
+
+	/**
+	 * Acquires the lock and checks whether the services are already closed. If they are
+	 * already closed, the method releases the lock and returns {@code false}.
+	 * 
+	 * @return True, if the lock was acquired and the services are not closed, false if the services are closed.
+	 */
+	boolean enterUnlessClosed() {
+		lock.lock();
+		if (!closed) {
+			return true;
+		} else {
+			lock.unlock();
+			return false;
+		}
+	}
+
+	/**
+	 * To be called at the end of every method that creates an HA service. Releases the lock.
+	 */
+	void exit() {
+		lock.unlock();
+	}
+
+	// ------------------------------------------------------------------------
+	//  Factory from Configuration
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Creates the high-availability services for a single-job Flink YARN application, to be
+	 * used in the Application Master that runs both ResourceManager and JobManager.
+	 * 
+	 * @param flinkConfig  The Flink configuration.
+	 * @param hadoopConfig The Hadoop configuration for the YARN cluster.
+	 * 
+	 * @return The created high-availability services.
+	 * 
+	 * @throws IOException Thrown, if the high-availability services could not be initialized.
+	 */
+	public static YarnHighAvailabilityServices forSingleJobAppMaster(
+			Configuration flinkConfig,
+			org.apache.hadoop.conf.Configuration hadoopConfig) throws IOException {
+
+		checkNotNull(flinkConfig, "flinkConfig");
+		checkNotNull(hadoopConfig, "hadoopConfig");
+
+		final HighAvailabilityMode mode = HighAvailabilityMode.fromConfig(flinkConfig);
+		switch (mode) {
+			case NONE:
+				return new YarnIntraNonHaMasterServices(flinkConfig, hadoopConfig);
+
+			case ZOOKEEPER:
+				throw  new UnsupportedOperationException("to be implemented");
+
+			default:
+				throw new IllegalConfigurationException("Unrecognized high availability mode: " + mode);
+		}
+	}
+
+	/**
+	 * Creates the high-availability services for the TaskManagers participating in
+	 * a Flink YARN application.
+	 *
+	 * @param flinkConfig  The Flink configuration.
+	 * @param hadoopConfig The Hadoop configuration for the YARN cluster.
+	 *
+	 * @return The created high-availability services.
+	 *
+	 * @throws IOException Thrown, if the high-availability services could not be initialized.
+	 */
+	public static YarnHighAvailabilityServices forYarnTaskManager(
+			Configuration flinkConfig,
+			org.apache.hadoop.conf.Configuration hadoopConfig) throws IOException {
+
+		checkNotNull(flinkConfig, "flinkConfig");
+		checkNotNull(hadoopConfig, "hadoopConfig");
+
+		final HighAvailabilityMode mode = HighAvailabilityMode.fromConfig(flinkConfig);
+		switch (mode) {
+			case NONE:
+				return new YarnPreConfiguredMasterNonHaServices(flinkConfig, hadoopConfig);
+
+			case ZOOKEEPER:
+				throw  new UnsupportedOperationException("to be implemented");
+
+			default:
+				throw new IllegalConfigurationException("Unrecognized high availability mode: " + mode);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/2a7dbda7/flink-yarn/src/main/java/org/apache/flink/yarn/highavailability/YarnIntraNonHaMasterServices.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/highavailability/YarnIntraNonHaMasterServices.java b/flink-yarn/src/main/java/org/apache/flink/yarn/highavailability/YarnIntraNonHaMasterServices.java
new file mode 100644
index 0000000..fd1a45e
--- /dev/null
+++ b/flink-yarn/src/main/java/org/apache/flink/yarn/highavailability/YarnIntraNonHaMasterServices.java
@@ -0,0 +1,188 @@
+/*
+ * 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.yarn.highavailability;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.IllegalConfigurationException;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
+import org.apache.flink.runtime.highavailability.ServicesThreadFactory;
+import org.apache.flink.runtime.highavailability.leaderelection.SingleLeaderElectionService;
+import org.apache.flink.runtime.leaderelection.LeaderElectionService;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+
+import java.io.IOException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+
+/**
+ * These YarnHighAvailabilityServices are for the Application Master in setups where there is one
+ * ResourceManager that is statically configured in the Flink configuration.
+ * 
+ * <h3>Handled failure types</h3>
+ * <ul>
+ *     <li><b>User code & operator failures:</b> Failed operators are recovered from checkpoints.</li>
+ *     <li><b>Task Manager Failures:</b> Failed Task Managers are restarted and their tasks are
+ *         recovered from checkpoints.</li>
+ * </ul>
+ *
+ * <h3>Non-recoverable failure types</h3>
+ * <ul>
+ *     <li><b>Application Master failures:</b> These failures cannot be recovered, because TaskManagers
+ *     have no way to discover the new Application Master's address.</li>
+ * </ul>
+ *
+ * <p>Internally, these services put their recovery data into YARN's working directory,
+ * except for checkpoints, which are in the configured checkpoint directory. That way,
+ * checkpoints can be resumed with a new job/application, even if the complete YARN application
+ * is killed and cleaned up. 
+ *
+ * <p>Because ResourceManager and JobManager run both in the same process (Application Master), they
+ * use an embedded leader election service to find each other.
+ * 
+ * <p>A typical YARN setup that uses these HA services first starts the ResourceManager
+ * inside the ApplicationMaster and puts its RPC endpoint address into the configuration with which
+ * the TaskManagers are started. Because of this static addressing scheme, the setup cannot handle failures
+ * of the JobManager and ResourceManager, which are running as part of the Application Master.
+ *
+ * @see HighAvailabilityServices
+ */
+public class YarnIntraNonHaMasterServices extends AbstractYarnNonHaServices {
+
+	/** The dispatcher thread pool for these services */
+	private final ExecutorService dispatcher;
+
+	/** The embedded leader election service used by JobManagers to find the resource manager */
+	private final SingleLeaderElectionService resourceManagerLeaderElectionService;
+
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Creates new YarnIntraNonHaMasterServices for the given Flink and YARN configuration.
+	 * 
+	 * This constructor initializes access to the HDFS to store recovery data, and creates the
+	 * embedded leader election services through which ResourceManager and JobManager find and
+	 * confirm each other.
+	 * 
+	 * @param config     The Flink configuration of this component / process.
+	 * @param hadoopConf The Hadoop configuration for the YARN cluster.
+	 *
+	 * @throws IOException
+	 *             Thrown, if the initialization of the Hadoop file system used by YARN fails.
+	 * @throws IllegalConfigurationException
+	 *             Thrown, if the Flink configuration does not properly describe the ResourceManager address and port.
+	 */
+	public YarnIntraNonHaMasterServices(
+			Configuration config,
+			org.apache.hadoop.conf.Configuration hadoopConf) throws IOException {
+
+		super(config, hadoopConf);
+
+		// track whether we successfully perform the initialization
+		boolean successful = false;
+
+		try {
+			this.dispatcher = Executors.newSingleThreadExecutor(new ServicesThreadFactory());
+			this.resourceManagerLeaderElectionService = new SingleLeaderElectionService(dispatcher, DEFAULT_LEADER_ID);
+
+			// all good!
+			successful = true;
+		}
+		finally {
+			if (!successful) {
+				// quietly undo what the parent constructor initialized
+				try {
+					super.close();
+				} catch (Throwable ignored) {}
+			}
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  Services
+	// ------------------------------------------------------------------------
+
+	@Override
+	public LeaderRetrievalService getResourceManagerLeaderRetriever() {
+		enter();
+		try {
+			return resourceManagerLeaderElectionService.createLeaderRetrievalService();
+		}
+		finally {
+			exit();
+		}
+	}
+
+	@Override
+	public LeaderElectionService getResourceManagerLeaderElectionService() {
+		enter();
+		try {
+			return resourceManagerLeaderElectionService;
+		}
+		finally {
+			exit();
+		}
+	}
+
+	@Override
+	public LeaderElectionService getJobManagerLeaderElectionService(JobID jobID) {
+		enter();
+		try {
+			throw new UnsupportedOperationException("needs refactoring to accept default address");
+		}
+		finally {
+			exit();
+		}
+	}
+
+	@Override
+	public LeaderRetrievalService getJobManagerLeaderRetriever(JobID jobID) {
+		enter();
+		try {
+			throw new UnsupportedOperationException("needs refactoring to accept default address");
+		}
+		finally {
+			exit();
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  shutdown
+	// ------------------------------------------------------------------------
+
+	@Override
+	public void close() throws Exception {
+		if (enterUnlessClosed()) {
+			try {
+				try {
+					// this class' own cleanup logic
+					resourceManagerLeaderElectionService.shutdown();
+					dispatcher.shutdownNow();
+				}
+				finally {
+					// in any case must we call the parent cleanup logic
+					super.close();
+				}
+			}
+			finally {
+				exit();
+			}
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/2a7dbda7/flink-yarn/src/main/java/org/apache/flink/yarn/highavailability/YarnPreConfiguredMasterNonHaServices.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/highavailability/YarnPreConfiguredMasterNonHaServices.java b/flink-yarn/src/main/java/org/apache/flink/yarn/highavailability/YarnPreConfiguredMasterNonHaServices.java
new file mode 100644
index 0000000..eb4b77e
--- /dev/null
+++ b/flink-yarn/src/main/java/org/apache/flink/yarn/highavailability/YarnPreConfiguredMasterNonHaServices.java
@@ -0,0 +1,172 @@
+/*
+ * 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.yarn.highavailability;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.IllegalConfigurationException;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
+import org.apache.flink.runtime.leaderelection.LeaderElectionService;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+import org.apache.flink.runtime.leaderretrieval.StandaloneLeaderRetrievalService;
+import org.apache.flink.runtime.rpc.RpcServiceUtils;
+import org.apache.flink.yarn.configuration.YarnConfigOptions;
+
+import java.io.IOException;
+
+/**
+ * These YarnHighAvailabilityServices are for use by the TaskManager in setups,
+ * where there is one ResourceManager that is statically configured in the Flink configuration.
+ * 
+ * <h3>Handled failure types</h3>
+ * <ul>
+ *     <li><b>User code & operator failures:</b> Failed operators are recovered from checkpoints.</li>
+ *     <li><b>Task Manager Failures:</b> Failed Task Managers are restarted and their tasks are
+ *         recovered from checkpoints.</li>
+ * </ul>
+ *
+ * <h3>Non-recoverable failure types</h3>
+ * <ul>
+ *     <li><b>Application Master failures:</b> These failures cannot be recovered, because TaskManagers
+ *     have no way to discover the new Application Master's address.</li>
+ * </ul>
+ *
+ * <p>Internally, these services put their recovery data into YARN's working directory,
+ * except for checkpoints, which are in the configured checkpoint directory. That way,
+ * checkpoints can be resumed with a new job/application, even if the complete YARN application
+ * is killed and cleaned up. 
+ *
+ * <p>A typical YARN setup that uses these HA services first starts the ResourceManager
+ * inside the ApplicationMaster and puts its RPC endpoint address into the configuration with which
+ * the TaskManagers are started. Because of this static addressing scheme, the setup cannot handle failures
+ * of the JobManager and ResourceManager, which are running as part of the Application Master.
+ *
+ * @see HighAvailabilityServices
+ */
+public class YarnPreConfiguredMasterNonHaServices extends AbstractYarnNonHaServices {
+
+	/** The RPC URL under which the single ResourceManager can be reached while available */ 
+	private final String resourceManagerRpcUrl;
+
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Creates new YarnPreConfiguredMasterHaServices for the given Flink and YARN configuration.
+	 * This constructor parses the ResourceManager address from the Flink configuration and sets
+	 * up the HDFS access to store recovery data in the YARN application's working directory.
+	 * 
+	 * @param config     The Flink configuration of this component / process.
+	 * @param hadoopConf The Hadoop configuration for the YARN cluster.
+	 *
+	 * @throws IOException
+	 *             Thrown, if the initialization of the Hadoop file system used by YARN fails.
+	 * @throws IllegalConfigurationException
+	 *             Thrown, if the Flink configuration does not properly describe the ResourceManager address and port.
+	 */
+	public YarnPreConfiguredMasterNonHaServices(
+			Configuration config,
+			org.apache.hadoop.conf.Configuration hadoopConf) throws IOException {
+
+		super(config, hadoopConf);
+
+		// track whether we successfully perform the initialization
+		boolean successful = false;
+
+		try {
+			// extract the hostname and port of the resource manager
+			final String rmHost = config.getString(YarnConfigOptions.APP_MASTER_RPC_ADDRESS);
+			final int rmPort = config.getInteger(YarnConfigOptions.APP_MASTER_RPC_PORT);
+
+			if (rmHost == null) {
+				throw new IllegalConfigurationException("Config parameter '" + 
+						YarnConfigOptions.APP_MASTER_RPC_ADDRESS.key() + "' is missing.");
+			}
+			if (rmPort < 0) {
+				throw new IllegalConfigurationException("Config parameter '" +
+						YarnConfigOptions.APP_MASTER_RPC_PORT.key() + "' is missing.");
+			}
+			if (rmPort <= 0 || rmPort >= 65536) {
+				throw new IllegalConfigurationException("Invalid value for '" + 
+						YarnConfigOptions.APP_MASTER_RPC_PORT.key() + "' - port must be in [1, 65535]");
+			}
+
+			this.resourceManagerRpcUrl = RpcServiceUtils.getRpcUrl(
+					rmHost, rmPort, RESOURCE_MANAGER_RPC_ENDPOINT_NAME, config);
+
+			// all well!
+			successful = true;
+		}
+		finally {
+			if (!successful) {
+				// quietly undo what the parent constructor initialized
+				try {
+					super.close();
+				} catch (Throwable ignored) {}
+			}
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  Services
+	// ------------------------------------------------------------------------
+
+	@Override
+	public LeaderRetrievalService getResourceManagerLeaderRetriever() {
+		enter();
+		try {
+			return new StandaloneLeaderRetrievalService(resourceManagerRpcUrl, DEFAULT_LEADER_ID);
+		}
+		finally {
+			exit();
+		}
+	}
+
+	@Override
+	public LeaderElectionService getResourceManagerLeaderElectionService() {
+		enter();
+		try {
+			throw new UnsupportedOperationException("Not supported on the TaskManager side");
+		}
+		finally {
+			exit();
+		}
+	}
+
+	@Override
+	public LeaderElectionService getJobManagerLeaderElectionService(JobID jobID) {
+		enter();
+		try {
+			throw new UnsupportedOperationException("needs refactoring to accept default address");
+		}
+		finally {
+			exit();
+		}
+	}
+
+	@Override
+	public LeaderRetrievalService getJobManagerLeaderRetriever(JobID jobID) {
+		enter();
+		try {
+			throw new UnsupportedOperationException("needs refactoring to accept default address");
+		}
+		finally {
+			exit();
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/2a7dbda7/flink-yarn/src/test/java/org/apache/flink/yarn/highavailability/YarnIntraNonHaMasterServicesTest.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/test/java/org/apache/flink/yarn/highavailability/YarnIntraNonHaMasterServicesTest.java b/flink-yarn/src/test/java/org/apache/flink/yarn/highavailability/YarnIntraNonHaMasterServicesTest.java
new file mode 100644
index 0000000..0e7bf0f
--- /dev/null
+++ b/flink-yarn/src/test/java/org/apache/flink/yarn/highavailability/YarnIntraNonHaMasterServicesTest.java
@@ -0,0 +1,149 @@
+/*
+ * 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.yarn.highavailability;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.runtime.leaderelection.LeaderContender;
+import org.apache.flink.runtime.leaderelection.LeaderElectionService;
+import org.apache.flink.util.StringUtils;
+
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+import java.io.File;
+import java.util.Random;
+import java.util.UUID;
+
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.timeout;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+public class YarnIntraNonHaMasterServicesTest {
+
+	private static final Random RND = new Random();
+
+	@ClassRule
+	public static final TemporaryFolder TEMP_DIR = new TemporaryFolder();
+
+	private static MiniDFSCluster HDFS_CLUSTER;
+
+	private static Path HDFS_ROOT_PATH;
+
+	private org.apache.hadoop.conf.Configuration hadoopConfig;
+
+	// ------------------------------------------------------------------------
+	//  Test setup and shutdown
+	// ------------------------------------------------------------------------
+
+	@BeforeClass
+	public static void createHDFS() throws Exception {
+		final File tempDir = TEMP_DIR.newFolder();
+
+		org.apache.hadoop.conf.Configuration hdConf = new org.apache.hadoop.conf.Configuration();
+		hdConf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, tempDir.getAbsolutePath());
+
+		MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(hdConf);
+		HDFS_CLUSTER = builder.build();
+		HDFS_ROOT_PATH = new Path(HDFS_CLUSTER.getURI());
+	}
+
+	@AfterClass
+	public static void destroyHDFS() {
+		if (HDFS_CLUSTER != null) {
+			HDFS_CLUSTER.shutdown();
+		}
+		HDFS_CLUSTER = null;
+		HDFS_ROOT_PATH = null;
+	}
+
+	@Before
+	public void initConfig() {
+		hadoopConfig = new org.apache.hadoop.conf.Configuration();
+		hadoopConfig.set(org.apache.hadoop.fs.FileSystem.FS_DEFAULT_NAME_KEY, HDFS_ROOT_PATH.toString());
+	}
+
+	// ------------------------------------------------------------------------
+	//  Tests
+	// ------------------------------------------------------------------------
+
+	@Test
+	public void testRepeatedClose() throws Exception {
+		final Configuration flinkConfig = new Configuration();
+
+		final YarnHighAvailabilityServices services = new YarnIntraNonHaMasterServices(flinkConfig, hadoopConfig);
+		services.closeAndCleanupAllData();
+
+		// this should not throw an exception
+		services.close();
+	}
+
+	@Test
+	public void testClosingReportsToLeader() throws Exception {
+		final Configuration flinkConfig = new Configuration();
+
+		try (YarnHighAvailabilityServices services = new YarnIntraNonHaMasterServices(flinkConfig, hadoopConfig)) {
+			final LeaderElectionService elector = services.getResourceManagerLeaderElectionService();
+			final LeaderContender contender = mockContender(elector);
+
+			elector.start(contender);
+			services.close();
+
+			verify(contender, timeout(100).times(1)).handleError(any(Exception.class));
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  utilities
+	// ------------------------------------------------------------------------
+
+	private static LeaderContender mockContender(final LeaderElectionService service) {
+		String address = StringUtils.getRandomString(RND, 5, 10, 'a', 'z');
+		return mockContender(service, address);
+	}
+
+	private static LeaderContender mockContender(final LeaderElectionService service, final String address) {
+		LeaderContender mockContender = mock(LeaderContender.class);
+
+		when(mockContender.getAddress()).thenReturn(address);
+
+		doAnswer(new Answer<Void>() {
+			@Override
+			public Void answer(InvocationOnMock invocation) throws Throwable {
+				final UUID uuid = (UUID) invocation.getArguments()[0];
+				service.confirmLeaderSessionID(uuid);
+				return null;
+			}
+		}).when(mockContender).grantLeadership(any(UUID.class));
+
+		return mockContender;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/2a7dbda7/flink-yarn/src/test/java/org/apache/flink/yarn/highavailability/YarnPreConfiguredMasterHaServicesTest.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/test/java/org/apache/flink/yarn/highavailability/YarnPreConfiguredMasterHaServicesTest.java b/flink-yarn/src/test/java/org/apache/flink/yarn/highavailability/YarnPreConfiguredMasterHaServicesTest.java
new file mode 100644
index 0000000..a13deac
--- /dev/null
+++ b/flink-yarn/src/test/java/org/apache/flink/yarn/highavailability/YarnPreConfiguredMasterHaServicesTest.java
@@ -0,0 +1,234 @@
+/*
+ * 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.yarn.highavailability;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.IllegalConfigurationException;
+import org.apache.flink.core.fs.FileSystem;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.runtime.leaderelection.LeaderContender;
+import org.apache.flink.runtime.leaderelection.LeaderElectionService;
+import org.apache.flink.util.TestLogger;
+import org.apache.flink.yarn.configuration.YarnConfigOptions;
+
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import static org.mockito.Mockito.*;
+
+
+public class YarnPreConfiguredMasterHaServicesTest extends TestLogger {
+
+	@ClassRule
+	public static final TemporaryFolder TEMP_DIR = new TemporaryFolder();
+
+	private static MiniDFSCluster HDFS_CLUSTER;
+
+	private static Path HDFS_ROOT_PATH;
+
+	private org.apache.hadoop.conf.Configuration hadoopConfig;
+
+	// ------------------------------------------------------------------------
+	//  Test setup and shutdown
+	// ------------------------------------------------------------------------
+
+	@BeforeClass
+	public static void createHDFS() throws Exception {
+		final File tempDir = TEMP_DIR.newFolder();
+
+		org.apache.hadoop.conf.Configuration hdConf = new org.apache.hadoop.conf.Configuration();
+		hdConf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, tempDir.getAbsolutePath());
+
+		MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(hdConf);
+		HDFS_CLUSTER = builder.build();
+		HDFS_ROOT_PATH = new Path(HDFS_CLUSTER.getURI());
+	}
+
+	@AfterClass
+	public static void destroyHDFS() {
+		if (HDFS_CLUSTER != null) {
+			HDFS_CLUSTER.shutdown();
+		}
+		HDFS_CLUSTER = null;
+		HDFS_ROOT_PATH = null;
+	}
+
+	@Before
+	public void initConfig() {
+		hadoopConfig = new org.apache.hadoop.conf.Configuration();
+		hadoopConfig.set(org.apache.hadoop.fs.FileSystem.FS_DEFAULT_NAME_KEY, HDFS_ROOT_PATH.toString());
+	}
+
+	// ------------------------------------------------------------------------
+	//  Tests
+	// ------------------------------------------------------------------------
+
+	@Test
+	public void testConstantResourceManagerName() throws Exception {
+		final Configuration flinkConfig = new Configuration();
+		flinkConfig.setString(YarnConfigOptions.APP_MASTER_RPC_ADDRESS, "localhost");
+		flinkConfig.setInteger(YarnConfigOptions.APP_MASTER_RPC_PORT, 1427);
+
+		YarnHighAvailabilityServices services1 = new YarnPreConfiguredMasterNonHaServices(flinkConfig, hadoopConfig);
+		YarnHighAvailabilityServices services2 = new YarnPreConfiguredMasterNonHaServices(flinkConfig, hadoopConfig);
+
+		try {
+			String rmName1 = services1.getResourceManagerEndpointName();
+			String rmName2 = services2.getResourceManagerEndpointName();
+
+			assertNotNull(rmName1);
+			assertNotNull(rmName2);
+			assertEquals(rmName1, rmName2);
+		}
+		finally {
+			services1.closeAndCleanupAllData();
+			services2.closeAndCleanupAllData();
+		}
+	}
+
+	@Test
+	public void testMissingRmConfiguration() throws Exception {
+		final Configuration flinkConfig = new Configuration();
+
+		// missing resource manager address
+		try {
+			new YarnPreConfiguredMasterNonHaServices(flinkConfig, hadoopConfig);
+			fail();
+		} catch (IllegalConfigurationException e) {
+			// expected
+		}
+
+		flinkConfig.setString(YarnConfigOptions.APP_MASTER_RPC_ADDRESS, "localhost");
+
+		// missing resource manager port
+		try {
+			new YarnPreConfiguredMasterNonHaServices(flinkConfig, hadoopConfig);
+			fail();
+		} catch (IllegalConfigurationException e) {
+			// expected
+		}
+
+		flinkConfig.setInteger(YarnConfigOptions.APP_MASTER_RPC_PORT, 1427);
+
+		// now everything is good ;-)
+		new YarnPreConfiguredMasterNonHaServices(flinkConfig, hadoopConfig).closeAndCleanupAllData();
+	}
+
+	@Test
+	public void testCloseAndCleanup() throws Exception {
+		final Configuration flinkConfig = new Configuration();
+		flinkConfig.setString(YarnConfigOptions.APP_MASTER_RPC_ADDRESS, "localhost");
+		flinkConfig.setInteger(YarnConfigOptions.APP_MASTER_RPC_PORT, 1427);
+
+		// create the services
+		YarnHighAvailabilityServices services = new YarnPreConfiguredMasterNonHaServices(flinkConfig, hadoopConfig);
+		services.closeAndCleanupAllData();
+
+		final FileSystem fileSystem = HDFS_ROOT_PATH.getFileSystem();
+		final Path workDir = new Path(HDFS_CLUSTER.getFileSystem().getWorkingDirectory().toString());
+		
+		try {
+			fileSystem.getFileStatus(new Path(workDir, YarnHighAvailabilityServices.FLINK_RECOVERY_DATA_DIR));
+			fail("Flink recovery data directory still exists");
+		}
+		catch (FileNotFoundException e) {
+			// expected, because the directory should have been cleaned up
+		}
+
+		assertTrue(services.isClosed());
+
+		// doing another cleanup when the services are closed should fail
+		try {
+			services.closeAndCleanupAllData();
+			fail("should fail with an IllegalStateException");
+		} catch (IllegalStateException e) {
+			// expected
+		}
+	}
+
+	@Test
+	public void testCallsOnClosedServices() throws Exception {
+		final Configuration flinkConfig = new Configuration();
+		flinkConfig.setString(YarnConfigOptions.APP_MASTER_RPC_ADDRESS, "localhost");
+		flinkConfig.setInteger(YarnConfigOptions.APP_MASTER_RPC_PORT, 1427);
+
+		YarnHighAvailabilityServices services = new YarnPreConfiguredMasterNonHaServices(flinkConfig, hadoopConfig);
+
+		// this method is not supported
+		try {
+			services.getSubmittedJobGraphStore();
+			fail();
+		} catch (UnsupportedOperationException ignored) {}
+
+
+		services.close();
+
+		// all these methods should fail now
+
+		try {
+			services.createBlobStore();
+			fail();
+		} catch (IllegalStateException ignored) {}
+
+		try {
+			services.getCheckpointRecoveryFactory();
+			fail();
+		} catch (IllegalStateException ignored) {}
+
+		try {
+			services.getJobManagerLeaderElectionService(new JobID());
+			fail();
+		} catch (IllegalStateException ignored) {}
+
+		try {
+			services.getJobManagerLeaderRetriever(new JobID());
+			fail();
+		} catch (IllegalStateException ignored) {}
+
+		try {
+			services.getRunningJobsRegistry();
+			fail();
+		} catch (IllegalStateException ignored) {}
+
+		try {
+			services.getResourceManagerLeaderElectionService();
+			fail();
+		} catch (IllegalStateException ignored) {}
+
+		try {
+			services.getResourceManagerLeaderRetriever();
+			fail();
+		} catch (IllegalStateException ignored) {}
+	}
+}


[25/52] [abbrv] flink git commit: Rebase fixes

Posted by se...@apache.org.
Rebase fixes


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

Branch: refs/heads/master
Commit: 57762350f5baa731d93f87da5e7182d41550c9ca
Parents: d8b22e0
Author: Till Rohrmann <tr...@apache.org>
Authored: Mon Oct 31 19:01:22 2016 +0100
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Dec 23 20:54:25 2016 +0100

----------------------------------------------------------------------
 .../apache/flink/runtime/instance/SlotDescriptor.java   |  4 ++--
 .../apache/flink/runtime/taskexecutor/TaskExecutor.java | 12 ++++++------
 2 files changed, 8 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/57762350/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
index 360cc26..47ce422 100644
--- 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
@@ -53,13 +53,13 @@ public class SlotDescriptor {
 		final TaskManagerLocation location,
 		final int slotNumber,
 		final ResourceProfile resourceProfile,
-		final TaskManagerGateway actorGateway)
+		final TaskManagerGateway taskManagerGateway)
 	{
 		this.jobID = checkNotNull(jobID);
 		this.taskManagerLocation = checkNotNull(location);
 		this.slotNumber = slotNumber;
 		this.resourceProfile = checkNotNull(resourceProfile);
-		this.taskManagerGateway = checkNotNull(actorGateway);
+		this.taskManagerGateway = checkNotNull(taskManagerGateway);
 	}
 
 	public SlotDescriptor(final SlotDescriptor other) {

http://git-wip-us.apache.org/repos/asf/flink/blob/57762350/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 b981829..c94113c 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
@@ -883,12 +883,12 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 					jobMasterLeaderId,
 					jobMasterGateway,
 					new TaskExecutionState(
-							task.getJobID(),
-							task.getExecutionId(),
-							task.getExecutionState(),
-							task.getFailureCause(),
-							accumulatorSnapshot,
-							task.getMetricGroup().getIOMetricGroup().createSnapshot()));
+						task.getJobID(),
+						task.getExecutionId(),
+						task.getExecutionState(),
+						task.getFailureCause(),
+						accumulatorSnapshot,
+						task.getMetricGroup().getIOMetricGroup().createSnapshot()));
 		} else {
 			log.error("Cannot find task with ID {} to unregister.", executionAttemptID);
 		}


[21/52] [abbrv] flink git commit: Rebasing fixes on latest master

Posted by se...@apache.org.
Rebasing fixes on latest master


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

Branch: refs/heads/master
Commit: 6f691ad5f60fd966f7cb72836ddf8fae8a5559c5
Parents: 6484f45
Author: Till Rohrmann <tr...@apache.org>
Authored: Thu Oct 20 19:54:12 2016 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Dec 23 20:54:24 2016 +0100

----------------------------------------------------------------------
 .../runtime/checkpoint/CheckpointCoordinatorGateway.java     | 8 ++++----
 .../runtime/taskexecutor/rpc/RpcCheckpointResponder.java     | 4 ++--
 .../org/apache/flink/runtime/jobmanager/JobSubmitTest.java   | 2 ++
 3 files changed, 8 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/6f691ad5/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 5ed7985..8d1423a 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
@@ -25,10 +25,10 @@ import org.apache.flink.runtime.rpc.RpcGateway;
 public interface CheckpointCoordinatorGateway extends RpcGateway {
 
 	void acknowledgeCheckpoint(
-			JobID jobID,
-			ExecutionAttemptID executionAttemptID,
-			CheckpointMetaData checkpointInfo,
-			SubtaskState checkpointStateHandles);
+			final JobID jobID,
+			final ExecutionAttemptID executionAttemptID,
+			final CheckpointMetaData checkpointMetaData,
+			final SubtaskState subtaskState);
 
 	void declineCheckpoint(
 			JobID jobID,

http://git-wip-us.apache.org/repos/asf/flink/blob/6f691ad5/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcCheckpointResponder.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcCheckpointResponder.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcCheckpointResponder.java
index 85bdbff..1ce4350 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcCheckpointResponder.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcCheckpointResponder.java
@@ -39,13 +39,13 @@ public class RpcCheckpointResponder implements CheckpointResponder {
 			JobID jobID,
 			ExecutionAttemptID executionAttemptID,
 			CheckpointMetaData checkpointMetaData,
-			SubtaskState checkpointStateHandles) {
+			SubtaskState subtaskState) {
 
 		checkpointCoordinatorGateway.acknowledgeCheckpoint(
 			jobID,
 			executionAttemptID,
 			checkpointMetaData,
-			checkpointStateHandles);
+			subtaskState);
 
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/6f691ad5/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobSubmitTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobSubmitTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobSubmitTest.java
index 53bd318..07e1644 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobSubmitTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobSubmitTest.java
@@ -171,6 +171,8 @@ public class JobSubmitTest {
 
 			JobVertex jobVertex = new JobVertex("Vertex that fails in initializeOnMaster") {
 
+				private static final long serialVersionUID = -3540303593784587652L;
+
 				@Override
 				public void initializeOnMaster(ClassLoader loader) throws Exception {
 					throw new RuntimeException("test exception");


[26/52] [abbrv] flink git commit: [FLINK-4986] Improvements to the JobMaster

Posted by se...@apache.org.
[FLINK-4986] Improvements to the JobMaster


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

Branch: refs/heads/master
Commit: 8730e200864427dd5d6ddb9f841978d68ab452bd
Parents: 91f1d09
Author: Stephan Ewen <se...@apache.org>
Authored: Mon Oct 17 20:26:58 2016 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Dec 23 20:54:25 2016 +0100

----------------------------------------------------------------------
 .../runtime/executiongraph/ExecutionGraph.java  |  15 +
 .../flink/runtime/jobmaster/JobMaster.java      | 336 ++++++++++---------
 2 files changed, 184 insertions(+), 167 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/8730e200/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 cbb4c7e..2025fc2 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
@@ -621,6 +621,21 @@ public class ExecutionGraph implements AccessExecutionGraph, Archiveable<Archive
 	}
 
 	/**
+	 * Gets the accumulator results.
+	 */
+	public Map<String, Object> getAccumulators() throws IOException {
+
+		Map<String, Accumulator<?, ?>> accumulatorMap = aggregateUserAccumulators();
+
+		Map<String, Object> result = new HashMap<>();
+		for (Map.Entry<String, Accumulator<?, ?>> entry : accumulatorMap.entrySet()) {
+			result.put(entry.getKey(), entry.getValue().getLocalValue());
+		}
+
+		return result;
+	}
+
+	/**
 	 * Gets a serialized accumulator map.
 	 * @return The accumulator map with serialized accumulator values.
 	 * @throws IOException

http://git-wip-us.apache.org/repos/asf/flink/blob/8730e200/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 458bf0c..0b3b68e 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,6 +18,7 @@
 
 package org.apache.flink.runtime.jobmaster;
 
+import org.apache.flink.api.common.JobExecutionResult;
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.api.common.restartstrategy.RestartStrategies;
 import org.apache.flink.api.common.time.Time;
@@ -33,7 +34,6 @@ import org.apache.flink.runtime.checkpoint.CheckpointMetaData;
 import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory;
 import org.apache.flink.runtime.checkpoint.SubtaskState;
 import org.apache.flink.runtime.client.JobExecutionException;
-import org.apache.flink.runtime.client.SerializedJobExecutionResult;
 import org.apache.flink.runtime.clusterframework.types.AllocationID;
 import org.apache.flink.runtime.clusterframework.types.ResourceID;
 import org.apache.flink.runtime.concurrent.BiFunction;
@@ -51,6 +51,7 @@ 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;
+import org.apache.flink.runtime.highavailability.LeaderIdMismatchException;
 import org.apache.flink.runtime.instance.Slot;
 import org.apache.flink.runtime.instance.SlotDescriptor;
 import org.apache.flink.runtime.instance.SlotPool;
@@ -90,7 +91,6 @@ import org.apache.flink.runtime.taskmanager.TaskExecutionState;
 import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
 import org.apache.flink.runtime.util.SerializedThrowable;
 import org.apache.flink.util.InstantiationUtil;
-import org.apache.flink.util.SerializedValue;
 import org.slf4j.Logger;
 
 import javax.annotation.Nullable;
@@ -246,6 +246,9 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 				-1,
 				log);
 
+		// register self as job status change listener
+		executionGraph.registerJobStatusListener(new JobManagerJobStatusListener());
+
 		this.slotPool = new SlotPool(executorService);
 		this.allocationTimeout = Time.of(5, TimeUnit.SECONDS);
 
@@ -269,13 +272,18 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 	 */
 	public void start(final UUID leaderSessionID) throws Exception {
 		if (LEADER_ID_UPDATER.compareAndSet(this, null, leaderSessionID)) {
-			super.start();
 
+			// make sure the slot pool now accepts messages for this leader  
 			slotPool.setJobManagerLeaderId(leaderSessionID);
-			log.info("Starting JobManager for job {} ({})", jobGraph.getName(), jobGraph.getJobID());
+
+			// make sure we receive RPC and async calls
+			super.start();
+
+			log.info("JobManager started as leader {} for job {}", leaderSessionID, jobGraph.getJobID());
 			getSelf().startJobExecution();
-		} else {
-			log.warn("Job already started with leaderId {}, ignoring this start request.", leaderSessionID);
+		}
+		else {
+			log.warn("Job already started with leader ID {}, ignoring this start request.", leaderSessionID);
 		}
 	}
 
@@ -297,48 +305,21 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 
 	@RpcMethod
 	public void startJobExecution() {
-		log.info("Starting execution of job {} ({}) with leaderId {}.",
-				jobGraph.getName(), jobGraph.getJobID(), leaderSessionID);
+		log.info("Starting execution of job {} ({})", jobGraph.getName(), jobGraph.getJobID());
 
 		try {
-			// 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 ready to go, try to establish connection with resource manager
+			//   - activate leader retrieval for the resource manager
+			//   - on notification of the leader, the connection will be established and
+			//     the slot pool will start requesting slots
 			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
-
+		}
+		catch (Throwable t) {
 			log.error("Failed to start job {} ({})", jobGraph.getName(), jobGraph.getJobID(), t);
 
-			executionGraph.fail(t);
-
-			final JobExecutionException rt;
-			if (t instanceof JobExecutionException) {
-				rt = (JobExecutionException) t;
-			} else {
-				rt = new JobExecutionException(jobGraph.getJobID(),
-						"Failed to start job " + jobGraph.getJobID() + " (" + jobGraph.getName() + ")", t);
-			}
-
-			// TODO: notify client about this failure
+			handleFatalError(new Exception(
+					"Could not start job execution: Failed to start leader service for Resource Manager", t));
 
-			jobCompletionActions.jobFailed(rt);
 			return;
 		}
 
@@ -348,7 +329,8 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 			public void run() {
 				try {
 					executionGraph.scheduleForExecution(new PooledSlotProvider(slotPool, allocationTimeout));
-				} catch (Throwable t) {
+				}
+				catch (Throwable t) {
 					executionGraph.fail(t);
 				}
 			}
@@ -386,6 +368,8 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 		}
 		closeResourceManagerConnection();
 
+		// TODO: in the future, the slot pool should not release the resources, so that
+		// TODO: the TaskManagers offer the resources to the new leader 
 		for (ResourceID taskManagerId : registeredTaskManagers.keySet()) {
 			slotPool.releaseResource(taskManagerId);
 		}
@@ -405,14 +389,8 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 			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);
-		}
+		checkNotNull(taskExecutionState, "taskExecutionState");
+		validateLeaderSessionId(leaderSessionID);
 
 		if (executionGraph.updateState(taskExecutionState)) {
 			return Acknowledge.get();
@@ -428,10 +406,7 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 			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);
-		}
+		validateLeaderSessionId(leaderSessionID);
 
 		final Execution execution = executionGraph.getRegisteredExecutions().get(executionAttempt);
 		if (execution == null) {
@@ -477,16 +452,12 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 		}
 	}
 
-	@RpcMethod
 	public ExecutionState requestPartitionState(
 			final UUID leaderSessionID,
 			final IntermediateDataSetID intermediateResultId,
 			final ResultPartitionID resultPartitionId) throws Exception {
 
-		if (!this.leaderSessionID.equals(leaderSessionID)) {
-			throw new Exception("Leader id not match, expected: " + this.leaderSessionID
-					+ ", actual: " + leaderSessionID);
-		}
+		validateLeaderSessionId(leaderSessionID);
 
 		final Execution execution = executionGraph.getRegisteredExecutions().get(resultPartitionId.getProducerId());
 		if (execution != null) {
@@ -520,10 +491,7 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 			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);
-		}
+		validateLeaderSessionId(leaderSessionID);
 
 		executionGraph.scheduleOrUpdateConsumers(partitionID);
 		return Acknowledge.get();
@@ -534,6 +502,7 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 		throw new UnsupportedOperationException();
 	}
 
+	// TODO: This method needs a leader session ID
 	@RpcMethod
 	public void acknowledgeCheckpoint(
 			final JobID jobID,
@@ -562,6 +531,7 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 		}
 	}
 
+	// TODO: This method needs a leader session ID
 	@RpcMethod
 	public void declineCheckpoint(
 			final JobID jobID,
@@ -657,10 +627,7 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 	public Iterable<SlotOffer> offerSlots(final ResourceID taskManagerId,
 			final Iterable<SlotOffer> slots, final UUID leaderId) throws Exception
 	{
-		if (!this.leaderSessionID.equals(leaderId)) {
-			throw new Exception("Leader id not match, expected: " + this.leaderSessionID
-					+ ", actual: " + leaderId);
-		}
+		validateLeaderSessionId(leaderSessionID);
 
 		Tuple2<TaskManagerLocation, TaskExecutorGateway> taskManager = registeredTaskManagers.get(taskManagerId);
 		if (taskManager == null) {
@@ -689,10 +656,7 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 			final UUID leaderId,
 			final Exception cause) throws Exception
 	{
-		if (!this.leaderSessionID.equals(leaderId)) {
-			throw new Exception("Leader id not match, expected: " + this.leaderSessionID
-					+ ", actual: " + leaderId);
-		}
+		validateLeaderSessionId(leaderSessionID);
 
 		if (!registeredTaskManagers.containsKey(taskManagerId)) {
 			throw new Exception("Unknown TaskManager " + taskManagerId);
@@ -782,62 +746,55 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 	}
 
 	private void jobStatusChanged(final JobStatus newJobStatus, long timestamp, final Throwable error) {
+		validateRunsInMainThread();
+
 		final JobID jobID = executionGraph.getJobID();
 		final String jobName = executionGraph.getJobName();
+
 		log.info("Status of job {} ({}) changed to {}.", jobID, jobName, newJobStatus, error);
 
 		if (newJobStatus.isGloballyTerminalState()) {
-			// TODO set job end time in JobInfo
-
-			/*
-			  TODO
-			  if (jobInfo.sessionAlive) {
-                jobInfo.setLastActive()
-                val lastActivity = jobInfo.lastActive
-                context.system.scheduler.scheduleOnce(jobInfo.sessionTimeout seconds) {
-                  // remove only if no activity occurred in the meantime
-                  if (lastActivity == jobInfo.lastActive) {
-                    self ! decorateMessage(RemoveJob(jobID, removeJobFromStateBackend = true))
-                  }
-                }(context.dispatcher)
-              } else {
-                self ! decorateMessage(RemoveJob(jobID, removeJobFromStateBackend = true))
-              }
-			 */
-
-			if (newJobStatus == JobStatus.FINISHED) {
-				try {
-					final Map<String, SerializedValue<Object>> accumulatorResults =
-							executionGraph.getAccumulatorsSerialized();
-					final SerializedJobExecutionResult result = new SerializedJobExecutionResult(
-							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);
+			switch (newJobStatus) {
+				case FINISHED:
+					try {
+						// TODO get correct job duration
+						// job done, let's get the accumulators
+						Map<String, Object> accumulatorResults = executionGraph.getAccumulators();
+						JobExecutionResult result = new JobExecutionResult(jobID, 0L, accumulatorResults); 
+
+						jobCompletionActions.jobFinished(result);
+					}
+					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. " +
+								"The job is registered as 'FINISHED (successful), but this notification describes " +
+								"a failure, since the resulting accumulators could not be fetched.", e);
+
+						jobCompletionActions.jobFailed(exception);
+					}
+					break;
+
+				case CANCELED: {
 					final JobExecutionException exception = new JobExecutionException(
-							jobID, "Failed to retrieve accumulator results.", e);
-					// TODO should we also notify client?
+						jobID, "Job was cancelled.", new Exception("The job was cancelled"));
+
 					jobCompletionActions.jobFailed(exception);
+					break;
 				}
-			} else if (newJobStatus == JobStatus.CANCELED) {
-				final Throwable unpackedError = SerializedThrowable.get(error, userCodeLoader);
-				final JobExecutionException exception = new JobExecutionException(
-						jobID, "Job was cancelled.", unpackedError);
-				// TODO should we also notify client?
-				jobCompletionActions.jobFailed(exception);
-			} else if (newJobStatus == JobStatus.FAILED) {
-				final Throwable unpackedError = SerializedThrowable.get(error, userCodeLoader);
-				final JobExecutionException exception = new JobExecutionException(
-						jobID, "Job execution failed.", unpackedError);
-				// TODO should we also notify client?
-				jobCompletionActions.jobFailed(exception);
-			} else {
-				final JobExecutionException exception = new JobExecutionException(
-						jobID, newJobStatus + " is not a terminal state.");
-				// TODO should we also notify client?
-				jobCompletionActions.jobFailed(exception);
-				throw new RuntimeException(exception);
+
+				case FAILED: {
+					final Throwable unpackedError = SerializedThrowable.get(error, userCodeLoader);
+					final JobExecutionException exception = new JobExecutionException(
+							jobID, "Job execution failed.", unpackedError);
+					jobCompletionActions.jobFailed(exception);
+					break;
+				}
+
+				default:
+					// this can happen only if the enum is buggy
+					throw new IllegalStateException(newJobStatus.toString());
 			}
 		}
 	}
@@ -845,57 +802,52 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 	private void notifyOfNewResourceManagerLeader(
 			final String resourceManagerAddress, final UUID resourceManagerLeaderId)
 	{
-		// IMPORTANT: executed by main thread to avoid concurrence
-		runAsync(new Runnable() {
-			@Override
-			public void run() {
-				if (resourceManagerConnection != null) {
-					if (resourceManagerAddress != null) {
-						if (resourceManagerAddress.equals(resourceManagerConnection.getTargetAddress())
-								&& 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 {
-						log.info("Current ResourceManager {} lost leader status. Waiting for new ResourceManager leader.",
-								resourceManagerConnection.getTargetAddress());
-					}
-				}
+		validateRunsInMainThread();
 
-				closeResourceManagerConnection();
-
-				if (resourceManagerAddress != null) {
-					log.info("Attempting to register at ResourceManager {}", resourceManagerAddress);
-					resourceManagerConnection = new ResourceManagerConnection(
-							log, jobGraph.getJobID(), leaderSessionID,
-							resourceManagerAddress, resourceManagerLeaderId, executionContext);
-					resourceManagerConnection.start();
+		if (resourceManagerConnection != null) {
+			if (resourceManagerAddress != null) {
+				if (resourceManagerAddress.equals(resourceManagerConnection.getTargetAddress())
+						&& 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 {
+				log.info("Current ResourceManager {} lost leader status. Waiting for new ResourceManager leader.",
+						resourceManagerConnection.getTargetAddress());
 			}
-		});
+		}
+
+		closeResourceManagerConnection();
+
+		if (resourceManagerAddress != null) {
+			log.info("Attempting to register at ResourceManager {}", resourceManagerAddress);
+			resourceManagerConnection = new ResourceManagerConnection(
+					log, jobGraph.getJobID(), getAddress(), leaderSessionID,
+					resourceManagerAddress, resourceManagerLeaderId, executionContext);
+			resourceManagerConnection.start();
+		}
 	}
 
 	private void onResourceManagerRegistrationSuccess(final JobMasterRegistrationSuccess success) {
-		getRpcService().execute(new Runnable() {
-			@Override
-			public void run() {
-				// 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()))
-				{
-					log.info("JobManager successfully registered at ResourceManager, leader id: {}.",
-							success.getResourceManagerLeaderId());
-					slotPool.setResourceManager(success.getResourceManagerLeaderId(),
-							resourceManagerConnection.getTargetGateway());
-				}
-			}
-		});
+		validateRunsInMainThread();
+	
+		// verify the response with current connection
+		if (resourceManagerConnection != null
+				&& resourceManagerConnection.getTargetLeaderId().equals(success.getResourceManagerLeaderId()))
+		{
+			log.info("JobManager successfully registered at ResourceManager, leader id: {}.",
+					success.getResourceManagerLeaderId());
+
+			slotPool.setResourceManager(
+					success.getResourceManagerLeaderId(), resourceManagerConnection.getTargetGateway());
+		}
 	}
 
 	private void closeResourceManagerConnection() {
+		validateRunsInMainThread();
+
 		if (resourceManagerConnection != null) {
 			resourceManagerConnection.close();
 			resourceManagerConnection = null;
@@ -903,32 +855,49 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 		slotPool.disconnectResourceManager();
 	}
 
+	private void validateLeaderSessionId(UUID leaderSessionID) throws LeaderIdMismatchException {
+		if (this.leaderSessionID == null || !this.leaderSessionID.equals(leaderSessionID)) {
+			throw new LeaderIdMismatchException(this.leaderSessionID, leaderSessionID);
+		}
+	}
+
 	//----------------------------------------------------------------------------------------------
 	// Utility classes
 	//----------------------------------------------------------------------------------------------
 
 	private class ResourceManagerLeaderListener implements LeaderRetrievalListener {
+
 		@Override
 		public void notifyLeaderAddress(final String leaderAddress, final UUID leaderSessionID) {
-			notifyOfNewResourceManagerLeader(leaderAddress, leaderSessionID);
+			runAsync(new Runnable() {
+				@Override
+				public void run() {
+					notifyOfNewResourceManagerLeader(leaderAddress, leaderSessionID);
+				}
+			});
 		}
 
 		@Override
 		public void handleError(final Exception exception) {
-			handleFatalError(exception);
+			handleFatalError(new Exception("Fatal error in the ResourceManager leader service", exception));
 		}
 	}
 
+	//----------------------------------------------------------------------------------------------
+
 	private class ResourceManagerConnection
 			extends RegisteredRpcConnection<ResourceManagerGateway, JobMasterRegistrationSuccess>
 	{
 		private final JobID jobID;
 
+		private final String jobManagerRpcAddress;
+
 		private final UUID jobManagerLeaderID;
 
 		ResourceManagerConnection(
 				final Logger log,
 				final JobID jobID,
+				final String jobManagerRpcAddress,
 				final UUID jobManagerLeaderID,
 				final String resourceManagerAddress,
 				final UUID resourceManagerLeaderID,
@@ -936,6 +905,7 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 		{
 			super(log, resourceManagerAddress, resourceManagerLeaderID, executor);
 			this.jobID = checkNotNull(jobID);
+			this.jobManagerRpcAddress = checkNotNull(jobManagerRpcAddress);
 			this.jobManagerLeaderID = checkNotNull(jobManagerLeaderID);
 		}
 
@@ -946,18 +916,29 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 					getTargetAddress(), getTargetLeaderId())
 			{
 				@Override
-				protected Future<RegistrationResponse> invokeRegistration(ResourceManagerGateway gateway, UUID leaderId,
-						long timeoutMillis) throws Exception
+				protected Future<RegistrationResponse> invokeRegistration(
+						ResourceManagerGateway gateway, UUID leaderId, long timeoutMillis) throws Exception
 				{
 					Time timeout = Time.milliseconds(timeoutMillis);
-					return gateway.registerJobManager(leaderId, jobManagerLeaderID, getAddress(), jobID, timeout);
+
+					return gateway.registerJobManager(
+						leaderId,
+						jobManagerLeaderID,
+						jobManagerRpcAddress,
+						jobID,
+						timeout);
 				}
 			};
 		}
 
 		@Override
 		protected void onRegistrationSuccess(final JobMasterRegistrationSuccess success) {
-			onResourceManagerRegistrationSuccess(success);
+			runAsync(new Runnable() {
+				@Override
+				public void run() {
+					onResourceManagerRegistrationSuccess(success);
+				}
+			});
 		}
 
 		@Override
@@ -965,4 +946,25 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 			handleFatalError(failure);
 		}
 	}
+
+	//----------------------------------------------------------------------------------------------
+
+	private class JobManagerJobStatusListener implements 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);
+				}
+			});
+		}
+	}
 }


[51/52] [abbrv] flink git commit: [tests] Harden TaskExecutorTest

Posted by se...@apache.org.
[tests] Harden TaskExecutorTest


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

Branch: refs/heads/master
Commit: 235a169691bd3c3ff2a25b7a7763c6900a0f2c6c
Parents: 368d0da
Author: Stephan Ewen <se...@apache.org>
Authored: Fri Dec 23 19:21:43 2016 +0100
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Dec 23 20:54:28 2016 +0100

----------------------------------------------------------------------
 .../runtime/taskexecutor/TaskExecutorTest.java  | 77 +++++++++++++++-----
 1 file changed, 57 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/235a1696/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java
index 2af97b5..aacd329 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java
@@ -21,10 +21,10 @@ package org.apache.flink.runtime.taskexecutor;
 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.runtime.blob.BlobKey;
-import org.apache.flink.runtime.clusterframework.types.AllocationID;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.blob.BlobKey;
 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.ResourceProfile;
 import org.apache.flink.runtime.clusterframework.types.SlotID;
@@ -75,14 +75,11 @@ import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
 import org.apache.flink.runtime.util.TestingFatalErrorHandler;
 import org.apache.flink.util.SerializedValue;
 import org.apache.flink.util.TestLogger;
-
 import org.junit.Ignore;
 import org.junit.Rule;
 import org.junit.Test;
-
 import org.junit.rules.TestName;
 import org.mockito.Matchers;
-import org.powermock.api.mockito.PowerMockito;
 
 import java.net.InetAddress;
 import java.net.URL;
@@ -90,9 +87,18 @@ import java.util.Arrays;
 import java.util.Collections;
 import java.util.UUID;
 
-import static org.junit.Assert.*;
-import static org.mockito.Mockito.*;
 import static org.hamcrest.Matchers.contains;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.any;
+import static org.mockito.Mockito.anyInt;
+import static org.mockito.Mockito.anyString;
+import static org.mockito.Mockito.eq;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
 
 public class TaskExecutorTest extends TestLogger {
 
@@ -107,10 +113,16 @@ public class TaskExecutorTest extends TestLogger {
 
 		final TestingSerialRpcService rpc = new TestingSerialRpcService();
 		try {
+			final FatalErrorHandler errorHandler = mock(FatalErrorHandler.class);
+
 			// register a mock resource manager gateway
 			ResourceManagerGateway rmGateway = mock(ResourceManagerGateway.class);
+			when(rmGateway.registerTaskExecutor(
+					any(UUID.class), anyString(), any(ResourceID.class), any(SlotReport.class), any(Time.class)))
+				.thenReturn(FlinkCompletableFuture.<RegistrationResponse>completed(new RegistrationResponse.Success()));
+
 			TaskManagerConfiguration taskManagerServicesConfiguration = mock(TaskManagerConfiguration.class);
-			PowerMockito.when(taskManagerServicesConfiguration.getNumberSlots()).thenReturn(1);
+			when(taskManagerServicesConfiguration.getNumberSlots()).thenReturn(1);
 
 			rpc.registerGateway(resourceManagerAddress, rmGateway);
 
@@ -123,6 +135,8 @@ public class TaskExecutorTest extends TestLogger {
 			final SlotReport slotReport = new SlotReport();
 			when(taskSlotTable.createSlotReport(any(ResourceID.class))).thenReturn(slotReport);
 
+			final TestingFatalErrorHandler testingFatalErrorHandler = new TestingFatalErrorHandler();
+
 			TaskExecutor taskManager = new TaskExecutor(
 				taskManagerServicesConfiguration,
 				taskManagerLocation,
@@ -138,13 +152,16 @@ public class TaskExecutorTest extends TestLogger {
 				taskSlotTable,
 				mock(JobManagerTable.class),
 				mock(JobLeaderService.class),
-				mock(FatalErrorHandler.class));
+				testingFatalErrorHandler);
 
 			taskManager.start();
 			String taskManagerAddress = taskManager.getAddress();
 
 			verify(rmGateway).registerTaskExecutor(
 					any(UUID.class), eq(taskManagerAddress), eq(resourceID), eq(slotReport), any(Time.class));
+
+			// check if a concurrent error occurred
+			testingFatalErrorHandler.rethrowError();
 		}
 		finally {
 			rpc.stopService();
@@ -165,6 +182,14 @@ public class TaskExecutorTest extends TestLogger {
 			// register the mock resource manager gateways
 			ResourceManagerGateway rmGateway1 = mock(ResourceManagerGateway.class);
 			ResourceManagerGateway rmGateway2 = mock(ResourceManagerGateway.class);
+
+			when(rmGateway1.registerTaskExecutor(
+					any(UUID.class), anyString(), any(ResourceID.class), any(SlotReport.class), any(Time.class)))
+					.thenReturn(FlinkCompletableFuture.<RegistrationResponse>completed(new RegistrationResponse.Success()));
+			when(rmGateway2.registerTaskExecutor(
+					any(UUID.class), anyString(), any(ResourceID.class), any(SlotReport.class), any(Time.class)))
+					.thenReturn(FlinkCompletableFuture.<RegistrationResponse>completed(new RegistrationResponse.Success()));
+
 			rpc.registerGateway(address1, rmGateway1);
 			rpc.registerGateway(address2, rmGateway2);
 
@@ -174,9 +199,9 @@ public class TaskExecutorTest extends TestLogger {
 			haServices.setResourceManagerLeaderRetriever(testLeaderService);
 
 			TaskManagerConfiguration taskManagerServicesConfiguration = mock(TaskManagerConfiguration.class);
-			PowerMockito.when(taskManagerServicesConfiguration.getNumberSlots()).thenReturn(1);
-			PowerMockito.when(taskManagerServicesConfiguration.getConfiguration()).thenReturn(new Configuration());
-			PowerMockito.when(taskManagerServicesConfiguration.getTmpDirectories()).thenReturn(new String[1]);
+			when(taskManagerServicesConfiguration.getNumberSlots()).thenReturn(1);
+			when(taskManagerServicesConfiguration.getConfiguration()).thenReturn(new Configuration());
+			when(taskManagerServicesConfiguration.getTmpDirectories()).thenReturn(new String[1]);
 
 			TaskManagerLocation taskManagerLocation = mock(TaskManagerLocation.class);
 			when(taskManagerLocation.getResourceID()).thenReturn(resourceID);
@@ -186,6 +211,8 @@ public class TaskExecutorTest extends TestLogger {
 			final SlotReport slotReport = new SlotReport();
 			when(taskSlotTable.createSlotReport(any(ResourceID.class))).thenReturn(slotReport);
 
+			final TestingFatalErrorHandler testingFatalErrorHandler = new TestingFatalErrorHandler();
+
 			TaskExecutor taskManager = new TaskExecutor(
 				taskManagerServicesConfiguration,
 				taskManagerLocation,
@@ -201,7 +228,7 @@ public class TaskExecutorTest extends TestLogger {
 				taskSlotTable,
 				mock(JobManagerTable.class),
 				mock(JobLeaderService.class),
-				mock(FatalErrorHandler.class));
+				testingFatalErrorHandler);
 
 			taskManager.start();
 			String taskManagerAddress = taskManager.getAddress();
@@ -225,6 +252,9 @@ public class TaskExecutorTest extends TestLogger {
 			verify(rmGateway2).registerTaskExecutor(
 					eq(leaderId2), eq(taskManagerAddress), eq(resourceID), eq(slotReport), any(Time.class));
 			assertNotNull(taskManager.getResourceManagerConnection());
+
+			// check if a concurrent error occurred
+			testingFatalErrorHandler.rethrowError();
 		}
 		finally {
 			rpc.stopService();
@@ -310,6 +340,7 @@ public class TaskExecutorTest extends TestLogger {
 		when(haServices.getResourceManagerLeaderRetriever()).thenReturn(mock(LeaderRetrievalService.class));
 
 		try {
+			final TestingFatalErrorHandler testingFatalErrorHandler = new TestingFatalErrorHandler();
 
 			TaskExecutor taskManager = new TaskExecutor(
 				taskManagerConfiguration,
@@ -326,7 +357,7 @@ public class TaskExecutorTest extends TestLogger {
 				taskSlotTable,
 				jobManagerTable,
 				mock(JobLeaderService.class),
-				mock(FatalErrorHandler.class));
+				testingFatalErrorHandler);
 
 			taskManager.start();
 
@@ -336,6 +367,8 @@ public class TaskExecutorTest extends TestLogger {
 
 			completionFuture.get();
 
+			// check if a concurrent error occurred
+			testingFatalErrorHandler.rethrowError();
 		} finally {
 			rpc.stopService();
 		}
@@ -452,10 +485,10 @@ public class TaskExecutorTest extends TestLogger {
 					(Iterable<SlotOffer>)Matchers.argThat(contains(slotOffer)),
 					eq(jobManagerLeaderId),
 					any(Time.class));
-		} finally {
+
 			// check if a concurrent error occurred
 			testingFatalErrorHandler.rethrowError();
-
+		} finally {
 			rpc.stopService();
 		}
 	}
@@ -559,10 +592,10 @@ public class TaskExecutorTest extends TestLogger {
 			assertTrue(taskSlotTable.existsActiveSlot(jobId, allocationId1));
 			assertFalse(taskSlotTable.existsActiveSlot(jobId, allocationId2));
 			assertTrue(taskSlotTable.isSlotFree(1));
-		} finally {
+
 			// check if a concurrent error occurred
 			testingFatalErrorHandler.rethrowError();
-
+		} finally {
 			rpc.stopService();
 		}
 	}
@@ -595,11 +628,13 @@ public class TaskExecutorTest extends TestLogger {
 			haServices.setResourceManagerLeaderRetriever(testLeaderService);
 
 			TaskManagerConfiguration taskManagerServicesConfiguration = mock(TaskManagerConfiguration.class);
-			PowerMockito.when(taskManagerServicesConfiguration.getNumberSlots()).thenReturn(1);
+			when(taskManagerServicesConfiguration.getNumberSlots()).thenReturn(1);
 
 			TaskManagerLocation taskManagerLocation = mock(TaskManagerLocation.class);
 			when(taskManagerLocation.getResourceID()).thenReturn(resourceID);
 
+			final TestingFatalErrorHandler testingFatalErrorHandler = new TestingFatalErrorHandler();
+
 			TaskExecutor taskManager = new TaskExecutor(
 				taskManagerServicesConfiguration,
 				taskManagerLocation,
@@ -615,7 +650,7 @@ public class TaskExecutorTest extends TestLogger {
 				mock(TaskSlotTable.class),
 				mock(JobManagerTable.class),
 				mock(JobLeaderService.class),
-				mock(FatalErrorHandler.class));
+				testingFatalErrorHandler);
 
 			taskManager.start();
 			String taskManagerAddress = taskManager.getAddress();
@@ -653,6 +688,8 @@ public class TaskExecutorTest extends TestLogger {
 				taskManager.requestSlot(unconfirmedFreeSlotID, jobId, new AllocationID(), jobManagerAddress, leaderId);
 			assertTrue(tmSlotRequestReply3 instanceof TMSlotRequestRegistered);
 
+			// check if a concurrent error occurred
+			testingFatalErrorHandler.rethrowError();
 		}
 		finally {
 			rpc.stopService();


[12/52] [abbrv] flink git commit: [FLINK-4689] [cluster management] Implement a simple slot provider for the new job manager

Posted by se...@apache.org.
[FLINK-4689] [cluster management] Implement a simple slot provider for the new job manager


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

Branch: refs/heads/master
Commit: 5cbec02f1055218db1531bd67ebfe69a24746852
Parents: 106cb9e
Author: Kurt Young <yk...@gmail.com>
Authored: Sun Oct 16 22:20:38 2016 +0800
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Dec 23 20:54:23 2016 +0100

----------------------------------------------------------------------
 .../apache/flink/runtime/instance/SlotPool.java |  1 -
 .../jobmanager/slots/PooledSlotProvider.java    | 73 ++++++++++++++++++++
 .../flink/runtime/jobmaster/JobMaster.java      | 24 ++++---
 3 files changed, 89 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/5cbec02f/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
index beca74d..7e7b21e 100644
--- 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
@@ -135,7 +135,6 @@ public class SlotPool implements SlotOwner {
 
 		internalAllocateSlot(jobID, allocationID, resourceProfile, future);
 
-		final SlotOwner owner = this;
 		return future.thenApplyAsync(
 			new ApplyFunction<SlotDescriptor, SimpleSlot>() {
 				@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/5cbec02f/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/slots/PooledSlotProvider.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/slots/PooledSlotProvider.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/slots/PooledSlotProvider.java
new file mode 100644
index 0000000..5655fc2
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/slots/PooledSlotProvider.java
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.jobmanager.slots;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.concurrent.Future;
+import org.apache.flink.runtime.concurrent.impl.FlinkCompletableFuture;
+import org.apache.flink.runtime.instance.SimpleSlot;
+import org.apache.flink.runtime.instance.SlotPool;
+import org.apache.flink.runtime.instance.SlotProvider;
+import org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException;
+import org.apache.flink.runtime.jobmanager.scheduler.ScheduledUnit;
+
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeoutException;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * A simple pool based slot provider with {@link SlotPool} as the underlying storage.
+ */
+public class PooledSlotProvider implements SlotProvider {
+
+	/** The pool which holds all the slots. */
+	private final SlotPool slotPool;
+
+	/** The timeout for allocation. */
+	private final Time timeout;
+
+	public PooledSlotProvider(final SlotPool slotPool, final Time timeout) {
+		this.slotPool = slotPool;
+		this.timeout = timeout;
+	}
+
+	@Override
+	public Future<SimpleSlot> allocateSlot(ScheduledUnit task,
+			boolean allowQueued) throws NoResourceAvailableException
+	{
+		checkNotNull(task);
+
+		final JobID jobID = task.getTaskToExecute().getVertex().getJobId();
+		final Future<SimpleSlot> future = slotPool.allocateSimpleSlot(jobID, ResourceProfile.UNKNOWN);
+		try {
+			final SimpleSlot slot = future.get(timeout.getSize(), timeout.getUnit());
+			return FlinkCompletableFuture.completed(slot);
+		} catch (InterruptedException e) {
+			throw new NoResourceAvailableException("Could not allocate a slot because it's interrupted.");
+		} catch (ExecutionException e) {
+			throw new NoResourceAvailableException("Could not allocate a slot because some error occurred " +
+					"during allocation, " + e.getMessage());
+		} catch (TimeoutException e) {
+			throw new NoResourceAvailableException("Could not allocate a slot within time limit: " + timeout);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/5cbec02f/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 81a3e23..e6720fd 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
@@ -50,6 +50,7 @@ import org.apache.flink.runtime.executiongraph.restart.RestartStrategy;
 import org.apache.flink.runtime.executiongraph.restart.RestartStrategyFactory;
 import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
 import org.apache.flink.runtime.instance.Slot;
+import org.apache.flink.runtime.instance.SlotPool;
 import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
 import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
 import org.apache.flink.runtime.jobgraph.JobGraph;
@@ -57,7 +58,7 @@ import org.apache.flink.runtime.jobgraph.JobStatus;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.jobmanager.OnCompletionActions;
 import org.apache.flink.runtime.jobmanager.PartitionProducerDisposedException;
-import org.apache.flink.runtime.jobmanager.scheduler.Scheduler;
+import org.apache.flink.runtime.jobmanager.slots.PooledSlotProvider;
 import org.apache.flink.runtime.jobmaster.message.ClassloadingProps;
 import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener;
 import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
@@ -84,7 +85,6 @@ import org.apache.flink.runtime.taskmanager.TaskExecutionState;
 import org.apache.flink.runtime.util.SerializedThrowable;
 import org.apache.flink.util.InstantiationUtil;
 import org.apache.flink.util.SerializedValue;
-
 import org.slf4j.Logger;
 
 import javax.annotation.Nullable;
@@ -93,6 +93,7 @@ import java.util.Map;
 import java.util.UUID;
 import java.util.concurrent.Executor;
 import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
 
 import static org.apache.flink.util.Preconditions.checkNotNull;
@@ -145,6 +146,9 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 	/** The execution graph of this job */
 	private final ExecutionGraph executionGraph;
 
+	private final SlotPool slotPool;
+
+	private final Time allocationTimeout;
 
 	private volatile UUID leaderSessionID;
 
@@ -156,8 +160,6 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 	/** Connection with ResourceManager, null if not located address yet or we close it initiative */
 	private ResourceManagerConnection resourceManagerConnection;
 
-	// TODO - we need to replace this with the slot pool
-	private final Scheduler scheduler;
 
 	// ------------------------------------------------------------------------
 
@@ -240,8 +242,8 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 				-1,
 				log);
 
-		// TODO - temp fix
-		this.scheduler = new Scheduler(executorService);
+		this.slotPool = new SlotPool(executorService);
+		this.allocationTimeout = Time.of(5, TimeUnit.SECONDS);
 	}
 
 	//----------------------------------------------------------------------------------------------
@@ -263,6 +265,7 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 		if (LEADER_ID_UPDATER.compareAndSet(this, null, leaderSessionID)) {
 			super.start();
 
+			slotPool.setJobManagerLeaderId(leaderSessionID);
 			log.info("Starting JobManager for job {} ({})", jobGraph.getName(), jobGraph.getJobID());
 			getSelf().startJobExecution();
 		} else {
@@ -338,7 +341,7 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 			@Override
 			public void run() {
 				try {
-					executionGraph.scheduleForExecution(scheduler);
+					executionGraph.scheduleForExecution(new PooledSlotProvider(slotPool, allocationTimeout));
 				} catch (Throwable t) {
 					executionGraph.fail(t);
 				}
@@ -366,6 +369,7 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 		((StartStoppable) getSelf()).stop();
 
 		leaderSessionID = null;
+		slotPool.setJobManagerLeaderId(null);
 		executionGraph.suspend(cause);
 
 		// disconnect from resource manager:
@@ -777,9 +781,12 @@ 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());
+					slotPool.setResourceManager(success.getResourceManagerLeaderId(),
+							resourceManagerConnection.getTargetGateway());
 				}
 			}
 		});
@@ -790,6 +797,7 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 			resourceManagerConnection.close();
 			resourceManagerConnection = null;
 		}
+		slotPool.disconnectResourceManager();
 	}
 
 	//----------------------------------------------------------------------------------------------


[49/52] [abbrv] flink git commit: [FLINK-4930] [client, yarn] Implement FLIP-6 YARN client

Posted by se...@apache.org.
[FLINK-4930] [client, yarn] Implement FLIP-6 YARN client

Summary: Implement FLIP-6 YARN client

Test Plan: NA

Reviewers: biao.liub

Differential Revision: http://phabricator.taobao.net/D6563


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

Branch: refs/heads/master
Commit: 3695a8e92e9c3deee368d9cc3ce89a5ab117d6a1
Parents: 2a7dbda
Author: shuai.xus <sh...@alibaba-inc.com>
Authored: Wed Nov 23 17:19:35 2016 +0800
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Dec 23 20:54:27 2016 +0100

----------------------------------------------------------------------
 .../org/apache/flink/client/CliFrontend.java    |   1 +
 .../yarn/AbstractYarnClusterDescriptor.java     | 101 +++++---
 .../apache/flink/yarn/YarnClusterClientV2.java  | 169 +++++++++++++
 .../flink/yarn/YarnClusterDescriptorV2.java     |  34 +++
 .../org/apache/flink/yarn/cli/FlinkYarnCLI.java | 253 +++++++++++++++++++
 5 files changed, 524 insertions(+), 34 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/3695a8e9/flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java b/flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java
index 1ec0674..dc3280e 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java
@@ -131,6 +131,7 @@ public class CliFrontend {
 		/** command line interface of the YARN session, with a special initialization here
 		 *  to prefix all options with y/yarn. */
 		loadCustomCommandLine("org.apache.flink.yarn.cli.FlinkYarnSessionCli", "y", "yarn");
+		loadCustomCommandLine("org.apache.flink.yarn.cli.FlinkYarnCLI", "y", "yarn");
 		customCommandLine.add(new DefaultCLI());
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/3695a8e9/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 ca18439..b4c87b8 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
@@ -25,6 +25,7 @@ import org.apache.flink.configuration.GlobalConfiguration;
 import org.apache.flink.configuration.HighAvailabilityOptions;
 import org.apache.flink.configuration.IllegalConfigurationException;
 import org.apache.flink.runtime.akka.AkkaUtils;
+import org.apache.flink.runtime.jobgraph.JobGraph;
 import org.apache.flink.runtime.jobmanager.HighAvailabilityMode;
 import org.apache.flink.runtime.security.SecurityUtils;
 import org.apache.hadoop.conf.Configuration;
@@ -60,6 +61,8 @@ import java.io.ByteArrayOutputStream;
 import java.io.File;
 import java.io.IOException;
 import java.io.PrintStream;
+import java.io.FileOutputStream;
+import java.io.ObjectOutputStream;
 import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
 import java.net.URISyntaxException;
@@ -460,28 +463,6 @@ public abstract class AbstractYarnClusterDescriptor implements ClusterDescriptor
 			flinkConfiguration.setString(dynProperty.getKey(), dynProperty.getValue());
 		}
 
-		// ------------------ Set default file system scheme -------------------------
-
-		try {
-			org.apache.flink.core.fs.FileSystem.setDefaultScheme(flinkConfiguration);
-		} catch (IOException e) {
-			throw new IOException("Error while setting the default " +
-				"filesystem scheme from configuration.", e);
-		}
-
-		// initialize file system
-		// Copy the application master jar to the filesystem
-		// Create a local resource to point to the destination jar path
-		final FileSystem fs = FileSystem.get(conf);
-
-		// hard coded check for the GoogleHDFS client because its not overriding the getScheme() method.
-		if (!fs.getClass().getSimpleName().equals("GoogleHadoopFileSystem") &&
-			fs.getScheme().startsWith("file")) {
-			LOG.warn("The file system scheme is '" + fs.getScheme() + "'. This indicates that the "
-				+ "specified Hadoop configuration path is wrong and the system is using the default Hadoop configuration values."
-				+ "The Flink YARN client needs to store its files in a distributed file system");
-		}
-
 		// ------------------ Check if the YARN ClusterClient has the requested resources --------------
 
 		// the yarnMinAllocationMB specifies the smallest possible container allocation size.
@@ -505,6 +486,7 @@ public abstract class AbstractYarnClusterDescriptor implements ClusterDescriptor
 		// Create application via yarnClient
 		final YarnClientApplication yarnApplication = yarnClient.createApplication();
 		GetNewApplicationResponse appResponse = yarnApplication.getNewApplicationResponse();
+		ApplicationSubmissionContext appContext = yarnApplication.getApplicationSubmissionContext();
 
 		Resource maxRes = appResponse.getMaximumResourceCapability();
 		final String NOTE = "Please check the 'yarn.scheduler.maximum-allocation-mb' and the 'yarn.nodemanager.resource.memory-mb' configuration values\n";
@@ -560,6 +542,45 @@ public abstract class AbstractYarnClusterDescriptor implements ClusterDescriptor
 			}
 		}
 
+		ApplicationReport report = startAppMaster(null, yarnClient);
+
+		String host = report.getHost();
+		int port = report.getRpcPort();
+
+		// Correctly initialize the Flink config
+		flinkConfiguration.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, host);
+		flinkConfiguration.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, port);
+
+		// the Flink cluster is deployed in YARN. Represent cluster
+		return createYarnClusterClient(this, yarnClient, report, flinkConfiguration, sessionFilesDir, true);
+	}
+
+	public ApplicationReport startAppMaster(JobGraph jobGraph, YarnClient yarnClient) throws Exception {
+
+		// ------------------ Set default file system scheme -------------------------
+
+		try {
+			org.apache.flink.core.fs.FileSystem.setDefaultScheme(flinkConfiguration);
+		} catch (IOException e) {
+			throw new IOException("Error while setting the default " +
+					"filesystem scheme from configuration.", e);
+		}
+
+		// initialize file system
+		// Copy the application master jar to the filesystem
+		// Create a local resource to point to the destination jar path
+		final FileSystem fs = FileSystem.get(conf);
+
+		// hard coded check for the GoogleHDFS client because its not overriding the getScheme() method.
+		if (!fs.getClass().getSimpleName().equals("GoogleHadoopFileSystem") &&
+				fs.getScheme().startsWith("file")) {
+			LOG.warn("The file system scheme is '" + fs.getScheme() + "'. This indicates that the "
+					+ "specified Hadoop configuration path is wrong and the system is using the default Hadoop configuration values."
+					+ "The Flink YARN client needs to store its files in a distributed file system");
+		}
+
+		final YarnClientApplication yarnApplication = yarnClient.createApplication();
+		ApplicationSubmissionContext appContext = yarnApplication.getApplicationSubmissionContext();
 		Set<File> effectiveShipFiles = new HashSet<>(shipFiles.size());
 		for (File file : shipFiles) {
 			effectiveShipFiles.add(file.getAbsoluteFile());
@@ -596,8 +617,8 @@ public abstract class AbstractYarnClusterDescriptor implements ClusterDescriptor
 			effectiveShipFiles.addAll(userJarFiles);
 		}
 
+
 		// Set-up ApplicationSubmissionContext for the application
-		ApplicationSubmissionContext appContext = yarnApplication.getApplicationSubmissionContext();
 
 		final ApplicationId appId = appContext.getApplicationId();
 
@@ -694,6 +715,27 @@ public abstract class AbstractYarnClusterDescriptor implements ClusterDescriptor
 		paths.add(remotePathConf);
 		classPathBuilder.append("flink-conf.yaml").append(File.pathSeparator);
 
+		// write job graph to tmp file and add it to local resource 
+		// TODO: need refine ?
+		if (jobGraph != null) {
+			try {
+				File fp = new File("/tmp/jobgraph-" + appId.toString());
+				FileOutputStream input = new FileOutputStream(fp);
+				ObjectOutputStream obInput = new ObjectOutputStream(input);
+				obInput.writeObject(jobGraph);
+				input.close();
+				LocalResource jobgraph = Records.newRecord(LocalResource.class);
+				Path remoteJobGraph =
+						Utils.setupLocalResource(fs, appId.toString(), new Path(fp.toURI()), jobgraph, fs.getHomeDirectory());
+				localResources.put("job.graph", jobgraph);
+				paths.add(remoteJobGraph);
+				classPathBuilder.append("job.graph").append(File.pathSeparator);
+			} catch (Exception e) {
+				LOG.warn("Add job graph to local resource fail");
+				throw e;
+			}
+		}
+
 		sessionFilesDir = new Path(fs.getHomeDirectory(), ".flink/" + appId.toString() + "/");
 
 		FsPermission permission = new FsPermission(FsAction.ALL, FsAction.NONE, FsAction.NONE);
@@ -835,7 +877,7 @@ public abstract class AbstractYarnClusterDescriptor implements ClusterDescriptor
 			LOG.debug("Application State: {}", appState);
 			switch(appState) {
 				case FAILED:
-				case FINISHED:
+				case FINISHED: //TODO: the finished state may be valid in flip-6
 				case KILLED:
 					throw new YarnDeploymentException("The YARN application unexpectedly switched to state "
 						+ appState + " during deployment. \n" +
@@ -871,16 +913,7 @@ public abstract class AbstractYarnClusterDescriptor implements ClusterDescriptor
 		} catch (IllegalStateException e) {
 			// we're already in the shut down hook.
 		}
-
-		String host = report.getHost();
-		int port = report.getRpcPort();
-
-		// Correctly initialize the Flink config
-		flinkConfiguration.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, host);
-		flinkConfiguration.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, port);
-
-		// the Flink cluster is deployed in YARN. Represent cluster
-		return createYarnClusterClient(this, yarnClient, report, flinkConfiguration, sessionFilesDir, true);
+		return report;
 	}
 
 	/**

http://git-wip-us.apache.org/repos/asf/flink/blob/3695a8e9/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterClientV2.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterClientV2.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterClientV2.java
new file mode 100644
index 0000000..daa2c3b
--- /dev/null
+++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterClientV2.java
@@ -0,0 +1,169 @@
+/*
+ * 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.yarn;
+
+import org.apache.flink.api.common.JobSubmissionResult;
+import org.apache.flink.client.program.ClusterClient;
+import org.apache.flink.client.program.ProgramInvocationException;
+import org.apache.flink.runtime.clusterframework.ApplicationStatus;
+import org.apache.flink.runtime.clusterframework.messages.GetClusterStatusResponse;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ApplicationReport;
+import org.apache.hadoop.yarn.api.records.YarnApplicationState;
+import org.apache.hadoop.yarn.client.api.YarnClient;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.net.URL;
+import java.util.List;
+
+/**
+ * Java representation of a running Flink job on YARN.
+ * Since flip-6, a flink job will be run as a yarn job by default, each job has a jobmaster, 
+ * so this class will be used as a client to communicate with yarn and start the job on yarn.
+ */
+public class YarnClusterClientV2 extends ClusterClient {
+
+	private static final Logger LOG = LoggerFactory.getLogger(YarnClusterClientV2.class);
+
+	private YarnClient yarnClient;
+
+	private final AbstractYarnClusterDescriptor clusterDescriptor;
+
+	private ApplicationId appId;
+
+	private String trackingURL;
+
+	/**
+	 * Create a client to communicate with YARN cluster.
+	 *
+	 * @param clusterDescriptor The descriptor used to create yarn job
+	 * @param flinkConfig Flink configuration
+	 * @throws java.io.IOException
+	 */
+	public YarnClusterClientV2(
+			final AbstractYarnClusterDescriptor clusterDescriptor,
+			org.apache.flink.configuration.Configuration flinkConfig) throws IOException {
+
+		super(flinkConfig);
+
+		this.clusterDescriptor = clusterDescriptor;
+		this.yarnClient = clusterDescriptor.getYarnClient();
+		this.trackingURL = "";
+	}
+
+	@Override
+	public org.apache.flink.configuration.Configuration getFlinkConfiguration() {
+		return flinkConfig;
+	}
+
+	@Override
+	public int getMaxSlots() {
+        // Now need not set max slot
+		return 0;
+	}
+
+	@Override
+	public boolean hasUserJarsInClassPath(List<URL> userJarFiles) {
+		return clusterDescriptor.hasUserJarFiles(userJarFiles);
+	}
+
+	@Override
+	protected JobSubmissionResult submitJob(JobGraph jobGraph, ClassLoader classLoader) throws ProgramInvocationException {
+		try {
+			// Create application via yarnClient
+			ApplicationReport report = this.clusterDescriptor.startAppMaster(jobGraph, yarnClient);
+			if (report.getYarnApplicationState().equals(YarnApplicationState.RUNNING)) {
+				appId = report.getApplicationId();
+				trackingURL = report.getTrackingUrl();
+				logAndSysout("Please refer to " + getWebInterfaceURL() 
+						+ " for the running status of job " +  jobGraph.getJobID().toString());
+				//TODO: not support attach mode now
+				return new JobSubmissionResult(jobGraph.getJobID());
+			}
+			else {
+				throw new ProgramInvocationException("Fail to submit the job.");
+			}
+		}
+		catch (Exception e) {
+			throw new ProgramInvocationException("Fail to submit the job", e.getCause());
+		}
+	}
+
+	@Override
+	public String getWebInterfaceURL() {
+		// there seems to be a difference between HD 2.2.0 and 2.6.0
+		if(!trackingURL.startsWith("http://")) {
+			return "http://" + trackingURL;
+		} else {
+			return trackingURL;
+		}
+	}
+
+	@Override
+	public String getClusterIdentifier() {
+		return "Yarn cluster with application id " + getApplicationId();
+	}
+
+	/**
+	 * This method is only available if the cluster hasn't been started in detached mode.
+	 */
+	@Override
+	public GetClusterStatusResponse getClusterStatus() {
+		throw new UnsupportedOperationException("Not support getClusterStatus since Flip-6.");
+	}
+
+	public ApplicationStatus getApplicationStatus() {
+		//TODO: this method is useful for later
+		return null;
+	}
+
+	@Override
+	public List<String> getNewMessages() {
+		throw new UnsupportedOperationException("Not support getNewMessages since Flip-6.");
+	}
+
+	@Override
+	public void finalizeCluster() {
+		throw new UnsupportedOperationException("Not support finalizeCluster since Flip-6.");
+	}
+
+	@Override
+	public boolean isDetached() {
+		return super.isDetached() || clusterDescriptor.isDetachedMode();
+	}
+
+	@Override
+	public void waitForClusterToBeReady() {
+		throw new UnsupportedOperationException("Not support waitForClusterToBeReady since Flip-6.");
+	}
+
+	@Override
+	public InetSocketAddress getJobManagerAddress() {
+		//TODO: just return a local address in order to be compatible with createClient in CliFrontend
+		return new InetSocketAddress("localhost", 0);
+	}
+
+	public ApplicationId getApplicationId() {
+		return appId;
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/3695a8e9/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptorV2.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptorV2.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptorV2.java
new file mode 100644
index 0000000..e3bd944
--- /dev/null
+++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptorV2.java
@@ -0,0 +1,34 @@
+/*
+ * 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.yarn;
+
+
+/**
+ * Implementation of {@link org.apache.flink.yarn.AbstractYarnClusterDescriptor} which is used to start the new application master for a job under flip-6.
+ * This implementation is now however tricky, since YarnClusterDescriptorV2 is related YarnClusterClientV2, but AbstractYarnClusterDescriptor is related
+ * to YarnClusterClient. We should let YarnClusterDescriptorV2 implements ClusterDescriptor<YarnClusterClientV2>.
+ * However, in order to use the code in AbstractYarnClusterDescriptor for setting environments and so on, we make YarnClusterDescriptorV2 as now.
+ */
+public class YarnClusterDescriptorV2 extends AbstractYarnClusterDescriptor {
+
+	@Override
+	protected Class<?> getApplicationMasterClass() {
+		return YarnFlinkApplicationMasterRunner.class;
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/3695a8e9/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnCLI.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnCLI.java b/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnCLI.java
new file mode 100644
index 0000000..ca5049c
--- /dev/null
+++ b/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnCLI.java
@@ -0,0 +1,253 @@
+/*
+ * 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.yarn.cli;
+
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.HelpFormatter;
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.Options;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.flink.client.cli.CliFrontendParser;
+import org.apache.flink.client.cli.CustomCommandLine;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.yarn.YarnClusterClientV2;
+import org.apache.flink.yarn.YarnClusterDescriptorV2;
+import org.apache.hadoop.fs.Path;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.UnsupportedEncodingException;
+import java.net.URL;
+import java.net.URLDecoder;
+import java.nio.charset.Charset;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.apache.flink.client.cli.CliFrontendParser.ADDRESS_OPTION;
+
+/**
+ * Class handling the command line interface to the YARN per job mode under flip-6.
+ */
+public class FlinkYarnCLI implements CustomCommandLine<YarnClusterClientV2> {
+	private static final Logger LOG = LoggerFactory.getLogger(FlinkYarnCLI.class);
+
+	/** The id for the CommandLine interface */
+	private static final String ID = "yarn";
+
+	private static final String YARN_DYNAMIC_PROPERTIES_SEPARATOR = "@@"; // this has to be a regex for String.split()
+
+	//------------------------------------ Command Line argument options -------------------------
+	// the prefix transformation is used by the CliFrontend static constructor.
+	private final Option QUEUE;
+	private final Option SHIP_PATH;
+	private final Option FLINK_JAR;
+	private final Option JM_MEMORY;
+	private final Option DETACHED;
+	private final Option ZOOKEEPER_NAMESPACE;
+
+	private final Options ALL_OPTIONS;
+
+	/**
+	 * Dynamic properties allow the user to specify additional configuration values with -D, such as
+	 *  -D fs.overwrite-files=true  -D taskmanager.network.numberOfBuffers=16368
+	 */
+	private final Option DYNAMIC_PROPERTIES;
+
+	//------------------------------------ Internal fields -------------------------
+	// use detach mode as default
+	private boolean detachedMode = true;
+
+	public FlinkYarnCLI(String shortPrefix, String longPrefix) {
+
+		QUEUE = new Option(shortPrefix + "qu", longPrefix + "queue", true, "Specify YARN queue.");
+		SHIP_PATH = new Option(shortPrefix + "t", longPrefix + "ship", true, "Ship files in the specified directory (t for transfer)");
+		FLINK_JAR = new Option(shortPrefix + "j", longPrefix + "jar", true, "Path to Flink jar file");
+		JM_MEMORY = new Option(shortPrefix + "jm", longPrefix + "jobManagerMemory", true, "Memory for JobManager Container [in MB]");
+		DYNAMIC_PROPERTIES = new Option(shortPrefix + "D", true, "Dynamic properties");
+		DETACHED = new Option(shortPrefix + "a", longPrefix + "attached", false, "Start attached");
+		ZOOKEEPER_NAMESPACE = new Option(shortPrefix + "z", longPrefix + "zookeeperNamespace", true, "Namespace to create the Zookeeper sub-paths for high availability mode");
+
+		ALL_OPTIONS = new Options();
+		ALL_OPTIONS.addOption(FLINK_JAR);
+		ALL_OPTIONS.addOption(JM_MEMORY);
+		ALL_OPTIONS.addOption(QUEUE);
+		ALL_OPTIONS.addOption(SHIP_PATH);
+		ALL_OPTIONS.addOption(DYNAMIC_PROPERTIES);
+		ALL_OPTIONS.addOption(DETACHED);
+		ALL_OPTIONS.addOption(ZOOKEEPER_NAMESPACE);
+	}
+
+	public YarnClusterDescriptorV2 createDescriptor(String defaultApplicationName, CommandLine cmd) {
+
+		YarnClusterDescriptorV2 yarnClusterDescriptor = new YarnClusterDescriptorV2();
+
+		// Jar Path
+		Path localJarPath;
+		if (cmd.hasOption(FLINK_JAR.getOpt())) {
+			String userPath = cmd.getOptionValue(FLINK_JAR.getOpt());
+			if (!userPath.startsWith("file://")) {
+				userPath = "file://" + userPath;
+			}
+			localJarPath = new Path(userPath);
+		} else {
+			LOG.info("No path for the flink jar passed. Using the location of "
+				+ yarnClusterDescriptor.getClass() + " to locate the jar");
+			String encodedJarPath =
+				yarnClusterDescriptor.getClass().getProtectionDomain().getCodeSource().getLocation().getPath();
+			try {
+				// we have to decode the url encoded parts of the path
+				String decodedPath = URLDecoder.decode(encodedJarPath, Charset.defaultCharset().name());
+				localJarPath = new Path(new File(decodedPath).toURI());
+			} catch (UnsupportedEncodingException e) {
+				throw new RuntimeException("Couldn't decode the encoded Flink dist jar path: " + encodedJarPath +
+					" Please supply a path manually via the -" + FLINK_JAR.getOpt() + " option.");
+			}
+		}
+
+		yarnClusterDescriptor.setLocalJarPath(localJarPath);
+
+		List<File> shipFiles = new ArrayList<>();
+		// path to directory to ship
+		if (cmd.hasOption(SHIP_PATH.getOpt())) {
+			String shipPath = cmd.getOptionValue(SHIP_PATH.getOpt());
+			File shipDir = new File(shipPath);
+			if (shipDir.isDirectory()) {
+				shipFiles.add(shipDir);
+			} else {
+				LOG.warn("Ship directory is not a directory. Ignoring it.");
+			}
+		}
+
+		yarnClusterDescriptor.addShipFiles(shipFiles);
+
+		// queue
+		if (cmd.hasOption(QUEUE.getOpt())) {
+			yarnClusterDescriptor.setQueue(cmd.getOptionValue(QUEUE.getOpt()));
+		}
+
+		// JobManager Memory
+		if (cmd.hasOption(JM_MEMORY.getOpt())) {
+			int jmMemory = Integer.valueOf(cmd.getOptionValue(JM_MEMORY.getOpt()));
+			yarnClusterDescriptor.setJobManagerMemory(jmMemory);
+		}
+
+		String[] dynamicProperties = null;
+		if (cmd.hasOption(DYNAMIC_PROPERTIES.getOpt())) {
+			dynamicProperties = cmd.getOptionValues(DYNAMIC_PROPERTIES.getOpt());
+		}
+		String dynamicPropertiesEncoded = StringUtils.join(dynamicProperties, YARN_DYNAMIC_PROPERTIES_SEPARATOR);
+
+		yarnClusterDescriptor.setDynamicPropertiesEncoded(dynamicPropertiesEncoded);
+
+		if (cmd.hasOption(DETACHED.getOpt()) || cmd.hasOption(CliFrontendParser.DETACHED_OPTION.getOpt())) {
+			// TODO: not support non detach mode now.
+			//this.detachedMode = false;
+		}
+		yarnClusterDescriptor.setDetachedMode(this.detachedMode);
+
+		if(defaultApplicationName != null) {
+			yarnClusterDescriptor.setName(defaultApplicationName);
+		}
+
+		if (cmd.hasOption(ZOOKEEPER_NAMESPACE.getOpt())) {
+			String zookeeperNamespace = cmd.getOptionValue(ZOOKEEPER_NAMESPACE.getOpt());
+			yarnClusterDescriptor.setZookeeperNamespace(zookeeperNamespace);
+		}
+
+		return yarnClusterDescriptor;
+	}
+
+	private void printUsage() {
+		System.out.println("Usage:");
+		HelpFormatter formatter = new HelpFormatter();
+		formatter.setWidth(200);
+		formatter.setLeftPadding(5);
+
+		formatter.setSyntaxPrefix("   Optional");
+		Options options = new Options();
+		addGeneralOptions(options);
+		addRunOptions(options);
+		formatter.printHelp(" ", options);
+	}
+
+	@Override
+	public boolean isActive(CommandLine commandLine, Configuration configuration) {
+		String jobManagerOption = commandLine.getOptionValue(ADDRESS_OPTION.getOpt(), null);
+		boolean yarnJobManager = ID.equals(jobManagerOption);
+		return yarnJobManager;
+	}
+
+	@Override
+	public String getId() {
+		return ID;
+	}
+
+	@Override
+	public void addRunOptions(Options baseOptions) {
+		for (Object option : ALL_OPTIONS.getOptions()) {
+			baseOptions.addOption((Option) option);
+		}
+	}
+
+	@Override
+	public void addGeneralOptions(Options baseOptions) {
+	}
+
+	@Override
+	public YarnClusterClientV2 retrieveCluster(
+			CommandLine cmdLine,
+			Configuration config) throws UnsupportedOperationException {
+
+		throw new UnsupportedOperationException("Not support retrieveCluster since Flip-6.");
+	}
+
+	@Override
+	public YarnClusterClientV2 createCluster(
+			String applicationName,
+			CommandLine cmdLine,
+			Configuration config,
+			List<URL> userJarFiles) {
+		Preconditions.checkNotNull(userJarFiles, "User jar files should not be null.");
+
+		YarnClusterDescriptorV2 yarnClusterDescriptor = createDescriptor(applicationName, cmdLine);
+		yarnClusterDescriptor.setFlinkConfiguration(config);
+		yarnClusterDescriptor.setProvidedUserJarFiles(userJarFiles);
+
+		YarnClusterClientV2 client = null;
+		try {
+			client = new YarnClusterClientV2(yarnClusterDescriptor, config);
+		}
+		catch (IOException e) {
+			throw new RuntimeException("Fail to create YarnClusterClientV2", e.getCause());
+		}
+		return client;
+
+	}
+
+	/**
+	 * Utility method
+	 */
+	private void logAndSysout(String message) {
+		LOG.info(message);
+		System.out.println(message);
+	}
+
+}


[02/52] [abbrv] 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/c6486067/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 b94f904..abc59cf 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,26 +18,19 @@
 
 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.CheckpointException;
 import org.apache.flink.runtime.checkpoint.CheckpointMetaData;
 import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory;
-import org.apache.flink.runtime.checkpoint.CompletedCheckpointStore;
 import org.apache.flink.runtime.checkpoint.SubtaskState;
-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;
@@ -48,9 +41,10 @@ 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.ExecutionGraphException;
+import org.apache.flink.runtime.executiongraph.ExecutionGraphBuilder;
 import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
 import org.apache.flink.runtime.executiongraph.IntermediateResult;
+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,16 +53,11 @@ 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.PartitionProducerDisposedException;
 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.TriggerSavepointResponse;
 import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener;
 import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
 import org.apache.flink.runtime.messages.Acknowledge;
@@ -84,22 +73,26 @@ 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.KeyGroupRange;
 import org.apache.flink.runtime.taskmanager.TaskExecutionState;
 import org.apache.flink.runtime.util.SerializedThrowable;
 import org.apache.flink.util.InstantiationUtil;
 import org.apache.flink.util.SerializedValue;
+
 import org.slf4j.Logger;
 
+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;
 
@@ -110,16 +103,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 */
@@ -128,37 +126,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;
 
@@ -168,22 +153,26 @@ 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,
+			Time rpcAskTimeout,
+			@Nullable JobManagerMetricGroup jobManagerMetricGroup,
+			OnCompletionActions jobCompletionActions,
+			FatalErrorHandler errorHandler,
+			ClassLoader userCodeLoader) throws Exception
 	{
 		super(rpcService);
 
@@ -191,293 +180,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);
-	}
-
-	//----------------------------------------------------------------------------------------------
-	// Lifecycle management
-	//----------------------------------------------------------------------------------------------
+		this.errorHandler = checkNotNull(errorHandler);
+		this.userCodeLoader = checkNotNull(userCodeLoader);
 
-	/**
-	 * 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());
+		final String jobName = jobGraph.getName();
+		final JobID jid = 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);
-			}
+		if (jobManagerMetricGroup != null) {
+			this.jobManagerMetricGroup = jobManagerMetricGroup;
+			this.jobMetricGroup = jobManagerMetricGroup.addJob(jobGraph);
+		} else {
+			this.jobManagerMetricGroup = new UnregisteredMetricsGroup();
+			this.jobMetricGroup = new UnregisteredMetricsGroup();
+		}
 
-			userCodeLoader = libraryCacheManager.getClassLoader(jobGraph.getJobID());
-			if (userCodeLoader == null) {
-				throw new JobSubmissionException(jobGraph.getJobID(),
-					"The user code class loader could not be initialized.");
-			}
+		log.info("Initializing job {} ({}).", jobName, jid);
 
-			if (jobGraph.getNumberOfVertices() == 0) {
-				throw new JobSubmissionException(jobGraph.getJobID(), "The given job is empty");
-			}
-
-			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());
-
-			if (t instanceof JobSubmissionException) {
-				throw (JobSubmissionException) t;
-			}
-			else {
-				throw new JobSubmissionException(jobGraph.getJobID(), "Failed to initialize job " +
-					jobGraph.getName() + " (" + jobGraph.getJobID() + ")", t);
-			}
 		}
+
+		this.executionGraph = ExecutionGraphBuilder.buildGraph(
+				null,
+				jobGraph,
+				configuration,
+				executorService,
+				executorService,
+				userCodeLoader,
+				checkpointRecoveryFactory,
+				rpcAskTimeout,
+				restartStrategy,
+				jobMetricGroup,
+				-1,
+				log);
+
+		// 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.
-	 */
-	@RpcMethod
-	public void startJob(final UUID leaderSessionID) {
-		log.info("Starting job {} ({}) with leaderId {}.", jobGraph.getName(), jobGraph.getJobID(), leaderSessionID);
+	//-- job starting and stopping  -----------------------------------------------------------------
 
-		this.leaderSessionID = leaderSessionID;
+	@RpcMethod
+	public void startJobExecution() {
+		log.info("Starting execution of job {} ({}) with leaderId {}.",
+				jobGraph.getName(), jobGraph.getJobID(), leaderSessionID);
 
 		try {
-			if (executionGraph != null) {
-				executionGraph = new ExecutionGraph(
-						executionContext,
-						executionContext,
-						jobGraph.getJobID(),
-						jobGraph.getName(),
-						jobGraph.getJobConfiguration(),
-						jobGraph.getSerializedExecutionConfig(),
-						timeout,
-						restartStrategy,
-						jobGraph.getUserJarBlobKeys(),
-						jobGraph.getClasspaths(),
-						userCodeLoader,
-						jobMetrics);
-			} else {
-				// TODO: update last active time in JobInfo
-			}
-
-			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);
-				}
-
-				String externalizedCheckpointsDir = configuration.getString(
-						ConfigConstants.CHECKPOINTS_DIRECTORY_KEY, null);
-
-				executionGraph.enableSnapshotCheckpointing(
-					snapshotSettings.getCheckpointInterval(),
-					snapshotSettings.getCheckpointTimeout(),
-					snapshotSettings.getMinPauseBetweenCheckpoints(),
-					snapshotSettings.getMaxConcurrentCheckpoints(),
-					snapshotSettings.getExternalizedCheckpointSettings(),
-					triggerVertices,
-					ackVertices,
-					confirmVertices,
-					checkpointIdCounter,
-					completedCheckpoints,
-					externalizedCheckpointsDir,
-					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
@@ -490,34 +336,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.
 	 *
@@ -525,26 +388,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,
@@ -583,7 +458,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;
 		}
@@ -591,16 +466,21 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 
 	@RpcMethod
 	public ExecutionState requestPartitionState(
-		final JobID ignored,
-		final IntermediateDataSetID intermediateResultId,
-		final ResultPartitionID resultPartitionId) throws PartitionProducerDisposedException {
+			final UUID leaderSessionID,
+			final IntermediateDataSetID intermediateResultId,
+			final ResultPartitionID resultPartitionId) 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(resultPartitionId.getProducerId());
 		if (execution != null) {
 			return execution.getState();
 		}
 		else {
-			final IntermediateResult intermediateResult = 
+			final IntermediateResult intermediateResult =
 					executionGraph.getAllIntermediateResults().get(intermediateResultId);
 
 			if (intermediateResult != null) {
@@ -623,7 +503,15 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 	}
 
 	@RpcMethod
-	public Acknowledge scheduleOrUpdateConsumers(ResultPartitionID partitionID) throws ExecutionGraphException {
+	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();
 	}
@@ -638,171 +526,118 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 			final JobID jobID,
 			final ExecutionAttemptID executionAttemptID,
 			final CheckpointMetaData checkpointInfo,
-			final SubtaskState checkpointStateHandles) {
+			final SubtaskState checkpointState) throws CheckpointException {
 
-		throw new UnsupportedOperationException();
+		final CheckpointCoordinator checkpointCoordinator = executionGraph.getCheckpointCoordinator();
+		final AcknowledgeCheckpoint ackMessage = 
+				new AcknowledgeCheckpoint(jobID, executionAttemptID, checkpointInfo, checkpointState);
+
+		if (checkpointCoordinator != null) {
+			getRpcService().execute(new Runnable() {
+				@Override
+				public void run() {
+					try {
+						checkpointCoordinator.receiveAcknowledgeMessage(ackMessage);
+					} catch (Throwable t) {
+						log.warn("Error while processing checkpoint acknowledgement message");
+					}
+				}
+			});
+		} else {
+			log.error("Received AcknowledgeCheckpoint message for job {} with no CheckpointCoordinator",
+					jobGraph.getJobID());
+		}
 	}
 
 	@RpcMethod
 	public void declineCheckpoint(
 			final JobID jobID,
 			final ExecutionAttemptID executionAttemptID,
-			final long checkpointId,
-			final Throwable cause) {
-
-		throw new UnsupportedOperationException();
-	}
-
-	//----------------------------------------------------------------------------------------------
-	// Internal methods
-	//----------------------------------------------------------------------------------------------
-
-	@RpcMethod
-	public void resourceRemoved(final ResourceID resourceId, final String message) {
-		// TODO: remove resource from slot pool
-	}
+			final long checkpointID,
+			final Throwable reason)
+	{
+		final DeclineCheckpoint decline = new DeclineCheckpoint(
+				jobID, executionAttemptID, checkpointID, reason);
+		final CheckpointCoordinator checkpointCoordinator = executionGraph.getCheckpointCoordinator();
 
-	@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);
-						}
+		if (checkpointCoordinator != null) {
+			getRpcService().execute(new Runnable() {
+				@Override
+				public void run() {
+					try {
+						checkpointCoordinator.receiveDeclineMessage(decline);
+					} catch (Exception e) {
+						log.error("Error in CheckpointCoordinator while processing {}", decline, 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 {
-							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",
+			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 " +
+		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 {
-		return null;
-	}
-
-	@RpcMethod
-	public DisposeSavepointResponse disposeSavepoint(final String savepointPath) {
-		// TODO
-		return null;
+		try {
+			executionGraph.getKvStateLocationRegistry().notifyKvStateUnregistered(
+					jobVertexId, keyGroupRange, registrationName);
+		} catch (Exception e) {
+			log.error("Failed to notify KvStateRegistry about registration {}.", registrationName);
+		}
 	}
 
 	@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());
-		}
 	}
 
 	//----------------------------------------------------------------------------------------------
@@ -815,12 +650,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();
@@ -848,36 +682,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);
@@ -886,7 +717,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() {
@@ -895,17 +726,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());
 					}
 				}
 
@@ -914,8 +743,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();
 				}
 			}
@@ -929,26 +758,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();
@@ -957,34 +774,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
 	//----------------------------------------------------------------------------------------------
 
@@ -1001,19 +790,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);
@@ -1023,12 +812,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/c6486067/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 5223b3e..daa33a3 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,7 +18,6 @@
 
 package org.apache.flink.runtime.jobmaster;
 
-import org.apache.flink.api.common.JobID;
 import org.apache.flink.api.common.time.Time;
 import org.apache.flink.runtime.checkpoint.CheckpointCoordinatorGateway;
 import org.apache.flink.runtime.clusterframework.types.ResourceID;
@@ -32,11 +31,7 @@ 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.TriggerSavepointResponse;
 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.query.KvStateID;
 import org.apache.flink.runtime.query.KvStateLocation;
 import org.apache.flink.runtime.query.KvStateServerAddress;
@@ -52,52 +47,54 @@ 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 producer of an intermediate result partition.
+	 * Requests the current state of the partition.
 	 * The state of a partition is currently bound to the state of the producing execution.
 	 *
-	 * @param jobId                TheID of job that the intermediate result partition belongs to.
+	 * @param leaderSessionID The leader id of JobManager
 	 * @param intermediateResultId The execution attempt ID of the task requesting the partition state.
 	 * @param partitionId          The partition ID of the partition to request the state of.
 	 * @return The future of the partition state
 	 */
 	Future<ExecutionState> requestPartitionState(
-			JobID jobId,
-			IntermediateDataSetID intermediateResultId,
-			ResultPartitionID partitionId);
+			final UUID leaderSessionID,
+			final IntermediateDataSetID intermediateResultId,
+			final ResultPartitionID partitionId);
 
 	/**
 	 * Notifies the JobManager about available data for a produced partition.
@@ -108,11 +105,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
@@ -123,36 +124,12 @@ public interface JobMasterGateway extends CheckpointCoordinatorGateway {
 	void disconnectTaskManager(ResourceID resourceID);
 
 	/**
-	 * 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;
+	Future<KvStateLocation> lookupKvStateLocation(final String registrationName);
 
 	/**
 	 * @param jobVertexId          JobVertexID the KvState instance belongs to.
@@ -162,11 +139,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.
@@ -174,24 +151,9 @@ public interface JobMasterGateway extends CheckpointCoordinatorGateway {
 	 * @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);
+			JobVertexID jobVertexId,
+			KeyGroupRange keyGroupRange,
+			String registrationName);
 
 	/**
 	 * Request the classloading props of this job.

http://git-wip-us.apache.org/repos/asf/flink/blob/c6486067/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/c6486067/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
index 520755d..572ba2f 100644
--- 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
@@ -23,7 +23,6 @@ import org.apache.flink.runtime.blob.BlobKey;
 import java.io.Serializable;
 import java.net.URL;
 import java.util.Collection;
-import java.util.List;
 
 /**
  * The response of classloading props request to JobManager.

http://git-wip-us.apache.org/repos/asf/flink/blob/c6486067/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/c6486067/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/c6486067/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/c6486067/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 72668d2..1b311e3 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 PartitionProducerStateChecker partitionStateChecker;
 
 	public JobManagerConnection(
-		JobMasterGateway jobMasterGateway,
-		TaskManagerActions taskManagerActions,
-		CheckpointResponder checkpointResponder,
-		LibraryCacheManager libraryCacheManager,
-		ResultPartitionConsumableNotifier resultPartitionConsumableNotifier,
-		PartitionProducerStateChecker partitionStateChecker) {
-
+			UUID jobMasterLeaderId,
+			JobMasterGateway jobMasterGateway,
+			TaskManagerActions taskManagerActions,
+			CheckpointResponder checkpointResponder,
+			LibraryCacheManager libraryCacheManager,
+			ResultPartitionConsumableNotifier resultPartitionConsumableNotifier,
+			PartitionProducerStateChecker 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;
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/c6486067/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 36f108e..2389291 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,37 +18,46 @@
 
 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.executiongraph.JobInformation;
-import org.apache.flink.runtime.executiongraph.TaskInformation;
-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;
 import org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager;
 import org.apache.flink.runtime.execution.librarycache.LibraryCacheManager;
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.executiongraph.JobInformation;
 import org.apache.flink.runtime.executiongraph.PartitionInfo;
+import org.apache.flink.runtime.executiongraph.TaskInformation;
 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.PartitionProducerStateChecker;
 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;
@@ -62,26 +71,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;
@@ -292,6 +291,7 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 				tdd.getAttemptNumber());
 
 		InputSplitProvider inputSplitProvider = new RpcInputSplitProvider(
+				jobManagerConnection.getJobMasterLeaderId(),
 				jobManagerConnection.getJobManagerGateway(),
 				jobInformation.getJobId(),
 				taskInformation.getJobVertexId(),
@@ -605,10 +605,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
@@ -620,7 +625,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) {
@@ -638,14 +647,15 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 			AccumulatorSnapshot accumulatorSnapshot = task.getAccumulatorRegistry().getSnapshot();
 
 			updateTaskExecutionState(
-				jobMasterGateway,
-				new TaskExecutionState(
-					task.getJobID(),
-					task.getExecutionId(),
-					task.getExecutionState(),
-					task.getFailureCause(),
-					accumulatorSnapshot,
-					task.getMetricGroup().getIOMetricGroup().createSnapshot()));
+					jobMasterLeaderId,
+					jobMasterGateway,
+					new TaskExecutionState(
+							task.getJobID(),
+							task.getExecutionId(),
+							task.getExecutionState(),
+							task.getFailureCause(),
+							accumulatorSnapshot,
+							task.getMetricGroup().getIOMetricGroup().createSnapshot()));
 		} else {
 			log.error("Cannot find task with ID {} to unregister.", executionAttemptID);
 		}
@@ -687,11 +697,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);
 
@@ -704,19 +717,21 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 			taskManagerConfiguration.getCleanupInterval());
 
 		ResultPartitionConsumableNotifier resultPartitionConsumableNotifier = new RpcResultPartitionConsumableNotifier(
-			jobMasterGateway,
-			getRpcService().getExecutor(),
-			taskManagerConfiguration.getTimeout());
+				jobMasterLeaderId,
+				jobMasterGateway,
+				getRpcService().getExecutor(),
+				taskManagerConfiguration.getTimeout());
 
-		PartitionProducerStateChecker partitionStateChecker = new RpcPartitionStateChecker(jobMasterGateway);
+		PartitionProducerStateChecker 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 {
@@ -808,9 +823,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);
 		}
 
@@ -819,7 +836,7 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 			runAsync(new Runnable() {
 				@Override
 				public void run() {
-					unregisterTaskAndNotifyFinalState(jobMasterGateway, executionAttemptID);
+					unregisterTaskAndNotifyFinalState(jobMasterLeaderId, jobMasterGateway, executionAttemptID);
 				}
 			});
 		}
@@ -842,7 +859,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/c6486067/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());


[29/52] [abbrv] flink git commit: [FLINK-4987] Harden SlotPool on JobMaster

Posted by se...@apache.org.
[FLINK-4987] Harden SlotPool on JobMaster


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

Branch: refs/heads/master
Commit: a1ba9f1126270d53168394211a99e354aa2cf20d
Parents: 8730e20
Author: Stephan Ewen <se...@apache.org>
Authored: Fri Oct 21 16:51:34 2016 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Dec 23 20:54:25 2016 +0100

----------------------------------------------------------------------
 .../clusterframework/types/ResourceProfile.java |   18 +-
 .../org/apache/flink/runtime/instance/Slot.java |   16 +
 .../flink/runtime/instance/SlotDescriptor.java  |  162 ---
 .../apache/flink/runtime/instance/SlotPool.java | 1026 +++++++++++-------
 .../flink/runtime/instance/SlotPoolGateway.java |   95 ++
 .../runtime/jobmanager/scheduler/Locality.java  |   26 +-
 .../scheduler/NoResourceAvailableException.java |    6 +-
 .../runtime/jobmanager/slots/AllocatedSlot.java |   58 +-
 .../jobmanager/slots/PooledSlotProvider.java    |   73 --
 .../jobmanager/slots/SlotAndLocality.java       |   55 +
 .../flink/runtime/jobmaster/JobMaster.java      |  102 +-
 .../apache/flink/runtime/rpc/RpcEndpoint.java   |   18 +-
 .../apache/flink/runtime/util/clock/Clock.java  |   40 +
 .../flink/runtime/util/clock/SystemClock.java   |   57 +
 .../types/ResourceProfileTest.java              |    5 +
 .../runtime/instance/AllocatedSlotsTest.java    |  270 ++---
 .../runtime/instance/AvailableSlotsTest.java    |  247 +++--
 .../flink/runtime/instance/SlotPoolTest.java    |  596 +++++-----
 .../runtime/minicluster/MiniClusterITCase.java  |   13 +-
 19 files changed, 1650 insertions(+), 1233 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/a1ba9f11/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 7a25de1..ddc7547 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
@@ -18,14 +18,21 @@
 
 package org.apache.flink.runtime.clusterframework.types;
 
+import javax.annotation.Nonnull;
 import java.io.Serializable;
 
 /**
  * Describe the resource profile of the slot, either when requiring or offering it. The profile can be
  * checked whether it can match another profile's requirement, and furthermore we may calculate a matching
  * score to decide which profile we should choose when we have lots of candidate slots.
+ * 
+ * <p>Resource Profiles have a total ordering, defined by comparing these fields in sequence:
+ * <ol>
+ *     <li>Memory Size</li>
+ *     <li>CPU cores</li>
+ * </ol>
  */
-public class ResourceProfile implements Serializable {
+public class ResourceProfile implements Serializable, Comparable<ResourceProfile> {
 
 	private static final long serialVersionUID = 1L;
 
@@ -90,11 +97,18 @@ public class ResourceProfile implements Serializable {
 		return cpuCores >= required.getCpuCores() && memoryInMB >= required.getMemoryInMB();
 	}
 
+	@Override
+	public int compareTo(@Nonnull ResourceProfile other) {
+		int cmp1 = Long.compare(this.memoryInMB, other.memoryInMB);
+		int cmp2 = Double.compare(this.cpuCores, other.cpuCores);
+		return (cmp1 != 0) ? cmp1 : cmp2; 
+	}
+
 	// ------------------------------------------------------------------------
 
 	@Override
 	public int hashCode() {
-		long cpuBits = Double.doubleToLongBits(cpuCores);
+		long cpuBits = Double.doubleToRawLongBits(cpuCores);
 		return (int) (cpuBits ^ (cpuBits >>> 32) ^ memoryInMB ^ (memoryInMB >> 32));
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/a1ba9f11/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Slot.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Slot.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Slot.java
index 8f8b897..d6d8f12 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Slot.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Slot.java
@@ -350,6 +350,22 @@ public abstract class Slot {
 	//  Utilities
 	// --------------------------------------------------------------------------------------------
 
+	/**
+	 * Slots must always has based on reference identity.
+	 */
+	@Override
+	public final int hashCode() {
+		return super.hashCode();
+	}
+
+	/**
+	 * Slots must always compare on referential equality.
+	 */
+	@Override
+	public final boolean equals(Object obj) {
+		return this == obj;
+	}
+
 	@Override
 	public String toString() {
 		return hierarchy() + " - " + getTaskManagerLocation() + " - " + getStateName(status);

http://git-wip-us.apache.org/repos/asf/flink/blob/a1ba9f11/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
deleted file mode 100644
index 47ce422..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotDescriptor.java
+++ /dev/null
@@ -1,162 +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.instance;
-
-import org.apache.flink.api.common.JobID;
-import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
-import org.apache.flink.runtime.jobmanager.slots.TaskManagerGateway;
-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 TaskManagerGateway taskManagerGateway;
-
-	public SlotDescriptor(
-		final JobID jobID,
-		final TaskManagerLocation location,
-		final int slotNumber,
-		final ResourceProfile resourceProfile,
-		final TaskManagerGateway taskManagerGateway)
-	{
-		this.jobID = checkNotNull(jobID);
-		this.taskManagerLocation = checkNotNull(location);
-		this.slotNumber = slotNumber;
-		this.resourceProfile = checkNotNull(resourceProfile);
-		this.taskManagerGateway = checkNotNull(taskManagerGateway);
-	}
-
-	public SlotDescriptor(final SlotDescriptor other) {
-		this.jobID = other.jobID;
-		this.taskManagerLocation = other.taskManagerLocation;
-		this.slotNumber = other.slotNumber;
-		this.resourceProfile = other.resourceProfile;
-		this.taskManagerGateway = other.taskManagerGateway;
-	}
-	
-	// 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.taskManagerGateway = slot.getTaskManagerGateway();
-	}
-
-	/**
-	 * 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 TaskManagerGateway getTaskManagerGateway() {
-		return taskManagerGateway;
-	}
-
-	@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/a1ba9f11/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
index 44df29b..5a3a321 100644
--- 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
@@ -25,25 +25,41 @@ 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.AcceptFunction;
 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.scheduler.Locality;
+import org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException;
+import org.apache.flink.runtime.jobmanager.scheduler.ScheduledUnit;
+import org.apache.flink.runtime.jobmanager.slots.AllocatedSlot;
+import org.apache.flink.runtime.jobmanager.slots.SlotAndLocality;
 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.RMSlotRequestRegistered;
 import org.apache.flink.runtime.resourcemanager.messages.jobmanager.RMSlotRequestRejected;
 import org.apache.flink.runtime.resourcemanager.messages.jobmanager.RMSlotRequestReply;
+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.taskexecutor.slot.SlotOffer;
+import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
+import org.apache.flink.runtime.util.clock.Clock;
+import org.apache.flink.runtime.util.clock.SystemClock;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.util.ArrayList;
+import java.util.Collections;
 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 java.util.concurrent.TimeoutException;
 
 import static org.apache.flink.util.Preconditions.checkArgument;
 import static org.apache.flink.util.Preconditions.checkNotNull;
@@ -58,18 +74,33 @@ import static org.apache.flink.util.Preconditions.checkNotNull;
  * <p>
  * All the allocation or the slot offering will be identified by self generated AllocationID, we will use it to
  * eliminate ambiguities.
+ * 
+ * TODO : Make pending requests location preference aware
+ * TODO : Make pass location preferences to ResourceManager when sending a slot request
  */
-public class SlotPool implements SlotOwner {
+public class SlotPool extends RpcEndpoint<SlotPoolGateway> {
+
+	/** The log for the pool - shared also with the internal classes */
+	static final Logger LOG = LoggerFactory.getLogger(SlotPool.class);
+
+	// ------------------------------------------------------------------------
 
-	private static final Logger LOG = LoggerFactory.getLogger(SlotPool.class);
+	private static final Time DEFAULT_SLOT_REQUEST_TIMEOUT = Time.minutes(5);
+
+	private static final Time DEFAULT_RM_ALLOCATION_TIMEOUT = Time.minutes(10);
+
+	private static final Time DEFAULT_RM_REQUEST_TIMEOUT = Time.seconds(10);
+
+	// ------------------------------------------------------------------------
 
 	private final Object lock = new Object();
 
-	/** The executor which is used to execute futures */
-	private final Executor executor;
+	private final JobID jobId;
 
-	/** All registered resources, slots will be accepted and used only if the resource is registered */
-	private final Set<ResourceID> registeredResources;
+	private final ProviderAndOwner providerAndOwner;
+
+	/** All registered TaskManagers, slots will be accepted and used only if the resource is registered */
+	private final HashSet<ResourceID> registeredTaskManagers;
 
 	/** The book-keeping of all allocated slots */
 	private final AllocatedSlots allocatedSlots;
@@ -78,10 +109,15 @@ public class SlotPool implements SlotOwner {
 	private final AvailableSlots availableSlots;
 
 	/** All pending requests waiting for slots */
-	private final Map<AllocationID, Tuple2<SlotRequest, FlinkCompletableFuture<SlotDescriptor>>> pendingRequests;
+	private final HashMap<AllocationID, PendingRequest> pendingRequests;
+
+	/** Timeout for request calls to the ResourceManager */
+	private final Time resourceManagerRequestsTimeout;
 
-	/** Timeout of slot allocation */
-	private final Time timeout;
+	/** Timeout for allocation round trips (RM -> launch TM -> offer slot) */
+	private final Time resourceManagerAllocationTimeout;
+
+	private final Clock clock;
 
 	/** the leader id of job manager */
 	private UUID jobManagerLeaderId;
@@ -92,177 +128,238 @@ public class SlotPool implements SlotOwner {
 	/** The gateway to communicate with resource manager */
 	private ResourceManagerGateway resourceManagerGateway;
 
-	public SlotPool(final Executor executor) {
-		this.executor = executor;
-		this.registeredResources = new HashSet<>();
+	// ------------------------------------------------------------------------
+
+	public SlotPool(RpcService rpcService, JobID jobId) {
+		this(rpcService, jobId, SystemClock.getInstance(),
+				DEFAULT_SLOT_REQUEST_TIMEOUT, DEFAULT_RM_ALLOCATION_TIMEOUT, DEFAULT_RM_REQUEST_TIMEOUT);
+	}
+
+	public SlotPool(
+			RpcService rpcService,
+			JobID jobId,
+			Clock clock,
+			Time slotRequestTimeout,
+			Time resourceManagerAllocationTimeout,
+			Time resourceManagerRequestTimeout) {
+
+		super(rpcService);
+
+		this.jobId = checkNotNull(jobId);
+		this.clock = checkNotNull(clock);
+		this.resourceManagerRequestsTimeout = checkNotNull(resourceManagerRequestTimeout);
+		this.resourceManagerAllocationTimeout = checkNotNull(resourceManagerAllocationTimeout);
+
+		this.registeredTaskManagers = 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;
+		this.providerAndOwner = new ProviderAndOwner(getSelf(), slotRequestTimeout);
 	}
 
 	// ------------------------------------------------------------------------
-	//  Slot Allocation
+	//  Starting and Stopping
 	// ------------------------------------------------------------------------
 
+	@Override
+	public void start() {
+		throw new UnsupportedOperationException("Should never call start() without leader ID");
+	}
+
 	/**
-	 * Try to allocate a simple slot with specified resource profile.
+	 * Start the slot pool to accept RPC calls.
 	 *
-	 * @param jobID           The job id which the slot allocated for
-	 * @param resourceProfile The needed resource profile
-	 * @return The future of allocated simple slot
+	 * @param jobManagerLeaderId The necessary leader id for running the job.
 	 */
-	public Future<SimpleSlot> allocateSimpleSlot(final JobID jobID, final ResourceProfile resourceProfile) {
-		return allocateSimpleSlot(jobID, resourceProfile, new AllocationID());
-	}
+	public void start(UUID jobManagerLeaderId) {
+		this.jobManagerLeaderId = jobManagerLeaderId;
 
+		// TODO - start should not throw an exception
+		try {
+			super.start();
+		} catch (Exception e) {
+			throw new RuntimeException("This should never happen", e);
+		}
+	}
 
 	/**
-	 * 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.
+	 * Suspends this pool, meaning it has lost its authority to accept and distribute slots.
 	 */
-	@VisibleForTesting
-	Future<SimpleSlot> allocateSimpleSlot(
-		final JobID jobID,
-		final ResourceProfile resourceProfile,
-		final AllocationID allocationID)
-	{
-		final FlinkCompletableFuture<SlotDescriptor> future = new FlinkCompletableFuture<>();
-
-		internalAllocateSlot(jobID, allocationID, resourceProfile, future);
-
-		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.getTaskManagerGateway());
-					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
-		);
+	@RpcMethod
+	public void suspend() {
+		validateRunsInMainThread();
+
+		// suspend this RPC endpoint
+		((StartStoppable) getSelf()).stop();
+
+		// do not accept any requests
+		jobManagerLeaderId = null;
+		resourceManagerLeaderId = null;
+		resourceManagerGateway = null;
+
+		// Clear (but not release!) the available slots. The TaskManagers should re-register them
+		// at the new leader JobManager/SlotPool
+		availableSlots.clear();
+		allocatedSlots.clear();
+		pendingRequests.clear();
 	}
 
+	// ------------------------------------------------------------------------
+	//  Getting PoolOwner and PoolProvider
+	// ------------------------------------------------------------------------
 
 	/**
-	 * 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
+	 * Gets the slot owner implementation for this pool.
+	 * 
+	 * <p>This method does not mutate state and can be called directly (no RPC indirection)
+	 * 
+	 * @return The slot owner implementation for this pool.
 	 */
-	public Future<SharedSlot> allocateSharedSlot(
-		final JobID jobID,
-		final ResourceProfile resourceProfile,
-		final SlotSharingGroupAssignment sharingGroupAssignment)
-	{
-		return allocateSharedSlot(jobID, resourceProfile, sharingGroupAssignment, new AllocationID());
+	public SlotOwner getSlotOwner() {
+		return providerAndOwner;
 	}
 
 	/**
-	 * 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.
+	 * Gets the slot provider implementation for this pool.
+	 *
+	 * <p>This method does not mutate state and can be called directly (no RPC indirection)
+	 *
+	 * @return The slot provider implementation for this pool.
 	 */
-	@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.getTaskManagerGateway(),
-							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
-		);
+	public SlotProvider getSlotProvider() {
+		return providerAndOwner;
 	}
 
-	/**
-	 * 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);
+	// ------------------------------------------------------------------------
+	//  Resource Manager Connection
+	// ------------------------------------------------------------------------
+
+	@RpcMethod
+	public void connectToResourceManager(UUID resourceManagerLeaderId, ResourceManagerGateway resourceManagerGateway) {
+		this.resourceManagerLeaderId = checkNotNull(resourceManagerLeaderId);
+		this.resourceManagerGateway = checkNotNull(resourceManagerGateway);
+	}
+
+	@RpcMethod
+	public void disconnectResourceManager() {
+		this.resourceManagerLeaderId = null;
+		this.resourceManagerGateway = null;
+	}
+
+	// ------------------------------------------------------------------------
+	//  Slot Allocation
+	// ------------------------------------------------------------------------
+
+	@RpcMethod
+	public Future<SimpleSlot> allocateSlot(
+			ScheduledUnit task,
+			ResourceProfile resources,
+			Iterable<TaskManagerLocation> locationPreferences) {
+
+		return internalAllocateSlot(task, resources, locationPreferences);
+	}
+
+	@RpcMethod
+	public void returnAllocatedSlot(Slot slot) {
+		internalReturnAllocatedSlot(slot);
+	}
+
+
+	Future<SimpleSlot> internalAllocateSlot(
+			ScheduledUnit task,
+			ResourceProfile resources,
+			Iterable<TaskManagerLocation> locationPreferences) {
+
+		// (1) do we have a slot available already?
+		SlotAndLocality slotFromPool = availableSlots.poll(resources, locationPreferences);
+		if (slotFromPool != null) {
+			SimpleSlot slot = createSimpleSlot(slotFromPool.slot(), slotFromPool.locality());
+			allocatedSlots.add(slot);
+			return FlinkCompletableFuture.completed(slot);
+		}
+
+		// (2) no slot available, and no resource manager connection
+		if (resourceManagerGateway == null) {
+			return FlinkCompletableFuture.completedExceptionally(
+					new NoResourceAvailableException("not connected to ResourceManager and no slot available"));
+			
+		}
+
+		// (3) we have a resource manager connection, so let's ask it for more resources
+		final FlinkCompletableFuture<SimpleSlot> future = new FlinkCompletableFuture<>();
+		final AllocationID allocationID = new AllocationID();
+
+		LOG.info("Requesting slot with profile {} from resource manager (request = {}).", resources, allocationID);
+
+		pendingRequests.put(allocationID, new PendingRequest(allocationID, future, resources));
+
+		Future<RMSlotRequestReply> rmResponse = resourceManagerGateway.requestSlot(
+				resourceManagerLeaderId, jobManagerLeaderId,
+				new SlotRequest(jobId, allocationID, resources),
+				resourceManagerRequestsTimeout);
+
+		// on success, trigger let the slot pool know
+		rmResponse.thenAcceptAsync(new AcceptFunction<RMSlotRequestReply>() {
+			@Override
+			public void accept(RMSlotRequestReply reply) {
+				if (reply.getAllocationID() != null && reply.getAllocationID().equals(allocationID)) {
+					if (reply instanceof RMSlotRequestRegistered) {
+						slotRequestToResourceManagerSuccess(allocationID);
+					}
+					else if (reply instanceof RMSlotRequestRejected) {
+						slotRequestToResourceManagerFailed(allocationID,
+								new Exception("ResourceManager rejected slot request"));
+					}
+					else {
+						slotRequestToResourceManagerFailed(allocationID, 
+								new Exception("Unknown ResourceManager response: " + reply));
+					}
 				}
 				else {
-					LOG.warn("Allocation[{}] Resource manager not available right now.", allocationID);
-					future.completeExceptionally(new Exception("Resource manager not available right now."));
+					future.completeExceptionally(new Exception(String.format(
+							"Bug: ResourceManager response had wrong AllocationID. Request: %s , Response: %s", 
+							allocationID, reply.getAllocationID())));
 				}
 			}
+		}, getMainThreadExecutor());
+
+		// on failure, fail the request future
+		rmResponse.exceptionallyAsync(new ApplyFunction<Throwable, Void>() {
+
+			@Override
+			public Void apply(Throwable failure) {
+				slotRequestToResourceManagerFailed(allocationID, failure);
+				return null;
+			}
+		}, getMainThreadExecutor());
+
+		return future;
+	}
+
+	private void slotRequestToResourceManagerSuccess(final AllocationID allocationID) {
+		// a request is pending from the ResourceManager to a (future) TaskManager
+		// we only add the watcher here in case that request times out
+		scheduleRunAsync(new Runnable() {
+			@Override
+			public void run() {
+				checkTimeoutSlotAllocation(allocationID);
+			}
+		}, resourceManagerAllocationTimeout);
+	}
+
+	private void slotRequestToResourceManagerFailed(AllocationID allocationID, Throwable failure) {
+		PendingRequest request = pendingRequests.remove(allocationID);
+		if (request != null) {
+			request.future().completeExceptionally(new NoResourceAvailableException(
+					"No pooled slot available and request to ResourceManager for new slot failed", failure));
+		}
+	}
+
+	private void checkTimeoutSlotAllocation(AllocationID allocationID) {
+		PendingRequest request = pendingRequests.remove(allocationID);
+		if (request != null && !request.future().isDone()) {
+			request.future().completeExceptionally(new TimeoutException("Slot allocation request timed out"));
 		}
 	}
 
@@ -275,123 +372,123 @@ public class SlotPool implements SlotOwner {
 	 * 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) {
+	private void internalReturnAllocatedSlot(Slot slot) {
 		checkNotNull(slot);
 		checkArgument(!slot.isAlive(), "slot is still alive");
-		checkArgument(slot.getOwner() == this, "slot belongs to the wrong pool.");
+		checkArgument(slot.getOwner() == providerAndOwner, "slot belongs to the wrong pool.");
 
+		// markReleased() is an atomic check-and-set operation, so that the slot is guaranteed
+		// to be returned only once
 		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;
+			if (allocatedSlots.remove(slot)) {
+				// this slot allocation is still valid, use the slot to fulfill another request
+				// or make it available again
+				final AllocatedSlot taskManagerSlot = slot.getAllocatedSlot();
+				final PendingRequest pendingRequest = pollMatchingPendingRequest(taskManagerSlot);
+	
+				if (pendingRequest != null) {
+					LOG.debug("Fulfilling pending request [{}] early with returned slot [{}]",
+							pendingRequest.allocationID(), taskManagerSlot.getSlotAllocationId());
+
+					pendingRequest.future().complete(createSimpleSlot(taskManagerSlot, Locality.UNKNOWN));
 				}
 				else {
-					throw new IllegalArgumentException("Slot was not allocated from this pool.");
+					LOG.debug("Adding returned slot [{}] to available slots", taskManagerSlot.getSlotAllocationId());
+					availableSlots.add(taskManagerSlot, clock.relativeTimeMillis());
 				}
 			}
-		}
-		else {
-			return false;
+			else {
+				LOG.debug("Returned slot's allocation has been failed. Dropping slot.");
+			}
 		}
 	}
 
-	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;
+	private PendingRequest pollMatchingPendingRequest(final AllocatedSlot slot) {
+		final ResourceProfile slotResources = slot.getResourceProfile();
+
+		for (PendingRequest request : pendingRequests.values()) {
+			if (slotResources.isMatching(request.resourceProfile())) {
+				pendingRequests.remove(request.allocationID());
+				return request;
 			}
 		}
+
+		// no request pending, or no request matches
 		return null;
 	}
 
-	/**
-	 * 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
+	@RpcMethod
+	public Iterable<SlotOffer> offerSlots(Iterable<Tuple2<AllocatedSlot, SlotOffer>> offers) {
+		validateRunsInMainThread();
+
+		final ArrayList<SlotOffer> result = new ArrayList<>();
+		for (Tuple2<AllocatedSlot, SlotOffer> offer : offers) {
+			if (offerSlot(offer.f0)) {
+				result.add(offer.f1);
+			}
 		}
-	}
 
+		return result.isEmpty() ? Collections.<SlotOffer>emptyList() : result;
+	}
+	
 	/**
 	 * 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
+	 * @param slot The offered slot
 	 * @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;
-			}
+	@RpcMethod
+	public boolean offerSlot(final AllocatedSlot slot) {
+		validateRunsInMainThread();
 
-			// check whether we have already using this slot
-			final Slot allocatedSlot = allocatedSlots.get(allocationID);
-			if (allocatedSlot != null) {
-				final SlotDescriptor allocatedSlotDescriptor = new SlotDescriptor(allocatedSlot);
+		// check if this TaskManager is valid
+		final ResourceID resourceID = slot.getTaskManagerId();
+		final AllocationID allocationID = slot.getSlotAllocationId();
 
-				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;
-				}
-			}
+		if (!registeredTaskManagers.contains(resourceID)) {
+			LOG.debug("Received outdated slot offering [{}] from unregistered TaskManager: {}",
+					slot.getSlotAllocationId(), slot);
+			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 already using this slot
+		if (allocatedSlots.contains(allocationID) || availableSlots.contains(allocationID)) {
+			LOG.debug("Received repeated offer for slot [{}]. Ignoring.", allocationID);
 
-			// 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;
-			}
+			// return true here so that the sender will get a positive acknowledgement to the retry
+			// and mark the offering as a success
+			return true;
+		}
 
-			// unwanted slot, rejecting this offer
-			return false;
+		// check whether we have request waiting for this slot
+		PendingRequest pendingRequest = pendingRequests.remove(allocationID);
+		if (pendingRequest != null) {
+			// we were waiting for this!
+			SimpleSlot resultSlot = createSimpleSlot(slot, Locality.UNKNOWN);
+			pendingRequest.future().complete(resultSlot);
+			allocatedSlots.add(resultSlot);
+		}
+		else {
+			// we were actually not waiting for this:
+			//   - could be that this request had been fulfilled
+			//   - we are receiving the slots from TaskManagers after becoming leaders
+			availableSlots.add(slot, clock.relativeTimeMillis());
 		}
+
+		// we accepted the request in any case. slot will be released after it idled for
+		// too long and timed out
+		return true;
 	}
 
+	
+	// TODO - periodic (every minute or so) catch slots that were lost (check all slots, if they have any task active)
+
+	// TODO - release slots that were not used to the resource manager
+
 	// ------------------------------------------------------------------------
 	//  Error Handling
 	// ------------------------------------------------------------------------
@@ -405,24 +502,29 @@ public class SlotPool implements SlotOwner {
 	 * @param allocationID Represents the allocation which should be failed
 	 * @param cause        The cause of the failure
 	 */
+	@RpcMethod
 	public void failAllocation(final AllocationID allocationID, final Exception cause) {
-		synchronized (lock) {
-			// 1. check whether the allocation still pending
-			Tuple2<SlotRequest, FlinkCompletableFuture<SlotDescriptor>> pendingRequest =
-					pendingRequests.get(allocationID);
-			if (pendingRequest != null) {
-				pendingRequest.f1.completeExceptionally(cause);
-				return;
+		final PendingRequest pendingRequest = pendingRequests.remove(allocationID);
+		if (pendingRequest != null) {
+			// request was still pending
+			LOG.debug("Failed pending request [{}] with ", allocationID, cause);
+			pendingRequest.future().completeExceptionally(cause);
+		}
+		else if (availableSlots.tryRemove(allocationID)) {
+			LOG.debug("Failed available slot [{}] with ", allocationID, cause);
+		}
+		else {
+			Slot slot = allocatedSlots.remove(allocationID);
+			if (slot != null) {
+				// release the slot.
+				// since it is not in 'allocatedSlots' any more, it will be dropped o return'
+				slot.releaseSlot();
+			}
+			else {
+				LOG.debug("Outdated request to fail slot [{}] with ", allocationID, cause);
 			}
-
-			// 2. check whether we have a free slot corresponding to this allocation id
-			// TODO: add allocation id to slot descriptor, so we can remove it by allocation id
-
-			// 3. check whether we have a in-use slot corresponding to this allocation id
-			// TODO: needs mechanism to release the in-use Slot but don't return it back to this pool
-
-			// TODO: add some unit tests when the previous two are ready, the allocation may failed at any phase
 		}
+		// TODO: add some unit tests when the previous two are ready, the allocation may failed at any phase
 	}
 
 	// ------------------------------------------------------------------------
@@ -435,10 +537,9 @@ public class SlotPool implements SlotOwner {
 	 *
 	 * @param resourceID The id of the TaskManager
 	 */
-	public void registerResource(final ResourceID resourceID) {
-		synchronized (lock) {
-			registeredResources.add(resourceID);
-		}
+	@RpcMethod
+	public void registerTaskManager(final ResourceID resourceID) {
+		registeredTaskManagers.add(resourceID);
 	}
 
 	/**
@@ -447,12 +548,12 @@ public class SlotPool implements SlotOwner {
 	 *
 	 * @param resourceID The id of the TaskManager
 	 */
-	public void releaseResource(final ResourceID resourceID) {
-		synchronized (lock) {
-			registeredResources.remove(resourceID);
-			availableSlots.removeByResource(resourceID);
+	@RpcMethod
+	public void releaseTaskManager(final ResourceID resourceID) {
+		if (registeredTaskManagers.remove(resourceID)) {
+			availableSlots.removeAllForTaskManager(resourceID);
 
-			final Set<Slot> allocatedSlotsForResource = allocatedSlots.getSlotsByResource(resourceID);
+			final Set<Slot> allocatedSlotsForResource = allocatedSlots.removeSlotsForTaskManager(resourceID);
 			for (Slot slot : allocatedSlotsForResource) {
 				slot.releaseSlot();
 			}
@@ -460,24 +561,15 @@ public class SlotPool implements SlotOwner {
 	}
 
 	// ------------------------------------------------------------------------
-	//  ResourceManager
+	//  Utilities
 	// ------------------------------------------------------------------------
 
-	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;
+	private SimpleSlot createSimpleSlot(AllocatedSlot slot, Locality locality) {
+		SimpleSlot result = new SimpleSlot(slot, providerAndOwner, slot.getSlotNumber());
+		if (locality != null) {
+			result.setLocality(locality);
 		}
+		return result;
 	}
 
 	// ------------------------------------------------------------------------
@@ -487,45 +579,34 @@ public class SlotPool implements SlotOwner {
 	/**
 	 * Organize allocated slots from different points of view.
 	 */
-	static class AllocatedSlots {
+	private static class AllocatedSlots {
 
 		/** All allocated slots organized by TaskManager's id */
-		private final Map<ResourceID, Set<Slot>> allocatedSlotsByResource;
-
-		/** All allocated slots organized by Slot object */
-		private final Map<Slot, AllocationID> allocatedSlots;
-
-		/** All allocated slot descriptors organized by Slot object */
-		private final Map<Slot, SlotDescriptor> allocatedSlotsWithDescriptor;
+		private final Map<ResourceID, Set<Slot>> allocatedSlotsByTaskManager;
 
 		/** 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.allocatedSlotsByTaskManager = new HashMap<>();
 			this.allocatedSlotsById = new HashMap<>();
 		}
 
 		/**
-		 * Add a new allocation
+		 * Adds a new slot to this collection.
 		 *
-		 * @param allocationID The allocation id
-		 * @param slot         The allocated slot
+		 * @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);
+		void add(Slot slot) {
+			allocatedSlotsById.put(slot.getAllocatedSlot().getSlotAllocationId(), slot);
 
 			final ResourceID resourceID = slot.getTaskManagerID();
-			Set<Slot> slotsForResource = allocatedSlotsByResource.get(resourceID);
-			if (slotsForResource == null) {
-				slotsForResource = new HashSet<>();
-				allocatedSlotsByResource.put(resourceID, slotsForResource);
+			Set<Slot> slotsForTaskManager = allocatedSlotsByTaskManager.get(resourceID);
+			if (slotsForTaskManager == null) {
+				slotsForTaskManager = new HashSet<>();
+				allocatedSlotsByTaskManager.put(resourceID, slotsForTaskManager);
 			}
-			slotsForResource.add(slot);
+			slotsForTaskManager.add(slot);
 		}
 
 		/**
@@ -541,11 +622,11 @@ public class SlotPool implements SlotOwner {
 		/**
 		 * Check whether we have allocated this slot
 		 *
-		 * @param slot The slot needs to checked
+		 * @param slotAllocationId The allocation id of the slot to check
 		 * @return True if we contains this slot
 		 */
-		boolean contains(final Slot slot) {
-			return allocatedSlots.containsKey(slot);
+		boolean contains(AllocationID slotAllocationId) {
+			return allocatedSlotsById.containsKey(slotAllocationId);
 		}
 
 		/**
@@ -553,25 +634,27 @@ public class SlotPool implements SlotOwner {
 		 *
 		 * @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");
-				}
+		boolean remove(final Slot slot) {
+			return remove(slot.getAllocatedSlot().getSlotAllocationId()) != null;
+		}
 
-				final ResourceID resourceID = slot.getTaskManagerID();
-				final Set<Slot> slotsForResource = allocatedSlotsByResource.get(resourceID);
-				slotsForResource.remove(slot);
-				if (slotsForResource.isEmpty()) {
-					allocatedSlotsByResource.remove(resourceID);
+		/**
+		 * Remove an allocation with slot.
+		 *
+		 * @param slotId The ID of the slot to be removed
+		 */
+		Slot remove(final AllocationID slotId) {
+			Slot slot = allocatedSlotsById.remove(slotId);
+			if (slot != null) {
+				final ResourceID taskManagerId = slot.getTaskManagerID();
+				Set<Slot> slotsForTM = allocatedSlotsByTaskManager.get(taskManagerId);
+				slotsForTM.remove(slot);
+				if (slotsForTM.isEmpty()) {
+					allocatedSlotsByTaskManager.get(taskManagerId);
 				}
-				
-				return descriptor;
-			} else {
+				return slot;
+			}
+			else {
 				return null;
 			}
 		}
@@ -582,119 +665,326 @@ public class SlotPool implements SlotOwner {
 		 * @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);
+		Set<Slot> removeSlotsForTaskManager(final ResourceID resourceID) {
+			Set<Slot> slotsForTaskManager = allocatedSlotsByTaskManager.remove(resourceID);
+			if (slotsForTaskManager != null) {
+				for (Slot slot : slotsForTaskManager) {
+					allocatedSlotsById.remove(slot.getAllocatedSlot().getSlotAllocationId());
+				}
+				return slotsForTaskManager;
 			}
 			else {
-				return new HashSet<>();
+				return Collections.emptySet();
 			}
 		}
 
+		void clear() {
+			allocatedSlotsById.clear();
+			allocatedSlotsByTaskManager.clear();
+		}
+
 		@VisibleForTesting
 		boolean containResource(final ResourceID resourceID) {
-			return allocatedSlotsByResource.containsKey(resourceID);
+			return allocatedSlotsByTaskManager.containsKey(resourceID);
 		}
 
 		@VisibleForTesting
 		int size() {
-			return allocatedSlots.size();
+			return allocatedSlotsById.size();
 		}
 	}
 
+	// ------------------------------------------------------------------------
+
 	/**
 	 * Organize all available slots from different points of view.
 	 */
-	static class AvailableSlots {
+	private static class AvailableSlots {
 
 		/** All available slots organized by TaskManager */
-		private final Map<ResourceID, Set<SlotDescriptor>> availableSlotsByResource;
+		private final HashMap<ResourceID, Set<AllocatedSlot>> availableSlotsByTaskManager;
+
+		/** All available slots organized by host */
+		private final HashMap<String, Set<AllocatedSlot>> availableSlotsByHost;
 
-		/** All available slots */
-		private final Set<SlotDescriptor> availableSlots;
+		/** The available slots, with the time when they were inserted */
+		private final HashMap<AllocationID, SlotAndTimestamp> availableSlots;
 
 		AvailableSlots() {
-			this.availableSlotsByResource = new HashMap<>();
-			this.availableSlots = new HashSet<>();
+			this.availableSlotsByTaskManager = new HashMap<>();
+			this.availableSlotsByHost = new HashMap<>();
+			this.availableSlots = new HashMap<>();
 		}
 
 		/**
-		 * Add an available slot.
+		 * Adds an available slot.
 		 *
-		 * @param descriptor The descriptor of the slot
+		 * @param slot The slot to add
 		 */
-		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);
+		void add(final AllocatedSlot slot, final long timestamp) {
+			checkNotNull(slot);
+
+			SlotAndTimestamp previous = availableSlots.put(
+					slot.getSlotAllocationId(), new SlotAndTimestamp(slot, timestamp));
+
+			if (previous == null) {
+				final ResourceID resourceID = slot.getTaskManagerLocation().getResourceID();
+				final String host = slot.getTaskManagerLocation().getFQDNHostname();
+
+				Set<AllocatedSlot> slotsForTaskManager = availableSlotsByTaskManager.get(resourceID);
+				if (slotsForTaskManager == null) {
+					slotsForTaskManager = new HashSet<>();
+					availableSlotsByTaskManager.put(resourceID, slotsForTaskManager);
+				}
+				slotsForTaskManager.add(slot);
+
+				Set<AllocatedSlot> slotsForHost = availableSlotsByHost.get(host);
+				if (slotsForHost == null) {
+					slotsForHost = new HashSet<>();
+					availableSlotsByHost.put(host, slotsForHost);
+				}
+				slotsForHost.add(slot);
+			}
+			else {
+				throw new IllegalStateException("slot already contained");
 			}
-			slotsForResource.add(descriptor);
 		}
 
 		/**
-		 * Check whether we have this slot
-		 *
-		 * @param slotDescriptor The descriptor of the slot
-		 * @return True if we contains this slot
+		 * Check whether we have this slot.
 		 */
-		boolean contains(final SlotDescriptor slotDescriptor) {
-			return availableSlots.contains(slotDescriptor);
+		boolean contains(AllocationID slotId) {
+			return availableSlots.containsKey(slotId);
 		}
 
 		/**
-		 * Poll a slot which matches the required resource profile
+		 * Poll a slot which matches the required resource profile. The polling tries to satisfy the
+		 * location preferences, by TaskManager and by host.
 		 *
-		 * @param resourceProfile The required resource profile
+		 * @param resourceProfile      The required resource profile.
+		 * @param locationPreferences  The location preferences, in order to be checked.
+		 * 
 		 * @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;
+		SlotAndLocality poll(ResourceProfile resourceProfile, Iterable<TaskManagerLocation> locationPreferences) {
+			// fast path if no slots are available
+			if (availableSlots.isEmpty()) {
+				return null;
+			}
+
+			boolean hadLocationPreference = false;
+
+			if (locationPreferences != null) {
+
+				// first search by TaskManager
+				for (TaskManagerLocation location : locationPreferences) {
+					hadLocationPreference = true;
+
+					final Set<AllocatedSlot> onTaskManager = availableSlotsByTaskManager.get(location.getResourceID());
+					if (onTaskManager != null) {
+						for (AllocatedSlot candidate : onTaskManager) {
+							if (candidate.getResourceProfile().isMatching(resourceProfile)) {
+								remove(candidate.getSlotAllocationId());
+								return new SlotAndLocality(candidate, Locality.LOCAL);
+							}
+						}
+					}
+				}
+
+				// now, search by host
+				for (TaskManagerLocation location : locationPreferences) {
+					final Set<AllocatedSlot> onHost = availableSlotsByHost.get(location.getFQDNHostname());
+					if (onHost != null) {
+						for (AllocatedSlot candidate : onHost) {
+							if (candidate.getResourceProfile().isMatching(resourceProfile)) {
+								remove(candidate.getSlotAllocationId());
+								return new SlotAndLocality(candidate, Locality.HOST_LOCAL);
+							}
+						}
+					}
+				}
+			}
+
+			// take any slot
+			for (SlotAndTimestamp candidate : availableSlots.values()) {
+				final AllocatedSlot slot = candidate.slot();
+
+				if (slot.getResourceProfile().isMatching(resourceProfile)) {
+					remove(slot.getSlotAllocationId());
+					return new SlotAndLocality(
+							slot, hadLocationPreference ? Locality.NON_LOCAL : Locality.UNCONSTRAINED);
 				}
 			}
+
+			// nothing available that matches
 			return null;
 		}
 
 		/**
 		 * Remove all available slots come from specified TaskManager.
 		 *
-		 * @param resourceID The id of the TaskManager
+		 * @param taskManager 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);
+		void removeAllForTaskManager(final ResourceID taskManager) {
+			// remove from the by-TaskManager view
+			final Set<AllocatedSlot> slotsForTm = availableSlotsByTaskManager.remove(taskManager);
+
+			if (slotsForTm != null && slotsForTm.size() > 0) {
+				final String host = slotsForTm.iterator().next().getTaskManagerLocation().getFQDNHostname();
+				final Set<AllocatedSlot> slotsForHost = availableSlotsByHost.get(host);
+
+				// remove from the base set and the by-host view
+				for (AllocatedSlot slot : slotsForTm) {
+					availableSlots.remove(slot.getSlotAllocationId());
+					slotsForHost.remove(slot);
+				}
+
+				if (slotsForHost.isEmpty()) {
+					availableSlotsByHost.remove(host);
 				}
 			}
 		}
 
-		private void remove(final SlotDescriptor slotDescriptor) {
-			availableSlots.remove(slotDescriptor);
+		boolean tryRemove(AllocationID slotId) {
+			final SlotAndTimestamp sat = availableSlots.remove(slotId);
+			if (sat != null) {
+				final AllocatedSlot slot = sat.slot();
+				final ResourceID resourceID = slot.getTaskManagerLocation().getResourceID();
+				final String host = slot.getTaskManagerLocation().getFQDNHostname();
+
+				final Set<AllocatedSlot> slotsForTm = availableSlotsByTaskManager.get(resourceID);
+				final Set<AllocatedSlot> slotsForHost = availableSlotsByHost.get(host);
+
+				slotsForTm.remove(slot);
+				slotsForHost.remove(slot);
+
+				if (slotsForTm.isEmpty()) {
+					availableSlotsByTaskManager.remove(resourceID);
+				}
+				if (slotsForHost.isEmpty()) {
+					availableSlotsByHost.remove(host);
+				}
+
+				return true;
+			}
+			else {
+				return false;
+			}
+		}
 
-			final ResourceID resourceID = slotDescriptor.getTaskManagerLocation().getResourceID();
-			final Set<SlotDescriptor> slotsForResource = checkNotNull(availableSlotsByResource.get(resourceID));
-			slotsForResource.remove(slotDescriptor);
-			if (slotsForResource.isEmpty()) {
-				availableSlotsByResource.remove(resourceID);
+		private void remove(AllocationID slotId) throws IllegalStateException {
+			if (!tryRemove(slotId)) {
+				throw new IllegalStateException("slot not contained");
 			}
 		}
 
 		@VisibleForTesting
-		boolean containResource(final ResourceID resourceID) {
-			return availableSlotsByResource.containsKey(resourceID);
+		boolean containsTaskManager(ResourceID resourceID) {
+			return availableSlotsByTaskManager.containsKey(resourceID);
 		}
 
 		@VisibleForTesting
 		int size() {
 			return availableSlots.size();
 		}
+
+		@VisibleForTesting
+		void clear() {
+			availableSlots.clear();
+			availableSlotsByTaskManager.clear();
+			availableSlotsByHost.clear();
+		}
+	}
+
+	// ------------------------------------------------------------------------
+
+	/**
+	 * An implementation of the {@link SlotOwner} and {@link SlotProvider} interfaces
+	 * that delegates methods as RPC calls to the SlotPool's RPC gateway.
+	 */
+	private static class ProviderAndOwner implements SlotOwner, SlotProvider {
+
+		private final SlotPoolGateway gateway;
+
+		private final Time timeout;
+
+		ProviderAndOwner(SlotPoolGateway gateway, Time timeout) {
+			this.gateway = gateway;
+			this.timeout = timeout;
+		}
+
+		@Override
+		public boolean returnAllocatedSlot(Slot slot) {
+			gateway.returnAllocatedSlot(slot);
+			return true;
+		}
+
+		@Override
+		public Future<SimpleSlot> allocateSlot(ScheduledUnit task, boolean allowQueued) {
+			return gateway.allocateSlot(
+					task, ResourceProfile.UNKNOWN, Collections.<TaskManagerLocation>emptyList(), timeout);
+		}
+	}
+
+	// ------------------------------------------------------------------------
+
+	/**
+	 * A pending request for a slot
+	 */
+	private static class PendingRequest {
+
+		private final AllocationID allocationID;
+
+		private final FlinkCompletableFuture<SimpleSlot> future;
+
+		private final ResourceProfile resourceProfile;
+
+		PendingRequest(
+				AllocationID allocationID,
+				FlinkCompletableFuture<SimpleSlot> future,
+				ResourceProfile resourceProfile) {
+			this.allocationID = allocationID;
+			this.future = future;
+			this.resourceProfile = resourceProfile;
+		}
+
+		public AllocationID allocationID() {
+			return allocationID;
+		}
+
+		public FlinkCompletableFuture<SimpleSlot> future() {
+			return future;
+		}
+
+		public ResourceProfile resourceProfile() {
+			return resourceProfile;
+		}
+	}
+
+	// ------------------------------------------------------------------------
+
+	/**
+	 * A slot, together with the timestamp when it was added
+	 */
+	private static class SlotAndTimestamp {
+
+		private final AllocatedSlot slot;
+
+		private final long timestamp;
+
+		SlotAndTimestamp(
+				AllocatedSlot slot,
+				long timestamp) {
+			this.slot = slot;
+			this.timestamp = timestamp;
+		}
+
+		public AllocatedSlot slot() {
+			return slot;
+		}
+
+		public long timestamp() {
+			return timestamp;
+		}
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/a1ba9f11/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotPoolGateway.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotPoolGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotPoolGateway.java
new file mode 100644
index 0000000..42942ca
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotPoolGateway.java
@@ -0,0 +1,95 @@
+/*
+ * 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.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.Future;
+import org.apache.flink.runtime.jobmanager.scheduler.ScheduledUnit;
+import org.apache.flink.runtime.jobmanager.slots.AllocatedSlot;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway;
+import org.apache.flink.runtime.rpc.RpcGateway;
+import org.apache.flink.runtime.rpc.RpcTimeout;
+import org.apache.flink.runtime.taskexecutor.slot.SlotOffer;
+import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
+
+import java.util.UUID;
+
+/**
+ * The gateway for calls on the {@link SlotPool}. 
+ */
+public interface SlotPoolGateway extends RpcGateway {
+
+	// ------------------------------------------------------------------------
+	//  shutdown
+	// ------------------------------------------------------------------------
+
+	void suspend();
+
+	// ------------------------------------------------------------------------
+	//  resource manager connection
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Connects the SlotPool to the given ResourceManager. After this method is called, the
+	 * SlotPool will be able to request resources from the given ResourceManager.
+	 * 
+	 * @param resourceManagerLeaderId The leader session ID of the resource manager.
+	 * @param resourceManagerGateway  The RPC gateway for the resource manager.
+	 */
+	void connectToResourceManager(UUID resourceManagerLeaderId, ResourceManagerGateway resourceManagerGateway);
+
+	/**
+	 * Disconnects the slot pool from its current Resource Manager. After this call, the pool will not
+	 * be able to request further slots from the Resource Manager, and all currently pending requests
+	 * to the resource manager will be canceled.
+	 * 
+	 * <p>The slot pool will still be able to serve slots from its internal pool.
+	 */
+	void disconnectResourceManager();
+
+	// ------------------------------------------------------------------------
+	//  registering / un-registering TaskManagers and slots
+	// ------------------------------------------------------------------------
+
+	void registerTaskManager(ResourceID resourceID);
+
+	void releaseTaskManager(ResourceID resourceID);
+
+	Future<Boolean> offerSlot(AllocatedSlot slot);
+
+	Future<Iterable<SlotOffer>> offerSlots(Iterable<Tuple2<AllocatedSlot, SlotOffer>> offers);
+	
+	void failAllocation(AllocationID allocationID, Exception cause);
+
+	// ------------------------------------------------------------------------
+	//  allocating and disposing slots
+	// ------------------------------------------------------------------------
+
+	Future<SimpleSlot> allocateSlot(
+			ScheduledUnit task,
+			ResourceProfile resources,
+			Iterable<TaskManagerLocation> locationPreferences,
+			@RpcTimeout Time timeout);
+
+	void returnAllocatedSlot(Slot slot);
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/a1ba9f11/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/Locality.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/Locality.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/Locality.java
index ec6e9b1..0ef2482 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/Locality.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/Locality.java
@@ -19,19 +19,19 @@
 package org.apache.flink.runtime.jobmanager.scheduler;
 
 public enum Locality {
-	
-	/**
-	 * No constraint existed on the task placement.
-	 */
+
+	/** No constraint existed on the task placement. */
 	UNCONSTRAINED,
-	
-	/**
-	 * The task was scheduled respecting its locality preferences.
-	 */
+
+	/** The task was scheduled into the same TaskManager as requested */
 	LOCAL,
-	
-	/**
-	 * The task was scheduled to a destination not included in its locality preferences.
-	 */
-	NON_LOCAL
+
+	/** The task was scheduled onto the same host as requested */
+	HOST_LOCAL,
+
+	/** The task was scheduled to a destination not included in its locality preferences. */
+	NON_LOCAL,
+
+	/** No locality information was provided, it is unknown if the locality was respected */
+	UNKNOWN
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/a1ba9f11/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/NoResourceAvailableException.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/NoResourceAvailableException.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/NoResourceAvailableException.java
index e45747b..546f31f 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/NoResourceAvailableException.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/NoResourceAvailableException.java
@@ -54,7 +54,11 @@ public class NoResourceAvailableException extends JobException {
 	public NoResourceAvailableException(String message) {
 		super(message);
 	}
-	
+
+	public NoResourceAvailableException(String message, Throwable cause) {
+		super(message, cause);
+	}
+
 	// --------------------------------------------------------------------------------------------
 	
 	@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/a1ba9f11/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/slots/AllocatedSlot.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/slots/AllocatedSlot.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/slots/AllocatedSlot.java
index 9419ab4..f477c49 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/slots/AllocatedSlot.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/slots/AllocatedSlot.java
@@ -20,7 +20,9 @@ package org.apache.flink.runtime.jobmanager.slots;
 
 import org.apache.flink.api.common.JobID;
 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.taskexecutor.TaskExecutorGateway;
 import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
 
 import static org.apache.flink.util.Preconditions.checkNotNull;
@@ -54,6 +56,9 @@ public class AllocatedSlot {
 	/** TEMP until the new RPC is in place: The actor gateway to communicate with the TaskManager */
 	private final TaskManagerGateway taskManagerGateway;
 
+	/** RPC gateway to call the TaskManager that holds this slot */
+	private final TaskExecutorGateway taskExecutorGateway;
+
 	/** The number of the slot on the TaskManager to which slot belongs. Purely informational. */
 	private final int slotNumber;
 
@@ -73,15 +78,23 @@ public class AllocatedSlot {
 		this.slotNumber = slotNumber;
 		this.resourceProfile = checkNotNull(resourceProfile);
 		this.taskManagerGateway = checkNotNull(taskManagerGateway);
+		this.taskExecutorGateway = null;
 	}
 
-	public AllocatedSlot(AllocatedSlot other) {
-		this.slotAllocationId = other.slotAllocationId;
-		this.jobID = other.jobID;
-		this.taskManagerLocation = other.taskManagerLocation;
-		this.slotNumber = other.slotNumber;
-		this.resourceProfile = other.resourceProfile;
-		this.taskManagerGateway = other.taskManagerGateway;
+	public AllocatedSlot(
+			AllocationID slotAllocationId,
+			JobID jobID,
+			TaskManagerLocation location,
+			int slotNumber,
+			ResourceProfile resourceProfile,
+			TaskExecutorGateway taskExecutorGateway) {
+		this.slotAllocationId = checkNotNull(slotAllocationId);
+		this.jobID = checkNotNull(jobID);
+		this.taskManagerLocation = checkNotNull(location);
+		this.slotNumber = slotNumber;
+		this.resourceProfile = checkNotNull(resourceProfile);
+		this.taskManagerGateway = null;
+		this.taskExecutorGateway = checkNotNull(taskExecutorGateway);
 	}
 
 	// ------------------------------------------------------------------------
@@ -96,6 +109,17 @@ public class AllocatedSlot {
 	}
 
 	/**
+	 * Gets the ID of the TaskManager on which this slot was allocated.
+	 * 
+	 * <p>This is equivalent to {@link #getTaskManagerLocation()#getTaskManagerId()}.
+	 * 
+	 * @return This slot's TaskManager's ID.
+	 */
+	public ResourceID getTaskManagerId() {
+		return getTaskManagerLocation().getResourceID();
+	}
+
+	/**
 	 * Returns the ID of the job this allocated slot belongs to.
 	 *
 	 * @return the ID of the job this allocated slot belongs to
@@ -142,8 +166,28 @@ public class AllocatedSlot {
 		return taskManagerGateway;
 	}
 
+	public TaskExecutorGateway getTaskExecutorGateway() {
+		return taskExecutorGateway;
+	}
+
 	// ------------------------------------------------------------------------
 
+	/**
+	 * This always returns a reference hash code.
+	 */
+	@Override
+	public final int hashCode() {
+		return super.hashCode();
+	}
+
+	/**
+	 * This always checks based on reference equality.
+	 */
+	@Override
+	public final boolean equals(Object obj) {
+		return this == obj;
+	}
+
 	@Override
 	public String toString() {
 		return "AllocatedSlot " + slotAllocationId + " @ " + taskManagerLocation + " - " + slotNumber;

http://git-wip-us.apache.org/repos/asf/flink/blob/a1ba9f11/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/slots/PooledSlotProvider.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/slots/PooledSlotProvider.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/slots/PooledSlotProvider.java
deleted file mode 100644
index 5655fc2..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/slots/PooledSlotProvider.java
+++ /dev/null
@@ -1,73 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.runtime.jobmanager.slots;
-
-import org.apache.flink.api.common.JobID;
-import org.apache.flink.api.common.time.Time;
-import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
-import org.apache.flink.runtime.concurrent.Future;
-import org.apache.flink.runtime.concurrent.impl.FlinkCompletableFuture;
-import org.apache.flink.runtime.instance.SimpleSlot;
-import org.apache.flink.runtime.instance.SlotPool;
-import org.apache.flink.runtime.instance.SlotProvider;
-import org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException;
-import org.apache.flink.runtime.jobmanager.scheduler.ScheduledUnit;
-
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.TimeoutException;
-
-import static org.apache.flink.util.Preconditions.checkNotNull;
-
-/**
- * A simple pool based slot provider with {@link SlotPool} as the underlying storage.
- */
-public class PooledSlotProvider implements SlotProvider {
-
-	/** The pool which holds all the slots. */
-	private final SlotPool slotPool;
-
-	/** The timeout for allocation. */
-	private final Time timeout;
-
-	public PooledSlotProvider(final SlotPool slotPool, final Time timeout) {
-		this.slotPool = slotPool;
-		this.timeout = timeout;
-	}
-
-	@Override
-	public Future<SimpleSlot> allocateSlot(ScheduledUnit task,
-			boolean allowQueued) throws NoResourceAvailableException
-	{
-		checkNotNull(task);
-
-		final JobID jobID = task.getTaskToExecute().getVertex().getJobId();
-		final Future<SimpleSlot> future = slotPool.allocateSimpleSlot(jobID, ResourceProfile.UNKNOWN);
-		try {
-			final SimpleSlot slot = future.get(timeout.getSize(), timeout.getUnit());
-			return FlinkCompletableFuture.completed(slot);
-		} catch (InterruptedException e) {
-			throw new NoResourceAvailableException("Could not allocate a slot because it's interrupted.");
-		} catch (ExecutionException e) {
-			throw new NoResourceAvailableException("Could not allocate a slot because some error occurred " +
-					"during allocation, " + e.getMessage());
-		} catch (TimeoutException e) {
-			throw new NoResourceAvailableException("Could not allocate a slot within time limit: " + timeout);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/a1ba9f11/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/slots/SlotAndLocality.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/slots/SlotAndLocality.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/slots/SlotAndLocality.java
new file mode 100644
index 0000000..3fe5346
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/slots/SlotAndLocality.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.jobmanager.slots;
+
+import org.apache.flink.runtime.jobmanager.scheduler.Locality;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * A combination of a {@link AllocatedSlot} and a {@link Locality}.
+ */
+public class SlotAndLocality {
+
+	private final AllocatedSlot slot;
+
+	private final Locality locality;
+
+	public SlotAndLocality(AllocatedSlot slot, Locality locality) {
+		this.slot = checkNotNull(slot);
+		this.locality = checkNotNull(locality);
+	}
+
+	// ------------------------------------------------------------------------
+
+	public AllocatedSlot slot() {
+		return slot;
+	}
+
+	public Locality locality() {
+		return locality;
+	}
+
+	// ------------------------------------------------------------------------
+
+	@Override
+	public String toString() {
+		return "Slot: " + slot + " (" + locality + ')';
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/a1ba9f11/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 0b3b68e..a620390 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
@@ -53,8 +53,8 @@ import org.apache.flink.runtime.executiongraph.restart.RestartStrategyFactory;
 import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
 import org.apache.flink.runtime.highavailability.LeaderIdMismatchException;
 import org.apache.flink.runtime.instance.Slot;
-import org.apache.flink.runtime.instance.SlotDescriptor;
 import org.apache.flink.runtime.instance.SlotPool;
+import org.apache.flink.runtime.instance.SlotPoolGateway;
 import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
 import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
 import org.apache.flink.runtime.jobgraph.JobGraph;
@@ -62,7 +62,7 @@ import org.apache.flink.runtime.jobgraph.JobStatus;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.jobmanager.OnCompletionActions;
 import org.apache.flink.runtime.jobmanager.PartitionProducerDisposedException;
-import org.apache.flink.runtime.jobmanager.slots.PooledSlotProvider;
+import org.apache.flink.runtime.jobmanager.slots.AllocatedSlot;
 import org.apache.flink.runtime.jobmaster.message.ClassloadingProps;
 import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener;
 import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
@@ -91,19 +91,18 @@ import org.apache.flink.runtime.taskmanager.TaskExecutionState;
 import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
 import org.apache.flink.runtime.util.SerializedThrowable;
 import org.apache.flink.util.InstantiationUtil;
+
 import org.slf4j.Logger;
 
 import javax.annotation.Nullable;
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.HashMap;
-import java.util.HashSet;
 import java.util.Map;
-import java.util.Set;
 import java.util.UUID;
 import java.util.concurrent.Callable;
 import java.util.concurrent.Executor;
 import java.util.concurrent.ExecutorService;
-import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
 
 import static org.apache.flink.util.Preconditions.checkNotNull;
@@ -160,7 +159,7 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 
 	private final SlotPool slotPool;
 
-	private final Time allocationTimeout;
+	private final SlotPoolGateway slotPoolGateway;
 
 	private volatile UUID leaderSessionID;
 
@@ -249,8 +248,8 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 		// register self as job status change listener
 		executionGraph.registerJobStatusListener(new JobManagerJobStatusListener());
 
-		this.slotPool = new SlotPool(executorService);
-		this.allocationTimeout = Time.of(5, TimeUnit.SECONDS);
+		this.slotPool = new SlotPool(rpcService, jobGraph.getJobID());
+		this.slotPoolGateway = slotPool.getSelf();
 
 		this.registeredTaskManagers = new HashMap<>(4);
 	}
@@ -272,10 +271,6 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 	 */
 	public void start(final UUID leaderSessionID) throws Exception {
 		if (LEADER_ID_UPDATER.compareAndSet(this, null, leaderSessionID)) {
-
-			// make sure the slot pool now accepts messages for this leader  
-			slotPool.setJobManagerLeaderId(leaderSessionID);
-
 			// make sure we receive RPC and async calls
 			super.start();
 
@@ -305,8 +300,18 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 
 	@RpcMethod
 	public void startJobExecution() {
+		// double check that the leader status did not change
+		if (leaderSessionID == null) {
+			log.info("Aborting job startup - JobManager lost leader status");
+			return;
+		}
+
 		log.info("Starting execution of job {} ({})", jobGraph.getName(), jobGraph.getJobID());
 
+		// start the slot pool make sure the slot pool now accepts messages for this leader
+		log.debug("Staring SlotPool component");
+		slotPool.start(leaderSessionID);
+
 		try {
 			// job is ready to go, try to establish connection with resource manager
 			//   - activate leader retrieval for the resource manager
@@ -328,7 +333,7 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 			@Override
 			public void run() {
 				try {
-					executionGraph.scheduleForExecution(new PooledSlotProvider(slotPool, allocationTimeout));
+					executionGraph.scheduleForExecution(slotPool.getSlotProvider());
 				}
 				catch (Throwable t) {
 					executionGraph.fail(t);
@@ -353,27 +358,26 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 			return;
 		}
 
-		// receive no more messages until started again, should be called before we clear self leader id
-		((StartStoppable) getSelf()).stop();
-
+		// not leader any more - should not accept any leader messages any more
 		leaderSessionID = null;
-		slotPool.setJobManagerLeaderId(null);
-		executionGraph.suspend(cause);
 
-		// disconnect from resource manager:
 		try {
 			resourceManagerLeaderRetriever.stop();
-		} catch (Exception e) {
-			log.warn("Failed to stop resource manager leader retriever when suspending.", e);
+		} catch (Throwable t) {
+			log.warn("Failed to stop resource manager leader retriever when suspending.", t);
 		}
-		closeResourceManagerConnection();
 
-		// TODO: in the future, the slot pool should not release the resources, so that
-		// TODO: the TaskManagers offer the resources to the new leader 
-		for (ResourceID taskManagerId : registeredTaskManagers.keySet()) {
-			slotPool.releaseResource(taskManagerId);
-		}
-		registeredTaskManagers.clear();
+		// tell the execution graph (JobManager is still processing messages here) 
+		executionGraph.suspend(cause);
+
+		// receive no more messages until started again, should be called before we clear self leader id
+		((StartStoppable) getSelf()).stop();
+
+		// the slot pool stops receiving messages and clears its pooled slots 
+		slotPoolGateway.suspend();
+
+		// disconnect from resource manager:
+		closeResourceManagerConnection();
 	}
 
 	//----------------------------------------------------------------------------------------------
@@ -452,6 +456,7 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 		}
 	}
 
+	@RpcMethod
 	public ExecutionState requestPartitionState(
 			final UUID leaderSessionID,
 			final IntermediateDataSetID intermediateResultId,
@@ -624,9 +629,11 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 	}
 
 	@RpcMethod
-	public Iterable<SlotOffer> offerSlots(final ResourceID taskManagerId,
-			final Iterable<SlotOffer> slots, final UUID leaderId) throws Exception
-	{
+	public Future<Iterable<SlotOffer>> offerSlots(
+			final ResourceID taskManagerId,
+			final Iterable<SlotOffer> slots,
+			final UUID leaderId) throws Exception {
+
 		validateLeaderSessionId(leaderSessionID);
 
 		Tuple2<TaskManagerLocation, TaskExecutorGateway> taskManager = registeredTaskManagers.get(taskManagerId);
@@ -634,20 +641,22 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 			throw new Exception("Unknown TaskManager " + taskManagerId);
 		}
 
-		final Set<SlotOffer> acceptedSlotOffers = new HashSet<>(4);
+		final JobID jid = jobGraph.getJobID();
+		final TaskManagerLocation taskManagerLocation = taskManager.f0;
+		final TaskExecutorGateway taskManagerGateway = taskManager.f1;
+
+		final ArrayList<Tuple2<AllocatedSlot, SlotOffer>> slotsAndOffers = new ArrayList<>();
+
 		for (SlotOffer slotOffer : slots) {
-			final SlotDescriptor slotDescriptor = new SlotDescriptor(
-					jobGraph.getJobID(),
-					taskManager.f0,
-					slotOffer.getSlotIndex(),
-					slotOffer.getResourceProfile(),
-					null); // TODO: replace the actor gateway with the new rpc gateway, it's ready (taskManager.f1)
-			if (slotPool.offerSlot(slotOffer.getAllocationId(), slotDescriptor)) {
-				acceptedSlotOffers.add(slotOffer);
-			}
+			final AllocatedSlot slot = new AllocatedSlot(
+					slotOffer.getAllocationId(), jid, taskManagerLocation,
+					slotOffer.getSlotIndex(), slotOffer.getResourceProfile(),
+					taskManagerGateway);
+
+			slotsAndOffers.add(new Tuple2<>(slot, slotOffer));
 		}
 
-		return acceptedSlotOffers;
+		return slotPoolGateway.offerSlots(slotsAndOffers);
 	}
 
 	@RpcMethod
@@ -662,7 +671,7 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 			throw new Exception("Unknown TaskManager " + taskManagerId);
 		}
 
-		slotPool.failAllocation(allocationId, cause);
+		slotPoolGateway.failAllocation(allocationId, cause);
 	}
 
 	@RpcMethod
@@ -708,7 +717,7 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 						return new RegistrationResponse.Decline("Invalid leader session id");
 					}
 
-					slotPool.registerResource(taskManagerId);
+					slotPoolGateway.registerTaskManager(taskManagerId);
 					registeredTaskManagers.put(taskManagerId, Tuple2.of(taskManagerLocation, taskExecutorGateway));
 					return new JMTMRegistrationSuccess(taskManagerId, libraryCacheManager.getBlobServerPort());
 				}
@@ -840,7 +849,7 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 			log.info("JobManager successfully registered at ResourceManager, leader id: {}.",
 					success.getResourceManagerLeaderId());
 
-			slotPool.setResourceManager(
+			slotPoolGateway.connectToResourceManager(
 					success.getResourceManagerLeaderId(), resourceManagerConnection.getTargetGateway());
 		}
 	}
@@ -852,7 +861,8 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 			resourceManagerConnection.close();
 			resourceManagerConnection = null;
 		}
-		slotPool.disconnectResourceManager();
+
+		slotPoolGateway.disconnectResourceManager();
 	}
 
 	private void validateLeaderSessionId(UUID leaderSessionID) throws LeaderIdMismatchException {

http://git-wip-us.apache.org/repos/asf/flink/blob/a1ba9f11/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java
index b971b96..f30e345 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java
@@ -25,6 +25,7 @@ import org.apache.flink.util.ReflectionUtil;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import javax.annotation.Nonnull;
 import java.util.concurrent.Callable;
 import java.util.concurrent.Executor;
 import java.util.concurrent.TimeUnit;
@@ -207,6 +208,17 @@ public abstract class RpcEndpoint<C extends RpcGateway> {
 	 * @param runnable Runnable to be executed
 	 * @param delay    The delay after which the runnable will be executed
 	 */
+	protected void scheduleRunAsync(Runnable runnable, Time delay) {
+		scheduleRunAsync(runnable, delay.getSize(), delay.getUnit());
+	}
+
+	/**
+	 * Execute the runnable in the main thread of the underlying RPC endpoint, with
+	 * a delay of the given number of milliseconds.
+	 *
+	 * @param runnable Runnable to be executed
+	 * @param delay    The delay after which the runnable will be executed
+	 */
 	protected void scheduleRunAsync(Runnable runnable, long delay, TimeUnit unit) {
 		((MainThreadExecutable) self).scheduleRunAsync(runnable, unit.toMillis(delay));
 	}
@@ -255,7 +267,7 @@ public abstract class RpcEndpoint<C extends RpcGateway> {
 	/**
 	 * Executor which executes runnables in the main thread context.
 	 */
-	private class MainThreadExecutor implements Executor {
+	private static class MainThreadExecutor implements Executor {
 
 		private final MainThreadExecutable gateway;
 
@@ -264,7 +276,7 @@ public abstract class RpcEndpoint<C extends RpcGateway> {
 		}
 
 		@Override
-		public void execute(Runnable runnable) {
+		public void execute(@Nonnull Runnable runnable) {
 			gateway.runAsync(runnable);
 		}
 	}
@@ -277,7 +289,7 @@ public abstract class RpcEndpoint<C extends RpcGateway> {
 	private Class<C> determineSelfGatewayType() {
 
 		// determine self gateway type
-		Class c = getClass();
+		Class<?> c = getClass();
 		Class<C> determinedSelfGatewayType;
 		do {
 			determinedSelfGatewayType = ReflectionUtil.getTemplateType1(c);


[28/52] [abbrv] flink git commit: [FLINK-4987] Harden SlotPool on JobMaster

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/a1ba9f11/flink-runtime/src/main/java/org/apache/flink/runtime/util/clock/Clock.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/clock/Clock.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/clock/Clock.java
new file mode 100644
index 0000000..d33bba4
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/clock/Clock.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.util.clock;
+
+/**
+ * A clock that gives access to time. This clock returns two flavors of time:
+ * 
+ * <p><b>Absolute Time:</b> This refers to real world wall clock time, and it typically
+ * derived from a system clock. It is subject to clock drift and inaccuracy, and can jump
+ * if the system clock is adjusted.
+ * 
+ * <p><b>Relative Time:</b> This time advances at the same speed as the <i>absolute time</i>,
+ * but the timestamps can only be referred to relative to each other. The timestamps have
+ * no absolute meaning and cannot be compared across JVM processes. The source for the
+ * timestamps is not affected by adjustments to the system clock, so it never jumps.
+ */
+public abstract class Clock {
+
+	public abstract long absoluteTimeMillis();
+
+	public abstract long relativeTimeMillis();
+
+	public abstract long relativeTimeNanos();
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/a1ba9f11/flink-runtime/src/main/java/org/apache/flink/runtime/util/clock/SystemClock.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/clock/SystemClock.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/clock/SystemClock.java
new file mode 100644
index 0000000..789a0b7
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/clock/SystemClock.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.util.clock;
+
+/**
+ * A clock that returns the time of the system / process.
+ * 
+ * <p>This clock uses {@link System#currentTimeMillis()} for <i>absolute time</i>
+ * and {@link System#nanoTime()} for <i>relative time</i>.
+ * 
+ * <p>This SystemClock exists as a singleton instance.
+ */
+public class SystemClock extends Clock {
+
+	private static final SystemClock INSTANCE = new SystemClock();
+
+	public static SystemClock getInstance() {
+		return INSTANCE;
+	}
+
+	// ------------------------------------------------------------------------
+
+	@Override
+	public long absoluteTimeMillis() {
+		return System.currentTimeMillis();
+	}
+
+	@Override
+	public long relativeTimeMillis() {
+		return System.nanoTime() / 1_000_000;
+	}
+
+	@Override
+	public long relativeTimeNanos() {
+		return System.nanoTime();
+	}
+
+	// ------------------------------------------------------------------------
+
+	private SystemClock() {}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/a1ba9f11/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/types/ResourceProfileTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/types/ResourceProfileTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/types/ResourceProfileTest.java
index bc5ddaa..cd1d895 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/types/ResourceProfileTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/types/ResourceProfileTest.java
@@ -46,4 +46,9 @@ public class ResourceProfileTest {
 		assertTrue(rp4.isMatching(rp3));
 		assertTrue(rp4.isMatching(rp4));
 	}
+
+	@Test
+	public void testUnknownMatchesUnknown() {
+		assertTrue(ResourceProfile.UNKNOWN.isMatching(ResourceProfile.UNKNOWN));
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/a1ba9f11/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
index 655a3ea..33ed679 100644
--- 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
@@ -1,135 +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;
-	}
-}
+///*
+// * 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/a1ba9f11/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
index 8e31085..4d58a31 100644
--- 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
@@ -1,124 +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.jobmanager.slots.TaskManagerGateway;
-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(TaskManagerGateway.class));
-	}
-}
+///*
+// * 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/a1ba9f11/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
index 30cdbd6..cc1d194 100644
--- 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
@@ -1,297 +1,299 @@
-/*
- * 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());
-	}
-
-}
+///*
+// * 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.ExecutorService;
+//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 ExecutorService 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());
+//	}
+//
+//}

http://git-wip-us.apache.org/repos/asf/flink/blob/a1ba9f11/flink-runtime/src/test/java/org/apache/flink/runtime/minicluster/MiniClusterITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/minicluster/MiniClusterITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/minicluster/MiniClusterITCase.java
index dd43337..f5b3892 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/minicluster/MiniClusterITCase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/minicluster/MiniClusterITCase.java
@@ -18,7 +18,9 @@
 
 package org.apache.flink.runtime.minicluster;
 
+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.runtime.jobgraph.JobGraph;
 import org.apache.flink.runtime.jobgraph.JobVertex;
 import org.apache.flink.runtime.testtasks.NoOpInvokable;
@@ -31,7 +33,7 @@ import org.junit.Test;
  */
 public class MiniClusterITCase extends TestLogger {
 
-//	@Test
+	@Test
 	public void runJobWithSingleRpcService() throws Exception {
 		MiniClusterConfiguration cfg = new MiniClusterConfiguration();
 
@@ -74,6 +76,13 @@ public class MiniClusterITCase extends TestLogger {
 		task.setMaxParallelism(1);
 		task.setInvokableClass(NoOpInvokable.class);
 
-		return new JobGraph(new JobID(), "Test Job", task);
+		JobGraph jg = new JobGraph(new JobID(), "Test Job", task);
+		jg.setAllowQueuedScheduling(true);
+
+		ExecutionConfig executionConfig = new ExecutionConfig();
+		executionConfig.setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, 1000));
+		jg.setExecutionConfig(executionConfig);
+
+		return jg;
 	}
 }


[11/52] [abbrv] flink git commit: [FLINK-4351] [cluster management] JobManager handle TaskManager's registration

Posted by se...@apache.org.
[FLINK-4351] [cluster management] JobManager handle TaskManager's registration


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

Branch: refs/heads/master
Commit: bc6823618a1cf01a3385478bb1913940d9f604bc
Parents: 5cbec02
Author: Kurt Young <yk...@gmail.com>
Authored: Sun Oct 16 23:00:57 2016 +0800
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Dec 23 20:54:23 2016 +0100

----------------------------------------------------------------------
 .../flink/runtime/jobmaster/JobMaster.java      | 75 +++++++++++++++++---
 .../runtime/jobmaster/JobMasterGateway.java     | 15 ++--
 .../runtime/taskexecutor/JobLeaderService.java  | 55 ++++++--------
 .../runtime/taskexecutor/TaskExecutor.java      |  2 +-
 .../taskexecutor/TaskManagerServices.java       |  2 +-
 .../runtime/taskexecutor/TaskExecutorTest.java  | 10 ++-
 6 files changed, 102 insertions(+), 57 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/bc682361/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 e6720fd..1fb5474 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
@@ -21,6 +21,7 @@ package org.apache.flink.runtime.jobmaster;
 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.api.java.tuple.Tuple2;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.core.io.InputSplit;
 import org.apache.flink.core.io.InputSplitAssigner;
@@ -36,7 +37,9 @@ import org.apache.flink.runtime.client.JobSubmissionException;
 import org.apache.flink.runtime.client.SerializedJobExecutionResult;
 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.concurrent.impl.FlinkCompletableFuture;
 import org.apache.flink.runtime.execution.ExecutionState;
 import org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager;
 import org.apache.flink.runtime.executiongraph.Execution;
@@ -81,7 +84,9 @@ 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.KeyGroupRange;
+import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway;
 import org.apache.flink.runtime.taskmanager.TaskExecutionState;
+import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
 import org.apache.flink.runtime.util.SerializedThrowable;
 import org.apache.flink.util.InstantiationUtil;
 import org.apache.flink.util.SerializedValue;
@@ -89,8 +94,10 @@ import org.slf4j.Logger;
 
 import javax.annotation.Nullable;
 import java.io.IOException;
+import java.util.HashMap;
 import java.util.Map;
 import java.util.UUID;
+import java.util.concurrent.Callable;
 import java.util.concurrent.Executor;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.TimeUnit;
@@ -122,6 +129,8 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 	/** Configuration of the JobManager */
 	private final Configuration configuration;
 
+	private final Time rpcTimeout;
+
 	/** Service to contend for and retrieve the leadership of JM and RM */
 	private final HighAvailabilityServices highAvailabilityServices;
 
@@ -152,7 +161,7 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 
 	private volatile UUID leaderSessionID;
 
-	// --------- resource manager --------
+	// --------- ResourceManager --------
 
 	/** Leader retriever service used to locate ResourceManager's address */
 	private LeaderRetrievalService resourceManagerLeaderRetriever;
@@ -160,6 +169,9 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 	/** Connection with ResourceManager, null if not located address yet or we close it initiative */
 	private ResourceManagerConnection resourceManagerConnection;
 
+	// --------- TaskManagers --------
+
+	private final Map<ResourceID, Tuple2<TaskManagerLocation, TaskExecutorGateway>> registeredTaskManagers;
 
 	// ------------------------------------------------------------------------
 
@@ -181,6 +193,7 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 
 		this.jobGraph = checkNotNull(jobGraph);
 		this.configuration = checkNotNull(configuration);
+		this.rpcTimeout = rpcAskTimeout;
 		this.highAvailabilityServices = checkNotNull(highAvailabilityService);
 		this.libraryCacheManager = checkNotNull(libraryCacheManager);
 		this.executionContext = checkNotNull(executorService);
@@ -244,6 +257,8 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 
 		this.slotPool = new SlotPool(executorService);
 		this.allocationTimeout = Time.of(5, TimeUnit.SECONDS);
+
+		this.registeredTaskManagers = new HashMap<>(4);
 	}
 
 	//----------------------------------------------------------------------------------------------
@@ -380,8 +395,10 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 		}
 		closeResourceManagerConnection();
 
-		// TODO: disconnect from all registered task managers
-
+		for (ResourceID taskManagerId : registeredTaskManagers.keySet()) {
+			slotPool.releaseResource(taskManagerId);
+		}
+		registeredTaskManagers.clear();
 	}
 
 	//----------------------------------------------------------------------------------------------
@@ -656,11 +673,53 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 	}
 
 	@RpcMethod
-	public RegistrationResponse registerTaskManager(
-		final String taskManagerAddress,
-		final ResourceID taskManagerProcessId,
-		final UUID leaderId) {
-		throw new UnsupportedOperationException("Has to be implemented.");
+	public Future<RegistrationResponse> registerTaskManager(
+			final TaskManagerLocation taskManagerLocation,
+			final UUID leaderId) throws Exception
+	{
+		if (!JobMaster.this.leaderSessionID.equals(leaderId)) {
+			log.warn("Discard registration from TaskExecutor {} at ({}) because the expected " +
+							"leader session ID {} did not equal the received leader session ID {}.",
+					taskManagerLocation.getResourceID(), taskManagerLocation.addressString(),
+					JobMaster.this.leaderSessionID, leaderId);
+			throw new Exception("Leader id not match, expected: " + JobMaster.this.leaderSessionID
+					+ ", actual: " + leaderId);
+		}
+
+		final ResourceID taskManagerId = taskManagerLocation.getResourceID();
+
+		if (registeredTaskManagers.containsKey(taskManagerId)) {
+			final RegistrationResponse response = new JMTMRegistrationSuccess(
+					taskManagerId, libraryCacheManager.getBlobServerPort());
+			return FlinkCompletableFuture.completed(response);
+		} else {
+			return getRpcService().execute(new Callable<TaskExecutorGateway>() {
+				@Override
+				public TaskExecutorGateway call() throws Exception {
+					return getRpcService().connect(taskManagerLocation.addressString(), TaskExecutorGateway.class)
+							.get(rpcTimeout.getSize(), rpcTimeout.getUnit());
+				}
+			}).handleAsync(new BiFunction<TaskExecutorGateway, Throwable, RegistrationResponse>() {
+				@Override
+				public RegistrationResponse apply(TaskExecutorGateway taskExecutorGateway, Throwable throwable) {
+					if (throwable != null) {
+						return new RegistrationResponse.Decline(throwable.getMessage());
+					}
+
+					if (!JobMaster.this.leaderSessionID.equals(leaderId)) {
+						log.warn("Discard registration from TaskExecutor {} at ({}) because the expected " +
+										"leader session ID {} did not equal the received leader session ID {}.",
+								taskManagerLocation.getResourceID(), taskManagerLocation.addressString(),
+								JobMaster.this.leaderSessionID, leaderId);
+						return new RegistrationResponse.Decline("Invalid leader session id");
+					}
+
+					slotPool.registerResource(taskManagerId);
+					registeredTaskManagers.put(taskManagerId, Tuple2.of(taskManagerLocation, taskExecutorGateway));
+					return new JMTMRegistrationSuccess(taskManagerId, libraryCacheManager.getBlobServerPort());
+				}
+			}, getMainThreadExecutor());
+		}
 	}
 
 	//----------------------------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/flink/blob/bc682361/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 17b4194..508e70a 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
@@ -41,6 +41,7 @@ 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;
+import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
 
 import java.util.UUID;
 
@@ -184,15 +185,13 @@ public interface JobMasterGateway extends CheckpointCoordinatorGateway {
 	/**
 	 * Register the task manager at the job manager.
 	 *
-	 * @param taskManagerAddress address of the task manager
-	 * @param taskManagerProcessId identifying the task manager
-	 * @param leaderId identifying the job leader
-	 * @param timeout for the rpc call
+	 * @param taskManagerLocation location of the task manager
+	 * @param leaderId            identifying the job leader
+	 * @param timeout             for the rpc call
 	 * @return Future registration response indicating whether the registration was successful or not
 	 */
 	Future<RegistrationResponse> registerTaskManager(
-		final String taskManagerAddress,
-		final ResourceID taskManagerProcessId,
-		final UUID leaderId,
-		@RpcTimeout final Time timeout);
+			final TaskManagerLocation taskManagerLocation,
+			final UUID leaderId,
+			@RpcTimeout final Time timeout);
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/bc682361/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/JobLeaderService.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/JobLeaderService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/JobLeaderService.java
index e7f52e2..14d36ab 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/JobLeaderService.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/JobLeaderService.java
@@ -21,7 +21,6 @@ package org.apache.flink.runtime.taskexecutor;
 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.ResourceID;
 import org.apache.flink.runtime.concurrent.Future;
 import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
 import org.apache.flink.runtime.jobmaster.JMTMRegistrationSuccess;
@@ -32,6 +31,7 @@ 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.rpc.RpcService;
+import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
 import org.apache.flink.util.Preconditions;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -53,8 +53,8 @@ public class JobLeaderService {
 
 	private static final Logger LOG = LoggerFactory.getLogger(JobLeaderService.class);
 
-	/** Process id of the owning process */
-	private final ResourceID ownerProcessId;
+	/** Self's location, used for the job manager connection */
+	private final TaskManagerLocation ownLocation;
 
 	/** The leader retrieval service and listener for each registered job */
 	private final Map<JobID, Tuple2<LeaderRetrievalService, JobLeaderService.JobManagerLeaderListener>> jobLeaderServices;
@@ -62,9 +62,6 @@ public class JobLeaderService {
 	/** Internal state of the service */
 	private volatile JobLeaderService.State state;
 
-	/** Address of the owner of this service. This address is used for the job manager connection */
-	private String ownerAddress;
-
 	/** Rpc service to use for establishing connections */
 	private RpcService rpcService;
 
@@ -74,14 +71,13 @@ public class JobLeaderService {
 	/** Job leader listener listening for job leader changes */
 	private JobLeaderListener jobLeaderListener;
 
-	public JobLeaderService(ResourceID ownerProcessId) {
-		this.ownerProcessId = Preconditions.checkNotNull(ownerProcessId);
+	public JobLeaderService(TaskManagerLocation location) {
+		this.ownLocation = Preconditions.checkNotNull(location);
 
 		jobLeaderServices = new HashMap<>(4);
 
 		state = JobLeaderService.State.CREATED;
 
-		ownerAddress = null;
 		rpcService = null;
 		highAvailabilityServices = null;
 		jobLeaderListener = null;
@@ -94,13 +90,11 @@ public class JobLeaderService {
 	/**
 	 * Start the job leader service with the given services.
 	 *
-	 * @param initialOwnerAddress to be used for establishing connections (source address)
 	 * @param initialRpcService to be used to create rpc connections
 	 * @param initialHighAvailabilityServices to create leader retrieval services for the different jobs
 	 * @param initialJobLeaderListener listening for job leader changes
 	 */
 	public void start(
-		final String initialOwnerAddress,
 		final RpcService initialRpcService,
 		final HighAvailabilityServices initialHighAvailabilityServices,
 		final JobLeaderListener initialJobLeaderListener) {
@@ -110,7 +104,6 @@ public class JobLeaderService {
 		} else {
 			LOG.info("Start job leader service.");
 
-			this.ownerAddress = Preconditions.checkNotNull(initialOwnerAddress);
 			this.rpcService = Preconditions.checkNotNull(initialRpcService);
 			this.highAvailabilityServices = Preconditions.checkNotNull(initialHighAvailabilityServices);
 			this.jobLeaderListener = Preconditions.checkNotNull(initialJobLeaderListener);
@@ -311,14 +304,13 @@ public class JobLeaderService {
 			@Override
 			protected RetryingRegistration<JobMasterGateway, JMTMRegistrationSuccess> generateRegistration() {
 				return new JobLeaderService.JobManagerRetryingRegistration(
-					LOG,
-					rpcService,
-					"JobManager",
-					JobMasterGateway.class,
-					getTargetAddress(),
-					getTargetLeaderId(),
-					ownerAddress,
-					ownerProcessId);
+						LOG,
+						rpcService,
+						"JobManager",
+						JobMasterGateway.class,
+						getTargetAddress(),
+						getTargetLeaderId(),
+						ownLocation);
 			}
 
 			@Override
@@ -349,10 +341,11 @@ public class JobLeaderService {
 	/**
 	 * Retrying registration for the job manager <--> task manager connection.
 	 */
-	private static final class JobManagerRetryingRegistration extends RetryingRegistration<JobMasterGateway, JMTMRegistrationSuccess> {
+	private static final class JobManagerRetryingRegistration
+			extends RetryingRegistration<JobMasterGateway, JMTMRegistrationSuccess>
+	{
 
-		private final String taskManagerAddress;
-		private final ResourceID taskManagerProcessId;
+		private final TaskManagerLocation taskManagerLocation;
 
 		JobManagerRetryingRegistration(
 			Logger log,
@@ -361,22 +354,18 @@ public class JobLeaderService {
 			Class<JobMasterGateway> targetType,
 			String targetAddress,
 			UUID leaderId,
-			String taskManagerAddress,
-			ResourceID taskManagerProcessId) {
+			TaskManagerLocation taskManagerLocation) {
 
 			super(log, rpcService, targetName, targetType, targetAddress, leaderId);
 
-			this.taskManagerAddress = Preconditions.checkNotNull(taskManagerAddress);
-			this.taskManagerProcessId = Preconditions.checkNotNull(taskManagerProcessId);
+			this.taskManagerLocation = Preconditions.checkNotNull(taskManagerLocation);
 		}
 
 		@Override
-		protected Future<RegistrationResponse> invokeRegistration(JobMasterGateway gateway, UUID leaderId, long timeoutMillis) throws Exception {
-			return gateway.registerTaskManager(
-				taskManagerAddress,
-				taskManagerProcessId,
-				leaderId,
-				Time.milliseconds(timeoutMillis));
+		protected Future<RegistrationResponse> invokeRegistration(
+				JobMasterGateway gateway, UUID leaderId, long timeoutMillis) throws Exception
+		{
+			return gateway.registerTaskManager(taskManagerLocation, leaderId, Time.milliseconds(timeoutMillis));
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/bc682361/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 5146e5b..1b1c02b 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
@@ -208,7 +208,7 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 		taskSlotTable.start(new SlotActionsImpl());
 
 		// start the job leader service
-		jobLeaderService.start(getAddress(), getRpcService(), haServices, new JobLeaderListenerImpl());
+		jobLeaderService.start(getRpcService(), haServices, new JobLeaderListenerImpl());
 	}
 
 	/**

http://git-wip-us.apache.org/repos/asf/flink/blob/bc682361/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServices.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServices.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServices.java
index 54f2332..b57fafe 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServices.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServices.java
@@ -210,7 +210,7 @@ public class TaskManagerServices {
 
 		final JobManagerTable jobManagerTable = new JobManagerTable();
 
-		final JobLeaderService jobLeaderService = new JobLeaderService(resourceID);
+		final JobLeaderService jobLeaderService = new JobLeaderService(taskManagerLocation);
 		
 		return new TaskManagerServices(
 			taskManagerLocation,

http://git-wip-us.apache.org/repos/asf/flink/blob/bc682361/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java
index 8d6cba2..a8da4fd 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java
@@ -365,7 +365,7 @@ public class TaskExecutorTest extends TestLogger {
 		final TimerService<AllocationID> timerService = mock(TimerService.class);
 		final TaskSlotTable taskSlotTable = new TaskSlotTable(Arrays.asList(mock(ResourceProfile.class)), timerService);
 		final JobManagerTable jobManagerTable = new JobManagerTable();
-		final JobLeaderService jobLeaderService = new JobLeaderService(resourceId);
+		final JobLeaderService jobLeaderService = new JobLeaderService(taskManagerLocation);
 		final TestingFatalErrorHandler testingFatalErrorHandler = new TestingFatalErrorHandler();
 
 		final TestingLeaderRetrievalService resourceManagerLeaderRetrievalService = new TestingLeaderRetrievalService();
@@ -394,8 +394,7 @@ public class TaskExecutorTest extends TestLogger {
 		final JobMasterGateway jobMasterGateway = mock(JobMasterGateway.class);
 
 		when(jobMasterGateway.registerTaskManager(
-			any(String.class),
-			eq(resourceId),
+			eq(taskManagerLocation),
 			eq(jobManagerLeaderId),
 			any(Time.class)
 		)).thenReturn(FlinkCompletableFuture.<RegistrationResponse>completed(new JMTMRegistrationSuccess(jmResourceId, blobPort)));
@@ -466,7 +465,7 @@ public class TaskExecutorTest extends TestLogger {
 		final TimerService<AllocationID> timerService = mock(TimerService.class);
 		final TaskSlotTable taskSlotTable = new TaskSlotTable(Arrays.asList(mock(ResourceProfile.class), mock(ResourceProfile.class)), timerService);
 		final JobManagerTable jobManagerTable = new JobManagerTable();
-		final JobLeaderService jobLeaderService = new JobLeaderService(resourceId);
+		final JobLeaderService jobLeaderService = new JobLeaderService(taskManagerLocation);
 		final TestingFatalErrorHandler testingFatalErrorHandler = new TestingFatalErrorHandler();
 
 		final String resourceManagerAddress = "rm";
@@ -499,8 +498,7 @@ public class TaskExecutorTest extends TestLogger {
 		final JobMasterGateway jobMasterGateway = mock(JobMasterGateway.class);
 
 		when(jobMasterGateway.registerTaskManager(
-			any(String.class),
-			eq(resourceId),
+			eq(taskManagerLocation),
 			eq(jobManagerLeaderId),
 			any(Time.class)
 		)).thenReturn(FlinkCompletableFuture.<RegistrationResponse>completed(new JMTMRegistrationSuccess(jmResourceId, blobPort)));


[13/52] [abbrv] flink git commit: [FLINK-4853] [rm] Clean up job manager registration at the resource manager

Posted by se...@apache.org.
[FLINK-4853] [rm] Clean up job manager registration at the resource manager

Introduce the JobLeaderIdService which automatically retrieves the current job leader id.
This job leader id is used to validate job manager registartion attempts. Additionally, it
is used to disconnect old job leaders from the resource manager.

Add comments

This closes #2657.


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

Branch: refs/heads/master
Commit: c001beccba7273c557f148becf61bddb2bf96503
Parents: 70af08c
Author: Till Rohrmann <tr...@apache.org>
Authored: Tue Oct 18 18:03:00 2016 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Dec 23 20:54:24 2016 +0100

----------------------------------------------------------------------
 .../flink/runtime/jobmaster/JobMaster.java      |  10 +-
 .../runtime/jobmaster/JobMasterGateway.java     |  12 +
 .../resourcemanager/JobLeaderIdActions.java     |  51 ++
 .../resourcemanager/JobLeaderIdService.java     | 285 ++++++++++
 .../resourcemanager/ResourceManager.java        | 538 ++++++++++---------
 .../resourcemanager/ResourceManagerGateway.java |   2 +-
 .../StandaloneResourceManager.java              |   5 +-
 .../exceptions/ResourceManagerRunner.java       |   5 +-
 .../registration/JobManagerRegistration.java    |  58 ++
 .../registration/JobMasterRegistration.java     |  62 ---
 .../flink/runtime/rpc/akka/AkkaRpcService.java  |   2 -
 .../resourcemanager/ResourceManagerHATest.java  |   8 +-
 .../ResourceManagerJobMasterTest.java           |  14 +-
 .../ResourceManagerTaskExecutorTest.java        |   2 +
 .../slotmanager/SlotProtocolTest.java           |  12 +-
 15 files changed, 736 insertions(+), 330 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/c001becc/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 c80cc51..458bf0c 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
@@ -752,6 +752,14 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 		}
 	}
 
+	@RpcMethod
+	public void disconnectResourceManager(
+			final UUID jobManagerLeaderId,
+			final UUID resourceManagerLeaderId,
+			final Exception cause) {
+		// TODO: Implement disconnect behaviour
+	}
+
 	//----------------------------------------------------------------------------------------------
 	// Internal methods
 	//----------------------------------------------------------------------------------------------
@@ -942,7 +950,7 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 						long timeoutMillis) throws Exception
 				{
 					Time timeout = Time.milliseconds(timeoutMillis);
-					return gateway.registerJobMaster(leaderId, jobManagerLeaderID, getAddress(), jobID, timeout);
+					return gateway.registerJobManager(leaderId, jobManagerLeaderID, getAddress(), jobID, timeout);
 				}
 			};
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/c001becc/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 2d7ebb9..5ab68fe 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
@@ -128,6 +128,18 @@ public interface JobMasterGateway extends CheckpointCoordinatorGateway {
 	void disconnectTaskManager(ResourceID resourceID);
 
 	/**
+	 * Disconnects the resource manager from the job manager because of the given cause.
+	 *
+	 * @param jobManagerLeaderId identifying the job manager leader id
+	 * @param resourceManagerLeaderId identifying the resource manager leader id
+	 * @param cause of the disconnect
+	 */
+	void disconnectResourceManager(
+		final UUID jobManagerLeaderId,
+		final UUID resourceManagerLeaderId,
+		final Exception cause);
+
+	/**
 	 * Requests a {@link KvStateLocation} for the specified {@link KvState} registration name.
 	 *
 	 * @param registrationName Name under which the KvState has been registered.

http://git-wip-us.apache.org/repos/asf/flink/blob/c001becc/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/JobLeaderIdActions.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/JobLeaderIdActions.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/JobLeaderIdActions.java
new file mode 100644
index 0000000..58777ef
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/JobLeaderIdActions.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager;
+
+import org.apache.flink.api.common.JobID;
+
+import java.util.UUID;
+
+/**
+ * Interface for actions called by the {@link JobLeaderIdService}.
+ */
+public interface JobLeaderIdActions {
+
+	/**
+	 * Callback when a monitored job leader lost its leadership.
+	 *
+	 * @param jobId identifying the job whose leader lost leadership
+	 * @param oldJobLeaderId of the job manager which lost leadership
+	 */
+	void jobLeaderLostLeadership(JobID jobId, UUID oldJobLeaderId);
+
+	/**
+	 * Request to remove the job from the {@link JobLeaderIdService}.
+	 *
+	 * @param jobId identifying the job to remove
+	 */
+	void removeJob(JobID jobId);
+
+	/**
+	 * Callback to report occurring errors.
+	 *
+	 * @param error which has occurred
+	 */
+	void handleError(Throwable error);
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/c001becc/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/JobLeaderIdService.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/JobLeaderIdService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/JobLeaderIdService.java
new file mode 100644
index 0000000..6c7e249
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/JobLeaderIdService.java
@@ -0,0 +1,285 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.concurrent.CompletableFuture;
+import org.apache.flink.runtime.concurrent.Future;
+import org.apache.flink.runtime.concurrent.impl.FlinkCompletableFuture;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
+import org.apache.flink.runtime.highavailability.RunningJobsRegistry;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.ExecutionException;
+
+/**
+ * Service which retrieves for a registered job the current job leader id (the leader id of the
+ * job manager responsible for the job). The leader id will be exposed as a future via the
+ * {@link #getLeaderId(JobID)}. The future will only be completed with an exception in case
+ * the service will be stopped.
+ */
+public class JobLeaderIdService {
+
+	private static final Logger LOG = LoggerFactory.getLogger(JobLeaderIdService.class);
+
+	/** High availability services to use by this service */
+	private final HighAvailabilityServices highAvailabilityServices;
+
+	/** Registry to retrieve running jobs */
+	private final RunningJobsRegistry runningJobsRegistry;
+
+	/** Map of currently monitored jobs */
+	private final Map<JobID, JobLeaderIdListener> jobLeaderIdListeners;
+
+	/** Actions to call when the job leader changes */
+	private JobLeaderIdActions jobLeaderIdActions;
+
+	public JobLeaderIdService(HighAvailabilityServices highAvailabilityServices) throws Exception {
+		this.highAvailabilityServices = Preconditions.checkNotNull(highAvailabilityServices);
+
+		this.runningJobsRegistry = highAvailabilityServices.getRunningJobsRegistry();
+
+		jobLeaderIdListeners = new HashMap<>(4);
+
+		jobLeaderIdActions = null;
+	}
+
+	/**
+	 * Start the service with the given job leader actions.
+	 *
+	 * @param initialJobLeaderIdActions to use for job leader id actions
+	 * @throws Exception which is thrown when clearing up old state
+	 */
+	public void start(JobLeaderIdActions initialJobLeaderIdActions) throws Exception {
+		if (isStarted()) {
+			clear();
+		}
+
+		this.jobLeaderIdActions = Preconditions.checkNotNull(initialJobLeaderIdActions);
+	}
+
+	/**
+	 * Stop the service.
+	 *
+	 * @throws Exception which is thrown in case a retrieval service cannot be stopped properly
+	 */
+	public void stop() throws Exception {
+		clear();
+
+		this.jobLeaderIdActions = null;
+	}
+
+	/**
+	 * Checks whether the service has been started.
+	 *
+	 * @return True if the service has been started; otherwise false
+	 */
+	public boolean isStarted() {
+		return jobLeaderIdActions == null;
+	}
+
+	/**
+	 * Stop and clear the currently registered job leader id listeners.
+	 *
+	 * @throws Exception which is thrown in case a retrieval service cannot be stopped properly
+	 */
+	public void clear() throws Exception {
+		Exception exception = null;
+
+		for (JobLeaderIdListener listener: jobLeaderIdListeners.values()) {
+			try {
+				listener.stop();
+			} catch (Exception e) {
+				exception = ExceptionUtils.firstOrSuppressed(e, exception);
+			}
+		}
+
+		if (exception != null) {
+			ExceptionUtils.rethrowException(exception, "Could not properly stop the " +
+				JobLeaderIdService.class.getSimpleName() + '.');
+		}
+
+		jobLeaderIdListeners.clear();
+	}
+
+	/**
+	 * Add a job to be monitored to retrieve the job leader id.
+	 *
+	 * @param jobId identifying the job to monitor
+	 * @throws Exception if the job could not be added to the service
+	 */
+	public void addJob(JobID jobId) throws Exception {
+		Preconditions.checkNotNull(jobLeaderIdActions);
+
+		LOG.debug("Add job {} to job leader id monitoring.", jobId);
+
+		if (!jobLeaderIdListeners.containsKey(jobId)) {
+			LeaderRetrievalService leaderRetrievalService = highAvailabilityServices.getJobManagerLeaderRetriever(jobId);
+
+			JobLeaderIdListener jobidListener = new JobLeaderIdListener(jobId, jobLeaderIdActions, leaderRetrievalService);
+			jobLeaderIdListeners.put(jobId, jobidListener);
+		}
+	}
+
+	/**
+	 * Remove the given job from being monitored by the service.
+	 *
+	 * @param jobId identifying the job to remove from monitor
+	 * @throws Exception if removing the job fails
+	 */
+	public void removeJob(JobID jobId) throws Exception {
+		LOG.debug("Remove job {} from job leader id monitoring.", jobId);
+
+		JobLeaderIdListener listener = jobLeaderIdListeners.remove(jobId);
+
+		if (listener != null) {
+			listener.stop();
+		}
+	}
+
+	/**
+	 * Check whether the given job is being monitored or not.
+	 *
+	 * @param jobId identifying the job
+	 * @return True if the job is being monitored; otherwise false
+	 */
+	public boolean containsJob(JobID jobId) {
+		return jobLeaderIdListeners.containsKey(jobId);
+	}
+
+	public Future<UUID> getLeaderId(JobID jobId) throws Exception {
+		if (!jobLeaderIdListeners.containsKey(jobId)) {
+			addJob(jobId);
+		}
+
+		JobLeaderIdListener listener = jobLeaderIdListeners.get(jobId);
+
+		return listener.getLeaderIdFuture();
+	}
+
+	// --------------------------------------------------------------------------------
+	// Static utility classes
+	// --------------------------------------------------------------------------------
+
+	/**
+	 * Listener which stores the current leader id and exposes them as a future value when
+	 * requested. The returned future will always be completed properly except when stopping the
+	 * listener.
+	 */
+	private final class JobLeaderIdListener implements LeaderRetrievalListener {
+		private final JobID jobId;
+		private final JobLeaderIdActions listenerJobLeaderIdActions;
+		private final LeaderRetrievalService leaderRetrievalService;
+
+		private volatile CompletableFuture<UUID> leaderIdFuture;
+		private volatile boolean running = true;
+
+		private JobLeaderIdListener(
+				JobID jobId,
+				JobLeaderIdActions listenerJobLeaderIdActions,
+				LeaderRetrievalService leaderRetrievalService) throws Exception {
+			this.jobId = Preconditions.checkNotNull(jobId);
+			this.listenerJobLeaderIdActions = Preconditions.checkNotNull(listenerJobLeaderIdActions);
+			this.leaderRetrievalService = Preconditions.checkNotNull(leaderRetrievalService);
+
+			leaderIdFuture = new FlinkCompletableFuture<>();
+
+			// start the leader service we're listening to
+			leaderRetrievalService.start(this);
+		}
+
+		public Future<UUID> getLeaderIdFuture() {
+			return leaderIdFuture;
+		}
+
+		public void stop() throws Exception {
+			running = false;
+			leaderRetrievalService.stop();
+			leaderIdFuture.completeExceptionally(new Exception("Job leader id service has been stopped."));
+		}
+
+		@Override
+		public void notifyLeaderAddress(String leaderAddress, UUID leaderSessionId) {
+			if (running) {
+				LOG.debug("Found a new job leader {}@{}.", leaderSessionId, leaderAddress);
+
+				UUID previousJobLeaderId = null;
+
+				if (leaderIdFuture.isDone()) {
+					try {
+						previousJobLeaderId = leaderIdFuture.getNow(null);
+					} catch (ExecutionException e) {
+						// this should never happen since we complete this future always properly
+						handleError(e);
+					}
+
+					leaderIdFuture = FlinkCompletableFuture.completed(leaderSessionId);
+				} else {
+					leaderIdFuture.complete(leaderSessionId);
+				}
+
+				try {
+					if (runningJobsRegistry.isJobRunning(jobId)) {
+						if (leaderSessionId == null) {
+							// there is no new leader
+							if (previousJobLeaderId != null) {
+								// we had a previous job leader, so notify about his lost leadership
+								listenerJobLeaderIdActions.jobLeaderLostLeadership(jobId, previousJobLeaderId);
+							}
+						} else {
+							if (previousJobLeaderId != null && !leaderSessionId.equals(previousJobLeaderId)) {
+								// we had a previous leader and he's not the same as the new leader
+								listenerJobLeaderIdActions.jobLeaderLostLeadership(jobId, previousJobLeaderId);
+							}
+						}
+					} else {
+						// the job is no longer running so remove it
+						listenerJobLeaderIdActions.removeJob(jobId);
+					}
+				} catch (IOException e) {
+					// cannot tell whether the job is still running or not so just remove the listener
+					LOG.debug("Encountered an error while checking the job registry for running jobs.", e);
+					listenerJobLeaderIdActions.removeJob(jobId);
+				}
+			} else {
+				LOG.debug("A leader id change {}@{} has been detected after the listener has been stopped.",
+					leaderSessionId, leaderAddress);
+			}
+		}
+
+		@Override
+		public void handleError(Exception exception) {
+			if (running) {
+				listenerJobLeaderIdActions.handleError(exception);
+			} else {
+				LOG.debug("An error occurred in the {} after the listener has been stopped.",
+					JobLeaderIdListener.class.getSimpleName(), exception);
+			}
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/c001becc/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java
index 4161972..7240087 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java
@@ -20,7 +20,6 @@ package org.apache.flink.runtime.resourcemanager;
 
 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.runtime.clusterframework.ApplicationStatus;
 import org.apache.flink.runtime.clusterframework.messages.InfoMessage;
 import org.apache.flink.runtime.clusterframework.types.ResourceID;
@@ -36,13 +35,11 @@ import org.apache.flink.runtime.instance.InstanceID;
 import org.apache.flink.runtime.jobmaster.JobMasterRegistrationSuccess;
 import org.apache.flink.runtime.leaderelection.LeaderContender;
 import org.apache.flink.runtime.leaderelection.LeaderElectionService;
-import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener;
-import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
 import org.apache.flink.runtime.metrics.MetricRegistry;
 import org.apache.flink.runtime.resourcemanager.exceptions.ResourceManagerException;
 import org.apache.flink.runtime.resourcemanager.messages.jobmanager.RMSlotRequestRejected;
 import org.apache.flink.runtime.resourcemanager.messages.jobmanager.RMSlotRequestReply;
-import org.apache.flink.runtime.resourcemanager.registration.JobMasterRegistration;
+import org.apache.flink.runtime.resourcemanager.registration.JobManagerRegistration;
 import org.apache.flink.runtime.resourcemanager.registration.WorkerRegistration;
 import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager;
 import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManagerFactory;
@@ -61,10 +58,8 @@ import org.apache.flink.util.ExceptionUtils;
 
 import java.io.Serializable;
 import java.util.HashMap;
-import java.util.Iterator;
 import java.util.Map;
 import java.util.UUID;
-import java.util.concurrent.Callable;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.Executor;
@@ -77,7 +72,7 @@ import static org.apache.flink.util.Preconditions.checkNotNull;
  *
  * It offers the following methods as part of its rpc interface to interact with him remotely:
  * <ul>
- *     <li>{@link #registerJobMaster(UUID, UUID, String, JobID)} registers a {@link JobMaster} at the resource manager</li>
+ *     <li>{@link #registerJobManager(UUID, UUID, String, JobID)} registers a {@link JobMaster} at the resource manager</li>
  *     <li>{@link #requestSlot(UUID, UUID, SlotRequest)} requests a slot from the resource manager</li>
  * </ul>
  */
@@ -89,10 +84,10 @@ public abstract class ResourceManager<WorkerType extends Serializable>
 	private final ResourceManagerConfiguration resourceManagerConfiguration;
 
 	/** All currently registered JobMasterGateways scoped by JobID. */
-	private final Map<JobID, JobMasterRegistration> jobMasters;
+	private final Map<JobID, JobManagerRegistration> jobManagerRegistrations;
 
-	/** LeaderListeners for all registered JobIDs. */
-	private final Map<JobID, JobIdLeaderListener> leaderListeners;
+	/** Service to retrieve the job leader ids */
+	private final JobLeaderIdService jobLeaderIdService;
 
 	/** All currently registered TaskExecutors with there framework specific worker information. */
 	private final Map<ResourceID, WorkerRegistration<WorkerType>> taskExecutors;
@@ -116,7 +111,7 @@ public abstract class ResourceManager<WorkerType extends Serializable>
 	private LeaderElectionService leaderElectionService;
 
 	/** ResourceManager's leader session id which is updated on leader election. */
-	private volatile UUID leaderSessionID;
+	private volatile UUID leaderSessionId;
 
 	/** All registered listeners for status updates of the ResourceManager. */
 	private ConcurrentMap<String, InfoMessageListenerRpcGateway> infoMessageListeners;
@@ -127,6 +122,7 @@ public abstract class ResourceManager<WorkerType extends Serializable>
 			HighAvailabilityServices highAvailabilityServices,
 			SlotManagerFactory slotManagerFactory,
 			MetricRegistry metricRegistry,
+			JobLeaderIdService jobLeaderIdService,
 			FatalErrorHandler fatalErrorHandler) {
 
 		super(rpcService);
@@ -135,12 +131,12 @@ public abstract class ResourceManager<WorkerType extends Serializable>
 		this.highAvailabilityServices = checkNotNull(highAvailabilityServices);
 		this.slotManagerFactory = checkNotNull(slotManagerFactory);
 		this.metricRegistry = checkNotNull(metricRegistry);
+		this.jobLeaderIdService = checkNotNull(jobLeaderIdService);
 		this.fatalErrorHandler = checkNotNull(fatalErrorHandler);
 
-		this.jobMasters = new HashMap<>(4);
-		this.leaderListeners = new HashMap<>(4);
+		this.jobManagerRegistrations = new HashMap<>(4);
 		this.taskExecutors = new HashMap<>(8);
-		this.leaderSessionID = null;
+		this.leaderSessionId = null;
 		infoMessageListeners = new ConcurrentHashMap<>(8);
 	}
 
@@ -168,11 +164,12 @@ public abstract class ResourceManager<WorkerType extends Serializable>
 		}
 
 		try {
-			// framework specific initialization
-			initialize();
+			jobLeaderIdService.start(new JobLeaderIdActionsImpl());
 		} catch (Exception e) {
-			throw new ResourceManagerException("Could not initialize the resource manager.", e);
+			throw new ResourceManagerException("Could not start the job leader id service.", e);
 		}
+
+		initialize();
 	}
 
 	@Override
@@ -180,6 +177,12 @@ public abstract class ResourceManager<WorkerType extends Serializable>
 		Exception exception = null;
 
 		try {
+			jobLeaderIdService.stop();
+		} catch (Exception e) {
+			exception = ExceptionUtils.firstOrSuppressed(e, exception);
+		}
+
+		try {
 			leaderElectionService.stop();
 		} catch (Exception e) {
 			exception = ExceptionUtils.firstOrSuppressed(e, exception);
@@ -202,101 +205,124 @@ public abstract class ResourceManager<WorkerType extends Serializable>
 	//  RPC methods
 	// ------------------------------------------------------------------------
 
-	/**
-	 * Register a {@link JobMaster} at the resource manager.
-	 *
-	 * @param resourceManagerLeaderId The fencing token for the ResourceManager leader
-	 * @param jobMasterAddress        The address of the JobMaster that registers
-	 * @param jobID                   The Job ID of the JobMaster that registers
-	 * @return Future registration response
-	 */
 	@RpcMethod
-	public Future<RegistrationResponse> registerJobMaster(
-		final UUID resourceManagerLeaderId, final UUID jobMasterLeaderId,
-		final String jobMasterAddress, final JobID jobID) {
+	public Future<RegistrationResponse> registerJobManager(
+			final UUID resourceManagerLeaderId,
+			final UUID jobManagerLeaderId,
+			final String jobManagerAddress,
+			final JobID jobId) {
+
+		checkNotNull(resourceManagerLeaderId);
+		checkNotNull(jobManagerLeaderId);
+		checkNotNull(jobManagerAddress);
+		checkNotNull(jobId);
+
+		if (isValid(resourceManagerLeaderId)) {
+			if (!jobLeaderIdService.containsJob(jobId)) {
+				try {
+					jobLeaderIdService.addJob(jobId);
+				} catch (Exception e) {
+					ResourceManagerException exception = new ResourceManagerException("Could not add the job " +
+						jobId + " to the job id leader service.", e);
+
+					onFatalErrorAsync(exception);
+
+					log.error("Could not add job {} to job leader id service.", jobId, e);
+					return FlinkCompletableFuture.completedExceptionally(exception);
+				}
+			}
 
-		checkNotNull(jobMasterAddress);
-		checkNotNull(jobID);
+			log.info("Registering job manager {}@{} for job {}.", jobManagerLeaderId, jobManagerAddress, jobId);
+
+			Future<UUID> jobLeaderIdFuture;
 
-		// create a leader retriever in case it doesn't exist
-		final JobIdLeaderListener jobIdLeaderListener;
-		if (leaderListeners.containsKey(jobID)) {
-			jobIdLeaderListener = leaderListeners.get(jobID);
-		} else {
 			try {
-				LeaderRetrievalService jobMasterLeaderRetriever =
-					highAvailabilityServices.getJobManagerLeaderRetriever(jobID);
-				jobIdLeaderListener = new JobIdLeaderListener(jobID, jobMasterLeaderRetriever);
+				jobLeaderIdFuture = jobLeaderIdService.getLeaderId(jobId);
 			} catch (Exception e) {
-				log.warn("Failed to start JobMasterLeaderRetriever for job id {}", jobID, e);
+				// we cannot check the job leader id so let's fail
+				// TODO: Maybe it's also ok to skip this check in case that we cannot check the leader id
+				ResourceManagerException exception = new ResourceManagerException("Cannot obtain the " +
+					"job leader id future to verify the correct job leader.", e);
+
+				onFatalErrorAsync(exception);
 
-				return FlinkCompletableFuture.<RegistrationResponse>completed(
-					new RegistrationResponse.Decline("Failed to retrieve JobMasterLeaderRetriever"));
+				log.debug("Could not obtain the job leader id future to verify the correct job leader.");
+				return FlinkCompletableFuture.completedExceptionally(exception);
 			}
 
-			leaderListeners.put(jobID, jobIdLeaderListener);
-		}
+			Future<JobMasterGateway> jobMasterGatewayFuture = getRpcService().connect(jobManagerAddress, JobMasterGateway.class);
 
-		return getRpcService()
-			.execute(new Callable<JobMasterGateway>() {
+			Future<RegistrationResponse> registrationResponseFuture = jobMasterGatewayFuture.thenCombineAsync(jobLeaderIdFuture, new BiFunction<JobMasterGateway, UUID, RegistrationResponse>() {
 				@Override
-				public JobMasterGateway call() throws Exception {
+				public RegistrationResponse apply(JobMasterGateway jobMasterGateway, UUID jobLeaderId) {
+					if (isValid(resourceManagerLeaderId)) {
+						if (jobLeaderId.equals(jobManagerLeaderId)) {
+							if (jobManagerRegistrations.containsKey(jobId)) {
+								JobManagerRegistration oldJobManagerRegistration = jobManagerRegistrations.get(jobId);
+
+								if (oldJobManagerRegistration.getLeaderID().equals(jobLeaderId)) {
+									// same registration
+									log.debug("Job manager {}@{} was already registered.", jobManagerLeaderId, jobManagerAddress);
+								} else {
+									// tell old job manager that he is no longer the job leader
+									disconnectJobManager(
+										oldJobManagerRegistration.getJobID(),
+										new Exception("New job leader for job " + jobId + " found."));
+
+									JobManagerRegistration jobManagerRegistration = new JobManagerRegistration(jobId, jobLeaderId, jobMasterGateway);
+									jobManagerRegistrations.put(jobId, jobManagerRegistration);
+								}
+							} else {
+								// new registration for the job
+								JobManagerRegistration jobManagerRegistration = new JobManagerRegistration(jobId, jobLeaderId, jobMasterGateway);
+
+								jobManagerRegistrations.put(jobId, jobManagerRegistration);
+							}
 
-					if (!leaderSessionID.equals(resourceManagerLeaderId)) {
-						log.warn("Discard registration from JobMaster {} at ({}) because the expected leader session ID {}" +
-								" did not equal the received leader session ID  {}",
-							jobID, jobMasterAddress, leaderSessionID, resourceManagerLeaderId);
-						throw new Exception("Invalid leader session id");
-					}
+							log.info("Registered job manager {}@{} for job {}.", jobManagerLeaderId, jobManagerAddress, jobId);
 
-					final Time timeout = resourceManagerConfiguration.getTimeout();
+							return new JobMasterRegistrationSuccess(
+								resourceManagerConfiguration.getHeartbeatInterval().toMilliseconds(),
+								getLeaderSessionId());
 
-					if (!jobIdLeaderListener.getLeaderID().get(timeout.getSize(), timeout.getUnit())
-							.equals(jobMasterLeaderId)) {
-						throw new Exception("Leader Id did not match");
+						} else {
+							log.debug("The job manager leader id {} did not match the job " +
+								"leader id {}.", jobManagerLeaderId, jobLeaderId);
+							return new RegistrationResponse.Decline("Job manager leader id did not match.");
+						}
+					} else {
+						log.debug("The resource manager leader id changed {}. Discarding job " +
+							"manager registration from {}.", getLeaderSessionId(), jobManagerAddress);
+						return new RegistrationResponse.Decline("Resource manager leader id changed.");
 					}
-
-					return getRpcService().connect(jobMasterAddress, JobMasterGateway.class)
-						.get(timeout.getSize(), timeout.getUnit());
 				}
-			})
-			.handleAsync(new BiFunction<JobMasterGateway, Throwable, RegistrationResponse>() {
-				@Override
-				public RegistrationResponse apply(JobMasterGateway jobMasterGateway, Throwable throwable) {
-
-				if (throwable != null) {
-					return new RegistrationResponse.Decline(throwable.getMessage());
-				} else {
-					if (!leaderSessionID.equals(resourceManagerLeaderId)) {
-						log.warn("Discard registration from JobMaster {} at ({}) because the expected leader session ID {}" +
-								" did not equal the received leader session ID  {}",
-							jobID, jobMasterAddress, leaderSessionID, resourceManagerLeaderId);
-						return new RegistrationResponse.Decline("Invalid leader session id");
-					}
+			}, getMainThreadExecutor());
 
-					try {
-						// LeaderID should be available now, but if not we fail the registration
-						UUID currentJobMasterLeaderId = jobIdLeaderListener.getLeaderID().getNow(null);
-						if (currentJobMasterLeaderId == null || !currentJobMasterLeaderId.equals(jobMasterLeaderId)) {
-							throw new Exception("Leader Id did not match");
+			// handle exceptions which might have occurred in one of the futures inputs of combine
+			return registrationResponseFuture.handleAsync(new BiFunction<RegistrationResponse, Throwable, RegistrationResponse>() {
+				@Override
+				public RegistrationResponse apply(RegistrationResponse registrationResponse, Throwable throwable) {
+					if (throwable != null) {
+						if (log.isDebugEnabled()) {
+							log.debug("Registration of job manager {}@{} failed.", jobManagerLeaderId, jobManagerAddress, throwable);
+						} else {
+							log.info("Registration of job manager {}@{} failed.", jobManagerLeaderId, jobManagerAddress);
 						}
-					} catch (Exception e) {
-						return new RegistrationResponse.Decline(e.getMessage());
-					}
-
-					final JobMasterRegistration registration =
-						new JobMasterRegistration(jobID, jobMasterLeaderId, jobMasterGateway);
 
-					final JobMasterRegistration existingRegistration = jobMasters.put(jobID, registration);
-					if (existingRegistration != null) {
-						log.info("Replacing JobMaster registration for newly registered JobMaster with JobID {}.", jobID);
+						return new RegistrationResponse.Decline(throwable.getMessage());
+					} else {
+						return registrationResponse;
 					}
-					return new JobMasterRegistrationSuccess(
-						resourceManagerConfiguration.getHeartbeatInterval().toMilliseconds(),
-						resourceManagerLeaderId);
 				}
-			}
-		}, getMainThreadExecutor());
+			}, getRpcService().getExecutor());
+		} else {
+			log.debug("Discard register job manager message from {}, because the leader id " +
+				"{} did not match the expected leader id {}.", jobManagerAddress,
+				resourceManagerLeaderId, leaderSessionId);
+
+			return FlinkCompletableFuture.<RegistrationResponse>completed(
+				new RegistrationResponse.Decline("Resource manager leader id did not match."));
+		}
 	}
 
 	/**
@@ -315,7 +341,7 @@ public abstract class ResourceManager<WorkerType extends Serializable>
 		final ResourceID resourceID,
 		final SlotReport slotReport) {
 
-		if (leaderSessionID.equals(resourceManagerLeaderId)) {
+		if (leaderSessionId.equals(resourceManagerLeaderId)) {
 			Future<TaskExecutorGateway> taskExecutorGatewayFuture = getRpcService().connect(taskExecutorAddress, TaskExecutorGateway.class);
 
 			return taskExecutorGatewayFuture.handleAsync(new BiFunction<TaskExecutorGateway, Throwable, RegistrationResponse>() {
@@ -346,12 +372,12 @@ public abstract class ResourceManager<WorkerType extends Serializable>
 		} else {
 			log.warn("Discard registration from TaskExecutor {} at ({}) because the expected leader session ID {} did " +
 					"not equal the received leader session ID  {}",
-				resourceID, taskExecutorAddress, leaderSessionID, resourceManagerLeaderId);
+				resourceID, taskExecutorAddress, leaderSessionId, resourceManagerLeaderId);
 
 			return FlinkCompletableFuture.<RegistrationResponse>completed(
 				new RegistrationResponse.Decline("Discard registration because the leader id " +
 					resourceManagerLeaderId + " does not match the expected leader id " +
-					leaderSessionID + '.'));
+					leaderSessionId + '.'));
 		}
 	}
 
@@ -368,11 +394,11 @@ public abstract class ResourceManager<WorkerType extends Serializable>
 			SlotRequest slotRequest) {
 
 		JobID jobId = slotRequest.getJobId();
-		JobMasterRegistration jobMasterRegistration = jobMasters.get(jobId);
+		JobManagerRegistration jobManagerRegistration = jobManagerRegistrations.get(jobId);
 
-		if (jobMasterRegistration != null
-				&& jobMasterLeaderID.equals(jobMasterRegistration.getLeaderID())
-				&& resourceManagerLeaderID.equals(leaderSessionID)) {
+		if (jobManagerRegistration != null
+				&& jobMasterLeaderID.equals(jobManagerRegistration.getLeaderID())
+				&& resourceManagerLeaderID.equals(leaderSessionId)) {
 			return slotManager.requestSlot(slotRequest);
 		} else {
 			log.info("Ignoring slot request for unknown JobMaster with JobID {}", jobId);
@@ -393,7 +419,7 @@ public abstract class ResourceManager<WorkerType extends Serializable>
 			final InstanceID instanceID,
 			final SlotID slotId) {
 
-		if (resourceManagerLeaderId.equals(leaderSessionID)) {
+		if (resourceManagerLeaderId.equals(leaderSessionId)) {
 			final ResourceID resourceId = slotId.getResourceID();
 			WorkerRegistration<WorkerType> registration = taskExecutors.get(resourceId);
 
@@ -413,7 +439,7 @@ public abstract class ResourceManager<WorkerType extends Serializable>
 		} else {
 			log.debug("Discarding notify slot available message for slot {}, because the " +
 				"leader id {} did not match the expected leader id {}.", slotId,
-				resourceManagerLeaderId, leaderSessionID);
+				resourceManagerLeaderId, leaderSessionId);
 		}
 	}
 
@@ -471,6 +497,150 @@ public abstract class ResourceManager<WorkerType extends Serializable>
 	}
 
 	// ------------------------------------------------------------------------
+	//  Testing methods
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Gets the leader session id of current resourceManager.
+	 *
+	 * @return return the leaderSessionId of current resourceManager, this returns null until the current resourceManager is granted leadership.
+	 */
+	@VisibleForTesting
+	UUID getLeaderSessionId() {
+		return leaderSessionId;
+	}
+
+	// ------------------------------------------------------------------------
+	//  Internal methods
+	// ------------------------------------------------------------------------
+
+	private void clearState() {
+		jobManagerRegistrations.clear();
+		taskExecutors.clear();
+		slotManager.clearState();
+
+		try {
+			jobLeaderIdService.clear();
+		} catch (Exception e) {
+			onFatalError(new ResourceManagerException("Could not properly clear the job leader id service.", e));
+		}
+
+		leaderSessionId = null;
+	}
+
+	/**
+	 * Disconnects the job manager which is connected for the given job from the resource manager.
+	 *
+	 * @param jobId identifying the job whose leader shall be disconnected
+	 */
+	protected void disconnectJobManager(JobID jobId, Exception cause) {
+		JobManagerRegistration jobManagerRegistration = jobManagerRegistrations.remove(jobId);
+
+		if (jobManagerRegistration != null) {
+			log.info("Disconnect job manager {}@{} for job {} from the resource manager.",
+				jobManagerRegistration.getLeaderID(),
+				jobManagerRegistration.getJobManagerGateway().getAddress(),
+				jobId);
+
+			JobMasterGateway jobMasterGateway = jobManagerRegistration.getJobManagerGateway();
+
+			// tell the job manager about the disconnect
+			jobMasterGateway.disconnectResourceManager(jobManagerRegistration.getLeaderID(), getLeaderSessionId(), cause);
+		} else {
+			log.debug("There was no registered job manager for job {}.", jobId);
+		}
+	}
+
+	/**
+	 * Checks whether the given resource manager leader id is matching the current leader id and
+	 * not null.
+	 *
+	 * @param resourceManagerLeaderId to check
+	 * @return True if the given leader id matches the actual leader id and is not null; otherwise false
+	 */
+	protected boolean isValid(UUID resourceManagerLeaderId) {
+		if (resourceManagerLeaderId == null) {
+			return false;
+		} else {
+			return resourceManagerLeaderId.equals(leaderSessionId);
+		}
+	}
+
+	protected void removeJob(JobID jobId) {
+		try {
+			jobLeaderIdService.removeJob(jobId);
+		} catch (Exception e) {
+			log.warn("Could not properly remove the job {} from the job leader id service.", jobId, e);
+		}
+
+		if (jobManagerRegistrations.containsKey(jobId)) {
+			disconnectJobManager(jobId, new Exception("Job " + jobId + "was removed"));
+		}
+	}
+
+	protected void jobLeaderLostLeadership(JobID jobId, UUID oldJobLeaderId) {
+		if (jobManagerRegistrations.containsKey(jobId)) {
+			JobManagerRegistration jobManagerRegistration = jobManagerRegistrations.get(jobId);
+
+			if (jobManagerRegistration.getLeaderID().equals(oldJobLeaderId)) {
+				disconnectJobManager(jobId, new Exception("Job leader lost leadership."));
+			} else {
+				log.debug("Discarding job leader lost leadership, because a new job leader was found for job {}. ", jobId);
+			}
+		} else {
+			log.debug("Discard job leader lost leadership for outdated leader {} for job {}.", oldJobLeaderId, jobId);
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  Info messaging
+	// ------------------------------------------------------------------------
+
+	public void sendInfoMessage(final String message) {
+		getRpcService().execute(new Runnable() {
+			@Override
+			public void run() {
+				InfoMessage infoMessage = new InfoMessage(message);
+				for (InfoMessageListenerRpcGateway listenerRpcGateway : infoMessageListeners.values()) {
+					listenerRpcGateway
+						.notifyInfoMessage(infoMessage);
+				}
+			}
+		});
+	}
+
+	// ------------------------------------------------------------------------
+	//  Error Handling
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Notifies the ResourceManager that a fatal error has occurred and it cannot proceed.
+	 * This method should be used when asynchronous threads want to notify the
+	 * ResourceManager of a fatal error.
+	 *
+	 * @param t The exception describing the fatal error
+	 */
+	void onFatalErrorAsync(final Throwable t) {
+		runAsync(new Runnable() {
+			@Override
+			public void run() {
+				onFatalError(t);
+			}
+		});
+	}
+
+	/**
+	 * Notifies the ResourceManager that a fatal error has occurred and it cannot proceed.
+	 * This method must only be called from within the ResourceManager's main thread.
+	 *
+	 * @param t The exception describing the fatal error
+	 */
+	void onFatalError(Throwable t) {
+		log.error("Fatal error occurred.", t);
+		fatalErrorHandler.onFatalError(t);
+	}
+
+	// ------------------------------------------------------------------------
 	//  Leader Contender
 	// ------------------------------------------------------------------------
 
@@ -487,11 +657,11 @@ public abstract class ResourceManager<WorkerType extends Serializable>
 				log.info("ResourceManager {} was granted leadership with leader session ID {}", getAddress(), newLeaderSessionID);
 
 				// clear the state if we've been the leader before
-				if (leaderSessionID != null) {
+				if (leaderSessionId != null) {
 					clearState();
 				}
 
-				leaderSessionID = newLeaderSessionID;
+				leaderSessionId = newLeaderSessionID;
 
 				getRpcService().execute(new Runnable() {
 					@Override
@@ -516,7 +686,7 @@ public abstract class ResourceManager<WorkerType extends Serializable>
 
 				clearState();
 
-				leaderSessionID = null;
+				leaderSessionId = null;
 			}
 		});
 	}
@@ -556,82 +726,15 @@ public abstract class ResourceManager<WorkerType extends Serializable>
 	}
 
 	// ------------------------------------------------------------------------
-	//  Error Handling
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Notifies the ResourceManager that a fatal error has occurred and it cannot proceed.
-	 * This method should be used when asynchronous threads want to notify the
-	 * ResourceManager of a fatal error.
-	 *
-	 * @param t The exception describing the fatal error
-	 */
-	void onFatalErrorAsync(final Throwable t) {
-		runAsync(new Runnable() {
-			@Override
-			public void run() {
-				onFatalError(t);
-			}
-		});
-	}
-
-	/**
-	 * Notifies the ResourceManager that a fatal error has occurred and it cannot proceed.
-	 * This method must only be called from within the ResourceManager's main thread.
-	 *
-	 * @param t The exception describing the fatal error
-	 */
-	void onFatalError(Throwable t) {
-		log.error("Fatal error occurred.", t);
-		fatalErrorHandler.onFatalError(t);
-	}
-
-	// ------------------------------------------------------------------------
-	//  Testing methods
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Gets the leader session id of current resourceManager.
-	 *
-	 * @return return the leaderSessionId of current resourceManager, this returns null until the current resourceManager is granted leadership.
-	 */
-	@VisibleForTesting
-	UUID getLeaderSessionID() {
-		return leaderSessionID;
-	}
-
-	// ------------------------------------------------------------------------
-	//  Internal methods
-	// ------------------------------------------------------------------------
-
-	private void clearState() {
-		jobMasters.clear();
-		taskExecutors.clear();
-		slotManager.clearState();
-		Iterator<JobIdLeaderListener> leaderListenerIterator =
-			leaderListeners.values().iterator();
-		while (leaderListenerIterator.hasNext()) {
-			JobIdLeaderListener listener = leaderListenerIterator.next();
-			try {
-				listener.stopService();
-			} catch (Exception e) {
-				onFatalError(e);
-			}
-			leaderListenerIterator.remove();
-		}
-		leaderSessionID = new UUID(0, 0);
-	}
-
-	// ------------------------------------------------------------------------
 	//  Framework specific behavior
 	// ------------------------------------------------------------------------
 
 	/**
 	 * Initializes the framework specific components.
 	 *
-	 * @throws Exception Exceptions during initialization cause the resource manager to fail.
+	 * @throws ResourceManagerException which occurs during initialization and causes the resource manager to fail.
 	 */
-	protected abstract void initialize() throws Exception;
+	protected abstract void initialize() throws ResourceManagerException;
 
 	/**
 	 * The framework specific code for shutting down the application. This should report the
@@ -659,24 +762,7 @@ public abstract class ResourceManager<WorkerType extends Serializable>
 	protected abstract WorkerType workerStarted(ResourceID resourceID);
 
 	// ------------------------------------------------------------------------
-	//  Info messaging
-	// ------------------------------------------------------------------------
-
-	public void sendInfoMessage(final String message) {
-		getRpcService().execute(new Runnable() {
-			@Override
-			public void run() {
-				InfoMessage infoMessage = new InfoMessage(message);
-				for (InfoMessageListenerRpcGateway listenerRpcGateway : infoMessageListeners.values()) {
-					listenerRpcGateway
-						.notifyInfoMessage(infoMessage);
-				}
-			}
-		});
-	}
-
-	// ------------------------------------------------------------------------
-	//  Resource Manager Services
+	//  Static utility classes
 	// ------------------------------------------------------------------------
 
 	protected ResourceManagerServices createResourceManagerServices() {
@@ -687,7 +773,7 @@ public abstract class ResourceManager<WorkerType extends Serializable>
 
 		@Override
 		public UUID getLeaderID() {
-			return ResourceManager.this.leaderSessionID;
+			return ResourceManager.this.leaderSessionId;
 		}
 
 		@Override
@@ -706,76 +792,26 @@ public abstract class ResourceManager<WorkerType extends Serializable>
 		}
 	}
 
-	/**
-	 * Leader instantiated for each connected JobMaster
-	 */
-	private class JobIdLeaderListener implements LeaderRetrievalListener {
-
-		private final JobID jobID;
-		private final LeaderRetrievalService retrievalService;
-
-		private final FlinkCompletableFuture<UUID> initialLeaderIdFuture;
-
-		private volatile UUID leaderID;
-
-		private JobIdLeaderListener(
-				JobID jobID,
-				LeaderRetrievalService retrievalService) throws Exception {
-			this.jobID = jobID;
-			this.retrievalService = retrievalService;
-			this.initialLeaderIdFuture = new FlinkCompletableFuture<>();
-			this.retrievalService.start(this);
-		}
+	private class JobLeaderIdActionsImpl implements JobLeaderIdActions {
 
-		public Future<UUID> getLeaderID() {
-			if (!initialLeaderIdFuture.isDone()) {
-				return initialLeaderIdFuture;
-			} else {
-				return FlinkCompletableFuture.completed(leaderID);
-			}
-		}
-
-		public JobID getJobID() {
-			return jobID;
-		}
-
-
-		public void stopService() throws Exception {
-			retrievalService.stop();
+		@Override
+		public void jobLeaderLostLeadership(final JobID jobId, final UUID oldJobLeaderId) {
+			ResourceManager.this.jobLeaderLostLeadership(jobId, oldJobLeaderId);
 		}
 
 		@Override
-		public void notifyLeaderAddress(final String leaderAddress, final UUID leaderSessionID) {
-			this.leaderID = leaderSessionID;
-
-			if (!initialLeaderIdFuture.isDone()) {
-				initialLeaderIdFuture.complete(leaderSessionID);
-			}
-
-			ResourceManager.this.runAsync(new Runnable() {
+		public void removeJob(final JobID jobId) {
+			runAsync(new Runnable() {
 				@Override
 				public void run() {
-					JobMasterRegistration jobMasterRegistration = ResourceManager.this.jobMasters.get(jobID);
-					if (jobMasterRegistration == null || !jobMasterRegistration.getLeaderID().equals(leaderSessionID)) {
-						// registration is not valid anymore, remove registration
-						ResourceManager.this.jobMasters.remove(jobID);
-						// leader listener is not necessary anymore
-						JobIdLeaderListener listener = ResourceManager.this.leaderListeners.remove(jobID);
-						if (listener != null) {
-							try {
-								listener.stopService();
-							} catch (Exception e) {
-								ResourceManager.this.handleError(e);
-							}
-						}
-					}
+					ResourceManager.this.removeJob(jobId);
 				}
 			});
 		}
 
 		@Override
-		public void handleError(final Exception exception) {
-			ResourceManager.this.handleError(exception);
+		public void handleError(Throwable error) {
+			onFatalErrorAsync(error);
 		}
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/c001becc/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java
index 968eeb8..0a37bb9 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java
@@ -49,7 +49,7 @@ public interface ResourceManagerGateway extends RpcGateway {
 	 * @param timeout                 Timeout for the future to complete
 	 * @return Future registration response
 	 */
-	Future<RegistrationResponse> registerJobMaster(
+	Future<RegistrationResponse> registerJobManager(
 		UUID resourceManagerLeaderId,
 		UUID jobMasterLeaderId,
 		String jobMasterAddress,

http://git-wip-us.apache.org/repos/asf/flink/blob/c001becc/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/StandaloneResourceManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/StandaloneResourceManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/StandaloneResourceManager.java
index 926be0d..73c8a2d 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/StandaloneResourceManager.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/StandaloneResourceManager.java
@@ -23,6 +23,7 @@ import org.apache.flink.runtime.clusterframework.types.ResourceID;
 import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
 import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
 import org.apache.flink.runtime.metrics.MetricRegistry;
+import org.apache.flink.runtime.resourcemanager.exceptions.ResourceManagerException;
 import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManagerFactory;
 import org.apache.flink.runtime.rpc.FatalErrorHandler;
 import org.apache.flink.runtime.rpc.RpcService;
@@ -41,6 +42,7 @@ public class StandaloneResourceManager extends ResourceManager<ResourceID> {
 			HighAvailabilityServices highAvailabilityServices,
 			SlotManagerFactory slotManagerFactory,
 			MetricRegistry metricRegistry,
+			JobLeaderIdService jobLeaderIdService,
 			FatalErrorHandler fatalErrorHandler) {
 		super(
 			rpcService,
@@ -48,11 +50,12 @@ public class StandaloneResourceManager extends ResourceManager<ResourceID> {
 			highAvailabilityServices,
 			slotManagerFactory,
 			metricRegistry,
+			jobLeaderIdService,
 			fatalErrorHandler);
 	}
 
 	@Override
-	protected void initialize() throws Exception {
+	protected void initialize() throws ResourceManagerException {
 		// nothing to initialize
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/c001becc/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/exceptions/ResourceManagerRunner.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/exceptions/ResourceManagerRunner.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/exceptions/ResourceManagerRunner.java
index 1e6f04c..0c7e4e4 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/exceptions/ResourceManagerRunner.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/exceptions/ResourceManagerRunner.java
@@ -21,6 +21,7 @@ package org.apache.flink.runtime.resourcemanager.exceptions;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
 import org.apache.flink.runtime.metrics.MetricRegistry;
+import org.apache.flink.runtime.resourcemanager.JobLeaderIdService;
 import org.apache.flink.runtime.resourcemanager.ResourceManager;
 import org.apache.flink.runtime.resourcemanager.ResourceManagerConfiguration;
 import org.apache.flink.runtime.resourcemanager.StandaloneResourceManager;
@@ -48,7 +49,7 @@ public class ResourceManagerRunner implements FatalErrorHandler {
 			final Configuration configuration,
 			final RpcService rpcService,
 			final HighAvailabilityServices highAvailabilityServices,
-			final MetricRegistry metricRegistry) throws ConfigurationException {
+			final MetricRegistry metricRegistry) throws Exception {
 
 		Preconditions.checkNotNull(configuration);
 		Preconditions.checkNotNull(rpcService);
@@ -57,6 +58,7 @@ public class ResourceManagerRunner implements FatalErrorHandler {
 
 		final ResourceManagerConfiguration resourceManagerConfiguration = ResourceManagerConfiguration.fromConfiguration(configuration);
 		final SlotManagerFactory slotManagerFactory = new DefaultSlotManager.Factory();
+		final JobLeaderIdService jobLeaderIdService = new JobLeaderIdService(highAvailabilityServices);
 
 		this.resourceManager = new StandaloneResourceManager(
 			rpcService,
@@ -64,6 +66,7 @@ public class ResourceManagerRunner implements FatalErrorHandler {
 			highAvailabilityServices,
 			slotManagerFactory,
 			metricRegistry,
+			jobLeaderIdService,
 			this);
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/c001becc/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/registration/JobManagerRegistration.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/registration/JobManagerRegistration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/registration/JobManagerRegistration.java
new file mode 100644
index 0000000..a1deb65
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/registration/JobManagerRegistration.java
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.registration;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.jobmaster.JobMasterGateway;
+import org.apache.flink.util.Preconditions;
+
+import java.util.UUID;
+
+/**
+ * Container for JobManager related registration information, such as the leader id or the job id.
+ */
+public class JobManagerRegistration {
+	private final JobID jobID;
+
+	private final UUID leaderID;
+
+	private final JobMasterGateway jobManagerGateway;
+
+	public JobManagerRegistration(
+			JobID jobID,
+			UUID leaderID,
+			JobMasterGateway jobManagerGateway) {
+		this.jobID = Preconditions.checkNotNull(jobID);
+		this.leaderID = Preconditions.checkNotNull(leaderID);
+		this.jobManagerGateway = Preconditions.checkNotNull(jobManagerGateway);
+	}
+
+	public JobID getJobID() {
+		return jobID;
+	}
+
+	public UUID getLeaderID() {
+		return leaderID;
+	}
+
+	public JobMasterGateway getJobManagerGateway() {
+		return jobManagerGateway;
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/c001becc/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/registration/JobMasterRegistration.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/registration/JobMasterRegistration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/registration/JobMasterRegistration.java
deleted file mode 100644
index f417935..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/registration/JobMasterRegistration.java
+++ /dev/null
@@ -1,62 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.runtime.resourcemanager.registration;
-
-import org.apache.flink.api.common.JobID;
-import org.apache.flink.runtime.jobmaster.JobMasterGateway;
-
-import java.util.UUID;
-
-/**
- * This class is responsible for grouping the JobMasterGateway and the JobMaster's
- * leader id
- */
-public class JobMasterRegistration {
-
-	private static final long serialVersionUID = -2062957799469434614L;
-
-	private final JobID jobID;
-
-	private final UUID leaderID;
-
-	private final JobMasterGateway jobMasterGateway;
-
-	public JobMasterRegistration(
-			JobID jobID,
-			UUID leaderID,
-			JobMasterGateway jobMasterGateway) {
-		this.jobID = jobID;
-		this.leaderID = leaderID;
-		this.jobMasterGateway = jobMasterGateway;
-	}
-
-	public JobID getJobID() {
-		return jobID;
-	}
-
-
-	public UUID getLeaderID() {
-		return leaderID;
-	}
-
-	public JobMasterGateway getJobMasterGateway() {
-		return jobMasterGateway;
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/c001becc/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java
index 44719c8..6e3fb40 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java
@@ -96,8 +96,6 @@ public class AkkaRpcService implements RpcService {
 
 		Address actorSystemAddress = AkkaUtils.getAddress(actorSystem);
 
-
-
 		if (actorSystemAddress.host().isDefined()) {
 			address = actorSystemAddress.host().get();
 		} else {

http://git-wip-us.apache.org/repos/asf/flink/blob/c001becc/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerHATest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerHATest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerHATest.java
index cb38e6e..2e52eeb 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerHATest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerHATest.java
@@ -49,6 +49,7 @@ public class ResourceManagerHATest {
 		ResourceManagerConfiguration resourceManagerConfiguration = new ResourceManagerConfiguration(Time.seconds(5L), Time.seconds(5L));
 		SlotManagerFactory slotManagerFactory = new TestingSlotManagerFactory();
 		MetricRegistry metricRegistry = mock(MetricRegistry.class);
+		JobLeaderIdService jobLeaderIdService = new JobLeaderIdService(highAvailabilityServices);
 		TestingFatalErrorHandler testingFatalErrorHandler = new TestingFatalErrorHandler();
 
 		final ResourceManager resourceManager =
@@ -58,17 +59,18 @@ public class ResourceManagerHATest {
 				highAvailabilityServices,
 				slotManagerFactory,
 				metricRegistry,
+				jobLeaderIdService,
 				testingFatalErrorHandler);
 		resourceManager.start();
 		// before grant leadership, resourceManager's leaderId is null
-		Assert.assertEquals(null, resourceManager.getLeaderSessionID());
+		Assert.assertEquals(null, resourceManager.getLeaderSessionId());
 		final UUID leaderId = UUID.randomUUID();
 		leaderElectionService.isLeader(leaderId);
 		// after grant leadership, resourceManager's leaderId has value
-		Assert.assertEquals(leaderId, resourceManager.getLeaderSessionID());
+		Assert.assertEquals(leaderId, resourceManager.getLeaderSessionId());
 		// then revoke leadership, resourceManager's leaderId is null again
 		leaderElectionService.notLeader();
-		Assert.assertEquals(null, resourceManager.getLeaderSessionID());
+		Assert.assertEquals(null, resourceManager.getLeaderSessionId());
 
 		if (testingFatalErrorHandler.hasExceptionOccurred()) {
 			testingFatalErrorHandler.rethrowError();

http://git-wip-us.apache.org/repos/asf/flink/blob/c001becc/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerJobMasterTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerJobMasterTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerJobMasterTest.java
index 7b8d254..2622634 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerJobMasterTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerJobMasterTest.java
@@ -71,8 +71,8 @@ public class ResourceManagerJobMasterTest {
 		final UUID rmLeaderSessionId = grantResourceManagerLeadership(resourceManagerLeaderElectionService);
 
 		// test response successful
-		Future<RegistrationResponse> successfulFuture = resourceManager.registerJobMaster(rmLeaderSessionId, jmLeaderID, jobMasterAddress, jobID);
-		RegistrationResponse response = successfulFuture.get(5, TimeUnit.SECONDS);
+		Future<RegistrationResponse> successfulFuture = resourceManager.registerJobManager(rmLeaderSessionId, jmLeaderID, jobMasterAddress, jobID);
+		RegistrationResponse response = successfulFuture.get(5L, TimeUnit.SECONDS);
 		assertTrue(response instanceof JobMasterRegistrationSuccess);
 
 		if (testingFatalErrorHandler.hasExceptionOccurred()) {
@@ -96,7 +96,7 @@ public class ResourceManagerJobMasterTest {
 
 		// test throw exception when receive a registration from job master which takes unmatched leaderSessionId
 		UUID differentLeaderSessionID = UUID.randomUUID();
-		Future<RegistrationResponse> unMatchedLeaderFuture = resourceManager.registerJobMaster(differentLeaderSessionID, jmLeaderID, jobMasterAddress, jobID);
+		Future<RegistrationResponse> unMatchedLeaderFuture = resourceManager.registerJobManager(differentLeaderSessionID, jmLeaderID, jobMasterAddress, jobID);
 		assertTrue(unMatchedLeaderFuture.get(5, TimeUnit.SECONDS) instanceof RegistrationResponse.Decline);
 
 		if (testingFatalErrorHandler.hasExceptionOccurred()) {
@@ -120,7 +120,7 @@ public class ResourceManagerJobMasterTest {
 
 		// test throw exception when receive a registration from job master which takes unmatched leaderSessionId
 		UUID differentLeaderSessionID = UUID.randomUUID();
-		Future<RegistrationResponse> unMatchedLeaderFuture = resourceManager.registerJobMaster(rmLeaderSessionId, differentLeaderSessionID, jobMasterAddress, jobID);
+		Future<RegistrationResponse> unMatchedLeaderFuture = resourceManager.registerJobManager(rmLeaderSessionId, differentLeaderSessionID, jobMasterAddress, jobID);
 		assertTrue(unMatchedLeaderFuture.get(5, TimeUnit.SECONDS) instanceof RegistrationResponse.Decline);
 
 		if (testingFatalErrorHandler.hasExceptionOccurred()) {
@@ -144,7 +144,7 @@ public class ResourceManagerJobMasterTest {
 
 		// test throw exception when receive a registration from job master which takes invalid address
 		String invalidAddress = "/jobMasterAddress2";
-		Future<RegistrationResponse> invalidAddressFuture = resourceManager.registerJobMaster(rmLeaderSessionId, jmLeaderSessionId, invalidAddress, jobID);
+		Future<RegistrationResponse> invalidAddressFuture = resourceManager.registerJobManager(rmLeaderSessionId, jmLeaderSessionId, invalidAddress, jobID);
 		assertTrue(invalidAddressFuture.get(5, TimeUnit.SECONDS) instanceof RegistrationResponse.Decline);
 
 		if (testingFatalErrorHandler.hasExceptionOccurred()) {
@@ -168,7 +168,7 @@ public class ResourceManagerJobMasterTest {
 
 		JobID unknownJobIDToHAServices = new JobID();
 		// verify return RegistrationResponse.Decline when failed to start a job master Leader retrieval listener
-		Future<RegistrationResponse> declineFuture = resourceManager.registerJobMaster(rmLeaderSessionId, jmLeaderSessionId, jobMasterAddress, unknownJobIDToHAServices);
+		Future<RegistrationResponse> declineFuture = resourceManager.registerJobManager(rmLeaderSessionId, jmLeaderSessionId, jobMasterAddress, unknownJobIDToHAServices);
 		RegistrationResponse response = declineFuture.get(5, TimeUnit.SECONDS);
 		assertTrue(response instanceof RegistrationResponse.Decline);
 
@@ -196,6 +196,7 @@ public class ResourceManagerJobMasterTest {
 		ResourceManagerConfiguration resourceManagerConfiguration = new ResourceManagerConfiguration(Time.seconds(5L), Time.seconds(5L));
 		SlotManagerFactory slotManagerFactory = new TestingSlotManagerFactory();
 		MetricRegistry metricRegistry = mock(MetricRegistry.class);
+		JobLeaderIdService jobLeaderIdService = new JobLeaderIdService(highAvailabilityServices);
 
 		ResourceManager resourceManager = new StandaloneResourceManager(
 			rpcService,
@@ -203,6 +204,7 @@ public class ResourceManagerJobMasterTest {
 			highAvailabilityServices,
 			slotManagerFactory,
 			metricRegistry,
+			jobLeaderIdService,
 			fatalErrorHandler);
 		resourceManager.start();
 		return resourceManager;

http://git-wip-us.apache.org/repos/asf/flink/blob/c001becc/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTaskExecutorTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTaskExecutorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTaskExecutorTest.java
index 4640eab..1016181 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTaskExecutorTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTaskExecutorTest.java
@@ -148,6 +148,7 @@ public class ResourceManagerTaskExecutorTest {
 		TestingSlotManagerFactory slotManagerFactory = new TestingSlotManagerFactory();
 		ResourceManagerConfiguration resourceManagerConfiguration = new ResourceManagerConfiguration(Time.seconds(5L), Time.seconds(5L));
 		MetricRegistry metricRegistry = mock(MetricRegistry.class);
+		JobLeaderIdService jobLeaderIdService = new JobLeaderIdService(highAvailabilityServices);
 
 
 		StandaloneResourceManager resourceManager =
@@ -157,6 +158,7 @@ public class ResourceManagerTaskExecutorTest {
 				highAvailabilityServices,
 				slotManagerFactory,
 				metricRegistry,
+				jobLeaderIdService,
 				fatalErrorHandler);
 		resourceManager.start();
 		return resourceManager;

http://git-wip-us.apache.org/repos/asf/flink/blob/c001becc/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotProtocolTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotProtocolTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotProtocolTest.java
index 08ceb86..a3ba436 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotProtocolTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotProtocolTest.java
@@ -32,6 +32,7 @@ import org.apache.flink.runtime.leaderelection.TestingLeaderElectionService;
 import org.apache.flink.runtime.leaderelection.TestingLeaderRetrievalService;
 import org.apache.flink.runtime.metrics.MetricRegistry;
 import org.apache.flink.runtime.registration.RegistrationResponse;
+import org.apache.flink.runtime.resourcemanager.JobLeaderIdService;
 import org.apache.flink.runtime.resourcemanager.ResourceManager;
 import org.apache.flink.runtime.resourcemanager.ResourceManagerConfiguration;
 import org.apache.flink.runtime.resourcemanager.ResourceManagerServices;
@@ -107,6 +108,7 @@ public class SlotProtocolTest extends TestLogger {
 			configureHA(testingHaServices, jobID, rmAddress, rmLeaderID, jmAddress, jmLeaderID);
 
 		ResourceManagerConfiguration resourceManagerConfiguration = new ResourceManagerConfiguration(Time.seconds(5L), Time.seconds(5L));
+		JobLeaderIdService jobLeaderIdService = new JobLeaderIdService(testingHaServices);
 
 		final TestingSlotManagerFactory slotManagerFactory = new TestingSlotManagerFactory();
 		SpiedResourceManager resourceManager =
@@ -116,12 +118,13 @@ public class SlotProtocolTest extends TestLogger {
 				testingHaServices,
 				slotManagerFactory,
 				mock(MetricRegistry.class),
+				jobLeaderIdService,
 				mock(FatalErrorHandler.class));
 		resourceManager.start();
 		rmLeaderElectionService.isLeader(rmLeaderID);
 
 		Future<RegistrationResponse> registrationFuture =
-			resourceManager.registerJobMaster(rmLeaderID, jmLeaderID, jmAddress, jobID);
+			resourceManager.registerJobManager(rmLeaderID, jmLeaderID, jmAddress, jobID);
 		try {
 			registrationFuture.get(5, TimeUnit.SECONDS);
 		} catch (Exception e) {
@@ -207,6 +210,8 @@ public class SlotProtocolTest extends TestLogger {
 
 		ResourceManagerConfiguration resourceManagerConfiguration = new ResourceManagerConfiguration(Time.seconds(5L), Time.seconds(5L));
 
+		JobLeaderIdService jobLeaderIdService = new JobLeaderIdService(testingHaServices);
+
 		TestingSlotManagerFactory slotManagerFactory = new TestingSlotManagerFactory();
 		ResourceManager<ResourceID> resourceManager =
 			Mockito.spy(new StandaloneResourceManager(
@@ -215,6 +220,7 @@ public class SlotProtocolTest extends TestLogger {
 				testingHaServices,
 				slotManagerFactory,
 				mock(MetricRegistry.class),
+				jobLeaderIdService,
 				mock(FatalErrorHandler.class)));
 		resourceManager.start();
 		rmLeaderElectionService.isLeader(rmLeaderID);
@@ -222,7 +228,7 @@ public class SlotProtocolTest extends TestLogger {
 		Thread.sleep(1000);
 
 		Future<RegistrationResponse> registrationFuture =
-			resourceManager.registerJobMaster(rmLeaderID, jmLeaderID, jmAddress, jobID);
+			resourceManager.registerJobManager(rmLeaderID, jmLeaderID, jmAddress, jobID);
 		try {
 			registrationFuture.get(5L, TimeUnit.SECONDS);
 		} catch (Exception e) {
@@ -290,6 +296,7 @@ public class SlotProtocolTest extends TestLogger {
 				HighAvailabilityServices highAvailabilityServices,
 				SlotManagerFactory slotManagerFactory,
 				MetricRegistry metricRegistry,
+				JobLeaderIdService jobLeaderIdService,
 				FatalErrorHandler fatalErrorHandler) {
 			super(
 				rpcService,
@@ -297,6 +304,7 @@ public class SlotProtocolTest extends TestLogger {
 				highAvailabilityServices,
 				slotManagerFactory,
 				metricRegistry,
+				jobLeaderIdService,
 				fatalErrorHandler);
 		}
 


[34/52] [abbrv] flink git commit: [FLINK-5238] [minicluster] MiniCluster starts local communication if only one TaskManager is used

Posted by se...@apache.org.
[FLINK-5238] [minicluster] MiniCluster starts local communication if only one TaskManager is used


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

Branch: refs/heads/master
Commit: 440e7571a9c91fd1e01addff1b5370563db96abc
Parents: fbfb234
Author: Stephan Ewen <se...@apache.org>
Authored: Fri Dec 2 17:00:25 2016 +0100
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Dec 23 20:54:26 2016 +0100

----------------------------------------------------------------------
 .../flink/runtime/minicluster/MiniCluster.java  |  4 +++-
 .../runtime/taskexecutor/TaskManagerRunner.java | 22 +++++++++++++++-----
 2 files changed, 20 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/440e7571/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java
index 1b9f265..29a6e59 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java
@@ -525,6 +525,7 @@ public class MiniCluster {
 			RpcService[] taskManagerRpcServices) throws Exception {
 
 		final TaskManagerRunner[] taskManagerRunners = new TaskManagerRunner[numTaskManagers];
+		final boolean localCommunication = numTaskManagers == 1;
 
 		for (int i = 0; i < numTaskManagers; i++) {
 			taskManagerRunners[i] = new TaskManagerRunner(
@@ -532,7 +533,8 @@ public class MiniCluster {
 				new ResourceID(UUID.randomUUID().toString()),
 				taskManagerRpcServices[i],
 				haServices,
-				metricRegistry);
+				metricRegistry,
+				localCommunication);
 
 			taskManagerRunners[i].start();
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/440e7571/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java
index a18ff40..1145a46 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java
@@ -66,11 +66,22 @@ public class TaskManagerRunner implements FatalErrorHandler {
 	private final TaskExecutor taskManager;
 
 	public TaskManagerRunner(
+			Configuration configuration,
+			ResourceID resourceID,
+			RpcService rpcService,
+			HighAvailabilityServices highAvailabilityServices,
+			MetricRegistry metricRegistry) throws Exception {
+
+		this(configuration, resourceID, rpcService, highAvailabilityServices, metricRegistry, false);
+	}
+
+	public TaskManagerRunner(
 		Configuration configuration,
 		ResourceID resourceID,
 		RpcService rpcService,
 		HighAvailabilityServices highAvailabilityServices,
-		MetricRegistry metricRegistry) throws Exception {
+		MetricRegistry metricRegistry,
+		boolean localCommunicationOnly) throws Exception {
 
 		this.configuration = Preconditions.checkNotNull(configuration);
 		this.resourceID = Preconditions.checkNotNull(resourceID);
@@ -80,10 +91,11 @@ public class TaskManagerRunner implements FatalErrorHandler {
 
 		InetAddress remoteAddress = InetAddress.getByName(rpcService.getAddress());
 
-		TaskManagerServicesConfiguration taskManagerServicesConfiguration = TaskManagerServicesConfiguration.fromConfiguration(
-			configuration,
-			remoteAddress,
-			false);
+		TaskManagerServicesConfiguration taskManagerServicesConfiguration = 
+				TaskManagerServicesConfiguration.fromConfiguration(
+						configuration,
+						remoteAddress,
+						localCommunicationOnly);
 
 		TaskManagerServices taskManagerServices = TaskManagerServices.fromConfiguration(
 			taskManagerServicesConfiguration,


[30/52] [abbrv] flink git commit: [FLINK-4954] [rpc] Discard messages when AkkaRpcActor is in state Processing.STOP

Posted by se...@apache.org.
[FLINK-4954] [rpc] Discard messages when AkkaRpcActor is in state Processing.STOP

When the AkkaRpcActor receives a message while being in state Processing.STOP it will discard
it and send an AkkaRpcException back to the caller. This replaces the old stashing behaviour
which had the problem that it was just a best effort approach to keep all received messages.
Distributed components should not rely on this behaviour. That's why it was replaced with discarding
messages.


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

Branch: refs/heads/master
Commit: 006a19d4e601a0917e073215d866b5b87dce375f
Parents: 522edae
Author: Till Rohrmann <tr...@apache.org>
Authored: Fri Oct 28 14:07:28 2016 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Dec 23 20:54:25 2016 +0100

----------------------------------------------------------------------
 .../flink/runtime/rpc/akka/AkkaRpcActor.java    | 23 ++++++-----
 .../rpc/akka/exceptions/AkkaRpcException.java   | 41 ++++++++++++++++++++
 .../rpc/exceptions/RpcConnectionException.java  |  4 +-
 .../runtime/rpc/exceptions/RpcException.java    | 39 +++++++++++++++++++
 .../runtime/rpc/akka/AkkaRpcActorTest.java      | 21 ++++++++--
 5 files changed, 112 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/006a19d4/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActor.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActor.java
index c21383a..fe6b23b 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActor.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActor.java
@@ -20,7 +20,7 @@ package org.apache.flink.runtime.rpc.akka;
 
 import akka.actor.ActorRef;
 import akka.actor.Status;
-import akka.actor.UntypedActorWithStash;
+import akka.actor.UntypedActor;
 import akka.dispatch.Futures;
 import akka.japi.Procedure;
 import akka.pattern.Patterns;
@@ -30,6 +30,7 @@ import org.apache.flink.runtime.concurrent.impl.FlinkFuture;
 import org.apache.flink.runtime.rpc.MainThreadValidatorUtil;
 import org.apache.flink.runtime.rpc.RpcEndpoint;
 import org.apache.flink.runtime.rpc.RpcGateway;
+import org.apache.flink.runtime.rpc.akka.exceptions.AkkaRpcException;
 import org.apache.flink.runtime.rpc.akka.messages.CallAsync;
 import org.apache.flink.runtime.rpc.akka.messages.LocalRpcInvocation;
 import org.apache.flink.runtime.rpc.akka.messages.Processing;
@@ -60,14 +61,14 @@ import static org.apache.flink.util.Preconditions.checkNotNull;
  * in the context of the actor thread.
  * <p>
  * The {@link Processing} message controls the processing behaviour of the akka rpc actor. A
- * {@link Processing#START} message unstashes all stashed messages and starts processing incoming
- * messages. A {@link Processing#STOP} message stops processing messages and stashes incoming
- * messages.
+ * {@link Processing#START} starts processing incoming messages. A {@link Processing#STOP} message
+ * stops processing messages. All messages which arrive when the processing is stopped, will be
+ * discarded.
  *
  * @param <C> Type of the {@link RpcGateway} associated with the {@link RpcEndpoint}
  * @param <T> Type of the {@link RpcEndpoint}
  */
-class AkkaRpcActor<C extends RpcGateway, T extends RpcEndpoint<C>> extends UntypedActorWithStash {
+class AkkaRpcActor<C extends RpcGateway, T extends RpcEndpoint<C>> extends UntypedActor {
 	
 	private static final Logger LOG = LoggerFactory.getLogger(AkkaRpcActor.class);
 
@@ -86,7 +87,7 @@ class AkkaRpcActor<C extends RpcGateway, T extends RpcEndpoint<C>> extends Untyp
 	}
 
 	@Override
-	public void postStop() {
+	public void postStop() throws Exception {
 		super.postStop();
 
 		// IMPORTANT: This only works if we don't use a restarting supervisor strategy. Otherwise
@@ -99,7 +100,6 @@ class AkkaRpcActor<C extends RpcGateway, T extends RpcEndpoint<C>> extends Untyp
 	@Override
 	public void onReceive(final Object message) {
 		if (message.equals(Processing.START)) {
-			unstashAll();
 			getContext().become(new Procedure<Object>() {
 				@Override
 				public void apply(Object msg) throws Exception {
@@ -111,10 +111,15 @@ class AkkaRpcActor<C extends RpcGateway, T extends RpcEndpoint<C>> extends Untyp
 				}
 			});
 		} else {
-			LOG.info("The rpc endpoint {} has not been started yet. Stashing message {} until processing is started.",
+			LOG.info("The rpc endpoint {} has not been started yet. Discarding message {} until processing is started.",
 				rpcEndpoint.getClass().getName(),
 				message.getClass().getName());
-			stash();
+
+			if (!getSender().equals(ActorRef.noSender())) {
+				// fail a possible future if we have a sender
+				getSender().tell(new Status.Failure(new AkkaRpcException("Discard message, because " +
+					"the rpc endpoint has not been started yet.")), getSelf());
+			}
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/006a19d4/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/exceptions/AkkaRpcException.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/exceptions/AkkaRpcException.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/exceptions/AkkaRpcException.java
new file mode 100644
index 0000000..f0d6548
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/exceptions/AkkaRpcException.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.rpc.akka.exceptions;
+
+import org.apache.flink.runtime.rpc.exceptions.RpcException;
+
+/**
+ * Base class for Akka RPC related exceptions.
+ */
+public class AkkaRpcException extends RpcException {
+
+	private static final long serialVersionUID = -3796329968494146418L;
+
+	public AkkaRpcException(String message) {
+		super(message);
+	}
+
+	public AkkaRpcException(String message, Throwable cause) {
+		super(message, cause);
+	}
+
+	public AkkaRpcException(Throwable cause) {
+		super(cause);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/006a19d4/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/exceptions/RpcConnectionException.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/exceptions/RpcConnectionException.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/exceptions/RpcConnectionException.java
index a22ebe7..4eaf34f 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/exceptions/RpcConnectionException.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/exceptions/RpcConnectionException.java
@@ -18,13 +18,11 @@
 
 package org.apache.flink.runtime.rpc.exceptions;
 
-import java.util.concurrent.ExecutionException;
-
 /**
  * Exception class which is thrown if a rpc connection failed. Usually this happens if the remote
  * host cannot be reached.
  */
-public class RpcConnectionException extends ExecutionException {
+public class RpcConnectionException extends RpcException {
 	private static final long serialVersionUID = -5500560405481142472L;
 
 	public RpcConnectionException(String message) {

http://git-wip-us.apache.org/repos/asf/flink/blob/006a19d4/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/exceptions/RpcException.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/exceptions/RpcException.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/exceptions/RpcException.java
new file mode 100644
index 0000000..652b3f5
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/exceptions/RpcException.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.rpc.exceptions;
+
+/**
+ * Base class for RPC related exceptions.
+ */
+public class RpcException extends Exception {
+
+	private static final long serialVersionUID = -7163591879289483630L;
+
+	public RpcException(String message) {
+		super(message);
+	}
+
+	public RpcException(String message, Throwable cause) {
+		super(message, cause);
+	}
+
+	public RpcException(Throwable cause) {
+		super(cause);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/006a19d4/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorTest.java
index d2dbab7..760e1a7 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorTest.java
@@ -27,6 +27,7 @@ import org.apache.flink.runtime.rpc.RpcEndpoint;
 import org.apache.flink.runtime.rpc.RpcGateway;
 import org.apache.flink.runtime.rpc.RpcMethod;
 import org.apache.flink.runtime.rpc.RpcService;
+import org.apache.flink.runtime.rpc.akka.exceptions.AkkaRpcException;
 import org.apache.flink.runtime.rpc.exceptions.RpcConnectionException;
 import org.apache.flink.util.TestLogger;
 import org.hamcrest.core.Is;
@@ -95,26 +96,38 @@ public class AkkaRpcActorTest extends TestLogger {
 	}
 
 	/**
-	 * Tests that the {@link AkkaRpcActor} stashes messages until the corresponding
+	 * Tests that the {@link AkkaRpcActor} discards messages until the corresponding
 	 * {@link RpcEndpoint} has been started.
 	 */
 	@Test
-	public void testMessageStashing() throws Exception {
+	public void testMessageDiscarding() throws Exception {
 		int expectedValue = 1337;
 
 		DummyRpcEndpoint rpcEndpoint = new DummyRpcEndpoint(akkaRpcService);
 
 		DummyRpcGateway rpcGateway = rpcEndpoint.getSelf();
 
-		// this message should not be processed until we've started the rpc endpoint
+		// this message should be discarded and completed with an AkkaRpcException
 		Future<Integer> result = rpcGateway.foobar();
 
+		try {
+			result.get(timeout.getSize(), timeout.getUnit());
+			fail("Expected an AkkaRpcException.");
+		} catch (ExecutionException ee) {
+			// expected this exception, because the endpoint has not been started
+			assertTrue(ee.getCause() instanceof AkkaRpcException);
+		}
+
 		// set a new value which we expect to be returned
 		rpcEndpoint.setFoobar(expectedValue);
 
-		// now process the rpc
+		// start the endpoint so that it can process messages
 		rpcEndpoint.start();
 
+		// send the rpc again
+		result = rpcGateway.foobar();
+
+		// now we should receive a result :-)
 		Integer actualValue = result.get(timeout.getSize(), timeout.getUnit());
 
 		assertThat("The new foobar value should have been returned.", actualValue, Is.is(expectedValue));


[08/52] [abbrv] flink git commit: [hotfix] [tests] Migrate some test tasks to Java

Posted by se...@apache.org.
[hotfix] [tests] Migrate some test tasks to Java


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

Branch: refs/heads/master
Commit: 208324d67c9d3cfa66a63fc637dfa280dbd3062c
Parents: 67ed642
Author: Stephan Ewen <se...@apache.org>
Authored: Fri Oct 14 23:54:29 2016 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Dec 23 20:54:23 2016 +0100

----------------------------------------------------------------------
 .../StackTraceSampleCoordinatorITCase.java      |  6 ++-
 .../checkpoint/CoordinatorShutdownTest.java     |  3 ++
 .../ExecutionGraphMetricsTest.java              |  4 +-
 .../ExecutionGraphRestartTest.java              | 23 ++++++------
 .../runtime/jobmanager/JobManagerTest.java      | 18 +++++----
 .../flink/runtime/jobmanager/JobSubmitTest.java |  7 ++--
 .../runtime/taskmanager/TaskManagerTest.java    |  5 ++-
 .../testtasks/BlockingNoOpInvokable.java        | 39 ++++++++++++++++++++
 .../flink/runtime/testtasks/NoOpInvokable.java  | 30 +++++++++++++++
 .../runtime/testtasks/WaitingNoOpInvokable.java | 34 +++++++++++++++++
 .../TaskManagerLossFailsTasksTest.scala         |  3 +-
 .../runtime/jobmanager/JobManagerITCase.scala   |  1 +
 .../apache/flink/runtime/jobmanager/Tasks.scala | 20 ----------
 .../JobSubmissionFailsITCase.java               |  6 +--
 .../JobManagerHACheckpointRecoveryITCase.java   |  4 +-
 .../JobManagerHAJobGraphRecoveryITCase.java     |  4 +-
 .../jobmanager/JobManagerFailsITCase.scala      |  2 +-
 .../taskmanager/TaskManagerFailsITCase.scala    |  3 +-
 18 files changed, 154 insertions(+), 58 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/208324d6/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/StackTraceSampleCoordinatorITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/StackTraceSampleCoordinatorITCase.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/StackTraceSampleCoordinatorITCase.java
index f31f41f..c9fa547 100644
--- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/StackTraceSampleCoordinatorITCase.java
+++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/StackTraceSampleCoordinatorITCase.java
@@ -32,14 +32,16 @@ import org.apache.flink.runtime.instance.ActorGateway;
 import org.apache.flink.runtime.instance.AkkaActorGateway;
 import org.apache.flink.runtime.jobgraph.JobGraph;
 import org.apache.flink.runtime.jobgraph.JobVertex;
-import org.apache.flink.runtime.jobmanager.Tasks;
 import org.apache.flink.runtime.messages.JobManagerMessages;
 import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages;
 import org.apache.flink.runtime.testingUtils.TestingUtils;
+import org.apache.flink.runtime.testtasks.BlockingNoOpInvokable;
 import org.apache.flink.util.TestLogger;
+
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
+
 import scala.concurrent.Await;
 import scala.concurrent.duration.FiniteDuration;
 
@@ -81,7 +83,7 @@ public class StackTraceSampleCoordinatorITCase extends TestLogger {
 			final int parallelism = 1;
 
 			final JobVertex task = new JobVertex("Task");
-			task.setInvokableClass(Tasks.BlockingNoOpInvokable.class);
+			task.setInvokableClass(BlockingNoOpInvokable.class);
 			task.setParallelism(parallelism);
 
 			jobGraph.addVertex(task);

http://git-wip-us.apache.org/repos/asf/flink/blob/208324d6/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CoordinatorShutdownTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CoordinatorShutdownTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CoordinatorShutdownTest.java
index 777ba9b..bd82d0c 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CoordinatorShutdownTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CoordinatorShutdownTest.java
@@ -32,7 +32,10 @@ import org.apache.flink.runtime.jobgraph.tasks.JobSnapshottingSettings;
 import org.apache.flink.runtime.messages.JobManagerMessages;
 import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
 import org.apache.flink.runtime.testingUtils.TestingUtils;
+import org.apache.flink.runtime.testtasks.NoOpInvokable;
+
 import org.junit.Test;
+
 import scala.concurrent.Await;
 import scala.concurrent.Future;
 import scala.concurrent.duration.FiniteDuration;

http://git-wip-us.apache.org/repos/asf/flink/blob/208324d6/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphMetricsTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphMetricsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphMetricsTest.java
index d8d8e24..0f7e75f 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphMetricsTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphMetricsTest.java
@@ -45,12 +45,12 @@ import org.apache.flink.runtime.instance.Slot;
 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.jobmanager.Tasks;
 import org.apache.flink.runtime.jobmanager.scheduler.ScheduledUnit;
 import org.apache.flink.runtime.jobmanager.scheduler.Scheduler;
 import org.apache.flink.runtime.metrics.MetricRegistry;
 import org.apache.flink.runtime.metrics.groups.JobManagerMetricGroup;
 import org.apache.flink.runtime.taskmanager.TaskExecutionState;
+import org.apache.flink.runtime.testtasks.NoOpInvokable;
 import org.apache.flink.util.SerializedValue;
 import org.apache.flink.util.TestLogger;
 
@@ -87,7 +87,7 @@ public class ExecutionGraphMetricsTest extends TestLogger {
 	
 			JobVertex jobVertex = new JobVertex("TestVertex");
 			jobVertex.setParallelism(parallelism);
-			jobVertex.setInvokableClass(Tasks.NoOpInvokable.class);
+			jobVertex.setInvokableClass(NoOpInvokable.class);
 			JobGraph jobGraph = new JobGraph("Test Job", jobVertex);
 	
 			Configuration config = new Configuration();

http://git-wip-us.apache.org/repos/asf/flink/blob/208324d6/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphRestartTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphRestartTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphRestartTest.java
index 52bfc96..11f12a5 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphRestartTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphRestartTest.java
@@ -37,16 +37,17 @@ import org.apache.flink.runtime.instance.Instance;
 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.jobmanager.Tasks;
 import org.apache.flink.runtime.jobmanager.scheduler.CoLocationConstraint;
 import org.apache.flink.runtime.jobmanager.scheduler.Scheduler;
 import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup;
 import org.apache.flink.runtime.jobmanager.slots.ActorTaskManagerGateway;
 import org.apache.flink.runtime.testingUtils.TestingUtils;
+import org.apache.flink.runtime.testtasks.NoOpInvokable;
 import org.apache.flink.util.SerializedValue;
 import org.apache.flink.util.TestLogger;
 
 import org.junit.Test;
+
 import scala.concurrent.Await;
 import scala.concurrent.Future;
 import scala.concurrent.duration.Deadline;
@@ -105,8 +106,8 @@ public class ExecutionGraphRestartTest extends TestLogger {
 		Scheduler scheduler = new Scheduler(TestingUtils.defaultExecutionContext());
 		scheduler.newInstanceAvailable(instance);
 
-		JobVertex groupVertex = newJobVertex("Task1", NUM_TASKS, Tasks.NoOpInvokable.class);
-		JobVertex groupVertex2 = newJobVertex("Task2", NUM_TASKS, Tasks.NoOpInvokable.class);
+		JobVertex groupVertex = newJobVertex("Task1", NUM_TASKS, NoOpInvokable.class);
+		JobVertex groupVertex2 = newJobVertex("Task2", NUM_TASKS, NoOpInvokable.class);
 
 		SlotSharingGroup sharingGroup = new SlotSharingGroup();
 		groupVertex.setSlotSharingGroup(sharingGroup);
@@ -242,7 +243,7 @@ public class ExecutionGraphRestartTest extends TestLogger {
 			new InfiniteDelayRestartStrategy());
 
 		JobVertex jobVertex = new JobVertex("NoOpInvokable");
-		jobVertex.setInvokableClass(Tasks.NoOpInvokable.class);
+		jobVertex.setInvokableClass(NoOpInvokable.class);
 		jobVertex.setParallelism(NUM_TASKS);
 
 		JobGraph jobGraph = new JobGraph("TestJob", jobVertex);
@@ -380,8 +381,8 @@ public class ExecutionGraphRestartTest extends TestLogger {
 		Scheduler scheduler = new Scheduler(TestingUtils.defaultExecutionContext());
 		scheduler.newInstanceAvailable(instance);
 
-		JobVertex sender = newJobVertex("Task1", 1, Tasks.NoOpInvokable.class);
-		JobVertex receiver = newJobVertex("Task2", 1, Tasks.NoOpInvokable.class);
+		JobVertex sender = newJobVertex("Task1", 1, NoOpInvokable.class);
+		JobVertex receiver = newJobVertex("Task2", 1, NoOpInvokable.class);
 		JobGraph jobGraph = new JobGraph("Pointwise job", sender, receiver);
 		ExecutionGraph eg = newExecutionGraph(new FixedDelayRestartStrategy(1, 1000));
 		eg.attachJobGraph(jobGraph.getVerticesSortedTopologicallyFromSources());
@@ -444,7 +445,7 @@ public class ExecutionGraphRestartTest extends TestLogger {
 		Scheduler scheduler = new Scheduler(TestingUtils.defaultExecutionContext());
 		scheduler.newInstanceAvailable(instance);
 
-		JobVertex vertex = newJobVertex("Test Vertex", 1, Tasks.NoOpInvokable.class);
+		JobVertex vertex = newJobVertex("Test Vertex", 1, NoOpInvokable.class);
 
 		ExecutionConfig executionConfig = new ExecutionConfig();
 		executionConfig.setRestartStrategy(RestartStrategies.fixedDelayRestart(
@@ -490,7 +491,7 @@ public class ExecutionGraphRestartTest extends TestLogger {
 		Scheduler scheduler = new Scheduler(TestingUtils.defaultExecutionContext());
 		scheduler.newInstanceAvailable(instance);
 
-		JobVertex vertex = newJobVertex("Test Vertex", 1, Tasks.NoOpInvokable.class);
+		JobVertex vertex = newJobVertex("Test Vertex", 1, NoOpInvokable.class);
 
 		ExecutionConfig executionConfig = new ExecutionConfig();
 		executionConfig.setRestartStrategy(RestartStrategies.fixedDelayRestart(
@@ -539,7 +540,7 @@ public class ExecutionGraphRestartTest extends TestLogger {
 		scheduler.newInstanceAvailable(instance);
 
 		JobVertex sender = new JobVertex("Task");
-		sender.setInvokableClass(Tasks.NoOpInvokable.class);
+		sender.setInvokableClass(NoOpInvokable.class);
 		sender.setParallelism(NUM_TASKS);
 
 		JobGraph jobGraph = new JobGraph("Pointwise job", sender);
@@ -655,7 +656,7 @@ public class ExecutionGraphRestartTest extends TestLogger {
 		Scheduler scheduler = new Scheduler(TestingUtils.defaultExecutionContext());
 		scheduler.newInstanceAvailable(instance);
 
-		JobVertex sender = newJobVertex("Task", NUM_TASKS, Tasks.NoOpInvokable.class);
+		JobVertex sender = newJobVertex("Task", NUM_TASKS, NoOpInvokable.class);
 
 		JobGraph jobGraph = new JobGraph("Pointwise job", sender);
 
@@ -672,7 +673,7 @@ public class ExecutionGraphRestartTest extends TestLogger {
 		return new Tuple2<>(eg, instance);
 	}
 
-	private static JobVertex newJobVertex(String task1, int numTasks, Class<Tasks.NoOpInvokable> invokable) {
+	private static JobVertex newJobVertex(String task1, int numTasks, Class<NoOpInvokable> invokable) {
 		JobVertex groupVertex = new JobVertex(task1);
 		groupVertex.setInvokableClass(invokable);
 		groupVertex.setParallelism(numTasks);

http://git-wip-us.apache.org/repos/asf/flink/blob/208324d6/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerTest.java
index 6d8c70b..3c17daf 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerTest.java
@@ -83,6 +83,8 @@ import org.apache.flink.runtime.testingUtils.TestingMemoryArchivist;
 import org.apache.flink.runtime.testingUtils.TestingTaskManager;
 import org.apache.flink.runtime.testingUtils.TestingTaskManagerMessages;
 import org.apache.flink.runtime.testingUtils.TestingUtils;
+import org.apache.flink.runtime.testtasks.BlockingNoOpInvokable;
+import org.apache.flink.runtime.testtasks.NoOpInvokable;
 import org.apache.flink.runtime.testutils.StoppableInvokable;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -171,7 +173,7 @@ public class JobManagerTest {
 						// Create a task
 						final JobVertex sender = new JobVertex("Sender");
 						sender.setParallelism(1);
-						sender.setInvokableClass(Tasks.BlockingNoOpInvokable.class); // just block
+						sender.setInvokableClass(BlockingNoOpInvokable.class); // just block
 						sender.createAndAddResultDataSet(rid, PIPELINED);
 
 						final JobGraph jobGraph = new JobGraph("Blocking test job", sender);
@@ -287,12 +289,12 @@ public class JobManagerTest {
 						// Create a task
 						final JobVertex sender = new JobVertex("Sender");
 						sender.setParallelism(1);
-						sender.setInvokableClass(Tasks.NoOpInvokable.class); // just finish
+						sender.setInvokableClass(NoOpInvokable.class); // just finish
 						sender.createAndAddResultDataSet(rid, PIPELINED);
 
 						final JobVertex sender2 = new JobVertex("Blocking Sender");
 						sender2.setParallelism(1);
-						sender2.setInvokableClass(Tasks.BlockingNoOpInvokable.class); // just block
+						sender2.setInvokableClass(BlockingNoOpInvokable.class); // just block
 						sender2.createAndAddResultDataSet(new IntermediateDataSetID(), PIPELINED);
 
 						final JobGraph jobGraph = new JobGraph("Fast finishing producer test job", sender, sender2);
@@ -378,12 +380,12 @@ public class JobManagerTest {
 						// Create a task
 						final JobVertex sender = new JobVertex("Sender");
 						sender.setParallelism(1);
-						sender.setInvokableClass(Tasks.NoOpInvokable.class); // just finish
+						sender.setInvokableClass(NoOpInvokable.class); // just finish
 						sender.createAndAddResultDataSet(rid, PIPELINED);
 
 						final JobVertex sender2 = new JobVertex("Blocking Sender");
 						sender2.setParallelism(1);
-						sender2.setInvokableClass(Tasks.BlockingNoOpInvokable.class); // just block
+						sender2.setInvokableClass(BlockingNoOpInvokable.class); // just block
 						sender2.createAndAddResultDataSet(new IntermediateDataSetID(), PIPELINED);
 
 						final JobGraph jobGraph = new JobGraph("Fast finishing producer test job", sender, sender2);
@@ -520,7 +522,7 @@ public class JobManagerTest {
 						// Create a task
 						final JobVertex sender = new JobVertex("Sender");
 						sender.setParallelism(1);
-						sender.setInvokableClass(Tasks.BlockingNoOpInvokable.class); // just block
+						sender.setInvokableClass(BlockingNoOpInvokable.class); // just block
 
 						final JobGraph jobGraph = new JobGraph("Non-Stoppable batching test job", sender);
 						final JobID jid = jobGraph.getJobID();
@@ -624,11 +626,11 @@ public class JobManagerTest {
 		JobGraph jobGraph = new JobGraph("croissant");
 		JobVertex jobVertex1 = new JobVertex("cappuccino");
 		jobVertex1.setParallelism(4);
-		jobVertex1.setInvokableClass(Tasks.BlockingNoOpInvokable.class);
+		jobVertex1.setInvokableClass(BlockingNoOpInvokable.class);
 
 		JobVertex jobVertex2 = new JobVertex("americano");
 		jobVertex2.setParallelism(4);
-		jobVertex2.setInvokableClass(Tasks.BlockingNoOpInvokable.class);
+		jobVertex2.setInvokableClass(BlockingNoOpInvokable.class);
 
 		jobGraph.addVertex(jobVertex1);
 		jobGraph.addVertex(jobVertex2);

http://git-wip-us.apache.org/repos/asf/flink/blob/208324d6/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobSubmitTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobSubmitTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobSubmitTest.java
index 1b8f0c3..53bd318 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobSubmitTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobSubmitTest.java
@@ -40,6 +40,7 @@ import org.apache.flink.runtime.jobgraph.tasks.ExternalizedCheckpointSettings;
 import org.apache.flink.runtime.jobgraph.tasks.JobSnapshottingSettings;
 import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
 import org.apache.flink.runtime.messages.JobManagerMessages;
+import org.apache.flink.runtime.testtasks.NoOpInvokable;
 import org.apache.flink.runtime.util.LeaderRetrievalUtils;
 import org.apache.flink.util.NetUtils;
 import org.junit.AfterClass;
@@ -112,7 +113,7 @@ public class JobSubmitTest {
 		try {
 			// create a simple job graph
 			JobVertex jobVertex = new JobVertex("Test Vertex");
-			jobVertex.setInvokableClass(Tasks.NoOpInvokable.class);
+			jobVertex.setInvokableClass(NoOpInvokable.class);
 			JobGraph jg = new JobGraph("test job", jobVertex);
 
 			// request the blob port from the job manager
@@ -176,7 +177,7 @@ public class JobSubmitTest {
 				}
 			};
 
-			jobVertex.setInvokableClass(Tasks.NoOpInvokable.class);
+			jobVertex.setInvokableClass(NoOpInvokable.class);
 			JobGraph jg = new JobGraph("test job", jobVertex);
 
 			// submit the job
@@ -219,7 +220,7 @@ public class JobSubmitTest {
 	private JobGraph createSimpleJobGraph() {
 		JobVertex jobVertex = new JobVertex("Vertex");
 
-		jobVertex.setInvokableClass(Tasks.NoOpInvokable.class);
+		jobVertex.setInvokableClass(NoOpInvokable.class);
 		List<JobVertexID> vertexIdList = Collections.singletonList(jobVertex.getID());
 
 		JobGraph jg = new JobGraph("test job", jobVertex);

http://git-wip-us.apache.org/repos/asf/flink/blob/208324d6/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java
index b06e474..99c1c1f 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java
@@ -72,6 +72,7 @@ import org.apache.flink.runtime.messages.TaskMessages.SubmitTask;
 import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages;
 import org.apache.flink.runtime.testingUtils.TestingTaskManagerMessages;
 import org.apache.flink.runtime.testingUtils.TestingUtils;
+import org.apache.flink.runtime.testtasks.BlockingNoOpInvokable;
 import org.apache.flink.runtime.testutils.StoppableInvokable;
 import org.apache.flink.types.IntValue;
 import org.apache.flink.util.NetUtils;
@@ -1121,7 +1122,7 @@ public class TaskManagerTest extends TestLogger {
 						0,
 						new Configuration(),
 						new Configuration(),
-						Tasks.BlockingNoOpInvokable.class.getName(),
+						BlockingNoOpInvokable.class.getName(),
 						Collections.<ResultPartitionDeploymentDescriptor>emptyList(),
 						Collections.<InputGateDeploymentDescriptor>emptyList(),
 						Collections.<BlobKey>emptyList(),
@@ -1227,7 +1228,7 @@ public class TaskManagerTest extends TestLogger {
 									// Look for BlockingNoOpInvokable#invoke
 									for (StackTraceElement elem : trace) {
 										if (elem.getClassName().equals(
-												Tasks.BlockingNoOpInvokable.class.getName())) {
+												BlockingNoOpInvokable.class.getName())) {
 
 											assertEquals("invoke", elem.getMethodName());
 											success = true;

http://git-wip-us.apache.org/repos/asf/flink/blob/208324d6/flink-runtime/src/test/java/org/apache/flink/runtime/testtasks/BlockingNoOpInvokable.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/testtasks/BlockingNoOpInvokable.java b/flink-runtime/src/test/java/org/apache/flink/runtime/testtasks/BlockingNoOpInvokable.java
new file mode 100644
index 0000000..c9adba8
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/testtasks/BlockingNoOpInvokable.java
@@ -0,0 +1,39 @@
+/*
+ * 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.testtasks;
+
+import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
+
+/**
+ * A task that does nothing but blocks indefinitely, until the executing thread is interrupted.
+ */
+public class BlockingNoOpInvokable extends AbstractInvokable {
+
+	@Override
+	public void invoke() throws Exception {
+		final Object o = new Object();
+		//noinspection SynchronizationOnLocalVariableOrMethodParameter
+		synchronized (o) {
+			//noinspection InfiniteLoopStatement
+			while (true) {
+				o.wait();
+			}
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/208324d6/flink-runtime/src/test/java/org/apache/flink/runtime/testtasks/NoOpInvokable.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/testtasks/NoOpInvokable.java b/flink-runtime/src/test/java/org/apache/flink/runtime/testtasks/NoOpInvokable.java
new file mode 100644
index 0000000..fa9949a
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/testtasks/NoOpInvokable.java
@@ -0,0 +1,30 @@
+/*
+ * 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.testtasks;
+
+import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
+
+/**
+ * A simple task that does nothing and finishes immediately.
+ */
+public class NoOpInvokable extends AbstractInvokable {
+
+	@Override
+	public void invoke() {}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/208324d6/flink-runtime/src/test/java/org/apache/flink/runtime/testtasks/WaitingNoOpInvokable.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/testtasks/WaitingNoOpInvokable.java b/flink-runtime/src/test/java/org/apache/flink/runtime/testtasks/WaitingNoOpInvokable.java
new file mode 100644
index 0000000..de7d59a
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/testtasks/WaitingNoOpInvokable.java
@@ -0,0 +1,34 @@
+/*
+ * 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.testtasks;
+
+import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
+
+/**
+ * A simple task that does nothing and finishes after a short delay of 100 milliseconds.
+ */
+public class WaitingNoOpInvokable extends AbstractInvokable {
+
+	private static final long waitingTime = 100L;
+
+	@Override
+	public void invoke() throws Exception {
+		Thread.sleep(waitingTime);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/208324d6/flink-runtime/src/test/scala/org/apache/flink/runtime/executiongraph/TaskManagerLossFailsTasksTest.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/executiongraph/TaskManagerLossFailsTasksTest.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/executiongraph/TaskManagerLossFailsTasksTest.scala
index 1cbd605..6f833f1 100644
--- a/flink-runtime/src/test/scala/org/apache/flink/runtime/executiongraph/TaskManagerLossFailsTasksTest.scala
+++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/executiongraph/TaskManagerLossFailsTasksTest.scala
@@ -28,6 +28,7 @@ import org.apache.flink.runtime.jobmanager.Tasks
 import org.apache.flink.runtime.jobmanager.scheduler.Scheduler
 import org.apache.flink.runtime.jobmanager.slots.ActorTaskManagerGateway
 import org.apache.flink.runtime.testingUtils.TestingUtils
+import org.apache.flink.runtime.testtasks.NoOpInvokable
 import org.apache.flink.util.SerializedValue
 import org.junit.runner.RunWith
 import org.scalatest.junit.JUnitRunner
@@ -51,7 +52,7 @@ class TaskManagerLossFailsTasksTest extends WordSpecLike with Matchers {
         scheduler.newInstanceAvailable(instance2)
 
         val sender = new JobVertex("Task")
-        sender.setInvokableClass(classOf[Tasks.NoOpInvokable])
+        sender.setInvokableClass(classOf[NoOpInvokable])
         sender.setParallelism(20)
 
         val jobGraph = new JobGraph("Pointwise job", sender)

http://git-wip-us.apache.org/repos/asf/flink/blob/208324d6/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/JobManagerITCase.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/JobManagerITCase.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/JobManagerITCase.scala
index 0569297..4aa0565 100644
--- a/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/JobManagerITCase.scala
+++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/JobManagerITCase.scala
@@ -33,6 +33,7 @@ import org.apache.flink.runtime.jobmanager.scheduler.{NoResourceAvailableExcepti
 import org.apache.flink.runtime.messages.JobManagerMessages._
 import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages._
 import org.apache.flink.runtime.testingUtils.{ScalaTestingUtils, TestingUtils}
+import org.apache.flink.runtime.testtasks._
 import org.junit.runner.RunWith
 import org.mockito.Mockito
 import org.mockito.Mockito._

http://git-wip-us.apache.org/repos/asf/flink/blob/208324d6/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/Tasks.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/Tasks.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/Tasks.scala
index 87c123a..fabd66b 100644
--- a/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/Tasks.scala
+++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/Tasks.scala
@@ -25,26 +25,6 @@ import org.apache.flink.types.IntValue
 
 
 object Tasks {
-  class BlockingNoOpInvokable extends AbstractInvokable {
-    override def invoke(): Unit = {
-      val o = new Object()
-      o.synchronized{
-        o.wait()
-      }
-    }
-  }
-
-  class NoOpInvokable extends AbstractInvokable{
-    override def invoke(): Unit = {}
-  }
-
-  class WaitingNoOpInvokable extends AbstractInvokable{
-    val waitingTime = 100L
-
-    override def invoke(): Unit = {
-      Thread.sleep(waitingTime)
-    }
-  }
 
   class Sender extends AbstractInvokable{
 

http://git-wip-us.apache.org/repos/asf/flink/blob/208324d6/flink-tests/src/test/java/org/apache/flink/test/failingPrograms/JobSubmissionFailsITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/failingPrograms/JobSubmissionFailsITCase.java b/flink-tests/src/test/java/org/apache/flink/test/failingPrograms/JobSubmissionFailsITCase.java
index 178656d..256b1ae 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/failingPrograms/JobSubmissionFailsITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/failingPrograms/JobSubmissionFailsITCase.java
@@ -25,9 +25,9 @@ import org.apache.flink.runtime.client.JobExecutionException;
 import org.apache.flink.runtime.client.JobSubmissionException;
 import org.apache.flink.runtime.jobgraph.JobVertex;
 import org.apache.flink.runtime.jobgraph.JobGraph;
-import org.apache.flink.runtime.jobmanager.Tasks;
 import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
 import org.apache.flink.runtime.testingUtils.TestingUtils;
+import org.apache.flink.runtime.testtasks.NoOpInvokable;
 
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -63,7 +63,7 @@ public class JobSubmissionFailsITCase {
 			cluster.start();
 			
 			final JobVertex jobVertex = new JobVertex("Working job vertex.");
-			jobVertex.setInvokableClass(Tasks.NoOpInvokable.class);
+			jobVertex.setInvokableClass(NoOpInvokable.class);
 			workingJobGraph = new JobGraph("Working testing job", jobVertex);
 		}
 		catch (Exception e) {
@@ -113,7 +113,7 @@ public class JobSubmissionFailsITCase {
 	public void testExceptionInInitializeOnMaster() {
 		try {
 			final JobVertex failingJobVertex = new FailingJobVertex("Failing job vertex");
-			failingJobVertex.setInvokableClass(Tasks.NoOpInvokable.class);
+			failingJobVertex.setInvokableClass(NoOpInvokable.class);
 
 			final JobGraph failingJobGraph = new JobGraph("Failing testing job", failingJobVertex);
 

http://git-wip-us.apache.org/repos/asf/flink/blob/208324d6/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHACheckpointRecoveryITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHACheckpointRecoveryITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHACheckpointRecoveryITCase.java
index 3f08b5a..418aa51 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHACheckpointRecoveryITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHACheckpointRecoveryITCase.java
@@ -31,10 +31,10 @@ import org.apache.flink.runtime.instance.AkkaActorGateway;
 import org.apache.flink.runtime.jobgraph.JobGraph;
 import org.apache.flink.runtime.jobgraph.JobStatus;
 import org.apache.flink.runtime.jobgraph.JobVertex;
-import org.apache.flink.runtime.jobmanager.Tasks;
 import org.apache.flink.runtime.leaderelection.TestingListener;
 import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
 import org.apache.flink.runtime.taskmanager.TaskManager;
+import org.apache.flink.runtime.testtasks.BlockingNoOpInvokable;
 import org.apache.flink.runtime.testutils.CommonTestUtils;
 import org.apache.flink.runtime.testutils.JobManagerActorTestUtils;
 import org.apache.flink.runtime.testutils.JobManagerProcess;
@@ -372,7 +372,7 @@ public class JobManagerHACheckpointRecoveryITCase extends TestLogger {
 
 			// Blocking JobGraph
 			JobVertex blockingVertex = new JobVertex("Blocking vertex");
-			blockingVertex.setInvokableClass(Tasks.BlockingNoOpInvokable.class);
+			blockingVertex.setInvokableClass(BlockingNoOpInvokable.class);
 			JobGraph jobGraph = new JobGraph(blockingVertex);
 
 			// Submit the job in detached mode

http://git-wip-us.apache.org/repos/asf/flink/blob/208324d6/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHAJobGraphRecoveryITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHAJobGraphRecoveryITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHAJobGraphRecoveryITCase.java
index bf39c4b..236e922 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHAJobGraphRecoveryITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHAJobGraphRecoveryITCase.java
@@ -38,7 +38,6 @@ 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.jobmanager.SubmittedJobGraph;
-import org.apache.flink.runtime.jobmanager.Tasks;
 import org.apache.flink.runtime.leaderelection.TestingListener;
 import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
 import org.apache.flink.runtime.messages.JobManagerMessages;
@@ -48,6 +47,7 @@ import org.apache.flink.runtime.messages.JobManagerMessages.SubmitJob;
 import org.apache.flink.runtime.taskmanager.TaskManager;
 import org.apache.flink.runtime.testingUtils.TestingCluster;
 import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages;
+import org.apache.flink.runtime.testtasks.BlockingNoOpInvokable;
 import org.apache.flink.runtime.testutils.CommonTestUtils;
 import org.apache.flink.runtime.testutils.JobManagerActorTestUtils;
 import org.apache.flink.runtime.testutils.JobManagerProcess;
@@ -461,7 +461,7 @@ public class JobManagerHAJobGraphRecoveryITCase extends TestLogger {
 		JobGraph jobGraph = new JobGraph("Blocking program");
 
 		JobVertex jobVertex = new JobVertex("Blocking Vertex");
-		jobVertex.setInvokableClass(Tasks.BlockingNoOpInvokable.class);
+		jobVertex.setInvokableClass(BlockingNoOpInvokable.class);
 
 		jobGraph.addVertex(jobVertex);
 

http://git-wip-us.apache.org/repos/asf/flink/blob/208324d6/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/jobmanager/JobManagerFailsITCase.scala
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/jobmanager/JobManagerFailsITCase.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/jobmanager/JobManagerFailsITCase.scala
index a8bc7a4..5db02d1 100644
--- a/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/jobmanager/JobManagerFailsITCase.scala
+++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/jobmanager/JobManagerFailsITCase.scala
@@ -23,7 +23,7 @@ import akka.testkit.{ImplicitSender, TestKit}
 import org.apache.flink.configuration.{ConfigConstants, Configuration}
 import org.apache.flink.runtime.akka.{AkkaUtils, ListeningBehaviour}
 import org.apache.flink.runtime.jobgraph.{JobGraph, JobVertex}
-import org.apache.flink.runtime.jobmanager.Tasks.{BlockingNoOpInvokable, NoOpInvokable}
+import org.apache.flink.runtime.testtasks.{BlockingNoOpInvokable, NoOpInvokable}
 import org.apache.flink.runtime.messages.Acknowledge
 import org.apache.flink.runtime.messages.JobManagerMessages._
 import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages.NotifyWhenAtLeastNumTaskManagerAreRegistered

http://git-wip-us.apache.org/repos/asf/flink/blob/208324d6/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/taskmanager/TaskManagerFailsITCase.scala
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/taskmanager/TaskManagerFailsITCase.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/taskmanager/TaskManagerFailsITCase.scala
index 3b39b3f..e141cc2 100644
--- a/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/taskmanager/TaskManagerFailsITCase.scala
+++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/taskmanager/TaskManagerFailsITCase.scala
@@ -25,7 +25,8 @@ import org.apache.flink.configuration.Configuration
 import org.apache.flink.runtime.akka.{AkkaUtils, ListeningBehaviour}
 import org.apache.flink.runtime.client.JobExecutionException
 import org.apache.flink.runtime.jobgraph.{DistributionPattern, JobGraph, JobVertex}
-import org.apache.flink.runtime.jobmanager.Tasks.{BlockingNoOpInvokable, BlockingReceiver, NoOpInvokable, Sender}
+import org.apache.flink.runtime.jobmanager.Tasks.{BlockingReceiver, Sender}
+import org.apache.flink.runtime.testtasks.{BlockingNoOpInvokable, NoOpInvokable}
 import org.apache.flink.runtime.messages.JobManagerMessages._
 import org.apache.flink.runtime.messages.TaskManagerMessages.{NotifyWhenRegisteredAtJobManager, RegisteredAtJobManager}
 import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages._


[14/52] [abbrv] flink git commit: [FLINK-4871] [mini cluster] Add memory calculation for TaskManagers to MiniCluster

Posted by se...@apache.org.
[FLINK-4871] [mini cluster] Add memory calculation for TaskManagers to MiniCluster

If the managed memory size for the task manager has not been set in the Configuration, then
it is automatically calculated by dividing the available memory by the number of distributed
components. Additionally this PR allows to provide a MetricRegistry to the TaskManagerRunner.
That way it is possible to use the MiniCluster's MetricRegistry.

Add memory calculation for task managers

This closes #2669.


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

Branch: refs/heads/master
Commit: d8b22e0448216ae54d0f0cf0ece3f2afe71f2593
Parents: c001bec
Author: Till Rohrmann <tr...@apache.org>
Authored: Thu Oct 20 11:07:08 2016 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Dec 23 20:54:24 2016 +0100

----------------------------------------------------------------------
 .../flink/runtime/minicluster/MiniCluster.java  |   7 +-
 .../minicluster/MiniClusterConfiguration.java   |  86 ++++++++++++++-
 .../resourcemanager/JobLeaderIdService.java     |   2 +-
 .../resourcemanager/ResourceManager.java        |   3 +-
 .../resourcemanager/ResourceManagerRunner.java  | 102 ++++++++++++++++++
 .../exceptions/ConfigurationException.java      |   2 +-
 .../exceptions/ResourceManagerRunner.java       | 105 -------------------
 .../runtime/taskexecutor/TaskExecutor.java      |  41 +++-----
 .../runtime/taskexecutor/TaskManagerRunner.java |  28 ++++-
 9 files changed, 229 insertions(+), 147 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/d8b22e04/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java
index b005330..611d4c4 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java
@@ -32,7 +32,7 @@ import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils;
 import org.apache.flink.runtime.jobgraph.JobGraph;
 import org.apache.flink.runtime.metrics.MetricRegistry;
 import org.apache.flink.runtime.metrics.MetricRegistryConfiguration;
-import org.apache.flink.runtime.resourcemanager.exceptions.ResourceManagerRunner;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerRunner;
 import org.apache.flink.runtime.rpc.RpcService;
 import org.apache.flink.runtime.rpc.akka.AkkaRpcService;
 import org.apache.flink.runtime.taskexecutor.TaskManagerRunner;
@@ -161,7 +161,7 @@ public class MiniCluster {
 			LOG.info("Starting Flink Mini Cluster");
 			LOG.debug("Using configuration {}", config);
 
-			final Configuration configuration = new UnmodifiableConfiguration(config.getConfiguration());
+			final Configuration configuration = new UnmodifiableConfiguration(config.generateConfiguration());
 			final Time rpcTimeout = config.getRpcTimeout();
 			final int numJobManagers = config.getNumJobManagers();
 			final int numTaskManagers = config.getNumTaskManagers();
@@ -468,7 +468,8 @@ public class MiniCluster {
 				configuration,
 				new ResourceID(UUID.randomUUID().toString()),
 				taskManagerRpcServices[i],
-				haServices);
+				haServices,
+				metricRegistry);
 
 			taskManagerRunners[i].start();
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/d8b22e04/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniClusterConfiguration.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniClusterConfiguration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniClusterConfiguration.java
index cfbbffb..3a03ca3 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniClusterConfiguration.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniClusterConfiguration.java
@@ -20,8 +20,11 @@ package org.apache.flink.runtime.minicluster;
 
 import org.apache.flink.api.common.time.Time;
 import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ConfigOptions;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.akka.AkkaUtils;
+import org.apache.flink.runtime.util.EnvironmentInformation;
 import scala.concurrent.duration.FiniteDuration;
 
 import static org.apache.flink.util.Preconditions.checkArgument;
@@ -41,6 +44,8 @@ public class MiniClusterConfiguration {
 
 	private String commonBindAddress;
 
+	private long managedMemoryPerTaskManager = -1;
+
 	// ------------------------------------------------------------------------
 	//  Construction
 	// ------------------------------------------------------------------------
@@ -96,14 +101,15 @@ public class MiniClusterConfiguration {
 		this.commonBindAddress = bindAddress;
 	}
 
+	public void setManagedMemoryPerTaskManager(long managedMemoryPerTaskManager) {
+		checkArgument(managedMemoryPerTaskManager > 0, "must have more than 0 MB of memory for the TaskManager.");
+		this.managedMemoryPerTaskManager = managedMemoryPerTaskManager;
+	}
+
 	// ------------------------------------------------------------------------
 	//  getters
 	// ------------------------------------------------------------------------
 
-	public Configuration getConfiguration() {
-		return config;
-	}
-
 	public boolean getUseSingleRpcSystem() {
 		return singleRpcService;
 	}
@@ -147,10 +153,23 @@ public class MiniClusterConfiguration {
 		return Time.of(duration.length(), duration.unit());
 	}
 
+	public long getManagedMemoryPerTaskManager() {
+		return getOrCalculateManagedMemoryPerTaskManager();
+	}
+
 	// ------------------------------------------------------------------------
 	//  utils
 	// ------------------------------------------------------------------------
 
+	public Configuration generateConfiguration() {
+		Configuration newConfiguration = new Configuration(config);
+		// set the memory
+		long memory = getOrCalculateManagedMemoryPerTaskManager();
+		newConfiguration.setLong(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, memory);
+
+		return newConfiguration;
+	}
+
 	@Override
 	public String toString() {
 		return "MiniClusterConfiguration {" +
@@ -162,4 +181,63 @@ public class MiniClusterConfiguration {
 				", config=" + config +
 				'}';
 	}
+
+	/**
+	 * Get or calculate the managed memory per task manager. The memory is calculated in the
+	 * following order:
+	 *
+	 * 1. Return {@link #managedMemoryPerTaskManager} if set
+	 * 2. Return config.getInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY) if set
+	 * 3. Distribute the available free memory equally among all components (JMs, RMs and TMs) and
+	 * calculate the managed memory from the share of memory for a single task manager.
+	 *
+	 * @return
+	 */
+	private long getOrCalculateManagedMemoryPerTaskManager() {
+		if (managedMemoryPerTaskManager == -1) {
+			// no memory set in the mini cluster configuration
+			final ConfigOption<Integer> memorySizeOption = ConfigOptions
+				.key(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY)
+				.defaultValue(-1);
+
+			int memorySize = config.getInteger(memorySizeOption);
+
+			if (memorySize == -1) {
+				// no memory set in the flink configuration
+				// share the available memory among all running components
+				final ConfigOption<Integer> bufferSizeOption = ConfigOptions
+					.key(ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY)
+					.defaultValue(ConfigConstants.DEFAULT_TASK_MANAGER_MEMORY_SEGMENT_SIZE);
+
+				final ConfigOption<Long> bufferMemoryOption = ConfigOptions
+					.key(ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY)
+					.defaultValue((long) ConfigConstants.DEFAULT_TASK_MANAGER_NETWORK_NUM_BUFFERS);
+
+				final ConfigOption<Float> memoryFractionOption = ConfigOptions
+					.key(ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY)
+					.defaultValue(ConfigConstants.DEFAULT_MEMORY_MANAGER_MEMORY_FRACTION);
+
+				float memoryFraction = config.getFloat(memoryFractionOption);
+				long networkBuffersMemory = config.getLong(bufferMemoryOption) * config.getInteger(bufferSizeOption);
+
+				long freeMemory = EnvironmentInformation.getSizeOfFreeHeapMemoryWithDefrag();
+
+				// we assign each component the same amount of free memory
+				// (might be a bit of overkill for the JMs and RMs)
+				long memoryPerComponent = freeMemory / (numTaskManagers + numResourceManagers + numJobManagers);
+
+				// subtract the network buffer memory
+				long memoryMinusNetworkBuffers = memoryPerComponent - networkBuffersMemory;
+
+				// calculate the managed memory size
+				long managedMemoryBytes = (long) (memoryMinusNetworkBuffers * memoryFraction);
+
+				return managedMemoryBytes >>> 20;
+			} else {
+				return memorySize;
+			}
+		} else {
+			return managedMemoryPerTaskManager;
+		}
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/d8b22e04/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/JobLeaderIdService.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/JobLeaderIdService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/JobLeaderIdService.java
index 6c7e249..56e72c0 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/JobLeaderIdService.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/JobLeaderIdService.java
@@ -59,7 +59,7 @@ public class JobLeaderIdService {
 	/** Actions to call when the job leader changes */
 	private JobLeaderIdActions jobLeaderIdActions;
 
-	public JobLeaderIdService(HighAvailabilityServices highAvailabilityServices) throws Exception {
+	public JobLeaderIdService(HighAvailabilityServices highAvailabilityServices) {
 		this.highAvailabilityServices = Preconditions.checkNotNull(highAvailabilityServices);
 
 		this.runningJobsRegistry = highAvailabilityServices.getRunningJobsRegistry();

http://git-wip-us.apache.org/repos/asf/flink/blob/d8b22e04/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java
index 7240087..a81c214 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java
@@ -156,8 +156,9 @@ public abstract class ResourceManager<WorkerType extends Serializable>
 			throw new ResourceManagerException("Could not create the slot manager.", e);
 		}
 
+		leaderElectionService = highAvailabilityServices.getResourceManagerLeaderElectionService();
+
 		try {
-			leaderElectionService = highAvailabilityServices.getResourceManagerLeaderElectionService();
 			leaderElectionService.start(this);
 		} catch (Exception e) {
 			throw new ResourceManagerException("Could not start the leader election service.", e);

http://git-wip-us.apache.org/repos/asf/flink/blob/d8b22e04/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerRunner.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerRunner.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerRunner.java
new file mode 100644
index 0000000..959b727
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerRunner.java
@@ -0,0 +1,102 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
+import org.apache.flink.runtime.metrics.MetricRegistry;
+import org.apache.flink.runtime.resourcemanager.exceptions.ConfigurationException;
+import org.apache.flink.runtime.resourcemanager.slotmanager.DefaultSlotManager;
+import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManagerFactory;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+import org.apache.flink.runtime.rpc.RpcService;
+import org.apache.flink.util.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Simple {@link StandaloneResourceManager} runner. It instantiates the resource manager's services
+ * and handles fatal errors by shutting the resource manager down.
+ */
+public class ResourceManagerRunner implements FatalErrorHandler {
+
+	private static final Logger LOG = LoggerFactory.getLogger(ResourceManagerRunner.class);
+
+	private final Object lock = new Object();
+
+	private final ResourceManager<?> resourceManager;
+
+	public ResourceManagerRunner(
+			final Configuration configuration,
+			final RpcService rpcService,
+			final HighAvailabilityServices highAvailabilityServices,
+			final MetricRegistry metricRegistry) throws ConfigurationException {
+
+		Preconditions.checkNotNull(configuration);
+		Preconditions.checkNotNull(rpcService);
+		Preconditions.checkNotNull(highAvailabilityServices);
+		Preconditions.checkNotNull(metricRegistry);
+
+		final ResourceManagerConfiguration resourceManagerConfiguration = ResourceManagerConfiguration.fromConfiguration(configuration);
+		final SlotManagerFactory slotManagerFactory = new DefaultSlotManager.Factory();
+		final JobLeaderIdService jobLeaderIdService = new JobLeaderIdService(highAvailabilityServices);
+
+		this.resourceManager = new StandaloneResourceManager(
+			rpcService,
+			resourceManagerConfiguration,
+			highAvailabilityServices,
+			slotManagerFactory,
+			metricRegistry,
+			jobLeaderIdService,
+			this);
+	}
+
+	//-------------------------------------------------------------------------------------
+	// Lifecycle management
+	//-------------------------------------------------------------------------------------
+
+	public void start() throws Exception {
+		resourceManager.start();
+	}
+
+	public void shutDown() throws Exception {
+		shutDownInternally();
+	}
+
+	private void shutDownInternally() throws Exception {
+		synchronized (lock) {
+			resourceManager.shutDown();
+		}
+	}
+
+	//-------------------------------------------------------------------------------------
+	// Fatal error handler
+	//-------------------------------------------------------------------------------------
+
+	@Override
+	public void onFatalError(Throwable exception) {
+		LOG.error("Encountered fatal error.", exception);
+
+		try {
+			shutDownInternally();
+		} catch (Exception e) {
+			LOG.error("Could not properly shut down the resource manager.", e);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/d8b22e04/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/exceptions/ConfigurationException.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/exceptions/ConfigurationException.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/exceptions/ConfigurationException.java
index f081fff..0007318 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/exceptions/ConfigurationException.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/exceptions/ConfigurationException.java
@@ -19,7 +19,7 @@
 package org.apache.flink.runtime.resourcemanager.exceptions;
 
 /**
- * Base class for configuration related exception which occur when creating a configuration.
+ * Exception which occures when creating a configuration object fails.
  */
 public class ConfigurationException extends Exception {
 	private static final long serialVersionUID = 3971647332059381556L;

http://git-wip-us.apache.org/repos/asf/flink/blob/d8b22e04/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/exceptions/ResourceManagerRunner.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/exceptions/ResourceManagerRunner.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/exceptions/ResourceManagerRunner.java
deleted file mode 100644
index 0c7e4e4..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/exceptions/ResourceManagerRunner.java
+++ /dev/null
@@ -1,105 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.runtime.resourcemanager.exceptions;
-
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
-import org.apache.flink.runtime.metrics.MetricRegistry;
-import org.apache.flink.runtime.resourcemanager.JobLeaderIdService;
-import org.apache.flink.runtime.resourcemanager.ResourceManager;
-import org.apache.flink.runtime.resourcemanager.ResourceManagerConfiguration;
-import org.apache.flink.runtime.resourcemanager.StandaloneResourceManager;
-import org.apache.flink.runtime.resourcemanager.slotmanager.DefaultSlotManager;
-import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManagerFactory;
-import org.apache.flink.runtime.rpc.FatalErrorHandler;
-import org.apache.flink.runtime.rpc.RpcService;
-import org.apache.flink.util.Preconditions;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Simple {@link StandaloneResourceManager} runner. It instantiates the resource manager's services
- * and handles fatal errors by shutting the resource manager down.
- */
-public class ResourceManagerRunner implements FatalErrorHandler {
-
-	private static final Logger LOG = LoggerFactory.getLogger(ResourceManagerRunner.class);
-
-	private final Object lock = new Object();
-
-	private final ResourceManager<?> resourceManager;
-
-	public ResourceManagerRunner(
-			final Configuration configuration,
-			final RpcService rpcService,
-			final HighAvailabilityServices highAvailabilityServices,
-			final MetricRegistry metricRegistry) throws Exception {
-
-		Preconditions.checkNotNull(configuration);
-		Preconditions.checkNotNull(rpcService);
-		Preconditions.checkNotNull(highAvailabilityServices);
-		Preconditions.checkNotNull(metricRegistry);
-
-		final ResourceManagerConfiguration resourceManagerConfiguration = ResourceManagerConfiguration.fromConfiguration(configuration);
-		final SlotManagerFactory slotManagerFactory = new DefaultSlotManager.Factory();
-		final JobLeaderIdService jobLeaderIdService = new JobLeaderIdService(highAvailabilityServices);
-
-		this.resourceManager = new StandaloneResourceManager(
-			rpcService,
-			resourceManagerConfiguration,
-			highAvailabilityServices,
-			slotManagerFactory,
-			metricRegistry,
-			jobLeaderIdService,
-			this);
-	}
-
-	//-------------------------------------------------------------------------------------
-	// Lifecycle management
-	//-------------------------------------------------------------------------------------
-
-	public void start() throws Exception {
-		resourceManager.start();
-	}
-
-	public void shutDown() throws Exception {
-		shutDownInternally();
-	}
-
-	private void shutDownInternally() throws Exception {
-		synchronized (lock) {
-			resourceManager.shutDown();
-		}
-	}
-
-	//-------------------------------------------------------------------------------------
-	// Fatal error handler
-	//-------------------------------------------------------------------------------------
-
-	@Override
-	public void onFatalError(Throwable exception) {
-		LOG.error("Encountered fatal error.", exception);
-
-		try {
-			shutDownInternally();
-		} catch (Exception e) {
-			LOG.error("Could not properly shut down the resource manager.", e);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/d8b22e04/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 8187fde..b981829 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
@@ -80,6 +80,7 @@ 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.util.ExceptionUtils;
 import org.apache.flink.util.Preconditions;
 
 import java.io.IOException;
@@ -217,47 +218,33 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 	 * Called to shut down the TaskManager. The method closes all TaskManager services.
 	 */
 	@Override
-	public void shutDown() {
+	public void shutDown() throws Exception {
 		log.info("Stopping TaskManager {}.", getAddress());
 
+		Exception exception = null;
+
 		taskSlotTable.stop();
 
 		if (isConnectedToResourceManager()) {
-			try {
-				resourceManagerConnection.close();
-			} catch (Exception e) {
-				log.error("Could not cleanly close the ResourceManager connection.", e);
-			}
+			resourceManagerConnection.close();
 		}
 
-		try {
-			ioManager.shutdown();
-		} catch (Exception e) {
-			log.error("IOManager did not shut down properly.", e);
-		}
+		ioManager.shutdown();
 
-		try {
-			memoryManager.shutdown();
-		} catch (Exception e) {
-			log.error("MemoryManager did not shut down properly.", e);
-		}
+		memoryManager.shutdown();
 
-		try {
-			networkEnvironment.shutdown();
-		} catch (Exception e) {
-			log.error("Network environment did not shut down properly.", e);
-		}
+		networkEnvironment.shutdown();
+
+		fileCache.shutdown();
 
 		try {
-			fileCache.shutdown();
+			super.shutDown();
 		} catch (Exception e) {
-			log.error("File cache did not shut down properly.", e);
+			exception = ExceptionUtils.firstOrSuppressed(e, exception);
 		}
 
-		try {
-			metricRegistry.shutdown();
-		} catch (Exception e) {
-			log.error("MetricRegistry did not shut down properly.", e);
+		if (exception != null) {
+			ExceptionUtils.rethrowException(exception, "Error while shutting the TaskExecutor down.");
 		}
 
 		log.info("Stopped TaskManager {}.", getAddress());

http://git-wip-us.apache.org/repos/asf/flink/blob/d8b22e04/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java
index 7d9ee55..99a7c5d 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java
@@ -24,9 +24,12 @@ import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.akka.AkkaUtils;
 import org.apache.flink.runtime.clusterframework.types.ResourceID;
 import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
+import org.apache.flink.runtime.metrics.MetricRegistry;
+import org.apache.flink.runtime.metrics.groups.TaskManagerMetricGroup;
 import org.apache.flink.runtime.rpc.FatalErrorHandler;
 import org.apache.flink.runtime.rpc.RpcService;
 import org.apache.flink.runtime.rpc.RpcServiceUtils;
+import org.apache.flink.runtime.taskexecutor.utils.TaskExecutorMetricsInitializer;
 import org.apache.flink.runtime.util.LeaderRetrievalUtils;
 
 import org.apache.flink.util.Preconditions;
@@ -66,7 +69,8 @@ public class TaskManagerRunner implements FatalErrorHandler {
 		Configuration configuration,
 		ResourceID resourceID,
 		RpcService rpcService,
-		HighAvailabilityServices highAvailabilityServices) throws Exception {
+		HighAvailabilityServices highAvailabilityServices,
+		MetricRegistry metricRegistry) throws Exception {
 
 		this.configuration = Preconditions.checkNotNull(configuration);
 		this.resourceID = Preconditions.checkNotNull(resourceID);
@@ -81,10 +85,20 @@ public class TaskManagerRunner implements FatalErrorHandler {
 			remoteAddress,
 			false);
 
-		TaskManagerServices taskManagerServices = TaskManagerServices.fromConfiguration(taskManagerServicesConfiguration, resourceID);
+		TaskManagerServices taskManagerServices = TaskManagerServices.fromConfiguration(
+			taskManagerServicesConfiguration,
+			resourceID);
 
 		TaskManagerConfiguration taskManagerConfiguration = TaskManagerConfiguration.fromConfiguration(configuration);
 
+		TaskManagerMetricGroup taskManagerMetricGroup = new TaskManagerMetricGroup(
+			metricRegistry,
+			taskManagerServices.getTaskManagerLocation().getHostname(),
+			resourceID.toString());
+
+		// Initialize the TM metrics
+		TaskExecutorMetricsInitializer.instantiateStatusMetrics(taskManagerMetricGroup, taskManagerServices.getNetworkEnvironment());
+
 		this.taskManager = new TaskExecutor(
 			taskManagerConfiguration,
 			taskManagerServices.getTaskManagerLocation(),
@@ -93,8 +107,8 @@ public class TaskManagerRunner implements FatalErrorHandler {
 			taskManagerServices.getIOManager(),
 			taskManagerServices.getNetworkEnvironment(),
 			highAvailabilityServices,
-			taskManagerServices.getMetricRegistry(),
-			taskManagerServices.getTaskManagerMetricGroup(),
+			metricRegistry,
+			taskManagerMetricGroup,
 			taskManagerServices.getBroadcastVariableManager(),
 			taskManagerServices.getFileCache(),
 			taskManagerServices.getTaskSlotTable(),
@@ -117,7 +131,11 @@ public class TaskManagerRunner implements FatalErrorHandler {
 
 	protected void shutDownInternally() {
 		synchronized(lock) {
-			taskManager.shutDown();
+			try {
+				taskManager.shutDown();
+			} catch (Exception e) {
+				LOG.error("Could not properly shut down the task manager.", e);
+			}
 		}
 	}
 


[52/52] [abbrv] flink git commit: [hotfix] [tests] Clean up lots of warnings

Posted by se...@apache.org.
[hotfix] [tests] Clean up lots of warnings


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

Branch: refs/heads/master
Commit: 368d0da1f2bb7b91b60b3bd5bd66b1fcd497a87c
Parents: afa2717
Author: Stephan Ewen <se...@apache.org>
Authored: Fri Dec 23 19:14:40 2016 +0100
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Dec 23 20:54:28 2016 +0100

----------------------------------------------------------------------
 .../savepoint/MigrationV0ToV1Test.java          |  3 +-
 .../hash/NonReusingHashJoinIteratorITCase.java  | 51 ++++++++------------
 2 files changed, 23 insertions(+), 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/368d0da1/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/savepoint/MigrationV0ToV1Test.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/savepoint/MigrationV0ToV1Test.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/savepoint/MigrationV0ToV1Test.java
index 4208fe5..512768d 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/savepoint/MigrationV0ToV1Test.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/savepoint/MigrationV0ToV1Test.java
@@ -59,6 +59,7 @@ import java.util.concurrent.ThreadLocalRandom;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 
+@SuppressWarnings("deprecation")
 public class MigrationV0ToV1Test {
 
 	@Rule
@@ -135,7 +136,7 @@ public class MigrationV0ToV1Test {
 						StreamStateHandle stateHandle = legacyOperatorState.get(c);
 						try (InputStream is = stateHandle.openInputStream()) {
 							Tuple4<Integer, Integer, Integer, Integer> expTestState = new Tuple4<>(0, t, p, c);
-							Tuple4<Integer, Integer, Integer, Integer> actTestState = null;
+							Tuple4<Integer, Integer, Integer, Integer> actTestState;
 							//check function state
 							if (p % 4 != 0) {
 								assertEquals(1, is.read());

http://git-wip-us.apache.org/repos/asf/flink/blob/368d0da1/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/NonReusingHashJoinIteratorITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/NonReusingHashJoinIteratorITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/NonReusingHashJoinIteratorITCase.java
index cc5c472..d45c8c8 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/NonReusingHashJoinIteratorITCase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/NonReusingHashJoinIteratorITCase.java
@@ -54,7 +54,8 @@ import java.util.Map.Entry;
 import org.apache.flink.api.common.typeutils.GenericPairComparator;
 import org.apache.flink.api.java.tuple.Tuple2;
 
-@SuppressWarnings({"serial"})
+@SuppressWarnings({"serial", "EqualsWhichDoesntCheckParameterClass", 
+		"StatementWithEmptyBody", "KeySetIterationMayUseEntrySet"})
 public class NonReusingHashJoinIteratorITCase {
 	
 	private static final int MEMORY_SIZE = 16000000;		// total memory
@@ -133,7 +134,7 @@ public class NonReusingHashJoinIteratorITCase {
 					collectTupleData(input1),
 					collectTupleData(input2));
 			
-			final FlatJoinFunction matcher = new TupleMatchRemovingJoin(expectedMatchesMap);
+			final TupleMatchRemovingJoin matcher = new TupleMatchRemovingJoin(expectedMatchesMap);
 			final Collector<Tuple2<Integer, String>> collector = new DiscardingOutputCollector<Tuple2<Integer, String>>();
 	
 			// reset the generators
@@ -150,7 +151,8 @@ public class NonReusingHashJoinIteratorITCase {
 						this.memoryManager, ioManager, this.parentTask, 1.0, false, false, true);
 			
 			iterator.open();
-			
+
+			//noinspection StatementWithEmptyBody
 			while (iterator.callWithNextKey(matcher, collector));
 			
 			iterator.close();
@@ -227,7 +229,7 @@ public class NonReusingHashJoinIteratorITCase {
 			input1 = new UnionIterator<>(inList1);
 			input2 = new UnionIterator<>(inList2);
 			
-			final FlatJoinFunction matcher = new TupleMatchRemovingJoin(expectedMatchesMap);
+			final TupleMatchRemovingJoin matcher = new TupleMatchRemovingJoin(expectedMatchesMap);
 			final Collector<Tuple2<Integer, String>> collector = new DiscardingOutputCollector<>();
 	
 			NonReusingBuildFirstHashJoinIterator<Tuple2<Integer, String>, Tuple2<Integer, String>, Tuple2<Integer, String>> iterator =
@@ -269,7 +271,7 @@ public class NonReusingHashJoinIteratorITCase {
 					collectTupleData(input1),
 					collectTupleData(input2));
 			
-			final FlatJoinFunction matcher = new TupleMatchRemovingJoin(expectedMatchesMap);
+			final TupleMatchRemovingJoin matcher = new TupleMatchRemovingJoin(expectedMatchesMap);
 			final Collector<Tuple2<Integer, String>> collector = new DiscardingOutputCollector<>();
 	
 			// reset the generators
@@ -363,7 +365,7 @@ public class NonReusingHashJoinIteratorITCase {
 			input1 = new UnionIterator<>(inList1);
 			input2 = new UnionIterator<>(inList2);
 			
-			final FlatJoinFunction matcher = new TupleMatchRemovingJoin(expectedMatchesMap);
+			final TupleMatchRemovingJoin matcher = new TupleMatchRemovingJoin(expectedMatchesMap);
 			final Collector<Tuple2<Integer, String>> collector = new DiscardingOutputCollector<>();
 
 			NonReusingBuildSecondHashJoinIterator<Tuple2<Integer, String>, Tuple2<Integer, String>, Tuple2<Integer, String>> iterator =
@@ -499,7 +501,7 @@ public class NonReusingHashJoinIteratorITCase {
 					collectTupleData(input1),
 					collectTupleData(input2));
 	
-			final FlatJoinFunction matcher = new TupleMatchRemovingJoin(expectedMatchesMap);
+			final TupleMatchRemovingJoin matcher = new TupleMatchRemovingJoin(expectedMatchesMap);
 			final Collector<Tuple2<Integer, String>> collector = new DiscardingOutputCollector<>();
 	
 			// reset the generators
@@ -548,7 +550,7 @@ public class NonReusingHashJoinIteratorITCase {
 				collectTupleData(input1),
 				collectTupleData(input2));
 	
-			final FlatJoinFunction matcher = new TupleMatchRemovingJoin(expectedMatchesMap);
+			final TupleMatchRemovingJoin matcher = new TupleMatchRemovingJoin(expectedMatchesMap);
 			final Collector<Tuple2<Integer, String>> collector = new DiscardingOutputCollector<>();
 	
 			// reset the generators
@@ -597,7 +599,7 @@ public class NonReusingHashJoinIteratorITCase {
 				collectTupleData(input1),
 				collectTupleData(input2));
 	
-			final FlatJoinFunction matcher = new TupleMatchRemovingJoin(expectedMatchesMap);
+			final TupleMatchRemovingJoin matcher = new TupleMatchRemovingJoin(expectedMatchesMap);
 			final Collector<Tuple2<Integer, String>> collector = new DiscardingOutputCollector<>();
 	
 			// reset the generators
@@ -646,7 +648,7 @@ public class NonReusingHashJoinIteratorITCase {
 					collectTupleData(input1),
 					collectTupleData(input2));
 	
-			final FlatJoinFunction matcher = new TupleMatchRemovingJoin(expectedMatchesMap);
+			final TupleMatchRemovingJoin matcher = new TupleMatchRemovingJoin(expectedMatchesMap);
 			final Collector<Tuple2<Integer, String>> collector = new DiscardingOutputCollector<>();
 	
 			// reset the generators
@@ -695,7 +697,7 @@ public class NonReusingHashJoinIteratorITCase {
 				collectTupleData(input1),
 				collectTupleData(input2));
 	
-			final FlatJoinFunction matcher = new TupleMatchRemovingJoin(expectedMatchesMap);
+			final TupleMatchRemovingJoin matcher = new TupleMatchRemovingJoin(expectedMatchesMap);
 			final Collector<Tuple2<Integer, String>> collector = new DiscardingOutputCollector<>();
 	
 			// reset the generators
@@ -744,7 +746,7 @@ public class NonReusingHashJoinIteratorITCase {
 				collectTupleData(input1),
 				collectTupleData(input2));
 	
-			final FlatJoinFunction matcher = new TupleMatchRemovingJoin(expectedMatchesMap);
+			final TupleMatchRemovingJoin matcher = new TupleMatchRemovingJoin(expectedMatchesMap);
 			final Collector<Tuple2<Integer, String>> collector = new DiscardingOutputCollector<>();
 	
 			// reset the generators
@@ -1012,23 +1014,12 @@ public class NonReusingHashJoinIteratorITCase {
 
 		@Override
 		public boolean equals(Object obj) {
-			TupleMatch o = (TupleMatch) obj;
+			TupleMatch that = (TupleMatch) obj;
 
-			if(left != null && o.left != null && right != null && o.right != null) {
-				return this.left.equals(o.left) && this.right.equals(o.right);
-			}
-			else if(left == null && o.left == null) {
-				return this.right.equals(o.right);
-			}
-			else if(right == null && o.right == null) {
-				return this.left.equals(o.left);
-			}
-			else if(left == null && o.left == null && right == null && o.right == null) {
-				return true;
-			}
-			else {
-				return false;
-			}
+			return (this.right == null ? that.right == null :
+							(that.right != null && this.right.equals(that.right))) &&
+					(this.left == null ? that.left == null :
+							(that.left != null && this.left.equals(that.left)));
 		}
 		
 		@Override
@@ -1041,7 +1032,7 @@ public class NonReusingHashJoinIteratorITCase {
 		@Override
 		public String toString() {
 			String s = left == null ? "<null>" : left;
-			s += ", " + right == null ? "<null>" : right;
+			s += ", " + (right == null ? "<null>" : right);
 			return s;
 		}
 	}
@@ -1056,7 +1047,7 @@ public class NonReusingHashJoinIteratorITCase {
 
 		public TupleIntPairMatch(int left, String right) {
 			this.left = left;
-			this.right = new String(right);
+			this.right = right;
 		}
 
 		@Override


[16/52] [abbrv] flink git commit: [FLINK-4851] [rm] Introduce FatalErrorHandler and MetricRegistry to RM

Posted by se...@apache.org.
[FLINK-4851] [rm] Introduce FatalErrorHandler and MetricRegistry to RM

This PR introduces a FatalErrorHandler and the MetricRegistry to the RM. The FatalErrorHandler is used to handle fatal errors. Additionally, the PR adds the MetricRegistry to the RM which can be used
to register metrics.

Apart from these changes the PR restructures the code of the RM a little bit and fixes some
blocking operations.

The PR also moves the TestingFatalErrorHandler into the util package of flink-runtime test. That
it is usable across multiple tests.

Introduce ResourceManagerRunner to handle errors in the ResourceManager

This closes #2655.


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

Branch: refs/heads/master
Commit: 70af08c62f1d30929a34314f8a5f704e2c44d5a1
Parents: a193425
Author: Till Rohrmann <tr...@apache.org>
Authored: Mon Oct 17 16:03:02 2016 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Dec 23 20:54:24 2016 +0100

----------------------------------------------------------------------
 .../flink/runtime/minicluster/MiniCluster.java  |  35 +-
 .../resourcemanager/ResourceManager.java        | 463 +++++++++++--------
 .../ResourceManagerConfiguration.java           |  86 ++++
 .../StandaloneResourceManager.java              |  25 +-
 .../exceptions/ConfigurationException.java      |  38 ++
 .../exceptions/ResourceManagerException.java    |  40 ++
 .../exceptions/ResourceManagerRunner.java       | 102 ++++
 .../registration/WorkerRegistration.java        |   5 +-
 .../resourcemanager/ResourceManagerHATest.java  |  45 +-
 .../ResourceManagerJobMasterTest.java           |  59 ++-
 .../ResourceManagerTaskExecutorTest.java        |  87 ++--
 .../resourcemanager/TestingResourceManager.java |  53 ---
 .../TestingSlotManagerFactory.java              |  30 ++
 .../slotmanager/SlotProtocolTest.java           |  47 +-
 .../runtime/taskexecutor/TaskExecutorTest.java  |  60 +--
 .../runtime/util/TestingFatalErrorHandler.java  |  83 ++++
 16 files changed, 863 insertions(+), 395 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/70af08c6/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java
index d63f9a7..b005330 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java
@@ -32,10 +32,7 @@ import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils;
 import org.apache.flink.runtime.jobgraph.JobGraph;
 import org.apache.flink.runtime.metrics.MetricRegistry;
 import org.apache.flink.runtime.metrics.MetricRegistryConfiguration;
-import org.apache.flink.runtime.resourcemanager.ResourceManager;
-import org.apache.flink.runtime.resourcemanager.StandaloneResourceManager;
-import org.apache.flink.runtime.resourcemanager.slotmanager.DefaultSlotManager;
-import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManagerFactory;
+import org.apache.flink.runtime.resourcemanager.exceptions.ResourceManagerRunner;
 import org.apache.flink.runtime.rpc.RpcService;
 import org.apache.flink.runtime.rpc.akka.AkkaRpcService;
 import org.apache.flink.runtime.taskexecutor.TaskManagerRunner;
@@ -52,7 +49,6 @@ import static org.apache.flink.util.ExceptionUtils.firstOrSuppressed;
 import static org.apache.flink.util.Preconditions.checkNotNull;
 import static org.apache.flink.util.Preconditions.checkState;
 
-
 public class MiniCluster {
 
 	private static final Logger LOG = LoggerFactory.getLogger(MiniCluster.class);
@@ -82,7 +78,7 @@ public class MiniCluster {
 	private HighAvailabilityServices haServices;
 
 	@GuardedBy("lock")
-	private ResourceManager<?>[] resourceManagers;
+	private ResourceManagerRunner[] resourceManagerRunners;
 
 	@GuardedBy("lock")
 	private TaskManagerRunner[] taskManagerRunners;
@@ -231,7 +227,7 @@ public class MiniCluster {
 
 				// bring up the ResourceManager(s)
 				LOG.info("Starting {} ResourceManger(s)", numResourceManagers);
-				resourceManagers = startResourceManagers(
+				resourceManagerRunners = startResourceManagers(
 						configuration, haServices, metricRegistry, numResourceManagers, resourceManagerRpcServices);
 
 				// bring up the TaskManager(s) for the mini cluster
@@ -303,8 +299,8 @@ public class MiniCluster {
 			jobDispatcher = null;
 		}
 
-		if (resourceManagers != null) {
-			for (ResourceManager<?> rm : resourceManagers) {
+		if (resourceManagerRunners != null) {
+			for (ResourceManagerRunner rm : resourceManagerRunners) {
 				if (rm != null) {
 					try {
 						rm.shutDown();
@@ -313,7 +309,7 @@ public class MiniCluster {
 					}
 				}
 			}
-			resourceManagers = null;
+			resourceManagerRunners = null;
 		}
 
 		// shut down the RpcServices
@@ -435,26 +431,27 @@ public class MiniCluster {
 		return new AkkaRpcService(actorSystem, askTimeout);
 	}
 
-	protected ResourceManager<?>[] startResourceManagers(
+	protected ResourceManagerRunner[] startResourceManagers(
 			Configuration configuration,
 			HighAvailabilityServices haServices,
 			MetricRegistry metricRegistry,
 			int numResourceManagers,
 			RpcService[] resourceManagerRpcServices) throws Exception {
 
-		final StandaloneResourceManager[] resourceManagers = new StandaloneResourceManager[numResourceManagers];
-		final SlotManagerFactory slotManagerFactory = new DefaultSlotManager.Factory(); 
+		final ResourceManagerRunner[] resourceManagerRunners = new ResourceManagerRunner[numResourceManagers];
 
 		for (int i = 0; i < numResourceManagers; i++) {
-			resourceManagers[i] = new StandaloneResourceManager(
-					resourceManagerRpcServices[i],
-					haServices,
-					slotManagerFactory);
 
-			resourceManagers[i].start();
+			resourceManagerRunners[i] = new ResourceManagerRunner(
+				configuration,
+				resourceManagerRpcServices[i],
+				haServices,
+				metricRegistry);
+
+			resourceManagerRunners[i].start();
 		}
 
-		return resourceManagers;
+		return resourceManagerRunners;
 	}
 
 	protected TaskManagerRunner[] startTaskManagers(

http://git-wip-us.apache.org/repos/asf/flink/blob/70af08c6/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java
index f1a5073..4161972 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java
@@ -38,12 +38,15 @@ import org.apache.flink.runtime.leaderelection.LeaderContender;
 import org.apache.flink.runtime.leaderelection.LeaderElectionService;
 import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener;
 import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+import org.apache.flink.runtime.metrics.MetricRegistry;
+import org.apache.flink.runtime.resourcemanager.exceptions.ResourceManagerException;
 import org.apache.flink.runtime.resourcemanager.messages.jobmanager.RMSlotRequestRejected;
 import org.apache.flink.runtime.resourcemanager.messages.jobmanager.RMSlotRequestReply;
 import org.apache.flink.runtime.resourcemanager.registration.JobMasterRegistration;
 import org.apache.flink.runtime.resourcemanager.registration.WorkerRegistration;
 import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager;
 import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManagerFactory;
+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;
@@ -54,6 +57,7 @@ import org.apache.flink.runtime.registration.RegistrationResponse;
 import org.apache.flink.runtime.taskexecutor.SlotReport;
 import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway;
 import org.apache.flink.runtime.taskexecutor.TaskExecutorRegistrationSuccess;
+import org.apache.flink.util.ExceptionUtils;
 
 import java.io.Serializable;
 import java.util.HashMap;
@@ -61,6 +65,8 @@ import java.util.Iterator;
 import java.util.Map;
 import java.util.UUID;
 import java.util.concurrent.Callable;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.Executor;
 
 import static org.apache.flink.util.Preconditions.checkNotNull;
@@ -79,8 +85,8 @@ public abstract class ResourceManager<WorkerType extends Serializable>
 		extends RpcEndpoint<ResourceManagerGateway>
 		implements LeaderContender {
 
-	/** The exit code with which the process is stopped in case of a fatal error. */
-	protected static final int EXIT_CODE_FATAL_ERROR = -13;
+	/** Configuration of the resource manager */
+	private final ResourceManagerConfiguration resourceManagerConfiguration;
 
 	/** All currently registered JobMasterGateways scoped by JobID. */
 	private final Map<JobID, JobMasterRegistration> jobMasters;
@@ -97,6 +103,12 @@ public abstract class ResourceManager<WorkerType extends Serializable>
 	/** The factory to construct the SlotManager. */
 	private final SlotManagerFactory slotManagerFactory;
 
+	/** Registry to use for metrics */
+	private final MetricRegistry metricRegistry;
+
+	/** Fatal error handler */
+	private final FatalErrorHandler fatalErrorHandler;
+
 	/** The SlotManager created by the slotManagerFactory when the ResourceManager is started. */
 	private SlotManager slotManager;
 
@@ -107,64 +119,89 @@ public abstract class ResourceManager<WorkerType extends Serializable>
 	private volatile UUID leaderSessionID;
 
 	/** All registered listeners for status updates of the ResourceManager. */
-	private Map<String, InfoMessageListenerRpcGateway> infoMessageListeners;
-
-	/** Default timeout for messages */
-	private final Time timeout = Time.seconds(5);
+	private ConcurrentMap<String, InfoMessageListenerRpcGateway> infoMessageListeners;
 
 	public ResourceManager(
 			RpcService rpcService,
+			ResourceManagerConfiguration resourceManagerConfiguration,
 			HighAvailabilityServices highAvailabilityServices,
-			SlotManagerFactory slotManagerFactory) {
+			SlotManagerFactory slotManagerFactory,
+			MetricRegistry metricRegistry,
+			FatalErrorHandler fatalErrorHandler) {
+
 		super(rpcService);
+
+		this.resourceManagerConfiguration = checkNotNull(resourceManagerConfiguration);
 		this.highAvailabilityServices = checkNotNull(highAvailabilityServices);
 		this.slotManagerFactory = checkNotNull(slotManagerFactory);
-		this.jobMasters = new HashMap<>();
-		this.leaderListeners = new HashMap<>();
-		this.taskExecutors = new HashMap<>();
-		this.leaderSessionID = new UUID(0, 0);
-		infoMessageListeners = new HashMap<>();
+		this.metricRegistry = checkNotNull(metricRegistry);
+		this.fatalErrorHandler = checkNotNull(fatalErrorHandler);
+
+		this.jobMasters = new HashMap<>(4);
+		this.leaderListeners = new HashMap<>(4);
+		this.taskExecutors = new HashMap<>(8);
+		this.leaderSessionID = null;
+		infoMessageListeners = new ConcurrentHashMap<>(8);
 	}
 
+	// ------------------------------------------------------------------------
+	//  RPC lifecycle methods
+	// ------------------------------------------------------------------------
+
 	@Override
-	public void start() {
+	public void start() throws Exception {
 		// start a leader
+		super.start();
+
 		try {
-			super.start();
 			// SlotManager should start first
 			slotManager = slotManagerFactory.create(createResourceManagerServices());
+		} catch (Exception e) {
+			throw new ResourceManagerException("Could not create the slot manager.", e);
+		}
+
+		try {
 			leaderElectionService = highAvailabilityServices.getResourceManagerLeaderElectionService();
 			leaderElectionService.start(this);
+		} catch (Exception e) {
+			throw new ResourceManagerException("Could not start the leader election service.", e);
+		}
+
+		try {
 			// framework specific initialization
 			initialize();
-		} catch (Throwable e) {
-			log.error("A fatal error happened when starting the ResourceManager", e);
-			throw new RuntimeException("A fatal error happened when starting the ResourceManager", e);
+		} catch (Exception e) {
+			throw new ResourceManagerException("Could not initialize the resource manager.", e);
 		}
 	}
 
 	@Override
-	public void shutDown() {
+	public void shutDown() throws Exception {
+		Exception exception = null;
+
 		try {
 			leaderElectionService.stop();
-			clearState();
+		} catch (Exception e) {
+			exception = ExceptionUtils.firstOrSuppressed(e, exception);
+		}
+
+		clearState();
+
+		try {
 			super.shutDown();
-		} catch (Throwable e) {
-			log.error("A fatal error happened when shutdown the ResourceManager", e);
-			throw new RuntimeException("A fatal error happened when shutdown the ResourceManager", e);
+		} catch (Exception e) {
+			exception = ExceptionUtils.firstOrSuppressed(e, exception);
 		}
-	}
 
-	/**
-	 * Gets the leader session id of current resourceManager.
-	 *
-	 * @return return the leaderSessionId of current resourceManager, this returns null until the current resourceManager is granted leadership.
-	 */
-	@VisibleForTesting
-	UUID getLeaderSessionID() {
-		return this.leaderSessionID;
+		if (exception != null) {
+			ExceptionUtils.rethrowException(exception, "Error while shutting the ResourceManager down.");
+		}
 	}
 
+	// ------------------------------------------------------------------------
+	//  RPC methods
+	// ------------------------------------------------------------------------
+
 	/**
 	 * Register a {@link JobMaster} at the resource manager.
 	 *
@@ -191,11 +228,12 @@ public abstract class ResourceManager<WorkerType extends Serializable>
 					highAvailabilityServices.getJobManagerLeaderRetriever(jobID);
 				jobIdLeaderListener = new JobIdLeaderListener(jobID, jobMasterLeaderRetriever);
 			} catch (Exception e) {
-				log.warn("Failed to start JobMasterLeaderRetriever for JobID {}", jobID);
-				FlinkCompletableFuture<RegistrationResponse> responseFuture = new FlinkCompletableFuture<>();
-				responseFuture.complete(new RegistrationResponse.Decline("Failed to retrieve JobMasterLeaderRetriever"));
-				return responseFuture;
+				log.warn("Failed to start JobMasterLeaderRetriever for job id {}", jobID, e);
+
+				return FlinkCompletableFuture.<RegistrationResponse>completed(
+					new RegistrationResponse.Decline("Failed to retrieve JobMasterLeaderRetriever"));
 			}
+
 			leaderListeners.put(jobID, jobIdLeaderListener);
 		}
 
@@ -211,6 +249,8 @@ public abstract class ResourceManager<WorkerType extends Serializable>
 						throw new Exception("Invalid leader session id");
 					}
 
+					final Time timeout = resourceManagerConfiguration.getTimeout();
+
 					if (!jobIdLeaderListener.getLeaderID().get(timeout.getSize(), timeout.getUnit())
 							.equals(jobMasterLeaderId)) {
 						throw new Exception("Leader Id did not match");
@@ -224,10 +264,9 @@ public abstract class ResourceManager<WorkerType extends Serializable>
 				@Override
 				public RegistrationResponse apply(JobMasterGateway jobMasterGateway, Throwable throwable) {
 
-					if (throwable != null) {
-						return new RegistrationResponse.Decline(throwable.getMessage());
-					}
-
+				if (throwable != null) {
+					return new RegistrationResponse.Decline(throwable.getMessage());
+				} else {
 					if (!leaderSessionID.equals(resourceManagerLeaderId)) {
 						log.warn("Discard registration from JobMaster {} at ({}) because the expected leader session ID {}" +
 								" did not equal the received leader session ID  {}",
@@ -252,10 +291,12 @@ public abstract class ResourceManager<WorkerType extends Serializable>
 					if (existingRegistration != null) {
 						log.info("Replacing JobMaster registration for newly registered JobMaster with JobID {}.", jobID);
 					}
-					return new JobMasterRegistrationSuccess(5000, resourceManagerLeaderId);
-
+					return new JobMasterRegistrationSuccess(
+						resourceManagerConfiguration.getHeartbeatInterval().toMilliseconds(),
+						resourceManagerLeaderId);
 				}
-			}, getMainThreadExecutor());
+			}
+		}, getMainThreadExecutor());
 	}
 
 	/**
@@ -274,38 +315,44 @@ public abstract class ResourceManager<WorkerType extends Serializable>
 		final ResourceID resourceID,
 		final SlotReport slotReport) {
 
-		return getRpcService().execute(new Callable<TaskExecutorGateway>() {
-			@Override
-			public TaskExecutorGateway call() throws Exception {
-				if (!leaderSessionID.equals(resourceManagerLeaderId)) {
-					log.warn("Discard registration from TaskExecutor {} at ({}) because the expected leader session ID {} did " +
-							"not equal the received leader session ID  {}",
-						resourceID, taskExecutorAddress, leaderSessionID, resourceManagerLeaderId);
-					throw new Exception("Invalid leader session id");
-				}
-				return getRpcService().connect(taskExecutorAddress, TaskExecutorGateway.class)
-					.get(timeout.toMilliseconds(), timeout.getUnit());
-			}
-		}).handleAsync(new BiFunction<TaskExecutorGateway, Throwable, RegistrationResponse>() {
-			@Override
-			public RegistrationResponse apply(TaskExecutorGateway taskExecutorGateway, Throwable throwable) {
-				if (throwable != null) {
-					return new RegistrationResponse.Decline(throwable.getMessage());
-				} else {
-					WorkerRegistration oldRegistration = taskExecutors.remove(resourceID);
-					if (oldRegistration != null) {
-						// TODO :: suggest old taskExecutor to stop itself
-						log.info("Replacing old instance of worker for ResourceID {}", resourceID);
+		if (leaderSessionID.equals(resourceManagerLeaderId)) {
+			Future<TaskExecutorGateway> taskExecutorGatewayFuture = getRpcService().connect(taskExecutorAddress, TaskExecutorGateway.class);
+
+			return taskExecutorGatewayFuture.handleAsync(new BiFunction<TaskExecutorGateway, Throwable, RegistrationResponse>() {
+				@Override
+				public RegistrationResponse apply(TaskExecutorGateway taskExecutorGateway, Throwable throwable) {
+					if (throwable != null) {
+						return new RegistrationResponse.Decline(throwable.getMessage());
+					} else {
+						WorkerRegistration<WorkerType> oldRegistration = taskExecutors.remove(resourceID);
+						if (oldRegistration != null) {
+							// TODO :: suggest old taskExecutor to stop itself
+							log.info("Replacing old instance of worker for ResourceID {}", resourceID);
+						}
+
+						WorkerType newWorker = workerStarted(resourceID);
+						WorkerRegistration<WorkerType> registration =
+							new WorkerRegistration<>(taskExecutorGateway, newWorker);
+
+						taskExecutors.put(resourceID, registration);
+						slotManager.registerTaskExecutor(resourceID, registration, slotReport);
+
+						return new TaskExecutorRegistrationSuccess(
+							registration.getInstanceID(),
+							resourceManagerConfiguration.getHeartbeatInterval().toMilliseconds());
 					}
-					WorkerType newWorker = workerStarted(resourceID);
-					WorkerRegistration<WorkerType> registration =
-						new WorkerRegistration<>(taskExecutorGateway, newWorker);
-					taskExecutors.put(resourceID, registration);
-					slotManager.registerTaskExecutor(resourceID, registration, slotReport);
-					return new TaskExecutorRegistrationSuccess(registration.getInstanceID(), 5000);
 				}
-			}
-		}, getMainThreadExecutor());
+			}, getMainThreadExecutor());
+		} else {
+			log.warn("Discard registration from TaskExecutor {} at ({}) because the expected leader session ID {} did " +
+					"not equal the received leader session ID  {}",
+				resourceID, taskExecutorAddress, leaderSessionID, resourceManagerLeaderId);
+
+			return FlinkCompletableFuture.<RegistrationResponse>completed(
+				new RegistrationResponse.Decline("Discard registration because the leader id " +
+					resourceManagerLeaderId + " does not match the expected leader id " +
+					leaderSessionID + '.'));
+		}
 	}
 
 	/**
@@ -337,33 +384,91 @@ public abstract class ResourceManager<WorkerType extends Serializable>
 	 * Notification from a TaskExecutor that a slot has become available
 	 * @param resourceManagerLeaderId TaskExecutor's resource manager leader id
 	 * @param instanceID TaskExecutor's instance id
-	 * @param slotID The slot id of the available slot
+	 * @param slotId The slot id of the available slot
 	 * @return SlotAvailableReply
 	 */
 	@RpcMethod
 	public void notifySlotAvailable(
 			final UUID resourceManagerLeaderId,
 			final InstanceID instanceID,
-			final SlotID slotID) {
+			final SlotID slotId) {
 
 		if (resourceManagerLeaderId.equals(leaderSessionID)) {
-			final ResourceID resourceId = slotID.getResourceID();
+			final ResourceID resourceId = slotId.getResourceID();
 			WorkerRegistration<WorkerType> registration = taskExecutors.get(resourceId);
 
 			if (registration != null) {
-				InstanceID registrationInstanceID = registration.getInstanceID();
-				if (registrationInstanceID.equals(instanceID)) {
-					runAsync(new Runnable() {
-						@Override
-						public void run() {
-							slotManager.notifySlotAvailable(resourceId, slotID);
-						}
-					});
+				InstanceID registrationId = registration.getInstanceID();
+
+				if (registrationId.equals(instanceID)) {
+					slotManager.notifySlotAvailable(resourceId, slotId);
+				} else {
+					log.debug("Invalid registration id for slot available message. This indicates an" +
+						" outdated request.");
 				}
+			} else {
+				log.debug("Could not find registration for resource id {}. Discarding the slot available" +
+					"message {}.", resourceId, slotId);
 			}
+		} else {
+			log.debug("Discarding notify slot available message for slot {}, because the " +
+				"leader id {} did not match the expected leader id {}.", slotId,
+				resourceManagerLeaderId, leaderSessionID);
+		}
+	}
+
+	/**
+	 * Registers an info message listener
+	 *
+	 * @param address address of infoMessage listener to register to this resource manager
+	 */
+	@RpcMethod
+	public void registerInfoMessageListener(final String address) {
+		if(infoMessageListeners.containsKey(address)) {
+			log.warn("Receive a duplicate registration from info message listener on ({})", address);
+		} else {
+			Future<InfoMessageListenerRpcGateway> infoMessageListenerRpcGatewayFuture = getRpcService().connect(address, InfoMessageListenerRpcGateway.class);
+
+			infoMessageListenerRpcGatewayFuture.thenAcceptAsync(new AcceptFunction<InfoMessageListenerRpcGateway>() {
+				@Override
+				public void accept(InfoMessageListenerRpcGateway gateway) {
+					log.info("Receive a registration from info message listener on ({})", address);
+					infoMessageListeners.put(address, gateway);
+				}
+			}, getMainThreadExecutor());
+
+			infoMessageListenerRpcGatewayFuture.exceptionallyAsync(new ApplyFunction<Throwable, Void>() {
+				@Override
+				public Void apply(Throwable failure) {
+					log.warn("Receive a registration from unreachable info message listener on ({})", address);
+					return null;
+				}
+			}, getRpcService().getExecutor());
 		}
 	}
 
+	/**
+	 * Unregisters an info message listener
+	 *
+	 * @param address of the  info message listener to unregister from this resource manager
+	 *
+	 */
+	@RpcMethod
+	public void unRegisterInfoMessageListener(final String address) {
+		infoMessageListeners.remove(address);
+	}
+
+	/**
+	 * Cleanup application and shut down cluster
+	 *
+	 * @param finalStatus
+	 * @param optionalDiagnostics
+	 */
+	@RpcMethod
+	public void shutDownCluster(final ApplicationStatus finalStatus, final String optionalDiagnostics) {
+		log.info("shut down cluster because application is in {}, diagnostics {}", finalStatus, optionalDiagnostics);
+		shutDownApplication(finalStatus, optionalDiagnostics);
+	}
 
 	// ------------------------------------------------------------------------
 	//  Leader Contender
@@ -372,23 +477,35 @@ public abstract class ResourceManager<WorkerType extends Serializable>
 	/**
 	 * Callback method when current resourceManager is granted leadership
 	 *
-	 * @param leaderSessionID unique leadershipID
+	 * @param newLeaderSessionID unique leadershipID
 	 */
 	@Override
-	public void grantLeadership(final UUID leaderSessionID) {
+	public void grantLeadership(final UUID newLeaderSessionID) {
 		runAsync(new Runnable() {
 			@Override
 			public void run() {
-				log.info("ResourceManager {} was granted leadership with leader session ID {}", getAddress(), leaderSessionID);
-				// confirming the leader session ID might be blocking,
-				leaderElectionService.confirmLeaderSessionID(leaderSessionID);
-				ResourceManager.this.leaderSessionID = leaderSessionID;
+				log.info("ResourceManager {} was granted leadership with leader session ID {}", getAddress(), newLeaderSessionID);
+
+				// clear the state if we've been the leader before
+				if (leaderSessionID != null) {
+					clearState();
+				}
+
+				leaderSessionID = newLeaderSessionID;
+
+				getRpcService().execute(new Runnable() {
+					@Override
+					public void run() {
+						// confirming the leader session ID might be blocking,
+						leaderElectionService.confirmLeaderSessionID(newLeaderSessionID);
+					}
+				});
 			}
 		});
 	}
 
 	/**
-	 * Callback method when current resourceManager lose leadership.
+	 * Callback method when current resourceManager loses leadership.
 	 */
 	@Override
 	public void revokeLeadership() {
@@ -396,7 +513,10 @@ public abstract class ResourceManager<WorkerType extends Serializable>
 			@Override
 			public void run() {
 				log.info("ResourceManager {} was revoked leadership.", getAddress());
+
 				clearState();
+
+				leaderSessionID = null;
 			}
 		});
 	}
@@ -408,106 +528,98 @@ public abstract class ResourceManager<WorkerType extends Serializable>
 	 */
 	@Override
 	public void handleError(final Exception exception) {
-		log.error("ResourceManager received an error from the LeaderElectionService.", exception);
-		// terminate ResourceManager in case of an error
-		shutDown();
+		onFatalErrorAsync(new ResourceManagerException("Received an error from the LeaderElectionService.", exception));
 	}
 
 	/**
-	 * Registers an infoMessage listener
+	 * This method should be called by the framework once it detects that a currently registered
+	 * task executor has failed.
 	 *
-	 * @param infoMessageListenerAddress address of infoMessage listener to register to this resource manager
+	 * @param resourceID Id of the worker that has failed.
+	 * @param message An informational message that explains why the worker failed.
 	 */
-	@RpcMethod
-	public void registerInfoMessageListener(final String infoMessageListenerAddress) {
-		if(infoMessageListeners.containsKey(infoMessageListenerAddress)) {
-			log.warn("Receive a duplicate registration from info message listener on ({})", infoMessageListenerAddress);
-		} else {
-			Future<InfoMessageListenerRpcGateway> infoMessageListenerRpcGatewayFuture = getRpcService().connect(infoMessageListenerAddress, InfoMessageListenerRpcGateway.class);
-
-			infoMessageListenerRpcGatewayFuture.thenAcceptAsync(new AcceptFunction<InfoMessageListenerRpcGateway>() {
-				@Override
-				public void accept(InfoMessageListenerRpcGateway gateway) {
-					log.info("Receive a registration from info message listener on ({})", infoMessageListenerAddress);
-					infoMessageListeners.put(infoMessageListenerAddress, gateway);
-				}
-			}, getMainThreadExecutor());
+	public void notifyWorkerFailed(final ResourceID resourceID, final String message) {
+		runAsync(new Runnable() {
+			@Override
+			public void run() {
+				WorkerRegistration<WorkerType> workerRegistration = taskExecutors.remove(resourceID);
 
-			infoMessageListenerRpcGatewayFuture.exceptionallyAsync(new ApplyFunction<Throwable, Void>() {
-				@Override
-				public Void apply(Throwable failure) {
-					log.warn("Receive a registration from unreachable info message listener on ({})", infoMessageListenerAddress);
-					return null;
+				if (workerRegistration != null) {
+					log.info("Task manager {} failed because {}.", resourceID, message);
+					// TODO :: suggest failed task executor to stop itself
+					slotManager.notifyTaskManagerFailure(resourceID);
+				} else {
+					log.debug("Could not find a registered task manager with the process id {}.", resourceID);
 				}
-			}, getMainThreadExecutor());
-		}
-	}
-
-	/**
-	 * Unregisters an infoMessage listener
-	 *
-	 * @param infoMessageListenerAddress address of infoMessage listener to unregister from this resource manager
-	 *
-	 */
-	@RpcMethod
-	public void unRegisterInfoMessageListener(final String infoMessageListenerAddress) {
-		infoMessageListeners.remove(infoMessageListenerAddress);
+			}
+		});
 	}
 
-	/**
-	 * Cleanup application and shut down cluster
-	 *
-	 * @param finalStatus
-	 * @param optionalDiagnostics
-	 */
-	@RpcMethod
-	public void shutDownCluster(final ApplicationStatus finalStatus, final String optionalDiagnostics) {
-		log.info("shut down cluster because application is in {}, diagnostics {}", finalStatus, optionalDiagnostics);
-		shutDownApplication(finalStatus, optionalDiagnostics);
-	}
+	// ------------------------------------------------------------------------
+	//  Error Handling
+	// ------------------------------------------------------------------------
 
 	/**
-	 * This method should be called by the framework once it detects that a currently registered task executor has failed.
+	 * Notifies the ResourceManager that a fatal error has occurred and it cannot proceed.
+	 * This method should be used when asynchronous threads want to notify the
+	 * ResourceManager of a fatal error.
 	 *
-	 * @param resourceID Id of the worker that has failed.
-	 * @param message An informational message that explains why the worker failed.
+	 * @param t The exception describing the fatal error
 	 */
-	public void notifyWorkerFailed(final ResourceID resourceID, String message) {
+	void onFatalErrorAsync(final Throwable t) {
 		runAsync(new Runnable() {
 			@Override
 			public void run() {
-				WorkerType worker = taskExecutors.remove(resourceID).getWorker();
-				if (worker != null) {
-					// TODO :: suggest failed task executor to stop itself
-					slotManager.notifyTaskManagerFailure(resourceID);
-				}
+				onFatalError(t);
 			}
 		});
 	}
 
 	/**
-	 * Gets the number of currently started TaskManagers.
+	 * Notifies the ResourceManager that a fatal error has occurred and it cannot proceed.
+	 * This method must only be called from within the ResourceManager's main thread.
 	 *
-	 * @return The number of currently started TaskManagers.
+	 * @param t The exception describing the fatal error
 	 */
-	public int getNumberOfStartedTaskManagers() {
-		return taskExecutors.size();
+	void onFatalError(Throwable t) {
+		log.error("Fatal error occurred.", t);
+		fatalErrorHandler.onFatalError(t);
 	}
 
+	// ------------------------------------------------------------------------
+	//  Testing methods
+	// ------------------------------------------------------------------------
+
 	/**
-	 * Notifies the resource manager of a fatal error.
+	 * Gets the leader session id of current resourceManager.
 	 *
-	 * <p><b>IMPORTANT:</b> This should not cleanly shut down this master, but exit it in
-	 * such a way that a high-availability setting would restart this or fail over
-	 * to another master.
+	 * @return return the leaderSessionId of current resourceManager, this returns null until the current resourceManager is granted leadership.
 	 */
-	public void onFatalError(final String message, final Throwable error) {
-		runAsync(new Runnable() {
-			@Override
-			public void run() {
-				fatalError(message, error);
+	@VisibleForTesting
+	UUID getLeaderSessionID() {
+		return leaderSessionID;
+	}
+
+	// ------------------------------------------------------------------------
+	//  Internal methods
+	// ------------------------------------------------------------------------
+
+	private void clearState() {
+		jobMasters.clear();
+		taskExecutors.clear();
+		slotManager.clearState();
+		Iterator<JobIdLeaderListener> leaderListenerIterator =
+			leaderListeners.values().iterator();
+		while (leaderListenerIterator.hasNext()) {
+			JobIdLeaderListener listener = leaderListenerIterator.next();
+			try {
+				listener.stopService();
+			} catch (Exception e) {
+				onFatalError(e);
 			}
-		});
+			leaderListenerIterator.remove();
+		}
+		leaderSessionID = new UUID(0, 0);
 	}
 
 	// ------------------------------------------------------------------------
@@ -522,15 +634,6 @@ public abstract class ResourceManager<WorkerType extends Serializable>
 	protected abstract void initialize() throws Exception;
 
 	/**
-	 * Notifies the resource master of a fatal error.
-	 *
-	 * <p><b>IMPORTANT:</b> This should not cleanly shut down this master, but exit it in
-	 * such a way that a high-availability setting would restart this or fail over
-	 * to another master.
-	 */
-	protected abstract void fatalError(String message, Throwable error);
-
-	/**
 	 * The framework specific code for shutting down the application. This should report the
 	 * application's final status and shut down the resource manager cleanly.
 	 *
@@ -560,7 +663,7 @@ public abstract class ResourceManager<WorkerType extends Serializable>
 	// ------------------------------------------------------------------------
 
 	public void sendInfoMessage(final String message) {
-		runAsync(new Runnable() {
+		getRpcService().execute(new Runnable() {
 			@Override
 			public void run() {
 				InfoMessage infoMessage = new InfoMessage(message);
@@ -675,23 +778,5 @@ public abstract class ResourceManager<WorkerType extends Serializable>
 			ResourceManager.this.handleError(exception);
 		}
 	}
-
-	private void clearState() {
-		jobMasters.clear();
-		taskExecutors.clear();
-		slotManager.clearState();
-		Iterator<JobIdLeaderListener> leaderListenerIterator =
-			leaderListeners.values().iterator();
-		while (leaderListenerIterator.hasNext()) {
-			JobIdLeaderListener listener = leaderListenerIterator.next();
-			try {
-				listener.stopService();
-			} catch (Exception e) {
-				handleError(e);
-			}
-			leaderListenerIterator.remove();
-		}
-		leaderSessionID = new UUID(0, 0);
-	}
 }
 

http://git-wip-us.apache.org/repos/asf/flink/blob/70af08c6/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerConfiguration.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerConfiguration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerConfiguration.java
new file mode 100644
index 0000000..920f1fc
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerConfiguration.java
@@ -0,0 +1,86 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager;
+
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ConfigOptions;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.resourcemanager.exceptions.ConfigurationException;
+import org.apache.flink.util.Preconditions;
+import scala.concurrent.duration.Duration;
+
+/**
+ * Resource manager configuration
+ */
+public class ResourceManagerConfiguration {
+
+	private final Time timeout;
+	private final Time heartbeatInterval;
+
+	public ResourceManagerConfiguration(Time timeout, Time heartbeatInterval) {
+		this.timeout = Preconditions.checkNotNull(timeout);
+		this.heartbeatInterval = Preconditions.checkNotNull(heartbeatInterval);
+	}
+
+	public Time getTimeout() {
+		return timeout;
+	}
+
+	public Time getHeartbeatInterval() {
+		return heartbeatInterval;
+	}
+
+	// --------------------------------------------------------------------------
+	// Static factory methods
+	// --------------------------------------------------------------------------
+
+	public static ResourceManagerConfiguration fromConfiguration(Configuration configuration) throws ConfigurationException {
+		ConfigOption<String> timeoutOption = ConfigOptions
+			.key(ConfigConstants.AKKA_ASK_TIMEOUT)
+			.defaultValue(ConfigConstants.DEFAULT_AKKA_ASK_TIMEOUT);
+
+		final String strTimeout = configuration.getString(timeoutOption);
+		final Time timeout;
+
+		try {
+			timeout = Time.milliseconds(Duration.apply(strTimeout).toMillis());
+		} catch (NumberFormatException e) {
+			throw new ConfigurationException("Could not parse the resource manager's timeout " +
+				"value " + timeoutOption + '.', e);
+		}
+
+		ConfigOption<String> heartbeatIntervalOption = ConfigOptions
+			.key(ConfigConstants.AKKA_WATCH_HEARTBEAT_INTERVAL)
+			.defaultValue(timeout.toString());
+
+		final String strHeartbeatInterval = configuration.getString(heartbeatIntervalOption);
+		final Time heartbeatInterval;
+
+		try {
+			heartbeatInterval = Time.milliseconds(Duration.apply(strHeartbeatInterval).toMillis());
+		} catch (NumberFormatException e) {
+			throw new ConfigurationException("Could not parse the resource manager's heartbeat interval " +
+				"value " + timeoutOption + '.', e);
+		}
+
+		return new ResourceManagerConfiguration(timeout, heartbeatInterval);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/70af08c6/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/StandaloneResourceManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/StandaloneResourceManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/StandaloneResourceManager.java
index f9f55f8..926be0d 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/StandaloneResourceManager.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/StandaloneResourceManager.java
@@ -22,7 +22,9 @@ import org.apache.flink.runtime.clusterframework.ApplicationStatus;
 import org.apache.flink.runtime.clusterframework.types.ResourceID;
 import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
 import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
+import org.apache.flink.runtime.metrics.MetricRegistry;
 import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManagerFactory;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
 import org.apache.flink.runtime.rpc.RpcService;
 
 /**
@@ -33,10 +35,20 @@ import org.apache.flink.runtime.rpc.RpcService;
  */
 public class StandaloneResourceManager extends ResourceManager<ResourceID> {
 
-	public StandaloneResourceManager(RpcService rpcService,
+	public StandaloneResourceManager(
+			RpcService rpcService,
+			ResourceManagerConfiguration resourceManagerConfiguration,
 			HighAvailabilityServices highAvailabilityServices,
-			SlotManagerFactory slotManagerFactory) {
-		super(rpcService, highAvailabilityServices, slotManagerFactory);
+			SlotManagerFactory slotManagerFactory,
+			MetricRegistry metricRegistry,
+			FatalErrorHandler fatalErrorHandler) {
+		super(
+			rpcService,
+			resourceManagerConfiguration,
+			highAvailabilityServices,
+			slotManagerFactory,
+			metricRegistry,
+			fatalErrorHandler);
 	}
 
 	@Override
@@ -45,13 +57,6 @@ public class StandaloneResourceManager extends ResourceManager<ResourceID> {
 	}
 
 	@Override
-	protected void fatalError(final String message, final Throwable error) {
-		log.error("FATAL ERROR IN RESOURCE MANAGER: " + message, error);
-		// kill this process
-		System.exit(EXIT_CODE_FATAL_ERROR);
-	}
-
-	@Override
 	protected void shutDownApplication(ApplicationStatus finalStatus, String optionalDiagnostics) {
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/70af08c6/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/exceptions/ConfigurationException.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/exceptions/ConfigurationException.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/exceptions/ConfigurationException.java
new file mode 100644
index 0000000..f081fff
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/exceptions/ConfigurationException.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.exceptions;
+
+/**
+ * Base class for configuration related exception which occur when creating a configuration.
+ */
+public class ConfigurationException extends Exception {
+	private static final long serialVersionUID = 3971647332059381556L;
+
+	public ConfigurationException(String message) {
+		super(message);
+	}
+
+	public ConfigurationException(String message, Throwable cause) {
+		super(message, cause);
+	}
+
+	public ConfigurationException(Throwable cause) {
+		super(cause);
+	}
+}

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

http://git-wip-us.apache.org/repos/asf/flink/blob/70af08c6/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/exceptions/ResourceManagerRunner.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/exceptions/ResourceManagerRunner.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/exceptions/ResourceManagerRunner.java
new file mode 100644
index 0000000..1e6f04c
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/exceptions/ResourceManagerRunner.java
@@ -0,0 +1,102 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.exceptions;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
+import org.apache.flink.runtime.metrics.MetricRegistry;
+import org.apache.flink.runtime.resourcemanager.ResourceManager;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerConfiguration;
+import org.apache.flink.runtime.resourcemanager.StandaloneResourceManager;
+import org.apache.flink.runtime.resourcemanager.slotmanager.DefaultSlotManager;
+import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManagerFactory;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+import org.apache.flink.runtime.rpc.RpcService;
+import org.apache.flink.util.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Simple {@link StandaloneResourceManager} runner. It instantiates the resource manager's services
+ * and handles fatal errors by shutting the resource manager down.
+ */
+public class ResourceManagerRunner implements FatalErrorHandler {
+
+	private static final Logger LOG = LoggerFactory.getLogger(ResourceManagerRunner.class);
+
+	private final Object lock = new Object();
+
+	private final ResourceManager<?> resourceManager;
+
+	public ResourceManagerRunner(
+			final Configuration configuration,
+			final RpcService rpcService,
+			final HighAvailabilityServices highAvailabilityServices,
+			final MetricRegistry metricRegistry) throws ConfigurationException {
+
+		Preconditions.checkNotNull(configuration);
+		Preconditions.checkNotNull(rpcService);
+		Preconditions.checkNotNull(highAvailabilityServices);
+		Preconditions.checkNotNull(metricRegistry);
+
+		final ResourceManagerConfiguration resourceManagerConfiguration = ResourceManagerConfiguration.fromConfiguration(configuration);
+		final SlotManagerFactory slotManagerFactory = new DefaultSlotManager.Factory();
+
+		this.resourceManager = new StandaloneResourceManager(
+			rpcService,
+			resourceManagerConfiguration,
+			highAvailabilityServices,
+			slotManagerFactory,
+			metricRegistry,
+			this);
+	}
+
+	//-------------------------------------------------------------------------------------
+	// Lifecycle management
+	//-------------------------------------------------------------------------------------
+
+	public void start() throws Exception {
+		resourceManager.start();
+	}
+
+	public void shutDown() throws Exception {
+		shutDownInternally();
+	}
+
+	private void shutDownInternally() throws Exception {
+		synchronized (lock) {
+			resourceManager.shutDown();
+		}
+	}
+
+	//-------------------------------------------------------------------------------------
+	// Fatal error handler
+	//-------------------------------------------------------------------------------------
+
+	@Override
+	public void onFatalError(Throwable exception) {
+		LOG.error("Encountered fatal error.", exception);
+
+		try {
+			shutDownInternally();
+		} catch (Exception e) {
+			LOG.error("Could not properly shut down the resource manager.", e);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/70af08c6/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/registration/WorkerRegistration.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/registration/WorkerRegistration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/registration/WorkerRegistration.java
index ff28f94..7ee7a1f 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/registration/WorkerRegistration.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/registration/WorkerRegistration.java
@@ -19,6 +19,7 @@
 package org.apache.flink.runtime.resourcemanager.registration;
 
 import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway;
+import org.apache.flink.util.Preconditions;
 
 import java.io.Serializable;
 
@@ -29,11 +30,11 @@ public class WorkerRegistration<WorkerType extends Serializable> extends TaskExe
 
 	private static final long serialVersionUID = -2062957799469434614L;
 
-	private WorkerType worker;
+	private final WorkerType worker;
 
 	public WorkerRegistration(TaskExecutorGateway taskExecutorGateway, WorkerType worker) {
 		super(taskExecutorGateway);
-		this.worker = worker;
+		this.worker = Preconditions.checkNotNull(worker);
 	}
 
 	public WorkerType getWorker() {

http://git-wip-us.apache.org/repos/asf/flink/blob/70af08c6/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerHATest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerHATest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerHATest.java
index ce1fdca..cb38e6e 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerHATest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerHATest.java
@@ -18,24 +18,20 @@
 
 package org.apache.flink.runtime.resourcemanager;
 
+import org.apache.flink.api.common.time.Time;
 import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices;
 import org.apache.flink.runtime.leaderelection.TestingLeaderElectionService;
-import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager;
+import org.apache.flink.runtime.metrics.MetricRegistry;
 import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManagerFactory;
-import org.apache.flink.runtime.rpc.MainThreadExecutable;
-import org.apache.flink.runtime.rpc.RpcEndpoint;
-import org.apache.flink.runtime.rpc.RpcGateway;
 import org.apache.flink.runtime.rpc.RpcService;
-import org.apache.flink.runtime.rpc.StartStoppable;
+import org.apache.flink.runtime.rpc.TestingSerialRpcService;
+import org.apache.flink.runtime.util.TestingFatalErrorHandler;
 import org.junit.Assert;
 import org.junit.Test;
 
 import java.util.UUID;
 
-import static org.mockito.Matchers.any;
-import static org.mockito.Mockito.doCallRealMethod;
 import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
 
 /**
  * resourceManager HA test, including grant leadership and revoke leadership
@@ -44,37 +40,38 @@ public class ResourceManagerHATest {
 
 	@Test
 	public void testGrantAndRevokeLeadership() throws Exception {
-		// mock a RpcService which will return a special RpcGateway when call its startServer method,
-		// the returned RpcGateway directly executes runAsync calls
-		TestingResourceManagerGatewayProxy gateway = mock(TestingResourceManagerGatewayProxy.class);
-		doCallRealMethod().when(gateway).runAsync(any(Runnable.class));
-
-		RpcService rpcService = mock(RpcService.class);
-		when(rpcService.startServer(any(RpcEndpoint.class))).thenReturn(gateway);
+		RpcService rpcService = new TestingSerialRpcService();
 
 		TestingLeaderElectionService leaderElectionService = new TestingLeaderElectionService();
 		TestingHighAvailabilityServices highAvailabilityServices = new TestingHighAvailabilityServices();
 		highAvailabilityServices.setResourceManagerLeaderElectionService(leaderElectionService);
 
+		ResourceManagerConfiguration resourceManagerConfiguration = new ResourceManagerConfiguration(Time.seconds(5L), Time.seconds(5L));
+		SlotManagerFactory slotManagerFactory = new TestingSlotManagerFactory();
+		MetricRegistry metricRegistry = mock(MetricRegistry.class);
+		TestingFatalErrorHandler testingFatalErrorHandler = new TestingFatalErrorHandler();
+
 		final ResourceManager resourceManager =
-			new TestingResourceManager(rpcService, highAvailabilityServices);
+			new StandaloneResourceManager(
+				rpcService,
+				resourceManagerConfiguration,
+				highAvailabilityServices,
+				slotManagerFactory,
+				metricRegistry,
+				testingFatalErrorHandler);
 		resourceManager.start();
 		// before grant leadership, resourceManager's leaderId is null
-		Assert.assertEquals(new UUID(0,0), resourceManager.getLeaderSessionID());
+		Assert.assertEquals(null, resourceManager.getLeaderSessionID());
 		final UUID leaderId = UUID.randomUUID();
 		leaderElectionService.isLeader(leaderId);
 		// after grant leadership, resourceManager's leaderId has value
 		Assert.assertEquals(leaderId, resourceManager.getLeaderSessionID());
 		// then revoke leadership, resourceManager's leaderId is null again
 		leaderElectionService.notLeader();
-		Assert.assertEquals(new UUID(0,0), resourceManager.getLeaderSessionID());
-	}
+		Assert.assertEquals(null, resourceManager.getLeaderSessionID());
 
-	private static abstract class TestingResourceManagerGatewayProxy implements MainThreadExecutable, StartStoppable, RpcGateway {
-		@Override
-		public void runAsync(Runnable runnable) {
-			runnable.run();
+		if (testingFatalErrorHandler.hasExceptionOccurred()) {
+			testingFatalErrorHandler.rethrowError();
 		}
 	}
-
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/70af08c6/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerJobMasterTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerJobMasterTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerJobMasterTest.java
index 14afd0e..7b8d254 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerJobMasterTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerJobMasterTest.java
@@ -19,14 +19,19 @@
 package org.apache.flink.runtime.resourcemanager;
 
 import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
 import org.apache.flink.runtime.concurrent.Future;
 import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices;
 import org.apache.flink.runtime.jobmaster.JobMasterGateway;
 import org.apache.flink.runtime.jobmaster.JobMasterRegistrationSuccess;
 import org.apache.flink.runtime.leaderelection.TestingLeaderElectionService;
 import org.apache.flink.runtime.leaderelection.TestingLeaderRetrievalService;
+import org.apache.flink.runtime.metrics.MetricRegistry;
+import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManagerFactory;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
 import org.apache.flink.runtime.rpc.TestingSerialRpcService;
 import org.apache.flink.runtime.registration.RegistrationResponse;
+import org.apache.flink.runtime.util.TestingFatalErrorHandler;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -61,13 +66,18 @@ public class ResourceManagerJobMasterTest {
 		TestingLeaderElectionService resourceManagerLeaderElectionService = new TestingLeaderElectionService();
 		UUID jmLeaderID = UUID.randomUUID();
 		TestingLeaderRetrievalService jobMasterLeaderRetrievalService = new TestingLeaderRetrievalService(jobMasterAddress, jmLeaderID);
-		final ResourceManager resourceManager = createAndStartResourceManager(resourceManagerLeaderElectionService, jobID, jobMasterLeaderRetrievalService);
+		TestingFatalErrorHandler testingFatalErrorHandler = new TestingFatalErrorHandler();
+		final ResourceManager resourceManager = createAndStartResourceManager(resourceManagerLeaderElectionService, jobID, jobMasterLeaderRetrievalService, testingFatalErrorHandler);
 		final UUID rmLeaderSessionId = grantResourceManagerLeadership(resourceManagerLeaderElectionService);
 
 		// test response successful
 		Future<RegistrationResponse> successfulFuture = resourceManager.registerJobMaster(rmLeaderSessionId, jmLeaderID, jobMasterAddress, jobID);
 		RegistrationResponse response = successfulFuture.get(5, TimeUnit.SECONDS);
 		assertTrue(response instanceof JobMasterRegistrationSuccess);
+
+		if (testingFatalErrorHandler.hasExceptionOccurred()) {
+			testingFatalErrorHandler.rethrowError();
+		}
 	}
 
 	/**
@@ -80,13 +90,18 @@ public class ResourceManagerJobMasterTest {
 		TestingLeaderElectionService resourceManagerLeaderElectionService = new TestingLeaderElectionService();
 		UUID jmLeaderID = UUID.randomUUID();
 		TestingLeaderRetrievalService jobMasterLeaderRetrievalService = new TestingLeaderRetrievalService(jobMasterAddress, jmLeaderID);
-		final ResourceManager resourceManager = createAndStartResourceManager(resourceManagerLeaderElectionService, jobID, jobMasterLeaderRetrievalService);
+		TestingFatalErrorHandler testingFatalErrorHandler = new TestingFatalErrorHandler();
+		final ResourceManager resourceManager = createAndStartResourceManager(resourceManagerLeaderElectionService, jobID, jobMasterLeaderRetrievalService, testingFatalErrorHandler);
 		final UUID rmLeaderSessionId = grantResourceManagerLeadership(resourceManagerLeaderElectionService);
 
 		// test throw exception when receive a registration from job master which takes unmatched leaderSessionId
 		UUID differentLeaderSessionID = UUID.randomUUID();
 		Future<RegistrationResponse> unMatchedLeaderFuture = resourceManager.registerJobMaster(differentLeaderSessionID, jmLeaderID, jobMasterAddress, jobID);
 		assertTrue(unMatchedLeaderFuture.get(5, TimeUnit.SECONDS) instanceof RegistrationResponse.Decline);
+
+		if (testingFatalErrorHandler.hasExceptionOccurred()) {
+			testingFatalErrorHandler.rethrowError();
+		}
 	}
 
 	/**
@@ -98,7 +113,8 @@ public class ResourceManagerJobMasterTest {
 		JobID jobID = mockJobMaster(jobMasterAddress);
 		TestingLeaderElectionService resourceManagerLeaderElectionService = new TestingLeaderElectionService();
 		TestingLeaderRetrievalService jobMasterLeaderRetrievalService = new TestingLeaderRetrievalService();
-		final ResourceManager resourceManager = createAndStartResourceManager(resourceManagerLeaderElectionService, jobID, jobMasterLeaderRetrievalService);
+		TestingFatalErrorHandler testingFatalErrorHandler = new TestingFatalErrorHandler();
+		final ResourceManager resourceManager = createAndStartResourceManager(resourceManagerLeaderElectionService, jobID, jobMasterLeaderRetrievalService, testingFatalErrorHandler);
 		final UUID rmLeaderSessionId = grantResourceManagerLeadership(resourceManagerLeaderElectionService);
 		final UUID jmLeaderSessionId = grantResourceManagerLeadership(resourceManagerLeaderElectionService);
 
@@ -106,6 +122,10 @@ public class ResourceManagerJobMasterTest {
 		UUID differentLeaderSessionID = UUID.randomUUID();
 		Future<RegistrationResponse> unMatchedLeaderFuture = resourceManager.registerJobMaster(rmLeaderSessionId, differentLeaderSessionID, jobMasterAddress, jobID);
 		assertTrue(unMatchedLeaderFuture.get(5, TimeUnit.SECONDS) instanceof RegistrationResponse.Decline);
+
+		if (testingFatalErrorHandler.hasExceptionOccurred()) {
+			testingFatalErrorHandler.rethrowError();
+		}
 	}
 
 	/**
@@ -117,7 +137,8 @@ public class ResourceManagerJobMasterTest {
 		JobID jobID = mockJobMaster(jobMasterAddress);
 		TestingLeaderElectionService resourceManagerLeaderElectionService = new TestingLeaderElectionService();
 		TestingLeaderRetrievalService jobMasterLeaderRetrievalService = new TestingLeaderRetrievalService();
-		final ResourceManager resourceManager = createAndStartResourceManager(resourceManagerLeaderElectionService, jobID, jobMasterLeaderRetrievalService);
+		TestingFatalErrorHandler testingFatalErrorHandler = new TestingFatalErrorHandler();
+		final ResourceManager resourceManager = createAndStartResourceManager(resourceManagerLeaderElectionService, jobID, jobMasterLeaderRetrievalService, testingFatalErrorHandler);
 		final UUID rmLeaderSessionId = grantResourceManagerLeadership(resourceManagerLeaderElectionService);
 		final UUID jmLeaderSessionId = grantResourceManagerLeadership(resourceManagerLeaderElectionService);
 
@@ -125,6 +146,10 @@ public class ResourceManagerJobMasterTest {
 		String invalidAddress = "/jobMasterAddress2";
 		Future<RegistrationResponse> invalidAddressFuture = resourceManager.registerJobMaster(rmLeaderSessionId, jmLeaderSessionId, invalidAddress, jobID);
 		assertTrue(invalidAddressFuture.get(5, TimeUnit.SECONDS) instanceof RegistrationResponse.Decline);
+
+		if (testingFatalErrorHandler.hasExceptionOccurred()) {
+			testingFatalErrorHandler.rethrowError();
+		}
 	}
 
 	/**
@@ -136,7 +161,8 @@ public class ResourceManagerJobMasterTest {
 		JobID jobID = mockJobMaster(jobMasterAddress);
 		TestingLeaderElectionService resourceManagerLeaderElectionService = new TestingLeaderElectionService();
 		TestingLeaderRetrievalService jobMasterLeaderRetrievalService = new TestingLeaderRetrievalService();
-		final ResourceManager resourceManager = createAndStartResourceManager(resourceManagerLeaderElectionService, jobID, jobMasterLeaderRetrievalService);
+		TestingFatalErrorHandler testingFatalErrorHandler = new TestingFatalErrorHandler();
+		final ResourceManager resourceManager = createAndStartResourceManager(resourceManagerLeaderElectionService, jobID, jobMasterLeaderRetrievalService, testingFatalErrorHandler);
 		final UUID rmLeaderSessionId = grantResourceManagerLeadership(resourceManagerLeaderElectionService);
 		final UUID jmLeaderSessionId = grantResourceManagerLeadership(resourceManagerLeaderElectionService);
 
@@ -145,6 +171,10 @@ public class ResourceManagerJobMasterTest {
 		Future<RegistrationResponse> declineFuture = resourceManager.registerJobMaster(rmLeaderSessionId, jmLeaderSessionId, jobMasterAddress, unknownJobIDToHAServices);
 		RegistrationResponse response = declineFuture.get(5, TimeUnit.SECONDS);
 		assertTrue(response instanceof RegistrationResponse.Decline);
+
+		if (testingFatalErrorHandler.hasExceptionOccurred()) {
+			testingFatalErrorHandler.rethrowError();
+		}
 	}
 
 	private JobID mockJobMaster(String jobMasterAddress) {
@@ -154,11 +184,26 @@ public class ResourceManagerJobMasterTest {
 		return jobID;
 	}
 
-	private ResourceManager createAndStartResourceManager(TestingLeaderElectionService resourceManagerLeaderElectionService, JobID jobID, TestingLeaderRetrievalService jobMasterLeaderRetrievalService) {
+	private ResourceManager createAndStartResourceManager(
+			TestingLeaderElectionService resourceManagerLeaderElectionService,
+			JobID jobID,
+			TestingLeaderRetrievalService jobMasterLeaderRetrievalService,
+			FatalErrorHandler fatalErrorHandler) throws Exception {
 		TestingHighAvailabilityServices highAvailabilityServices = new TestingHighAvailabilityServices();
 		highAvailabilityServices.setResourceManagerLeaderElectionService(resourceManagerLeaderElectionService);
 		highAvailabilityServices.setJobMasterLeaderRetriever(jobID, jobMasterLeaderRetrievalService);
-		ResourceManager resourceManager = new TestingResourceManager(rpcService, highAvailabilityServices);
+
+		ResourceManagerConfiguration resourceManagerConfiguration = new ResourceManagerConfiguration(Time.seconds(5L), Time.seconds(5L));
+		SlotManagerFactory slotManagerFactory = new TestingSlotManagerFactory();
+		MetricRegistry metricRegistry = mock(MetricRegistry.class);
+
+		ResourceManager resourceManager = new StandaloneResourceManager(
+			rpcService,
+			resourceManagerConfiguration,
+			highAvailabilityServices,
+			slotManagerFactory,
+			metricRegistry,
+			fatalErrorHandler);
 		resourceManager.start();
 		return resourceManager;
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/70af08c6/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTaskExecutorTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTaskExecutorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTaskExecutorTest.java
index a577c26..4640eab 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTaskExecutorTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTaskExecutorTest.java
@@ -18,15 +18,19 @@
 
 package org.apache.flink.runtime.resourcemanager;
 
+import org.apache.flink.api.common.time.Time;
 import org.apache.flink.runtime.clusterframework.types.ResourceID;
 import org.apache.flink.runtime.concurrent.Future;
 import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices;
 import org.apache.flink.runtime.leaderelection.TestingLeaderElectionService;
+import org.apache.flink.runtime.metrics.MetricRegistry;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
 import org.apache.flink.runtime.rpc.TestingSerialRpcService;
 import org.apache.flink.runtime.registration.RegistrationResponse;
 import org.apache.flink.runtime.taskexecutor.SlotReport;
 import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway;
 import org.apache.flink.runtime.taskexecutor.TaskExecutorRegistrationSuccess;
+import org.apache.flink.runtime.util.TestingFatalErrorHandler;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -34,7 +38,6 @@ import org.junit.Test;
 import java.util.UUID;
 import java.util.concurrent.TimeUnit;
 
-import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotEquals;
 import static org.junit.Assert.assertTrue;
 import static org.mockito.Mockito.mock;
@@ -53,13 +56,16 @@ public class ResourceManagerTaskExecutorTest {
 
 	private UUID leaderSessionId;
 
+	private TestingFatalErrorHandler testingFatalErrorHandler;
+
 	@Before
 	public void setup() throws Exception {
 		rpcService = new TestingSerialRpcService();
 
 		taskExecutorResourceID = mockTaskExecutor(taskExecutorAddress);
 		TestingLeaderElectionService rmLeaderElectionService = new TestingLeaderElectionService();
-		resourceManager = createAndStartResourceManager(rmLeaderElectionService);
+		testingFatalErrorHandler = new TestingFatalErrorHandler();
+		resourceManager = createAndStartResourceManager(rmLeaderElectionService, testingFatalErrorHandler);
 		leaderSessionId = grantLeadership(rmLeaderElectionService);
 	}
 
@@ -73,18 +79,24 @@ public class ResourceManagerTaskExecutorTest {
 	 */
 	@Test
 	public void testRegisterTaskExecutor() throws Exception {
-		// test response successful
-		Future<RegistrationResponse> successfulFuture =
-			resourceManager.registerTaskExecutor(leaderSessionId, taskExecutorAddress, taskExecutorResourceID, slotReport);
-		RegistrationResponse response = successfulFuture.get(5, TimeUnit.SECONDS);
-		assertTrue(response instanceof TaskExecutorRegistrationSuccess);
-
-		// test response successful with instanceID not equal to previous when receive duplicate registration from taskExecutor
-		Future<RegistrationResponse> duplicateFuture =
-			resourceManager.registerTaskExecutor(leaderSessionId, taskExecutorAddress, taskExecutorResourceID, slotReport);
-		RegistrationResponse duplicateResponse = duplicateFuture.get();
-		assertTrue(duplicateResponse instanceof TaskExecutorRegistrationSuccess);
-		assertNotEquals(((TaskExecutorRegistrationSuccess) response).getRegistrationId(), ((TaskExecutorRegistrationSuccess) duplicateResponse).getRegistrationId());
+		try {
+			// test response successful
+			Future<RegistrationResponse> successfulFuture =
+				resourceManager.registerTaskExecutor(leaderSessionId, taskExecutorAddress, taskExecutorResourceID, slotReport);
+			RegistrationResponse response = successfulFuture.get(5, TimeUnit.SECONDS);
+			assertTrue(response instanceof TaskExecutorRegistrationSuccess);
+
+			// test response successful with instanceID not equal to previous when receive duplicate registration from taskExecutor
+			Future<RegistrationResponse> duplicateFuture =
+				resourceManager.registerTaskExecutor(leaderSessionId, taskExecutorAddress, taskExecutorResourceID, slotReport);
+			RegistrationResponse duplicateResponse = duplicateFuture.get();
+			assertTrue(duplicateResponse instanceof TaskExecutorRegistrationSuccess);
+			assertNotEquals(((TaskExecutorRegistrationSuccess) response).getRegistrationId(), ((TaskExecutorRegistrationSuccess) duplicateResponse).getRegistrationId());
+		} finally {
+			if (testingFatalErrorHandler.hasExceptionOccurred()) {
+				testingFatalErrorHandler.rethrowError();
+			}
+		}
 	}
 
 	/**
@@ -92,11 +104,17 @@ public class ResourceManagerTaskExecutorTest {
 	 */
 	@Test
 	public void testRegisterTaskExecutorWithUnmatchedLeaderSessionId() throws Exception {
-		// test throw exception when receive a registration from taskExecutor which takes unmatched leaderSessionId
-		UUID differentLeaderSessionID = UUID.randomUUID();
-		Future<RegistrationResponse> unMatchedLeaderFuture =
-			resourceManager.registerTaskExecutor(differentLeaderSessionID, taskExecutorAddress, taskExecutorResourceID, slotReport);
-		assertTrue(unMatchedLeaderFuture.get(5, TimeUnit.SECONDS) instanceof RegistrationResponse.Decline);
+		try {
+			// test throw exception when receive a registration from taskExecutor which takes unmatched leaderSessionId
+			UUID differentLeaderSessionID = UUID.randomUUID();
+			Future<RegistrationResponse> unMatchedLeaderFuture =
+				resourceManager.registerTaskExecutor(differentLeaderSessionID, taskExecutorAddress, taskExecutorResourceID, slotReport);
+			assertTrue(unMatchedLeaderFuture.get(5, TimeUnit.SECONDS) instanceof RegistrationResponse.Decline);
+		} finally {
+			if (testingFatalErrorHandler.hasExceptionOccurred()) {
+				testingFatalErrorHandler.rethrowError();
+			}
+		}
 	}
 
 	/**
@@ -104,11 +122,17 @@ public class ResourceManagerTaskExecutorTest {
 	 */
 	@Test
 	public void testRegisterTaskExecutorFromInvalidAddress() throws Exception {
-		// test throw exception when receive a registration from taskExecutor which takes invalid address
-		String invalidAddress = "/taskExecutor2";
-		Future<RegistrationResponse> invalidAddressFuture =
-			resourceManager.registerTaskExecutor(leaderSessionId, invalidAddress, taskExecutorResourceID, slotReport);
-		assertTrue(invalidAddressFuture.get(5, TimeUnit.SECONDS) instanceof RegistrationResponse.Decline);
+		try {
+			// test throw exception when receive a registration from taskExecutor which takes invalid address
+			String invalidAddress = "/taskExecutor2";
+			Future<RegistrationResponse> invalidAddressFuture =
+				resourceManager.registerTaskExecutor(leaderSessionId, invalidAddress, taskExecutorResourceID, slotReport);
+			assertTrue(invalidAddressFuture.get(5, TimeUnit.SECONDS) instanceof RegistrationResponse.Decline);
+		} finally {
+			if (testingFatalErrorHandler.hasExceptionOccurred()) {
+				testingFatalErrorHandler.rethrowError();
+			}
+		}
 	}
 
 	private ResourceID mockTaskExecutor(String taskExecutorAddress) {
@@ -118,11 +142,22 @@ public class ResourceManagerTaskExecutorTest {
 		return taskExecutorResourceID;
 	}
 
-	private StandaloneResourceManager createAndStartResourceManager(TestingLeaderElectionService rmLeaderElectionService) {
+	private StandaloneResourceManager createAndStartResourceManager(TestingLeaderElectionService rmLeaderElectionService, FatalErrorHandler fatalErrorHandler) throws Exception {
 		TestingHighAvailabilityServices highAvailabilityServices = new TestingHighAvailabilityServices();
 		highAvailabilityServices.setResourceManagerLeaderElectionService(rmLeaderElectionService);
+		TestingSlotManagerFactory slotManagerFactory = new TestingSlotManagerFactory();
+		ResourceManagerConfiguration resourceManagerConfiguration = new ResourceManagerConfiguration(Time.seconds(5L), Time.seconds(5L));
+		MetricRegistry metricRegistry = mock(MetricRegistry.class);
+
+
 		StandaloneResourceManager resourceManager =
-			new TestingResourceManager(rpcService, highAvailabilityServices);
+			new StandaloneResourceManager(
+				rpcService,
+				resourceManagerConfiguration,
+				highAvailabilityServices,
+				slotManagerFactory,
+				metricRegistry,
+				fatalErrorHandler);
 		resourceManager.start();
 		return resourceManager;
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/70af08c6/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/TestingResourceManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/TestingResourceManager.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/TestingResourceManager.java
deleted file mode 100644
index 6b4ca14..0000000
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/TestingResourceManager.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.runtime.resourcemanager;
-
-import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
-import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices;
-import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager;
-import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManagerFactory;
-import org.apache.flink.runtime.rpc.RpcService;
-
-public class TestingResourceManager extends StandaloneResourceManager {
-
-	public TestingResourceManager(RpcService rpcService) {
-		this(rpcService, new TestingHighAvailabilityServices());
-	}
-
-	public TestingResourceManager(
-			RpcService rpcService,
-			HighAvailabilityServices highAvailabilityServices) {
-		this(rpcService, highAvailabilityServices, new TestingSlotManagerFactory());
-	}
-
-	public TestingResourceManager(
-			RpcService rpcService,
-			HighAvailabilityServices highAvailabilityServices,
-			SlotManagerFactory slotManagerFactory) {
-		super(rpcService, highAvailabilityServices, slotManagerFactory);
-	}
-
-	private static class TestingSlotManagerFactory implements SlotManagerFactory {
-
-		@Override
-		public SlotManager create(ResourceManagerServices rmServices) {
-			return new TestingSlotManager(rmServices);
-		}
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/70af08c6/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/TestingSlotManagerFactory.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/TestingSlotManagerFactory.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/TestingSlotManagerFactory.java
new file mode 100644
index 0000000..6b5f6b2
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/TestingSlotManagerFactory.java
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager;
+
+import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager;
+import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManagerFactory;
+
+public class TestingSlotManagerFactory implements SlotManagerFactory {
+
+	@Override
+	public SlotManager create(ResourceManagerServices rmServices) {
+		return new TestingSlotManager(rmServices);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/70af08c6/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotProtocolTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotProtocolTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotProtocolTest.java
index 86cd1f8..08ceb86 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotProtocolTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotProtocolTest.java
@@ -30,14 +30,18 @@ import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices
 import org.apache.flink.runtime.jobmaster.JobMasterGateway;
 import org.apache.flink.runtime.leaderelection.TestingLeaderElectionService;
 import org.apache.flink.runtime.leaderelection.TestingLeaderRetrievalService;
+import org.apache.flink.runtime.metrics.MetricRegistry;
 import org.apache.flink.runtime.registration.RegistrationResponse;
+import org.apache.flink.runtime.resourcemanager.ResourceManager;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerConfiguration;
 import org.apache.flink.runtime.resourcemanager.ResourceManagerServices;
 import org.apache.flink.runtime.resourcemanager.SlotRequest;
-import org.apache.flink.runtime.resourcemanager.TestingResourceManager;
+import org.apache.flink.runtime.resourcemanager.StandaloneResourceManager;
 import org.apache.flink.runtime.resourcemanager.TestingSlotManager;
 import org.apache.flink.runtime.resourcemanager.messages.jobmanager.RMSlotRequestReply;
 import org.apache.flink.runtime.resourcemanager.messages.taskexecutor.TMSlotRequestReply;
 import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorRegistration;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
 import org.apache.flink.runtime.rpc.RpcService;
 import org.apache.flink.runtime.rpc.TestingSerialRpcService;
 import org.apache.flink.runtime.taskexecutor.SlotReport;
@@ -102,9 +106,17 @@ public class SlotProtocolTest extends TestLogger {
 		TestingLeaderElectionService rmLeaderElectionService =
 			configureHA(testingHaServices, jobID, rmAddress, rmLeaderID, jmAddress, jmLeaderID);
 
+		ResourceManagerConfiguration resourceManagerConfiguration = new ResourceManagerConfiguration(Time.seconds(5L), Time.seconds(5L));
+
 		final TestingSlotManagerFactory slotManagerFactory = new TestingSlotManagerFactory();
 		SpiedResourceManager resourceManager =
-			new SpiedResourceManager(testRpcService, testingHaServices, slotManagerFactory);
+			new SpiedResourceManager(
+				testRpcService,
+				resourceManagerConfiguration,
+				testingHaServices,
+				slotManagerFactory,
+				mock(MetricRegistry.class),
+				mock(FatalErrorHandler.class));
 		resourceManager.start();
 		rmLeaderElectionService.isLeader(rmLeaderID);
 
@@ -193,16 +205,26 @@ public class SlotProtocolTest extends TestLogger {
 			.thenReturn(new FlinkCompletableFuture<TMSlotRequestReply>());
 		testRpcService.registerGateway(tmAddress, taskExecutorGateway);
 
+		ResourceManagerConfiguration resourceManagerConfiguration = new ResourceManagerConfiguration(Time.seconds(5L), Time.seconds(5L));
+
 		TestingSlotManagerFactory slotManagerFactory = new TestingSlotManagerFactory();
-		TestingResourceManager resourceManager =
-			Mockito.spy(new TestingResourceManager(testRpcService, testingHaServices, slotManagerFactory));
+		ResourceManager<ResourceID> resourceManager =
+			Mockito.spy(new StandaloneResourceManager(
+				testRpcService,
+				resourceManagerConfiguration,
+				testingHaServices,
+				slotManagerFactory,
+				mock(MetricRegistry.class),
+				mock(FatalErrorHandler.class)));
 		resourceManager.start();
 		rmLeaderElectionService.isLeader(rmLeaderID);
 
+		Thread.sleep(1000);
+
 		Future<RegistrationResponse> registrationFuture =
 			resourceManager.registerJobMaster(rmLeaderID, jmLeaderID, jmAddress, jobID);
 		try {
-			registrationFuture.get(5, TimeUnit.SECONDS);
+			registrationFuture.get(5L, TimeUnit.SECONDS);
 		} catch (Exception e) {
 			Assert.fail("JobManager registration Future didn't become ready.");
 		}
@@ -258,15 +280,24 @@ public class SlotProtocolTest extends TestLogger {
 		return rmLeaderElectionService;
 	}
 
-	private static class SpiedResourceManager extends TestingResourceManager {
+	private static class SpiedResourceManager extends StandaloneResourceManager {
 
 		private int startNewWorkerCalled = 0;
 
 		public SpiedResourceManager(
 				RpcService rpcService,
+				ResourceManagerConfiguration resourceManagerConfiguration,
 				HighAvailabilityServices highAvailabilityServices,
-				SlotManagerFactory slotManagerFactory) {
-			super(rpcService, highAvailabilityServices, slotManagerFactory);
+				SlotManagerFactory slotManagerFactory,
+				MetricRegistry metricRegistry,
+				FatalErrorHandler fatalErrorHandler) {
+			super(
+				rpcService,
+				resourceManagerConfiguration,
+				highAvailabilityServices,
+				slotManagerFactory,
+				metricRegistry,
+				fatalErrorHandler);
 		}
 
 

http://git-wip-us.apache.org/repos/asf/flink/blob/70af08c6/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java
index 638ec56..1ef7140 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java
@@ -67,12 +67,12 @@ import org.apache.flink.runtime.rpc.FatalErrorHandler;
 import org.apache.flink.runtime.rpc.TestingSerialRpcService;
 import org.apache.flink.runtime.taskexecutor.slot.SlotOffer;
 import org.apache.flink.runtime.taskexecutor.slot.TaskSlotTable;
-import org.apache.flink.runtime.taskexecutor.exceptions.SlotAllocationException;
 import org.apache.flink.runtime.taskexecutor.slot.TimerService;
 import org.apache.flink.runtime.taskmanager.CheckpointResponder;
 import org.apache.flink.runtime.taskmanager.Task;
 import org.apache.flink.runtime.taskmanager.TaskManagerActions;
 import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
+import org.apache.flink.runtime.util.TestingFatalErrorHandler;
 import org.apache.flink.util.SerializedValue;
 import org.apache.flink.util.TestLogger;
 
@@ -83,15 +83,12 @@ import org.junit.Test;
 import org.junit.rules.TestName;
 import org.mockito.Matchers;
 import org.powermock.api.mockito.PowerMockito;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 import java.net.InetAddress;
 import java.net.URL;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.UUID;
-import java.util.concurrent.atomic.AtomicReference;
 
 import static org.junit.Assert.*;
 import static org.mockito.Mockito.*;
@@ -449,7 +446,7 @@ public class TaskExecutorTest extends TestLogger {
 					any(Time.class));
 		} finally {
 			// check if a concurrent error occurred
-			testingFatalErrorHandler.rethrowException();
+			testingFatalErrorHandler.rethrowError();
 
 			rpc.stopService();
 		}
@@ -556,63 +553,12 @@ public class TaskExecutorTest extends TestLogger {
 			assertTrue(taskSlotTable.isSlotFree(1));
 		} finally {
 			// check if a concurrent error occurred
-			testingFatalErrorHandler.rethrowException();
+			testingFatalErrorHandler.rethrowError();
 
 			rpc.stopService();
 		}
 	}
 
-	private static class TestingFatalErrorHandler implements FatalErrorHandler {
-		private static final Logger LOG = LoggerFactory.getLogger(TestingFatalErrorHandler.class);
-		private final AtomicReference<Throwable> atomicThrowable;
-
-		public TestingFatalErrorHandler() {
-			atomicThrowable = new AtomicReference<>(null);
-		}
-
-		public void rethrowException() throws TestingException {
-			Throwable throwable = atomicThrowable.get();
-
-			if (throwable != null) {
-				throw new TestingException(throwable);
-			}
-		}
-
-		public boolean hasExceptionOccurred() {
-			return atomicThrowable.get() != null;
-		}
-
-		public Throwable getException() {
-			return atomicThrowable.get();
-		}
-
-		@Override
-		public void onFatalError(Throwable exception) {
-			LOG.error("OnFatalError:", exception);
-			atomicThrowable.compareAndSet(null, exception);
-		}
-
-		//------------------------------------------------------------------
-		// static utility classes
-		//------------------------------------------------------------------
-
-		private static final class TestingException extends Exception {
-			public TestingException(String message) {
-				super(message);
-			}
-
-			public TestingException(String message, Throwable cause) {
-				super(message, cause);
-			}
-
-			public TestingException(Throwable cause) {
-				super(cause);
-			}
-
-			private static final long serialVersionUID = -4648195335470914498L;
-		}
-	}
-
 	/**
 	 * Tests that all allocation requests for slots are ignored if the slot has been reported as
 	 * free by the TaskExecutor but this report hasn't been confirmed by the ResourceManager.


[15/52] [abbrv] flink git commit: [FLINK-4851] [rm] Introduce FatalErrorHandler and MetricRegistry to RM

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/70af08c6/flink-runtime/src/test/java/org/apache/flink/runtime/util/TestingFatalErrorHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/util/TestingFatalErrorHandler.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/TestingFatalErrorHandler.java
new file mode 100644
index 0000000..616313c
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/TestingFatalErrorHandler.java
@@ -0,0 +1,83 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.util;
+
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.concurrent.atomic.AtomicReference;
+
+/**
+ * Testing fatal error handler which records the occurred exceptions during the execution of the
+ * tests. Captured exceptions are thrown as a {@link TestingException}.
+ */
+public class TestingFatalErrorHandler implements FatalErrorHandler {
+	private static final Logger LOG = LoggerFactory.getLogger(TestingFatalErrorHandler.class);
+	private final AtomicReference<Throwable> atomicThrowable;
+
+	public TestingFatalErrorHandler() {
+		atomicThrowable = new AtomicReference<>(null);
+	}
+
+	public void rethrowError() throws TestingException {
+		Throwable throwable = atomicThrowable.get();
+
+		if (throwable != null) {
+			throw new TestingException(throwable);
+		}
+	}
+
+	public boolean hasExceptionOccurred() {
+		return atomicThrowable.get() != null;
+	}
+
+	public Throwable getException() {
+		return atomicThrowable.get();
+	}
+
+	@Override
+	public void onFatalError(Throwable exception) {
+		LOG.error("OnFatalError:", exception);
+
+		if (!atomicThrowable.compareAndSet(null, exception)) {
+			atomicThrowable.get().addSuppressed(exception);
+		}
+	}
+
+	//------------------------------------------------------------------
+	// static utility classes
+	//------------------------------------------------------------------
+
+	private static final class TestingException extends Exception {
+		public TestingException(String message) {
+			super(message);
+		}
+
+		public TestingException(String message, Throwable cause) {
+			super(message, cause);
+		}
+
+		public TestingException(Throwable cause) {
+			super(cause);
+		}
+
+		private static final long serialVersionUID = -4648195335470914498L;
+	}
+}


[03/52] [abbrv] 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/c6486067
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/c6486067
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/c6486067

Branch: refs/heads/master
Commit: c64860677fb576f00c700a62a4ebb4ed415bebda
Parents: 4e5f423
Author: Kurt Young <yk...@gmail.com>
Authored: Wed Oct 12 23:25:16 2016 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Dec 23 20:54:22 2016 +0100

----------------------------------------------------------------------
 .../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 +-
 .../HighAvailabilityServices.java               |  13 +
 .../runtime/highavailability/NonHaServices.java |  13 +
 .../highavailability/RunningJobsRegistry.java   |  66 ++
 .../highavailability/ZookeeperHaServices.java   | 104 ++-
 .../highavailability/nonha/NonHaRegistry.java   |  62 ++
 .../runtime/jobmanager/OnCompletionActions.java |   3 +-
 .../runtime/jobmaster/JobManagerRunner.java     | 269 ++++--
 .../runtime/jobmaster/JobManagerServices.java   |  86 +-
 .../flink/runtime/jobmaster/JobMaster.java      | 831 +++++++------------
 .../runtime/jobmaster/JobMasterGateway.java     | 112 +--
 .../jobmaster/MiniClusterJobDispatcher.java     |  61 +-
 .../jobmaster/message/ClassloadingProps.java    |   1 -
 .../message/DisposeSavepointResponse.java       |  49 --
 .../message/TriggerSavepointResponse.java       |  74 --
 .../apache/flink/runtime/rpc/RpcService.java    |   4 +-
 .../taskexecutor/JobManagerConnection.java      |  25 +-
 .../runtime/taskexecutor/TaskExecutor.java      | 103 ++-
 .../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     |  56 +-
 .../runtime/jobmaster/JobManagerRunnerTest.java |  24 +
 30 files changed, 1184 insertions(+), 960 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/c6486067/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/c6486067/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 b800500..33f9db7 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.runtime.net.SSLUtils;
 import org.apache.flink.util.NetUtils;
@@ -45,6 +49,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
@@ -96,9 +101,16 @@ 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);
 
 		this.blobServiceConfiguration = config;
 
@@ -107,14 +119,6 @@ public class BlobServer extends Thread implements BlobService {
 		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);
@@ -135,13 +139,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);
 
 		if (config.getBoolean(ConfigConstants.BLOB_SERVICE_SSL_ENABLED,
 				ConfigConstants.DEFAULT_BLOB_SERVICE_SSL_ENABLED)) {
@@ -451,4 +449,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/c6486067/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/c6486067/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/c6486067/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/c6486067/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/c6486067/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/c6486067/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/c6486067/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/c6486067/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 d25965d..d32068e 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,16 +35,57 @@ import org.apache.flink.runtime.leaderelection.LeaderElectionService;
 import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
 import org.apache.flink.runtime.util.ZooKeeperUtils;
 
+import java.io.IOException;
 import java.util.concurrent.Executor;
 
 import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.StringUtils.isNullOrWhitespaceOnly;
 
 /**
- * An implementation of the {@link HighAvailabilityServices} 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;
@@ -54,24 +102,28 @@ public class ZookeeperHaServices implements HighAvailabilityServices {
 		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
@@ -84,7 +136,43 @@ public class ZookeeperHaServices implements HighAvailabilityServices {
 		return ZooKeeperUtils.createSubmittedJobGraphs(client, configuration, executor);
 	}
 
-	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/c6486067/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/c6486067/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/c6486067/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..74ca6f3 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,140 @@ 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.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 +201,6 @@ public class JobManagerRunner implements LeaderContender, OnCompletionActions {
 	//----------------------------------------------------------------------------------------------
 
 	public void start() throws Exception {
-		jobManager.init();
-		jobManager.start();
-
 		try {
 			leaderElectionService.start(this);
 		}
@@ -114,11 +211,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 +221,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 +257,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 +273,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 +289,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 +304,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 +360,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 +395,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 +410,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/c6486067/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..fff75d5 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,20 @@
 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.runtime.checkpoint.savepoint.SavepointStore;
+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.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;
 
@@ -40,34 +47,81 @@ public class JobManagerServices {
 
 	public final RestartStrategyFactory restartStrategyFactory;
 
-	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 {
+			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),
+			Time.of(timeout.length(), timeout.unit()));
 	}
 }


[31/52] [abbrv] flink git commit: [FLINK-5171] [runtime] fix wrong use of Preconditions.checkState in TaskManagerRunner

Posted by se...@apache.org.
[FLINK-5171] [runtime] fix wrong use of Preconditions.checkState in TaskManagerRunner

This closes #2880.


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

Branch: refs/heads/master
Commit: b58c6dcb820b7434fbec206820c57845c06d84d6
Parents: 006a19d
Author: shuai.xus <sh...@alibaba-inc.com>
Authored: Mon Nov 28 17:21:26 2016 +0800
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Dec 23 20:54:25 2016 +0100

----------------------------------------------------------------------
 .../org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java   | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/b58c6dcb/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java
index 99a7c5d..a18ff40 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java
@@ -185,7 +185,7 @@ public class TaskManagerRunner implements FatalErrorHandler {
 
 		final int rpcPort = configuration.getInteger(ConfigConstants.TASK_MANAGER_IPC_PORT_KEY, 0);
 
-		Preconditions.checkState(rpcPort < 0 || rpcPort >65535, "Invalid value for " +
+		Preconditions.checkState(rpcPort >= 0 && rpcPort <= 65535, "Invalid value for " +
 				"'%s' (port for the TaskManager actor system) : %d - Leave config parameter empty or " +
 				"use 0 to let the system choose port automatically.",
 			ConfigConstants.TASK_MANAGER_IPC_PORT_KEY, rpcPort);


[46/52] [abbrv] flink git commit: [FLINK-4928] [yarn] Implement FLIP-6 YARN Application Master Runner

Posted by se...@apache.org.
[FLINK-4928] [yarn] Implement FLIP-6 YARN Application Master Runner


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

Branch: refs/heads/master
Commit: 0113e5a467858f9cd435e80df2c2626170e5de62
Parents: e28b116
Author: shuai.xus <sh...@alibaba-inc.com>
Authored: Thu Nov 3 16:24:47 2016 +0800
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Dec 23 20:54:27 2016 +0100

----------------------------------------------------------------------
 ...bstractYarnFlinkApplicationMasterRunner.java | 213 +++++++++++++
 .../yarn/YarnFlinkApplicationMasterRunner.java  | 316 +++++++++++++++++++
 2 files changed, 529 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/0113e5a4/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnFlinkApplicationMasterRunner.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnFlinkApplicationMasterRunner.java b/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnFlinkApplicationMasterRunner.java
new file mode 100644
index 0000000..923694e
--- /dev/null
+++ b/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnFlinkApplicationMasterRunner.java
@@ -0,0 +1,213 @@
+/*
+ * 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.yarn;
+
+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.clusterframework.BootstrapTools;
+import org.apache.flink.runtime.security.SecurityContext;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.yarn.cli.FlinkYarnSessionCli;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.yarn.api.ApplicationConstants.Environment;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.util.Map;
+
+/**
+ * This class is the executable entry point for the YARN application master.
+ * It starts actor system and the actors for {@link org.apache.flink.runtime.jobmaster.JobMaster}
+ * and {@link YarnResourceManager}.
+ *
+ * The JobMasters handles Flink job execution, while the YarnResourceManager handles container
+ * allocation and failure detection.
+ */
+public abstract class AbstractYarnFlinkApplicationMasterRunner {
+
+	/** Logger */
+	protected static final Logger LOG = LoggerFactory.getLogger(AbstractYarnFlinkApplicationMasterRunner.class);
+
+	/** The process environment variables */
+	protected static final Map<String, String> ENV = System.getenv();
+
+	/** The exit code returned if the initialization of the application master failed */
+	protected static final int INIT_ERROR_EXIT_CODE = 31;
+
+	/** The host name passed by env */
+	protected String appMasterHostname;
+
+	/**
+	 * The instance entry point for the YARN application master. Obtains user group
+	 * information and calls the main work method {@link #runApplicationMaster(org.apache.flink.configuration.Configuration)} as a
+	 * privileged action.
+	 *
+	 * @param args The command line arguments.
+	 * @return The process exit code.
+	 */
+	protected int run(String[] args) {
+		try {
+			LOG.debug("All environment variables: {}", ENV);
+
+			final String yarnClientUsername = ENV.get(YarnConfigKeys.ENV_HADOOP_USER_NAME);
+			Preconditions.checkArgument(yarnClientUsername != null, "YARN client user name environment variable {} not set",
+				YarnConfigKeys.ENV_HADOOP_USER_NAME);
+
+			final String currDir = ENV.get(Environment.PWD.key());
+			Preconditions.checkArgument(currDir != null, "Current working directory variable (%s) not set", Environment.PWD.key());
+			LOG.debug("Current working directory: {}", currDir);
+
+			final String remoteKeytabPath = ENV.get(YarnConfigKeys.KEYTAB_PATH);
+			LOG.debug("Remote keytab path obtained {}", remoteKeytabPath);
+
+			final String remoteKeytabPrincipal = ENV.get(YarnConfigKeys.KEYTAB_PRINCIPAL);
+			LOG.info("Remote keytab principal obtained {}", remoteKeytabPrincipal);
+
+			String keytabPath = null;
+			if(remoteKeytabPath != null) {
+				File f = new File(currDir, Utils.KEYTAB_FILE_NAME);
+				keytabPath = f.getAbsolutePath();
+				LOG.debug("Keytab path: {}", keytabPath);
+			}
+
+			UserGroupInformation currentUser = UserGroupInformation.getCurrentUser();
+
+			LOG.info("YARN daemon is running as: {} Yarn client user obtainer: {}",
+					currentUser.getShortUserName(), yarnClientUsername );
+
+			SecurityContext.SecurityConfiguration sc = new SecurityContext.SecurityConfiguration();
+
+			//To support Yarn Secure Integration Test Scenario
+			File krb5Conf = new File(currDir, Utils.KRB5_FILE_NAME);
+			if(krb5Conf.exists() && krb5Conf.canRead()) {
+				String krb5Path = krb5Conf.getAbsolutePath();
+				LOG.info("KRB5 Conf: {}", krb5Path);
+				org.apache.hadoop.conf.Configuration conf = new org.apache.hadoop.conf.Configuration();
+				conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION, "kerberos");
+				conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHORIZATION, "true");
+				sc.setHadoopConfiguration(conf);
+			}
+
+			// Flink configuration
+			final Map<String, String> dynamicProperties =
+					FlinkYarnSessionCli.getDynamicProperties(ENV.get(YarnConfigKeys.ENV_DYNAMIC_PROPERTIES));
+			LOG.debug("YARN dynamic properties: {}", dynamicProperties);
+
+			final Configuration flinkConfig = createConfiguration(currDir, dynamicProperties);
+			if(keytabPath != null && remoteKeytabPrincipal != null) {
+				flinkConfig.setString(ConfigConstants.SECURITY_KEYTAB_KEY, keytabPath);
+				flinkConfig.setString(ConfigConstants.SECURITY_PRINCIPAL_KEY, remoteKeytabPrincipal);
+			}
+
+			SecurityContext.install(sc.setFlinkConfiguration(flinkConfig));
+
+			// Note that we use the "appMasterHostname" given by YARN here, to make sure
+			// we use the hostnames given by YARN consistently throughout akka.
+			// for akka "localhost" and "localhost.localdomain" are different actors.
+			this.appMasterHostname = ENV.get(Environment.NM_HOST.key());
+			Preconditions.checkArgument(appMasterHostname != null,
+					"ApplicationMaster hostname variable %s not set", Environment.NM_HOST.key());
+			LOG.info("YARN assigned hostname for application master: {}", appMasterHostname);
+
+			return SecurityContext.getInstalled().runSecured(new SecurityContext.FlinkSecuredRunner<Integer>() {
+				@Override
+				public Integer run() {
+					return runApplicationMaster(flinkConfig);
+				}
+			});
+
+		}
+		catch (Throwable t) {
+			// make sure that everything whatever ends up in the log
+			LOG.error("YARN Application Master initialization failed", t);
+			return INIT_ERROR_EXIT_CODE;
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  Core work method
+	// ------------------------------------------------------------------------
+
+	/**
+	 * The main work method, must run as a privileged action.
+	 *
+	 * @return The return code for the Java process.
+	 */
+	protected abstract int runApplicationMaster(Configuration config);
+
+	// ------------------------------------------------------------------------
+	//  Utilities
+	// ------------------------------------------------------------------------
+	/**
+	 * @param baseDirectory  The working directory
+	 * @param additional Additional parameters
+	 * 
+	 * @return The configuration to be used by the TaskExecutors.
+	 */
+	private static Configuration createConfiguration(String baseDirectory, Map<String, String> additional) {
+		LOG.info("Loading config from directory {}.", baseDirectory);
+
+		Configuration configuration = GlobalConfiguration.loadConfiguration(baseDirectory);
+
+		configuration.setString(ConfigConstants.FLINK_BASE_DIR_PATH_KEY, baseDirectory);
+
+		// add dynamic properties to JobManager configuration.
+		for (Map.Entry<String, String> property : additional.entrySet()) {
+			configuration.setString(property.getKey(), property.getValue());
+		}
+
+		// override zookeeper namespace with user cli argument (if provided)
+		String cliZKNamespace = ENV.get(YarnConfigKeys.ENV_ZOOKEEPER_NAMESPACE);
+		if (cliZKNamespace != null && !cliZKNamespace.isEmpty()) {
+			configuration.setString(HighAvailabilityOptions.HA_CLUSTER_ID, cliZKNamespace);
+		}
+
+		// if a web monitor shall be started, set the port to random binding
+		if (configuration.getInteger(ConfigConstants.JOB_MANAGER_WEB_PORT_KEY, 0) >= 0) {
+			configuration.setInteger(ConfigConstants.JOB_MANAGER_WEB_PORT_KEY, 0);
+		}
+
+		// if the user has set the deprecated YARN-specific config keys, we add the 
+		// corresponding generic config keys instead. that way, later code needs not
+		// deal with deprecated config keys
+
+		BootstrapTools.substituteDeprecatedConfigKey(configuration,
+			ConfigConstants.YARN_HEAP_CUTOFF_RATIO,
+			ConfigConstants.CONTAINERIZED_HEAP_CUTOFF_RATIO);
+
+		BootstrapTools.substituteDeprecatedConfigKey(configuration,
+			ConfigConstants.YARN_HEAP_CUTOFF_MIN,
+			ConfigConstants.CONTAINERIZED_HEAP_CUTOFF_MIN);
+
+		BootstrapTools.substituteDeprecatedConfigPrefix(configuration,
+			ConfigConstants.YARN_APPLICATION_MASTER_ENV_PREFIX,
+			ConfigConstants.CONTAINERIZED_MASTER_ENV_PREFIX);
+
+		BootstrapTools.substituteDeprecatedConfigPrefix(configuration,
+			ConfigConstants.YARN_TASK_MANAGER_ENV_PREFIX,
+			ConfigConstants.CONTAINERIZED_TASK_MANAGER_ENV_PREFIX);
+
+		return configuration;
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/0113e5a4/flink-yarn/src/main/java/org/apache/flink/yarn/YarnFlinkApplicationMasterRunner.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnFlinkApplicationMasterRunner.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnFlinkApplicationMasterRunner.java
new file mode 100644
index 0000000..e58c77e
--- /dev/null
+++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnFlinkApplicationMasterRunner.java
@@ -0,0 +1,316 @@
+/*
+ * 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.yarn;
+
+import akka.actor.ActorSystem;
+import org.apache.flink.api.common.JobExecutionResult;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.akka.AkkaUtils;
+import org.apache.flink.runtime.client.JobExecutionException;
+import org.apache.flink.runtime.clusterframework.ApplicationStatus;
+import org.apache.flink.runtime.clusterframework.BootstrapTools;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobmanager.OnCompletionActions;
+import org.apache.flink.runtime.jobmaster.JobManagerRunner;
+import org.apache.flink.runtime.metrics.MetricRegistry;
+import org.apache.flink.runtime.metrics.MetricRegistryConfiguration;
+import org.apache.flink.runtime.resourcemanager.JobLeaderIdService;
+import org.apache.flink.runtime.resourcemanager.ResourceManager;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerConfiguration;
+import org.apache.flink.runtime.resourcemanager.exceptions.ConfigurationException;
+import org.apache.flink.runtime.resourcemanager.slotmanager.DefaultSlotManager;
+import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManagerFactory;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+import org.apache.flink.runtime.rpc.RpcService;
+import org.apache.flink.runtime.rpc.akka.AkkaRpcService;
+import org.apache.flink.runtime.util.EnvironmentInformation;
+import org.apache.flink.runtime.util.JvmShutdownSafeguard;
+import org.apache.flink.runtime.util.SignalHandler;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.concurrent.duration.FiniteDuration;
+
+import javax.annotation.concurrent.GuardedBy;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.ObjectInputStream;
+
+/**
+ * This class is the executable entry point for the YARN application master.
+ * It starts actor system and the actors for {@link org.apache.flink.runtime.jobmaster.JobManagerRunner}
+ * and {@link org.apache.flink.yarn.YarnResourceManager}.
+ *
+ * The JobMasnagerRunner start a {@link org.apache.flink.runtime.jobmaster.JobMaster}
+ * JobMaster handles Flink job execution, while the YarnResourceManager handles container
+ * allocation and failure detection.
+ */
+public class YarnFlinkApplicationMasterRunner extends AbstractYarnFlinkApplicationMasterRunner
+		implements OnCompletionActions, FatalErrorHandler {
+
+	/** Logger */
+	protected static final Logger LOG = LoggerFactory.getLogger(YarnFlinkApplicationMasterRunner.class);
+
+	/** The job graph file path */
+	private static final String JOB_GRAPH_FILE_PATH = "flink.jobgraph.path";
+
+	/** The lock to guard startup / shutdown / manipulation methods */
+	private final Object lock = new Object();
+
+	@GuardedBy("lock")
+	private MetricRegistry metricRegistry;
+
+	@GuardedBy("lock")
+	private HighAvailabilityServices haServices;
+
+	@GuardedBy("lock")
+	private RpcService commonRpcService;
+
+	@GuardedBy("lock")
+	private ResourceManager resourceManager;
+
+	@GuardedBy("lock")
+	private JobManagerRunner jobManagerRunner;
+
+	@GuardedBy("lock")
+	private JobGraph jobGraph;
+
+	// ------------------------------------------------------------------------
+	//  Program entry point
+	// ------------------------------------------------------------------------
+
+	/**
+	 * The entry point for the YARN application master.
+	 *
+	 * @param args The command line arguments.
+	 */
+	public static void main(String[] args) {
+		EnvironmentInformation.logEnvironmentInfo(LOG, "YARN ApplicationMaster runner", args);
+		SignalHandler.register(LOG);
+		JvmShutdownSafeguard.installAsShutdownHook(LOG);
+
+		// run and exit with the proper return code
+		int returnCode = new YarnFlinkApplicationMasterRunner().run(args);
+		System.exit(returnCode);
+	}
+
+	@Override
+	protected int runApplicationMaster(Configuration config) {
+
+		try {
+			// ---- (1) create common services
+
+			// try to start the rpc service
+			// using the port range definition from the config.
+			final String amPortRange = config.getString(
+					ConfigConstants.YARN_APPLICATION_MASTER_PORT,
+					ConfigConstants.DEFAULT_YARN_JOB_MANAGER_PORT);
+
+			synchronized (lock) {
+				haServices = HighAvailabilityServicesUtils.createAvailableOrEmbeddedServices(config);
+				metricRegistry = new MetricRegistry(MetricRegistryConfiguration.fromConfiguration(config));
+				commonRpcService = createRpcService(config, appMasterHostname, amPortRange);
+
+				// ---- (2) init resource manager -------
+				resourceManager = createResourceManager(config);
+
+				// ---- (3) init job master parameters
+				jobManagerRunner = createJobManagerRunner(config);
+
+				// ---- (4) start the resource manager  and job manager runner:
+				resourceManager.start();
+				LOG.debug("YARN Flink Resource Manager started");
+
+				jobManagerRunner.start();
+				LOG.debug("Job Manager Runner started");
+
+				// ---- (5) start the web monitor
+				// TODO: add web monitor
+			}
+
+			// wait for resource manager to finish
+			resourceManager.getTerminationFuture().get();
+			// everything started, we can wait until all is done or the process is killed
+			LOG.info("YARN Application Master finished");
+		}
+		catch (Throwable t) {
+			// make sure that everything whatever ends up in the log
+			LOG.error("YARN Application Master initialization failed", t);
+			shutdown(ApplicationStatus.FAILED, t.getMessage());
+			return INIT_ERROR_EXIT_CODE;
+		}
+
+		return 0;
+	}
+
+	// ------------------------------------------------------------------------
+	//  Utilities
+	// ------------------------------------------------------------------------
+
+	protected RpcService createRpcService(
+			Configuration configuration,
+			String bindAddress,
+			String portRange) throws Exception{
+		ActorSystem actorSystem = BootstrapTools.startActorSystem(configuration, bindAddress, portRange, LOG);
+		FiniteDuration duration = AkkaUtils.getTimeout(configuration);
+		return new AkkaRpcService(actorSystem, Time.of(duration.length(), duration.unit()));
+	}
+
+	private ResourceManager createResourceManager(Configuration config) throws ConfigurationException {
+		final ResourceManagerConfiguration resourceManagerConfiguration = ResourceManagerConfiguration.fromConfiguration(config);
+		final SlotManagerFactory slotManagerFactory = new DefaultSlotManager.Factory();
+		final JobLeaderIdService jobLeaderIdService = new JobLeaderIdService(haServices);
+
+		return new YarnResourceManager(config,
+				ENV,
+				commonRpcService,
+				resourceManagerConfiguration,
+				haServices,
+				slotManagerFactory,
+				metricRegistry,
+				jobLeaderIdService,
+				this);
+	}
+
+	private JobManagerRunner createJobManagerRunner(Configuration config) throws Exception{
+		// first get JobGraph from local resources
+		//TODO: generate the job graph from user's jar
+		jobGraph = loadJobGraph(config);
+
+		// we first need to mark the job as running in the HA services, so that the
+		// JobManager leader will recognize that it as work to do
+		try {
+			haServices.getRunningJobsRegistry().setJobRunning(jobGraph.getJobID());
+		}
+		catch (Throwable t) {
+			throw new JobExecutionException(jobGraph.getJobID(),
+					"Could not register the job at the high-availability services", t);
+		}
+
+		// now the JobManagerRunner
+		return new JobManagerRunner(
+				jobGraph, config,
+				commonRpcService,
+				haServices,
+				this,
+				this);
+	}
+
+	protected void shutdown(ApplicationStatus status, String msg) {
+		synchronized (lock) {
+			if (jobManagerRunner != null) {
+				try {
+					jobManagerRunner.shutdown();
+				} catch (Throwable tt) {
+					LOG.warn("Failed to stop the JobManagerRunner", tt);
+				}
+			}
+			if (resourceManager != null) {
+				try {
+					resourceManager.shutDownCluster(status, msg);
+					resourceManager.shutDown();
+				} catch (Throwable tt) {
+					LOG.warn("Failed to stop the ResourceManager", tt);
+				}
+			}
+			if (commonRpcService != null) {
+				try {
+					commonRpcService.stopService();
+				} catch (Throwable tt) {
+					LOG.error("Error shutting down resource manager rpc service", tt);
+				}
+			}
+			if (haServices != null) {
+				try {
+					haServices.shutdown();
+				} catch (Throwable tt) {
+					LOG.warn("Failed to stop the HA service", tt);
+				}
+			}
+			if (metricRegistry != null) {
+				try {
+					metricRegistry.shutdown();
+				} catch (Throwable tt) {
+					LOG.warn("Failed to stop the metrics registry", tt);
+				}
+			}
+		}
+	}
+
+	private static JobGraph loadJobGraph(Configuration config) throws Exception {
+		JobGraph jg = null;
+		String jobGraphFile = config.getString(JOB_GRAPH_FILE_PATH, "job.graph");
+		if (jobGraphFile != null) {
+			File fp = new File(jobGraphFile);
+			if (fp.isFile()) {
+				FileInputStream input = new FileInputStream(fp);
+				ObjectInputStream obInput = new ObjectInputStream(input);
+				jg = (JobGraph) obInput.readObject();
+				input.close();
+			}
+		}
+		if (jg == null) {
+			throw new Exception("Fail to load job graph " + jobGraphFile);
+		}
+		return jg;
+	}
+
+	//-------------------------------------------------------------------------------------
+	// Fatal error handler
+	//-------------------------------------------------------------------------------------
+
+	@Override
+	public void onFatalError(Throwable exception) {
+		LOG.error("Encountered fatal error.", exception);
+
+		shutdown(ApplicationStatus.FAILED, exception.getMessage());
+	}
+
+	//----------------------------------------------------------------------------------------------
+	// Result and error handling methods
+	//----------------------------------------------------------------------------------------------
+
+	/**
+	 * Job completion notification triggered by JobManager
+	 */
+	@Override
+	public void jobFinished(JobExecutionResult result) {
+		shutdown(ApplicationStatus.SUCCEEDED, null);
+	}
+
+	/**
+	 * Job completion notification triggered by JobManager
+	 */
+	@Override
+	public void jobFailed(Throwable cause) {
+		shutdown(ApplicationStatus.FAILED, cause.getMessage());
+	}
+
+	/**
+	 * Job completion notification triggered by self
+	 */
+	@Override
+	public void jobFinishedByOther() {
+		shutdown(ApplicationStatus.UNKNOWN, null);
+	}
+}


[40/52] [abbrv] flink git commit: [FLINK-5093] Add proper shutdown of scheduled executor service in TimerService

Posted by se...@apache.org.
[FLINK-5093] Add proper shutdown of scheduled executor service in TimerService


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

Branch: refs/heads/master
Commit: 4d3a3eeb7b2cfc2ead2a388f745a05184c7878ce
Parents: eefcbbd
Author: Till Rohrmann <tr...@apache.org>
Authored: Mon Nov 28 15:25:57 2016 +0100
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Dec 23 20:54:26 2016 +0100

----------------------------------------------------------------------
 .../taskexecutor/TaskManagerServices.java       |  4 ++-
 .../TaskManagerServicesConfiguration.java       | 20 ++++++++++++--
 .../runtime/taskexecutor/slot/TimerService.java | 29 +++++++++++++++++---
 .../taskexecutor/TaskExecutorITCase.java        |  2 +-
 .../taskexecutor/slot/TimerServiceTest.java     |  2 +-
 5 files changed, 47 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/4d3a3eeb/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServices.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServices.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServices.java
index b57fafe..ae5a383 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServices.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServices.java
@@ -204,7 +204,9 @@ public class TaskManagerServices {
 			resourceProfiles.add(new ResourceProfile(1.0, 42L));
 		}
 
-		final TimerService<AllocationID> timerService = new TimerService<>(new ScheduledThreadPoolExecutor(1));
+		final TimerService<AllocationID> timerService = new TimerService<>(
+			new ScheduledThreadPoolExecutor(1),
+			taskManagerServicesConfiguration.getTimerServiceShutdownTimeout());
 
 		final TaskSlotTable taskSlotTable = new TaskSlotTable(resourceProfiles, timerService);
 

http://git-wip-us.apache.org/repos/asf/flink/blob/4d3a3eeb/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesConfiguration.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesConfiguration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesConfiguration.java
index 3190a93..2c76372 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesConfiguration.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesConfiguration.java
@@ -26,6 +26,7 @@ import org.apache.flink.core.memory.HeapMemorySegment;
 import org.apache.flink.core.memory.HybridMemorySegment;
 import org.apache.flink.core.memory.MemorySegmentFactory;
 import org.apache.flink.core.memory.MemoryType;
+import org.apache.flink.runtime.akka.AkkaUtils;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.io.network.netty.NettyConfig;
 import org.apache.flink.runtime.memory.MemoryManager;
@@ -37,6 +38,7 @@ import java.io.File;
 import java.net.InetAddress;
 import java.net.InetSocketAddress;
 
+import static org.apache.flink.util.Preconditions.checkArgument;
 import static org.apache.flink.util.Preconditions.checkNotNull;
 
 /**
@@ -63,6 +65,8 @@ public class TaskManagerServicesConfiguration {
 
 	private final MetricRegistryConfiguration metricRegistryConfiguration;
 
+	private final long timerServiceShutdownTimeout;
+
 	public TaskManagerServicesConfiguration(
 			InetAddress taskManagerAddress,
 			String[] tmpDirPaths,
@@ -72,7 +76,8 @@ public class TaskManagerServicesConfiguration {
 			long configuredMemory,
 			boolean preAllocateMemory,
 			float memoryFraction,
-			MetricRegistryConfiguration metricRegistryConfiguration) {
+			MetricRegistryConfiguration metricRegistryConfiguration,
+			long timerServiceShutdownTimeout) {
 
 		this.taskManagerAddress = checkNotNull(taskManagerAddress);
 		this.tmpDirPaths = checkNotNull(tmpDirPaths);
@@ -85,6 +90,10 @@ public class TaskManagerServicesConfiguration {
 		this.memoryFraction = memoryFraction;
 
 		this.metricRegistryConfiguration = checkNotNull(metricRegistryConfiguration);
+
+		checkArgument(timerServiceShutdownTimeout >= 0L, "The timer " +
+			"service shutdown timeout must be greater or equal to 0.");
+		this.timerServiceShutdownTimeout = timerServiceShutdownTimeout;
 	}
 
 	// --------------------------------------------------------------------------------------------
@@ -128,6 +137,10 @@ public class TaskManagerServicesConfiguration {
 		return metricRegistryConfiguration;
 	}
 
+	public long getTimerServiceShutdownTimeout() {
+		return timerServiceShutdownTimeout;
+	}
+
 	// --------------------------------------------------------------------------------------------
 	//  Parsing of Flink configuration
 	// --------------------------------------------------------------------------------------------
@@ -188,7 +201,7 @@ public class TaskManagerServicesConfiguration {
 
 		final MetricRegistryConfiguration metricRegistryConfiguration = MetricRegistryConfiguration.fromConfiguration(configuration);
 
-
+		long timerServiceShutdownTimeout = AkkaUtils.getTimeout(configuration).toMillis();
 
 		return new TaskManagerServicesConfiguration(
 			remoteAddress,
@@ -199,7 +212,8 @@ public class TaskManagerServicesConfiguration {
 			configuredMemory,
 			preAllocateMemory,
 			memoryFraction,
-			metricRegistryConfiguration);
+			metricRegistryConfiguration,
+			timerServiceShutdownTimeout);
 	}
 
 	// --------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/flink/blob/4d3a3eeb/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/slot/TimerService.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/slot/TimerService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/slot/TimerService.java
index 14c9ab1..8ec9a2e 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/slot/TimerService.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/slot/TimerService.java
@@ -19,6 +19,8 @@
 package org.apache.flink.runtime.taskexecutor.slot;
 
 import org.apache.flink.util.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.util.HashMap;
 import java.util.Map;
@@ -35,18 +37,28 @@ import java.util.concurrent.TimeUnit;
  */
 public class TimerService<K> {
 
+	private static final Logger LOG = LoggerFactory.getLogger(TimerService.class);
+
 	/** Executor service for the scheduled timeouts */
 	private final ScheduledExecutorService scheduledExecutorService;
 
+	/** Timeout for the shutdown of the service. */
+	private final long shutdownTimeout;
+
 	/** Map of currently active timeouts */
 	private final Map<K, Timeout<K>> timeouts;
 
 	/** Listener which is notified about occurring timeouts */
 	private TimeoutListener<K> timeoutListener;
 
-	public TimerService(final ScheduledExecutorService scheduledExecutorService) {
+	public TimerService(
+			final ScheduledExecutorService scheduledExecutorService,
+			final long shutdownTimeout) {
 		this.scheduledExecutorService = Preconditions.checkNotNull(scheduledExecutorService);
 
+		Preconditions.checkArgument(shutdownTimeout >= 0L, "The shut down timeout must be larger than or equal than 0.");
+		this.shutdownTimeout = shutdownTimeout;
+
 		this.timeouts = new HashMap<>(16);
 		this.timeoutListener = null;
 	}
@@ -65,6 +77,17 @@ public class TimerService<K> {
 		timeoutListener = null;
 
 		scheduledExecutorService.shutdown();
+
+		try {
+			if(!scheduledExecutorService.awaitTermination(shutdownTimeout, TimeUnit.MILLISECONDS)) {
+				LOG.debug("The scheduled executor service did not properly terminate. Shutting " +
+					"it down now.");
+				scheduledExecutorService.shutdownNow();
+			}
+		} catch (InterruptedException e) {
+			LOG.debug("Could not properly await the termination of the scheduled executor service.", e);
+			scheduledExecutorService.shutdownNow();
+		}
 	}
 
 	/**
@@ -103,9 +126,7 @@ public class TimerService<K> {
 	 */
 	protected void unregisterAllTimeouts() {
 		for (Timeout<K> timeout : timeouts.values()) {
-			if (timeout != null) {
-				timeout.cancel();
-			}
+			timeout.cancel();
 		}
 		timeouts.clear();
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/4d3a3eeb/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorITCase.java
index 050db44..36fd65b 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorITCase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorITCase.java
@@ -106,7 +106,7 @@ public class TaskExecutorITCase {
 		final TaskManagerMetricGroup taskManagerMetricGroup = mock(TaskManagerMetricGroup.class);
 		final BroadcastVariableManager broadcastVariableManager = mock(BroadcastVariableManager.class);
 		final FileCache fileCache = mock(FileCache.class);
-		final TaskSlotTable taskSlotTable = new TaskSlotTable(Arrays.asList(resourceProfile), new TimerService<AllocationID>(scheduledExecutorService));
+		final TaskSlotTable taskSlotTable = new TaskSlotTable(Arrays.asList(resourceProfile), new TimerService<AllocationID>(scheduledExecutorService, 100L));
 		final JobManagerTable jobManagerTable = new JobManagerTable();
 		final JobLeaderService jobLeaderService = new JobLeaderService(taskManagerLocation);
 

http://git-wip-us.apache.org/repos/asf/flink/blob/4d3a3eeb/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/slot/TimerServiceTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/slot/TimerServiceTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/slot/TimerServiceTest.java
index 9dd5f39..cad3624 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/slot/TimerServiceTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/slot/TimerServiceTest.java
@@ -48,7 +48,7 @@ public class TimerServiceTest {
 		ScheduledFuture scheduledFuture = mock(ScheduledFuture.class);
 		when(scheduledExecutorService.schedule(any(Runnable.class), anyLong(), any(TimeUnit.class)))
 			.thenReturn(scheduledFuture);
-		TimerService<AllocationID> timerService = new TimerService<>(scheduledExecutorService);
+		TimerService<AllocationID> timerService = new TimerService<>(scheduledExecutorService, 100L);
 		TimeoutListener<AllocationID> listener = mock(TimeoutListener.class);
 
 		timerService.start(listener);


[22/52] [abbrv] flink git commit: [FLINK-5170] [akka] Fix hostname usage in AkkaUtils.getConfig

Posted by se...@apache.org.
[FLINK-5170] [akka] Fix hostname usage in AkkaUtils.getConfig

This closes #2879.


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

Branch: refs/heads/master
Commit: 9ad9fa95ed0485a985250d2735c86154009fd515
Parents: b58c6dc
Author: shuai.xus <sh...@alibaba-inc.com>
Authored: Mon Nov 28 17:14:35 2016 +0800
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Dec 23 20:54:25 2016 +0100

----------------------------------------------------------------------
 .../src/main/scala/org/apache/flink/runtime/akka/AkkaUtils.scala   | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/9ad9fa95/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 b25e29e..4fe39b6 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
@@ -115,7 +115,7 @@ object AkkaUtils {
   }
 
   def getAkkaConfig(configuration: Configuration, hostname: String, port: Int): Config = {
-    getAkkaConfig(configuration, if (hostname == null) Some((hostname, port)) else None)
+    getAkkaConfig(configuration, if (hostname != null) Some((hostname, port)) else None)
   }
 
   /**


[45/52] [abbrv] flink git commit: [FLINK-5190] [runtime] fix ZooKeeperLeaderRetrievalService close the zk client when stopping bug

Posted by se...@apache.org.
[FLINK-5190] [runtime] fix ZooKeeperLeaderRetrievalService close the zk client when stopping bug


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

Branch: refs/heads/master
Commit: cbfb807d65b68b2b6157e1b1d42606123ea499ad
Parents: 95c6829
Author: shuai.xus <sh...@alibaba-inc.com>
Authored: Wed Nov 30 17:02:49 2016 +0800
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Dec 23 20:54:27 2016 +0100

----------------------------------------------------------------------
 .../runtime/leaderretrieval/ZooKeeperLeaderRetrievalService.java    | 1 -
 1 file changed, 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/cbfb807d/flink-runtime/src/main/java/org/apache/flink/runtime/leaderretrieval/ZooKeeperLeaderRetrievalService.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/leaderretrieval/ZooKeeperLeaderRetrievalService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/leaderretrieval/ZooKeeperLeaderRetrievalService.java
index 4587bad..f74fb1a 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/leaderretrieval/ZooKeeperLeaderRetrievalService.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/leaderretrieval/ZooKeeperLeaderRetrievalService.java
@@ -96,7 +96,6 @@ public class ZooKeeperLeaderRetrievalService implements LeaderRetrievalService,
 		client.getConnectionStateListenable().removeListener(connectionStateListener);
 
 		cache.close();
-		client.close();
 	}
 
 	@Override


[05/52] [abbrv] flink git commit: [hotfix] [cluster management] Remove scala dependencies from MiniCluster.java

Posted by se...@apache.org.
[hotfix] [cluster management] Remove scala dependencies from MiniCluster.java


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

Branch: refs/heads/master
Commit: aaf80a23e0cf47a4e330eb4f973f21a0524b78b6
Parents: af924b4
Author: Till Rohrmann <tr...@apache.org>
Authored: Mon Oct 17 14:41:52 2016 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Dec 23 20:54:23 2016 +0100

----------------------------------------------------------------------
 .../flink/runtime/minicluster/MiniCluster.java     | 12 ++++--------
 .../org/apache/flink/runtime/akka/AkkaUtils.scala  | 17 +++++++++++++++++
 .../runtime/minicluster/MiniClusterITCase.java     |  2 +-
 3 files changed, 22 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/aaf80a23/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java
index 1ee38e0..d85234d 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java
@@ -35,9 +35,6 @@ import org.apache.flink.runtime.rpc.RpcService;
 import org.apache.flink.runtime.rpc.akka.AkkaRpcService;
 import org.apache.flink.util.ExceptionUtils;
 
-import scala.Option;
-import scala.Tuple2;
-
 import javax.annotation.concurrent.GuardedBy;
 
 import static org.apache.flink.util.ExceptionUtils.firstOrSuppressed;
@@ -348,7 +345,7 @@ public class MiniCluster {
 	/**
 	 * Factory method to instantiate the RPC service.
 	 * 
-	 * @param config
+	 * @param configuration
 	 *            The configuration of the mini cluster
 	 * @param askTimeout
 	 *            The default RPC timeout for asynchronous "ask" requests.
@@ -360,17 +357,16 @@ public class MiniCluster {
 	 * @return The instantiated RPC service
 	 */
 	protected RpcService createRpcService(
-			Configuration config,
+			Configuration configuration,
 			Time askTimeout,
 			boolean remoteEnabled,
 			String bindAddress) {
 
 		ActorSystem actorSystem;
 		if (remoteEnabled) {
-			Tuple2<String, Object> remoteSettings = new Tuple2<String, Object>(bindAddress, 0);
-			actorSystem = AkkaUtils.createActorSystem(config, Option.apply(remoteSettings));
+			actorSystem = AkkaUtils.createActorSystem(configuration, bindAddress, 0);
 		} else {
-			actorSystem = AkkaUtils.createLocalActorSystem(config);
+			actorSystem = AkkaUtils.createLocalActorSystem(configuration);
 		}
 
 		return new AkkaRpcService(actorSystem, askTimeout);

http://git-wip-us.apache.org/repos/asf/flink/blob/aaf80a23/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 5210427..b25e29e 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
@@ -58,6 +58,23 @@ object AkkaUtils {
   }
 
   /**
+    * Creates an actor system bound to the given hostname and port.
+    *
+    * @param configuration instance containing the user provided configuration values
+    * @param hostname of the network interface to bind to
+    * @param port of to bind to
+    * @return created actor system
+    */
+  def createActorSystem(
+      configuration: Configuration,
+      hostname: String,
+      port: Int)
+    : ActorSystem = {
+
+    createActorSystem(configuration, Some((hostname, port)))
+  }
+
+  /**
    * Creates an actor system. If a listening address is specified, then the actor system will listen
    * on that address for messages from a remote actor system. If not, then a local actor system
    * will be instantiated.

http://git-wip-us.apache.org/repos/asf/flink/blob/aaf80a23/flink-runtime/src/test/java/org/apache/flink/runtime/minicluster/MiniClusterITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/minicluster/MiniClusterITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/minicluster/MiniClusterITCase.java
index dd43337..ef53547 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/minicluster/MiniClusterITCase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/minicluster/MiniClusterITCase.java
@@ -31,7 +31,7 @@ import org.junit.Test;
  */
 public class MiniClusterITCase extends TestLogger {
 
-//	@Test
+	@Test
 	public void runJobWithSingleRpcService() throws Exception {
 		MiniClusterConfiguration cfg = new MiniClusterConfiguration();
 


[18/52] [abbrv] flink git commit: [FLINK-4836] [cluster management] Start ResourceManager in MiniCluster

Posted by se...@apache.org.
[FLINK-4836] [cluster management] Start ResourceManager in MiniCluster


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

Branch: refs/heads/master
Commit: 6484f453cfeb4f8e04a231f41cc004b309bbdd37
Parents: 49a2968
Author: Stephan Ewen <se...@apache.org>
Authored: Mon Oct 17 17:02:33 2016 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Dec 23 20:54:24 2016 +0100

----------------------------------------------------------------------
 .../flink/runtime/minicluster/MiniCluster.java  | 126 +++++++++++++++----
 .../minicluster/MiniClusterJobDispatcher.java   |   4 +-
 2 files changed, 102 insertions(+), 28 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/6484f453/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java
index 1ffcd12..d63f9a7 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java
@@ -32,11 +32,18 @@ import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils;
 import org.apache.flink.runtime.jobgraph.JobGraph;
 import org.apache.flink.runtime.metrics.MetricRegistry;
 import org.apache.flink.runtime.metrics.MetricRegistryConfiguration;
+import org.apache.flink.runtime.resourcemanager.ResourceManager;
+import org.apache.flink.runtime.resourcemanager.StandaloneResourceManager;
+import org.apache.flink.runtime.resourcemanager.slotmanager.DefaultSlotManager;
+import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManagerFactory;
 import org.apache.flink.runtime.rpc.RpcService;
 import org.apache.flink.runtime.rpc.akka.AkkaRpcService;
 import org.apache.flink.runtime.taskexecutor.TaskManagerRunner;
 import org.apache.flink.util.ExceptionUtils;
 
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 import javax.annotation.concurrent.GuardedBy;
 
 import java.util.UUID;
@@ -48,13 +55,15 @@ import static org.apache.flink.util.Preconditions.checkState;
 
 public class MiniCluster {
 
+	private static final Logger LOG = LoggerFactory.getLogger(MiniCluster.class);
+
 	/** The lock to guard startup / shutdown / manipulation methods */
 	private final Object lock = new Object();
 
 	/** The configuration for this mini cluster */
 	private final MiniClusterConfiguration config;
 
-	@GuardedBy("lock")
+	@GuardedBy("lock") 
 	private MetricRegistry metricRegistry;
 
 	@GuardedBy("lock")
@@ -73,6 +82,9 @@ public class MiniCluster {
 	private HighAvailabilityServices haServices;
 
 	@GuardedBy("lock")
+	private ResourceManager<?>[] resourceManagers;
+
+	@GuardedBy("lock")
 	private TaskManagerRunner[] taskManagerRunners;
 
 	@GuardedBy("lock")
@@ -98,6 +110,7 @@ public class MiniCluster {
 	}
 
 	/**
+	 * Creates a new Flink mini cluster based on the given configuration.
 	 * 
 	 * @param config The configuration for the mini cluster
 	 */
@@ -149,6 +162,9 @@ public class MiniCluster {
 		synchronized (lock) {
 			checkState(!running, "FlinkMiniCluster is already running");
 
+			LOG.info("Starting Flink Mini Cluster");
+			LOG.debug("Using configuration {}", config);
+
 			final Configuration configuration = new UnmodifiableConfiguration(config.getConfiguration());
 			final Time rpcTimeout = config.getRpcTimeout();
 			final int numJobManagers = config.getNumJobManagers();
@@ -210,13 +226,21 @@ public class MiniCluster {
 				}
 
 				// create the high-availability services
+				LOG.info("Starting high-availability services");
 				haServices = HighAvailabilityServicesUtils.createAvailableOrEmbeddedServices(configuration);
 
-				// bring up the task managers for the mini cluster
+				// bring up the ResourceManager(s)
+				LOG.info("Starting {} ResourceManger(s)", numResourceManagers);
+				resourceManagers = startResourceManagers(
+						configuration, haServices, metricRegistry, numResourceManagers, resourceManagerRpcServices);
+
+				// bring up the TaskManager(s) for the mini cluster
+				LOG.info("Starting {} TaskManger(s)", numTaskManagers);
 				taskManagerRunners = startTaskManagers(
 						configuration, haServices, metricRegistry, numTaskManagers, taskManagerRpcServices);
 
 				// bring up the dispatcher that launches JobManagers when jobs submitted
+				LOG.info("Starting job dispatcher for {} JobManger(s)", numJobManagers);
 				jobDispatcher = new MiniClusterJobDispatcher(
 						configuration, haServices, metricRegistry, numJobManagers, jobManagerRpcServices);
 			}
@@ -232,6 +256,8 @@ public class MiniCluster {
 
 			// now officially mark this as running
 			running = true;
+
+			LOG.info("Flink Mini Cluster started successfully");
 		}
 	}
 
@@ -247,11 +273,13 @@ public class MiniCluster {
 	public void shutdown() throws Exception {
 		synchronized (lock) {
 			if (running) {
+				LOG.info("Shutting down Flink Mini Cluster");
 				try {
 					shutdownInternally();
 				} finally {
 					running = false;
 				}
+				LOG.info("Flink Mini Cluster is shut down");
 			}
 		}
 	}
@@ -270,11 +298,34 @@ public class MiniCluster {
 			try {
 				jobDispatcher.shutdown();
 			} catch (Exception e) {
-				exception = firstOrSuppressed(e, exception);
+				exception = e;
 			}
 			jobDispatcher = null;
 		}
 
+		if (resourceManagers != null) {
+			for (ResourceManager<?> rm : resourceManagers) {
+				if (rm != null) {
+					try {
+						rm.shutDown();
+					} catch (Throwable t) {
+						exception = firstOrSuppressed(t, exception);
+					}
+				}
+			}
+			resourceManagers = null;
+		}
+
+		// shut down the RpcServices
+		exception = shutDownRpc(commonRpcService, exception);
+		exception = shutDownRpcs(jobManagerRpcServices, exception);
+		exception = shutDownRpcs(taskManagerRpcServices, exception);
+		exception = shutDownRpcs(resourceManagerRpcServices, exception);
+		commonRpcService = null;
+		jobManagerRpcServices = null;
+		taskManagerRpcServices = null;
+		resourceManagerRpcServices = null;
+
 		// shut down high-availability services
 		if (haServices != null) {
 			try {
@@ -285,24 +336,6 @@ public class MiniCluster {
 			haServices = null;
 		}
 
-		// shut down the RpcServices
-		if (commonRpcService != null) {
-			exception = shutDownRpc(commonRpcService, exception);
-			commonRpcService = null;
-		}
-		if (jobManagerRpcServices != null) {
-			for (RpcService service : jobManagerRpcServices) {
-				exception = shutDownRpc(service, exception);
-			}
-			jobManagerRpcServices = null;
-		}
-		if (taskManagerRpcServices != null) {
-			for (RpcService service : taskManagerRpcServices) {
-				exception = shutDownRpc(service, exception);
-			}
-			taskManagerRpcServices = null;
-		}
-
 		// metrics shutdown
 		if (metricRegistry != null) {
 			metricRegistry.shutdown();
@@ -402,6 +435,28 @@ public class MiniCluster {
 		return new AkkaRpcService(actorSystem, askTimeout);
 	}
 
+	protected ResourceManager<?>[] startResourceManagers(
+			Configuration configuration,
+			HighAvailabilityServices haServices,
+			MetricRegistry metricRegistry,
+			int numResourceManagers,
+			RpcService[] resourceManagerRpcServices) throws Exception {
+
+		final StandaloneResourceManager[] resourceManagers = new StandaloneResourceManager[numResourceManagers];
+		final SlotManagerFactory slotManagerFactory = new DefaultSlotManager.Factory(); 
+
+		for (int i = 0; i < numResourceManagers; i++) {
+			resourceManagers[i] = new StandaloneResourceManager(
+					resourceManagerRpcServices[i],
+					haServices,
+					slotManagerFactory);
+
+			resourceManagers[i].start();
+		}
+
+		return resourceManagers;
+	}
+
 	protected TaskManagerRunner[] startTaskManagers(
 			Configuration configuration,
 			HighAvailabilityServices haServices,
@@ -429,15 +484,34 @@ public class MiniCluster {
 	// ------------------------------------------------------------------------
 
 	private static Throwable shutDownRpc(RpcService rpcService, Throwable priorException) {
-		try {
-			if (rpcService != null) {
+		if (rpcService != null) {
+			try {
 				rpcService.stopService();
 			}
-			return priorException;
+			catch (Throwable t) {
+				return firstOrSuppressed(t, priorException);
+			}
 		}
-		catch (Throwable t) {
-			return firstOrSuppressed(t, priorException);
+
+		return priorException;
+	}
+
+	private static Throwable shutDownRpcs(RpcService[] rpcServices, Throwable priorException) {
+		if (rpcServices != null) {
+			Throwable exception = priorException;
+
+			for (RpcService service : rpcServices) {
+				try {
+					if (service != null) {
+						service.stopService();
+					}
+				}
+				catch (Throwable t) {
+					exception = firstOrSuppressed(t, exception);
+				}
+			}
 		}
+		return priorException;
 	}
 
 	private static MiniClusterConfiguration createConfig(Configuration cfg, boolean singleRpcService) {

http://git-wip-us.apache.org/repos/asf/flink/blob/6484f453/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniClusterJobDispatcher.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniClusterJobDispatcher.java b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniClusterJobDispatcher.java
index d0df293..8ac8eba 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniClusterJobDispatcher.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniClusterJobDispatcher.java
@@ -176,7 +176,7 @@ public class MiniClusterJobDispatcher {
 	public void runDetached(JobGraph job) throws JobExecutionException {
 		checkNotNull(job);
 
-		LOG.info("Received job for detached execution {} ({})", job.getName(), job.getJobID());
+		LOG.info("Received job for detached execution: {} ({})", job.getName(), job.getJobID());
 
 		synchronized (lock) {
 			checkState(!shutdown, "mini cluster is shut down");
@@ -201,7 +201,7 @@ public class MiniClusterJobDispatcher {
 	public JobExecutionResult runJobBlocking(JobGraph job) throws JobExecutionException, InterruptedException {
 		checkNotNull(job);
 		
-		LOG.info("Received job for blocking execution {} ({})", job.getName(), job.getJobID());
+		LOG.info("Received job for blocking execution: {} ({})", job.getName(), job.getJobID());
 		final BlockingJobSync sync = new BlockingJobSync(job.getJobID(), numJobManagers);
 
 		synchronized (lock) {