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

[01/50] [abbrv] flink git commit: [FLINK-4347][cluster management] Implement SlotManager core [Forced Update!]

Repository: flink
Updated Branches:
  refs/heads/flip-6 48c936eed -> 7aca811df (forced update)


[FLINK-4347][cluster management] Implement SlotManager core

This closes #2388


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

Branch: refs/heads/flip-6
Commit: 6e22c64dc6f5c2c252df46a5a32adf20da7ef162
Parents: ce6114d
Author: Kurt Young <yk...@gmail.com>
Authored: Thu Aug 18 15:48:30 2016 +0800
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Oct 14 15:14:39 2016 +0200

----------------------------------------------------------------------
 .../runtime/clusterframework/SlotManager.java   | 525 ++++++++++++++++++
 .../clusterframework/types/ResourceID.java      |   4 +-
 .../clusterframework/types/ResourceSlot.java    |  66 +++
 .../runtime/clusterframework/types/SlotID.java  |  14 +-
 .../rpc/resourcemanager/SlotRequest.java        |  51 +-
 .../runtime/rpc/taskexecutor/SlotReport.java    |  56 ++
 .../runtime/rpc/taskexecutor/SlotStatus.java    | 129 +++++
 .../clusterframework/SlotManagerTest.java       | 540 +++++++++++++++++++
 8 files changed, 1377 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/6e22c64d/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/SlotManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/SlotManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/SlotManager.java
new file mode 100644
index 0000000..cc140a1
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/SlotManager.java
@@ -0,0 +1,525 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.clusterframework;
+
+import org.apache.flink.annotation.VisibleForTesting;
+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.ResourceSlot;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.rpc.resourcemanager.ResourceManagerGateway;
+import org.apache.flink.runtime.rpc.resourcemanager.SlotRequest;
+import org.apache.flink.runtime.rpc.taskexecutor.SlotReport;
+import org.apache.flink.runtime.rpc.taskexecutor.SlotStatus;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.Map;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * SlotManager is responsible for receiving slot requests and do slot allocations. It allows to request
+ * slots from registered TaskManagers and issues container allocation requests in case of there are not
+ * enough available slots. Besides, it should sync its slot allocation with TaskManager's heartbeat.
+ * <p>
+ * The main operation principle of SlotManager is:
+ * <ul>
+ * <li>1. All slot allocation status should be synced with TaskManager, which is the ground truth.</li>
+ * <li>2. All slots that have registered must be tracked, either by free pool or allocated pool.</li>
+ * <li>3. All slot requests will be handled by best efforts, there is no guarantee that one request will be
+ * fulfilled in time or correctly allocated. Conflicts or timeout or some special error will happen, it should
+ * be handled outside SlotManager. SlotManager will make each decision based on the information it currently
+ * holds.</li>
+ * </ul>
+ * <b>IMPORTANT:</b> This class is <b>Not Thread-safe</b>.
+ */
+public abstract class SlotManager {
+
+	private static final Logger LOG = LoggerFactory.getLogger(SlotManager.class);
+
+	/** Gateway to communicate with ResourceManager */
+	private final ResourceManagerGateway resourceManagerGateway;
+
+	/** All registered slots, including free and allocated slots */
+	private final Map<ResourceID, Map<SlotID, ResourceSlot>> registeredSlots;
+
+	/** All pending slot requests, waiting available slots to fulfil */
+	private final Map<AllocationID, SlotRequest> pendingSlotRequests;
+
+	/** All free slots that can be used to be allocated */
+	private final Map<SlotID, ResourceSlot> freeSlots;
+
+	/** All allocations, we can lookup allocations either by SlotID or AllocationID */
+	private final AllocationMap allocationMap;
+
+	public SlotManager(ResourceManagerGateway resourceManagerGateway) {
+		this.resourceManagerGateway = checkNotNull(resourceManagerGateway);
+		this.registeredSlots = new HashMap<>(16);
+		this.pendingSlotRequests = new LinkedHashMap<>(16);
+		this.freeSlots = new HashMap<>(16);
+		this.allocationMap = new AllocationMap();
+	}
+
+	// ------------------------------------------------------------------------
+	//  slot managements
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Request a slot with requirements, we may either fulfill the request or pending it. Trigger container
+	 * allocation if we don't have enough resource. If we have free slot which can match the request, record
+	 * this allocation and forward the request to TaskManager through ResourceManager (we want this done by
+	 * RPC's main thread to avoid race condition).
+	 *
+	 * @param request The detailed request of the slot
+	 */
+	public void requestSlot(final SlotRequest request) {
+		if (isRequestDuplicated(request)) {
+			LOG.warn("Duplicated slot request, AllocationID:{}", request.getAllocationId());
+			return;
+		}
+
+		// try to fulfil the request with current free slots
+		ResourceSlot slot = chooseSlotToUse(request, freeSlots);
+		if (slot != null) {
+			LOG.info("Assigning SlotID({}) to AllocationID({}), JobID:{}", slot.getSlotId(),
+				request.getAllocationId(), request.getJobId());
+
+			// record this allocation in bookkeeping
+			allocationMap.addAllocation(slot.getSlotId(), request.getAllocationId());
+
+			// remove selected slot from free pool
+			freeSlots.remove(slot.getSlotId());
+
+			// TODO: send slot request to TaskManager
+		} else {
+			LOG.info("Cannot fulfil slot request, try to allocate a new container for it, " +
+				"AllocationID:{}, JobID:{}", request.getAllocationId(), request.getJobId());
+			allocateContainer(request.getResourceProfile());
+			pendingSlotRequests.put(request.getAllocationId(), request);
+		}
+	}
+
+	/**
+	 * Sync slot status with TaskManager's SlotReport.
+	 */
+	public void updateSlotStatus(final SlotReport slotReport) {
+		for (SlotStatus slotStatus : slotReport.getSlotsStatus()) {
+			updateSlotStatus(slotStatus);
+		}
+	}
+
+	/**
+	 * The slot request to TaskManager may be either failed by rpc communication (timeout, network error, etc.)
+	 * or really rejected by TaskManager. We shall retry this request by:
+	 * <ul>
+	 * <li>1. verify and clear all the previous allocate information for this request
+	 * <li>2. try to request slot again
+	 * </ul>
+	 * <p>
+	 * This may cause some duplicate allocation, e.g. the slot request to TaskManager is successful but the response
+	 * is lost somehow, so we may request a slot in another TaskManager, this causes two slots assigned to one request,
+	 * but it can be taken care of by rejecting registration at JobManager.
+	 *
+	 * @param originalRequest The original slot request
+	 * @param slotId          The target SlotID
+	 */
+	public void handleSlotRequestFailedAtTaskManager(final SlotRequest originalRequest, final SlotID slotId) {
+		final AllocationID originalAllocationId = originalRequest.getAllocationId();
+		LOG.info("Slot request failed at TaskManager, SlotID:{}, AllocationID:{}, JobID:{}",
+			slotId, originalAllocationId, originalRequest.getJobId());
+
+		// verify the allocation info before we do anything
+		if (freeSlots.containsKey(slotId)) {
+			// this slot is currently empty, no need to de-allocate it from our allocations
+			LOG.info("Original slot is somehow empty, retrying this request");
+
+			// before retry, we should double check whether this request was allocated by some other ways
+			if (!allocationMap.isAllocated(originalAllocationId)) {
+				requestSlot(originalRequest);
+			} else {
+				LOG.info("The failed request has somehow been allocated, SlotID:{}",
+					allocationMap.getSlotID(originalAllocationId));
+			}
+		} else if (allocationMap.isAllocated(slotId)) {
+			final AllocationID currentAllocationId = allocationMap.getAllocationID(slotId);
+
+			// check whether we have an agreement on whom this slot belongs to
+			if (originalAllocationId.equals(currentAllocationId)) {
+				LOG.info("De-allocate this request and retry");
+				allocationMap.removeAllocation(currentAllocationId);
+
+				// put this slot back to free pool
+				ResourceSlot slot = checkNotNull(getRegisteredSlot(slotId));
+				freeSlots.put(slotId, slot);
+
+				// retry the request
+				requestSlot(originalRequest);
+			} else {
+				// the slot is taken by someone else, no need to de-allocate it from our allocations
+				LOG.info("Original slot is taken by someone else, current AllocationID:{}", currentAllocationId);
+
+				// before retry, we should double check whether this request was allocated by some other ways
+				if (!allocationMap.isAllocated(originalAllocationId)) {
+					requestSlot(originalRequest);
+				} else {
+					LOG.info("The failed request is somehow been allocated, SlotID:{}",
+						allocationMap.getSlotID(originalAllocationId));
+				}
+			}
+		} else {
+			LOG.error("BUG! {} is neither in free pool nor in allocated pool", slotId);
+		}
+	}
+
+	/**
+	 * Callback for TaskManager failures. In case that a TaskManager fails, we have to clean up all its slots.
+	 *
+	 * @param resourceId The ResourceID of the TaskManager
+	 */
+	public void notifyTaskManagerFailure(final ResourceID resourceId) {
+		LOG.info("Resource:{} been notified failure", resourceId);
+		final Map<SlotID, ResourceSlot> slotIdsToRemove = registeredSlots.remove(resourceId);
+		if (slotIdsToRemove != null) {
+			for (SlotID slotId : slotIdsToRemove.keySet()) {
+				LOG.info("Removing Slot:{} upon resource failure", slotId);
+				if (freeSlots.containsKey(slotId)) {
+					freeSlots.remove(slotId);
+				} else if (allocationMap.isAllocated(slotId)) {
+					allocationMap.removeAllocation(slotId);
+				} else {
+					LOG.error("BUG! {} is neither in free pool nor in allocated pool", slotId);
+				}
+			}
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  internal behaviors
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Update slot status based on TaskManager's report. There are mainly two situations when we receive the report:
+	 * <ul>
+	 * <li>1. The slot is newly registered.</li>
+	 * <li>2. The slot has registered, it contains its current status.</li>
+	 * </ul>
+	 * <p>
+	 * Regarding 1: It's fairly simple, we just record this slot's status, and trigger schedule if slot is empty.
+	 * <p>
+	 * Regarding 2: It will cause some weird situation since we may have some time-gap on how the slot's status really
+	 * is. We may have some updates on the slot's allocation, but it doesn't reflected by TaskManager's heartbeat yet,
+	 * and we may make some wrong decision if we cannot guarantee we have the exact status about all the slots. So
+	 * the principle here is: We always trust TaskManager's heartbeat, we will correct our information based on that
+	 * and take next action based on the diff between our information and heartbeat status.
+	 *
+	 * @param reportedStatus Reported slot status
+	 */
+	void updateSlotStatus(final SlotStatus reportedStatus) {
+		final SlotID slotId = reportedStatus.getSlotID();
+		final ResourceSlot slot = new ResourceSlot(slotId, reportedStatus.getProfiler());
+
+		if (registerNewSlot(slot)) {
+			// we have a newly registered slot
+			LOG.info("New slot appeared, SlotID:{}, AllocationID:{}", slotId, reportedStatus.getAllocationID());
+
+			if (reportedStatus.getAllocationID() != null) {
+				// slot in use, record this in bookkeeping
+				allocationMap.addAllocation(slotId, reportedStatus.getAllocationID());
+			} else {
+				handleFreeSlot(new ResourceSlot(slotId, reportedStatus.getProfiler()));
+			}
+		} else {
+			// slot exists, update current information
+			if (reportedStatus.getAllocationID() != null) {
+				// slot is reported in use
+				final AllocationID reportedAllocationId = reportedStatus.getAllocationID();
+
+				// check whether we also thought this slot is in use
+				if (allocationMap.isAllocated(slotId)) {
+					// we also think that slot is in use, check whether the AllocationID matches
+					final AllocationID currentAllocationId = allocationMap.getAllocationID(slotId);
+
+					if (!reportedAllocationId.equals(currentAllocationId)) {
+						LOG.info("Slot allocation info mismatch! SlotID:{}, current:{}, reported:{}",
+							slotId, currentAllocationId, reportedAllocationId);
+
+						// seems we have a disagreement about the slot assignments, need to correct it
+						allocationMap.removeAllocation(slotId);
+						allocationMap.addAllocation(slotId, reportedAllocationId);
+					}
+				} else {
+					LOG.info("Slot allocation info mismatch! SlotID:{}, current:null, reported:{}",
+						slotId, reportedAllocationId);
+
+					// we thought the slot is free, should correct this information
+					allocationMap.addAllocation(slotId, reportedStatus.getAllocationID());
+
+					// remove this slot from free slots pool
+					freeSlots.remove(slotId);
+				}
+			} else {
+				// slot is reported empty
+
+				// check whether we also thought this slot is empty
+				if (allocationMap.isAllocated(slotId)) {
+					LOG.info("Slot allocation info mismatch! SlotID:{}, current:{}, reported:null",
+						slotId, allocationMap.getAllocationID(slotId));
+
+					// we thought the slot is in use, correct it
+					allocationMap.removeAllocation(slotId);
+
+					// we have a free slot!
+					handleFreeSlot(new ResourceSlot(slotId, reportedStatus.getProfiler()));
+				}
+			}
+		}
+	}
+
+	/**
+	 * When we have a free slot, try to fulfill the pending request first. If any request can be fulfilled,
+	 * record this allocation in bookkeeping and send slot request to TaskManager, else we just add this slot
+	 * to the free pool.
+	 *
+	 * @param freeSlot The free slot
+	 */
+	private void handleFreeSlot(final ResourceSlot freeSlot) {
+		SlotRequest chosenRequest = chooseRequestToFulfill(freeSlot, pendingSlotRequests);
+
+		if (chosenRequest != null) {
+			pendingSlotRequests.remove(chosenRequest.getAllocationId());
+
+			LOG.info("Assigning SlotID({}) to AllocationID({}), JobID:{}", freeSlot.getSlotId(),
+				chosenRequest.getAllocationId(), chosenRequest.getJobId());
+			allocationMap.addAllocation(freeSlot.getSlotId(), chosenRequest.getAllocationId());
+
+			// TODO: send slot request to TaskManager
+		} else {
+			freeSlots.put(freeSlot.getSlotId(), freeSlot);
+		}
+	}
+
+	/**
+	 * Check whether the request is duplicated. We use AllocationID to identify slot request, for each
+	 * formerly received slot request, it is either in pending list or already been allocated.
+	 *
+	 * @param request The slot request
+	 * @return <tt>true</tt> if the request is duplicated
+	 */
+	private boolean isRequestDuplicated(final SlotRequest request) {
+		final AllocationID allocationId = request.getAllocationId();
+		return pendingSlotRequests.containsKey(allocationId)
+			|| allocationMap.isAllocated(allocationId);
+	}
+
+	/**
+	 * Try to register slot, and tell if this slot is newly registered.
+	 *
+	 * @param slot The ResourceSlot which will be checked and registered
+	 * @return <tt>true</tt> if we meet a new slot
+	 */
+	private boolean registerNewSlot(final ResourceSlot slot) {
+		final SlotID slotId = slot.getSlotId();
+		final ResourceID resourceId = slotId.getResourceID();
+		if (!registeredSlots.containsKey(resourceId)) {
+			registeredSlots.put(resourceId, new HashMap<SlotID, ResourceSlot>());
+		}
+		return registeredSlots.get(resourceId).put(slotId, slot) == null;
+	}
+
+	private ResourceSlot getRegisteredSlot(final SlotID slotId) {
+		final ResourceID resourceId = slotId.getResourceID();
+		if (!registeredSlots.containsKey(resourceId)) {
+			return null;
+		}
+		return registeredSlots.get(resourceId).get(slotId);
+	}
+
+	// ------------------------------------------------------------------------
+	//  Framework specific behavior
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Choose a slot to use among all free slots, the behavior is framework specified.
+	 *
+	 * @param request   The slot request
+	 * @param freeSlots All slots which can be used
+	 * @return The slot we choose to use, <tt>null</tt> if we did not find a match
+	 */
+	protected abstract ResourceSlot chooseSlotToUse(final SlotRequest request,
+		final Map<SlotID, ResourceSlot> freeSlots);
+
+	/**
+	 * Choose a pending request to fulfill when we have a free slot, the behavior is framework specified.
+	 *
+	 * @param offeredSlot     The free slot
+	 * @param pendingRequests All the pending slot requests
+	 * @return The chosen SlotRequest, <tt>null</tt> if we did not find a match
+	 */
+	protected abstract SlotRequest chooseRequestToFulfill(final ResourceSlot offeredSlot,
+		final Map<AllocationID, SlotRequest> pendingRequests);
+
+	/**
+	 * The framework specific code for allocating a container for specified resource profile.
+	 *
+	 * @param resourceProfile The resource profile
+	 */
+	protected abstract void allocateContainer(final ResourceProfile resourceProfile);
+
+
+	// ------------------------------------------------------------------------
+	//  Helper classes
+	// ------------------------------------------------------------------------
+
+	/**
+	 * We maintain all the allocations with SlotID and AllocationID. We are able to get or remove the allocation info
+	 * either by SlotID or AllocationID.
+	 */
+	private static class AllocationMap {
+
+		/** All allocated slots (by SlotID) */
+		private final Map<SlotID, AllocationID> allocatedSlots;
+
+		/** All allocated slots (by AllocationID), it'a a inverse view of allocatedSlots */
+		private final Map<AllocationID, SlotID> allocatedSlotsByAllocationId;
+
+		AllocationMap() {
+			this.allocatedSlots = new HashMap<>(16);
+			this.allocatedSlotsByAllocationId = new HashMap<>(16);
+		}
+
+		/**
+		 * Add a allocation
+		 *
+		 * @param slotId       The slot id
+		 * @param allocationId The allocation id
+		 */
+		void addAllocation(final SlotID slotId, final AllocationID allocationId) {
+			allocatedSlots.put(slotId, allocationId);
+			allocatedSlotsByAllocationId.put(allocationId, slotId);
+		}
+
+		/**
+		 * De-allocation with slot id
+		 *
+		 * @param slotId The slot id
+		 */
+		void removeAllocation(final SlotID slotId) {
+			if (allocatedSlots.containsKey(slotId)) {
+				final AllocationID allocationId = allocatedSlots.get(slotId);
+				allocatedSlots.remove(slotId);
+				allocatedSlotsByAllocationId.remove(allocationId);
+			}
+		}
+
+		/**
+		 * De-allocation with allocation id
+		 *
+		 * @param allocationId The allocation id
+		 */
+		void removeAllocation(final AllocationID allocationId) {
+			if (allocatedSlotsByAllocationId.containsKey(allocationId)) {
+				SlotID slotId = allocatedSlotsByAllocationId.get(allocationId);
+				allocatedSlotsByAllocationId.remove(allocationId);
+				allocatedSlots.remove(slotId);
+			}
+		}
+
+		/**
+		 * Check whether allocation exists by slot id
+		 *
+		 * @param slotId The slot id
+		 * @return true if the allocation exists
+		 */
+		boolean isAllocated(final SlotID slotId) {
+			return allocatedSlots.containsKey(slotId);
+		}
+
+		/**
+		 * Check whether allocation exists by allocation id
+		 *
+		 * @param allocationId The allocation id
+		 * @return true if the allocation exists
+		 */
+		boolean isAllocated(final AllocationID allocationId) {
+			return allocatedSlotsByAllocationId.containsKey(allocationId);
+		}
+
+		AllocationID getAllocationID(final SlotID slotId) {
+			return allocatedSlots.get(slotId);
+		}
+
+		SlotID getSlotID(final AllocationID allocationId) {
+			return allocatedSlotsByAllocationId.get(allocationId);
+		}
+
+		public int size() {
+			return allocatedSlots.size();
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  Testing utilities
+	// ------------------------------------------------------------------------
+
+	@VisibleForTesting
+	boolean isAllocated(final SlotID slotId) {
+		return allocationMap.isAllocated(slotId);
+	}
+
+	@VisibleForTesting
+	boolean isAllocated(final AllocationID allocationId) {
+		return allocationMap.isAllocated(allocationId);
+	}
+
+	/**
+	 * Add free slots directly to the free pool, this will not trigger pending requests allocation
+	 *
+	 * @param slot The resource slot
+	 */
+	@VisibleForTesting
+	void addFreeSlot(final ResourceSlot slot) {
+		final ResourceID resourceId = slot.getResourceID();
+		final SlotID slotId = slot.getSlotId();
+
+		if (!registeredSlots.containsKey(resourceId)) {
+			registeredSlots.put(resourceId, new HashMap<SlotID, ResourceSlot>());
+		}
+		registeredSlots.get(resourceId).put(slot.getSlotId(), slot);
+		freeSlots.put(slotId, slot);
+	}
+
+	@VisibleForTesting
+	int getAllocatedSlotCount() {
+		return allocationMap.size();
+	}
+
+	@VisibleForTesting
+	int getFreeSlotCount() {
+		return freeSlots.size();
+	}
+
+	@VisibleForTesting
+	int getPendingRequestCount() {
+		return pendingSlotRequests.size();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/6e22c64d/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceID.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceID.java b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceID.java
index 8cf9ccb..6b8a037 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceID.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceID.java
@@ -63,9 +63,7 @@ public final class ResourceID implements ResourceIDRetrievable, Serializable {
 
 	@Override
 	public String toString() {
-		return "ResourceID{" +
-			"resourceId='" + resourceId + '\'' +
-			'}';
+		return resourceId;
 	}
 
 	/**

http://git-wip-us.apache.org/repos/asf/flink/blob/6e22c64d/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceSlot.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceSlot.java b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceSlot.java
new file mode 100644
index 0000000..8a6db5f
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceSlot.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.clusterframework.types;
+
+import java.io.Serializable;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * A ResourceSlot represents a slot located in TaskManager from ResourceManager's view. It has a unique
+ * identification and resource profile which we can compare to the resource request.
+ */
+public class ResourceSlot implements ResourceIDRetrievable, Serializable {
+
+	private static final long serialVersionUID = -5853720153136840674L;
+
+	/** The unique identification of this slot */
+	private final SlotID slotId;
+
+	/** The resource profile of this slot */
+	private final ResourceProfile resourceProfile;
+
+	public ResourceSlot(SlotID slotId, ResourceProfile resourceProfile) {
+		this.slotId = checkNotNull(slotId);
+		this.resourceProfile = checkNotNull(resourceProfile);
+	}
+
+	@Override
+	public ResourceID getResourceID() {
+		return slotId.getResourceID();
+	}
+
+	public SlotID getSlotId() {
+		return slotId;
+	}
+
+	public ResourceProfile getResourceProfile() {
+		return resourceProfile;
+	}
+
+	/**
+	 * Check whether required resource profile can be matched by this slot.
+	 *
+	 * @param required The required resource profile
+	 * @return true if requirement can be matched
+	 */
+	public boolean isMatchingRequirement(ResourceProfile required) {
+		return resourceProfile.isMatching(required);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/6e22c64d/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/SlotID.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/SlotID.java b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/SlotID.java
index d1b072d..e831a5d 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/SlotID.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/SlotID.java
@@ -75,9 +75,15 @@ public class SlotID implements ResourceIDRetrievable, Serializable {
 
 	@Override
 	public String toString() {
-		return "SlotID{" +
-			"resourceId=" + resourceId +
-			", slotId=" + slotId +
-			'}';
+		return resourceId + "_" + slotId;
+	}
+
+	/**
+	 * Generate a random slot id.
+	 *
+	 * @return A random slot id.
+	 */
+	public static SlotID generate() {
+		return new SlotID(ResourceID.generate(), 0);
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/6e22c64d/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/SlotRequest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/SlotRequest.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/SlotRequest.java
index d8fe268..74c7c39 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/SlotRequest.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/SlotRequest.java
@@ -18,8 +18,57 @@
 
 package org.apache.flink.runtime.rpc.resourcemanager;
 
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+
 import java.io.Serializable;
 
-public class SlotRequest implements Serializable{
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * This describes the requirement of the slot, mainly used by JobManager requesting slot from ResourceManager.
+ */
+public class SlotRequest implements Serializable {
+
 	private static final long serialVersionUID = -6586877187990445986L;
+
+	/** The JobID of the slot requested for */
+	private final JobID jobId;
+
+	/** The unique identification of this request */
+	private final AllocationID allocationId;
+
+	/** The resource profile of the required slot */
+	private final ResourceProfile resourceProfile;
+
+	public SlotRequest(JobID jobId, AllocationID allocationId, ResourceProfile resourceProfile) {
+		this.jobId = checkNotNull(jobId);
+		this.allocationId = checkNotNull(allocationId);
+		this.resourceProfile = checkNotNull(resourceProfile);
+	}
+
+	/**
+	 * Get the JobID of the slot requested for.
+	 * @return The job id
+	 */
+	public JobID getJobId() {
+		return jobId;
+	}
+
+	/**
+	 * Get the unique identification of this request
+	 * @return the allocation id
+	 */
+	public AllocationID getAllocationId() {
+		return allocationId;
+	}
+
+	/**
+	 * Get the resource profile of the desired slot
+	 * @return The resource profile
+	 */
+	public ResourceProfile getResourceProfile() {
+		return resourceProfile;
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/6e22c64d/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/SlotReport.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/SlotReport.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/SlotReport.java
new file mode 100644
index 0000000..c372ecb
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/SlotReport.java
@@ -0,0 +1,56 @@
+/*
+ * 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.taskexecutor;
+
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+
+import java.io.Serializable;
+import java.util.List;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * A report about the current status of all slots of the TaskExecutor, describing
+ * which slots are available and allocated, and what jobs (JobManagers) the allocated slots
+ * have been allocated to.
+ */
+public class SlotReport implements Serializable {
+
+	private static final long serialVersionUID = -3150175198722481689L;
+
+	/** The slots status of the TaskManager */
+	private final List<SlotStatus> slotsStatus;
+
+	/** The resource id which identifies the TaskManager */
+	private final ResourceID resourceID;
+
+	public SlotReport(final List<SlotStatus> slotsStatus, final ResourceID resourceID) {
+		this.slotsStatus = checkNotNull(slotsStatus);
+		this.resourceID = checkNotNull(resourceID);
+	}
+
+	public List<SlotStatus> getSlotsStatus() {
+		return slotsStatus;
+	}
+
+	public ResourceID getResourceID() {
+		return resourceID;
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/6e22c64d/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/SlotStatus.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/SlotStatus.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/SlotStatus.java
new file mode 100644
index 0000000..e8e2084
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/SlotStatus.java
@@ -0,0 +1,129 @@
+/*
+ * 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.taskexecutor;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+
+import java.io.Serializable;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * This describes the slot current status which located in TaskManager.
+ */
+public class SlotStatus implements Serializable {
+
+	private static final long serialVersionUID = 5099191707339664493L;
+
+	/** slotID to identify a slot */
+	private final SlotID slotID;
+
+	/** the resource profile of the slot */
+	private final ResourceProfile profiler;
+
+	/** if the slot is allocated, allocationId identify its allocation; else, allocationId is null */
+	private final AllocationID allocationID;
+
+	/** if the slot is allocated, jobId identify which job this slot is allocated to; else, jobId is null */
+	private final JobID jobID;
+
+	public SlotStatus(SlotID slotID, ResourceProfile profiler) {
+		this(slotID, profiler, null, null);
+	}
+
+	public SlotStatus(SlotID slotID, ResourceProfile profiler, AllocationID allocationID, JobID jobID) {
+		this.slotID = checkNotNull(slotID, "slotID cannot be null");
+		this.profiler = checkNotNull(profiler, "profile cannot be null");
+		this.allocationID = allocationID;
+		this.jobID = jobID;
+	}
+
+	/**
+	 * Get the unique identification of this slot
+	 *
+	 * @return The slot id
+	 */
+	public SlotID getSlotID() {
+		return slotID;
+	}
+
+	/**
+	 * Get the resource profile of this slot
+	 *
+	 * @return The resource profile
+	 */
+	public ResourceProfile getProfiler() {
+		return profiler;
+	}
+
+	/**
+	 * Get the allocation id of this slot
+	 *
+	 * @return The allocation id if this slot is allocated, otherwise null
+	 */
+	public AllocationID getAllocationID() {
+		return allocationID;
+	}
+
+	/**
+	 * Get the job id of the slot allocated for
+	 *
+	 * @return The job id if this slot is allocated, otherwise null
+	 */
+	public JobID getJobID() {
+		return jobID;
+	}
+
+	@Override
+	public boolean equals(Object o) {
+		if (this == o) {
+			return true;
+		}
+		if (o == null || getClass() != o.getClass()) {
+			return false;
+		}
+
+		SlotStatus that = (SlotStatus) o;
+
+		if (!slotID.equals(that.slotID)) {
+			return false;
+		}
+		if (!profiler.equals(that.profiler)) {
+			return false;
+		}
+		if (allocationID != null ? !allocationID.equals(that.allocationID) : that.allocationID != null) {
+			return false;
+		}
+		return jobID != null ? jobID.equals(that.jobID) : that.jobID == null;
+
+	}
+
+	@Override
+	public int hashCode() {
+		int result = slotID.hashCode();
+		result = 31 * result + profiler.hashCode();
+		result = 31 * result + (allocationID != null ? allocationID.hashCode() : 0);
+		result = 31 * result + (jobID != null ? jobID.hashCode() : 0);
+		return result;
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/6e22c64d/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/SlotManagerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/SlotManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/SlotManagerTest.java
new file mode 100644
index 0000000..2ee280f
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/SlotManagerTest.java
@@ -0,0 +1,540 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.clusterframework;
+
+import org.apache.flink.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.clusterframework.types.ResourceSlot;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.rpc.resourcemanager.ResourceManagerGateway;
+import org.apache.flink.runtime.rpc.resourcemanager.SlotRequest;
+import org.apache.flink.runtime.rpc.taskexecutor.SlotStatus;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+
+public class SlotManagerTest {
+
+	private static final double DEFAULT_TESTING_CPU_CORES = 1.0;
+
+	private static final long DEFAULT_TESTING_MEMORY = 512;
+
+	private static final ResourceProfile DEFAULT_TESTING_PROFILE =
+		new ResourceProfile(DEFAULT_TESTING_CPU_CORES, DEFAULT_TESTING_MEMORY);
+
+	private static final ResourceProfile DEFAULT_TESTING_BIG_PROFILE =
+		new ResourceProfile(DEFAULT_TESTING_CPU_CORES * 2, DEFAULT_TESTING_MEMORY * 2);
+
+	private ResourceManagerGateway resourceManagerGateway;
+
+	@Before
+	public void setUp() {
+		resourceManagerGateway = mock(ResourceManagerGateway.class);
+	}
+
+	/**
+	 * Tests that there are no free slots when we request, need to allocate from cluster manager master
+	 */
+	@Test
+	public void testRequestSlotWithoutFreeSlot() {
+		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
+		slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE));
+
+		assertEquals(0, slotManager.getAllocatedSlotCount());
+		assertEquals(0, slotManager.getFreeSlotCount());
+		assertEquals(1, slotManager.getPendingRequestCount());
+		assertEquals(1, slotManager.getAllocatedContainers().size());
+		assertEquals(DEFAULT_TESTING_PROFILE, slotManager.getAllocatedContainers().get(0));
+	}
+
+	/**
+	 * Tests that there are some free slots when we request, and the request is fulfilled immediately
+	 */
+	@Test
+	public void testRequestSlotWithFreeSlot() {
+		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
+
+		directlyProvideFreeSlots(slotManager, DEFAULT_TESTING_PROFILE, 1);
+		assertEquals(1, slotManager.getFreeSlotCount());
+
+		slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE));
+		assertEquals(1, slotManager.getAllocatedSlotCount());
+		assertEquals(0, slotManager.getFreeSlotCount());
+		assertEquals(0, slotManager.getPendingRequestCount());
+		assertEquals(0, slotManager.getAllocatedContainers().size());
+	}
+
+	/**
+	 * Tests that there are some free slots when we request, but none of them are suitable
+	 */
+	@Test
+	public void testRequestSlotWithoutSuitableSlot() {
+		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
+
+		directlyProvideFreeSlots(slotManager, DEFAULT_TESTING_PROFILE, 2);
+		assertEquals(2, slotManager.getFreeSlotCount());
+
+		slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_BIG_PROFILE));
+		assertEquals(0, slotManager.getAllocatedSlotCount());
+		assertEquals(2, slotManager.getFreeSlotCount());
+		assertEquals(1, slotManager.getPendingRequestCount());
+		assertEquals(1, slotManager.getAllocatedContainers().size());
+		assertEquals(DEFAULT_TESTING_BIG_PROFILE, slotManager.getAllocatedContainers().get(0));
+	}
+
+	/**
+	 * Tests that we send duplicated slot request
+	 */
+	@Test
+	public void testDuplicatedSlotRequest() {
+		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
+		directlyProvideFreeSlots(slotManager, DEFAULT_TESTING_PROFILE, 1);
+
+		SlotRequest request1 = new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE);
+		SlotRequest request2 = new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_BIG_PROFILE);
+
+		slotManager.requestSlot(request1);
+		slotManager.requestSlot(request2);
+		slotManager.requestSlot(request2);
+		slotManager.requestSlot(request1);
+
+		assertEquals(1, slotManager.getAllocatedSlotCount());
+		assertEquals(0, slotManager.getFreeSlotCount());
+		assertEquals(1, slotManager.getPendingRequestCount());
+		assertEquals(1, slotManager.getAllocatedContainers().size());
+		assertEquals(DEFAULT_TESTING_BIG_PROFILE, slotManager.getAllocatedContainers().get(0));
+	}
+
+	/**
+	 * Tests that we send multiple slot requests
+	 */
+	@Test
+	public void testRequestMultipleSlots() {
+		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
+		directlyProvideFreeSlots(slotManager, DEFAULT_TESTING_PROFILE, 5);
+
+		// request 3 normal slots
+		for (int i = 0; i < 3; ++i) {
+			slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE));
+		}
+
+		// request 2 big slots
+		for (int i = 0; i < 2; ++i) {
+			slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_BIG_PROFILE));
+		}
+
+		// request 1 normal slot again
+		slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE));
+
+		assertEquals(4, slotManager.getAllocatedSlotCount());
+		assertEquals(1, slotManager.getFreeSlotCount());
+		assertEquals(2, slotManager.getPendingRequestCount());
+		assertEquals(2, slotManager.getAllocatedContainers().size());
+		assertEquals(DEFAULT_TESTING_BIG_PROFILE, slotManager.getAllocatedContainers().get(0));
+		assertEquals(DEFAULT_TESTING_BIG_PROFILE, slotManager.getAllocatedContainers().get(1));
+	}
+
+	/**
+	 * Tests that a new slot appeared in SlotReport, and we used it to fulfill a pending request
+	 */
+	@Test
+	public void testNewlyAppearedFreeSlotFulfillPendingRequest() {
+		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
+		slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE));
+		assertEquals(1, slotManager.getPendingRequestCount());
+
+		SlotID slotId = SlotID.generate();
+		SlotStatus slotStatus = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE);
+		slotManager.updateSlotStatus(slotStatus);
+
+		assertEquals(1, slotManager.getAllocatedSlotCount());
+		assertEquals(0, slotManager.getFreeSlotCount());
+		assertEquals(0, slotManager.getPendingRequestCount());
+		assertTrue(slotManager.isAllocated(slotId));
+	}
+
+	/**
+	 * Tests that a new slot appeared in SlotReport, but we have no pending request
+	 */
+	@Test
+	public void testNewlyAppearedFreeSlot() {
+		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
+		SlotID slotId = SlotID.generate();
+		SlotStatus slotStatus = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE);
+		slotManager.updateSlotStatus(slotStatus);
+
+		assertEquals(0, slotManager.getAllocatedSlotCount());
+		assertEquals(1, slotManager.getFreeSlotCount());
+	}
+
+	/**
+	 * Tests that a new slot appeared in SlotReport, but it't not suitable for all the pending requests
+	 */
+	@Test
+	public void testNewlyAppearedFreeSlotNotMatchPendingRequests() {
+		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
+		slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_BIG_PROFILE));
+		assertEquals(1, slotManager.getPendingRequestCount());
+
+		SlotID slotId = SlotID.generate();
+		SlotStatus slotStatus = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE);
+		slotManager.updateSlotStatus(slotStatus);
+
+		assertEquals(0, slotManager.getAllocatedSlotCount());
+		assertEquals(1, slotManager.getFreeSlotCount());
+		assertEquals(1, slotManager.getPendingRequestCount());
+		assertFalse(slotManager.isAllocated(slotId));
+	}
+
+	/**
+	 * Tests that a new slot appeared in SlotReport, and it's been reported using by some job
+	 */
+	@Test
+	public void testNewlyAppearedInUseSlot() {
+		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
+
+		SlotID slotId = SlotID.generate();
+		SlotStatus slotStatus = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE, new AllocationID(), new JobID());
+		slotManager.updateSlotStatus(slotStatus);
+
+		assertEquals(1, slotManager.getAllocatedSlotCount());
+		assertEquals(0, slotManager.getFreeSlotCount());
+		assertTrue(slotManager.isAllocated(slotId));
+	}
+
+	/**
+	 * Tests that we had a slot in-use, and it's confirmed by SlotReport
+	 */
+	@Test
+	public void testExistingInUseSlotUpdateStatus() {
+		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
+		SlotRequest request = new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE);
+		slotManager.requestSlot(request);
+
+		// make this slot in use
+		SlotID slotId = SlotID.generate();
+		SlotStatus slotStatus = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE);
+		slotManager.updateSlotStatus(slotStatus);
+
+		assertEquals(1, slotManager.getAllocatedSlotCount());
+		assertEquals(0, slotManager.getFreeSlotCount());
+		assertTrue(slotManager.isAllocated(slotId));
+
+		// slot status is confirmed
+		SlotStatus slotStatus2 = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE,
+			request.getAllocationId(), request.getJobId());
+		slotManager.updateSlotStatus(slotStatus2);
+
+		assertEquals(1, slotManager.getAllocatedSlotCount());
+		assertEquals(0, slotManager.getFreeSlotCount());
+		assertTrue(slotManager.isAllocated(slotId));
+	}
+
+	/**
+	 * Tests that we had a slot in-use, but it's empty according to the SlotReport
+	 */
+	@Test
+	public void testExistingInUseSlotAdjustedToEmpty() {
+		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
+		SlotRequest request1 = new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE);
+		slotManager.requestSlot(request1);
+
+		// make this slot in use
+		SlotID slotId = SlotID.generate();
+		SlotStatus slotStatus = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE);
+		slotManager.updateSlotStatus(slotStatus);
+
+		// another request pending
+		SlotRequest request2 = new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE);
+		slotManager.requestSlot(request2);
+
+		assertEquals(1, slotManager.getAllocatedSlotCount());
+		assertEquals(0, slotManager.getFreeSlotCount());
+		assertEquals(1, slotManager.getPendingRequestCount());
+		assertTrue(slotManager.isAllocated(slotId));
+		assertTrue(slotManager.isAllocated(request1.getAllocationId()));
+
+
+		// but slot is reported empty again, request2 will be fulfilled, request1 will be missing
+		slotManager.updateSlotStatus(slotStatus);
+
+		assertEquals(1, slotManager.getAllocatedSlotCount());
+		assertEquals(0, slotManager.getFreeSlotCount());
+		assertEquals(0, slotManager.getPendingRequestCount());
+		assertTrue(slotManager.isAllocated(slotId));
+		assertTrue(slotManager.isAllocated(request2.getAllocationId()));
+	}
+
+	/**
+	 * Tests that we had a slot in use, and it's also reported in use by TaskManager, but the allocation
+	 * information didn't match.
+	 */
+	@Test
+	public void testExistingInUseSlotWithDifferentAllocationInfo() {
+		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
+		SlotRequest request = new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE);
+		slotManager.requestSlot(request);
+
+		// make this slot in use
+		SlotID slotId = SlotID.generate();
+		SlotStatus slotStatus = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE);
+		slotManager.updateSlotStatus(slotStatus);
+
+		assertEquals(1, slotManager.getAllocatedSlotCount());
+		assertEquals(0, slotManager.getFreeSlotCount());
+		assertEquals(0, slotManager.getPendingRequestCount());
+		assertTrue(slotManager.isAllocated(slotId));
+		assertTrue(slotManager.isAllocated(request.getAllocationId()));
+
+		SlotStatus slotStatus2 = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE, new AllocationID(), new JobID());
+		// update slot status with different allocation info
+		slotManager.updateSlotStatus(slotStatus2);
+
+		// original request is missing and won't be allocated
+		assertEquals(1, slotManager.getAllocatedSlotCount());
+		assertEquals(0, slotManager.getFreeSlotCount());
+		assertEquals(0, slotManager.getPendingRequestCount());
+		assertTrue(slotManager.isAllocated(slotId));
+		assertFalse(slotManager.isAllocated(request.getAllocationId()));
+		assertTrue(slotManager.isAllocated(slotStatus2.getAllocationID()));
+	}
+
+	/**
+	 * Tests that we had a free slot, and it's confirmed by SlotReport
+	 */
+	@Test
+	public void testExistingEmptySlotUpdateStatus() {
+		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
+		ResourceSlot slot = new ResourceSlot(SlotID.generate(), DEFAULT_TESTING_PROFILE);
+		slotManager.addFreeSlot(slot);
+
+		SlotStatus slotStatus = new SlotStatus(slot.getSlotId(), DEFAULT_TESTING_PROFILE);
+		slotManager.updateSlotStatus(slotStatus);
+
+		assertEquals(0, slotManager.getAllocatedSlotCount());
+		assertEquals(1, slotManager.getFreeSlotCount());
+		assertEquals(0, slotManager.getPendingRequestCount());
+	}
+
+	/**
+	 * Tests that we had a free slot, and it's reported in-use by TaskManager
+	 */
+	@Test
+	public void testExistingEmptySlotAdjustedToInUse() {
+		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
+		ResourceSlot slot = new ResourceSlot(SlotID.generate(), DEFAULT_TESTING_PROFILE);
+		slotManager.addFreeSlot(slot);
+
+		SlotStatus slotStatus = new SlotStatus(slot.getSlotId(), DEFAULT_TESTING_PROFILE,
+			new AllocationID(), new JobID());
+		slotManager.updateSlotStatus(slotStatus);
+
+		assertEquals(1, slotManager.getAllocatedSlotCount());
+		assertEquals(0, slotManager.getFreeSlotCount());
+		assertEquals(0, slotManager.getPendingRequestCount());
+		assertTrue(slotManager.isAllocated(slot.getSlotId()));
+	}
+
+	/**
+	 * Tests that we did some allocation but failed / rejected by TaskManager, request will retry
+	 */
+	@Test
+	public void testSlotAllocationFailedAtTaskManager() {
+		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
+		ResourceSlot slot = new ResourceSlot(SlotID.generate(), DEFAULT_TESTING_PROFILE);
+		slotManager.addFreeSlot(slot);
+
+		SlotRequest request = new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE);
+		slotManager.requestSlot(request);
+
+		assertEquals(1, slotManager.getAllocatedSlotCount());
+		assertEquals(0, slotManager.getFreeSlotCount());
+		assertEquals(0, slotManager.getPendingRequestCount());
+		assertTrue(slotManager.isAllocated(slot.getSlotId()));
+
+		slotManager.handleSlotRequestFailedAtTaskManager(request, slot.getSlotId());
+
+		assertEquals(1, slotManager.getAllocatedSlotCount());
+		assertEquals(0, slotManager.getFreeSlotCount());
+		assertEquals(0, slotManager.getPendingRequestCount());
+	}
+
+
+	/**
+	 * Tests that we did some allocation but failed / rejected by TaskManager, and slot is occupied by another request
+	 */
+	@Test
+	public void testSlotAllocationFailedAtTaskManagerOccupiedByOther() {
+		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
+		ResourceSlot slot = new ResourceSlot(SlotID.generate(), DEFAULT_TESTING_PROFILE);
+		slotManager.addFreeSlot(slot);
+
+		SlotRequest request = new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE);
+		slotManager.requestSlot(request);
+
+		// slot is set empty by heartbeat
+		SlotStatus slotStatus = new SlotStatus(slot.getSlotId(), slot.getResourceProfile());
+		slotManager.updateSlotStatus(slotStatus);
+
+		// another request took this slot
+		SlotRequest request2 = new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE);
+		slotManager.requestSlot(request2);
+
+		assertEquals(1, slotManager.getAllocatedSlotCount());
+		assertEquals(0, slotManager.getFreeSlotCount());
+		assertEquals(0, slotManager.getPendingRequestCount());
+		assertFalse(slotManager.isAllocated(request.getAllocationId()));
+		assertTrue(slotManager.isAllocated(request2.getAllocationId()));
+
+		// original request should be pended
+		slotManager.handleSlotRequestFailedAtTaskManager(request, slot.getSlotId());
+
+		assertEquals(1, slotManager.getAllocatedSlotCount());
+		assertEquals(0, slotManager.getFreeSlotCount());
+		assertEquals(1, slotManager.getPendingRequestCount());
+		assertFalse(slotManager.isAllocated(request.getAllocationId()));
+		assertTrue(slotManager.isAllocated(request2.getAllocationId()));
+	}
+
+	@Test
+	public void testNotifyTaskManagerFailure() {
+		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
+
+		ResourceID resource1 = ResourceID.generate();
+		ResourceID resource2 = ResourceID.generate();
+
+		ResourceSlot slot11 = new ResourceSlot(new SlotID(resource1, 1), DEFAULT_TESTING_PROFILE);
+		ResourceSlot slot12 = new ResourceSlot(new SlotID(resource1, 2), DEFAULT_TESTING_PROFILE);
+		ResourceSlot slot21 = new ResourceSlot(new SlotID(resource2, 1), DEFAULT_TESTING_PROFILE);
+		ResourceSlot slot22 = new ResourceSlot(new SlotID(resource2, 2), DEFAULT_TESTING_PROFILE);
+
+		slotManager.addFreeSlot(slot11);
+		slotManager.addFreeSlot(slot21);
+
+		slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE));
+		slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE));
+
+		assertEquals(2, slotManager.getAllocatedSlotCount());
+		assertEquals(0, slotManager.getFreeSlotCount());
+		assertEquals(0, slotManager.getPendingRequestCount());
+
+		slotManager.addFreeSlot(slot12);
+		slotManager.addFreeSlot(slot22);
+
+		assertEquals(2, slotManager.getAllocatedSlotCount());
+		assertEquals(2, slotManager.getFreeSlotCount());
+		assertEquals(0, slotManager.getPendingRequestCount());
+
+		slotManager.notifyTaskManagerFailure(resource2);
+
+		assertEquals(1, slotManager.getAllocatedSlotCount());
+		assertEquals(1, slotManager.getFreeSlotCount());
+		assertEquals(0, slotManager.getPendingRequestCount());
+
+		// notify an not exist resource failure
+		slotManager.notifyTaskManagerFailure(ResourceID.generate());
+
+		assertEquals(1, slotManager.getAllocatedSlotCount());
+		assertEquals(1, slotManager.getFreeSlotCount());
+		assertEquals(0, slotManager.getPendingRequestCount());
+	}
+
+	// ------------------------------------------------------------------------
+	//  testing utilities
+	// ------------------------------------------------------------------------
+
+	private void directlyProvideFreeSlots(
+		final SlotManager slotManager,
+		final ResourceProfile resourceProfile,
+		final int freeSlotNum)
+	{
+		for (int i = 0; i < freeSlotNum; ++i) {
+			slotManager.addFreeSlot(new ResourceSlot(SlotID.generate(), new ResourceProfile(resourceProfile)));
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  testing classes
+	// ------------------------------------------------------------------------
+
+	private static class TestingSlotManager extends SlotManager {
+
+		private final List<ResourceProfile> allocatedContainers;
+
+		TestingSlotManager(ResourceManagerGateway resourceManagerGateway) {
+			super(resourceManagerGateway);
+			this.allocatedContainers = new LinkedList<>();
+		}
+
+		/**
+		 * Choose slot randomly if it matches requirement
+		 *
+		 * @param request   The slot request
+		 * @param freeSlots All slots which can be used
+		 * @return The chosen slot or null if cannot find a match
+		 */
+		@Override
+		protected ResourceSlot chooseSlotToUse(SlotRequest request, Map<SlotID, ResourceSlot> freeSlots) {
+			for (ResourceSlot slot : freeSlots.values()) {
+				if (slot.isMatchingRequirement(request.getResourceProfile())) {
+					return slot;
+				}
+			}
+			return null;
+		}
+
+		/**
+		 * Choose request randomly if offered slot can match its requirement
+		 *
+		 * @param offeredSlot     The free slot
+		 * @param pendingRequests All the pending slot requests
+		 * @return The chosen request's AllocationID or null if cannot find a match
+		 */
+		@Override
+		protected SlotRequest chooseRequestToFulfill(ResourceSlot offeredSlot,
+			Map<AllocationID, SlotRequest> pendingRequests)
+		{
+			for (Map.Entry<AllocationID, SlotRequest> pendingRequest : pendingRequests.entrySet()) {
+				if (offeredSlot.isMatchingRequirement(pendingRequest.getValue().getResourceProfile())) {
+					return pendingRequest.getValue();
+				}
+			}
+			return null;
+		}
+
+		@Override
+		protected void allocateContainer(ResourceProfile resourceProfile) {
+			allocatedContainers.add(resourceProfile);
+		}
+
+		List<ResourceProfile> getAllocatedContainers() {
+			return allocatedContainers;
+		}
+	}
+}


[09/50] [abbrv] flink git commit: [FLINK-4658] [rpc] Allow RpcService to execute Runnables and Callables in its executor

Posted by se...@apache.org.
[FLINK-4658] [rpc] Allow RpcService to execute Runnables and Callables in its executor

This closes #2531.


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

Branch: refs/heads/flip-6
Commit: f4dc47411fc06b5f3ee0ff34c2ebf02bd459ffa8
Parents: 507e86c
Author: Till Rohrmann <tr...@apache.org>
Authored: Wed Sep 21 18:16:27 2016 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Oct 14 15:14:40 2016 +0200

----------------------------------------------------------------------
 .../apache/flink/runtime/rpc/RpcService.java    | 31 ++++++++++++++
 .../flink/runtime/rpc/akka/AkkaRpcService.java  | 14 ++++++
 .../flink/runtime/rpc/AsyncCallsTest.java       |  1 -
 .../runtime/rpc/TestingSerialRpcService.java    | 16 +++++++
 .../runtime/rpc/akka/AkkaRpcServiceTest.java    | 45 ++++++++++++++++++++
 5 files changed, 106 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/f4dc4741/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 a367ff2..437e08b 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
@@ -21,6 +21,7 @@ package org.apache.flink.runtime.rpc;
 import org.apache.flink.runtime.concurrent.Future;
 import org.apache.flink.runtime.rpc.exceptions.RpcConnectionException;
 
+import java.util.concurrent.Callable;
 import java.util.concurrent.Executor;
 import java.util.concurrent.TimeUnit;
 
@@ -89,4 +90,34 @@ public interface RpcService {
 	 * @param delay    The delay after which the runnable will be executed
 	 */
 	void scheduleRunnable(Runnable runnable, long delay, TimeUnit unit);
+
+	/**
+	 * Execute the given runnable in the executor of the RPC service. This method can be used to run
+	 * code outside of the main thread of a {@link RpcEndpoint}.
+	 *
+	 * <p><b>IMPORTANT:</b> This executor does not isolate the method invocations against
+	 * any concurrent invocations and is therefore not suitable to run completion methods of futures
+	 * that modify state of an {@link RpcEndpoint}. For such operations, one needs to use the
+	 * {@link RpcEndpoint#getMainThreadExecutor() MainThreadExecutionContext} of that
+	 * {@code RpcEndpoint}.
+	 *
+	 * @param runnable to execute
+	 */
+	void execute(Runnable runnable);
+
+	/**
+	 * Execute the given callable and return its result as a {@link Future}. This method can be used
+	 * to run code outside of the main thread of a {@link RpcEndpoint}.
+	 *
+	 * <p><b>IMPORTANT:</b> This executor does not isolate the method invocations against
+	 * any concurrent invocations and is therefore not suitable to run completion methods of futures
+	 * that modify state of an {@link RpcEndpoint}. For such operations, one needs to use the
+	 * {@link RpcEndpoint#getMainThreadExecutor() MainThreadExecutionContext} of that
+	 * {@code RpcEndpoint}.
+	 *
+	 * @param callable to execute
+	 * @param <T> is the return value type
+	 * @return Future containing the callable's future result
+	 */
+	<T> Future<T> execute(Callable<T> callable);
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/f4dc4741/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 36f1115..cee19c4 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
@@ -25,6 +25,7 @@ import akka.actor.ActorSystem;
 import akka.actor.Identify;
 import akka.actor.PoisonPill;
 import akka.actor.Props;
+import akka.dispatch.Futures;
 import akka.dispatch.Mapper;
 
 import akka.pattern.Patterns;
@@ -48,6 +49,7 @@ import java.lang.reflect.InvocationHandler;
 import java.lang.reflect.Proxy;
 import java.util.HashSet;
 import java.util.Set;
+import java.util.concurrent.Callable;
 import java.util.concurrent.Executor;
 import java.util.concurrent.TimeUnit;
 
@@ -223,4 +225,16 @@ public class AkkaRpcService implements RpcService {
 
 		actorSystem.scheduler().scheduleOnce(new FiniteDuration(delay, unit), runnable, actorSystem.dispatcher());
 	}
+
+	@Override
+	public void execute(Runnable runnable) {
+		actorSystem.dispatcher().execute(runnable);
+	}
+
+	@Override
+	public <T> Future<T> execute(Callable<T> callable) {
+		scala.concurrent.Future<T> scalaFuture = Futures.future(callable, actorSystem.dispatcher());
+
+		return new FlinkFuture<>(scalaFuture);
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/f4dc4741/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/AsyncCallsTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/AsyncCallsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/AsyncCallsTest.java
index 7c6b0ee..e8255d4 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/AsyncCallsTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/AsyncCallsTest.java
@@ -19,7 +19,6 @@
 package org.apache.flink.runtime.rpc;
 
 import akka.actor.ActorSystem;
-import akka.util.Timeout;
 
 import org.apache.flink.api.common.time.Time;
 import org.apache.flink.core.testutils.OneShotLatch;

http://git-wip-us.apache.org/repos/asf/flink/blob/f4dc4741/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 957453a..c58ea20 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
@@ -65,6 +65,22 @@ public class TestingSerialRpcService implements RpcService {
 	}
 
 	@Override
+	public void execute(Runnable runnable) {
+		runnable.run();
+	}
+
+	@Override
+	public <T> Future<T> execute(Callable<T> callable) {
+		try {
+			T result = callable.call();
+
+			return FlinkCompletableFuture.completed(result);
+		} catch (Exception e) {
+			return FlinkCompletableFuture.completedExceptionally(e);
+		}
+	}
+
+	@Override
 	public Executor getExecutor() {
 		return executorService;
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/f4dc4741/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java
index 4e9e518..5550cb5 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java
@@ -22,13 +22,18 @@ import akka.actor.ActorSystem;
 import org.apache.flink.api.common.time.Time;
 import org.apache.flink.core.testutils.OneShotLatch;
 import org.apache.flink.runtime.akka.AkkaUtils;
+import org.apache.flink.runtime.concurrent.Future;
 import org.apache.flink.util.TestLogger;
 
 import org.junit.AfterClass;
 import org.junit.Test;
 
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
 
+import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
 public class AkkaRpcServiceTest extends TestLogger {
@@ -70,4 +75,44 @@ public class AkkaRpcServiceTest extends TestLogger {
 
 		assertTrue("call was not properly delayed", ((stop - start) / 1000000) >= delay);
 	}
+
+	/**
+	 * Tests that the {@link AkkaRpcService} can execute runnables
+	 */
+	@Test
+	public void testExecuteRunnable() throws Exception {
+		final OneShotLatch latch = new OneShotLatch();
+
+		akkaRpcService.execute(new Runnable() {
+			@Override
+			public void run() {
+				latch.trigger();
+			}
+		});
+
+		latch.await(30L, TimeUnit.SECONDS);
+	}
+
+	/**
+	 * Tests that the {@link AkkaRpcService} can execute callables and returns their result as
+	 * a {@link Future}.
+	 */
+	@Test
+	public void testExecuteCallable() throws InterruptedException, ExecutionException, TimeoutException {
+		final OneShotLatch latch = new OneShotLatch();
+		final int expected = 42;
+
+		Future<Integer> result = akkaRpcService.execute(new Callable<Integer>() {
+			@Override
+			public Integer call() throws Exception {
+				latch.trigger();
+				return expected;
+			}
+		});
+
+		int actual = result.get(30L, TimeUnit.SECONDS);
+
+		assertEquals(expected, actual);
+		assertTrue(latch.isTriggered());
+	}
 }


[30/50] [abbrv] flink git commit: [FLINK-4530] [rpc] Generalize TaskExecutorToResourceManagerConnection to be reusable

Posted by se...@apache.org.
[FLINK-4530] [rpc] Generalize TaskExecutorToResourceManagerConnection to be reusable

This closes #2520


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

Branch: refs/heads/flip-6
Commit: b8c6b9986522aea96c79c774eeea5dc57a7bfc64
Parents: 07512e0
Author: zhuhaifengleon <zh...@gmail.com>
Authored: Mon Sep 26 17:43:44 2016 +0800
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Oct 14 15:14:41 2016 +0200

----------------------------------------------------------------------
 .../JobMasterToResourceManagerConnection.java   | 117 +++++++++++
 .../registration/RegisteredRpcConnection.java   | 192 +++++++++++++++++++
 .../runtime/taskexecutor/TaskExecutor.java      |   4 +-
 ...TaskExecutorToResourceManagerConnection.java | 127 +++---------
 .../RegisteredRpcConnectionTest.java            | 183 ++++++++++++++++++
 .../registration/RetryingRegistrationTest.java  |   6 +-
 6 files changed, 519 insertions(+), 110 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/b8c6b998/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterToResourceManagerConnection.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterToResourceManagerConnection.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterToResourceManagerConnection.java
new file mode 100644
index 0000000..71fce8c
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterToResourceManagerConnection.java
@@ -0,0 +1,117 @@
+/*
+ * 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.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.RpcService;
+import org.apache.flink.runtime.concurrent.Future;
+
+import org.slf4j.Logger;
+
+import java.util.UUID;
+import java.util.concurrent.Executor;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * The connection between a JobMaster and the ResourceManager.
+ */
+public class JobMasterToResourceManagerConnection 
+		extends RegisteredRpcConnection<ResourceManagerGateway, JobMasterRegistrationSuccess> {
+
+	/** the JobMaster whose connection to the ResourceManager this represents */
+	private final JobMaster jobMaster;
+
+	private final JobID jobID;
+
+	private final UUID jobMasterLeaderId;
+
+	public JobMasterToResourceManagerConnection(
+			Logger log,
+			JobID jobID,
+			JobMaster jobMaster,
+			UUID jobMasterLeaderId,
+			String resourceManagerAddress,
+			UUID resourceManagerLeaderId,
+			Executor executor) {
+
+		super(log, resourceManagerAddress, resourceManagerLeaderId, executor);
+		this.jobMaster = checkNotNull(jobMaster);
+		this.jobID = checkNotNull(jobID);
+		this.jobMasterLeaderId = checkNotNull(jobMasterLeaderId);
+	}
+
+	@Override
+	protected RetryingRegistration<ResourceManagerGateway, JobMasterRegistrationSuccess> generateRegistration() {
+		return new JobMasterToResourceManagerConnection.ResourceManagerRegistration(
+			log, jobMaster.getRpcService(),
+			getTargetAddress(), getTargetLeaderId(),
+			jobMaster.getAddress(),jobID, jobMasterLeaderId);
+	}
+
+	@Override
+	protected void onRegistrationSuccess(JobMasterRegistrationSuccess success) {
+	}
+
+	@Override
+	protected void onRegistrationFailure(Throwable failure) {
+	}
+
+	// ------------------------------------------------------------------------
+	//  Utilities
+	// ------------------------------------------------------------------------
+
+	private static class ResourceManagerRegistration
+		extends RetryingRegistration<ResourceManagerGateway, JobMasterRegistrationSuccess> {
+
+		private final String jobMasterAddress;
+
+		private final JobID jobID;
+
+		private final UUID jobMasterLeaderId;
+
+		ResourceManagerRegistration(
+			Logger log,
+			RpcService rpcService,
+			String targetAddress,
+			UUID leaderId,
+			String jobMasterAddress,
+			JobID jobID,
+			UUID jobMasterLeaderId) {
+
+			super(log, rpcService, "ResourceManager", ResourceManagerGateway.class, targetAddress, leaderId);
+			this.jobMasterAddress = checkNotNull(jobMasterAddress);
+			this.jobID = checkNotNull(jobID);
+			this.jobMasterLeaderId = checkNotNull(jobMasterLeaderId);
+		}
+
+		@Override
+		protected Future<RegistrationResponse> invokeRegistration(
+			ResourceManagerGateway gateway, UUID leaderId, long timeoutMillis) throws Exception {
+
+			Time timeout = Time.milliseconds(timeoutMillis);
+			return gateway.registerJobMaster(leaderId, jobMasterLeaderId,jobMasterAddress, jobID, timeout);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/b8c6b998/flink-runtime/src/main/java/org/apache/flink/runtime/registration/RegisteredRpcConnection.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/registration/RegisteredRpcConnection.java b/flink-runtime/src/main/java/org/apache/flink/runtime/registration/RegisteredRpcConnection.java
new file mode 100644
index 0000000..76093b0
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/registration/RegisteredRpcConnection.java
@@ -0,0 +1,192 @@
+/*
+ * 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.registration;
+
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.runtime.rpc.RpcGateway;
+import org.apache.flink.runtime.concurrent.AcceptFunction;
+import org.apache.flink.runtime.concurrent.ApplyFunction;
+import org.apache.flink.runtime.concurrent.Future;
+
+import org.slf4j.Logger;
+
+import java.util.UUID;
+import java.util.concurrent.Executor;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * This utility class implements the basis of RPC connecting from one component to another component,
+ * for example the RPC connection from TaskExecutor to ResourceManager.
+ * This {@code RegisteredRpcConnection} implements registration and get target gateway .
+ *
+ * <p>The registration gives access to a future that is completed upon successful registration.
+ * The RPC connection can be closed, for example when the target where it tries to register
+ * at looses leader status.
+ *
+ * @param <Gateway> The type of the gateway to connect to.
+ * @param <Success> The type of the successful registration responses.
+ */
+public abstract class RegisteredRpcConnection<Gateway extends RpcGateway, Success extends RegistrationResponse.Success> {
+
+	/** the logger for all log messages of this class */
+	protected final Logger log;
+
+	/** the target component leaderID, for example the ResourceManager leaderID */
+	private final UUID targetLeaderId;
+
+	/** the target component Address, for example the ResourceManager Address */
+	private final String targetAddress;
+
+	/** Execution context to be used to execute the on complete action of the ResourceManagerRegistration */
+	private final Executor executor;
+
+	/** the Registration of this RPC connection */
+	private RetryingRegistration<Gateway, Success> pendingRegistration;
+
+	/** the gateway to register, it's null until the registration is completed */
+	private volatile Gateway targetGateway;
+
+	/** flag indicating that the RPC connection is closed */
+	private volatile boolean closed;
+
+	// ------------------------------------------------------------------------
+
+	public RegisteredRpcConnection(
+		Logger log,
+		String targetAddress,
+		UUID targetLeaderId,
+		Executor executor)
+	{
+		this.log = checkNotNull(log);
+		this.targetAddress = checkNotNull(targetAddress);
+		this.targetLeaderId = checkNotNull(targetLeaderId);
+		this.executor = checkNotNull(executor);
+	}
+
+	// ------------------------------------------------------------------------
+	//  Life cycle
+	// ------------------------------------------------------------------------
+
+	@SuppressWarnings("unchecked")
+	public void start() {
+		checkState(!closed, "The RPC connection is already closed");
+		checkState(!isConnected() && pendingRegistration == null, "The RPC connection is already started");
+
+		pendingRegistration = checkNotNull(generateRegistration());
+		pendingRegistration.startRegistration();
+
+		Future<Tuple2<Gateway, Success>> future = pendingRegistration.getFuture();
+
+		future.thenAcceptAsync(new AcceptFunction<Tuple2<Gateway, Success>>() {
+			@Override
+			public void accept(Tuple2<Gateway, Success> result) {
+				targetGateway = result.f0;
+				onRegistrationSuccess(result.f1);
+			}
+		}, executor);
+
+		// this future should only ever fail if there is a bug, not if the registration is declined
+		future.exceptionallyAsync(new ApplyFunction<Throwable, Void>() {
+			@Override
+			public Void apply(Throwable failure) {
+				onRegistrationFailure(failure);
+				return null;
+			}
+		}, executor);
+	}
+
+	/**
+	 * This method generate a specific Registration, for example TaskExecutor Registration at the ResourceManager
+	 */
+	protected abstract RetryingRegistration<Gateway, Success> generateRegistration();
+
+	/**
+	 * This method handle the Registration Response
+	 */
+	protected abstract void onRegistrationSuccess(Success success);
+
+	/**
+	 * This method handle the Registration failure
+	 */
+	protected abstract void onRegistrationFailure(Throwable failure);
+
+	/**
+	 * close connection
+	 */
+	public void close() {
+		closed = true;
+
+		// make sure we do not keep re-trying forever
+		if (pendingRegistration != null) {
+			pendingRegistration.cancel();
+		}
+	}
+
+	public boolean isClosed() {
+		return closed;
+	}
+
+	// ------------------------------------------------------------------------
+	//  Properties
+	// ------------------------------------------------------------------------
+
+	public UUID getTargetLeaderId() {
+		return targetLeaderId;
+	}
+
+	public String getTargetAddress() {
+		return targetAddress;
+	}
+
+	/**
+	 * Gets the RegisteredGateway. This returns null until the registration is completed.
+	 */
+	public Gateway getTargetGateway() {
+		return targetGateway;
+	}
+
+	public boolean isConnected() {
+		return targetGateway != null;
+	}
+
+	// ------------------------------------------------------------------------
+
+	@Override
+	public String toString() {
+		String connectionInfo = "(ADDRESS: " + targetAddress + " LEADERID: " + targetLeaderId + ")";
+
+		if (isConnected()) {
+			connectionInfo = "RPC connection to " + targetGateway.getClass().getSimpleName() + " " + connectionInfo;
+		} else {
+			connectionInfo = "RPC connection to " + connectionInfo;
+		}
+
+		if (isClosed()) {
+			connectionInfo = connectionInfo + " is closed";
+		} else if (isConnected()){
+			connectionInfo = connectionInfo + " is established";
+		} else {
+			connectionInfo = connectionInfo + " is connecting";
+		}
+
+		return connectionInfo;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/b8c6b998/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 9e3c3b9..9d9ad2a 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
@@ -178,12 +178,12 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 			if (newLeaderAddress != null) {
 				// the resource manager switched to a new leader
 				log.info("ResourceManager leader changed from {} to {}. Registering at new leader.",
-					resourceManagerConnection.getResourceManagerAddress(), newLeaderAddress);
+					resourceManagerConnection.getTargetAddress(), newLeaderAddress);
 			}
 			else {
 				// address null means that the current leader is lost without a new leader being there, yet
 				log.info("Current ResourceManager {} lost leader status. Waiting for new ResourceManager leader.",
-					resourceManagerConnection.getResourceManagerAddress());
+					resourceManagerConnection.getTargetAddress());
 			}
 
 			// drop the current connection or connection attempt

http://git-wip-us.apache.org/repos/asf/flink/blob/b8c6b998/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 647359d..b4b3bae 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
@@ -19,16 +19,14 @@
 package org.apache.flink.runtime.taskexecutor;
 
 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.AcceptFunction;
-import org.apache.flink.runtime.concurrent.ApplyFunction;
-import org.apache.flink.runtime.concurrent.Future;
 import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.registration.RegisteredRpcConnection;
 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.slf4j.Logger;
 
@@ -36,115 +34,46 @@ import java.util.UUID;
 import java.util.concurrent.Executor;
 
 import static org.apache.flink.util.Preconditions.checkNotNull;
-import static org.apache.flink.util.Preconditions.checkState;
 
 /**
  * The connection between a TaskExecutor and the ResourceManager.
  */
-public class TaskExecutorToResourceManagerConnection {
-
-	/** the logger for all log messages of this class */
-	private final Logger log;
+public class TaskExecutorToResourceManagerConnection
+		extends RegisteredRpcConnection<ResourceManagerGateway, TaskExecutorRegistrationSuccess> {
 
 	/** the TaskExecutor whose connection to the ResourceManager this represents */
 	private final TaskExecutor taskExecutor;
 
-	private final UUID resourceManagerLeaderId;
-
-	private final String resourceManagerAddress;
-
-	/** Execution context to be used to execute the on complete action of the ResourceManagerRegistration */
-	private final Executor executor;
-
-	private TaskExecutorToResourceManagerConnection.ResourceManagerRegistration pendingRegistration;
-
-	private volatile ResourceManagerGateway registeredResourceManager;
-
 	private InstanceID registrationId;
 
-	/** flag indicating that the connection is closed */
-	private volatile boolean closed;
-
-
 	public TaskExecutorToResourceManagerConnection(
-		Logger log,
-		TaskExecutor taskExecutor,
-		String resourceManagerAddress,
-		UUID resourceManagerLeaderId,
-		Executor executor) {
+			Logger log,
+			TaskExecutor taskExecutor,
+			String resourceManagerAddress,
+			UUID resourceManagerLeaderId,
+			Executor executor) {
 
-		this.log = checkNotNull(log);
+		super(log, resourceManagerAddress, resourceManagerLeaderId, executor);
 		this.taskExecutor = checkNotNull(taskExecutor);
-		this.resourceManagerAddress = checkNotNull(resourceManagerAddress);
-		this.resourceManagerLeaderId = checkNotNull(resourceManagerLeaderId);
-		this.executor = checkNotNull(executor);
-	}
-
-	// ------------------------------------------------------------------------
-	//  Life cycle
-	// ------------------------------------------------------------------------
-
-	@SuppressWarnings("unchecked")
-	public void start() {
-		checkState(!closed, "The connection is already closed");
-		checkState(!isRegistered() && pendingRegistration == null, "The connection is already started");
-
-		pendingRegistration = new TaskExecutorToResourceManagerConnection.ResourceManagerRegistration(
-				log, taskExecutor.getRpcService(),
-				resourceManagerAddress, resourceManagerLeaderId,
-				taskExecutor.getAddress(), taskExecutor.getResourceID());
-		pendingRegistration.startRegistration();
-
-		Future<Tuple2<ResourceManagerGateway, TaskExecutorRegistrationSuccess>> future = pendingRegistration.getFuture();
-
-		future.thenAcceptAsync(new AcceptFunction<Tuple2<ResourceManagerGateway, TaskExecutorRegistrationSuccess>>() {
-			@Override
-			public void accept(Tuple2<ResourceManagerGateway, TaskExecutorRegistrationSuccess> result) {
-				registrationId = result.f1.getRegistrationId();
-				registeredResourceManager = result.f0;
-			}
-		}, executor);
-		
-		// this future should only ever fail if there is a bug, not if the registration is declined
-		future.exceptionallyAsync(new ApplyFunction<Throwable, Void>() {
-			@Override
-			public Void apply(Throwable failure) {
-				taskExecutor.onFatalErrorAsync(failure);
-				return null;
-			}
-		}, executor);
-	}
-
-	public void close() {
-		closed = true;
-
-		// make sure we do not keep re-trying forever
-		if (pendingRegistration != null) {
-			pendingRegistration.cancel();
-		}
 	}
 
-	public boolean isClosed() {
-		return closed;
-	}
 
-	// ------------------------------------------------------------------------
-	//  Properties
-	// ------------------------------------------------------------------------
-
-	public UUID getResourceManagerLeaderId() {
-		return resourceManagerLeaderId;
+	@Override
+	protected RetryingRegistration<ResourceManagerGateway, TaskExecutorRegistrationSuccess> generateRegistration() {
+		return new TaskExecutorToResourceManagerConnection.ResourceManagerRegistration(
+			log, taskExecutor.getRpcService(),
+			getTargetAddress(), getTargetLeaderId(),
+			taskExecutor.getAddress(),taskExecutor.getResourceID());
 	}
 
-	public String getResourceManagerAddress() {
-		return resourceManagerAddress;
+	@Override
+	protected void onRegistrationSuccess(TaskExecutorRegistrationSuccess success) {
+		registrationId = success.getRegistrationId();
 	}
 
-	/**
-	 * Gets the ResourceManagerGateway. This returns null until the registration is completed.
-	 */
-	public ResourceManagerGateway getResourceManager() {
-		return registeredResourceManager;
+	@Override
+	protected void onRegistrationFailure(Throwable failure) {
+		taskExecutor.onFatalErrorAsync(failure);
 	}
 
 	/**
@@ -155,18 +84,6 @@ public class TaskExecutorToResourceManagerConnection {
 		return registrationId;
 	}
 
-	public boolean isRegistered() {
-		return registeredResourceManager != null;
-	}
-
-	// ------------------------------------------------------------------------
-
-	@Override
-	public String toString() {
-		return String.format("Connection to ResourceManager %s (leaderId=%s)",
-				resourceManagerAddress, resourceManagerLeaderId); 
-	}
-
 	// ------------------------------------------------------------------------
 	//  Utilities
 	// ------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/flink/blob/b8c6b998/flink-runtime/src/test/java/org/apache/flink/runtime/registration/RegisteredRpcConnectionTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/registration/RegisteredRpcConnectionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/registration/RegisteredRpcConnectionTest.java
new file mode 100644
index 0000000..8558205
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/registration/RegisteredRpcConnectionTest.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.registration;
+
+import org.apache.flink.runtime.registration.RetryingRegistrationTest.TestRegistrationSuccess;
+import org.apache.flink.runtime.rpc.RpcService;
+import org.apache.flink.runtime.rpc.TestingRpcService;
+import org.apache.flink.util.TestLogger;
+import org.junit.Test;
+import org.slf4j.LoggerFactory;
+
+import java.util.UUID;
+import java.util.concurrent.Executor;
+
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNull;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyLong;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+/**
+ * Tests for RegisteredRpcConnection, validating the successful, failure and close behavior.
+ */
+public class RegisteredRpcConnectionTest extends TestLogger {
+
+	@Test
+	public void testSuccessfulRpcConnection() throws Exception {
+		final String testRpcConnectionEndpointAddress = "<TestRpcConnectionEndpointAddress>";
+		final UUID leaderId = UUID.randomUUID();
+		final String connectionID = "Test RPC Connection ID";
+
+		// an endpoint that immediately returns success
+		TestRegistrationGateway testGateway = new TestRegistrationGateway(new RetryingRegistrationTest.TestRegistrationSuccess(connectionID));
+		TestingRpcService rpcService = new TestingRpcService();
+
+		try {
+			rpcService.registerGateway(testRpcConnectionEndpointAddress, testGateway);
+
+			TestRpcConnection connection = new TestRpcConnection(testRpcConnectionEndpointAddress, leaderId, rpcService.getExecutor(), rpcService);
+			connection.start();
+
+			//wait for connection established
+			Thread.sleep(RetryingRegistrationTest.TestRetryingRegistration.MAX_TIMEOUT);
+
+			// validate correct invocation and result
+			assertTrue(connection.isConnected());
+			assertEquals(testRpcConnectionEndpointAddress, connection.getTargetAddress());
+			assertEquals(leaderId, connection.getTargetLeaderId());
+			assertEquals(testGateway, connection.getTargetGateway());
+			assertEquals(connectionID, connection.getConnectionId());
+		}
+		finally {
+			testGateway.stop();
+			rpcService.stopService();
+		}
+	}
+
+	@Test
+	public void testRpcConnectionFailures() throws Exception {
+		final String connectionFailureMessage = "Test RPC Connection failure";
+		final String testRpcConnectionEndpointAddress = "<TestRpcConnectionEndpointAddress>";
+		final UUID leaderId = UUID.randomUUID();
+
+		TestingRpcService rpcService = new TestingRpcService();
+
+		try {
+			// gateway that upon calls Throw an exception
+			TestRegistrationGateway testGateway = mock(TestRegistrationGateway.class);
+			when(testGateway.registrationCall(any(UUID.class), anyLong())).thenThrow(
+				new RuntimeException(connectionFailureMessage));
+
+			rpcService.registerGateway(testRpcConnectionEndpointAddress, testGateway);
+
+			TestRpcConnection connection = new TestRpcConnection(testRpcConnectionEndpointAddress, leaderId, rpcService.getExecutor(), rpcService);
+			connection.start();
+
+			//wait for connection failure
+			Thread.sleep(RetryingRegistrationTest.TestRetryingRegistration.MAX_TIMEOUT);
+
+			// validate correct invocation and result
+			assertFalse(connection.isConnected());
+			assertEquals(testRpcConnectionEndpointAddress, connection.getTargetAddress());
+			assertEquals(leaderId, connection.getTargetLeaderId());
+			assertNull(connection.getTargetGateway());
+			assertEquals(connectionFailureMessage, connection.getFailareMessage());
+		}
+		finally {
+			rpcService.stopService();
+		}
+	}
+
+	@Test
+	public void testRpcConnectionClose() throws Exception {
+		final String testRpcConnectionEndpointAddress = "<TestRpcConnectionEndpointAddress>";
+		final UUID leaderId = UUID.randomUUID();
+		final String connectionID = "Test RPC Connection ID";
+
+		TestRegistrationGateway testGateway = new TestRegistrationGateway(new RetryingRegistrationTest.TestRegistrationSuccess(connectionID));
+		TestingRpcService rpcService = new TestingRpcService();
+
+		try{
+			rpcService.registerGateway(testRpcConnectionEndpointAddress, testGateway);
+
+			TestRpcConnection connection = new TestRpcConnection(testRpcConnectionEndpointAddress, leaderId, rpcService.getExecutor(), rpcService);
+			connection.start();
+			//close the connection
+			connection.close();
+
+			// validate connection is closed
+			assertEquals(testRpcConnectionEndpointAddress, connection.getTargetAddress());
+			assertEquals(leaderId, connection.getTargetLeaderId());
+			assertTrue(connection.isClosed());
+		}
+		finally {
+			testGateway.stop();
+			rpcService.stopService();
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  test RegisteredRpcConnection
+	// ------------------------------------------------------------------------
+
+	private static class TestRpcConnection extends RegisteredRpcConnection<TestRegistrationGateway, TestRegistrationSuccess> {
+
+		private final RpcService rpcService;
+
+		private String connectionId;
+
+		private String failureMessage;
+
+		public TestRpcConnection(String targetAddress,
+								 UUID targetLeaderId,
+								 Executor executor,
+								 RpcService rpcService)
+		{
+			super(LoggerFactory.getLogger(RegisteredRpcConnectionTest.class), targetAddress, targetLeaderId, executor);
+			this.rpcService = rpcService;
+		}
+
+		@Override
+		protected RetryingRegistration<TestRegistrationGateway, RetryingRegistrationTest.TestRegistrationSuccess> generateRegistration() {
+			return new RetryingRegistrationTest.TestRetryingRegistration(rpcService, getTargetAddress(), getTargetLeaderId());
+		}
+
+		@Override
+		protected void onRegistrationSuccess(RetryingRegistrationTest.TestRegistrationSuccess success) {
+			connectionId = success.getCorrelationId();
+		}
+
+		@Override
+		protected void onRegistrationFailure(Throwable failure) {
+			failureMessage = failure.getMessage();
+		}
+
+		public String getConnectionId() {
+			return connectionId;
+		}
+
+		public String getFailareMessage() {
+			return failureMessage;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/b8c6b998/flink-runtime/src/test/java/org/apache/flink/runtime/registration/RetryingRegistrationTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/registration/RetryingRegistrationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/registration/RetryingRegistrationTest.java
index e56a9ec..6d6bbef 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/registration/RetryingRegistrationTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/registration/RetryingRegistrationTest.java
@@ -298,12 +298,12 @@ public class RetryingRegistrationTest extends TestLogger {
 	//  test registration
 	// ------------------------------------------------------------------------
 
-	private static class TestRegistrationSuccess extends RegistrationResponse.Success {
+	protected static class TestRegistrationSuccess extends RegistrationResponse.Success {
 		private static final long serialVersionUID = 5542698790917150604L;
 
 		private final String correlationId;
 
-		private TestRegistrationSuccess(String correlationId) {
+		public TestRegistrationSuccess(String correlationId) {
 			this.correlationId = correlationId;
 		}
 
@@ -312,7 +312,7 @@ public class RetryingRegistrationTest extends TestLogger {
 		}
 	}
 
-	private static class TestRetryingRegistration extends RetryingRegistration<TestRegistrationGateway, TestRegistrationSuccess> {
+	protected static class TestRetryingRegistration extends RetryingRegistration<TestRegistrationGateway, TestRegistrationSuccess> {
 
 		// we use shorter timeouts here to speed up the tests
 		static final long INITIAL_TIMEOUT = 20;


[08/50] [abbrv] flink git commit: [FLINK-4516] leader election of resourcemanager

Posted by se...@apache.org.
[FLINK-4516] leader election of resourcemanager

- add serial rpc service
- add a special rpcService implementation which directly executes the asynchronous calls serially one by one, it is just for testcase
- Change ResourceManagerLeaderContender code and TestingSerialRpcService code
- override shutdown logic to stop leadershipService
- use a mocked RpcService rather than TestingSerialRpcService for resourceManager HA test

This closes #2427


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

Branch: refs/heads/flip-6
Commit: 4b077af4dfab4e9c6e1ae4e6f4865f1f319deb0c
Parents: 6e22c64
Author: beyond1920 <be...@126.com>
Authored: Sat Aug 27 14:14:28 2016 +0800
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Oct 14 15:14:39 2016 +0200

----------------------------------------------------------------------
 .../HighAvailabilityServices.java               |   7 +
 .../runtime/highavailability/NonHaServices.java |   5 +
 .../rpc/resourcemanager/ResourceManager.java    | 111 +++++-
 .../TestingHighAvailabilityServices.java        |  19 +-
 .../runtime/rpc/TestingSerialRpcService.java    | 369 +++++++++++++++++++
 .../resourcemanager/ResourceManagerHATest.java  |  76 ++++
 6 files changed, 578 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/4b077af4/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 73e4f1f..298147c 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
@@ -40,6 +40,13 @@ public interface HighAvailabilityServices {
 	LeaderRetrievalService getResourceManagerLeaderRetriever() throws Exception;
 
 	/**
+	 * Gets the leader election service for the cluster's resource manager.
+	 * @return
+	 * @throws Exception
+	 */
+	LeaderElectionService getResourceManagerLeaderElectionService() throws Exception;
+
+	/**
 	 * Gets the leader election service for the given job.
 	 *
 	 * @param jobID The identifier of the job running the election.

http://git-wip-us.apache.org/repos/asf/flink/blob/4b077af4/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 3d2769b..292a404 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
@@ -61,6 +61,11 @@ public class NonHaServices implements HighAvailabilityServices {
 	}
 
 	@Override
+	public LeaderElectionService getResourceManagerLeaderElectionService() throws Exception {
+		return new StandaloneLeaderElectionService();
+	}
+
+	@Override
 	public LeaderElectionService getJobMasterLeaderElectionService(JobID jobID) throws Exception {
 		return new StandaloneLeaderElectionService();
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/4b077af4/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManager.java
index 6f34465..f7147c9 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManager.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManager.java
@@ -20,24 +20,26 @@ package org.apache.flink.runtime.rpc.resourcemanager;
 
 import akka.dispatch.Mapper;
 
+import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
 import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.leaderelection.LeaderContender;
+import org.apache.flink.runtime.leaderelection.LeaderElectionService;
 import org.apache.flink.runtime.rpc.RpcMethod;
 import org.apache.flink.runtime.rpc.RpcEndpoint;
 import org.apache.flink.runtime.rpc.RpcService;
 import org.apache.flink.runtime.rpc.jobmaster.JobMaster;
 import org.apache.flink.runtime.rpc.jobmaster.JobMasterGateway;
 import org.apache.flink.runtime.rpc.taskexecutor.TaskExecutorRegistrationSuccess;
-import org.apache.flink.util.Preconditions;
 
-import scala.concurrent.ExecutionContext;
-import scala.concurrent.ExecutionContext$;
 import scala.concurrent.Future;
 
 import java.util.HashMap;
 import java.util.Map;
 import java.util.UUID;
-import java.util.concurrent.ExecutorService;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
 
 /**
  * ResourceManager implementation. The resource manager is responsible for resource de-/allocation
@@ -50,16 +52,51 @@ import java.util.concurrent.ExecutorService;
  * </ul>
  */
 public class ResourceManager extends RpcEndpoint<ResourceManagerGateway> {
-	private final ExecutionContext executionContext;
 	private final Map<JobMasterGateway, InstanceID> jobMasterGateways;
+	private final HighAvailabilityServices highAvailabilityServices;
+	private LeaderElectionService leaderElectionService = null;
+	private UUID leaderSessionID = null;
 
-	public ResourceManager(RpcService rpcService, ExecutorService executorService) {
+	public ResourceManager(RpcService rpcService, HighAvailabilityServices highAvailabilityServices) {
 		super(rpcService);
-		this.executionContext = ExecutionContext$.MODULE$.fromExecutor(
-			Preconditions.checkNotNull(executorService));
+		this.highAvailabilityServices = checkNotNull(highAvailabilityServices);
 		this.jobMasterGateways = new HashMap<>();
 	}
 
+	@Override
+	public void start() {
+		// start a leader
+		try {
+			super.start();
+			leaderElectionService = highAvailabilityServices.getResourceManagerLeaderElectionService();
+			leaderElectionService.start(new ResourceManagerLeaderContender());
+		} 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);
+		}
+	}
+
+	@Override
+	public void shutDown() {
+		try {
+			leaderElectionService.stop();
+			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);
+		}
+	}
+
+	/**
+	 * 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;
+	}
+
 	/**
 	 * Register a {@link JobMaster} at the resource manager.
 	 *
@@ -116,4 +153,62 @@ public class ResourceManager extends RpcEndpoint<ResourceManagerGateway> {
 
 		return new TaskExecutorRegistrationSuccess(new InstanceID(), 5000);
 	}
+
+	private class ResourceManagerLeaderContender implements LeaderContender {
+
+		/**
+		 * Callback method when current resourceManager is granted leadership
+		 *
+		 * @param leaderSessionID unique leadershipID
+		 */
+		@Override
+		public void grantLeadership(final UUID leaderSessionID) {
+			runAsync(new Runnable() {
+				@Override
+				public void run() {
+					log.info("ResourceManager {} was granted leadership with leader session ID {}", getAddress(), leaderSessionID);
+					ResourceManager.this.leaderSessionID = leaderSessionID;
+					// confirming the leader session ID might be blocking,
+					leaderElectionService.confirmLeaderSessionID(leaderSessionID);
+				}
+			});
+		}
+
+		/**
+		 * Callback method when current resourceManager lose leadership.
+		 */
+		@Override
+		public void revokeLeadership() {
+			runAsync(new Runnable() {
+				@Override
+				public void run() {
+					log.info("ResourceManager {} was revoked leadership.", getAddress());
+					jobMasterGateways.clear();
+					leaderSessionID = null;
+				}
+			});
+		}
+
+		@Override
+		public String getAddress() {
+			return ResourceManager.this.getAddress();
+		}
+
+		/**
+		 * Handles error occurring in the leader election service
+		 *
+		 * @param exception Exception being thrown in the leader election service
+		 */
+		@Override
+		public void handleError(final Exception exception) {
+			runAsync(new Runnable() {
+				@Override
+				public void run() {
+					log.error("ResourceManager received an error from the LeaderElectionService.", exception);
+					// terminate ResourceManager in case of an error
+					shutDown();
+				}
+			});
+		}
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/4b077af4/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 4d654a3..3162f40 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
@@ -32,6 +32,8 @@ public class TestingHighAvailabilityServices implements HighAvailabilityServices
 
 	private volatile LeaderElectionService jobMasterLeaderElectionService;
 
+	private volatile LeaderElectionService resourceManagerLeaderElectionService;
+
 
 	// ------------------------------------------------------------------------
 	//  Setters for mock / testing implementations
@@ -44,7 +46,11 @@ public class TestingHighAvailabilityServices implements HighAvailabilityServices
 	public void setJobMasterLeaderElectionService(LeaderElectionService leaderElectionService) {
 		this.jobMasterLeaderElectionService = leaderElectionService;
 	}
-	
+
+	public void setResourceManagerLeaderElectionService(LeaderElectionService leaderElectionService) {
+		this.resourceManagerLeaderElectionService = leaderElectionService;
+	}
+
 	// ------------------------------------------------------------------------
 	//  HA Services Methods
 	// ------------------------------------------------------------------------
@@ -69,4 +75,15 @@ public class TestingHighAvailabilityServices implements HighAvailabilityServices
 			throw new IllegalStateException("JobMasterLeaderElectionService has not been set");
 		}
 	}
+
+	@Override
+	public LeaderElectionService getResourceManagerLeaderElectionService() throws Exception {
+		LeaderElectionService service = resourceManagerLeaderElectionService;
+
+		if (service != null) {
+			return service;
+		} else {
+			throw new IllegalStateException("ResourceManagerLeaderElectionService has not been set");
+		}
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/4b077af4/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
new file mode 100644
index 0000000..7bdbb99
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingSerialRpcService.java
@@ -0,0 +1,369 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.rpc;
+
+import akka.dispatch.ExecutionContexts;
+import akka.dispatch.Futures;
+import akka.util.Timeout;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.runtime.util.DirectExecutorService;
+import org.apache.flink.util.Preconditions;
+import scala.concurrent.Await;
+import scala.concurrent.ExecutionContext;
+import scala.concurrent.Future;
+import scala.concurrent.duration.FiniteDuration;
+
+import java.lang.annotation.Annotation;
+import java.lang.reflect.InvocationHandler;
+import java.lang.reflect.Method;
+import java.lang.reflect.Proxy;
+import java.util.BitSet;
+import java.util.UUID;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+
+/**
+ * An RPC Service implementation for testing. This RPC service directly executes all asynchronous calls one by one in the main thread.
+ */
+public class TestingSerialRpcService implements RpcService {
+
+	private final DirectExecutorService executorService;
+	private final ConcurrentHashMap<String, RpcGateway> registeredConnections;
+
+	public TestingSerialRpcService() {
+		executorService = new DirectExecutorService();
+		this.registeredConnections = new ConcurrentHashMap<>();
+	}
+
+	@Override
+	public void scheduleRunnable(final Runnable runnable, final long delay, final TimeUnit unit) {
+		try {
+			unit.sleep(delay);
+			runnable.run();
+		} catch (Throwable e) {
+			throw new RuntimeException(e);
+		}
+	}
+
+	@Override
+	public ExecutionContext getExecutionContext() {
+		return ExecutionContexts.fromExecutorService(executorService);
+	}
+
+	@Override
+	public void stopService() {
+		executorService.shutdown();
+		registeredConnections.clear();
+	}
+
+	@Override
+	public void stopServer(RpcGateway selfGateway) {
+
+	}
+
+	@Override
+	public <C extends RpcGateway, S extends RpcEndpoint<C>> C startServer(S rpcEndpoint) {
+		final String address = UUID.randomUUID().toString();
+
+		InvocationHandler akkaInvocationHandler = new TestingSerialInvocationHandler(address, rpcEndpoint);
+		ClassLoader classLoader = getClass().getClassLoader();
+
+		@SuppressWarnings("unchecked")
+		C self = (C) Proxy.newProxyInstance(
+			classLoader,
+			new Class<?>[]{
+				rpcEndpoint.getSelfGatewayType(),
+				MainThreadExecutor.class,
+				StartStoppable.class,
+				RpcGateway.class
+			},
+			akkaInvocationHandler);
+
+		return self;
+	}
+
+	@Override
+	public <C extends RpcGateway> Future<C> connect(String address, Class<C> clazz) {
+		RpcGateway gateway = registeredConnections.get(address);
+
+		if (gateway != null) {
+			if (clazz.isAssignableFrom(gateway.getClass())) {
+				@SuppressWarnings("unchecked")
+				C typedGateway = (C) gateway;
+				return Futures.successful(typedGateway);
+			} else {
+				return Futures.failed(
+					new Exception("Gateway registered under " + address + " is not of type " + clazz));
+			}
+		} else {
+			return Futures.failed(new Exception("No gateway registered under that name"));
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	// connections
+	// ------------------------------------------------------------------------
+
+	public void registerGateway(String address, RpcGateway gateway) {
+		checkNotNull(address);
+		checkNotNull(gateway);
+
+		if (registeredConnections.putIfAbsent(address, gateway) != null) {
+			throw new IllegalStateException("a gateway is already registered under " + address);
+		}
+	}
+
+	private static class TestingSerialInvocationHandler<C extends RpcGateway, T extends RpcEndpoint<C>> implements InvocationHandler, RpcGateway, MainThreadExecutor, StartStoppable {
+
+		private final T rpcEndpoint;
+
+		/** default timeout for asks */
+		private final Timeout timeout;
+
+		private final String address;
+
+		private TestingSerialInvocationHandler(String address, T rpcEndpoint) {
+			this(address, rpcEndpoint, new Timeout(new FiniteDuration(10, TimeUnit.SECONDS)));
+		}
+
+		private TestingSerialInvocationHandler(String address, T rpcEndpoint, Timeout timeout) {
+			this.rpcEndpoint = rpcEndpoint;
+			this.timeout = timeout;
+			this.address = address;
+		}
+
+		@Override
+		public Object invoke(Object proxy, Method method, Object[] args) throws Throwable {
+			Class<?> declaringClass = method.getDeclaringClass();
+			if (declaringClass.equals(MainThreadExecutor.class) ||
+				declaringClass.equals(Object.class) || declaringClass.equals(StartStoppable.class) ||
+				declaringClass.equals(RpcGateway.class)) {
+				return method.invoke(this, args);
+			} else {
+				final String methodName = method.getName();
+				Class<?>[] parameterTypes = method.getParameterTypes();
+				Annotation[][] parameterAnnotations = method.getParameterAnnotations();
+				Timeout futureTimeout = extractRpcTimeout(parameterAnnotations, args, timeout);
+
+				final Tuple2<Class<?>[], Object[]> filteredArguments = filterArguments(
+					parameterTypes,
+					parameterAnnotations,
+					args);
+
+				Class<?> returnType = method.getReturnType();
+
+				if (returnType.equals(Future.class)) {
+					try {
+						Object result = handleRpcInvocationSync(methodName, filteredArguments.f0, filteredArguments.f1, futureTimeout);
+						return Futures.successful(result);
+					} catch (Throwable e) {
+						return Futures.failed(e);
+					}
+				} else {
+					return handleRpcInvocationSync(methodName, filteredArguments.f0, filteredArguments.f1, futureTimeout);
+				}
+			}
+		}
+
+		/**
+		 * Handle rpc invocations by looking up the rpc method on the rpc endpoint and calling this
+		 * method with the provided method arguments. If the method has a return value, it is returned
+		 * to the sender of the call.
+		 */
+		private Object handleRpcInvocationSync(final String methodName,
+			final Class<?>[] parameterTypes,
+			final Object[] args,
+			final Timeout futureTimeout) throws Exception {
+			final Method rpcMethod = lookupRpcMethod(methodName, parameterTypes);
+			Object result = rpcMethod.invoke(rpcEndpoint, args);
+
+			if (result != null && result instanceof Future) {
+				Future<?> future = (Future<?>) result;
+				return Await.result(future, futureTimeout.duration());
+			} else {
+				return result;
+			}
+		}
+
+		@Override
+		public void runAsync(Runnable runnable) {
+			runnable.run();
+		}
+
+		@Override
+		public <V> Future<V> callAsync(Callable<V> callable, Timeout callTimeout) {
+			try {
+				return Futures.successful(callable.call());
+			} catch (Throwable e) {
+				return Futures.failed(e);
+			}
+		}
+
+		@Override
+		public void scheduleRunAsync(final Runnable runnable, final long delay) {
+			try {
+				TimeUnit.MILLISECONDS.sleep(delay);
+				runnable.run();
+			} catch (Throwable e) {
+				throw new RuntimeException(e);
+			}
+		}
+
+		@Override
+		public String getAddress() {
+			return address;
+		}
+
+		@Override
+		public void start() {
+			// do nothing
+		}
+
+		@Override
+		public void stop() {
+			// do nothing
+		}
+
+		/**
+		 * Look up the rpc method on the given {@link RpcEndpoint} instance.
+		 *
+		 * @param methodName     Name of the method
+		 * @param parameterTypes Parameter types of the method
+		 * @return Method of the rpc endpoint
+		 * @throws NoSuchMethodException Thrown if the method with the given name and parameter types
+		 *                               cannot be found at the rpc endpoint
+		 */
+		private Method lookupRpcMethod(final String methodName,
+			final Class<?>[] parameterTypes) throws NoSuchMethodException {
+			return rpcEndpoint.getClass().getMethod(methodName, parameterTypes);
+		}
+
+		// ------------------------------------------------------------------------
+		//  Helper methods
+		// ------------------------------------------------------------------------
+
+		/**
+		 * Extracts the {@link RpcTimeout} annotated rpc timeout value from the list of given method
+		 * arguments. If no {@link RpcTimeout} annotated parameter could be found, then the default
+		 * timeout is returned.
+		 *
+		 * @param parameterAnnotations Parameter annotations
+		 * @param args                 Array of arguments
+		 * @param defaultTimeout       Default timeout to return if no {@link RpcTimeout} annotated parameter
+		 *                             has been found
+		 * @return Timeout extracted from the array of arguments or the default timeout
+		 */
+		private static Timeout extractRpcTimeout(Annotation[][] parameterAnnotations, Object[] args,
+			Timeout defaultTimeout) {
+			if (args != null) {
+				Preconditions.checkArgument(parameterAnnotations.length == args.length);
+
+				for (int i = 0; i < parameterAnnotations.length; i++) {
+					if (isRpcTimeout(parameterAnnotations[i])) {
+						if (args[i] instanceof FiniteDuration) {
+							return new Timeout((FiniteDuration) args[i]);
+						} else {
+							throw new RuntimeException("The rpc timeout parameter must be of type " +
+								FiniteDuration.class.getName() + ". The type " + args[i].getClass().getName() +
+								" is not supported.");
+						}
+					}
+				}
+			}
+
+			return defaultTimeout;
+		}
+
+		/**
+		 * Removes all {@link RpcTimeout} annotated parameters from the parameter type and argument
+		 * list.
+		 *
+		 * @param parameterTypes       Array of parameter types
+		 * @param parameterAnnotations Array of parameter annotations
+		 * @param args                 Arary of arguments
+		 * @return Tuple of filtered parameter types and arguments which no longer contain the
+		 * {@link RpcTimeout} annotated parameter types and arguments
+		 */
+		private static Tuple2<Class<?>[], Object[]> filterArguments(
+			Class<?>[] parameterTypes,
+			Annotation[][] parameterAnnotations,
+			Object[] args) {
+
+			Class<?>[] filteredParameterTypes;
+			Object[] filteredArgs;
+
+			if (args == null) {
+				filteredParameterTypes = parameterTypes;
+				filteredArgs = null;
+			} else {
+				Preconditions.checkArgument(parameterTypes.length == parameterAnnotations.length);
+				Preconditions.checkArgument(parameterAnnotations.length == args.length);
+
+				BitSet isRpcTimeoutParameter = new BitSet(parameterTypes.length);
+				int numberRpcParameters = parameterTypes.length;
+
+				for (int i = 0; i < parameterTypes.length; i++) {
+					if (isRpcTimeout(parameterAnnotations[i])) {
+						isRpcTimeoutParameter.set(i);
+						numberRpcParameters--;
+					}
+				}
+
+				if (numberRpcParameters == parameterTypes.length) {
+					filteredParameterTypes = parameterTypes;
+					filteredArgs = args;
+				} else {
+					filteredParameterTypes = new Class<?>[numberRpcParameters];
+					filteredArgs = new Object[numberRpcParameters];
+					int counter = 0;
+
+					for (int i = 0; i < parameterTypes.length; i++) {
+						if (!isRpcTimeoutParameter.get(i)) {
+							filteredParameterTypes[counter] = parameterTypes[i];
+							filteredArgs[counter] = args[i];
+							counter++;
+						}
+					}
+				}
+			}
+			return Tuple2.of(filteredParameterTypes, filteredArgs);
+		}
+
+		/**
+		 * Checks whether any of the annotations is of type {@link RpcTimeout}
+		 *
+		 * @param annotations Array of annotations
+		 * @return True if {@link RpcTimeout} was found; otherwise false
+		 */
+		private static boolean isRpcTimeout(Annotation[] annotations) {
+			for (Annotation annotation : annotations) {
+				if (annotation.annotationType().equals(RpcTimeout.class)) {
+					return true;
+				}
+			}
+
+			return false;
+		}
+
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4b077af4/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManagerHATest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManagerHATest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManagerHATest.java
new file mode 100644
index 0000000..dfffeda
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManagerHATest.java
@@ -0,0 +1,76 @@
+/*
+ * 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.resourcemanager;
+
+import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices;
+import org.apache.flink.runtime.leaderelection.TestingLeaderElectionService;
+import org.apache.flink.runtime.rpc.MainThreadExecutor;
+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.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
+ */
+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 execute runAsync call
+		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);
+
+		TestingLeaderElectionService leaderElectionService = new TestingLeaderElectionService();
+		TestingHighAvailabilityServices highAvailabilityServices = new TestingHighAvailabilityServices();
+		highAvailabilityServices.setResourceManagerLeaderElectionService(leaderElectionService);
+
+		final ResourceManager resourceManager = new ResourceManager(rpcService, highAvailabilityServices);
+		resourceManager.start();
+		// before grant leadership, resourceManager's leaderId is null
+		Assert.assertNull(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.assertNull(resourceManager.getLeaderSessionID());
+	}
+
+	private static abstract class TestingResourceManagerGatewayProxy implements MainThreadExecutor, StartStoppable, RpcGateway {
+		@Override
+		public void runAsync(Runnable runnable) {
+			runnable.run();
+		}
+	}
+
+}


[18/50] [abbrv] flink git commit: [hotfix] [taskmanager] Fixes TaskManager component creation at startup

Posted by se...@apache.org.
[hotfix] [taskmanager] Fixes TaskManager component creation at startup


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

Branch: refs/heads/flip-6
Commit: 9c07278a547c7231d9761a8841fa03dfb554c76c
Parents: 3be561f
Author: Till Rohrmann <tr...@apache.org>
Authored: Thu Sep 8 18:43:15 2016 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Oct 14 15:14:40 2016 +0200

----------------------------------------------------------------------
 .../runtime/taskexecutor/TaskExecutor.java      | 189 ++++++++++++++++---
 .../taskexecutor/TaskExecutorConfiguration.java |   9 -
 2 files changed, 159 insertions(+), 39 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/9c07278a/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 735730b..a455fe2 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
@@ -19,9 +19,19 @@
 package org.apache.flink.runtime.taskexecutor;
 
 import akka.actor.ActorSystem;
-import akka.dispatch.ExecutionContexts$;
 import akka.util.Timeout;
 import com.typesafe.config.Config;
+import org.apache.flink.runtime.io.network.ConnectionManager;
+import org.apache.flink.runtime.io.network.LocalConnectionManager;
+import org.apache.flink.runtime.io.network.TaskEventDispatcher;
+import org.apache.flink.runtime.io.network.buffer.NetworkBufferPool;
+import org.apache.flink.runtime.io.network.netty.NettyConnectionManager;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionManager;
+import org.apache.flink.runtime.query.KvStateRegistry;
+import org.apache.flink.runtime.query.netty.DisabledKvStateRequestStats;
+import org.apache.flink.runtime.query.netty.KvStateServer;
+import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
+import org.apache.flink.util.Preconditions;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -37,7 +47,6 @@ import org.apache.flink.runtime.akka.AkkaUtils;
 import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.runtime.clusterframework.types.ResourceID;
 import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
-import org.apache.flink.runtime.instance.InstanceConnectionInfo;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager.IOMode;
 import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
@@ -61,7 +70,6 @@ import org.apache.flink.util.NetUtils;
 import scala.Tuple2;
 import scala.Option;
 import scala.Some;
-import scala.concurrent.ExecutionContext;
 import scala.concurrent.duration.Duration;
 import scala.concurrent.duration.FiniteDuration;
 
@@ -70,9 +78,9 @@ import java.io.IOException;
 import java.net.InetAddress;
 import java.net.InetSocketAddress;
 import java.util.UUID;
-import java.util.concurrent.ForkJoinPool;
 import java.util.concurrent.TimeUnit;
 
+import static org.apache.flink.util.Preconditions.checkArgument;
 import static org.apache.flink.util.Preconditions.checkNotNull;
 
 /**
@@ -86,6 +94,8 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 	/** The unique resource ID of this TaskExecutor */
 	private final ResourceID resourceID;
 
+	private final TaskManagerLocation taskManagerLocation;
+
 	/** The access to the leader election and metadata storage services */
 	private final HighAvailabilityServices haServices;
 
@@ -113,22 +123,26 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 	public TaskExecutor(
 			TaskExecutorConfiguration taskExecutorConfig,
 			ResourceID resourceID,
+			TaskManagerLocation taskManagerLocation,
 			MemoryManager memoryManager,
 			IOManager ioManager,
 			NetworkEnvironment networkEnvironment,
-			int numberOfSlots,
 			RpcService rpcService,
 			HighAvailabilityServices haServices) {
 
 		super(rpcService);
 
+		checkArgument(taskExecutorConfig.getNumberOfSlots() > 0, "The number of slots has to be larger than 0.");
+
 		this.taskExecutorConfig = checkNotNull(taskExecutorConfig);
 		this.resourceID = checkNotNull(resourceID);
+		this.taskManagerLocation = checkNotNull(taskManagerLocation);
 		this.memoryManager = checkNotNull(memoryManager);
 		this.ioManager = checkNotNull(ioManager);
 		this.networkEnvironment = checkNotNull(networkEnvironment);
-		this.numberOfSlots = checkNotNull(numberOfSlots);
 		this.haServices = checkNotNull(haServices);
+
+		this.numberOfSlots = taskExecutorConfig.getNumberOfSlots();
 	}
 
 	// ------------------------------------------------------------------------
@@ -360,10 +374,10 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 	 *                                      then a HighAvailabilityServices is constructed from the configuration.
 	 * @param localTaskManagerCommunication     If true, the TaskManager will not initiate the TCP network stack.
 	 * @return An ActorRef to the TaskManager actor.
-	 * @throws org.apache.flink.configuration.IllegalConfigurationException     Thrown, if the given config contains illegal values.
-	 * @throws java.io.IOException      Thrown, if any of the I/O components (such as buffer pools,
+	 * @throws IllegalConfigurationException     Thrown, if the given config contains illegal values.
+	 * @throws IOException      Thrown, if any of the I/O components (such as buffer pools,
 	 *                                       I/O manager, ...) cannot be properly started.
-	 * @throws java.lang.Exception      Thrown is some other error occurs while parsing the configuration
+	 * @throws Exception      Thrown is some other error occurs while parsing the configuration
 	 *                                      or starting the TaskManager components.
 	 */
 	public static TaskExecutor startTaskManagerComponentsAndActor(
@@ -377,19 +391,105 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 		final TaskExecutorConfiguration taskExecutorConfig = parseTaskManagerConfiguration(
 			configuration, taskManagerHostname, localTaskManagerCommunication);
 
+		TaskManagerComponents taskManagerComponents = createTaskManagerComponents(
+			resourceID,
+			InetAddress.getByName(taskManagerHostname),
+			taskExecutorConfig,
+			configuration);
+
+		final TaskExecutor taskExecutor = new TaskExecutor(
+			taskExecutorConfig,
+			resourceID,
+			taskManagerComponents.getTaskManagerLocation(),
+			taskManagerComponents.getMemoryManager(),
+			taskManagerComponents.getIOManager(),
+			taskManagerComponents.getNetworkEnvironment(),
+			rpcService,
+			haServices);
+
+		return taskExecutor;
+	}
+
+	/**
+	 * Creates and returns the task manager components.
+	 *
+	 * @param resourceID resource ID of the task manager
+	 * @param taskManagerAddress address of the task manager
+	 * @param taskExecutorConfig task manager configuration
+	 * @param configuration of Flink
+	 * @return task manager components
+	 * @throws Exception
+	 */
+	private static TaskExecutor.TaskManagerComponents createTaskManagerComponents(
+		ResourceID resourceID,
+		InetAddress taskManagerAddress,
+		TaskExecutorConfiguration taskExecutorConfig,
+		Configuration configuration) throws Exception {
 		MemoryType memType = taskExecutorConfig.getNetworkConfig().memoryType();
 
 		// pre-start checks
 		checkTempDirs(taskExecutorConfig.getTmpDirPaths());
 
-		ExecutionContext executionContext = ExecutionContexts$.MODULE$.fromExecutor(new ForkJoinPool());
+		NetworkEnvironmentConfiguration networkEnvironmentConfiguration = taskExecutorConfig.getNetworkConfig();
+
+		NetworkBufferPool networkBufferPool = new NetworkBufferPool(
+			networkEnvironmentConfiguration.numNetworkBuffers(),
+			networkEnvironmentConfiguration.networkBufferSize(),
+			networkEnvironmentConfiguration.memoryType());
+
+		ConnectionManager connectionManager;
+
+		if (networkEnvironmentConfiguration.nettyConfig().isDefined()) {
+			connectionManager = new NettyConnectionManager(networkEnvironmentConfiguration.nettyConfig().get());
+		} else {
+			connectionManager = new LocalConnectionManager();
+		}
+
+		ResultPartitionManager resultPartitionManager = new ResultPartitionManager();
+		TaskEventDispatcher taskEventDispatcher = new TaskEventDispatcher();
+
+		KvStateRegistry kvStateRegistry = new KvStateRegistry();
+
+		KvStateServer kvStateServer;
+
+		if (networkEnvironmentConfiguration.nettyConfig().isDefined()) {
+			NettyConfig nettyConfig = networkEnvironmentConfiguration.nettyConfig().get();
+
+			int numNetworkThreads = networkEnvironmentConfiguration.queryServerNetworkThreads() == 0 ?
+				nettyConfig.getNumberOfSlots() : networkEnvironmentConfiguration.queryServerNetworkThreads();
+
+			int numQueryThreads = networkEnvironmentConfiguration.queryServerQueryThreads() == 0 ?
+				nettyConfig.getNumberOfSlots() : networkEnvironmentConfiguration.queryServerQueryThreads();
+
+			kvStateServer = new KvStateServer(
+				taskManagerAddress,
+				networkEnvironmentConfiguration.queryServerPort(),
+				numNetworkThreads,
+				numQueryThreads,
+				kvStateRegistry,
+				new DisabledKvStateRequestStats());
+		} else {
+			kvStateServer = null;
+		}
 
 		// we start the network first, to make sure it can allocate its buffers first
 		final NetworkEnvironment network = new NetworkEnvironment(
-			executionContext,
-			taskExecutorConfig.getTimeout(),
-			taskExecutorConfig.getNetworkConfig(),
-			taskExecutorConfig.getConnectionInfo());
+			networkBufferPool,
+			connectionManager,
+			resultPartitionManager,
+			taskEventDispatcher,
+			kvStateRegistry,
+			kvStateServer,
+			networkEnvironmentConfiguration.ioMode(),
+			networkEnvironmentConfiguration.partitionRequestInitialBackoff(),
+			networkEnvironmentConfiguration.partitinRequestMaxBackoff());
+
+		network.start();
+
+		TaskManagerLocation taskManagerLocation = new TaskManagerLocation(
+			resourceID,
+			taskManagerAddress,
+			network.getConnectionManager().getDataPort());
 
 		// computing the amount of memory to use depends on how much memory is available
 		// it strictly needs to happen AFTER the network stack has been initialized
@@ -473,17 +573,7 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 		// start the I/O manager, it will create some temp directories.
 		final IOManager ioManager = new IOManagerAsync(taskExecutorConfig.getTmpDirPaths());
 
-		final TaskExecutor taskExecutor = new TaskExecutor(
-			taskExecutorConfig,
-			resourceID,
-			memoryManager,
-			ioManager,
-			network,
-			taskExecutorConfig.getNumberOfSlots(),
-			rpcService,
-			haServices);
-
-		return taskExecutor;
+		return new TaskExecutor.TaskManagerComponents(taskManagerLocation, memoryManager, ioManager, network);
 	}
 
 	// --------------------------------------------------------------------------
@@ -519,7 +609,7 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 			"Leave config parameter empty or use 0 to let the system choose a port automatically.");
 
 		InetAddress taskManagerAddress = InetAddress.getByName(taskManagerHostname);
-		final InstanceConnectionInfo connectionInfo = new InstanceConnectionInfo(taskManagerAddress, dataport);
+		final InetSocketAddress taskManagerInetSocketAddress = new InetSocketAddress(taskManagerAddress, dataport);
 
 		// ----> memory / network stack (shuffles/broadcasts), task slots, temp directories
 
@@ -576,7 +666,12 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 
 		final NettyConfig nettyConfig;
 		if (!localTaskManagerCommunication) {
-			nettyConfig = new NettyConfig(connectionInfo.address(), connectionInfo.dataPort(), pageSize, slots, configuration);
+			nettyConfig = new NettyConfig(
+				taskManagerInetSocketAddress.getAddress(),
+				taskManagerInetSocketAddress.getPort(),
+				pageSize,
+				slots,
+				configuration);
 		} else {
 			nettyConfig = null;
 		}
@@ -613,8 +708,9 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 			queryServerPort,
 			queryServerNetworkThreads,
 			queryServerQueryThreads,
-			localTaskManagerCommunication ? Option.<NettyConfig>empty() : new Some<>(nettyConfig),
-			new Tuple2<>(500, 3000));
+			Option.apply(nettyConfig),
+			500,
+			30000);
 
 		// ----> timeouts, library caching, profiling
 
@@ -695,7 +791,6 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 		return new TaskExecutorConfiguration(
 			tmpDirs,
 			cleanupInterval,
-			connectionInfo,
 			networkConfig,
 			timeout,
 			finiteRegistrationDuration,
@@ -829,4 +924,38 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 			onFatalErrorAsync(exception);
 		}
 	}
+
+	private static class TaskManagerComponents {
+		private final TaskManagerLocation taskManagerLocation;
+		private final MemoryManager memoryManager;
+		private final IOManager ioManager;
+		private final NetworkEnvironment networkEnvironment;
+
+		private TaskManagerComponents(
+				TaskManagerLocation taskManagerLocation,
+				MemoryManager memoryManager,
+				IOManager ioManager,
+				NetworkEnvironment networkEnvironment) {
+			this.taskManagerLocation = Preconditions.checkNotNull(taskManagerLocation);
+			this.memoryManager = Preconditions.checkNotNull(memoryManager);
+			this.ioManager = Preconditions.checkNotNull(ioManager);
+			this.networkEnvironment = Preconditions.checkNotNull(networkEnvironment);
+		}
+
+		public MemoryManager getMemoryManager() {
+			return memoryManager;
+		}
+
+		public IOManager getIOManager() {
+			return ioManager;
+		}
+
+		public NetworkEnvironment getNetworkEnvironment() {
+			return networkEnvironment;
+		}
+
+		public TaskManagerLocation getTaskManagerLocation() {
+			return taskManagerLocation;
+		}
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/9c07278a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorConfiguration.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorConfiguration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorConfiguration.java
index 3707a47..c97c893 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorConfiguration.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorConfiguration.java
@@ -19,7 +19,6 @@
 package org.apache.flink.runtime.taskexecutor;
 
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.instance.InstanceConnectionInfo;
 import org.apache.flink.runtime.taskmanager.NetworkEnvironmentConfiguration;
 
 import scala.concurrent.duration.FiniteDuration;
@@ -52,12 +51,9 @@ public class TaskExecutorConfiguration implements Serializable {
 
 	private final NetworkEnvironmentConfiguration networkConfig;
 
-	private final InstanceConnectionInfo connectionInfo;
-
 	public TaskExecutorConfiguration(
 			String[] tmpDirPaths,
 			long cleanupInterval,
-			InstanceConnectionInfo connectionInfo,
 			NetworkEnvironmentConfiguration networkConfig,
 			FiniteDuration timeout,
 			FiniteDuration maxRegistrationDuration,
@@ -66,7 +62,6 @@ public class TaskExecutorConfiguration implements Serializable {
 
 		this (tmpDirPaths,
 			cleanupInterval,
-			connectionInfo,
 			networkConfig,
 			timeout,
 			maxRegistrationDuration,
@@ -80,7 +75,6 @@ public class TaskExecutorConfiguration implements Serializable {
 	public TaskExecutorConfiguration(
 			String[] tmpDirPaths,
 			long cleanupInterval,
-			InstanceConnectionInfo connectionInfo,
 			NetworkEnvironmentConfiguration networkConfig,
 			FiniteDuration timeout,
 			FiniteDuration maxRegistrationDuration,
@@ -92,7 +86,6 @@ public class TaskExecutorConfiguration implements Serializable {
 
 		this.tmpDirPaths = checkNotNull(tmpDirPaths);
 		this.cleanupInterval = checkNotNull(cleanupInterval);
-		this.connectionInfo = checkNotNull(connectionInfo);
 		this.networkConfig = checkNotNull(networkConfig);
 		this.timeout = checkNotNull(timeout);
 		this.maxRegistrationDuration = maxRegistrationDuration;
@@ -115,8 +108,6 @@ public class TaskExecutorConfiguration implements Serializable {
 		return cleanupInterval;
 	}
 
-	public InstanceConnectionInfo getConnectionInfo() { return connectionInfo; }
-
 	public NetworkEnvironmentConfiguration getNetworkConfig() { return networkConfig; }
 
 	public FiniteDuration getTimeout() {


[28/50] [abbrv] flink git commit: [FLINK-4687] [rpc] Add getAddress to RpcService

Posted by se...@apache.org.
[FLINK-4687] [rpc] Add getAddress to RpcService

This closes #2551.


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

Branch: refs/heads/flip-6
Commit: 07512e06acfc4bb3c48f1286ce52478c64ffb259
Parents: 59d9e67
Author: Till Rohrmann <tr...@apache.org>
Authored: Mon Sep 26 18:01:47 2016 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Oct 14 15:14:41 2016 +0200

----------------------------------------------------------------------
 .../org/apache/flink/runtime/rpc/RpcService.java    |  8 ++++++++
 .../flink/runtime/rpc/akka/AkkaRpcService.java      | 16 ++++++++++++++++
 .../apache/flink/runtime/rpc/TestingRpcService.java |  5 +++--
 .../flink/runtime/rpc/TestingSerialRpcService.java  |  6 ++++++
 .../flink/runtime/rpc/akka/AkkaRpcActorTest.java    |  8 ++++----
 .../flink/runtime/rpc/akka/AkkaRpcServiceTest.java  |  5 +++++
 6 files changed, 42 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/07512e06/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 437e08b..96844ed 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,6 +33,14 @@ 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 Address of the rpc service or empty string if local rpc service
+	 */
+	String getAddress();
+
+	/**
 	 * Connect to a remote rpc server under the provided address. Returns a rpc gateway which can
 	 * be used to communicate with the rpc server. If the connection failed, then the returned
 	 * future is failed with a {@link RpcConnectionException}.

http://git-wip-us.apache.org/repos/asf/flink/blob/07512e06/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 cee19c4..6825557 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
@@ -22,6 +22,7 @@ import akka.actor.ActorIdentity;
 import akka.actor.ActorRef;
 import akka.actor.ActorSelection;
 import akka.actor.ActorSystem;
+import akka.actor.Address;
 import akka.actor.Identify;
 import akka.actor.PoisonPill;
 import akka.actor.Props;
@@ -75,6 +76,8 @@ public class AkkaRpcService implements RpcService {
 	private final Set<ActorRef> actors = new HashSet<>(4);
 	private final long maximumFramesize;
 
+	private final String address;
+
 	private volatile boolean stopped;
 
 	public AkkaRpcService(final ActorSystem actorSystem, final Time timeout) {
@@ -87,6 +90,19 @@ public class AkkaRpcService implements RpcService {
 			// only local communication
 			maximumFramesize = Long.MAX_VALUE;
 		}
+
+		Address actorSystemAddress = AkkaUtils.getAddress(actorSystem);
+
+		if (actorSystemAddress.host().isDefined()) {
+			address = actorSystemAddress.host().get();
+		} else {
+			address = "";
+		}
+	}
+
+	@Override
+	public String getAddress() {
+		return address;
 	}
 
 	// this method does not mutate state and is thus thread-safe

http://git-wip-us.apache.org/repos/asf/flink/blob/07512e06/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingRpcService.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingRpcService.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingRpcService.java
index f164056..47c9e24 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingRpcService.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingRpcService.java
@@ -25,6 +25,7 @@ import org.apache.flink.runtime.concurrent.Future;
 import org.apache.flink.runtime.concurrent.impl.FlinkCompletableFuture;
 import org.apache.flink.runtime.rpc.akka.AkkaRpcService;
 
+import java.net.UnknownHostException;
 import java.util.concurrent.ConcurrentHashMap;
 
 import static org.apache.flink.util.Preconditions.checkNotNull;
@@ -57,14 +58,14 @@ public class TestingRpcService extends AkkaRpcService {
 	/**
 	 * Creates a new {@code TestingRpcService}. 
 	 */
-	public TestingRpcService() {
+	public TestingRpcService() throws UnknownHostException {
 		this(new Configuration());
 	}
 
 	/**
 	 * Creates a new {@code TestingRpcService}, using the given configuration. 
 	 */
-	public TestingRpcService(Configuration configuration) {
+	public TestingRpcService(Configuration configuration) throws UnknownHostException {
 		super(AkkaUtils.createLocalActorSystem(configuration), Time.seconds(10));
 
 		this.registeredConnections = new ConcurrentHashMap<>();

http://git-wip-us.apache.org/repos/asf/flink/blob/07512e06/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 c58ea20..5b8e6e6 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
@@ -30,6 +30,7 @@ import java.lang.annotation.Annotation;
 import java.lang.reflect.InvocationHandler;
 import java.lang.reflect.Method;
 import java.lang.reflect.Proxy;
+import java.net.InetAddress;
 import java.util.BitSet;
 import java.util.UUID;
 import java.util.concurrent.Callable;
@@ -121,6 +122,11 @@ public class TestingSerialRpcService implements RpcService {
 	}
 
 	@Override
+	public String getAddress() {
+		return "";
+	}
+
+	@Override
 	public <C extends RpcGateway> Future<C> connect(String address, Class<C> clazz) {
 		RpcGateway gateway = registeredConnections.get(address);
 

http://git-wip-us.apache.org/repos/asf/flink/blob/07512e06/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 1e8c9a6..5d76024 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
@@ -133,7 +133,7 @@ public class AkkaRpcActorTest extends TestLogger {
 
 		Future<WrongRpcGateway> futureGateway = akkaRpcService.connect(rpcEndpoint.getAddress(), WrongRpcGateway.class);
 
-		WrongRpcGateway gateway = Await.result(futureGateway, timeout.duration());
+		WrongRpcGateway gateway = futureGateway.get(timeout.getSize(), timeout.getUnit());
 
 		// since it is a tell operation we won't receive a RpcConnectionException, it's only logged
 		gateway.tell("foobar");
@@ -141,10 +141,10 @@ public class AkkaRpcActorTest extends TestLogger {
 		Future<Boolean> result = gateway.barfoo();
 
 		try {
-			Await.result(result, timeout.duration());
+			result.get(timeout.getSize(), timeout.getUnit());
 			fail("We expected a RpcConnectionException.");
-		} catch (RpcConnectionException rpcConnectionException) {
-			// we expect this exception here
+		} catch (ExecutionException executionException) {
+			assertTrue(executionException.getCause() instanceof RpcConnectionException);
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/07512e06/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java
index 5550cb5..3388011 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java
@@ -115,4 +115,9 @@ public class AkkaRpcServiceTest extends TestLogger {
 		assertEquals(expected, actual);
 		assertTrue(latch.isTriggered());
 	}
+
+	@Test
+	public void testGetAddress() {
+		assertEquals(AkkaUtils.getAddress(actorSystem).host().get(), akkaRpcService.getAddress());
+	}
 }


[12/50] [abbrv] flink git commit: [hotfix] Add methods defined in the gateway to the ResourceManager and TaskExecutor

Posted by se...@apache.org.
[hotfix] Add methods defined in the gateway to the ResourceManager and TaskExecutor


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

Branch: refs/heads/flip-6
Commit: fbd38671f33d2579cf2856c7f9b50ffc0335113f
Parents: b2c5c0f
Author: Till Rohrmann <tr...@apache.org>
Authored: Wed Sep 21 14:14:05 2016 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Oct 14 15:14:40 2016 +0200

----------------------------------------------------------------------
 .../runtime/resourcemanager/ResourceManager.java     |  1 +
 .../resourcemanager/ResourceManagerGateway.java      |  2 +-
 .../flink/runtime/taskexecutor/TaskExecutor.java     | 15 +++++++++++++++
 .../runtime/taskexecutor/TaskExecutorGateway.java    |  6 +++---
 4 files changed, 20 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/fbd38671/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 29aba1a..d9a7134 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
@@ -18,6 +18,7 @@
 
 package org.apache.flink.runtime.resourcemanager;
 
+import akka.dispatch.Futures;
 import akka.dispatch.Mapper;
 
 import org.apache.flink.annotation.VisibleForTesting;

http://git-wip-us.apache.org/repos/asf/flink/blob/fbd38671/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 e5c8b64..c8e3488 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
@@ -58,7 +58,7 @@ public interface ResourceManagerGateway extends RpcGateway {
 	 * @param slotRequest Slot request
 	 * @return Future slot assignment
 	 */
-	Future<SlotRequestRegistered> requestSlot(SlotRequest slotRequest);
+	Future<SlotRequestReply> requestSlot(SlotRequest slotRequest);
 
 	/**
 	 * 

http://git-wip-us.apache.org/repos/asf/flink/blob/fbd38671/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 a455fe2..fadae5f 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
@@ -21,6 +21,7 @@ package org.apache.flink.runtime.taskexecutor;
 import akka.actor.ActorSystem;
 import akka.util.Timeout;
 import com.typesafe.config.Config;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
 import org.apache.flink.runtime.io.network.ConnectionManager;
 import org.apache.flink.runtime.io.network.LocalConnectionManager;
 import org.apache.flink.runtime.io.network.TaskEventDispatcher;
@@ -30,6 +31,8 @@ import org.apache.flink.runtime.io.network.partition.ResultPartitionManager;
 import org.apache.flink.runtime.query.KvStateRegistry;
 import org.apache.flink.runtime.query.netty.DisabledKvStateRequestStats;
 import org.apache.flink.runtime.query.netty.KvStateServer;
+import org.apache.flink.runtime.resourcemanager.SlotRequestRegistered;
+import org.apache.flink.runtime.resourcemanager.SlotRequestReply;
 import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
 import org.apache.flink.util.Preconditions;
 import org.slf4j.Logger;
@@ -201,6 +204,18 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 	}
 
 	/**
+	 * Requests a slot from the TaskManager
+	 *
+	 * @param allocationID id for the request
+	 * @param resourceManagerLeaderID current leader id of the ResourceManager
+	 * @return answer to the slot request
+	 */
+	@RpcMethod
+	public SlotRequestReply requestSlot(AllocationID allocationID, UUID resourceManagerLeaderID) {
+		return new SlotRequestRegistered(allocationID);
+	}
+
+	/**
 	 * Starts and runs the TaskManager.
 	 * <p/>
 	 * This method first tries to select the network interface to use for the TaskManager

http://git-wip-us.apache.org/repos/asf/flink/blob/fbd38671/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 7257436..65323a8 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
@@ -39,12 +39,12 @@ public interface TaskExecutorGateway extends RpcGateway {
 	void notifyOfNewResourceManagerLeader(String address, UUID resourceManagerLeaderId);
 
 	/**
-	 * Send by the ResourceManager to the TaskExecutor
+	 * Requests a slot from the TaskManager
+	 *
 	 * @param allocationID id for the request
 	 * @param resourceManagerLeaderID current leader id of the ResourceManager
-	 * @return SlotRequestReply Answer to the request
+	 * @return answer to the slot request
 	 */
-
 	Future<SlotRequestReply> requestSlot(
 		AllocationID allocationID,
 		UUID resourceManagerLeaderID,


[21/50] [abbrv] flink git commit: [FLINK-4505] [cluster mngt] Separate TaskManager service configuration from TaskManagerConfiguration; Implement TaskManagerRunner

Posted by se...@apache.org.
[FLINK-4505] [cluster mngt] Separate TaskManager service configuration from TaskManagerConfiguration; Implement TaskManagerRunner

Refactors the startup logic so that is easier to reuse.

This closes #2461.


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

Branch: refs/heads/flip-6
Commit: bb781aef10b6c7099c83678a95c1d0db79cdbe3d
Parents: c34f13c
Author: Till Rohrmann <tr...@apache.org>
Authored: Wed Sep 21 12:33:15 2016 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Oct 14 15:14:41 2016 +0200

----------------------------------------------------------------------
 .../HighAvailabilityServicesUtils.java          |  41 +
 .../flink/runtime/rpc/RpcServiceUtils.java      |  73 ++
 .../flink/runtime/rpc/akka/AkkaRpcService.java  |   2 +
 .../runtime/taskexecutor/TaskExecutor.java      |  51 +-
 .../taskexecutor/TaskExecutorConfiguration.java | 142 ----
 .../taskexecutor/TaskManagerConfiguration.java  | 205 +++++
 .../runtime/taskexecutor/TaskManagerRunner.java | 172 +++++
 .../taskexecutor/TaskManagerServices.java       | 320 ++++++++
 .../TaskManagerServicesConfiguration.java       | 325 ++++++++
 .../runtime/taskmanager/TaskManagerRunner.java  | 749 -------------------
 .../runtime/util/LeaderRetrievalUtils.java      |   7 +
 .../apache/flink/runtime/akka/AkkaUtils.scala   |   4 +
 .../NetworkEnvironmentConfiguration.scala       |   2 +-
 .../flink/runtime/taskmanager/TaskManager.scala |   6 +-
 .../io/network/NetworkEnvironmentTest.java      |   4 +-
 .../runtime/rpc/TestingSerialRpcService.java    |   1 -
 .../runtime/taskexecutor/TaskExecutorTest.java  |  29 +-
 ...askManagerComponentsStartupShutdownTest.java |   3 +-
 .../TaskManagerConfigurationTest.java           |   1 -
 19 files changed, 1195 insertions(+), 942 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/bb781aef/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
new file mode 100644
index 0000000..f3da847
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServicesUtils.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.highavailability;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.jobmanager.HighAvailabilityMode;
+import org.apache.flink.runtime.util.LeaderRetrievalUtils;
+
+public class HighAvailabilityServicesUtils {
+
+	public static HighAvailabilityServices createHighAvailabilityServices(Configuration configuration) throws Exception {
+		HighAvailabilityMode highAvailabilityMode = LeaderRetrievalUtils.getRecoveryMode(configuration);
+
+		switch(highAvailabilityMode) {
+			case NONE:
+				final String resourceManagerAddress = null;
+				return new NonHaServices(resourceManagerAddress);
+			case ZOOKEEPER:
+				throw new UnsupportedOperationException("ZooKeeper high availability services " +
+					"have not been implemented yet.");
+			default:
+				throw new Exception("Recovery mode " + highAvailabilityMode + " is not supported.");
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/bb781aef/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
new file mode 100644
index 0000000..d40e336
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcServiceUtils.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.rpc;
+
+import akka.actor.ActorSystem;
+import com.typesafe.config.Config;
+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.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;
+
+public class RpcServiceUtils {
+	private static final Logger LOG = LoggerFactory.getLogger(RpcServiceUtils.class);
+
+	/**
+	 * Utility method to create RPC service from configuration and hostname, port.
+	 *
+	 * @param hostname   The hostname/address that describes the TaskManager's data location.
+	 * @param port           If true, the TaskManager will not initiate the TCP network stack.
+	 * @param configuration                 The configuration for the TaskManager.
+	 * @return   The rpc service which is used to start and connect to the TaskManager RpcEndpoint .
+	 * @throws IOException      Thrown, if the actor system can not bind to the address
+	 * @throws Exception      Thrown is some other error occurs while creating akka actor system
+	 */
+	public static RpcService createRpcService(String hostname, int port, Configuration configuration) throws Exception {
+		LOG.info("Starting AkkaRpcService at {}.", NetUtils.hostAndPortToUrlString(hostname, port));
+
+		final ActorSystem actorSystem;
+
+		try {
+			Config akkaConfig = AkkaUtils.getAkkaConfig(configuration, hostname, port);
+
+			LOG.debug("Using akka configuration \n {}.", akkaConfig);
+
+			actorSystem = AkkaUtils.createActorSystem(akkaConfig);
+		} catch (Throwable t) {
+			if (t instanceof ChannelException) {
+				Throwable cause = t.getCause();
+				if (cause != null && t.getCause() instanceof java.net.BindException) {
+					String address = NetUtils.hostAndPortToUrlString(hostname, port);
+					throw new IOException("Unable to bind AkkaRpcService actor system to address " +
+						address + " - " + cause.getMessage(), t);
+				}
+			}
+			throw new Exception("Could not create TaskManager actor system", t);
+		}
+
+		final Time timeout = Time.milliseconds(AkkaUtils.getTimeout(configuration).toMillis());
+		return new AkkaRpcService(actorSystem, timeout);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/bb781aef/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 6825557..fb7896a 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
@@ -93,6 +93,8 @@ 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/bb781aef/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 8ce2780..7df0a91 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,16 +18,14 @@
 
 package org.apache.flink.runtime.taskexecutor;
 
-import org.apache.flink.api.common.time.Time;
-import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory;
 import org.apache.flink.runtime.clusterframework.types.AllocationID;
-import org.apache.flink.runtime.jobmanager.SubmittedJobGraphStore;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.metrics.MetricRegistry;
 import org.apache.flink.runtime.resourcemanager.SlotRequestRegistered;
 import org.apache.flink.runtime.resourcemanager.SlotRequestReply;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
 import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
-import org.jboss.netty.channel.ChannelException;
 import org.apache.flink.annotation.VisibleForTesting;
-import org.apache.flink.runtime.clusterframework.types.ResourceID;
 import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.io.network.NetworkEnvironment;
@@ -39,7 +37,7 @@ import org.apache.flink.runtime.rpc.RpcService;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import java.net.BindException;
+
 import java.util.UUID;
 
 import static org.apache.flink.util.Preconditions.checkArgument;
@@ -60,7 +58,7 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 	private final HighAvailabilityServices haServices;
 
 	/** The task manager configuration */
-	private final TaskExecutorConfiguration taskExecutorConfig;
+	private final TaskManagerConfiguration taskManagerConfiguration;
 
 	/** The I/O manager component in the task manager */
 	private final IOManager ioManager;
@@ -71,9 +69,15 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 	/** The network component in the task manager */
 	private final NetworkEnvironment networkEnvironment;
 
+	/** The metric registry in the task manager */
+	private final MetricRegistry metricRegistry;
+
 	/** The number of slots in the task manager, should be 1 for YARN */
 	private final int numberOfSlots;
 
+	/** The fatal error handler to use in case of a fatal error */
+	private final FatalErrorHandler fatalErrorHandler;
+
 	// --------- resource manager --------
 
 	private TaskExecutorToResourceManagerConnection resourceManagerConnection;
@@ -81,26 +85,30 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 	// ------------------------------------------------------------------------
 
 	public TaskExecutor(
-		TaskExecutorConfiguration taskExecutorConfig,
+		TaskManagerConfiguration taskManagerConfiguration,
 		TaskManagerLocation taskManagerLocation,
 		RpcService rpcService,
 		MemoryManager memoryManager,
 		IOManager ioManager,
 		NetworkEnvironment networkEnvironment,
-		HighAvailabilityServices haServices) {
+		HighAvailabilityServices haServices,
+		MetricRegistry metricRegistry,
+		FatalErrorHandler fatalErrorHandler) {
 
 		super(rpcService);
 
-		checkArgument(taskExecutorConfig.getNumberOfSlots() > 0, "The number of slots has to be larger than 0.");
+		checkArgument(taskManagerConfiguration.getNumberSlots() > 0, "The number of slots has to be larger than 0.");
 
-		this.taskExecutorConfig = checkNotNull(taskExecutorConfig);
+		this.taskManagerConfiguration = checkNotNull(taskManagerConfiguration);
 		this.taskManagerLocation = checkNotNull(taskManagerLocation);
 		this.memoryManager = checkNotNull(memoryManager);
 		this.ioManager = checkNotNull(ioManager);
 		this.networkEnvironment = checkNotNull(networkEnvironment);
 		this.haServices = checkNotNull(haServices);
+		this.metricRegistry = checkNotNull(metricRegistry);
+		this.fatalErrorHandler = checkNotNull(fatalErrorHandler);
 
-		this.numberOfSlots =  taskExecutorConfig.getNumberOfSlots();
+		this.numberOfSlots =  taskManagerConfiguration.getNumberSlots();
 	}
 
 	// ------------------------------------------------------------------------
@@ -158,6 +166,7 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 		}
 	}
 
+	/**
 	 * Requests a slot from the TaskManager
 	 *
 	 * @param allocationID id for the request
@@ -169,22 +178,6 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 		return new SlotRequestRegistered(allocationID);
 	}
 
-	/**
-			public LeaderRetrievalService getJobMasterLeaderRetriever(JobID jobID) throws Exception {
-				return null;
-			}
-
-			@Override
-				return null;
-			}
-
-			@Override
-			public CheckpointRecoveryFactory getCheckpointRecoveryFactory() throws Exception {
-				return null;
-			}
-
-			@Override
-			public SubmittedJobGraphStore getSubmittedJobGraphStore() throws Exception {
 	// ------------------------------------------------------------------------
 	//  Properties
 	// ------------------------------------------------------------------------
@@ -222,7 +215,7 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 	void onFatalError(Throwable t) {
 		// to be determined, probably delegate to a fatal error handler that 
 		// would either log (mini cluster) ot kill the process (yarn, mesos, ...)
-		log.error("FATAL ERROR", t);
+		fatalErrorHandler.onFatalError(t);
 	}
 
 	// ------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/flink/blob/bb781aef/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorConfiguration.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorConfiguration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorConfiguration.java
deleted file mode 100644
index c97c893..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorConfiguration.java
+++ /dev/null
@@ -1,142 +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.taskexecutor;
-
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.taskmanager.NetworkEnvironmentConfiguration;
-
-import scala.concurrent.duration.FiniteDuration;
-
-import java.io.Serializable;
-import java.util.concurrent.TimeUnit;
-
-import static org.apache.flink.util.Preconditions.checkNotNull;
-
-/**
- * {@link TaskExecutor} Configuration
- */
-public class TaskExecutorConfiguration implements Serializable {
-
-	private static final long serialVersionUID = 1L;
-
-	private final String[] tmpDirPaths;
-
-	private final long cleanupInterval;
-
-	private final int numberOfSlots;
-
-	private final Configuration configuration;
-
-	private final FiniteDuration timeout;
-	private final FiniteDuration maxRegistrationDuration;
-	private final FiniteDuration initialRegistrationPause;
-	private final FiniteDuration maxRegistrationPause;
-	private final FiniteDuration refusedRegistrationPause;
-
-	private final NetworkEnvironmentConfiguration networkConfig;
-
-	public TaskExecutorConfiguration(
-			String[] tmpDirPaths,
-			long cleanupInterval,
-			NetworkEnvironmentConfiguration networkConfig,
-			FiniteDuration timeout,
-			FiniteDuration maxRegistrationDuration,
-			int numberOfSlots,
-			Configuration configuration) {
-
-		this (tmpDirPaths,
-			cleanupInterval,
-			networkConfig,
-			timeout,
-			maxRegistrationDuration,
-			numberOfSlots,
-			configuration,
-			new FiniteDuration(500, TimeUnit.MILLISECONDS),
-			new FiniteDuration(30, TimeUnit.SECONDS),
-			new FiniteDuration(10, TimeUnit.SECONDS));
-	}
-
-	public TaskExecutorConfiguration(
-			String[] tmpDirPaths,
-			long cleanupInterval,
-			NetworkEnvironmentConfiguration networkConfig,
-			FiniteDuration timeout,
-			FiniteDuration maxRegistrationDuration,
-			int numberOfSlots,
-			Configuration configuration,
-			FiniteDuration initialRegistrationPause,
-			FiniteDuration maxRegistrationPause,
-			FiniteDuration refusedRegistrationPause) {
-
-		this.tmpDirPaths = checkNotNull(tmpDirPaths);
-		this.cleanupInterval = checkNotNull(cleanupInterval);
-		this.networkConfig = checkNotNull(networkConfig);
-		this.timeout = checkNotNull(timeout);
-		this.maxRegistrationDuration = maxRegistrationDuration;
-		this.numberOfSlots = checkNotNull(numberOfSlots);
-		this.configuration = checkNotNull(configuration);
-		this.initialRegistrationPause = checkNotNull(initialRegistrationPause);
-		this.maxRegistrationPause = checkNotNull(maxRegistrationPause);
-		this.refusedRegistrationPause = checkNotNull(refusedRegistrationPause);
-	}
-
-	// --------------------------------------------------------------------------------------------
-	//  Properties
-	// --------------------------------------------------------------------------------------------
-
-	public String[] getTmpDirPaths() {
-		return tmpDirPaths;
-	}
-
-	public long getCleanupInterval() {
-		return cleanupInterval;
-	}
-
-	public NetworkEnvironmentConfiguration getNetworkConfig() { return networkConfig; }
-
-	public FiniteDuration getTimeout() {
-		return timeout;
-	}
-
-	public FiniteDuration getMaxRegistrationDuration() {
-		return maxRegistrationDuration;
-	}
-
-	public int getNumberOfSlots() {
-		return numberOfSlots;
-	}
-
-	public Configuration getConfiguration() {
-		return configuration;
-	}
-
-	public FiniteDuration getInitialRegistrationPause() {
-		return initialRegistrationPause;
-	}
-
-	public FiniteDuration getMaxRegistrationPause() {
-		return maxRegistrationPause;
-	}
-
-	public FiniteDuration getRefusedRegistrationPause() {
-		return refusedRegistrationPause;
-	}
-
-}
-

http://git-wip-us.apache.org/repos/asf/flink/blob/bb781aef/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerConfiguration.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerConfiguration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerConfiguration.java
new file mode 100644
index 0000000..32eb8c1
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerConfiguration.java
@@ -0,0 +1,205 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.taskexecutor;
+
+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.util.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.concurrent.duration.Duration;
+
+import java.io.File;
+
+/**
+ * Configuration object for {@link TaskExecutor}.
+ */
+public class TaskManagerConfiguration {
+
+	private static final Logger LOG = LoggerFactory.getLogger(TaskManagerConfiguration.class);
+
+	private final int numberSlots;
+
+	private final String[] tmpDirPaths;
+
+	private final Time timeout;
+	private final Time maxRegistrationDuration;
+	private final Time initialRegistrationPause;
+	private final Time maxRegistrationPause;
+	private final Time refusedRegistrationPause;
+
+	private final long cleanupInterval;
+
+	public TaskManagerConfiguration(
+		int numberSlots,
+		String[] tmpDirPaths,
+		Time timeout,
+		Time maxRegistrationDuration,
+		Time initialRegistrationPause,
+		Time maxRegistrationPause,
+		Time refusedRegistrationPause,
+		long cleanupInterval) {
+
+		this.numberSlots = numberSlots;
+		this.tmpDirPaths = Preconditions.checkNotNull(tmpDirPaths);
+		this.timeout = Preconditions.checkNotNull(timeout);
+		this.maxRegistrationDuration = Preconditions.checkNotNull(maxRegistrationDuration);
+		this.initialRegistrationPause = Preconditions.checkNotNull(initialRegistrationPause);
+		this.maxRegistrationPause = Preconditions.checkNotNull(maxRegistrationPause);
+		this.refusedRegistrationPause = Preconditions.checkNotNull(refusedRegistrationPause);
+		this.cleanupInterval = Preconditions.checkNotNull(cleanupInterval);
+	}
+
+	public int getNumberSlots() {
+		return numberSlots;
+	}
+
+	public String[] getTmpDirPaths() {
+		return tmpDirPaths;
+	}
+
+	public Time getTimeout() {
+		return timeout;
+	}
+
+	public Time getMaxRegistrationDuration() {
+		return maxRegistrationDuration;
+	}
+
+	public Time getInitialRegistrationPause() {
+		return initialRegistrationPause;
+	}
+
+	public Time getMaxRegistrationPause() {
+		return maxRegistrationPause;
+	}
+
+	public Time getRefusedRegistrationPause() {
+		return refusedRegistrationPause;
+	}
+
+	public long getCleanupInterval() {
+		return cleanupInterval;
+	}
+
+	// --------------------------------------------------------------------------------------------
+	//  Static factory methods
+	// --------------------------------------------------------------------------------------------
+
+	public static TaskManagerConfiguration fromConfiguration(Configuration configuration) {
+		int numberSlots = configuration.getInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 1);
+
+		if (numberSlots == -1) {
+			numberSlots = 1;
+		}
+
+		final String[] tmpDirPaths = configuration.getString(
+			ConfigConstants.TASK_MANAGER_TMP_DIR_KEY,
+			ConfigConstants.DEFAULT_TASK_MANAGER_TMP_PATH).split(",|" + File.pathSeparator);
+
+		final Time timeout;
+
+		try {
+			timeout = Time.milliseconds(AkkaUtils.getTimeout(configuration).toMillis());
+		} catch (Exception e) {
+			throw new IllegalArgumentException(
+				"Invalid format for '" + ConfigConstants.AKKA_ASK_TIMEOUT +
+					"'.Use formats like '50 s' or '1 min' to specify the timeout.");
+		}
+
+		LOG.info("Messages have a max timeout of " + timeout);
+
+		final long cleanupInterval = configuration.getLong(
+			ConfigConstants.LIBRARY_CACHE_MANAGER_CLEANUP_INTERVAL,
+			ConfigConstants.DEFAULT_LIBRARY_CACHE_MANAGER_CLEANUP_INTERVAL) * 1000;
+
+		final Time finiteRegistrationDuration;
+
+		try {
+			Duration maxRegistrationDuration = Duration.create(configuration.getString(
+				ConfigConstants.TASK_MANAGER_MAX_REGISTRATION_DURATION,
+				ConfigConstants.DEFAULT_TASK_MANAGER_MAX_REGISTRATION_DURATION));
+			if (maxRegistrationDuration.isFinite()) {
+				finiteRegistrationDuration = Time.seconds(maxRegistrationDuration.toSeconds());
+			} else {
+				finiteRegistrationDuration = null;
+			}
+		} catch (NumberFormatException e) {
+			throw new IllegalArgumentException("Invalid format for parameter " +
+				ConfigConstants.TASK_MANAGER_MAX_REGISTRATION_DURATION, e);
+		}
+
+		final Time initialRegistrationPause;
+		try {
+			Duration pause = Duration.create(configuration.getString(
+				ConfigConstants.TASK_MANAGER_INITIAL_REGISTRATION_PAUSE,
+				ConfigConstants.DEFAULT_TASK_MANAGER_INITIAL_REGISTRATION_PAUSE));
+			if (pause.isFinite()) {
+				initialRegistrationPause = Time.seconds(pause.toSeconds());
+			} else {
+				throw new IllegalArgumentException("The initial registration pause must be finite: " + pause);
+			}
+		} catch (NumberFormatException e) {
+			throw new IllegalArgumentException("Invalid format for parameter " +
+				ConfigConstants.TASK_MANAGER_INITIAL_REGISTRATION_PAUSE, e);
+		}
+
+		final Time maxRegistrationPause;
+		try {
+			Duration pause = Duration.create(configuration.getString(
+				ConfigConstants.TASK_MANAGER_MAX_REGISTARTION_PAUSE,
+				ConfigConstants.DEFAULT_TASK_MANAGER_MAX_REGISTRATION_PAUSE));
+			if (pause.isFinite()) {
+				maxRegistrationPause = Time.seconds(pause.toSeconds());
+			} else {
+				throw new IllegalArgumentException("The maximum registration pause must be finite: " + pause);
+			}
+		} catch (NumberFormatException e) {
+			throw new IllegalArgumentException("Invalid format for parameter " +
+				ConfigConstants.TASK_MANAGER_INITIAL_REGISTRATION_PAUSE, e);
+		}
+
+		final Time refusedRegistrationPause;
+		try {
+			Duration pause = Duration.create(configuration.getString(
+				ConfigConstants.TASK_MANAGER_REFUSED_REGISTRATION_PAUSE,
+				ConfigConstants.DEFAULT_TASK_MANAGER_REFUSED_REGISTRATION_PAUSE));
+			if (pause.isFinite()) {
+				refusedRegistrationPause = Time.seconds(pause.toSeconds());
+			} else {
+				throw new IllegalArgumentException("The refused registration pause must be finite: " + pause);
+			}
+		} catch (NumberFormatException e) {
+			throw new IllegalArgumentException("Invalid format for parameter " +
+				ConfigConstants.TASK_MANAGER_INITIAL_REGISTRATION_PAUSE, e);
+		}
+
+		return new TaskManagerConfiguration(
+			numberSlots,
+			tmpDirPaths,
+			timeout,
+			finiteRegistrationDuration,
+			initialRegistrationPause,
+			maxRegistrationPause,
+			refusedRegistrationPause,
+			cleanupInterval);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/bb781aef/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
new file mode 100644
index 0000000..8ac0ddd
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.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.runtime.taskexecutor;
+
+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.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
+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.util.LeaderRetrievalUtils;
+
+import org.apache.flink.util.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.net.InetAddress;
+import java.util.concurrent.Executor;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * This class is the executable entry point for the task manager in yarn or standalone mode.
+ * It constructs the related components (network, I/O manager, memory manager, RPC service, HA service)
+ * and starts them.
+ */
+public class TaskManagerRunner implements FatalErrorHandler {
+
+	private static final Logger LOG = LoggerFactory.getLogger(TaskManagerRunner.class);
+
+	private final Object lock = new Object();
+
+	private final Configuration configuration;
+
+	private final ResourceID resourceID;
+
+	private final RpcService rpcService;
+
+	private final HighAvailabilityServices highAvailabilityServices;
+
+	/** Executor used to run future callbacks */
+	private final Executor executor;
+
+	private final TaskExecutor taskManager;
+
+	public TaskManagerRunner(
+		Configuration configuration,
+		ResourceID resourceID,
+		RpcService rpcService,
+		HighAvailabilityServices highAvailabilityServices,
+		Executor executor) throws Exception {
+
+		this.configuration = Preconditions.checkNotNull(configuration);
+		this.resourceID = Preconditions.checkNotNull(resourceID);
+		this.rpcService = Preconditions.checkNotNull(rpcService);
+		this.highAvailabilityServices = Preconditions.checkNotNull(highAvailabilityServices);
+		this.executor = rpcService.getExecutor();
+
+		InetAddress remoteAddress = InetAddress.getByName(rpcService.getAddress());
+
+		TaskManagerServicesConfiguration taskManagerServicesConfiguration = TaskManagerServicesConfiguration.fromConfiguration(
+			configuration,
+			remoteAddress,
+			false);
+
+		TaskManagerServices taskManagerServices = TaskManagerServices.fromConfiguration(taskManagerServicesConfiguration, resourceID);
+
+		TaskManagerConfiguration taskManagerConfiguration = TaskManagerConfiguration.fromConfiguration(configuration);
+
+		this.taskManager = new TaskExecutor(
+			taskManagerConfiguration,
+			taskManagerServices.getTaskManagerLocation(),
+			rpcService,
+			taskManagerServices.getMemoryManager(),
+			taskManagerServices.getIOManager(),
+			taskManagerServices.getNetworkEnvironment(),
+			highAvailabilityServices,
+			taskManagerServices.getMetricRegistry(),
+			this);
+	}
+
+	// --------------------------------------------------------------------------------------------
+	//  Lifecycle management
+	// --------------------------------------------------------------------------------------------
+
+	public void start() {
+		taskManager.start();
+	}
+
+	public void shutDown(Throwable cause) {
+		shutDownInternally();
+	}
+
+	protected void shutDownInternally() {
+		synchronized(lock) {
+			taskManager.shutDown();
+		}
+	}
+
+	// --------------------------------------------------------------------------------------------
+	//  FatalErrorHandler methods
+	// --------------------------------------------------------------------------------------------
+
+	@Override
+	public void onFatalError(Throwable exception) {
+		LOG.error("Fatal error occurred while executing the TaskManager. Shutting it down...", exception);
+		shutDown(exception);
+	}
+
+	// --------------------------------------------------------------------------------------------
+	//  Static utilities
+	// --------------------------------------------------------------------------------------------
+
+	/**
+	 * Create a RPC service for the task manager.
+	 *
+	 * @param configuration The configuration for the TaskManager.
+	 * @param haServices to use for the task manager hostname retrieval
+	 */
+	public static RpcService createRpcService(
+		final Configuration configuration,
+		final HighAvailabilityServices haServices) throws Exception {
+
+		checkNotNull(configuration);
+		checkNotNull(haServices);
+
+		String taskManagerHostname = configuration.getString(ConfigConstants.TASK_MANAGER_HOSTNAME_KEY, null);
+
+		if (taskManagerHostname != null) {
+			LOG.info("Using configured hostname/address for TaskManager: {}.", taskManagerHostname);
+		} else {
+			Time lookupTimeout = Time.milliseconds(AkkaUtils.getLookupTimeout(configuration).toMillis());
+
+			InetAddress taskManagerAddress = LeaderRetrievalUtils.findConnectingAddress(
+				haServices.getResourceManagerLeaderRetriever(),
+				lookupTimeout);
+
+			taskManagerHostname = taskManagerAddress.getHostName();
+
+			LOG.info("TaskManager will use hostname/address '{}' ({}) for communication.",
+				taskManagerHostname, taskManagerAddress.getHostAddress());
+		}
+
+		final int rpcPort = configuration.getInteger(ConfigConstants.TASK_MANAGER_IPC_PORT_KEY, 0);
+
+		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);
+
+		return RpcServiceUtils.createRpcService(taskManagerHostname, rpcPort, configuration);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/bb781aef/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
new file mode 100644
index 0000000..ff7f7d5
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServices.java
@@ -0,0 +1,320 @@
+/*
+ * 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.core.memory.MemoryType;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.io.disk.iomanager.IOManager;
+import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
+import org.apache.flink.runtime.io.network.ConnectionManager;
+import org.apache.flink.runtime.io.network.LocalConnectionManager;
+import org.apache.flink.runtime.io.network.NetworkEnvironment;
+import org.apache.flink.runtime.io.network.TaskEventDispatcher;
+import org.apache.flink.runtime.io.network.buffer.NetworkBufferPool;
+import org.apache.flink.runtime.io.network.netty.NettyConfig;
+import org.apache.flink.runtime.io.network.netty.NettyConnectionManager;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionManager;
+import org.apache.flink.runtime.memory.MemoryManager;
+import org.apache.flink.runtime.metrics.MetricRegistry;
+import org.apache.flink.runtime.query.KvStateRegistry;
+import org.apache.flink.runtime.query.netty.DisabledKvStateRequestStats;
+import org.apache.flink.runtime.query.netty.KvStateServer;
+import org.apache.flink.runtime.taskmanager.NetworkEnvironmentConfiguration;
+import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
+import org.apache.flink.runtime.util.EnvironmentInformation;
+import org.apache.flink.util.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+
+/**
+ * Container for {@link TaskExecutor} services such as the {@link MemoryManager}, {@link IOManager},
+ * {@link NetworkEnvironment} and the {@link MetricRegistry}.
+ */
+public class TaskManagerServices {
+	private static final Logger LOG = LoggerFactory.getLogger(TaskManagerServices.class);
+
+	/** TaskManager services */
+	private final TaskManagerLocation taskManagerLocation;
+	private final MemoryManager memoryManager;
+	private final IOManager ioManager;
+	private final NetworkEnvironment networkEnvironment;
+	private final MetricRegistry metricRegistry;
+
+	private TaskManagerServices(
+		TaskManagerLocation taskManagerLocation,
+		MemoryManager memoryManager,
+		IOManager ioManager,
+		NetworkEnvironment networkEnvironment,
+		MetricRegistry metricRegistry) {
+
+		this.taskManagerLocation = Preconditions.checkNotNull(taskManagerLocation);
+		this.memoryManager = Preconditions.checkNotNull(memoryManager);
+		this.ioManager = Preconditions.checkNotNull(ioManager);
+		this.networkEnvironment = Preconditions.checkNotNull(networkEnvironment);
+		this.metricRegistry = Preconditions.checkNotNull(metricRegistry);
+	}
+
+	// --------------------------------------------------------------------------------------------
+	//  Getter/Setter
+	// --------------------------------------------------------------------------------------------
+
+	public MemoryManager getMemoryManager() {
+		return memoryManager;
+	}
+
+	public IOManager getIOManager() {
+		return ioManager;
+	}
+
+	public NetworkEnvironment getNetworkEnvironment() {
+		return networkEnvironment;
+	}
+
+	public TaskManagerLocation getTaskManagerLocation() {
+		return taskManagerLocation;
+	}
+
+	public MetricRegistry getMetricRegistry() {
+		return metricRegistry;
+	}
+
+	// --------------------------------------------------------------------------------------------
+	//  Static factory methods for task manager services
+	// --------------------------------------------------------------------------------------------
+
+	/**
+	 * Creates and returns the task manager services.
+	 *
+	 * @param resourceID resource ID of the task manager
+	 * @param taskManagerServicesConfiguration task manager configuration
+	 * @return task manager components
+	 * @throws Exception
+	 */
+	public static TaskManagerServices fromConfiguration(
+		TaskManagerServicesConfiguration taskManagerServicesConfiguration,
+		ResourceID resourceID) throws Exception {
+
+		final NetworkEnvironment network = createNetworkEnvironment(taskManagerServicesConfiguration);
+
+		network.start();
+
+		final TaskManagerLocation taskManagerLocation = new TaskManagerLocation(
+			resourceID,
+			taskManagerServicesConfiguration.getTaskManagerAddress(),
+			network.getConnectionManager().getDataPort());
+
+		// this call has to happen strictly after the network stack has been initialized
+		final MemoryManager memoryManager = createMemoryManager(taskManagerServicesConfiguration);
+
+		// start the I/O manager, it will create some temp directories.
+		final IOManager ioManager = new IOManagerAsync(taskManagerServicesConfiguration.getTmpDirPaths());
+
+		MetricRegistry metricsRegistry = new MetricRegistry(taskManagerServicesConfiguration.getMetricRegistryConfiguration());
+
+		return new TaskManagerServices(taskManagerLocation, memoryManager, ioManager, network, metricsRegistry);
+	}
+
+	/**
+	 * Creates a {@link MemoryManager} from the given {@link TaskManagerServicesConfiguration}.
+	 *
+	 * @param taskManagerServicesConfiguration to create the memory manager from
+	 * @return Memory manager
+	 * @throws Exception
+	 */
+	private static MemoryManager createMemoryManager(TaskManagerServicesConfiguration taskManagerServicesConfiguration) throws Exception {
+		// computing the amount of memory to use depends on how much memory is available
+		// it strictly needs to happen AFTER the network stack has been initialized
+
+		MemoryType memType = taskManagerServicesConfiguration.getNetworkConfig().memoryType();
+
+		// check if a value has been configured
+		long configuredMemory = taskManagerServicesConfiguration.getConfiguredMemory();
+
+		final long memorySize;
+
+		boolean preAllocateMemory = taskManagerServicesConfiguration.isPreAllocateMemory();
+
+		if (configuredMemory > 0) {
+			if (preAllocateMemory) {
+				LOG.info("Using {} MB for managed memory." , configuredMemory);
+			} else {
+				LOG.info("Limiting managed memory to {} MB, memory will be allocated lazily." , configuredMemory);
+			}
+			memorySize = configuredMemory << 20; // megabytes to bytes
+		} else {
+			float memoryFraction = taskManagerServicesConfiguration.getMemoryFraction();
+
+			if (memType == MemoryType.HEAP) {
+				long relativeMemSize = (long) (EnvironmentInformation.getSizeOfFreeHeapMemoryWithDefrag() * memoryFraction);
+				if (preAllocateMemory) {
+					LOG.info("Using {} of the currently free heap space for managed heap memory ({} MB)." ,
+						memoryFraction , relativeMemSize >> 20);
+				} else {
+					LOG.info("Limiting managed memory to {} of the currently free heap space ({} MB), " +
+						"memory will be allocated lazily." , memoryFraction , relativeMemSize >> 20);
+				}
+				memorySize = relativeMemSize;
+			} else if (memType == MemoryType.OFF_HEAP) {
+				// The maximum heap memory has been adjusted according to the fraction
+				long maxMemory = EnvironmentInformation.getMaxJvmHeapMemory();
+				long directMemorySize = (long) (maxMemory / (1.0 - memoryFraction) * memoryFraction);
+				if (preAllocateMemory) {
+					LOG.info("Using {} of the maximum memory size for managed off-heap memory ({} MB)." ,
+						memoryFraction, directMemorySize >> 20);
+				} else {
+					LOG.info("Limiting managed memory to {} of the maximum memory size ({} MB)," +
+						" memory will be allocated lazily.", memoryFraction, directMemorySize >> 20);
+				}
+				memorySize = directMemorySize;
+			} else {
+				throw new RuntimeException("No supported memory type detected.");
+			}
+		}
+
+		// now start the memory manager
+		final MemoryManager memoryManager;
+		try {
+			memoryManager = new MemoryManager(
+				memorySize,
+				taskManagerServicesConfiguration.getNumberOfSlots(),
+				taskManagerServicesConfiguration.getNetworkConfig().networkBufferSize(),
+				memType,
+				preAllocateMemory);
+		} catch (OutOfMemoryError e) {
+			if (memType == MemoryType.HEAP) {
+				throw new Exception("OutOfMemory error (" + e.getMessage() +
+					") while allocating the TaskManager heap memory (" + memorySize + " bytes).", e);
+			} else if (memType == MemoryType.OFF_HEAP) {
+				throw new Exception("OutOfMemory error (" + e.getMessage() +
+					") while allocating the TaskManager off-heap memory (" + memorySize +
+					" bytes).Try increasing the maximum direct memory (-XX:MaxDirectMemorySize)", e);
+			} else {
+				throw e;
+			}
+		}
+		return memoryManager;
+	}
+
+	/**
+	 * Creates the {@link NetworkEnvironment} from the given {@link TaskManagerServicesConfiguration}.
+	 *
+	 * @param taskManagerServicesConfiguration to construct the network environment from
+	 * @return Network environment
+	 * @throws IOException
+	 */
+	private static NetworkEnvironment createNetworkEnvironment(TaskManagerServicesConfiguration taskManagerServicesConfiguration) throws IOException {
+		// pre-start checks
+		checkTempDirs(taskManagerServicesConfiguration.getTmpDirPaths());
+
+		NetworkEnvironmentConfiguration networkEnvironmentConfiguration = taskManagerServicesConfiguration.getNetworkConfig();
+
+		NetworkBufferPool networkBufferPool = new NetworkBufferPool(
+			networkEnvironmentConfiguration.numNetworkBuffers(),
+			networkEnvironmentConfiguration.networkBufferSize(),
+			networkEnvironmentConfiguration.memoryType());
+
+		ConnectionManager connectionManager;
+
+		if (networkEnvironmentConfiguration.nettyConfig() != null) {
+			connectionManager = new NettyConnectionManager(networkEnvironmentConfiguration.nettyConfig());
+		} else {
+			connectionManager = new LocalConnectionManager();
+		}
+
+		ResultPartitionManager resultPartitionManager = new ResultPartitionManager();
+		TaskEventDispatcher taskEventDispatcher = new TaskEventDispatcher();
+
+		KvStateRegistry kvStateRegistry = new KvStateRegistry();
+
+		KvStateServer kvStateServer;
+
+		if (networkEnvironmentConfiguration.nettyConfig() != null) {
+			NettyConfig nettyConfig = networkEnvironmentConfiguration.nettyConfig();
+
+			int numNetworkThreads = networkEnvironmentConfiguration.queryServerNetworkThreads() == 0 ?
+				nettyConfig.getNumberOfSlots() : networkEnvironmentConfiguration.queryServerNetworkThreads();
+
+			int numQueryThreads = networkEnvironmentConfiguration.queryServerQueryThreads() == 0 ?
+				nettyConfig.getNumberOfSlots() : networkEnvironmentConfiguration.queryServerQueryThreads();
+
+			kvStateServer = new KvStateServer(
+				taskManagerServicesConfiguration.getTaskManagerAddress(),
+				networkEnvironmentConfiguration.queryServerPort(),
+				numNetworkThreads,
+				numQueryThreads,
+				kvStateRegistry,
+				new DisabledKvStateRequestStats());
+		} else {
+			kvStateServer = null;
+		}
+
+		// we start the network first, to make sure it can allocate its buffers first
+		final NetworkEnvironment network = new NetworkEnvironment(
+			networkBufferPool,
+			connectionManager,
+			resultPartitionManager,
+			taskEventDispatcher,
+			kvStateRegistry,
+			kvStateServer,
+			networkEnvironmentConfiguration.ioMode(),
+			networkEnvironmentConfiguration.partitionRequestInitialBackoff(),
+			networkEnvironmentConfiguration.partitinRequestMaxBackoff());
+
+		return network;
+	}
+
+	/**
+	 * Validates that all the directories denoted by the strings do actually exist, are proper
+	 * directories (not files), and are writable.
+	 *
+	 * @param tmpDirs       The array of directory paths to check.
+	 * @throws Exception    Thrown if any of the directories does not exist or is not writable
+	 *                   or is a file, rather than a directory.
+	 */
+	private static void checkTempDirs(String[] tmpDirs) throws IOException {
+		for (String dir : tmpDirs) {
+			if (dir != null && !dir.equals("")) {
+				File file = new File(dir);
+				if (!file.exists()) {
+					throw new IOException("Temporary file directory " + file.getAbsolutePath() + " does not exist.");
+				}
+				if (!file.isDirectory()) {
+					throw new IOException("Temporary file directory " + file.getAbsolutePath() + " is not a directory.");
+				}
+				if (!file.canWrite()) {
+					throw new IOException("Temporary file directory " + file.getAbsolutePath() + " is not writable.");
+				}
+
+				if (LOG.isInfoEnabled()) {
+					long totalSpaceGb = file.getTotalSpace() >> 30;
+					long usableSpaceGb = file.getUsableSpace() >> 30;
+					double usablePercentage = (double)usableSpaceGb / totalSpaceGb * 100;
+					String path = file.getAbsolutePath();
+					LOG.info(String.format("Temporary file directory '%s': total %d GB, " + "usable %d GB (%.2f%% usable)",
+						path, totalSpaceGb, usableSpaceGb, usablePercentage));
+				}
+			} else {
+				throw new IllegalArgumentException("Temporary file directory #$id is null.");
+			}
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/bb781aef/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
new file mode 100644
index 0000000..66d969a
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesConfiguration.java
@@ -0,0 +1,325 @@
+/*
+ * 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.configuration.ConfigConstants;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.IllegalConfigurationException;
+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.io.disk.iomanager.IOManager;
+import org.apache.flink.runtime.io.network.netty.NettyConfig;
+import org.apache.flink.runtime.memory.MemoryManager;
+import org.apache.flink.runtime.metrics.MetricRegistryConfiguration;
+import org.apache.flink.runtime.taskmanager.NetworkEnvironmentConfiguration;
+import org.apache.flink.util.MathUtils;
+
+import java.io.File;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Configuration for the task manager services such as the network environment, the memory manager,
+ * the io manager and the metric registry
+ */
+public class TaskManagerServicesConfiguration {
+
+	private final InetAddress taskManagerAddress;
+
+	private final String[] tmpDirPaths;
+
+	private final int numberOfSlots;
+
+	private final NetworkEnvironmentConfiguration networkConfig;
+
+	private final long configuredMemory;
+
+	private final boolean preAllocateMemory;
+
+	private final float memoryFraction;
+
+	private final MetricRegistryConfiguration metricRegistryConfiguration;
+
+	public TaskManagerServicesConfiguration(
+		InetAddress taskManagerAddress,
+		String[] tmpDirPaths,
+		NetworkEnvironmentConfiguration networkConfig,
+		int numberOfSlots,
+		long configuredMemory,
+		boolean preAllocateMemory,
+		float memoryFraction,
+		MetricRegistryConfiguration metricRegistryConfiguration) {
+
+		this.taskManagerAddress = checkNotNull(taskManagerAddress);
+		this.tmpDirPaths = checkNotNull(tmpDirPaths);
+		this.networkConfig = checkNotNull(networkConfig);
+		this.numberOfSlots = checkNotNull(numberOfSlots);
+
+		this.configuredMemory = configuredMemory;
+		this.preAllocateMemory = preAllocateMemory;
+		this.memoryFraction = memoryFraction;
+
+		this.metricRegistryConfiguration = checkNotNull(metricRegistryConfiguration);
+	}
+
+	// --------------------------------------------------------------------------------------------
+	//  Getter/Setter
+	// --------------------------------------------------------------------------------------------
+
+
+	public InetAddress getTaskManagerAddress() {
+		return taskManagerAddress;
+	}
+
+	public String[] getTmpDirPaths() {
+		return tmpDirPaths;
+	}
+
+	public NetworkEnvironmentConfiguration getNetworkConfig() { return networkConfig; }
+
+	public int getNumberOfSlots() {
+		return numberOfSlots;
+	}
+
+	public float getMemoryFraction() {
+		return memoryFraction;
+	}
+
+	public long getConfiguredMemory() {
+		return configuredMemory;
+	}
+
+	public boolean isPreAllocateMemory() {
+		return preAllocateMemory;
+	}
+
+	public MetricRegistryConfiguration getMetricRegistryConfiguration() {
+		return metricRegistryConfiguration;
+	}
+
+	// --------------------------------------------------------------------------------------------
+	//  Parsing of Flink configuration
+	// --------------------------------------------------------------------------------------------
+
+	/**
+	 * Utility method to extract TaskManager config parameters from the configuration and to
+	 * sanity check them.
+	 *
+	 * @param configuration The configuration.
+	 * @param remoteAddress identifying the IP address under which the TaskManager will be accessible
+	 * @param localCommunication True, to skip initializing the network stack.
+	 *                                      Use only in cases where only one task manager runs.
+	 * @return TaskExecutorConfiguration that wrappers InstanceConnectionInfo, NetworkEnvironmentConfiguration, etc.
+	 */
+	public static TaskManagerServicesConfiguration fromConfiguration(
+		Configuration configuration,
+		InetAddress remoteAddress,
+		boolean localCommunication) throws Exception {
+
+		// we need this because many configs have been written with a "-1" entry
+		int slots = configuration.getInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 1);
+		if (slots == -1) {
+			slots = 1;
+		}
+
+		final String[] tmpDirs = configuration.getString(
+			ConfigConstants.TASK_MANAGER_TMP_DIR_KEY,
+			ConfigConstants.DEFAULT_TASK_MANAGER_TMP_PATH).split(",|" + File.pathSeparator);
+
+		final NetworkEnvironmentConfiguration networkConfig = parseNetworkEnvironmentConfiguration(
+			configuration,
+			localCommunication,
+			remoteAddress,
+			slots);
+
+		// extract memory settings
+		long configuredMemory = configuration.getLong(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, -1L);
+		checkConfigParameter(configuredMemory == -1 || configuredMemory > 0, configuredMemory,
+			ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY,
+			"MemoryManager needs at least one MB of memory. " +
+				"If you leave this config parameter empty, the system automatically " +
+				"pick a fraction of the available memory.");
+
+		boolean preAllocateMemory = configuration.getBoolean(
+			ConfigConstants.TASK_MANAGER_MEMORY_PRE_ALLOCATE_KEY,
+			ConfigConstants.DEFAULT_TASK_MANAGER_MEMORY_PRE_ALLOCATE);
+
+		float memoryFraction = configuration.getFloat(
+			ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY,
+			ConfigConstants.DEFAULT_MEMORY_MANAGER_MEMORY_FRACTION);
+		checkConfigParameter(memoryFraction > 0.0f && memoryFraction < 1.0f, memoryFraction,
+			ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY,
+			"MemoryManager fraction of the free memory must be between 0.0 and 1.0");
+
+		final MetricRegistryConfiguration metricRegistryConfiguration = MetricRegistryConfiguration.fromConfiguration(configuration);
+
+		return new TaskManagerServicesConfiguration(
+			remoteAddress,
+			tmpDirs,
+			networkConfig,
+			slots,
+			configuredMemory,
+			preAllocateMemory,
+			memoryFraction,
+			metricRegistryConfiguration);
+	}
+
+	// --------------------------------------------------------------------------
+	//  Parsing and checking the TaskManager Configuration
+	// --------------------------------------------------------------------------
+
+	/**
+	 * Creates the {@link NetworkEnvironmentConfiguration} from the given {@link Configuration}.
+	 *
+	 * @param configuration to create the network environment configuration from
+	 * @param localTaskManagerCommunication true if task manager communication is local
+	 * @param taskManagerAddress address of the task manager
+	 * @param slots to start the task manager with
+	 * @return Network environment configuration
+	 */
+	private static NetworkEnvironmentConfiguration parseNetworkEnvironmentConfiguration(
+		Configuration configuration,
+		boolean localTaskManagerCommunication,
+		InetAddress taskManagerAddress,
+		int slots) throws Exception {
+
+		// ----> hosts / ports for communication and data exchange
+
+		int dataport = configuration.getInteger(ConfigConstants.TASK_MANAGER_DATA_PORT_KEY,
+			ConfigConstants.DEFAULT_TASK_MANAGER_DATA_PORT);
+
+		checkConfigParameter(dataport > 0, dataport, ConfigConstants.TASK_MANAGER_DATA_PORT_KEY,
+			"Leave config parameter empty or use 0 to let the system choose a port automatically.");
+
+		checkConfigParameter(slots >= 1, slots, ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS,
+			"Number of task slots must be at least one.");
+
+		final int numNetworkBuffers = configuration.getInteger(
+			ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY,
+			ConfigConstants.DEFAULT_TASK_MANAGER_NETWORK_NUM_BUFFERS);
+
+		checkConfigParameter(numNetworkBuffers > 0, numNetworkBuffers,
+			ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY, "");
+
+		final int pageSize = configuration.getInteger(
+			ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY,
+			ConfigConstants.DEFAULT_TASK_MANAGER_MEMORY_SEGMENT_SIZE);
+
+		// check page size of for minimum size
+		checkConfigParameter(pageSize >= MemoryManager.MIN_PAGE_SIZE, pageSize,
+			ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY,
+			"Minimum memory segment size is " + MemoryManager.MIN_PAGE_SIZE);
+
+		// check page size for power of two
+		checkConfigParameter(MathUtils.isPowerOf2(pageSize), pageSize,
+			ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY,
+			"Memory segment size must be a power of 2.");
+
+		// check whether we use heap or off-heap memory
+		final MemoryType memType;
+		if (configuration.getBoolean(ConfigConstants.TASK_MANAGER_MEMORY_OFF_HEAP_KEY, false)) {
+			memType = MemoryType.OFF_HEAP;
+		} else {
+			memType = MemoryType.HEAP;
+		}
+
+		// initialize the memory segment factory accordingly
+		if (memType == MemoryType.HEAP) {
+			if (!MemorySegmentFactory.initializeIfNotInitialized(HeapMemorySegment.FACTORY)) {
+				throw new Exception("Memory type is set to heap memory, but memory segment " +
+					"factory has been initialized for off-heap memory segments");
+			}
+		} else {
+			if (!MemorySegmentFactory.initializeIfNotInitialized(HybridMemorySegment.FACTORY)) {
+				throw new Exception("Memory type is set to off-heap memory, but memory segment " +
+					"factory has been initialized for heap memory segments");
+			}
+		}
+
+		final NettyConfig nettyConfig;
+		if (!localTaskManagerCommunication) {
+			final InetSocketAddress taskManagerInetSocketAddress = new InetSocketAddress(taskManagerAddress, dataport);
+
+			nettyConfig = new NettyConfig(taskManagerInetSocketAddress.getAddress(),
+				taskManagerInetSocketAddress.getPort(), pageSize, slots, configuration);
+		} else {
+			nettyConfig = null;
+		}
+
+		// Default spill I/O mode for intermediate results
+		final String syncOrAsync = configuration.getString(
+			ConfigConstants.TASK_MANAGER_NETWORK_DEFAULT_IO_MODE,
+			ConfigConstants.DEFAULT_TASK_MANAGER_NETWORK_DEFAULT_IO_MODE);
+
+		final IOManager.IOMode ioMode;
+		if (syncOrAsync.equals("async")) {
+			ioMode = IOManager.IOMode.ASYNC;
+		} else {
+			ioMode = IOManager.IOMode.SYNC;
+		}
+
+		final int queryServerPort =  configuration.getInteger(
+			ConfigConstants.QUERYABLE_STATE_SERVER_PORT,
+			ConfigConstants.DEFAULT_QUERYABLE_STATE_SERVER_PORT);
+
+		final int queryServerNetworkThreads =  configuration.getInteger(
+			ConfigConstants.QUERYABLE_STATE_SERVER_NETWORK_THREADS,
+			ConfigConstants.DEFAULT_QUERYABLE_STATE_SERVER_NETWORK_THREADS);
+
+		final int queryServerQueryThreads =  configuration.getInteger(
+			ConfigConstants.QUERYABLE_STATE_SERVER_QUERY_THREADS,
+			ConfigConstants.DEFAULT_QUERYABLE_STATE_SERVER_QUERY_THREADS);
+
+		return new NetworkEnvironmentConfiguration(
+			numNetworkBuffers,
+			pageSize,
+			memType,
+			ioMode,
+			queryServerPort,
+			queryServerNetworkThreads,
+			queryServerQueryThreads,
+			nettyConfig,
+			500,
+			3000);
+	}
+
+	/**
+	 * Validates a condition for a config parameter and displays a standard exception, if the
+	 * the condition does not hold.
+	 *
+	 * @param condition             The condition that must hold. If the condition is false, an exception is thrown.
+	 * @param parameter         The parameter value. Will be shown in the exception message.
+	 * @param name              The name of the config parameter. Will be shown in the exception message.
+	 * @param errorMessage  The optional custom error message to append to the exception message.
+	 */
+	private static void checkConfigParameter(
+		boolean condition,
+		Object parameter,
+		String name,
+		String errorMessage) {
+		if (!condition) {
+			throw new IllegalConfigurationException("Invalid configuration value for " + name + " : " + parameter + " - " + errorMessage);
+		}
+	}
+}
+


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

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


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

Branch: refs/heads/flip-6
Commit: 7aca811df96ee0628fc4d274971b3ffc6d4b6eb7
Parents: 0615b62
Author: Kurt Young <yk...@gmail.com>
Authored: Thu Oct 13 04:59:46 2016 +0800
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Oct 14 15:32:27 2016 +0200

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


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

http://git-wip-us.apache.org/repos/asf/flink/blob/7aca811d/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotDescriptor.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotDescriptor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotDescriptor.java
new file mode 100644
index 0000000..be7cf96
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotDescriptor.java
@@ -0,0 +1,161 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.instance;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * The description of slots, TaskManagers offer one or more task slots, which define a slice of
+ * their resources. This description will contain some static information about the slot, such
+ * as the location and numeric id of the slot, rpc gateway to communicate with the TaskManager which
+ * owns the slot.
+ */
+public class SlotDescriptor {
+
+	/** The ID of the job this slice belongs to. */
+	private final JobID jobID;
+
+	/** The location information of the TaskManager to which this slot belongs */
+	private final TaskManagerLocation taskManagerLocation;
+
+	/** The number of the slot on which the task is deployed */
+	private final int slotNumber;
+
+	/** The resource profile of the slot provides */
+	private final ResourceProfile resourceProfile;
+
+	/** TEMP until the new RPC is in place: The actor gateway to communicate with the TaskManager */
+	private final ActorGateway taskManagerActorGateway;
+
+	public SlotDescriptor(
+		final JobID jobID,
+		final TaskManagerLocation location,
+		final int slotNumber,
+		final ResourceProfile resourceProfile,
+		final ActorGateway actorGateway)
+	{
+		this.jobID = checkNotNull(jobID);
+		this.taskManagerLocation = checkNotNull(location);
+		this.slotNumber = slotNumber;
+		this.resourceProfile = checkNotNull(resourceProfile);
+		this.taskManagerActorGateway = checkNotNull(actorGateway);
+	}
+
+	public SlotDescriptor(final SlotDescriptor other) {
+		this.jobID = other.jobID;
+		this.taskManagerLocation = other.taskManagerLocation;
+		this.slotNumber = other.slotNumber;
+		this.resourceProfile = other.resourceProfile;
+		this.taskManagerActorGateway = other.taskManagerActorGateway;
+	}
+	
+	// TODO - temporary workaround until we have the SlotDesriptor in the Slot
+	public SlotDescriptor(final Slot slot) {
+		this.jobID = slot.getJobID();
+		this.taskManagerLocation = slot.getTaskManagerLocation();
+		this.slotNumber = slot.getRootSlotNumber();
+		this.resourceProfile = new ResourceProfile(0, 0);
+		this.taskManagerActorGateway = slot.getTaskManagerActorGateway();
+	}
+
+	/**
+	 * Returns the ID of the job this allocated slot belongs to.
+	 *
+	 * @return the ID of the job this allocated slot belongs to
+	 */
+	public JobID getJobID() {
+		return jobID;
+	}
+
+	/**
+	 * Gets the number of the slot.
+	 *
+	 * @return The number of the slot on the TaskManager.
+	 */
+	public int getSlotNumber() {
+		return slotNumber;
+	}
+
+	/**
+	 * Gets the resource profile of the slot.
+	 *
+	 * @return The resource profile of the slot.
+	 */
+	public ResourceProfile getResourceProfile() {
+		return resourceProfile;
+	}
+
+	/**
+	 * Gets the location info of the TaskManager that offers this slot.
+	 *
+	 * @return The location info of the TaskManager that offers this slot
+	 */
+	public TaskManagerLocation getTaskManagerLocation() {
+		return taskManagerLocation;
+	}
+
+	/**
+	 * Gets the actor gateway that can be used to send messages to the TaskManager.
+	 * <p>
+	 * This method should be removed once the new interface-based RPC abstraction is in place
+	 *
+	 * @return The actor gateway that can be used to send messages to the TaskManager.
+	 */
+	public ActorGateway getTaskManagerActorGateway() {
+		return taskManagerActorGateway;
+	}
+
+	@Override
+	public boolean equals(Object o) {
+		if (this == o) {
+			return true;
+		}
+		if (o == null || getClass() != o.getClass()) {
+			return false;
+		}
+
+		SlotDescriptor that = (SlotDescriptor) o;
+
+		if (slotNumber != that.slotNumber) {
+			return false;
+		}
+		if (!jobID.equals(that.jobID)) {
+			return false;
+		}
+		return taskManagerLocation.equals(that.taskManagerLocation);
+
+	}
+
+	@Override
+	public int hashCode() {
+		int result = jobID.hashCode();
+		result = 31 * result + taskManagerLocation.hashCode();
+		result = 31 * result + slotNumber;
+		return result;
+	}
+
+	@Override
+	public String toString() {
+		return taskManagerLocation + " - " + slotNumber;
+	}
+}

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

http://git-wip-us.apache.org/repos/asf/flink/blob/7aca811d/flink-runtime/src/main/scala/org/apache/flink/runtime/akka/AkkaUtils.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/akka/AkkaUtils.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/akka/AkkaUtils.scala
index 9209d15..2461340 100644
--- a/flink-runtime/src/main/scala/org/apache/flink/runtime/akka/AkkaUtils.scala
+++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/akka/AkkaUtils.scala
@@ -583,8 +583,8 @@ object AkkaUtils {
   }
 
   def formatDurationParingErrorMessage: String = {
-    "Duration format must be \"val unit\", where 'val' is a number and 'unit' is " + 
-      "(d|day)|(h|hour)|(min|minute)|s|sec|second)|(ms|milli|millisecond)|"+
+    "Duration format must be \"val unit\", where 'val' is a number and 'unit' is " +
+      "(d|day)|(h|hour)|(min|minute)|s|sec|second)|(ms|milli|millisecond)|" +
       "(�s|micro|microsecond)|(ns|nano|nanosecond)"
   }
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/7aca811d/flink-runtime/src/test/java/org/apache/flink/runtime/instance/AllocatedSlotsTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/AllocatedSlotsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/AllocatedSlotsTest.java
new file mode 100644
index 0000000..655a3ea
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/AllocatedSlotsTest.java
@@ -0,0 +1,135 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.instance;
+
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+public class AllocatedSlotsTest {
+
+	@Test
+	public void testOperations() throws Exception {
+		SlotPool.AllocatedSlots allocatedSlots = new SlotPool.AllocatedSlots();
+
+		final AllocationID allocation1 = new AllocationID();
+		final ResourceID resource1 = new ResourceID("resource1");
+		final Slot slot1 = createSlot(resource1);
+
+		allocatedSlots.add(allocation1, new SlotDescriptor(slot1), slot1);
+
+		assertTrue(allocatedSlots.contains(slot1));
+		assertTrue(allocatedSlots.containResource(resource1));
+
+		assertEquals(slot1, allocatedSlots.get(allocation1));
+		assertEquals(1, allocatedSlots.getSlotsByResource(resource1).size());
+		assertEquals(1, allocatedSlots.size());
+
+		final AllocationID allocation2 = new AllocationID();
+		final Slot slot2 = createSlot(resource1);
+
+		allocatedSlots.add(allocation2, new SlotDescriptor(slot2), slot2);
+
+		assertTrue(allocatedSlots.contains(slot1));
+		assertTrue(allocatedSlots.contains(slot2));
+		assertTrue(allocatedSlots.containResource(resource1));
+
+		assertEquals(slot1, allocatedSlots.get(allocation1));
+		assertEquals(slot2, allocatedSlots.get(allocation2));
+		assertEquals(2, allocatedSlots.getSlotsByResource(resource1).size());
+		assertEquals(2, allocatedSlots.size());
+
+		final AllocationID allocation3 = new AllocationID();
+		final ResourceID resource2 = new ResourceID("resource2");
+		final Slot slot3 = createSlot(resource2);
+
+		allocatedSlots.add(allocation3, new SlotDescriptor(slot2), slot3);
+
+		assertTrue(allocatedSlots.contains(slot1));
+		assertTrue(allocatedSlots.contains(slot2));
+		assertTrue(allocatedSlots.contains(slot3));
+		assertTrue(allocatedSlots.containResource(resource1));
+		assertTrue(allocatedSlots.containResource(resource2));
+
+		assertEquals(slot1, allocatedSlots.get(allocation1));
+		assertEquals(slot2, allocatedSlots.get(allocation2));
+		assertEquals(slot3, allocatedSlots.get(allocation3));
+		assertEquals(2, allocatedSlots.getSlotsByResource(resource1).size());
+		assertEquals(1, allocatedSlots.getSlotsByResource(resource2).size());
+		assertEquals(3, allocatedSlots.size());
+
+		allocatedSlots.remove(slot2);
+
+		assertTrue(allocatedSlots.contains(slot1));
+		assertFalse(allocatedSlots.contains(slot2));
+		assertTrue(allocatedSlots.contains(slot3));
+		assertTrue(allocatedSlots.containResource(resource1));
+		assertTrue(allocatedSlots.containResource(resource2));
+
+		assertEquals(slot1, allocatedSlots.get(allocation1));
+		assertNull(allocatedSlots.get(allocation2));
+		assertEquals(slot3, allocatedSlots.get(allocation3));
+		assertEquals(1, allocatedSlots.getSlotsByResource(resource1).size());
+		assertEquals(1, allocatedSlots.getSlotsByResource(resource2).size());
+		assertEquals(2, allocatedSlots.size());
+
+		allocatedSlots.remove(slot1);
+
+		assertFalse(allocatedSlots.contains(slot1));
+		assertFalse(allocatedSlots.contains(slot2));
+		assertTrue(allocatedSlots.contains(slot3));
+		assertFalse(allocatedSlots.containResource(resource1));
+		assertTrue(allocatedSlots.containResource(resource2));
+
+		assertNull(allocatedSlots.get(allocation1));
+		assertNull(allocatedSlots.get(allocation2));
+		assertEquals(slot3, allocatedSlots.get(allocation3));
+		assertEquals(0, allocatedSlots.getSlotsByResource(resource1).size());
+		assertEquals(1, allocatedSlots.getSlotsByResource(resource2).size());
+		assertEquals(1, allocatedSlots.size());
+
+		allocatedSlots.remove(slot3);
+
+		assertFalse(allocatedSlots.contains(slot1));
+		assertFalse(allocatedSlots.contains(slot2));
+		assertFalse(allocatedSlots.contains(slot3));
+		assertFalse(allocatedSlots.containResource(resource1));
+		assertFalse(allocatedSlots.containResource(resource2));
+
+		assertNull(allocatedSlots.get(allocation1));
+		assertNull(allocatedSlots.get(allocation2));
+		assertNull(allocatedSlots.get(allocation3));
+		assertEquals(0, allocatedSlots.getSlotsByResource(resource1).size());
+		assertEquals(0, allocatedSlots.getSlotsByResource(resource2).size());
+		assertEquals(0, allocatedSlots.size());
+	}
+
+	private Slot createSlot(final ResourceID resourceId) {
+		Slot slot = mock(Slot.class);
+		when(slot.getTaskManagerID()).thenReturn(resourceId);
+		return slot;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/7aca811d/flink-runtime/src/test/java/org/apache/flink/runtime/instance/AvailableSlotsTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/AvailableSlotsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/AvailableSlotsTest.java
new file mode 100644
index 0000000..872810f
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/AvailableSlotsTest.java
@@ -0,0 +1,123 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.instance;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+public class AvailableSlotsTest {
+
+	static final ResourceProfile DEFAULT_TESTING_PROFILE = new ResourceProfile(1.0, 512);
+
+	static final ResourceProfile DEFAULT_TESTING_BIG_PROFILE = new ResourceProfile(2.0, 1024);
+
+	@Test
+	public void testAddAndRemove() throws Exception {
+		SlotPool.AvailableSlots availableSlots = new SlotPool.AvailableSlots();
+
+		final ResourceID resource1 = new ResourceID("resource1");
+		final ResourceID resource2 = new ResourceID("resource2");
+
+		final SlotDescriptor slot1 = createSlotDescriptor(resource1);
+		final SlotDescriptor slot2 = createSlotDescriptor(resource1);
+		final SlotDescriptor slot3 = createSlotDescriptor(resource2);
+
+		availableSlots.add(slot1);
+		availableSlots.add(slot2);
+		availableSlots.add(slot3);
+
+		assertEquals(3, availableSlots.size());
+		assertTrue(availableSlots.contains(slot1));
+		assertTrue(availableSlots.contains(slot2));
+		assertTrue(availableSlots.contains(slot3));
+		assertTrue(availableSlots.containResource(resource1));
+		assertTrue(availableSlots.containResource(resource2));
+
+		availableSlots.removeByResource(resource1);
+
+		assertEquals(1, availableSlots.size());
+		assertFalse(availableSlots.contains(slot1));
+		assertFalse(availableSlots.contains(slot2));
+		assertTrue(availableSlots.contains(slot3));
+		assertFalse(availableSlots.containResource(resource1));
+		assertTrue(availableSlots.containResource(resource2));
+
+		availableSlots.removeByResource(resource2);
+
+		assertEquals(0, availableSlots.size());
+		assertFalse(availableSlots.contains(slot1));
+		assertFalse(availableSlots.contains(slot2));
+		assertFalse(availableSlots.contains(slot3));
+		assertFalse(availableSlots.containResource(resource1));
+		assertFalse(availableSlots.containResource(resource2));
+	}
+
+	@Test
+	public void testPollFreeSlot() {
+		SlotPool.AvailableSlots availableSlots = new SlotPool.AvailableSlots();
+
+		final ResourceID resource1 = new ResourceID("resource1");
+		final SlotDescriptor slot1 = createSlotDescriptor(resource1);
+
+		availableSlots.add(slot1);
+
+		assertEquals(1, availableSlots.size());
+		assertTrue(availableSlots.contains(slot1));
+		assertTrue(availableSlots.containResource(resource1));
+
+		assertNull(availableSlots.poll(DEFAULT_TESTING_BIG_PROFILE));
+
+		assertEquals(slot1, availableSlots.poll(DEFAULT_TESTING_PROFILE));
+		assertEquals(0, availableSlots.size());
+		assertFalse(availableSlots.contains(slot1));
+		assertFalse(availableSlots.containResource(resource1));
+	}
+
+	static SlotDescriptor createSlotDescriptor(final ResourceID resourceID) {
+		return createSlotDescriptor(resourceID, new JobID());
+	}
+
+	static SlotDescriptor createSlotDescriptor(final ResourceID resourceID, final JobID jobID) {
+		return createSlotDescriptor(resourceID, jobID, DEFAULT_TESTING_PROFILE);
+	}
+
+	static SlotDescriptor createSlotDescriptor(final ResourceID resourceID, final JobID jobID,
+		final ResourceProfile resourceProfile)
+	{
+		return createSlotDescriptor(resourceID, jobID, resourceProfile, 0);
+	}
+
+	static SlotDescriptor createSlotDescriptor(final ResourceID resourceID, final JobID jobID,
+		final ResourceProfile resourceProfile, final int slotNumber)
+	{
+		TaskManagerLocation location = mock(TaskManagerLocation.class);
+		when(location.getResourceID()).thenReturn(resourceID);
+		return new SlotDescriptor(jobID, location, slotNumber, resourceProfile, mock(ActorGateway.class));
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/7aca811d/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SlotPoolTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SlotPoolTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SlotPoolTest.java
new file mode 100644
index 0000000..30cdbd6
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SlotPoolTest.java
@@ -0,0 +1,297 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.instance;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.concurrent.BiFunction;
+import org.apache.flink.runtime.concurrent.Future;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway;
+import org.apache.flink.runtime.resourcemanager.SlotRequest;
+import org.apache.flink.util.TestLogger;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.UUID;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.flink.runtime.instance.AvailableSlotsTest.DEFAULT_TESTING_PROFILE;
+import static org.apache.flink.runtime.instance.AvailableSlotsTest.createSlotDescriptor;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+public class SlotPoolTest extends TestLogger {
+
+	private Executor executor;
+
+	private SlotPool slotPool;
+
+	private ResourceManagerGateway resourceManagerGateway;
+
+	@Before
+	public void setUp() throws Exception {
+		this.executor = Executors.newFixedThreadPool(1);
+		this.slotPool = new SlotPool(executor);
+		this.resourceManagerGateway = mock(ResourceManagerGateway.class);
+		when(resourceManagerGateway
+			.requestSlot(any(UUID.class), any(UUID.class), any(SlotRequest.class), any(Time.class)))
+			.thenReturn(mock(Future.class));
+		slotPool.setResourceManager(UUID.randomUUID(), resourceManagerGateway);
+		slotPool.setJobManagerLeaderId(UUID.randomUUID());
+	}
+
+	@After
+	public void tearDown() throws Exception {
+	}
+
+	@Test
+	public void testAllocateSimpleSlot() throws Exception {
+		ResourceID resourceID = new ResourceID("resource");
+		slotPool.registerResource(resourceID);
+
+		JobID jobID = new JobID();
+		AllocationID allocationID = new AllocationID();
+		Future<SimpleSlot> future = slotPool.allocateSimpleSlot(jobID, DEFAULT_TESTING_PROFILE, allocationID);
+		assertFalse(future.isDone());
+		verify(resourceManagerGateway).requestSlot(any(UUID.class), any(UUID.class), any(SlotRequest.class), any(Time.class));
+
+		SlotDescriptor slotDescriptor = createSlotDescriptor(resourceID, jobID, DEFAULT_TESTING_PROFILE);
+		assertTrue(slotPool.offerSlot(allocationID, slotDescriptor));
+
+		SimpleSlot slot = future.get(1, TimeUnit.SECONDS);
+		assertTrue(future.isDone());
+		assertTrue(slot.isAlive());
+		assertEquals(resourceID, slot.getTaskManagerID());
+		assertEquals(jobID, slot.getJobID());
+		assertEquals(slotPool, slot.getOwner());
+	}
+
+	@Test
+	public void testAllocateSharedSlot() throws Exception {
+		ResourceID resourceID = new ResourceID("resource");
+		slotPool.registerResource(resourceID);
+
+		JobVertexID vid = new JobVertexID();
+		SlotSharingGroup sharingGroup = new SlotSharingGroup(vid);
+		SlotSharingGroupAssignment assignment = sharingGroup.getTaskAssignment();
+
+		JobID jobID = new JobID();
+		AllocationID allocationID = new AllocationID();
+		Future<SharedSlot> future = slotPool.allocateSharedSlot(jobID, DEFAULT_TESTING_PROFILE, assignment, allocationID);
+
+		assertFalse(future.isDone());
+		verify(resourceManagerGateway).requestSlot(any(UUID.class), any(UUID.class), any(SlotRequest.class), any(Time.class));
+
+		SlotDescriptor slotDescriptor = createSlotDescriptor(resourceID, jobID, DEFAULT_TESTING_PROFILE);
+		assertTrue(slotPool.offerSlot(allocationID, slotDescriptor));
+
+		SharedSlot slot = future.get(1, TimeUnit.SECONDS);
+		assertTrue(future.isDone());
+		assertTrue(slot.isAlive());
+		assertEquals(resourceID, slot.getTaskManagerID());
+		assertEquals(jobID, slot.getJobID());
+		assertEquals(slotPool, slot.getOwner());
+
+		SimpleSlot simpleSlot = slot.allocateSubSlot(vid);
+		assertNotNull(simpleSlot);
+		assertTrue(simpleSlot.isAlive());
+	}
+
+	@Test
+	public void testAllocateSlotWithoutResourceManager() throws Exception {
+		slotPool.disconnectResourceManager();
+		Future<SimpleSlot> future = slotPool.allocateSimpleSlot(new JobID(), DEFAULT_TESTING_PROFILE);
+		future.handleAsync(
+			new BiFunction<SimpleSlot, Throwable, Void>() {
+				@Override
+				public Void apply(SimpleSlot simpleSlot, Throwable throwable) {
+					assertNull(simpleSlot);
+					assertNotNull(throwable);
+					return null;
+				}
+			},
+			executor);
+		try {
+			future.get(1, TimeUnit.SECONDS);
+			fail("We expected a ExecutionException.");
+		} catch (ExecutionException ex) {
+			// we expect the exception
+		}
+	}
+
+	@Test
+	public void testAllocationFulfilledByReturnedSlot() throws Exception {
+		ResourceID resourceID = new ResourceID("resource");
+		slotPool.registerResource(resourceID);
+
+		JobID jobID = new JobID();
+
+		AllocationID allocationID1 = new AllocationID();
+		Future<SimpleSlot> future1 = slotPool.allocateSimpleSlot(jobID, DEFAULT_TESTING_PROFILE, allocationID1);
+
+		AllocationID allocationID2 = new AllocationID();
+		Future<SimpleSlot> future2 = slotPool.allocateSimpleSlot(jobID, DEFAULT_TESTING_PROFILE, allocationID2);
+
+		assertFalse(future1.isDone());
+		assertFalse(future2.isDone());
+		verify(resourceManagerGateway, times(2))
+			.requestSlot(any(UUID.class), any(UUID.class), any(SlotRequest.class), any(Time.class));
+
+		SlotDescriptor slotDescriptor = createSlotDescriptor(resourceID, jobID, DEFAULT_TESTING_PROFILE);
+		assertTrue(slotPool.offerSlot(allocationID1, slotDescriptor));
+
+		SimpleSlot slot1 = future1.get(1, TimeUnit.SECONDS);
+		assertTrue(future1.isDone());
+		assertFalse(future2.isDone());
+
+		// return this slot to pool
+		slot1.releaseSlot();
+
+		// second allocation fulfilled by previous slot returning
+		SimpleSlot slot2 = future2.get(1, TimeUnit.SECONDS);
+		assertTrue(future2.isDone());
+
+		assertNotEquals(slot1, slot2);
+		assertTrue(slot1.isReleased());
+		assertTrue(slot2.isAlive());
+		assertEquals(slot1.getTaskManagerID(), slot2.getTaskManagerID());
+		assertEquals(slot1.getSlotNumber(), slot2.getSlotNumber());
+	}
+
+	@Test
+	public void testAllocateWithFreeSlot() throws Exception {
+		ResourceID resourceID = new ResourceID("resource");
+		slotPool.registerResource(resourceID);
+
+		JobID jobID = new JobID();
+		AllocationID allocationID1 = new AllocationID();
+		Future<SimpleSlot> future1 = slotPool.allocateSimpleSlot(jobID, DEFAULT_TESTING_PROFILE, allocationID1);
+		assertFalse(future1.isDone());
+
+		SlotDescriptor slotDescriptor = createSlotDescriptor(resourceID, jobID, DEFAULT_TESTING_PROFILE);
+		assertTrue(slotPool.offerSlot(allocationID1, slotDescriptor));
+
+		SimpleSlot slot1 = future1.get(1, TimeUnit.SECONDS);
+		assertTrue(future1.isDone());
+
+		// return this slot to pool
+		slot1.releaseSlot();
+
+		AllocationID allocationID2 = new AllocationID();
+		Future<SimpleSlot> future2 = slotPool.allocateSimpleSlot(jobID, DEFAULT_TESTING_PROFILE, allocationID2);
+
+		// second allocation fulfilled by previous slot returning
+		SimpleSlot slot2 = future2.get(1, TimeUnit.SECONDS);
+		assertTrue(future2.isDone());
+
+		assertNotEquals(slot1, slot2);
+		assertTrue(slot1.isReleased());
+		assertTrue(slot2.isAlive());
+		assertEquals(slot1.getTaskManagerID(), slot2.getTaskManagerID());
+		assertEquals(slot1.getSlotNumber(), slot2.getSlotNumber());
+	}
+
+	@Test
+	public void testOfferSlot() throws Exception {
+		ResourceID resourceID = new ResourceID("resource");
+		slotPool.registerResource(resourceID);
+
+		JobID jobID = new JobID();
+		AllocationID allocationID = new AllocationID();
+		Future<SimpleSlot> future = slotPool.allocateSimpleSlot(jobID, DEFAULT_TESTING_PROFILE, allocationID);
+		assertFalse(future.isDone());
+		verify(resourceManagerGateway).requestSlot(any(UUID.class), any(UUID.class), any(SlotRequest.class), any(Time.class));
+
+		// slot from unregistered resource
+		SlotDescriptor invalid = createSlotDescriptor(new ResourceID("unregistered"), jobID, DEFAULT_TESTING_PROFILE);
+		assertFalse(slotPool.offerSlot(allocationID, invalid));
+
+		SlotDescriptor slotDescriptor = createSlotDescriptor(resourceID, jobID, DEFAULT_TESTING_PROFILE);
+
+		// reject offering with mismatch allocation id
+		assertFalse(slotPool.offerSlot(new AllocationID(), slotDescriptor));
+
+		// accepted slot
+		assertTrue(slotPool.offerSlot(allocationID, slotDescriptor));
+		SimpleSlot slot = future.get(1, TimeUnit.SECONDS);
+		assertTrue(future.isDone());
+		assertTrue(slot.isAlive());
+
+		// conflict offer with using slot
+		SlotDescriptor conflict = createSlotDescriptor(resourceID, jobID, DEFAULT_TESTING_PROFILE);
+		assertFalse(slotPool.offerSlot(allocationID, conflict));
+
+		// duplicated offer with using slot
+		assertTrue(slotPool.offerSlot(allocationID, slotDescriptor));
+		assertTrue(future.isDone());
+		assertTrue(slot.isAlive());
+
+		// duplicated offer with free slot
+		slot.releaseSlot();
+		assertTrue(slot.isReleased());
+		assertTrue(slotPool.offerSlot(allocationID, slotDescriptor));
+	}
+
+	@Test
+	public void testReleaseResource() throws Exception {
+		ResourceID resourceID = new ResourceID("resource");
+		slotPool.registerResource(resourceID);
+
+		JobID jobID = new JobID();
+
+		AllocationID allocationID1 = new AllocationID();
+		Future<SimpleSlot> future1 = slotPool.allocateSimpleSlot(jobID, DEFAULT_TESTING_PROFILE, allocationID1);
+
+		AllocationID allocationID2 = new AllocationID();
+		Future<SimpleSlot> future2 = slotPool.allocateSimpleSlot(jobID, DEFAULT_TESTING_PROFILE, allocationID2);
+
+		SlotDescriptor slotDescriptor = createSlotDescriptor(resourceID, jobID, DEFAULT_TESTING_PROFILE);
+		assertTrue(slotPool.offerSlot(allocationID1, slotDescriptor));
+
+		SimpleSlot slot1 = future1.get(1, TimeUnit.SECONDS);
+		assertTrue(future1.isDone());
+		assertFalse(future2.isDone());
+
+		slotPool.releaseResource(resourceID);
+		assertTrue(slot1.isReleased());
+
+		// slot released and not usable, second allocation still not fulfilled
+		Thread.sleep(10);
+		assertFalse(future2.isDone());
+	}
+
+}


[35/50] [abbrv] flink git commit: [FLINK-4657] Implement HighAvailabilityServices based on ZooKeeper

Posted by se...@apache.org.
[FLINK-4657] Implement HighAvailabilityServices based on ZooKeeper

[FLINK-4657] Implement a few rpc calls for JobMaster

[FLINK-4657][cluster management] Address review comments

[FLINK-4657][cluster management] Throw exception when error occurred when request input split


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

Branch: refs/heads/flip-6
Commit: 214113eb494d83046e6b0b2dc4df49fc72d869f8
Parents: 291daf6
Author: Kurt Young <yk...@gmail.com>
Authored: Mon Sep 26 10:59:16 2016 +0800
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Oct 14 15:14:42 2016 +0200

----------------------------------------------------------------------
 .../HighAvailabilityServices.java               |   7 +-
 .../runtime/highavailability/NonHaServices.java |   4 +-
 .../highavailability/ZookeeperHaServices.java   |  82 ++++++++++
 .../StandaloneSubmittedJobGraphStore.java       |   5 +
 .../jobmanager/SubmittedJobGraphStore.java      |   8 +
 .../ZooKeeperSubmittedJobGraphStore.java        |   7 +
 .../runtime/jobmaster/JobManagerRunner.java     |  18 +--
 .../flink/runtime/jobmaster/JobMaster.java      | 161 ++++++++++++++++++-
 .../runtime/jobmaster/JobMasterGateway.java     |  54 ++++++-
 .../jobmaster/message/NextInputSplit.java       |  39 +++++
 .../resourcemanager/ResourceManager.java        |   6 +-
 .../flink/runtime/util/ZooKeeperUtils.java      |  82 ++++++++--
 .../flink/runtime/taskmanager/TaskManager.scala |   6 +-
 .../TestingHighAvailabilityServices.java        |  20 +--
 .../jobmanager/JobManagerHARecoveryTest.java    |   3 +-
 .../jobmaster/JobManagerRunnerMockTest.java     |  11 +-
 .../slotmanager/SlotProtocolTest.java           |   2 +-
 17 files changed, 455 insertions(+), 60 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/214113eb/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 d67e927..a26886a 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
@@ -32,6 +32,7 @@ import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
  *     <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>
  * </ul>
  */
 public interface HighAvailabilityServices {
@@ -48,12 +49,10 @@ public interface HighAvailabilityServices {
 	 * @return
 	 * @throws Exception
 	 */
-	LeaderRetrievalService getJobMasterLeaderRetriever(JobID jobID) throws Exception;
+	LeaderRetrievalService getJobManagerLeaderRetriever(JobID jobID) throws Exception;
 
 	/**
 	 * Gets the leader election service for the cluster's resource manager.
-	 * @return
-	 * @throws Exception
 	 */
 	LeaderElectionService getResourceManagerLeaderElectionService() throws Exception;
 
@@ -62,7 +61,7 @@ public interface HighAvailabilityServices {
 	 *
 	 * @param jobID The identifier of the job running the election.
 	 */
-	LeaderElectionService getJobMasterLeaderElectionService(JobID jobID) throws Exception;
+	LeaderElectionService getJobManagerLeaderElectionService(JobID jobID) throws Exception;
 
 	/**
 	 * Gets the checkpoint recovery factory for the job manager

http://git-wip-us.apache.org/repos/asf/flink/blob/214113eb/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 a2c9cc4..2c6295c 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
@@ -79,7 +79,7 @@ public class NonHaServices implements HighAvailabilityServices {
 	}
 
 	@Override
-	public LeaderRetrievalService getJobMasterLeaderRetriever(JobID jobID) throws Exception {
+	public LeaderRetrievalService getJobManagerLeaderRetriever(JobID jobID) throws Exception {
 		return new StandaloneLeaderRetrievalService(jobMastersAddress.get(jobID), new UUID(0, 0));
 	}
 
@@ -89,7 +89,7 @@ public class NonHaServices implements HighAvailabilityServices {
 	}
 
 	@Override
-	public LeaderElectionService getJobMasterLeaderElectionService(JobID jobID) throws Exception {
+	public LeaderElectionService getJobManagerLeaderElectionService(JobID jobID) throws Exception {
 		return new StandaloneLeaderElectionService();
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/214113eb/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
new file mode 100644
index 0000000..d26b668
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/ZookeeperHaServices.java
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.highavailability;
+
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory;
+import org.apache.flink.runtime.checkpoint.ZooKeeperCheckpointRecoveryFactory;
+import org.apache.flink.runtime.jobmanager.SubmittedJobGraphStore;
+import org.apache.flink.runtime.leaderelection.LeaderElectionService;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+import org.apache.flink.runtime.util.ZooKeeperUtils;
+
+/**
+ * An implementation of the {@link HighAvailabilityServices} with zookeeper.
+ */
+public class ZookeeperHaServices implements HighAvailabilityServices {
+
+	private static final String DEFAULT_RESOURCE_MANAGER_PATH_SUFFIX = "/resource-manager";
+
+	/** The ZooKeeper client to use */
+	private final CuratorFramework client;
+
+	/** The runtime configuration */
+	private final Configuration configuration;
+
+	public ZookeeperHaServices(final CuratorFramework client, final Configuration configuration) {
+		this.client = client;
+		this.configuration = configuration;
+	}
+
+	@Override
+	public LeaderRetrievalService getResourceManagerLeaderRetriever() throws Exception {
+		return ZooKeeperUtils.createLeaderRetrievalService(client, configuration, DEFAULT_RESOURCE_MANAGER_PATH_SUFFIX);
+	}
+
+	@Override
+	public LeaderRetrievalService getJobManagerLeaderRetriever(JobID jobID) throws Exception {
+		return ZooKeeperUtils.createLeaderRetrievalService(client, configuration, getPathSuffixForJob(jobID));
+	}
+
+	@Override
+	public LeaderElectionService getResourceManagerLeaderElectionService() throws Exception {
+		return ZooKeeperUtils.createLeaderElectionService(client, configuration, DEFAULT_RESOURCE_MANAGER_PATH_SUFFIX);
+	}
+
+	@Override
+	public LeaderElectionService getJobManagerLeaderElectionService(JobID jobID) throws Exception {
+		return ZooKeeperUtils.createLeaderElectionService(client, configuration, getPathSuffixForJob(jobID));
+	}
+
+	@Override
+	public CheckpointRecoveryFactory getCheckpointRecoveryFactory() throws Exception {
+		return new ZooKeeperCheckpointRecoveryFactory(client, configuration);
+	}
+
+	@Override
+	public SubmittedJobGraphStore getSubmittedJobGraphStore() throws Exception {
+		return ZooKeeperUtils.createSubmittedJobGraphs(client, configuration);
+	}
+
+	private static String getPathSuffixForJob(final JobID jobID) {
+		return String.format("/job-managers/%s", jobID);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/214113eb/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/StandaloneSubmittedJobGraphStore.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/StandaloneSubmittedJobGraphStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/StandaloneSubmittedJobGraphStore.java
index 3041cde..00df935 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/StandaloneSubmittedJobGraphStore.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/StandaloneSubmittedJobGraphStore.java
@@ -62,4 +62,9 @@ public class StandaloneSubmittedJobGraphStore implements SubmittedJobGraphStore
 	public List<SubmittedJobGraph> recoverJobGraphs() throws Exception {
 		return Collections.emptyList();
 	}
+
+	@Override
+	public boolean contains(JobID jobId) throws Exception {
+		return false;
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/214113eb/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/SubmittedJobGraphStore.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/SubmittedJobGraphStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/SubmittedJobGraphStore.java
index bd628cd..4d544ae 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/SubmittedJobGraphStore.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/SubmittedJobGraphStore.java
@@ -64,6 +64,14 @@ public interface SubmittedJobGraphStore {
 	void removeJobGraph(JobID jobId) throws Exception;
 
 	/**
+	 * Check whether the given {@link JobID} is exist.
+	 *
+	 * <p>It's also a flag indicates whether we should recover this job before we can do anything else, since all
+	 * global terminated job will be removed from this store.
+	 */
+	boolean contains(final JobID jobId) throws Exception;
+
+	/**
 	 * A listener for {@link SubmittedJobGraph} instances. This is used to react to races between
 	 * multiple running {@link SubmittedJobGraphStore} instances (on multiple job managers).
 	 */

http://git-wip-us.apache.org/repos/asf/flink/blob/214113eb/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/ZooKeeperSubmittedJobGraphStore.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/ZooKeeperSubmittedJobGraphStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/ZooKeeperSubmittedJobGraphStore.java
index ec05f1e..92093c5 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/ZooKeeperSubmittedJobGraphStore.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/ZooKeeperSubmittedJobGraphStore.java
@@ -266,6 +266,13 @@ public class ZooKeeperSubmittedJobGraphStore implements SubmittedJobGraphStore {
 		}
 	}
 
+	@Override
+	public boolean contains(JobID jobId) throws Exception {
+		checkNotNull(jobId, "Job ID");
+		String path = getPathForJob(jobId);
+		return jobGraphsInZooKeeper.exists(path) != -1;
+	}
+
 	/**
 	 * Monitors ZooKeeper for changes.
 	 *

http://git-wip-us.apache.org/repos/asf/flink/blob/214113eb/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 6944d85..a096932 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,20 +21,18 @@ 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.checkpoint.CheckpointRecoveryFactory;
 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.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.rpc.RpcService;
-
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.util.UUID;
-import java.util.concurrent.Executor;
 
 /**
  * The runner for the job manager. It deals with job level leader election and make underlying job manager
@@ -52,11 +50,8 @@ public class JobManagerRunner implements LeaderContender, OnCompletionActions {
 
 	private final OnCompletionActions toNotify;
 
-	/** The execution context which is used to execute futures */
-	private final Executor executionContext;
-
-	// TODO: use this to decide whether the job is finished by other
-	private final CheckpointRecoveryFactory checkpointRecoveryFactory;
+	/** Used to check whether a job needs to be run */
+	private final SubmittedJobGraphStore submittedJobGraphStore;
 
 	/** Leader election for this job */
 	private final LeaderElectionService leaderElectionService;
@@ -87,9 +82,8 @@ public class JobManagerRunner implements LeaderContender, OnCompletionActions {
 	{
 		this.jobGraph = jobGraph;
 		this.toNotify = toNotify;
-		this.executionContext = rpcService.getExecutor();
-		this.checkpointRecoveryFactory = haServices.getCheckpointRecoveryFactory();
-		this.leaderElectionService = haServices.getJobMasterLeaderElectionService(jobGraph.getJobID());
+		this.submittedJobGraphStore = haServices.getSubmittedJobGraphStore();
+		this.leaderElectionService = haServices.getJobManagerLeaderElectionService(jobGraph.getJobID());
 
 		this.jobManager = new JobMaster(
 			jobGraph, configuration, rpcService, haServices,
@@ -271,7 +265,7 @@ public class JobManagerRunner implements LeaderContender, OnCompletionActions {
 
 	@VisibleForTesting
 	boolean isJobFinishedByOthers() {
-		// TODO
+		// TODO: Fix
 		return false;
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/214113eb/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 1e01c55..e67a167 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
@@ -24,6 +24,8 @@ 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.CheckpointIDCounter;
@@ -35,23 +37,32 @@ import org.apache.flink.runtime.checkpoint.stats.DisabledCheckpointStatsTracker;
 import org.apache.flink.runtime.checkpoint.stats.SimpleCheckpointStatsTracker;
 import org.apache.flink.runtime.client.JobExecutionException;
 import org.apache.flink.runtime.client.JobSubmissionException;
+import org.apache.flink.runtime.client.SerializedJobExecutionResult;
+import org.apache.flink.runtime.execution.ExecutionState;
 import org.apache.flink.runtime.concurrent.Future;
 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.ExecutionJobVertex;
 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.io.network.PartitionState;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
+import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
 import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobgraph.JobStatus;
 import org.apache.flink.runtime.jobgraph.JobVertex;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.jobgraph.jsonplan.JsonPlanGenerator;
 import org.apache.flink.runtime.jobgraph.tasks.JobSnapshottingSettings;
 import org.apache.flink.runtime.jobmanager.OnCompletionActions;
 import org.apache.flink.runtime.jobmanager.scheduler.Scheduler;
+import org.apache.flink.runtime.jobmaster.message.NextInputSplit;
 import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener;
 import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
-import org.apache.flink.runtime.messages.Acknowledge;
 import org.apache.flink.runtime.metrics.groups.JobManagerMetricGroup;
 import org.apache.flink.runtime.registration.RegisteredRpcConnection;
 import org.apache.flink.runtime.registration.RegistrationResponse;
@@ -61,13 +72,18 @@ 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.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 scala.concurrent.ExecutionContext$;
 import scala.concurrent.duration.FiniteDuration;
 
+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;
 
@@ -491,9 +507,12 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 	 * @return Acknowledge the task execution state update
 	 */
 	@RpcMethod
-	public Acknowledge updateTaskExecutionState(TaskExecutionState taskExecutionState) {
-		System.out.println("TaskExecutionState: " + taskExecutionState);
-		return Acknowledge.get();
+	public boolean updateTaskExecutionState(final TaskExecutionState taskExecutionState) {
+		if (taskExecutionState == null) {
+			return false;
+		} else {
+			return executionGraph.updateState(taskExecutionState);
+		}
 	}
 
 	//----------------------------------------------------------------------------------------------\u2028
@@ -511,6 +530,140 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 		});
 	}
 
+	@RpcMethod
+	public NextInputSplit requestNextInputSplit(
+		final JobVertexID vertexID,
+		final ExecutionAttemptID executionAttempt) throws Exception
+	{
+		final Execution execution = executionGraph.getRegisteredExecutions().get(executionAttempt);
+		if (execution == null) {
+			// can happen when JobManager had already unregistered this execution upon on task failure,
+			// but TaskManager get some delay to aware of that situation
+			if (log.isDebugEnabled()) {
+				log.debug("Can not find Execution for attempt {}.", executionAttempt);
+			}
+			// but we should TaskManager be aware of this
+			throw new Exception("Can not find Execution for attempt " + executionAttempt);
+		}
+
+		final ExecutionJobVertex vertex = executionGraph.getJobVertex(vertexID);
+		if (vertex == null) {
+			log.error("Cannot find execution vertex for vertex ID {}.", vertexID);
+			throw new Exception("Cannot find execution vertex for vertex ID " + vertexID);
+		}
+
+		final InputSplitAssigner splitAssigner = vertex.getSplitAssigner();
+		if (splitAssigner == null) {
+			log.error("No InputSplitAssigner for vertex ID {}.", vertexID);
+			throw new Exception("No InputSplitAssigner for vertex ID " + vertexID);
+		}
+
+		final Slot slot = execution.getAssignedResource();
+		final int taskId = execution.getVertex().getParallelSubtaskIndex();
+		final String host = slot != null ? slot.getTaskManagerLocation().getHostname() : null;
+		final InputSplit nextInputSplit = splitAssigner.getNextInputSplit(host, taskId);
+
+		if (log.isDebugEnabled()) {
+			log.debug("Send next input split {}.", nextInputSplit);
+		}
+
+		try {
+			final byte[] serializedInputSplit = InstantiationUtil.serializeObject(nextInputSplit);
+			return new NextInputSplit(serializedInputSplit);
+		} 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);
+			vertex.fail(reason);
+			throw reason;
+		}
+	}
+
+	@RpcMethod
+	public PartitionState requestPartitionState(
+		final ResultPartitionID partitionId,
+		final ExecutionAttemptID taskExecutionId,
+		final IntermediateDataSetID taskResultId)
+	{
+		final Execution execution = executionGraph.getRegisteredExecutions().get(partitionId.getProducerId());
+		final ExecutionState state = execution != null ? execution.getState() : null;
+		return new PartitionState(taskResultId, partitionId.getPartitionId(), state);
+	}
+
+	@RpcMethod
+	public void scheduleOrUpdateConsumers(final ResultPartitionID partitionID) {
+		executionGraph.scheduleOrUpdateConsumers(partitionID);
+	}
+
+	//----------------------------------------------------------------------------------------------
+	// Internal methods
+	//----------------------------------------------------------------------------------------------
+
+	// 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();
+		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);
+					final JobExecutionException exception = new JobExecutionException(
+						jobID, "Failed to retrieve accumulator results.", e);
+					// TODO should we also notify client?
+					jobCompletionActions.jobFailed(exception);
+				}
+			}
+			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);
+			}
+		}
+	}
+
 	private void notifyOfNewResourceManagerLeader(
 		final String resourceManagerAddress, final UUID resourceManagerLeaderId)
 	{

http://git-wip-us.apache.org/repos/asf/flink/blob/214113eb/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 6587ccb..686a3f3 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
@@ -19,7 +19,15 @@
 package org.apache.flink.runtime.jobmaster;
 
 import org.apache.flink.runtime.concurrent.Future;
-import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
+import org.apache.flink.runtime.executiongraph.ExecutionVertex;
+import org.apache.flink.runtime.io.network.PartitionState;
+import org.apache.flink.runtime.io.network.partition.ResultPartition;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
+import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.jobmaster.message.NextInputSplit;
 import org.apache.flink.runtime.rpc.RpcGateway;
 import org.apache.flink.runtime.taskmanager.TaskExecutionState;
 
@@ -47,7 +55,47 @@ public interface JobMasterGateway extends RpcGateway {
 	 * Updates the task execution state for a given task.
 	 *
 	 * @param taskExecutionState New task execution state for a given task
-	 * @return Future acknowledge of the task execution state update
+	 * @return Future flag of the task execution state update result
 	 */
-	Future<Acknowledge> updateTaskExecutionState(TaskExecutionState taskExecutionState);
+	Future<Boolean> updateTaskExecutionState(TaskExecutionState taskExecutionState);
+
+	/**
+	 * Requesting next input split for the {@link ExecutionJobVertex}. The next input split is sent back to the sender
+	 * as a {@link NextInputSplit} message.
+	 *
+	 * @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.
+	 * @throws Exception if some error occurred or information mismatch.
+	 */
+	Future<NextInputSplit> requestNextInputSplit(
+		final JobVertexID vertexID,
+		final ExecutionAttemptID executionAttempt) throws Exception;
+
+	/**
+	 * Requests the current state of the partition.
+	 * The state of a partition is currently bound to the state of the producing execution.
+	 *
+	 * @param partitionId     The partition ID of the partition to request the state of.
+	 * @param taskExecutionId The execution attempt ID of the task requesting the partition state.
+	 * @param taskResultId    The input gate ID of the task requesting the partition state.
+	 * @return The future of the partition state
+	 */
+	Future<PartitionState> requestPartitionState(
+		final ResultPartitionID partitionId,
+		final ExecutionAttemptID taskExecutionId,
+		final IntermediateDataSetID taskResultId);
+
+	/**
+	 * Notifies the JobManager about available data for a produced partition.
+	 * <p>
+	 * There is a call to this method for each {@link ExecutionVertex} instance once per produced
+	 * {@link ResultPartition} instance, either when first producing data (for pipelined executions)
+	 * or when all data has been produced (for staged executions).
+	 * <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
+	 */
+	void scheduleOrUpdateConsumers(final ResultPartitionID partitionID);
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/214113eb/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/message/NextInputSplit.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/message/NextInputSplit.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/message/NextInputSplit.java
new file mode 100644
index 0000000..fe511ed
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/message/NextInputSplit.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.jobmaster.message;
+
+import java.io.Serializable;
+
+/**
+ * Contains the next input split for a task.
+ */
+public class NextInputSplit implements Serializable {
+
+	private static final long serialVersionUID = -1355784074565856240L;
+
+	private final byte[] splitData;
+
+	public NextInputSplit(final byte[] splitData) {
+		this.splitData = splitData;
+	}
+
+	public byte[] getSplitData() {
+		return splitData;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/214113eb/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 f695de4..f45afa3 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
@@ -129,7 +129,7 @@ public abstract class ResourceManager<WorkerType extends ResourceIDRetrievable>
 		try {
 			leaderElectionService.stop();
 			for (JobID jobID : jobMasterGateways.keySet()) {
-				highAvailabilityServices.getJobMasterLeaderRetriever(jobID).stop();
+				highAvailabilityServices.getJobManagerLeaderRetriever(jobID).stop();
 			}
 			super.shutDown();
 		} catch (Throwable e) {
@@ -179,7 +179,7 @@ public abstract class ResourceManager<WorkerType extends ResourceIDRetrievable>
 					final LeaderConnectionInfo jobMasterLeaderInfo;
 					try {
 						jobMasterLeaderInfo = LeaderRetrievalUtils.retrieveLeaderConnectionInfo(
-							highAvailabilityServices.getJobMasterLeaderRetriever(jobID), new FiniteDuration(5, TimeUnit.SECONDS));
+							highAvailabilityServices.getJobManagerLeaderRetriever(jobID), new FiniteDuration(5, TimeUnit.SECONDS));
 					} catch (Exception e) {
 						log.warn("Failed to start JobMasterLeaderRetriever for JobID {}", jobID);
 						throw new Exception("Failed to retrieve JobMasterLeaderRetriever");
@@ -203,7 +203,7 @@ public abstract class ResourceManager<WorkerType extends ResourceIDRetrievable>
 						if (!jobMasterLeaderRetrievalListeners.containsKey(jobID)) {
 							JobMasterLeaderListener jobMasterLeaderListener = new JobMasterLeaderListener(jobID);
 							try {
-								LeaderRetrievalService jobMasterLeaderRetriever = highAvailabilityServices.getJobMasterLeaderRetriever(jobID);
+								LeaderRetrievalService jobMasterLeaderRetriever = highAvailabilityServices.getJobManagerLeaderRetriever(jobID);
 								jobMasterLeaderRetriever.start(jobMasterLeaderListener);
 							} catch (Exception e) {
 								log.warn("Failed to start JobMasterLeaderRetriever for JobID {}", jobID);

http://git-wip-us.apache.org/repos/asf/flink/blob/214113eb/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 67fc397..c5d44b2 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
@@ -132,13 +132,46 @@ public class ZooKeeperUtils {
 	 * @throws Exception
 	 */
 	public static ZooKeeperLeaderRetrievalService createLeaderRetrievalService(
-			Configuration configuration) throws Exception {
-		CuratorFramework client = startCuratorFramework(configuration);
+		final Configuration configuration) throws Exception
+	{
+		final CuratorFramework client = startCuratorFramework(configuration);
+		return createLeaderRetrievalService(client, configuration);
+	}
+
+	/**
+	 * Creates a {@link ZooKeeperLeaderRetrievalService} instance.
+	 *
+	 * @param client        The {@link CuratorFramework} ZooKeeper client to use
+	 * @param configuration {@link Configuration} object containing the configuration values
+	 * @return {@link ZooKeeperLeaderRetrievalService} instance.
+	 * @throws Exception
+	 */
+	public static ZooKeeperLeaderRetrievalService createLeaderRetrievalService(
+		final CuratorFramework client,
+		final Configuration configuration) throws Exception
+	{
+		return createLeaderRetrievalService(client, configuration, "");
+	}
+
+	/**
+	 * Creates a {@link ZooKeeperLeaderRetrievalService} instance.
+	 *
+	 * @param client        The {@link CuratorFramework} ZooKeeper client to use
+	 * @param configuration {@link Configuration} object containing the configuration values
+	 * @param pathSuffix    The path suffix which we want to append
+	 * @return {@link ZooKeeperLeaderRetrievalService} instance.
+	 * @throws Exception
+	 */
+	public static ZooKeeperLeaderRetrievalService createLeaderRetrievalService(
+		final CuratorFramework client,
+		final Configuration configuration,
+		final String pathSuffix) throws Exception
+	{
 		String leaderPath = ConfigurationUtil.getStringWithDeprecatedKeys(
-				configuration,
-				ConfigConstants.HA_ZOOKEEPER_LEADER_PATH,
-				ConfigConstants.DEFAULT_ZOOKEEPER_LEADER_PATH,
-				ConfigConstants.ZOOKEEPER_LEADER_PATH);
+			configuration,
+			ConfigConstants.HA_ZOOKEEPER_LEADER_PATH,
+			ConfigConstants.DEFAULT_ZOOKEEPER_LEADER_PATH,
+			ConfigConstants.ZOOKEEPER_LEADER_PATH) + pathSuffix;
 
 		return new ZooKeeperLeaderRetrievalService(client, leaderPath);
 	}
@@ -171,16 +204,33 @@ public class ZooKeeperUtils {
 			CuratorFramework client,
 			Configuration configuration) throws Exception {
 
-		String latchPath = ConfigurationUtil.getStringWithDeprecatedKeys(
-				configuration,
-				ConfigConstants.HA_ZOOKEEPER_LATCH_PATH,
-				ConfigConstants.DEFAULT_ZOOKEEPER_LATCH_PATH,
-				ConfigConstants.ZOOKEEPER_LATCH_PATH);
-		String leaderPath = ConfigurationUtil.getStringWithDeprecatedKeys(
-				configuration,
-				ConfigConstants.HA_ZOOKEEPER_LEADER_PATH,
-				ConfigConstants.DEFAULT_ZOOKEEPER_LEADER_PATH,
-				ConfigConstants.ZOOKEEPER_LEADER_PATH);
+		return createLeaderElectionService(client, configuration, "");
+	}
+
+	/**
+	 * Creates a {@link ZooKeeperLeaderElectionService} instance.
+	 *
+	 * @param client        The {@link CuratorFramework} ZooKeeper client to use
+	 * @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 latchPath = ConfigurationUtil.getStringWithDeprecatedKeys(
+			configuration,
+			ConfigConstants.HA_ZOOKEEPER_LATCH_PATH,
+			ConfigConstants.DEFAULT_ZOOKEEPER_LATCH_PATH,
+			ConfigConstants.ZOOKEEPER_LATCH_PATH) + pathSuffix;
+		final String leaderPath = ConfigurationUtil.getStringWithDeprecatedKeys(
+			configuration,
+			ConfigConstants.HA_ZOOKEEPER_LEADER_PATH,
+			ConfigConstants.DEFAULT_ZOOKEEPER_LEADER_PATH,
+			ConfigConstants.ZOOKEEPER_LEADER_PATH) + pathSuffix;
 
 		return new ZooKeeperLeaderElectionService(client, latchPath, leaderPath);
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/214113eb/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala
index d16c1b0..7a764ca 100644
--- a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala
+++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala
@@ -52,6 +52,8 @@ import org.apache.flink.runtime.filecache.FileCache
 import org.apache.flink.runtime.instance.{AkkaActorGateway, HardwareDescription, InstanceID}
 import org.apache.flink.runtime.io.disk.iomanager.IOManager
 import org.apache.flink.runtime.io.network.NetworkEnvironment
+import org.apache.flink.runtime.io.network.netty.PartitionStateChecker
+import org.apache.flink.runtime.io.network.partition.ResultPartitionConsumableNotifier
 import org.apache.flink.runtime.jobgraph.IntermediateDataSetID
 import org.apache.flink.runtime.leaderretrieval.{LeaderRetrievalListener, LeaderRetrievalService}
 import org.apache.flink.runtime.memory.MemoryManager
@@ -944,12 +946,12 @@ class TaskManager(
 
     val partitionStateChecker = new ActorGatewayPartitionStateChecker(
       jobManagerGateway,
-      config.timeout)
+      new FiniteDuration(config.getTimeout().toMilliseconds, TimeUnit.MILLISECONDS))
 
     val resultPartitionConsumableNotifier = new ActorGatewayResultPartitionConsumableNotifier(
       context.dispatcher,
       jobManagerGateway,
-      config.timeout)
+      new FiniteDuration(config.getTimeout().toMilliseconds, TimeUnit.MILLISECONDS))
 
     connectionUtils = Some(
       (checkpointResponder,

http://git-wip-us.apache.org/repos/asf/flink/blob/214113eb/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 1a5450d..faf69cc 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
@@ -36,7 +36,7 @@ public class TestingHighAvailabilityServices implements HighAvailabilityServices
 
 	private ConcurrentHashMap<JobID, LeaderRetrievalService> jobMasterLeaderRetrievers = new ConcurrentHashMap<>();
 
-	private volatile LeaderElectionService jobMasterLeaderElectionService;
+	private ConcurrentHashMap<JobID, LeaderElectionService> jobManagerLeaderElectionServices = new ConcurrentHashMap<>();
 
 	private volatile LeaderElectionService resourceManagerLeaderElectionService;
 
@@ -56,8 +56,8 @@ public class TestingHighAvailabilityServices implements HighAvailabilityServices
 		this.jobMasterLeaderRetrievers.put(jobID, jobMasterLeaderRetriever);
 	}
 
-	public void setJobMasterLeaderElectionService(LeaderElectionService leaderElectionService) {
-		this.jobMasterLeaderElectionService = leaderElectionService;
+	public void setJobMasterLeaderElectionService(JobID jobID, LeaderElectionService leaderElectionService) {
+		this.jobManagerLeaderElectionServices.put(jobID, leaderElectionService);
 	}
 
 	public void setResourceManagerLeaderElectionService(LeaderElectionService leaderElectionService) {
@@ -87,7 +87,7 @@ public class TestingHighAvailabilityServices implements HighAvailabilityServices
 	}
 
 	@Override
-	public LeaderRetrievalService getJobMasterLeaderRetriever(JobID jobID) throws Exception {
+	public LeaderRetrievalService getJobManagerLeaderRetriever(JobID jobID) throws Exception {
 		LeaderRetrievalService service = this.jobMasterLeaderRetrievers.get(jobID);
 		if (service != null) {
 			return service;
@@ -97,24 +97,24 @@ public class TestingHighAvailabilityServices implements HighAvailabilityServices
 	}
 
 	@Override
-	public LeaderElectionService getJobMasterLeaderElectionService(JobID jobID) throws Exception {
-		LeaderElectionService service = jobMasterLeaderElectionService;
+	public LeaderElectionService getResourceManagerLeaderElectionService() throws Exception {
+		LeaderElectionService service = resourceManagerLeaderElectionService;
 
 		if (service != null) {
 			return service;
 		} else {
-			throw new IllegalStateException("JobMasterLeaderElectionService has not been set");
+			throw new IllegalStateException("ResourceManagerLeaderElectionService has not been set");
 		}
 	}
 
 	@Override
-	public LeaderElectionService getResourceManagerLeaderElectionService() throws Exception {
-		LeaderElectionService service = resourceManagerLeaderElectionService;
+	public LeaderElectionService getJobManagerLeaderElectionService(JobID jobID) throws Exception {
+		LeaderElectionService service = this.jobManagerLeaderElectionServices.get(jobID);
 
 		if (service != null) {
 			return service;
 		} else {
-			throw new IllegalStateException("ResourceManagerLeaderElectionService has not been set");
+			throw new IllegalStateException("JobMasterLeaderElectionService has not been set");
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/214113eb/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerHARecoveryTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerHARecoveryTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerHARecoveryTest.java
index 5ec6991..8419abe 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerHARecoveryTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerHARecoveryTest.java
@@ -406,7 +406,8 @@ public class JobManagerHARecoveryTest {
 			storedJobs.remove(jobId);
 		}
 
-		boolean contains(JobID jobId) {
+		@Override
+		public boolean contains(JobID jobId) {
 			return storedJobs.containsKey(jobId);
 		}
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/214113eb/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 bfe5f55..3a769bb 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
@@ -24,6 +24,7 @@ import org.apache.flink.configuration.Configuration;
 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.jobmanager.SubmittedJobGraphStore;
 import org.apache.flink.runtime.leaderelection.LeaderElectionService;
 import org.apache.flink.runtime.rpc.RpcService;
 import org.junit.After;
@@ -57,6 +58,8 @@ public class JobManagerRunnerMockTest {
 
 	private LeaderElectionService leaderElectionService;
 
+	private SubmittedJobGraphStore submittedJobGraphStore;
+
 	private TestingOnCompletionActions jobCompletion;
 
 	@Before
@@ -72,8 +75,12 @@ public class JobManagerRunnerMockTest {
 		leaderElectionService = mock(LeaderElectionService.class);
 		when(leaderElectionService.hasLeadership()).thenReturn(true);
 
+		submittedJobGraphStore = mock(SubmittedJobGraphStore.class);
+		when(submittedJobGraphStore.contains(any(JobID.class))).thenReturn(true);
+
 		HighAvailabilityServices haServices = mock(HighAvailabilityServices.class);
-		when(haServices.getJobMasterLeaderElectionService(any(JobID.class))).thenReturn(leaderElectionService);
+		when(haServices.getJobManagerLeaderElectionService(any(JobID.class))).thenReturn(leaderElectionService);
+		when(haServices.getSubmittedJobGraphStore()).thenReturn(submittedJobGraphStore);
 
 		runner = PowerMockito.spy(new JobManagerRunner(
 			new JobGraph("test"),
@@ -127,7 +134,7 @@ public class JobManagerRunnerMockTest {
 	public void testJobFinishedByOtherBeforeGrantLeadership() throws Exception {
 		runner.start();
 
-		when(runner.isJobFinishedByOthers()).thenReturn(true);
+		when(submittedJobGraphStore.contains(any(JobID.class))).thenReturn(false);
 		runner.grantLeadership(UUID.randomUUID());
 
 		// runner should shutdown automatic and informed the job completion

http://git-wip-us.apache.org/repos/asf/flink/blob/214113eb/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 e3018c9..805ea71 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
@@ -237,7 +237,7 @@ public class SlotProtocolTest extends TestLogger {
 		testingHA.setResourceManagerLeaderRetriever(rmLeaderRetrievalService);
 
 		final TestingLeaderElectionService jmLeaderElectionService = new TestingLeaderElectionService();
-		testingHA.setJobMasterLeaderElectionService(jmLeaderElectionService);
+		testingHA.setJobMasterLeaderElectionService(jobID, jmLeaderElectionService);
 		final TestingLeaderRetrievalService jmLeaderRetrievalService = new TestingLeaderRetrievalService(jmAddress, jmID);
 		testingHA.setJobMasterLeaderRetriever(jobID, jmLeaderRetrievalService);
 


[29/50] [abbrv] flink git commit: [FLINK-4580] [rpc] Report rpc invocation exceptions to the caller

Posted by se...@apache.org.
[FLINK-4580] [rpc] Report rpc invocation exceptions to the caller

This closes #2526.


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

Branch: refs/heads/flip-6
Commit: 59d9e67b5811da2a864f7784a685e3c829f4f039
Parents: 360eaf8
Author: Till Rohrmann <tr...@apache.org>
Authored: Wed Sep 21 15:18:27 2016 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Oct 14 15:14:41 2016 +0200

----------------------------------------------------------------------
 .../flink/runtime/rpc/akka/AkkaRpcActor.java    | 53 ++++++++++++--------
 .../runtime/rpc/akka/AkkaRpcActorTest.java      | 34 +++++++++++++
 2 files changed, 66 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/59d9e67b/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 59daa46..1b456a7 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
@@ -35,6 +35,7 @@ import org.apache.flink.runtime.rpc.akka.messages.Processing;
 import org.apache.flink.runtime.rpc.akka.messages.RpcInvocation;
 import org.apache.flink.runtime.rpc.akka.messages.RunAsync;
 
+import org.apache.flink.runtime.rpc.exceptions.RpcConnectionException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -86,11 +87,11 @@ class AkkaRpcActor<C extends RpcGateway, T extends RpcEndpoint<C>> extends Untyp
 			unstashAll();
 			getContext().become(new Procedure<Object>() {
 				@Override
-				public void apply(Object message) throws Exception {
-					if (message.equals(Processing.STOP)) {
+				public void apply(Object msg) throws Exception {
+					if (msg.equals(Processing.STOP)) {
 						getContext().unbecome();
 					} else {
-						handleMessage(message);
+						handleMessage(msg);
 					}
 				}
 			});
@@ -130,21 +131,36 @@ class AkkaRpcActor<C extends RpcGateway, T extends RpcEndpoint<C>> extends Untyp
 	 * @param rpcInvocation Rpc invocation message
 	 */
 	private void handleRpcInvocation(RpcInvocation rpcInvocation) {
+		Method rpcMethod = null;
+
 		try {
 			String methodName = rpcInvocation.getMethodName();
 			Class<?>[] parameterTypes = rpcInvocation.getParameterTypes();
 
-			Method rpcMethod = lookupRpcMethod(methodName, parameterTypes);
+			rpcMethod = lookupRpcMethod(methodName, parameterTypes);
+		} catch(ClassNotFoundException e) {
+			LOG.error("Could not load method arguments.", e);
+
+			RpcConnectionException rpcException = new RpcConnectionException("Could not load method arguments.", e);
+			getSender().tell(new Status.Failure(rpcException), getSelf());
+		} catch (IOException e) {
+			LOG.error("Could not deserialize rpc invocation message.", e);
+
+			RpcConnectionException rpcException = new RpcConnectionException("Could not deserialize rpc invocation message.", e);
+			getSender().tell(new Status.Failure(rpcException), getSelf());
+		} catch (final NoSuchMethodException e) {
+			LOG.error("Could not find rpc method for rpc invocation.", e);
+
+			RpcConnectionException rpcException = new RpcConnectionException("Could not find rpc method for rpc invocation.", e);
+			getSender().tell(new Status.Failure(rpcException), getSelf());
+		}
 
-			if (rpcMethod.getReturnType().equals(Void.TYPE)) {
-				// No return value to send back
-				try {
+		if (rpcMethod != null) {
+			try {
+				if (rpcMethod.getReturnType().equals(Void.TYPE)) {
+					// No return value to send back
 					rpcMethod.invoke(rpcEndpoint, rpcInvocation.getArgs());
-				} catch (Throwable e) {
-					LOG.error("Error while executing remote procedure call {}.", rpcMethod, e);
-				}
-			} else {
-				try {
+				} else {
 					Object result = rpcMethod.invoke(rpcEndpoint, rpcInvocation.getArgs());
 
 					if (result instanceof Future) {
@@ -169,17 +185,12 @@ class AkkaRpcActor<C extends RpcGateway, T extends RpcEndpoint<C>> extends Untyp
 						// tell the sender the result of the computation
 						getSender().tell(new Status.Success(result), getSelf());
 					}
-				} catch (Throwable e) {
-					// tell the sender about the failure
-					getSender().tell(new Status.Failure(e), getSelf());
 				}
+			} catch (Throwable e) {
+				LOG.error("Error while executing remote procedure call {}.", rpcMethod, e);
+				// tell the sender about the failure
+				getSender().tell(new Status.Failure(e), getSelf());
 			}
-		} catch(ClassNotFoundException e) {
-			LOG.error("Could not load method arguments.", e);
-		} catch (IOException e) {
-			LOG.error("Could not deserialize rpc invocation message.", e);
-		} catch (final NoSuchMethodException e) {
-			LOG.error("Could not find rpc method for rpc invocation: {}.", rpcInvocation, e);
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/59d9e67b/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 5624d12..1e8c9a6 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
@@ -119,10 +119,44 @@ public class AkkaRpcActorTest extends TestLogger {
 		rpcEndpoint.shutDown();
 	}
 
+	/**
+	 * Tests that we receive a RpcConnectionException when calling a rpc method (with return type)
+	 * on a wrong rpc endpoint.
+	 *
+	 * @throws Exception
+	 */
+	@Test
+	public void testWrongGatewayEndpointConnection() throws Exception {
+		DummyRpcEndpoint rpcEndpoint = new DummyRpcEndpoint(akkaRpcService);
+
+		rpcEndpoint.start();
+
+		Future<WrongRpcGateway> futureGateway = akkaRpcService.connect(rpcEndpoint.getAddress(), WrongRpcGateway.class);
+
+		WrongRpcGateway gateway = Await.result(futureGateway, timeout.duration());
+
+		// since it is a tell operation we won't receive a RpcConnectionException, it's only logged
+		gateway.tell("foobar");
+
+		Future<Boolean> result = gateway.barfoo();
+
+		try {
+			Await.result(result, timeout.duration());
+			fail("We expected a RpcConnectionException.");
+		} catch (RpcConnectionException rpcConnectionException) {
+			// we expect this exception here
+		}
+	}
+
 	private interface DummyRpcGateway extends RpcGateway {
 		Future<Integer> foobar();
 	}
 
+	private interface WrongRpcGateway extends RpcGateway {
+		Future<Boolean> barfoo();
+		void tell(String message);
+	}
+
 	private static class DummyRpcEndpoint extends RpcEndpoint<DummyRpcGateway> {
 
 		private volatile int _foobar = 42;


[43/50] [abbrv] flink git commit: [FLINK-4738] [TaskManager] Port TaskManager logic to new Flip-6 TaskManager

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/a00619a4/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/exceptions/CheckpointException.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/exceptions/CheckpointException.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/exceptions/CheckpointException.java
new file mode 100644
index 0000000..80f2aa0
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/exceptions/CheckpointException.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.taskexecutor.exceptions;
+
+import org.apache.flink.runtime.taskexecutor.TaskExecutor;
+
+/**
+ * Exception indicating a problem with checkpointing on the {@link TaskExecutor} side.
+ */
+public class CheckpointException extends TaskManagerException {
+
+	private static final long serialVersionUID = 3366394086880327955L;
+
+	public CheckpointException(String message) {
+		super(message);
+	}
+
+	public CheckpointException(String message, Throwable cause) {
+		super(message, cause);
+	}
+
+	public CheckpointException(Throwable cause) {
+		super(cause);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/a00619a4/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/exceptions/PartitionException.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/exceptions/PartitionException.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/exceptions/PartitionException.java
new file mode 100644
index 0000000..eecd0ae
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/exceptions/PartitionException.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.taskexecutor.exceptions;
+
+import org.apache.flink.runtime.taskexecutor.TaskExecutor;
+
+/**
+ * Exception indicating a problem with the result partitions on the {@link TaskExecutor} side.
+ */
+public class PartitionException extends TaskManagerException {
+
+	private static final long serialVersionUID = 6248696963418276618L;
+
+	public PartitionException(String message) {
+		super(message);
+	}
+
+	public PartitionException(String message, Throwable cause) {
+		super(message, cause);
+	}
+
+	public PartitionException(Throwable cause) {
+		super(cause);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/a00619a4/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/exceptions/TaskException.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/exceptions/TaskException.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/exceptions/TaskException.java
new file mode 100644
index 0000000..a4a89c2
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/exceptions/TaskException.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.taskexecutor.exceptions;
+
+import org.apache.flink.runtime.taskexecutor.TaskExecutor;
+
+/**
+ * Exception indicating a task related problem on the {@link TaskExecutor}.
+ */
+public class TaskException extends TaskManagerException {
+
+	private static final long serialVersionUID = 968001398103156856L;
+
+	public TaskException(String message) {
+		super(message);
+	}
+
+	public TaskException(String message, Throwable cause) {
+		super(message, cause);
+	}
+
+	public TaskException(Throwable cause) {
+		super(cause);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/a00619a4/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/exceptions/TaskManagerException.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/exceptions/TaskManagerException.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/exceptions/TaskManagerException.java
new file mode 100644
index 0000000..62d186e
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/exceptions/TaskManagerException.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.taskexecutor.exceptions;
+
+import org.apache.flink.runtime.taskexecutor.TaskExecutor;
+
+/**
+ * Base exception thrown by the {@link TaskExecutor}.
+ */
+public class TaskManagerException extends Exception {
+
+	private static final long serialVersionUID = -2997745772227694731L;
+
+	public TaskManagerException(String message) {
+		super(message);
+	}
+
+	public TaskManagerException(String message, Throwable cause) {
+		super(message, cause);
+	}
+
+	public TaskManagerException(Throwable cause) {
+		super(cause);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/a00619a4/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/exceptions/TaskSubmissionException.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/exceptions/TaskSubmissionException.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/exceptions/TaskSubmissionException.java
new file mode 100644
index 0000000..23f7812
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/exceptions/TaskSubmissionException.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.taskexecutor.exceptions;
+
+import org.apache.flink.runtime.taskmanager.Task;
+
+/**
+ * Exception indicating a problem with the {@link Task} submission at the {@link TaskException}.
+ */
+public class TaskSubmissionException extends TaskManagerException {
+
+	private static final long serialVersionUID = 4589813591317690486L;
+
+	public TaskSubmissionException(String message) {
+		super(message);
+	}
+
+	public TaskSubmissionException(String message, Throwable cause) {
+		super(message, cause);
+	}
+
+	public TaskSubmissionException(Throwable cause) {
+		super(cause);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/a00619a4/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
new file mode 100644
index 0000000..246c11d
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcCheckpointResponder.java
@@ -0,0 +1,56 @@
+/*
+ * 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.rpc;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.checkpoint.CheckpointCoordinatorGateway;
+import org.apache.flink.runtime.checkpoint.CheckpointMetaData;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.state.CheckpointStateHandles;
+import org.apache.flink.runtime.taskmanager.CheckpointResponder;
+import org.apache.flink.util.Preconditions;
+
+public class RpcCheckpointResponder implements CheckpointResponder {
+
+	private final CheckpointCoordinatorGateway checkpointCoordinatorGateway;
+
+	public RpcCheckpointResponder(CheckpointCoordinatorGateway checkpointCoordinatorGateway) {
+		this.checkpointCoordinatorGateway = Preconditions.checkNotNull(checkpointCoordinatorGateway);
+	}
+
+	@Override
+	public void acknowledgeCheckpoint(
+			JobID jobID,
+			ExecutionAttemptID executionAttemptID,
+			CheckpointMetaData checkpointMetaData,
+			CheckpointStateHandles checkpointStateHandles) {
+
+		checkpointCoordinatorGateway.acknowledgeCheckpoint(
+			jobID,
+			executionAttemptID,
+			checkpointMetaData,
+			checkpointStateHandles);
+
+	}
+
+	@Override
+	public void declineCheckpoint(JobID jobID, ExecutionAttemptID executionAttemptID, CheckpointMetaData checkpoint) {
+		checkpointCoordinatorGateway.declineCheckpoint(jobID, executionAttemptID, checkpoint);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/a00619a4/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
new file mode 100644
index 0000000..4850d63
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcInputSplitProvider.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.taskexecutor.rpc;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.core.io.InputSplit;
+import org.apache.flink.runtime.concurrent.Future;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider;
+import org.apache.flink.runtime.jobgraph.tasks.InputSplitProviderException;
+import org.apache.flink.runtime.jobmaster.JobMasterGateway;
+import org.apache.flink.runtime.jobmaster.SerializedInputSplit;
+import org.apache.flink.util.InstantiationUtil;
+import org.apache.flink.util.Preconditions;
+
+public class RpcInputSplitProvider implements InputSplitProvider {
+	private final JobMasterGateway jobMasterGateway;
+	private final JobID jobID;
+	private final JobVertexID jobVertexID;
+	private final ExecutionAttemptID executionAttemptID;
+	private final Time timeout;
+
+	public RpcInputSplitProvider(
+			JobMasterGateway jobMasterGateway,
+			JobID jobID,
+			JobVertexID jobVertexID,
+			ExecutionAttemptID executionAttemptID,
+			Time timeout) {
+		this.jobMasterGateway = Preconditions.checkNotNull(jobMasterGateway);
+		this.jobID = Preconditions.checkNotNull(jobID);
+		this.jobVertexID = Preconditions.checkNotNull(jobVertexID);
+		this.executionAttemptID = Preconditions.checkNotNull(executionAttemptID);
+		this.timeout = Preconditions.checkNotNull(timeout);
+	}
+
+
+	@Override
+	public InputSplit getNextInputSplit(ClassLoader userCodeClassLoader) throws InputSplitProviderException {
+		Preconditions.checkNotNull(userCodeClassLoader);
+
+		Future<SerializedInputSplit> futureInputSplit = jobMasterGateway.requestNextInputSplit(jobVertexID, executionAttemptID);
+
+		try {
+			SerializedInputSplit serializedInputSplit = futureInputSplit.get(timeout.getSize(), timeout.getUnit());
+
+			if (serializedInputSplit.isEmpty()) {
+				return null;
+			} else {
+				return InstantiationUtil.deserializeObject(serializedInputSplit.getInputSplitData(), userCodeClassLoader);
+			}
+		} catch (Exception e) {
+			throw new InputSplitProviderException("Requesting the next input split failed.", e);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/a00619a4/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcKvStateRegistryListener.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcKvStateRegistryListener.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcKvStateRegistryListener.java
new file mode 100644
index 0000000..3692a71
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcKvStateRegistryListener.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.taskexecutor.rpc;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.query.KvStateID;
+import org.apache.flink.runtime.query.KvStateRegistryGateway;
+import org.apache.flink.runtime.query.KvStateRegistryListener;
+import org.apache.flink.runtime.query.KvStateServerAddress;
+import org.apache.flink.runtime.state.KeyGroupRange;
+import org.apache.flink.util.Preconditions;
+
+public class RpcKvStateRegistryListener implements KvStateRegistryListener {
+
+	private final KvStateRegistryGateway kvStateRegistryGateway;
+	private final KvStateServerAddress kvStateServerAddress;
+
+	public RpcKvStateRegistryListener(
+			KvStateRegistryGateway kvStateRegistryGateway,
+			KvStateServerAddress kvStateServerAddress) {
+		this.kvStateRegistryGateway = Preconditions.checkNotNull(kvStateRegistryGateway);
+		this.kvStateServerAddress = Preconditions.checkNotNull(kvStateServerAddress);
+	}
+
+	@Override
+	public void notifyKvStateRegistered(
+			JobID jobId,
+			JobVertexID jobVertexId,
+			KeyGroupRange keyGroupRange,
+			String registrationName,
+			KvStateID kvStateId) {
+		kvStateRegistryGateway.notifyKvStateRegistered(
+			jobId,
+			jobVertexId,
+			keyGroupRange,
+			registrationName,
+			kvStateId,
+			kvStateServerAddress);
+
+	}
+
+	@Override
+	public void notifyKvStateUnregistered(
+		JobID jobId,
+		JobVertexID jobVertexId,
+		KeyGroupRange keyGroupRange,
+		String registrationName) {
+
+		kvStateRegistryGateway.notifyKvStateUnregistered(
+			jobId,
+			jobVertexId,
+			keyGroupRange,
+			registrationName);
+
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/a00619a4/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
new file mode 100644
index 0000000..ab111ad
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcPartitionStateChecker.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.taskexecutor.rpc;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.concurrent.Future;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.io.network.PartitionState;
+import org.apache.flink.runtime.io.network.netty.PartitionStateChecker;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
+import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
+import org.apache.flink.runtime.jobmaster.JobMasterGateway;
+import org.apache.flink.util.Preconditions;
+
+public class RpcPartitionStateChecker implements PartitionStateChecker {
+
+	private final JobMasterGateway jobMasterGateway;
+
+	public RpcPartitionStateChecker(JobMasterGateway jobMasterGateway) {
+		this.jobMasterGateway = Preconditions.checkNotNull(jobMasterGateway);
+	}
+
+	@Override
+	public Future<PartitionState> requestPartitionState(
+		JobID jobId,
+		ExecutionAttemptID executionId,
+		IntermediateDataSetID resultId,
+		ResultPartitionID partitionId) {
+
+		return jobMasterGateway.requestPartitionState(partitionId, executionId, resultId);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/a00619a4/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
new file mode 100644
index 0000000..29ad3b6
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcResultPartitionConsumableNotifier.java
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.taskexecutor.rpc;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.concurrent.ApplyFunction;
+import org.apache.flink.runtime.concurrent.Future;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionConsumableNotifier;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
+import org.apache.flink.runtime.jobmaster.JobMasterGateway;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.taskmanager.TaskActions;
+import org.apache.flink.util.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.concurrent.Executor;
+
+public class RpcResultPartitionConsumableNotifier implements ResultPartitionConsumableNotifier {
+
+	private static final Logger LOG = LoggerFactory.getLogger(RpcResultPartitionConsumableNotifier.class);
+
+	private final JobMasterGateway jobMasterGateway;
+	private final Executor executor;
+	private final Time timeout;
+
+	public RpcResultPartitionConsumableNotifier(
+			JobMasterGateway jobMasterGateway,
+			Executor executor,
+			Time timeout) {
+		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);
+
+		acknowledgeFuture.exceptionallyAsync(new ApplyFunction<Throwable, Void>() {
+			@Override
+			public Void apply(Throwable value) {
+				LOG.error("Could not schedule or update consumers at the JobManager.", value);
+
+				taskActions.failExternally(new RuntimeException("Could not notify JobManager to schedule or update consumers.", value));
+
+				return null;
+			}
+		}, executor);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/a00619a4/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/utils/TaskExecutorMetricsInitializer.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/utils/TaskExecutorMetricsInitializer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/utils/TaskExecutorMetricsInitializer.java
new file mode 100644
index 0000000..1f8d5ed
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/utils/TaskExecutorMetricsInitializer.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.runtime.taskexecutor.utils;
+
+import com.sun.management.OperatingSystemMXBean;
+import org.apache.flink.metrics.Gauge;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.runtime.io.network.NetworkEnvironment;
+import org.apache.flink.runtime.taskexecutor.TaskExecutor;
+import org.apache.flink.util.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.management.AttributeNotFoundException;
+import javax.management.InstanceNotFoundException;
+import javax.management.MBeanException;
+import javax.management.MBeanServer;
+import javax.management.MalformedObjectNameException;
+import javax.management.ObjectName;
+import javax.management.ReflectionException;
+import java.lang.management.ClassLoadingMXBean;
+import java.lang.management.GarbageCollectorMXBean;
+import java.lang.management.ManagementFactory;
+import java.lang.management.MemoryMXBean;
+import java.lang.management.ThreadMXBean;
+import java.util.List;
+
+/**
+ * Utility class ot initialize {@link TaskExecutor} specific metrics.
+ */
+public class TaskExecutorMetricsInitializer {
+	private static final Logger LOG = LoggerFactory.getLogger(TaskExecutorMetricsInitializer.class);
+
+	public static void instantiateStatusMetrics(
+		MetricGroup taskManagerMetricGroup,
+		NetworkEnvironment network) {
+		MetricGroup status = taskManagerMetricGroup.addGroup("Status");
+
+		instantiateNetworkMetrics(status.addGroup("Network"), network);
+
+		MetricGroup jvm = status.addGroup("JVM");
+
+		instantiateClassLoaderMetrics(jvm.addGroup("ClassLoader"));
+		instantiateGarbageCollectorMetrics(jvm.addGroup("GarbageCollector"));
+		instantiateMemoryMetrics(jvm.addGroup("Memory"));
+		instantiateThreadMetrics(jvm.addGroup("Threads"));
+		instantiateCPUMetrics(jvm.addGroup("CPU"));
+	}
+
+	private static void instantiateNetworkMetrics(
+		MetricGroup metrics,
+		final NetworkEnvironment network) {
+		metrics.<Long, Gauge<Long>>gauge("TotalMemorySegments", new Gauge<Long> () {
+			@Override
+			public Long getValue() {
+				return (long) network.getNetworkBufferPool().getTotalNumberOfMemorySegments();
+			}
+		});
+
+		metrics.<Long, Gauge<Long>>gauge("AvailableMemorySegments", new Gauge<Long> () {
+			@Override
+			public Long getValue() {
+				return (long) network.getNetworkBufferPool().getNumberOfAvailableMemorySegments();
+			}
+		});
+	}
+
+	private static void instantiateClassLoaderMetrics(MetricGroup metrics) {
+		final ClassLoadingMXBean mxBean = ManagementFactory.getClassLoadingMXBean();
+
+		metrics.<Long, Gauge<Long>>gauge("ClassesLoaded", new Gauge<Long> () {
+			@Override
+			public Long getValue() {
+				return mxBean.getTotalLoadedClassCount();
+			}
+		});
+
+		metrics.<Long, Gauge<Long>>gauge("ClassesUnloaded", new Gauge<Long> () {
+			@Override
+			public Long getValue() {
+				return mxBean.getUnloadedClassCount();
+			}
+		});
+	}
+
+	private static void instantiateGarbageCollectorMetrics(MetricGroup metrics) {
+		List<GarbageCollectorMXBean> garbageCollectors = ManagementFactory.getGarbageCollectorMXBeans();
+
+		for (final GarbageCollectorMXBean garbageCollector: garbageCollectors) {
+			MetricGroup gcGroup = metrics.addGroup(garbageCollector.getName());
+
+			gcGroup.<Long, Gauge<Long>>gauge("Count", new Gauge<Long> () {
+				@Override
+				public Long getValue() {
+					return garbageCollector.getCollectionCount();
+				}
+			});
+
+			gcGroup.<Long, Gauge<Long>>gauge("Time", new Gauge<Long> () {
+				@Override
+				public Long getValue() {
+					return garbageCollector.getCollectionTime();
+				}
+			});
+		}
+	}
+
+	private static void instantiateMemoryMetrics(MetricGroup metrics) {
+		final MemoryMXBean mxBean = ManagementFactory.getMemoryMXBean();
+
+		MetricGroup heap = metrics.addGroup("Heap");
+
+		heap.<Long, Gauge<Long>>gauge("Used", new Gauge<Long> () {
+			@Override
+			public Long getValue() {
+				return mxBean.getHeapMemoryUsage().getUsed();
+			}
+		});
+		heap.<Long, Gauge<Long>>gauge("Committed", new Gauge<Long> () {
+			@Override
+			public Long getValue() {
+				return mxBean.getHeapMemoryUsage().getCommitted();
+			}
+		});
+		heap.<Long, Gauge<Long>>gauge("Max", new Gauge<Long> () {
+			@Override
+			public Long getValue() {
+				return mxBean.getHeapMemoryUsage().getMax();
+			}
+		});
+
+		MetricGroup nonHeap = metrics.addGroup("NonHeap");
+
+		nonHeap.<Long, Gauge<Long>>gauge("Used", new Gauge<Long> () {
+			@Override
+			public Long getValue() {
+				return mxBean.getNonHeapMemoryUsage().getUsed();
+			}
+		});
+		nonHeap.<Long, Gauge<Long>>gauge("Committed", new Gauge<Long> () {
+			@Override
+			public Long getValue() {
+				return mxBean.getNonHeapMemoryUsage().getCommitted();
+			}
+		});
+		nonHeap.<Long, Gauge<Long>>gauge("Max", new Gauge<Long> () {
+			@Override
+			public Long getValue() {
+				return mxBean.getNonHeapMemoryUsage().getMax();
+			}
+		});
+
+		final MBeanServer con = ManagementFactory.getPlatformMBeanServer();
+
+		final String directBufferPoolName = "java.nio:type=BufferPool,name=direct";
+
+		try {
+			final ObjectName directObjectName = new ObjectName(directBufferPoolName);
+
+			MetricGroup direct = metrics.addGroup("Direct");
+
+			direct.<Long, Gauge<Long>>gauge("Count", new TaskExecutorMetricsInitializer.AttributeGauge<>(con, directObjectName, "Count", -1L));
+			direct.<Long, Gauge<Long>>gauge("MemoryUsed", new TaskExecutorMetricsInitializer.AttributeGauge<>(con, directObjectName, "MemoryUsed", -1L));
+			direct.<Long, Gauge<Long>>gauge("TotalCapacity", new TaskExecutorMetricsInitializer.AttributeGauge<>(con, directObjectName, "TotalCapacity", -1L));
+		} catch (MalformedObjectNameException e) {
+			LOG.warn("Could not create object name {}.", directBufferPoolName, e);
+		}
+
+		final String mappedBufferPoolName = "java.nio:type=BufferPool,name=mapped";
+
+		try {
+			final ObjectName mappedObjectName = new ObjectName(mappedBufferPoolName);
+
+			MetricGroup mapped = metrics.addGroup("Mapped");
+
+			mapped.<Long, Gauge<Long>>gauge("Count", new TaskExecutorMetricsInitializer.AttributeGauge<>(con, mappedObjectName, "Count", -1L));
+			mapped.<Long, Gauge<Long>>gauge("MemoryUsed", new TaskExecutorMetricsInitializer.AttributeGauge<>(con, mappedObjectName, "MemoryUsed", -1L));
+			mapped.<Long, Gauge<Long>>gauge("TotalCapacity", new TaskExecutorMetricsInitializer.AttributeGauge<>(con, mappedObjectName, "TotalCapacity", -1L));
+		} catch (MalformedObjectNameException e) {
+			LOG.warn("Could not create object name {}.", mappedBufferPoolName, e);
+		}
+	}
+
+	private static void instantiateThreadMetrics(MetricGroup metrics) {
+		final ThreadMXBean mxBean = ManagementFactory.getThreadMXBean();
+
+		metrics.<Integer, Gauge<Integer>>gauge("Count", new Gauge<Integer> () {
+			@Override
+			public Integer getValue() {
+				return mxBean.getThreadCount();
+			}
+		});
+	}
+
+	private static void instantiateCPUMetrics(MetricGroup metrics) {
+		try {
+			final OperatingSystemMXBean mxBean = (OperatingSystemMXBean) ManagementFactory.getOperatingSystemMXBean();
+
+			metrics.<Double, Gauge<Double>>gauge("Load", new Gauge<Double> () {
+				@Override
+				public Double getValue() {
+					return mxBean.getProcessCpuLoad();
+				}
+			});
+			metrics.<Long, Gauge<Long>>gauge("Time", new Gauge<Long> () {
+				@Override
+				public Long getValue() {
+					return mxBean.getProcessCpuTime();
+				}
+			});
+		} catch (Exception e) {
+			LOG.warn("Cannot access com.sun.management.OperatingSystemMXBean.getProcessCpuLoad()" +
+				" - CPU load metrics will not be available.", e);
+		}
+	}
+
+	private static final class AttributeGauge<T> implements Gauge<T> {
+		private final MBeanServer server;
+		private final ObjectName objectName;
+		private final String attributeName;
+		private final T errorValue;
+
+		private AttributeGauge(MBeanServer server, ObjectName objectName, String attributeName, T errorValue) {
+			this.server = Preconditions.checkNotNull(server);
+			this.objectName = Preconditions.checkNotNull(objectName);
+			this.attributeName = Preconditions.checkNotNull(attributeName);
+			this.errorValue = errorValue;
+		}
+
+		@SuppressWarnings("unchecked")
+		@Override
+		public T getValue() {
+			try {
+				return (T) server.getAttribute(objectName, attributeName);
+			} catch (MBeanException | AttributeNotFoundException | InstanceNotFoundException | ReflectionException e) {
+				LOG.warn("Could not read attribute {}.", attributeName, e);
+				return errorValue;
+			}
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/a00619a4/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/ActorGatewayTaskManagerActions.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/ActorGatewayTaskManagerActions.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/ActorGatewayTaskManagerActions.java
new file mode 100644
index 0000000..b3a0cbb
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/ActorGatewayTaskManagerActions.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.taskmanager;
+
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.instance.ActorGateway;
+import org.apache.flink.runtime.messages.TaskManagerMessages;
+import org.apache.flink.runtime.messages.TaskMessages;
+import org.apache.flink.util.Preconditions;
+
+/**
+ * Implementation using {@link ActorGateway} to forward the messages.
+ */
+public class ActorGatewayTaskManagerActions implements TaskManagerActions {
+
+	private final ActorGateway actorGateway;
+
+	public ActorGatewayTaskManagerActions(ActorGateway actorGateway) {
+		this.actorGateway = Preconditions.checkNotNull(actorGateway);
+	}
+
+	@Override
+	public void notifyFinalState(ExecutionAttemptID executionAttemptID) {
+		actorGateway.tell(new TaskMessages.TaskInFinalState(executionAttemptID));
+	}
+
+	@Override
+	public void notifyFatalError(String message, Throwable cause) {
+		actorGateway.tell(new TaskManagerMessages.FatalError(message, cause));
+	}
+
+	@Override
+	public void failTask(ExecutionAttemptID executionAttemptID, Throwable cause) {
+		actorGateway.tell(new TaskMessages.FailTask(executionAttemptID, cause));
+	}
+
+	@Override
+	public void updateTaskExecutionState(TaskExecutionState taskExecutionState) {
+		TaskMessages.UpdateTaskExecutionState actorMessage = new TaskMessages.UpdateTaskExecutionState(taskExecutionState);
+
+		actorGateway.tell(actorMessage);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/a00619a4/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/ActorGatewayTaskManagerConnection.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/ActorGatewayTaskManagerConnection.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/ActorGatewayTaskManagerConnection.java
deleted file mode 100644
index cddac55..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/ActorGatewayTaskManagerConnection.java
+++ /dev/null
@@ -1,59 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.runtime.taskmanager;
-
-import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
-import org.apache.flink.runtime.instance.ActorGateway;
-import org.apache.flink.runtime.messages.TaskManagerMessages;
-import org.apache.flink.runtime.messages.TaskMessages;
-import org.apache.flink.util.Preconditions;
-
-/**
- * Implementation using {@link ActorGateway} to forward the messages.
- */
-public class ActorGatewayTaskManagerConnection implements TaskManagerConnection {
-
-	private final ActorGateway actorGateway;
-
-	public ActorGatewayTaskManagerConnection(ActorGateway actorGateway) {
-		this.actorGateway = Preconditions.checkNotNull(actorGateway);
-	}
-
-	@Override
-	public void notifyFinalState(ExecutionAttemptID executionAttemptID) {
-		actorGateway.tell(new TaskMessages.TaskInFinalState(executionAttemptID));
-	}
-
-	@Override
-	public void notifyFatalError(String message, Throwable cause) {
-		actorGateway.tell(new TaskManagerMessages.FatalError(message, cause));
-	}
-
-	@Override
-	public void failTask(ExecutionAttemptID executionAttemptID, Throwable cause) {
-		actorGateway.tell(new TaskMessages.FailTask(executionAttemptID, cause));
-	}
-
-	@Override
-	public void updateTaskExecutionState(TaskExecutionState taskExecutionState) {
-		TaskMessages.UpdateTaskExecutionState actorMessage = new TaskMessages.UpdateTaskExecutionState(taskExecutionState);
-
-		actorGateway.tell(actorMessage);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/a00619a4/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java
index 02a41b5..977e563 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java
@@ -25,6 +25,7 @@ import org.apache.flink.api.common.cache.DistributedCache;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.core.fs.Path;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
 import org.apache.flink.runtime.checkpoint.CheckpointMetaData;
 import org.apache.flink.runtime.concurrent.BiFunction;
 import org.apache.flink.runtime.io.network.PartitionState;
@@ -131,6 +132,9 @@ public class Task implements Runnable, TaskActions {
 	/** The execution attempt of the parallel subtask */
 	private final ExecutionAttemptID executionId;
 
+	/** ID which identifies the slot in which the task is supposed to run */
+	private final AllocationID allocationID;
+
 	/** TaskInfo object for this task */
 	private final TaskInfo taskInfo;
 
@@ -176,7 +180,7 @@ public class Task implements Runnable, TaskActions {
 	private final Map<IntermediateDataSetID, SingleInputGate> inputGatesById;
 
 	/** Connection to the task manager */
-	private final TaskManagerConnection taskManagerConnection;
+	private final TaskManagerActions taskManagerActions;
 
 	/** Input split provider for the task */
 	private final InputSplitProvider inputSplitProvider;
@@ -259,7 +263,7 @@ public class Task implements Runnable, TaskActions {
 		IOManager ioManager,
 		NetworkEnvironment networkEnvironment,
 		BroadcastVariableManager bcVarManager,
-		TaskManagerConnection taskManagerConnection,
+		TaskManagerActions taskManagerActions,
 		InputSplitProvider inputSplitProvider,
 		CheckpointResponder checkpointResponder,
 		LibraryCacheManager libraryCache,
@@ -274,6 +278,7 @@ public class Task implements Runnable, TaskActions {
 		this.jobId = checkNotNull(tdd.getJobID());
 		this.vertexId = checkNotNull(tdd.getVertexID());
 		this.executionId  = checkNotNull(tdd.getExecutionId());
+		this.allocationID = checkNotNull(tdd.getAllocationID());
 		this.taskNameWithSubtask = taskInfo.getTaskNameWithSubtasks();
 		this.jobConfiguration = checkNotNull(tdd.getJobConfiguration());
 		this.taskConfiguration = checkNotNull(tdd.getTaskConfiguration());
@@ -296,7 +301,7 @@ public class Task implements Runnable, TaskActions {
 
 		this.inputSplitProvider = checkNotNull(inputSplitProvider);
 		this.checkpointResponder = checkNotNull(checkpointResponder);
-		this.taskManagerConnection = checkNotNull(taskManagerConnection);
+		this.taskManagerActions = checkNotNull(taskManagerActions);
 
 		this.libraryCache = checkNotNull(libraryCache);
 		this.fileCache = checkNotNull(fileCache);
@@ -380,6 +385,10 @@ public class Task implements Runnable, TaskActions {
 		return executionId;
 	}
 
+	public AllocationID getAllocationID() {
+		return allocationID;
+	}
+
 	public TaskInfo getTaskInfo() {
 		return taskInfo;
 	}
@@ -600,7 +609,7 @@ public class Task implements Runnable, TaskActions {
 
 			// notify everyone that we switched to running
 			notifyObservers(ExecutionState.RUNNING, null);
-			taskManagerConnection.updateTaskExecutionState(new TaskExecutionState(jobId, executionId, ExecutionState.RUNNING));
+			taskManagerActions.updateTaskExecutionState(new TaskExecutionState(jobId, executionId, ExecutionState.RUNNING));
 
 			// make sure the user code classloader is accessible thread-locally
 			executingThread.setContextClassLoader(userCodeClassLoader);
@@ -793,11 +802,11 @@ public class Task implements Runnable, TaskActions {
 	}
 
 	private void notifyFinalState() {
-		taskManagerConnection.notifyFinalState(executionId);
+		taskManagerActions.notifyFinalState(executionId);
 	}
 
 	private void notifyFatalError(String message, Throwable cause) {
-		taskManagerConnection.notifyFatalError(message, cause);
+		taskManagerActions.notifyFatalError(message, cause);
 	}
 
 	// ----------------------------------------------------------------------------------------------------------------
@@ -823,7 +832,7 @@ public class Task implements Runnable, TaskActions {
 						((StoppableTask)Task.this.invokable).stop();
 					} catch(RuntimeException e) {
 						LOG.error("Stopping task " + taskNameWithSubtask + " failed.", e);
-						taskManagerConnection.failTask(executionId, e);
+						taskManagerActions.failTask(executionId, e);
 					}
 				}
 			};

http://git-wip-us.apache.org/repos/asf/flink/blob/a00619a4/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskExecutionState.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskExecutionState.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskExecutionState.java
index 60aadf5..877cc1e 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskExecutionState.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskExecutionState.java
@@ -24,6 +24,8 @@ import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.runtime.util.SerializedThrowable;
 
+import java.io.Serializable;
+
 /**
  * This class represents an update about a task's execution state.
  *
@@ -34,7 +36,7 @@ import org.apache.flink.runtime.util.SerializedThrowable;
  * exception field transient and deserialized it lazily, with the
  * appropriate class loader.
  */
-public class TaskExecutionState implements java.io.Serializable {
+public class TaskExecutionState implements Serializable {
 
 	private static final long serialVersionUID = 1L;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/a00619a4/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskInputSplitProvider.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskInputSplitProvider.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskInputSplitProvider.java
index 60beae0..31c518a 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskInputSplitProvider.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskInputSplitProvider.java
@@ -24,6 +24,7 @@ import org.apache.flink.api.common.JobID;
 import org.apache.flink.runtime.instance.ActorGateway;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider;
+import org.apache.flink.runtime.jobgraph.tasks.InputSplitProviderException;
 import org.apache.flink.runtime.messages.JobManagerMessages;
 import org.apache.flink.util.InstantiationUtil;
 
@@ -63,35 +64,45 @@ public class TaskInputSplitProvider implements InputSplitProvider {
 	}
 
 	@Override
-	public InputSplit getNextInputSplit(ClassLoader userCodeClassLoader) {
+	public InputSplit getNextInputSplit(ClassLoader userCodeClassLoader) throws InputSplitProviderException {
 		Preconditions.checkNotNull(userCodeClassLoader);
 
+		final Future<Object> response = jobManager.ask(
+			new JobManagerMessages.RequestNextInputSplit(jobID, vertexID, executionID),
+			timeout);
+
+		final Object result;
+
 		try {
-			final Future<Object> response = jobManager.ask(
-					new JobManagerMessages.RequestNextInputSplit(jobID, vertexID, executionID),
-					timeout);
+			result = Await.result(response, timeout);
+		} catch (Exception e) {
+			throw new InputSplitProviderException("Did not receive next input split from JobManager.", e);
+		}
 
-			final Object result = Await.result(response, timeout);
+		if(result instanceof JobManagerMessages.NextInputSplit){
+			final JobManagerMessages.NextInputSplit nextInputSplit =
+				(JobManagerMessages.NextInputSplit) result;
 
-			if(result instanceof JobManagerMessages.NextInputSplit){
-				final JobManagerMessages.NextInputSplit nextInputSplit =
-					(JobManagerMessages.NextInputSplit) result;
+			byte[] serializedData = nextInputSplit.splitData();
 
-				byte[] serializedData = nextInputSplit.splitData();
+			if(serializedData == null) {
+				return null;
+			} else {
+				final Object deserialized;
 
-				if(serializedData == null) {
-					return null;
-				} else {
-					Object deserialized = InstantiationUtil.deserializeObject(serializedData,
+				try {
+					deserialized = InstantiationUtil.deserializeObject(serializedData,
 						userCodeClassLoader);
-					return (InputSplit) deserialized;
+				} catch (Exception e) {
+					throw new InputSplitProviderException("Could not deserialize the serialized input split.", e);
 				}
-			} else {
-				throw new Exception("RequestNextInputSplit requires a response of type " +
-					"NextInputSplit. Instead response is of type " + result.getClass() + '.');
+
+				return (InputSplit) deserialized;
 			}
-		} catch (Exception e) {
-			throw new RuntimeException("Requesting the next InputSplit failed.", e);
+		} else {
+			throw new InputSplitProviderException("RequestNextInputSplit requires a response of type " +
+				"NextInputSplit. Instead response is of type " + result.getClass() + '.');
 		}
+
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/a00619a4/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManagerActions.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManagerActions.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManagerActions.java
new file mode 100644
index 0000000..2f3a0cb
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManagerActions.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.taskmanager;
+
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+
+/**
+ * Interface for the communication of the {@link Task} with the {@link TaskManager}.
+ */
+public interface TaskManagerActions {
+
+	/**
+	 * Notifies the task manager that the given task is in a final state.
+	 *
+	 * @param executionAttemptID Execution attempt ID of the task
+	 */
+	void notifyFinalState(ExecutionAttemptID executionAttemptID);
+
+	/**
+	 * Notifies the task manager about a fatal error occurred in the task.
+	 *
+	 * @param message Message to report
+	 * @param cause Cause of the fatal error
+	 */
+	void notifyFatalError(String message, Throwable cause);
+
+	/**
+	 * Tells the task manager to fail the given task.
+	 *
+	 * @param executionAttemptID Execution attempt ID of the task to fail
+	 * @param cause Cause of the failure
+	 */
+	void failTask(ExecutionAttemptID executionAttemptID, Throwable cause);
+
+	/**
+	 * Notifies the task manager about the task execution state update.
+	 *
+	 * @param taskExecutionState Task execution state update
+	 */
+	void updateTaskExecutionState(TaskExecutionState taskExecutionState);
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/a00619a4/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManagerConnection.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManagerConnection.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManagerConnection.java
deleted file mode 100644
index dc1b40f..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManagerConnection.java
+++ /dev/null
@@ -1,57 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.runtime.taskmanager;
-
-import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
-
-/**
- * Interface for the communication of the {@link Task} with the {@link TaskManager}.
- */
-public interface TaskManagerConnection {
-
-	/**
-	 * Notifies the task manager that the given task is in a final state.
-	 *
-	 * @param executionAttemptID Execution attempt ID of the task
-	 */
-	void notifyFinalState(ExecutionAttemptID executionAttemptID);
-
-	/**
-	 * Notifies the task manager about a fatal error occurred in the task.
-	 *
-	 * @param message Message to report
-	 * @param cause Cause of the fatal error
-	 */
-	void notifyFatalError(String message, Throwable cause);
-
-	/**
-	 * Tells the task manager to fail the given task.
-	 *
-	 * @param executionAttemptID Execution attempt ID of the task to fail
-	 * @param cause Cause of the failure
-	 */
-	void failTask(ExecutionAttemptID executionAttemptID, Throwable cause);
-
-	/**
-	 * Notifies the task manager about the task execution state update.
-	 *
-	 * @param taskExecutionState Task execution state update
-	 */
-	void updateTaskExecutionState(TaskExecutionState taskExecutionState);
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/a00619a4/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala
index 7a764ca..da8c14e 100644
--- a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala
+++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala
@@ -25,7 +25,6 @@ import java.net.{InetAddress, InetSocketAddress}
 import java.util
 import java.util.UUID
 import java.util.concurrent.TimeUnit
-import javax.management.ObjectName
 
 import _root_.akka.actor._
 import _root_.akka.pattern.ask
@@ -37,7 +36,6 @@ import com.fasterxml.jackson.databind.ObjectMapper
 import grizzled.slf4j.Logger
 import org.apache.flink.configuration._
 import org.apache.flink.core.fs.FileSystem
-import org.apache.flink.metrics.{MetricGroup, Gauge => FlinkGauge}
 import org.apache.flink.runtime.accumulators.AccumulatorSnapshot
 import org.apache.flink.runtime.clusterframework.messages.StopCluster
 import org.apache.flink.runtime.clusterframework.types.ResourceID
@@ -59,7 +57,7 @@ import org.apache.flink.runtime.leaderretrieval.{LeaderRetrievalListener, Leader
 import org.apache.flink.runtime.memory.MemoryManager
 import org.apache.flink.runtime.messages.Messages._
 import org.apache.flink.runtime.messages.RegistrationMessages._
-import org.apache.flink.runtime.messages.StackTraceSampleMessages.{ResponseStackTraceSampleFailure, ResponseStackTraceSampleSuccess, SampleTaskStackTrace, StackTraceSampleMessages, TriggerStackTraceSample}
+import org.apache.flink.runtime.messages.StackTraceSampleMessages._
 import org.apache.flink.runtime.messages.TaskManagerMessages._
 import org.apache.flink.runtime.messages.TaskMessages._
 import org.apache.flink.runtime.messages.checkpoint.{AbstractCheckpointMessage, NotifyCheckpointComplete, TriggerCheckpoint}
@@ -68,7 +66,8 @@ import org.apache.flink.runtime.metrics.groups.TaskManagerMetricGroup
 import org.apache.flink.runtime.process.ProcessReaper
 import org.apache.flink.runtime.security.SecurityContext.{FlinkSecuredRunner, SecurityConfiguration}
 import org.apache.flink.runtime.security.SecurityContext
-import org.apache.flink.runtime.taskexecutor.{TaskManagerConfiguration, TaskManagerServices, TaskManagerServicesConfiguration}
+import org.apache.flink.runtime.taskexecutor._
+import org.apache.flink.runtime.taskexecutor.utils.TaskExecutorMetricsInitializer
 import org.apache.flink.runtime.util._
 import org.apache.flink.runtime.{FlinkActor, LeaderSessionMessageFilter, LogMessages}
 import org.apache.flink.util.NetUtils
@@ -150,7 +149,7 @@ class TaskManager(
   protected val bcVarManager = new BroadcastVariableManager()
 
   /** Handler for distributed files cached by this TaskManager */
-  protected val fileCache = new FileCache(config.getConfiguration())
+  protected val fileCache = new FileCache(config.getTmpDirPaths())
 
   /** Registry of metrics periodically transmitted to the JobManager */
   private val metricRegistry = TaskManager.createMetricsRegistry()
@@ -196,7 +195,7 @@ class TaskManager(
     CheckpointResponder,
     PartitionStateChecker,
     ResultPartitionConsumableNotifier,
-    TaskManagerConnection)] = None
+    TaskManagerActions)] = None
 
   // --------------------------------------------------------------------------
   //  Actor messages and life cycle
@@ -940,9 +939,9 @@ class TaskManager(
     val jobManagerGateway = new AkkaActorGateway(jobManager, leaderSessionID.orNull)
     val taskManagerGateway = new AkkaActorGateway(self, leaderSessionID.orNull)
 
-    val checkpointResponder = new ActorGatewayCheckpointResponder(jobManagerGateway);
+    val checkpointResponder = new ActorGatewayCheckpointResponder(jobManagerGateway)
 
-    val taskManagerConnection = new ActorGatewayTaskManagerConnection(taskManagerGateway)
+    val taskManagerConnection = new ActorGatewayTaskManagerActions(taskManagerGateway)
 
     val partitionStateChecker = new ActorGatewayPartitionStateChecker(
       jobManagerGateway,
@@ -998,7 +997,7 @@ class TaskManager(
     taskManagerMetricGroup = 
       new TaskManagerMetricGroup(metricsRegistry, this.runtimeInfo.getHostname, id.toString)
     
-    TaskManager.instantiateStatusMetrics(taskManagerMetricGroup, network)
+    TaskExecutorMetricsInitializer.instantiateStatusMetrics(taskManagerMetricGroup, network)
     
     // watch job manager to detect when it dies
     context.watch(jobManager)
@@ -2008,23 +2007,23 @@ object TaskManager {
 
     // Pre-processing steps for registering cpuLoad
     val osBean: OperatingSystemMXBean = ManagementFactory.getOperatingSystemMXBean()
-        
-    val fetchCPULoadMethod: Option[Method] = 
+
+    val fetchCPULoadMethod: Option[Method] =
       try {
         Class.forName("com.sun.management.OperatingSystemMXBean")
           .getMethods()
-          .find( _.getName() == "getProcessCpuLoad" )
+          .find(_.getName() == "getProcessCpuLoad")
       }
       catch {
         case t: Throwable =>
           LOG.warn("Cannot access com.sun.management.OperatingSystemMXBean.getProcessCpuLoad()" +
-            " - CPU load metrics will not be available.")
+                     " - CPU load metrics will not be available.")
           None
       }
 
     metricRegistry.register("cpuLoad", new Gauge[Double] {
       override def getValue: Double = {
-        try{
+        try {
           fetchCPULoadMethod.map(_.invoke(osBean).asInstanceOf[Double]).getOrElse(-1.0)
         }
         catch {
@@ -2036,146 +2035,4 @@ object TaskManager {
     })
     metricRegistry
   }
-
-  private def instantiateStatusMetrics(
-      taskManagerMetricGroup: MetricGroup,
-      network: NetworkEnvironment)
-    : Unit = {
-    val status = taskManagerMetricGroup
-      .addGroup("Status")
-
-    instantiateNetworkMetrics(status.addGroup("Network"), network)
-
-    val jvm = status
-      .addGroup("JVM")
-
-    instantiateClassLoaderMetrics(jvm.addGroup("ClassLoader"))
-    instantiateGarbageCollectorMetrics(jvm.addGroup("GarbageCollector"))
-    instantiateMemoryMetrics(jvm.addGroup("Memory"))
-    instantiateThreadMetrics(jvm.addGroup("Threads"))
-    instantiateCPUMetrics(jvm.addGroup("CPU"))
-  }
-
-  private def instantiateNetworkMetrics(
-        metrics: MetricGroup,
-        network: NetworkEnvironment)
-    : Unit = {
-    metrics.gauge[Long, FlinkGauge[Long]]("TotalMemorySegments", new FlinkGauge[Long] {
-      override def getValue: Long = network.getNetworkBufferPool.getTotalNumberOfMemorySegments
-    })
-    metrics.gauge[Long, FlinkGauge[Long]]("AvailableMemorySegments", new FlinkGauge[Long] {
-      override def getValue: Long = network.getNetworkBufferPool.getNumberOfAvailableMemorySegments
-    })
-  }
-
-  private def instantiateClassLoaderMetrics(metrics: MetricGroup) {
-    val mxBean = ManagementFactory.getClassLoadingMXBean
-
-    metrics.gauge[Long, FlinkGauge[Long]]("ClassesLoaded", new FlinkGauge[Long] {
-      override def getValue: Long = mxBean.getTotalLoadedClassCount
-    })
-    metrics.gauge[Long, FlinkGauge[Long]]("ClassesUnloaded", new FlinkGauge[Long] {
-      override def getValue: Long = mxBean.getUnloadedClassCount
-    })
-  }
-
-  private def instantiateGarbageCollectorMetrics(metrics: MetricGroup) {
-    val garbageCollectors = ManagementFactory.getGarbageCollectorMXBeans
-
-    for (garbageCollector <- garbageCollectors.asScala) {
-      val gcGroup = metrics.addGroup(garbageCollector.getName)
-      gcGroup.gauge[Long, FlinkGauge[Long]]("Count", new FlinkGauge[Long] {
-        override def getValue: Long = garbageCollector.getCollectionCount
-      })
-      gcGroup.gauge[Long, FlinkGauge[Long]]("Time", new FlinkGauge[Long] {
-        override def getValue: Long = garbageCollector.getCollectionTime
-      })
-    }
-  }
-
-  private def instantiateMemoryMetrics(metrics: MetricGroup) {
-    val mxBean = ManagementFactory.getMemoryMXBean
-    val heap = metrics.addGroup("Heap")
-    heap.gauge[Long, FlinkGauge[Long]]("Used", new FlinkGauge[Long] {
-      override def getValue: Long = mxBean.getHeapMemoryUsage.getUsed
-    })
-    heap.gauge[Long, FlinkGauge[Long]]("Committed", new FlinkGauge[Long] {
-      override def getValue: Long = mxBean.getHeapMemoryUsage.getCommitted
-    })
-    heap.gauge[Long, FlinkGauge[Long]]("Max", new FlinkGauge[Long] {
-      override def getValue: Long = mxBean.getHeapMemoryUsage.getMax
-    })
-
-    val nonHeap = metrics.addGroup("NonHeap")
-    nonHeap.gauge[Long, FlinkGauge[Long]]("Used", new FlinkGauge[Long] {
-      override def getValue: Long = mxBean.getNonHeapMemoryUsage.getUsed
-    })
-    nonHeap.gauge[Long, FlinkGauge[Long]]("Committed", new FlinkGauge[Long] {
-      override def getValue: Long = mxBean.getNonHeapMemoryUsage.getCommitted
-    })
-    nonHeap.gauge[Long, FlinkGauge[Long]]("Max", new FlinkGauge[Long] {
-      override def getValue: Long = mxBean.getNonHeapMemoryUsage.getMax
-    })
-
-    val con = ManagementFactory.getPlatformMBeanServer;
-
-    val directObjectName = new ObjectName("java.nio:type=BufferPool,name=direct")
-
-    val direct = metrics.addGroup("Direct")
-    direct.gauge[Long, FlinkGauge[Long]]("Count", new FlinkGauge[Long] {
-      override def getValue: Long = con
-        .getAttribute(directObjectName, "Count").asInstanceOf[Long]
-    })
-    direct.gauge[Long, FlinkGauge[Long]]("MemoryUsed", new FlinkGauge[Long] {
-      override def getValue: Long = con
-        .getAttribute(directObjectName, "MemoryUsed").asInstanceOf[Long]
-    })
-    direct.gauge[Long, FlinkGauge[Long]]("TotalCapacity", new FlinkGauge[Long] {
-      override def getValue: Long = con
-        .getAttribute(directObjectName, "TotalCapacity").asInstanceOf[Long]
-    })
-
-    val mappedObjectName = new ObjectName("java.nio:type=BufferPool,name=mapped")
-
-    val mapped = metrics.addGroup("Mapped")
-    mapped.gauge[Long, FlinkGauge[Long]]("Count", new FlinkGauge[Long] {
-      override def getValue: Long = con
-        .getAttribute(mappedObjectName, "Count").asInstanceOf[Long]
-    })
-    mapped.gauge[Long, FlinkGauge[Long]]("MemoryUsed", new FlinkGauge[Long] {
-      override def getValue: Long = con
-        .getAttribute(mappedObjectName, "MemoryUsed").asInstanceOf[Long]
-    })
-    mapped.gauge[Long, FlinkGauge[Long]]("TotalCapacity", new FlinkGauge[Long] {
-      override def getValue: Long = con
-        .getAttribute(mappedObjectName, "TotalCapacity").asInstanceOf[Long]
-    })
-  }
-
-  private def instantiateThreadMetrics(metrics: MetricGroup): Unit = {
-    val mxBean = ManagementFactory.getThreadMXBean
-
-    metrics.gauge[Int, FlinkGauge[Int]]("Count", new FlinkGauge[Int] {
-      override def getValue: Int = mxBean.getThreadCount
-    })
-  }
-
-  private def instantiateCPUMetrics(metrics: MetricGroup): Unit = {
-    try {
-      val mxBean = ManagementFactory.getOperatingSystemMXBean
-        .asInstanceOf[com.sun.management.OperatingSystemMXBean]
-
-      metrics.gauge[Double, FlinkGauge[Double]]("Load", new FlinkGauge[Double] {
-          override def getValue: Double = mxBean.getProcessCpuLoad
-        })
-      metrics.gauge[Long, FlinkGauge[Long]]("Time", new FlinkGauge[Long] {
-          override def getValue: Long = mxBean.getProcessCpuTime
-        })
-    }
-    catch {
-     case t: Throwable =>
-       LOG.warn("Cannot access com.sun.management.OperatingSystemMXBean.getProcessCpuLoad()" +
-        " - CPU load metrics will not be available.") 
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/a00619a4/flink-runtime/src/test/java/org/apache/flink/runtime/filecache/FileCacheDeleteValidationTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/filecache/FileCacheDeleteValidationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/filecache/FileCacheDeleteValidationTest.java
index c369674..4db0d93 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/filecache/FileCacheDeleteValidationTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/filecache/FileCacheDeleteValidationTest.java
@@ -21,7 +21,6 @@ package org.apache.flink.runtime.filecache;
 import java.io.File;
 import java.util.concurrent.Future;
 
-import org.apache.flink.configuration.Configuration;
 import org.apache.flink.core.fs.Path;
 import org.apache.flink.api.common.cache.DistributedCache.DistributedCacheEntry;
 import org.apache.flink.api.common.JobID;
@@ -62,8 +61,9 @@ public class FileCacheDeleteValidationTest {
 	
 	@Before
 	public void setup() {
+		String[] tmpDirectories = System.getProperty("java.io.tmpdir").split(",|" + File.pathSeparator);
 		try {
-			fileCache = new FileCache(new Configuration());
+			fileCache = new FileCache(tmpDirectories);
 		}
 		catch (Exception e) {
 			e.printStackTrace();

http://git-wip-us.apache.org/repos/asf/flink/blob/a00619a4/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 f8a0b6a..30dfef5 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
@@ -42,7 +42,6 @@ import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 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;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/a00619a4/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 7710fa9..f5fe52c 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
@@ -20,8 +20,11 @@ package org.apache.flink.runtime.taskexecutor;
 
 import org.apache.flink.api.common.time.Time;
 import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.broadcast.BroadcastVariableManager;
 import org.apache.flink.runtime.clusterframework.types.ResourceID;
 import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.filecache.FileCache;
 import org.apache.flink.runtime.highavailability.NonHaServices;
 import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
@@ -29,6 +32,7 @@ import org.apache.flink.runtime.io.network.NetworkEnvironment;
 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.resourcemanager.ResourceManagerGateway;
 import org.apache.flink.runtime.resourcemanager.messages.taskexecutor.TMSlotRequestRegistered;
 import org.apache.flink.runtime.resourcemanager.messages.taskexecutor.TMSlotRequestRejected;
@@ -38,7 +42,6 @@ import org.apache.flink.runtime.rpc.TestingSerialRpcService;
 import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
 import org.apache.flink.util.TestLogger;
 
-import org.hamcrest.Matchers;
 import org.junit.Test;
 
 import org.powermock.api.mockito.PowerMockito;
@@ -60,10 +63,14 @@ public class TaskExecutorTest extends TestLogger {
 			ResourceManagerGateway rmGateway = mock(ResourceManagerGateway.class);
 			TaskManagerConfiguration taskManagerServicesConfiguration = mock(TaskManagerConfiguration.class);
 			PowerMockito.when(taskManagerServicesConfiguration.getNumberSlots()).thenReturn(1);
+			PowerMockito.when(taskManagerServicesConfiguration.getConfiguration()).thenReturn(new Configuration());
+			PowerMockito.when(taskManagerServicesConfiguration.getTmpDirPaths()).thenReturn(new String[1]);
+
 			rpc.registerGateway(resourceManagerAddress, rmGateway);
 
 			TaskManagerLocation taskManagerLocation = mock(TaskManagerLocation.class);
 			when(taskManagerLocation.getResourceID()).thenReturn(resourceID);
+			when(taskManagerLocation.getHostname()).thenReturn("foobar");
 
 			NonHaServices haServices = new NonHaServices(resourceManagerAddress);
 
@@ -76,6 +83,9 @@ public class TaskExecutorTest extends TestLogger {
 				mock(NetworkEnvironment.class),
 				haServices,
 				mock(MetricRegistry.class),
+				mock(TaskManagerMetricGroup.class),
+				mock(BroadcastVariableManager.class),
+				mock(FileCache.class),
 				mock(FatalErrorHandler.class));
 
 			taskManager.start();
@@ -113,9 +123,12 @@ public class TaskExecutorTest extends TestLogger {
 
 			TaskManagerConfiguration taskManagerServicesConfiguration = mock(TaskManagerConfiguration.class);
 			PowerMockito.when(taskManagerServicesConfiguration.getNumberSlots()).thenReturn(1);
+			PowerMockito.when(taskManagerServicesConfiguration.getConfiguration()).thenReturn(new Configuration());
+			PowerMockito.when(taskManagerServicesConfiguration.getTmpDirPaths()).thenReturn(new String[1]);
 
 			TaskManagerLocation taskManagerLocation = mock(TaskManagerLocation.class);
 			when(taskManagerLocation.getResourceID()).thenReturn(resourceID);
+			when(taskManagerLocation.getHostname()).thenReturn("foobar");
 
 			TaskExecutor taskManager = new TaskExecutor(
 				taskManagerServicesConfiguration,
@@ -126,6 +139,9 @@ public class TaskExecutorTest extends TestLogger {
 				mock(NetworkEnvironment.class),
 				haServices,
 				mock(MetricRegistry.class),
+				mock(TaskManagerMetricGroup.class),
+				mock(BroadcastVariableManager.class),
+				mock(FileCache.class),
 				mock(FatalErrorHandler.class));
 
 			taskManager.start();
@@ -182,9 +198,12 @@ public class TaskExecutorTest extends TestLogger {
 
 			TaskManagerConfiguration taskManagerServicesConfiguration = mock(TaskManagerConfiguration.class);
 			PowerMockito.when(taskManagerServicesConfiguration.getNumberSlots()).thenReturn(1);
+			PowerMockito.when(taskManagerServicesConfiguration.getConfiguration()).thenReturn(new Configuration());
+			PowerMockito.when(taskManagerServicesConfiguration.getTmpDirPaths()).thenReturn(new String[1]);
 
 			TaskManagerLocation taskManagerLocation = mock(TaskManagerLocation.class);
 			when(taskManagerLocation.getResourceID()).thenReturn(resourceID);
+			when(taskManagerLocation.getHostname()).thenReturn("foobar");
 
 			TaskExecutor taskManager = new TaskExecutor(
 				taskManagerServicesConfiguration,
@@ -195,6 +214,9 @@ public class TaskExecutorTest extends TestLogger {
 				mock(NetworkEnvironment.class),
 				haServices,
 				mock(MetricRegistry.class),
+				mock(TaskManagerMetricGroup.class),
+				mock(BroadcastVariableManager.class),
+				mock(FileCache.class),
 				mock(FatalErrorHandler.class));
 
 			taskManager.start();

http://git-wip-us.apache.org/repos/asf/flink/blob/a00619a4/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskAsyncCallTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskAsyncCallTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskAsyncCallTest.java
index e2abe88..9a79935 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskAsyncCallTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskAsyncCallTest.java
@@ -173,7 +173,7 @@ public class TaskAsyncCallTest {
 			mock(IOManager.class),
 			networkEnvironment,
 			mock(BroadcastVariableManager.class),
-			mock(TaskManagerConnection.class),
+			mock(TaskManagerActions.class),
 			mock(InputSplitProvider.class),
 			mock(CheckpointResponder.class),
 			libCache,

http://git-wip-us.apache.org/repos/asf/flink/blob/a00619a4/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskInputSplitProviderTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskInputSplitProviderTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskInputSplitProviderTest.java
index 642300d..777633d 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskInputSplitProviderTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskInputSplitProviderTest.java
@@ -24,6 +24,7 @@ import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
 import org.apache.flink.runtime.instance.BaseTestingActorGateway;
 import org.apache.flink.runtime.instance.ActorGateway;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.jobgraph.tasks.InputSplitProviderException;
 import org.apache.flink.runtime.messages.JobManagerMessages;
 import org.apache.flink.runtime.testingUtils.TestingUtils;
 import org.junit.Test;
@@ -36,7 +37,7 @@ import java.util.concurrent.TimeUnit;
 public class TaskInputSplitProviderTest {
 
 	@Test
-	public void testRequestNextInputSplitWithInvalidExecutionID() {
+	public void testRequestNextInputSplitWithInvalidExecutionID() throws InputSplitProviderException {
 
 		final JobID jobID = new JobID();
 		final JobVertexID vertexID = new JobVertexID();

http://git-wip-us.apache.org/repos/asf/flink/blob/a00619a4/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskStopTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskStopTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskStopTest.java
index 9791cee..5d3eb3a 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskStopTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskStopTest.java
@@ -20,6 +20,7 @@ package org.apache.flink.runtime.taskmanager;
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.api.common.TaskInfo;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
 import org.apache.flink.runtime.io.network.netty.PartitionStateChecker;
 import org.apache.flink.runtime.io.network.partition.ResultPartitionConsumableNotifier;
 import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider;
@@ -68,6 +69,7 @@ public class TaskStopTest {
 		when(tddMock.getTaskConfiguration()).thenReturn(mock(Configuration.class));
 		when(tddMock.getSerializedExecutionConfig()).thenReturn(mock(SerializedValue.class));
 		when(tddMock.getInvokableClassName()).thenReturn("className");
+		when(tddMock.getAllocationID()).thenReturn(mock(AllocationID.class));
 
 		task = new Task(
 			tddMock,
@@ -75,7 +77,7 @@ public class TaskStopTest {
 			mock(IOManager.class),
 			mock(NetworkEnvironment.class),
 			mock(BroadcastVariableManager.class),
-			mock(TaskManagerConnection.class),
+			mock(TaskManagerActions.class),
 			mock(InputSplitProvider.class),
 			mock(CheckpointResponder.class),
 			mock(LibraryCacheManager.class),

http://git-wip-us.apache.org/repos/asf/flink/blob/a00619a4/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java
index 9a13cde..fe618ff 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java
@@ -97,7 +97,7 @@ public class TaskTest {
 	private ActorGateway listenerGateway;
 
 	private ActorGatewayTaskExecutionStateListener listener;
-	private ActorGatewayTaskManagerConnection taskManagerConnection;
+	private ActorGatewayTaskManagerActions taskManagerConnection;
 
 	private BlockingQueue<Object> taskManagerMessages;
 	private BlockingQueue<Object> jobManagerMessages;
@@ -113,7 +113,7 @@ public class TaskTest {
 		listenerGateway = new ForwardingActorGateway(listenerMessages);
 
 		listener = new ActorGatewayTaskExecutionStateListener(listenerGateway);
-		taskManagerConnection = new ActorGatewayTaskManagerConnection(taskManagerGateway);
+		taskManagerConnection = new ActorGatewayTaskManagerActions(taskManagerGateway);
 		
 		awaitLatch = new OneShotLatch();
 		triggerLatch = new OneShotLatch();

http://git-wip-us.apache.org/repos/asf/flink/blob/a00619a4/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/InputFormatSourceFunction.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/InputFormatSourceFunction.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/InputFormatSourceFunction.java
index 343affe..c067ca7 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/InputFormatSourceFunction.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/InputFormatSourceFunction.java
@@ -26,6 +26,7 @@ import org.apache.flink.configuration.Configuration;
 import org.apache.flink.core.io.InputSplit;
 import org.apache.flink.metrics.Counter;
 import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider;
+import org.apache.flink.runtime.jobgraph.tasks.InputSplitProviderException;
 import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
 
 import java.util.Iterator;
@@ -146,7 +147,12 @@ public class InputFormatSourceFunction<OUT> extends RichParallelSourceFunction<O
 					return true;
 				}
 
-				InputSplit split = provider.getNextInputSplit(getRuntimeContext().getUserCodeClassLoader());
+				final InputSplit split;
+				try {
+					split = provider.getNextInputSplit(getRuntimeContext().getUserCodeClassLoader());
+				} catch (InputSplitProviderException e) {
+					throw new RuntimeException("Could not retrieve next input split.", e);
+				}
 
 				if (split != null) {
 					this.nextSplit = split;

http://git-wip-us.apache.org/repos/asf/flink/blob/a00619a4/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/InterruptSensitiveRestoreTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/InterruptSensitiveRestoreTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/InterruptSensitiveRestoreTest.java
index b5b6582..ffda126 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/InterruptSensitiveRestoreTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/InterruptSensitiveRestoreTest.java
@@ -47,7 +47,7 @@ import org.apache.flink.runtime.state.OperatorStateHandle;
 import org.apache.flink.runtime.state.StreamStateHandle;
 import org.apache.flink.runtime.taskmanager.CheckpointResponder;
 import org.apache.flink.runtime.taskmanager.Task;
-import org.apache.flink.runtime.taskmanager.TaskManagerConnection;
+import org.apache.flink.runtime.taskmanager.TaskManagerActions;
 import org.apache.flink.runtime.taskmanager.TaskManagerRuntimeInfo;
 import org.apache.flink.runtime.util.EnvironmentInformation;
 import org.apache.flink.streaming.api.TimeCharacteristic;
@@ -59,6 +59,7 @@ import org.apache.flink.util.SerializedValue;
 import org.junit.Test;
 
 import java.io.EOFException;
+import java.io.File;
 import java.io.IOException;
 import java.io.Serializable;
 import java.net.URL;
@@ -153,17 +154,19 @@ public class InterruptSensitiveRestoreTest {
 		when(networkEnvironment.createKvStateTaskRegistry(any(JobID.class), any(JobVertexID.class)))
 				.thenReturn(mock(TaskKvStateRegistry.class));
 
+		String[] tmpDirectories = EnvironmentInformation.getTemporaryFileDirectory().split(",|" + File.pathSeparator);
+
 		return new Task(
 				tdd,
 				mock(MemoryManager.class),
 				mock(IOManager.class),
 				networkEnvironment,
 				mock(BroadcastVariableManager.class),
-				mock(TaskManagerConnection.class),
+				mock(TaskManagerActions.class),
 				mock(InputSplitProvider.class),
 				mock(CheckpointResponder.class),
 				new FallbackLibraryCacheManager(),
-				new FileCache(new Configuration()),
+				new FileCache(tmpDirectories),
 				new TaskManagerRuntimeInfo(
 						"localhost", new Configuration(), EnvironmentInformation.getTemporaryFileDirectory()),
 				new UnregisteredTaskMetricsGroup(),
@@ -266,4 +269,4 @@ public class InterruptSensitiveRestoreTest {
 			fail("should never be called");
 		}
 	}
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/a00619a4/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java
index 8aae19f..106d3df 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java
@@ -51,7 +51,7 @@ import org.apache.flink.runtime.taskmanager.CheckpointResponder;
 import org.apache.flink.runtime.taskmanager.Task;
 import org.apache.flink.runtime.taskmanager.TaskExecutionState;
 import org.apache.flink.runtime.taskmanager.TaskExecutionStateListener;
-import org.apache.flink.runtime.taskmanager.TaskManagerConnection;
+import org.apache.flink.runtime.taskmanager.TaskManagerActions;
 import org.apache.flink.runtime.taskmanager.TaskManagerRuntimeInfo;
 import org.apache.flink.streaming.api.TimeCharacteristic;
 import org.apache.flink.streaming.api.functions.source.SourceFunction;
@@ -238,7 +238,7 @@ public class StreamTaskTest {
 			mock(IOManager.class),
 			network,
 			mock(BroadcastVariableManager.class),
-			mock(TaskManagerConnection.class),
+			mock(TaskManagerActions.class),
 			mock(InputSplitProvider.class),
 			mock(CheckpointResponder.class),
 			libCache,


[22/50] [abbrv] flink git commit: [FLINK-4505] [cluster mngt] Implement TaskManager component's startup

Posted by se...@apache.org.
[FLINK-4505] [cluster mngt] Implement TaskManager component's startup

The TaskManagerRunner now contains the startup logic for the TaskManager's components.


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

Branch: refs/heads/flip-6
Commit: c34f13c26bf215fbde6c3f28547f1a31ca6cbc99
Parents: b8c6b99
Author: \u6dd8\u6c5f <ta...@alibaba-inc.com>
Authored: Fri Sep 2 18:00:49 2016 +0800
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Oct 14 15:14:41 2016 +0200

----------------------------------------------------------------------
 .../runtime/taskexecutor/TaskExecutor.java      | 766 +------------------
 .../runtime/taskmanager/TaskManagerRunner.java  | 749 ++++++++++++++++++
 .../runtime/taskexecutor/TaskExecutorTest.java  |  53 +-
 3 files changed, 804 insertions(+), 764 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/c34f13c2/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 9d9ad2a..8ce2780 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,74 +18,29 @@
 
 package org.apache.flink.runtime.taskexecutor;
 
-import akka.actor.ActorSystem;
-import com.typesafe.config.Config;
 import org.apache.flink.api.common.time.Time;
 import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory;
 import org.apache.flink.runtime.clusterframework.types.AllocationID;
-import org.apache.flink.runtime.io.network.ConnectionManager;
-import org.apache.flink.runtime.io.network.LocalConnectionManager;
-import org.apache.flink.runtime.io.network.TaskEventDispatcher;
-import org.apache.flink.runtime.io.network.buffer.NetworkBufferPool;
-import org.apache.flink.runtime.io.network.netty.NettyConnectionManager;
-import org.apache.flink.runtime.io.network.partition.ResultPartitionManager;
 import org.apache.flink.runtime.jobmanager.SubmittedJobGraphStore;
-import org.apache.flink.runtime.query.KvStateRegistry;
-import org.apache.flink.runtime.query.netty.DisabledKvStateRequestStats;
-import org.apache.flink.runtime.query.netty.KvStateServer;
 import org.apache.flink.runtime.resourcemanager.SlotRequestRegistered;
 import org.apache.flink.runtime.resourcemanager.SlotRequestReply;
 import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
-import org.apache.flink.util.Preconditions;
 import org.jboss.netty.channel.ChannelException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.flink.api.common.JobID;
-import org.apache.flink.configuration.ConfigConstants;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.configuration.IllegalConfigurationException;
-import org.apache.flink.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.annotation.VisibleForTesting;
 import org.apache.flink.runtime.clusterframework.types.ResourceID;
 import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
-import org.apache.flink.runtime.io.disk.iomanager.IOManager.IOMode;
-import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
 import org.apache.flink.runtime.io.network.NetworkEnvironment;
-import org.apache.flink.runtime.io.network.netty.NettyConfig;
-import org.apache.flink.runtime.leaderelection.LeaderElectionService;
 import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener;
-import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
 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.runtime.rpc.akka.AkkaRpcService;
-import org.apache.flink.runtime.taskmanager.MemoryLogger;
-import org.apache.flink.runtime.util.EnvironmentInformation;
-import org.apache.flink.runtime.util.LeaderRetrievalUtils;
-import org.apache.flink.runtime.taskmanager.NetworkEnvironmentConfiguration;
-import org.apache.flink.util.MathUtils;
-import org.apache.flink.util.NetUtils;
-
-import scala.Tuple2;
-import scala.Option;
-import scala.Some;
-import scala.concurrent.duration.Duration;
-import scala.concurrent.duration.FiniteDuration;
-
-import java.io.File;
-import java.io.IOException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 import java.net.BindException;
-import java.net.InetAddress;
-import java.net.InetSocketAddress;
 import java.util.UUID;
-import java.util.concurrent.TimeUnit;
 
 import static org.apache.flink.util.Preconditions.checkArgument;
 import static org.apache.flink.util.Preconditions.checkNotNull;
@@ -98,12 +53,10 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 
 	private static final Logger LOG = LoggerFactory.getLogger(TaskExecutor.class);
 
-	/** The unique resource ID of this TaskExecutor */
-	private final ResourceID resourceID;
-
+	/** The connection information of this task manager */
 	private final TaskManagerLocation taskManagerLocation;
 
-	/** The access to the leader election and metadata storage services */
+	/** The access to the leader election and retrieval services */
 	private final HighAvailabilityServices haServices;
 
 	/** The task manager configuration */
@@ -128,28 +81,26 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 	// ------------------------------------------------------------------------
 
 	public TaskExecutor(
-			TaskExecutorConfiguration taskExecutorConfig,
-			ResourceID resourceID,
-			TaskManagerLocation taskManagerLocation,
-			MemoryManager memoryManager,
-			IOManager ioManager,
-			NetworkEnvironment networkEnvironment,
-			RpcService rpcService,
-			HighAvailabilityServices haServices) {
+		TaskExecutorConfiguration taskExecutorConfig,
+		TaskManagerLocation taskManagerLocation,
+		RpcService rpcService,
+		MemoryManager memoryManager,
+		IOManager ioManager,
+		NetworkEnvironment networkEnvironment,
+		HighAvailabilityServices haServices) {
 
 		super(rpcService);
 
 		checkArgument(taskExecutorConfig.getNumberOfSlots() > 0, "The number of slots has to be larger than 0.");
 
 		this.taskExecutorConfig = checkNotNull(taskExecutorConfig);
-		this.resourceID = checkNotNull(resourceID);
 		this.taskManagerLocation = checkNotNull(taskManagerLocation);
 		this.memoryManager = checkNotNull(memoryManager);
 		this.ioManager = checkNotNull(ioManager);
 		this.networkEnvironment = checkNotNull(networkEnvironment);
 		this.haServices = checkNotNull(haServices);
 
-		this.numberOfSlots = taskExecutorConfig.getNumberOfSlots();
+		this.numberOfSlots =  taskExecutorConfig.getNumberOfSlots();
 	}
 
 	// ------------------------------------------------------------------------
@@ -207,7 +158,6 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 		}
 	}
 
-	/**
 	 * Requests a slot from the TaskManager
 	 *
 	 * @param allocationID id for the request
@@ -220,126 +170,11 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 	}
 
 	/**
-	 * Starts and runs the TaskManager.
-	 * <p/>
-	 * This method first tries to select the network interface to use for the TaskManager
-	 * communication. The network interface is used both for the actor communication
-	 * (coordination) as well as for the data exchange between task managers. Unless
-	 * the hostname/interface is explicitly configured in the configuration, this
-	 * method will try out various interfaces and methods to connect to the JobManager
-	 * and select the one where the connection attempt is successful.
-	 * <p/>
-	 * After selecting the network interface, this method brings up an actor system
-	 * for the TaskManager and its actors, starts the TaskManager's services
-	 * (library cache, shuffle network stack, ...), and starts the TaskManager itself.
-	 *
-	 * @param configuration    The configuration for the TaskManager.
-	 * @param resourceID       The id of the resource which the task manager will run on.
-	 */
-	public static void selectNetworkInterfaceAndRunTaskManager(
-		Configuration configuration,
-		ResourceID resourceID) throws Exception {
-
-		final InetSocketAddress taskManagerAddress = selectNetworkInterfaceAndPort(configuration);
-
-		runTaskManager(taskManagerAddress.getHostName(), resourceID, taskManagerAddress.getPort(), configuration);
-	}
-
-	private static InetSocketAddress selectNetworkInterfaceAndPort(Configuration configuration)
-		throws Exception {
-		String taskManagerHostname = configuration.getString(ConfigConstants.TASK_MANAGER_HOSTNAME_KEY, null);
-		if (taskManagerHostname != null) {
-			LOG.info("Using configured hostname/address for TaskManager: " + taskManagerHostname);
-		} else {
-			LeaderRetrievalService leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(configuration);
-			FiniteDuration lookupTimeout = AkkaUtils.getLookupTimeout(configuration);
-
-			InetAddress taskManagerAddress = LeaderRetrievalUtils.findConnectingAddress(leaderRetrievalService, lookupTimeout);
-			taskManagerHostname = taskManagerAddress.getHostName();
-			LOG.info("TaskManager will use hostname/address '{}' ({}) for communication.",
-				taskManagerHostname, taskManagerAddress.getHostAddress());
-		}
-
-		// if no task manager port has been configured, use 0 (system will pick any free port)
-		final int actorSystemPort = configuration.getInteger(ConfigConstants.TASK_MANAGER_IPC_PORT_KEY, 0);
-		if (actorSystemPort < 0 || actorSystemPort > 65535) {
-			throw new IllegalConfigurationException("Invalid value for '" +
-				ConfigConstants.TASK_MANAGER_IPC_PORT_KEY +
-				"' (port for the TaskManager actor system) : " + actorSystemPort +
-				" - Leave config parameter empty or use 0 to let the system choose a port automatically.");
-		}
-
-		return new InetSocketAddress(taskManagerHostname, actorSystemPort);
-	}
-
-	/**
-	 * Starts and runs the TaskManager. Brings up an actor system for the TaskManager and its
-	 * actors, starts the TaskManager's services (library cache, shuffle network stack, ...),
-	 * and starts the TaskManager itself.
-	 * <p/>
-	 * This method will also spawn a process reaper for the TaskManager (kill the process if
-	 * the actor fails) and optionally start the JVM memory logging thread.
-	 *
-	 * @param taskManagerHostname The hostname/address of the interface where the actor system
-	 *                            will communicate.
-	 * @param resourceID          The id of the resource which the task manager will run on.
-	 * @param actorSystemPort   The port at which the actor system will communicate.
-	 * @param configuration       The configuration for the TaskManager.
-	 */
-	private static void runTaskManager(
-		String taskManagerHostname,
-		ResourceID resourceID,
-		int actorSystemPort,
-		final Configuration configuration) throws Exception {
-
-		LOG.info("Starting TaskManager");
-
-		// Bring up the TaskManager actor system first, bind it to the given address.
-
-		LOG.info("Starting TaskManager actor system at " +
-			NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort));
-
-		final ActorSystem taskManagerSystem;
-		try {
-			Tuple2<String, Object> address = new Tuple2<String, Object>(taskManagerHostname, actorSystemPort);
-			Config akkaConfig = AkkaUtils.getAkkaConfig(configuration, new Some<>(address));
-			LOG.debug("Using akka configuration\n " + akkaConfig);
-			taskManagerSystem = AkkaUtils.createActorSystem(akkaConfig);
-		} catch (Throwable t) {
-			if (t instanceof ChannelException) {
-				Throwable cause = t.getCause();
-				if (cause != null && t.getCause() instanceof BindException) {
-					String address = NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort);
-					throw new IOException("Unable to bind TaskManager actor system to address " +
-						address + " - " + cause.getMessage(), t);
-				}
-			}
-			throw new Exception("Could not create TaskManager actor system", t);
-		}
-
-		// start akka rpc service based on actor system
-		final Time timeout = Time.milliseconds(AkkaUtils.getTimeout(configuration).toMillis());
-		final AkkaRpcService akkaRpcService = new AkkaRpcService(taskManagerSystem, timeout);
-
-		// start high availability service to implement getResourceManagerLeaderRetriever method only
-		final HighAvailabilityServices haServices = new HighAvailabilityServices() {
-			@Override
-			public LeaderRetrievalService getResourceManagerLeaderRetriever() throws Exception {
-				return LeaderRetrievalUtils.createLeaderRetrievalService(configuration);
-			}
-
-			@Override
 			public LeaderRetrievalService getJobMasterLeaderRetriever(JobID jobID) throws Exception {
 				return null;
 			}
 
 			@Override
-			public LeaderElectionService getResourceManagerLeaderElectionService() throws Exception {
-				return null;
-			}
-
-			@Override
-			public LeaderElectionService getJobMasterLeaderElectionService(JobID jobID) throws Exception {
 				return null;
 			}
 
@@ -350,552 +185,12 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 
 			@Override
 			public SubmittedJobGraphStore getSubmittedJobGraphStore() throws Exception {
-				return null;
-			}
-		};
-
-		// start all the TaskManager services (network stack,  library cache, ...)
-		// and the TaskManager actor
-		try {
-			LOG.info("Starting TaskManager actor");
-			TaskExecutor taskExecutor = startTaskManagerComponentsAndActor(
-				configuration,
-				resourceID,
-				akkaRpcService,
-				taskManagerHostname,
-				haServices,
-				false);
-
-			taskExecutor.start();
-
-			// if desired, start the logging daemon that periodically logs the memory usage information
-			if (LOG.isInfoEnabled() && configuration.getBoolean(
-				ConfigConstants.TASK_MANAGER_DEBUG_MEMORY_USAGE_START_LOG_THREAD,
-				ConfigConstants.DEFAULT_TASK_MANAGER_DEBUG_MEMORY_USAGE_START_LOG_THREAD)) {
-				LOG.info("Starting periodic memory usage logger");
-
-				long interval = configuration.getLong(
-					ConfigConstants.TASK_MANAGER_DEBUG_MEMORY_USAGE_LOG_INTERVAL_MS,
-					ConfigConstants.DEFAULT_TASK_MANAGER_DEBUG_MEMORY_USAGE_LOG_INTERVAL_MS);
-
-				MemoryLogger logger = new MemoryLogger(LOG, interval, taskManagerSystem);
-				logger.start();
-			}
-
-			// block until everything is done
-			taskManagerSystem.awaitTermination();
-		} catch (Throwable t) {
-			LOG.error("Error while starting up taskManager", t);
-			try {
-				taskManagerSystem.shutdown();
-			} catch (Throwable tt) {
-				LOG.warn("Could not cleanly shut down actor system", tt);
-			}
-			throw t;
-		}
-	}
-
-	// --------------------------------------------------------------------------
-	//  Starting and running the TaskManager
-	// --------------------------------------------------------------------------
-
-	/**
-	 * @param configuration                 The configuration for the TaskManager.
-	 * @param resourceID                    The id of the resource which the task manager will run on.
-	 * @param rpcService                  The rpc service which is used to start and connect to the TaskManager RpcEndpoint .
-	 * @param taskManagerHostname       The hostname/address that describes the TaskManager's data location.
-	 * @param haServices        Optionally, a high availability service can be provided. If none is given,
-	 *                                      then a HighAvailabilityServices is constructed from the configuration.
-	 * @param localTaskManagerCommunication     If true, the TaskManager will not initiate the TCP network stack.
-	 * @return An ActorRef to the TaskManager actor.
-	 * @throws IllegalConfigurationException     Thrown, if the given config contains illegal values.
-	 * @throws IOException      Thrown, if any of the I/O components (such as buffer pools,
-	 *                                       I/O manager, ...) cannot be properly started.
-	 * @throws Exception      Thrown is some other error occurs while parsing the configuration
-	 *                                      or starting the TaskManager components.
-	 */
-	public static TaskExecutor startTaskManagerComponentsAndActor(
-		Configuration configuration,
-		ResourceID resourceID,
-		RpcService rpcService,
-		String taskManagerHostname,
-		HighAvailabilityServices haServices,
-		boolean localTaskManagerCommunication) throws Exception {
-
-		final TaskExecutorConfiguration taskExecutorConfig = parseTaskManagerConfiguration(
-			configuration, taskManagerHostname, localTaskManagerCommunication);
-
-		TaskManagerComponents taskManagerComponents = createTaskManagerComponents(
-			resourceID,
-			InetAddress.getByName(taskManagerHostname),
-			taskExecutorConfig,
-			configuration);
-
-		final TaskExecutor taskExecutor = new TaskExecutor(
-			taskExecutorConfig,
-			resourceID,
-			taskManagerComponents.getTaskManagerLocation(),
-			taskManagerComponents.getMemoryManager(),
-			taskManagerComponents.getIOManager(),
-			taskManagerComponents.getNetworkEnvironment(),
-			rpcService,
-			haServices);
-
-		return taskExecutor;
-	}
-
-	/**
-	 * Creates and returns the task manager components.
-	 *
-	 * @param resourceID resource ID of the task manager
-	 * @param taskManagerAddress address of the task manager
-	 * @param taskExecutorConfig task manager configuration
-	 * @param configuration of Flink
-	 * @return task manager components
-	 * @throws Exception
-	 */
-	private static TaskExecutor.TaskManagerComponents createTaskManagerComponents(
-		ResourceID resourceID,
-		InetAddress taskManagerAddress,
-		TaskExecutorConfiguration taskExecutorConfig,
-		Configuration configuration) throws Exception {
-		MemoryType memType = taskExecutorConfig.getNetworkConfig().memoryType();
-
-		// pre-start checks
-		checkTempDirs(taskExecutorConfig.getTmpDirPaths());
-
-		NetworkEnvironmentConfiguration networkEnvironmentConfiguration = taskExecutorConfig.getNetworkConfig();
-
-		NetworkBufferPool networkBufferPool = new NetworkBufferPool(
-			networkEnvironmentConfiguration.numNetworkBuffers(),
-			networkEnvironmentConfiguration.networkBufferSize(),
-			networkEnvironmentConfiguration.memoryType());
-
-		ConnectionManager connectionManager;
-
-		if (networkEnvironmentConfiguration.nettyConfig().isDefined()) {
-			connectionManager = new NettyConnectionManager(networkEnvironmentConfiguration.nettyConfig().get());
-		} else {
-			connectionManager = new LocalConnectionManager();
-		}
-
-		ResultPartitionManager resultPartitionManager = new ResultPartitionManager();
-		TaskEventDispatcher taskEventDispatcher = new TaskEventDispatcher();
-
-		KvStateRegistry kvStateRegistry = new KvStateRegistry();
-
-		KvStateServer kvStateServer;
-
-		if (networkEnvironmentConfiguration.nettyConfig().isDefined()) {
-			NettyConfig nettyConfig = networkEnvironmentConfiguration.nettyConfig().get();
-
-			int numNetworkThreads = networkEnvironmentConfiguration.queryServerNetworkThreads() == 0 ?
-				nettyConfig.getNumberOfSlots() : networkEnvironmentConfiguration.queryServerNetworkThreads();
-
-			int numQueryThreads = networkEnvironmentConfiguration.queryServerQueryThreads() == 0 ?
-				nettyConfig.getNumberOfSlots() : networkEnvironmentConfiguration.queryServerQueryThreads();
-
-			kvStateServer = new KvStateServer(
-				taskManagerAddress,
-				networkEnvironmentConfiguration.queryServerPort(),
-				numNetworkThreads,
-				numQueryThreads,
-				kvStateRegistry,
-				new DisabledKvStateRequestStats());
-		} else {
-			kvStateServer = null;
-		}
-
-		// we start the network first, to make sure it can allocate its buffers first
-		final NetworkEnvironment network = new NetworkEnvironment(
-			networkBufferPool,
-			connectionManager,
-			resultPartitionManager,
-			taskEventDispatcher,
-			kvStateRegistry,
-			kvStateServer,
-			networkEnvironmentConfiguration.ioMode(),
-			networkEnvironmentConfiguration.partitionRequestInitialBackoff(),
-			networkEnvironmentConfiguration.partitinRequestMaxBackoff());
-
-		network.start();
-
-		TaskManagerLocation taskManagerLocation = new TaskManagerLocation(
-			resourceID,
-			taskManagerAddress,
-			network.getConnectionManager().getDataPort());
-
-		// computing the amount of memory to use depends on how much memory is available
-		// it strictly needs to happen AFTER the network stack has been initialized
-
-		// check if a value has been configured
-		long configuredMemory = configuration.getLong(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, -1L);
-		checkConfigParameter(configuredMemory == -1 || configuredMemory > 0, configuredMemory,
-			ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY,
-			"MemoryManager needs at least one MB of memory. " +
-				"If you leave this config parameter empty, the system automatically " +
-				"pick a fraction of the available memory.");
-
-		final long memorySize;
-		boolean preAllocateMemory = configuration.getBoolean(
-			ConfigConstants.TASK_MANAGER_MEMORY_PRE_ALLOCATE_KEY,
-			ConfigConstants.DEFAULT_TASK_MANAGER_MEMORY_PRE_ALLOCATE);
-		if (configuredMemory > 0) {
-			if (preAllocateMemory) {
-				LOG.info("Using {} MB for managed memory." , configuredMemory);
-			} else {
-				LOG.info("Limiting managed memory to {} MB, memory will be allocated lazily." , configuredMemory);
-			}
-			memorySize = configuredMemory << 20; // megabytes to bytes
-		} else {
-			float fraction = configuration.getFloat(
-				ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY,
-				ConfigConstants.DEFAULT_MEMORY_MANAGER_MEMORY_FRACTION);
-			checkConfigParameter(fraction > 0.0f && fraction < 1.0f, fraction,
-				ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY,
-				"MemoryManager fraction of the free memory must be between 0.0 and 1.0");
-
-			if (memType == MemoryType.HEAP) {
-				long relativeMemSize = (long) (EnvironmentInformation.getSizeOfFreeHeapMemoryWithDefrag() * fraction);
-				if (preAllocateMemory) {
-					LOG.info("Using {} of the currently free heap space for managed heap memory ({} MB)." ,
-						fraction , relativeMemSize >> 20);
-				} else {
-					LOG.info("Limiting managed memory to {} of the currently free heap space ({} MB), " +
-						"memory will be allocated lazily." , fraction , relativeMemSize >> 20);
-				}
-				memorySize = relativeMemSize;
-			} else if (memType == MemoryType.OFF_HEAP) {
-				// The maximum heap memory has been adjusted according to the fraction
-				long maxMemory = EnvironmentInformation.getMaxJvmHeapMemory();
-				long directMemorySize = (long) (maxMemory / (1.0 - fraction) * fraction);
-				if (preAllocateMemory) {
-					LOG.info("Using {} of the maximum memory size for managed off-heap memory ({} MB)." ,
-						fraction, directMemorySize >> 20);
-				} else {
-					LOG.info("Limiting managed memory to {} of the maximum memory size ({} MB)," +
-						" memory will be allocated lazily.", fraction, directMemorySize >> 20);
-				}
-				memorySize = directMemorySize;
-			} else {
-				throw new RuntimeException("No supported memory type detected.");
-			}
-		}
-
-		// now start the memory manager
-		final MemoryManager memoryManager;
-		try {
-			memoryManager = new MemoryManager(
-				memorySize,
-				taskExecutorConfig.getNumberOfSlots(),
-				taskExecutorConfig.getNetworkConfig().networkBufferSize(),
-				memType,
-				preAllocateMemory);
-		} catch (OutOfMemoryError e) {
-			if (memType == MemoryType.HEAP) {
-				throw new Exception("OutOfMemory error (" + e.getMessage() +
-					") while allocating the TaskManager heap memory (" + memorySize + " bytes).", e);
-			} else if (memType == MemoryType.OFF_HEAP) {
-				throw new Exception("OutOfMemory error (" + e.getMessage() +
-					") while allocating the TaskManager off-heap memory (" + memorySize +
-					" bytes).Try increasing the maximum direct memory (-XX:MaxDirectMemorySize)", e);
-			} else {
-				throw e;
-			}
-		}
-
-		// start the I/O manager, it will create some temp directories.
-		final IOManager ioManager = new IOManagerAsync(taskExecutorConfig.getTmpDirPaths());
-
-		return new TaskExecutor.TaskManagerComponents(taskManagerLocation, memoryManager, ioManager, network);
-	}
-
-	// --------------------------------------------------------------------------
-	//  Parsing and checking the TaskManager Configuration
-	// --------------------------------------------------------------------------
-
-	/**
-	 * Utility method to extract TaskManager config parameters from the configuration and to
-	 * sanity check them.
-	 *
-	 * @param configuration                 The configuration.
-	 * @param taskManagerHostname           The host name under which the TaskManager communicates.
-	 * @param localTaskManagerCommunication             True, to skip initializing the network stack.
-	 *                                      Use only in cases where only one task manager runs.
-	 * @return TaskExecutorConfiguration that wrappers InstanceConnectionInfo, NetworkEnvironmentConfiguration, etc.
-	 */
-	private static TaskExecutorConfiguration parseTaskManagerConfiguration(
-		Configuration configuration,
-		String taskManagerHostname,
-		boolean localTaskManagerCommunication) throws Exception {
-
-		// ------- read values from the config and check them ---------
-		//                      (a lot of them)
-
-		// ----> hosts / ports for communication and data exchange
-
-		int dataport = configuration.getInteger(ConfigConstants.TASK_MANAGER_DATA_PORT_KEY,
-			ConfigConstants.DEFAULT_TASK_MANAGER_DATA_PORT);
-		if (dataport == 0) {
-			dataport = NetUtils.getAvailablePort();
-		}
-		checkConfigParameter(dataport > 0, dataport, ConfigConstants.TASK_MANAGER_DATA_PORT_KEY,
-			"Leave config parameter empty or use 0 to let the system choose a port automatically.");
-
-		InetAddress taskManagerAddress = InetAddress.getByName(taskManagerHostname);
-		final InetSocketAddress taskManagerInetSocketAddress = new InetSocketAddress(taskManagerAddress, dataport);
-
-		// ----> memory / network stack (shuffles/broadcasts), task slots, temp directories
-
-		// we need this because many configs have been written with a "-1" entry
-		int slots = configuration.getInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 1);
-		if (slots == -1) {
-			slots = 1;
-		}
-		checkConfigParameter(slots >= 1, slots, ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS,
-			"Number of task slots must be at least one.");
-
-		final int numNetworkBuffers = configuration.getInteger(
-			ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY,
-			ConfigConstants.DEFAULT_TASK_MANAGER_NETWORK_NUM_BUFFERS);
-		checkConfigParameter(numNetworkBuffers > 0, numNetworkBuffers,
-			ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY, "");
-
-		final int pageSize = configuration.getInteger(
-			ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY,
-			ConfigConstants.DEFAULT_TASK_MANAGER_MEMORY_SEGMENT_SIZE);
-		// check page size of for minimum size
-		checkConfigParameter(pageSize >= MemoryManager.MIN_PAGE_SIZE, pageSize,
-			ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY,
-			"Minimum memory segment size is " + MemoryManager.MIN_PAGE_SIZE);
-		// check page size for power of two
-		checkConfigParameter(MathUtils.isPowerOf2(pageSize), pageSize,
-			ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY,
-			"Memory segment size must be a power of 2.");
-
-		// check whether we use heap or off-heap memory
-		final MemoryType memType;
-		if (configuration.getBoolean(ConfigConstants.TASK_MANAGER_MEMORY_OFF_HEAP_KEY, false)) {
-			memType = MemoryType.OFF_HEAP;
-		} else {
-			memType = MemoryType.HEAP;
-		}
-
-		// initialize the memory segment factory accordingly
-		if (memType == MemoryType.HEAP) {
-			if (!MemorySegmentFactory.initializeIfNotInitialized(HeapMemorySegment.FACTORY)) {
-				throw new Exception("Memory type is set to heap memory, but memory segment " +
-					"factory has been initialized for off-heap memory segments");
-			}
-		} else {
-			if (!MemorySegmentFactory.initializeIfNotInitialized(HybridMemorySegment.FACTORY)) {
-				throw new Exception("Memory type is set to off-heap memory, but memory segment " +
-					"factory has been initialized for heap memory segments");
-			}
-		}
-
-		final String[] tmpDirs = configuration.getString(
-			ConfigConstants.TASK_MANAGER_TMP_DIR_KEY,
-			ConfigConstants.DEFAULT_TASK_MANAGER_TMP_PATH).split(",|" + File.pathSeparator);
-
-		final NettyConfig nettyConfig;
-		if (!localTaskManagerCommunication) {
-			nettyConfig = new NettyConfig(
-				taskManagerInetSocketAddress.getAddress(),
-				taskManagerInetSocketAddress.getPort(),
-				pageSize,
-				slots,
-				configuration);
-		} else {
-			nettyConfig = null;
-		}
-
-		// Default spill I/O mode for intermediate results
-		final String syncOrAsync = configuration.getString(
-			ConfigConstants.TASK_MANAGER_NETWORK_DEFAULT_IO_MODE,
-			ConfigConstants.DEFAULT_TASK_MANAGER_NETWORK_DEFAULT_IO_MODE);
-
-		final IOMode ioMode;
-		if (syncOrAsync.equals("async")) {
-			ioMode = IOManager.IOMode.ASYNC;
-		} else {
-			ioMode = IOManager.IOMode.SYNC;
-		}
-
-		final int queryServerPort =  configuration.getInteger(
-			ConfigConstants.QUERYABLE_STATE_SERVER_PORT,
-			ConfigConstants.DEFAULT_QUERYABLE_STATE_SERVER_PORT);
-
-		final int queryServerNetworkThreads =  configuration.getInteger(
-			ConfigConstants.QUERYABLE_STATE_SERVER_NETWORK_THREADS,
-			ConfigConstants.DEFAULT_QUERYABLE_STATE_SERVER_NETWORK_THREADS);
-
-		final int queryServerQueryThreads =  configuration.getInteger(
-			ConfigConstants.QUERYABLE_STATE_SERVER_QUERY_THREADS,
-			ConfigConstants.DEFAULT_QUERYABLE_STATE_SERVER_QUERY_THREADS);
-
-		final NetworkEnvironmentConfiguration networkConfig = new NetworkEnvironmentConfiguration(
-			numNetworkBuffers,
-			pageSize,
-			memType,
-			ioMode,
-			queryServerPort,
-			queryServerNetworkThreads,
-			queryServerQueryThreads,
-			Option.apply(nettyConfig),
-			500,
-			30000);
-
-		// ----> timeouts, library caching, profiling
-
-		final FiniteDuration timeout;
-		try {
-			timeout = AkkaUtils.getTimeout(configuration);
-		} catch (Exception e) {
-			throw new IllegalArgumentException(
-				"Invalid format for '" + ConfigConstants.AKKA_ASK_TIMEOUT +
-					"'.Use formats like '50 s' or '1 min' to specify the timeout.");
-		}
-		LOG.info("Messages between TaskManager and JobManager have a max timeout of " + timeout);
-
-		final long cleanupInterval = configuration.getLong(
-			ConfigConstants.LIBRARY_CACHE_MANAGER_CLEANUP_INTERVAL,
-			ConfigConstants.DEFAULT_LIBRARY_CACHE_MANAGER_CLEANUP_INTERVAL) * 1000;
-
-		final FiniteDuration finiteRegistrationDuration;
-		try {
-			Duration maxRegistrationDuration = Duration.create(configuration.getString(
-				ConfigConstants.TASK_MANAGER_MAX_REGISTRATION_DURATION,
-				ConfigConstants.DEFAULT_TASK_MANAGER_MAX_REGISTRATION_DURATION));
-			if (maxRegistrationDuration.isFinite()) {
-				finiteRegistrationDuration = new FiniteDuration(maxRegistrationDuration.toSeconds(), TimeUnit.SECONDS);
-			} else {
-				finiteRegistrationDuration = null;
-			}
-		} catch (NumberFormatException e) {
-			throw new IllegalArgumentException("Invalid format for parameter " +
-				ConfigConstants.TASK_MANAGER_MAX_REGISTRATION_DURATION, e);
-		}
-
-		final FiniteDuration initialRegistrationPause;
-		try {
-			Duration pause = Duration.create(configuration.getString(
-				ConfigConstants.TASK_MANAGER_INITIAL_REGISTRATION_PAUSE,
-				ConfigConstants.DEFAULT_TASK_MANAGER_INITIAL_REGISTRATION_PAUSE));
-			if (pause.isFinite()) {
-				initialRegistrationPause = new FiniteDuration(pause.toSeconds(), TimeUnit.SECONDS);
-			} else {
-				throw new IllegalArgumentException("The initial registration pause must be finite: " + pause);
-			}
-		} catch (NumberFormatException e) {
-			throw new IllegalArgumentException("Invalid format for parameter " +
-				ConfigConstants.TASK_MANAGER_INITIAL_REGISTRATION_PAUSE, e);
-		}
-
-		final FiniteDuration maxRegistrationPause;
-		try {
-			Duration pause = Duration.create(configuration.getString(
-				ConfigConstants.TASK_MANAGER_MAX_REGISTARTION_PAUSE,
-				ConfigConstants.DEFAULT_TASK_MANAGER_MAX_REGISTRATION_PAUSE));
-			if (pause.isFinite()) {
-				maxRegistrationPause = new FiniteDuration(pause.toSeconds(), TimeUnit.SECONDS);
-			} else {
-				throw new IllegalArgumentException("The maximum registration pause must be finite: " + pause);
-			}
-		} catch (NumberFormatException e) {
-			throw new IllegalArgumentException("Invalid format for parameter " +
-				ConfigConstants.TASK_MANAGER_INITIAL_REGISTRATION_PAUSE, e);
-		}
-
-		final FiniteDuration refusedRegistrationPause;
-		try {
-			Duration pause = Duration.create(configuration.getString(
-				ConfigConstants.TASK_MANAGER_REFUSED_REGISTRATION_PAUSE,
-				ConfigConstants.DEFAULT_TASK_MANAGER_REFUSED_REGISTRATION_PAUSE));
-			if (pause.isFinite()) {
-				refusedRegistrationPause = new FiniteDuration(pause.toSeconds(), TimeUnit.SECONDS);
-			} else {
-				throw new IllegalArgumentException("The refused registration pause must be finite: " + pause);
-			}
-		} catch (NumberFormatException e) {
-			throw new IllegalArgumentException("Invalid format for parameter " +
-				ConfigConstants.TASK_MANAGER_INITIAL_REGISTRATION_PAUSE, e);
-		}
-
-		return new TaskExecutorConfiguration(
-			tmpDirs,
-			cleanupInterval,
-			networkConfig,
-			timeout,
-			finiteRegistrationDuration,
-			slots,
-			configuration,
-			initialRegistrationPause,
-			maxRegistrationPause,
-			refusedRegistrationPause);
-	}
-
-	/**
-	 * Validates a condition for a config parameter and displays a standard exception, if the
-	 * the condition does not hold.
-	 *
-	 * @param condition    The condition that must hold. If the condition is false, an exception is thrown.
-	 * @param parameter    The parameter value. Will be shown in the exception message.
-	 * @param name         The name of the config parameter. Will be shown in the exception message.
-	 * @param errorMessage The optional custom error message to append to the exception message.
-	 */
-	private static void checkConfigParameter(
-		boolean condition,
-		Object parameter,
-		String name,
-		String errorMessage) {
-		if (!condition) {
-			throw new IllegalConfigurationException("Invalid configuration value for " + name + " : " + parameter + " - " + errorMessage);
-		}
-	}
-
-	/**
-	 * Validates that all the directories denoted by the strings do actually exist, are proper
-	 * directories (not files), and are writable.
-	 *
-	 * @param tmpDirs The array of directory paths to check.
-	 * @throws Exception Thrown if any of the directories does not exist or is not writable
-	 *                   or is a file, rather than a directory.
-	 */
-	private static void checkTempDirs(String[] tmpDirs) throws IOException {
-		for (String dir : tmpDirs) {
-			if (dir != null && !dir.equals("")) {
-				File file = new File(dir);
-				if (!file.exists()) {
-					throw new IOException("Temporary file directory " + file.getAbsolutePath() + " does not exist.");
-				}
-				if (!file.isDirectory()) {
-					throw new IOException("Temporary file directory " + file.getAbsolutePath() + " is not a directory.");
-				}
-				if (!file.canWrite()) {
-					throw new IOException("Temporary file directory " + file.getAbsolutePath() + " is not writable.");
-				}
-
-				if (LOG.isInfoEnabled()) {
-					long totalSpaceGb = file.getTotalSpace() >> 30;
-					long usableSpaceGb = file.getUsableSpace() >> 30;
-					double usablePercentage = (double)usableSpaceGb / totalSpaceGb * 100;
-					String path = file.getAbsolutePath();
-					LOG.info(String.format("Temporary file directory '%s': total %d GB, " + "usable %d GB (%.2f%% usable)",
-						path, totalSpaceGb, usableSpaceGb, usablePercentage));
-				}
-			} else {
-				throw new IllegalArgumentException("Temporary file directory #$id is null.");
-			}
-		}
-	}
-
 	// ------------------------------------------------------------------------
 	//  Properties
 	// ------------------------------------------------------------------------
 
 	public ResourceID getResourceID() {
-		return resourceID;
+		return taskManagerLocation.getResourceID();
 	}
 
 	// ------------------------------------------------------------------------
@@ -959,37 +254,4 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 		}
 	}
 
-	private static class TaskManagerComponents {
-		private final TaskManagerLocation taskManagerLocation;
-		private final MemoryManager memoryManager;
-		private final IOManager ioManager;
-		private final NetworkEnvironment networkEnvironment;
-
-		private TaskManagerComponents(
-				TaskManagerLocation taskManagerLocation,
-				MemoryManager memoryManager,
-				IOManager ioManager,
-				NetworkEnvironment networkEnvironment) {
-			this.taskManagerLocation = Preconditions.checkNotNull(taskManagerLocation);
-			this.memoryManager = Preconditions.checkNotNull(memoryManager);
-			this.ioManager = Preconditions.checkNotNull(ioManager);
-			this.networkEnvironment = Preconditions.checkNotNull(networkEnvironment);
-		}
-
-		public MemoryManager getMemoryManager() {
-			return memoryManager;
-		}
-
-		public IOManager getIOManager() {
-			return ioManager;
-		}
-
-		public NetworkEnvironment getNetworkEnvironment() {
-			return networkEnvironment;
-		}
-
-		public TaskManagerLocation getTaskManagerLocation() {
-			return taskManagerLocation;
-		}
-	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/c34f13c2/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManagerRunner.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManagerRunner.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManagerRunner.java
new file mode 100644
index 0000000..4f756fb
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManagerRunner.java
@@ -0,0 +1,749 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.taskmanager;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.IllegalConfigurationException;
+import org.apache.flink.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.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
+import org.apache.flink.runtime.io.disk.iomanager.IOManager;
+import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
+import org.apache.flink.runtime.io.network.ConnectionManager;
+import org.apache.flink.runtime.io.network.LocalConnectionManager;
+import org.apache.flink.runtime.io.network.NetworkEnvironment;
+import org.apache.flink.runtime.io.network.TaskEventDispatcher;
+import org.apache.flink.runtime.io.network.buffer.NetworkBufferPool;
+import org.apache.flink.runtime.io.network.netty.NettyConfig;
+import org.apache.flink.runtime.io.network.netty.NettyConnectionManager;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionManager;
+import org.apache.flink.runtime.leaderelection.LeaderElectionService;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+import org.apache.flink.runtime.memory.MemoryManager;
+import org.apache.flink.runtime.query.KvStateRegistry;
+import org.apache.flink.runtime.query.netty.DisabledKvStateRequestStats;
+import org.apache.flink.runtime.query.netty.KvStateServer;
+import org.apache.flink.runtime.rpc.RpcService;
+import org.apache.flink.runtime.rpc.akka.AkkaRpcService;
+import org.apache.flink.runtime.taskexecutor.TaskExecutor;
+import org.apache.flink.runtime.taskexecutor.TaskExecutorConfiguration;
+import org.apache.flink.runtime.util.EnvironmentInformation;
+import org.apache.flink.runtime.util.LeaderRetrievalUtils;
+import org.apache.flink.util.MathUtils;
+import org.apache.flink.util.NetUtils;
+
+import akka.actor.ActorSystem;
+import akka.util.Timeout;
+import com.typesafe.config.Config;
+import org.apache.flink.util.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.Option;
+import scala.Some;
+import scala.Tuple2;
+import scala.concurrent.duration.Duration;
+import scala.concurrent.duration.FiniteDuration;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * This class is the executable entry point for the task manager in yarn or standalone mode.
+ * It constructs the related components (network, I/O manager, memory manager, RPC service, HA service)
+ * and starts them.
+ */
+public class TaskManagerRunner {
+
+	private static final Logger LOG = LoggerFactory.getLogger(TaskManagerRunner.class);
+
+	/**
+	 * Constructs related components of the TaskManager and starts them.
+	 *
+	 * @param configuration                 The configuration for the TaskManager.
+	 * @param resourceID                    The id of the resource which the task manager will run on.
+	 * @param rpcService                    Optionally, The rpc service which is used to start and connect to the TaskManager RpcEndpoint .
+	 *                                                 If none is given, then a RpcService is constructed from the configuration.
+	 * @param taskManagerHostname   Optionally, The hostname/address that describes the TaskManager's data location.
+	 *                                                 If none is given, it can be got from the configuration.
+	 * @param localTaskManagerCommunication      If true, the TaskManager will not initiate the TCP network stack.
+	 * @param haServices                    Optionally, a high availability service can be provided. If none is given,
+	 *                                                 then a HighAvailabilityServices is constructed from the configuration.
+	 */
+	public static void createAndStartComponents(
+		final Configuration configuration,
+		final ResourceID resourceID,
+		RpcService rpcService,
+		String taskManagerHostname,
+		boolean localTaskManagerCommunication,
+		HighAvailabilityServices haServices) throws Exception {
+
+		checkNotNull(configuration);
+		checkNotNull(resourceID);
+
+		if (taskManagerHostname == null || taskManagerHostname.isEmpty()) {
+			taskManagerHostname = selectNetworkInterface(configuration);
+		}
+
+		if (rpcService == null) {
+			// if no task manager port has been configured, use 0 (system will pick any free port)
+			final int actorSystemPort = configuration.getInteger(ConfigConstants.TASK_MANAGER_IPC_PORT_KEY, 0);
+			if (actorSystemPort < 0 || actorSystemPort > 65535) {
+				throw new IllegalConfigurationException("Invalid value for '" +
+					ConfigConstants.TASK_MANAGER_IPC_PORT_KEY +
+					"' (port for the TaskManager actor system) : " + actorSystemPort +
+					" - Leave config parameter empty or use 0 to let the system choose a port automatically.");
+			}
+			rpcService = createRpcService(configuration, taskManagerHostname, actorSystemPort);
+		}
+
+		if(haServices == null) {
+			// start high availability service to implement getResourceManagerLeaderRetriever method only
+			haServices = new HighAvailabilityServices() {
+				@Override
+				public LeaderRetrievalService getResourceManagerLeaderRetriever() throws Exception {
+					return LeaderRetrievalUtils.createLeaderRetrievalService(configuration);
+				}
+
+				@Override
+				public LeaderElectionService getResourceManagerLeaderElectionService() throws Exception {
+					return null;
+				}
+
+				@Override
+				public LeaderElectionService getJobMasterLeaderElectionService(JobID jobID) throws Exception {
+					return null;
+				}
+			};
+		}
+
+		createAndStartTaskManagerComponents(
+			configuration,
+			resourceID,
+			rpcService,
+			taskManagerHostname,
+			haServices,
+			localTaskManagerCommunication);
+	}
+
+	/**
+	 * <p/>
+	 * This method tries to select the network interface to use for the TaskManager
+	 * communication. The network interface is used both for the actor communication
+	 * (coordination) as well as for the data exchange between task managers. Unless
+	 * the hostname/interface is explicitly configured in the configuration, this
+	 * method will try out various interfaces and methods to connect to the JobManager
+	 * and select the one where the connection attempt is successful.
+	 * <p/>
+	 *
+	 * @param configuration    The configuration for the TaskManager.
+	 * @return  The host name under which the TaskManager communicates.
+	 */
+	private static String selectNetworkInterface(Configuration configuration) throws Exception {
+		String taskManagerHostname = configuration.getString(ConfigConstants.TASK_MANAGER_HOSTNAME_KEY, null);
+		if (taskManagerHostname != null) {
+			LOG.info("Using configured hostname/address for TaskManager: " + taskManagerHostname);
+		} else {
+			LeaderRetrievalService leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(configuration);
+			FiniteDuration lookupTimeout = AkkaUtils.getLookupTimeout(configuration);
+
+			InetAddress taskManagerAddress = LeaderRetrievalUtils.findConnectingAddress(leaderRetrievalService, lookupTimeout);
+			taskManagerHostname = taskManagerAddress.getHostName();
+			LOG.info("TaskManager will use hostname/address '{}' ({}) for communication.",
+				taskManagerHostname, taskManagerAddress.getHostAddress());
+		}
+
+		return taskManagerHostname;
+	}
+
+	/**
+	 * Utility method to create RPC service from configuration and hostname, port.
+	 *
+	 * @param configuration                 The configuration for the TaskManager.
+	 * @param taskManagerHostname   The hostname/address that describes the TaskManager's data location.
+	 * @param actorSystemPort           If true, the TaskManager will not initiate the TCP network stack.
+	 * @return   The rpc service which is used to start and connect to the TaskManager RpcEndpoint .
+	 * @throws java.io.IOException      Thrown, if the actor system can not bind to the address
+	 * @throws java.lang.Exception      Thrown is some other error occurs while creating akka actor system
+	 */
+	private static RpcService createRpcService(Configuration configuration, String taskManagerHostname, int actorSystemPort)
+		throws Exception{
+
+		// Bring up the TaskManager actor system first, bind it to the given address.
+
+		LOG.info("Starting TaskManager actor system at " +
+			NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort));
+
+		final ActorSystem taskManagerSystem;
+		try {
+			Tuple2<String, Object> address = new Tuple2<String, Object>(taskManagerHostname, actorSystemPort);
+			Config akkaConfig = AkkaUtils.getAkkaConfig(configuration, new Some<>(address));
+			LOG.debug("Using akka configuration\n " + akkaConfig);
+			taskManagerSystem = AkkaUtils.createActorSystem(akkaConfig);
+		} catch (Throwable t) {
+			if (t instanceof org.jboss.netty.channel.ChannelException) {
+				Throwable cause = t.getCause();
+				if (cause != null && t.getCause() instanceof java.net.BindException) {
+					String address = NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort);
+					throw new IOException("Unable to bind TaskManager actor system to address " +
+						address + " - " + cause.getMessage(), t);
+				}
+			}
+			throw new Exception("Could not create TaskManager actor system", t);
+		}
+
+		// start akka rpc service based on actor system
+		final Timeout timeout = new Timeout(AkkaUtils.getTimeout(configuration).toMillis(), TimeUnit.MILLISECONDS);
+		final AkkaRpcService akkaRpcService = new AkkaRpcService(taskManagerSystem, timeout);
+
+		return akkaRpcService;
+	}
+
+	/**
+	 * @param configuration                 The configuration for the TaskManager.
+	 * @param resourceID                    The id of the resource which the task manager will run on.
+	 * @param rpcService                    The rpc service which is used to start and connect to the TaskManager RpcEndpoint .
+	 * @param taskManagerHostname   The hostname/address that describes the TaskManager's data location.
+	 * @param haServices                    Optionally, a high availability service can be provided. If none is given,
+	 *                                                  then a HighAvailabilityServices is constructed from the configuration.
+	 * @param localTaskManagerCommunication     If true, the TaskManager will not initiate the TCP network stack.
+	 * @throws IllegalConfigurationException        Thrown, if the given config contains illegal values.
+	 * @throws IOException      Thrown, if any of the I/O components (such as buffer pools, I/O manager, ...)
+	 *                                              cannot be properly started.
+	 * @throws Exception      Thrown is some other error occurs while parsing the configuration or
+	 *                                              starting the TaskManager components.
+	 */
+	private static void createAndStartTaskManagerComponents(
+		Configuration configuration,
+		ResourceID resourceID,
+		RpcService rpcService,
+		String taskManagerHostname,
+		HighAvailabilityServices haServices,
+		boolean localTaskManagerCommunication) throws Exception {
+
+		final TaskExecutorConfiguration taskManagerConfig = parseTaskManagerConfiguration(
+			configuration, taskManagerHostname, localTaskManagerCommunication);
+
+		TaskManagerComponents taskManagerComponents = createTaskManagerComponents(
+			resourceID,
+			InetAddress.getByName(taskManagerHostname),
+			taskManagerConfig,
+			configuration);
+
+		final TaskExecutor taskExecutor = new TaskExecutor(
+			taskManagerConfig,
+			taskManagerComponents.getTaskManagerLocation(),
+			rpcService, taskManagerComponents.getMemoryManager(),
+			taskManagerComponents.getIOManager(),
+			taskManagerComponents.getNetworkEnvironment(),
+			haServices);
+
+		taskExecutor.start();
+	}
+
+	/**
+	 * Creates and returns the task manager components.
+	 *
+	 * @param resourceID resource ID of the task manager
+	 * @param taskManagerAddress address of the task manager
+	 * @param taskExecutorConfig task manager configuration
+	 * @param configuration of Flink
+	 * @return task manager components
+	 * @throws Exception
+	 */
+	private static TaskManagerComponents createTaskManagerComponents(
+		ResourceID resourceID,
+		InetAddress taskManagerAddress,
+		TaskExecutorConfiguration taskExecutorConfig,
+		Configuration configuration) throws Exception {
+
+		MemoryType memType = taskExecutorConfig.getNetworkConfig().memoryType();
+
+		// pre-start checks
+		checkTempDirs(taskExecutorConfig.getTmpDirPaths());
+
+		NetworkEnvironmentConfiguration networkEnvironmentConfiguration = taskExecutorConfig.getNetworkConfig();
+
+		NetworkBufferPool networkBufferPool = new NetworkBufferPool(
+			networkEnvironmentConfiguration.numNetworkBuffers(),
+			networkEnvironmentConfiguration.networkBufferSize(),
+			networkEnvironmentConfiguration.memoryType());
+
+		ConnectionManager connectionManager;
+
+		if (networkEnvironmentConfiguration.nettyConfig().isDefined()) {
+			connectionManager = new NettyConnectionManager(networkEnvironmentConfiguration.nettyConfig().get());
+		} else {
+			connectionManager = new LocalConnectionManager();
+		}
+
+		ResultPartitionManager resultPartitionManager = new ResultPartitionManager();
+		TaskEventDispatcher taskEventDispatcher = new TaskEventDispatcher();
+
+		KvStateRegistry kvStateRegistry = new KvStateRegistry();
+
+		KvStateServer kvStateServer;
+
+		if (networkEnvironmentConfiguration.nettyConfig().isDefined()) {
+			NettyConfig nettyConfig = networkEnvironmentConfiguration.nettyConfig().get();
+
+			int numNetworkThreads = networkEnvironmentConfiguration.queryServerNetworkThreads() == 0 ?
+				nettyConfig.getNumberOfSlots() : networkEnvironmentConfiguration.queryServerNetworkThreads();
+
+			int numQueryThreads = networkEnvironmentConfiguration.queryServerQueryThreads() == 0 ?
+				nettyConfig.getNumberOfSlots() : networkEnvironmentConfiguration.queryServerQueryThreads();
+
+			kvStateServer = new KvStateServer(
+				taskManagerAddress,
+				networkEnvironmentConfiguration.queryServerPort(),
+				numNetworkThreads,
+				numQueryThreads,
+				kvStateRegistry,
+				new DisabledKvStateRequestStats());
+		} else {
+			kvStateServer = null;
+		}
+
+		// we start the network first, to make sure it can allocate its buffers first
+		final NetworkEnvironment network = new NetworkEnvironment(
+			networkBufferPool,
+			connectionManager,
+			resultPartitionManager,
+			taskEventDispatcher,
+			kvStateRegistry,
+			kvStateServer,
+			networkEnvironmentConfiguration.ioMode(),
+			networkEnvironmentConfiguration.partitionRequestInitialBackoff(),
+			networkEnvironmentConfiguration.partitinRequestMaxBackoff());
+
+		network.start();
+
+		final TaskManagerLocation taskManagerLocation = new TaskManagerLocation(
+			resourceID,
+			taskManagerAddress,
+			network.getConnectionManager().getDataPort());
+
+		// computing the amount of memory to use depends on how much memory is available
+		// it strictly needs to happen AFTER the network stack has been initialized
+
+		// check if a value has been configured
+		long configuredMemory = configuration.getLong(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, -1L);
+		checkConfigParameter(configuredMemory == -1 || configuredMemory > 0, configuredMemory,
+			ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY,
+			"MemoryManager needs at least one MB of memory. " +
+				"If you leave this config parameter empty, the system automatically " +
+				"pick a fraction of the available memory.");
+
+		final long memorySize;
+		boolean preAllocateMemory = configuration.getBoolean(
+			ConfigConstants.TASK_MANAGER_MEMORY_PRE_ALLOCATE_KEY,
+			ConfigConstants.DEFAULT_TASK_MANAGER_MEMORY_PRE_ALLOCATE);
+		if (configuredMemory > 0) {
+			if (preAllocateMemory) {
+				LOG.info("Using {} MB for managed memory." , configuredMemory);
+			} else {
+				LOG.info("Limiting managed memory to {} MB, memory will be allocated lazily." , configuredMemory);
+			}
+			memorySize = configuredMemory << 20; // megabytes to bytes
+		} else {
+			float fraction = configuration.getFloat(
+				ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY,
+				ConfigConstants.DEFAULT_MEMORY_MANAGER_MEMORY_FRACTION);
+			checkConfigParameter(fraction > 0.0f && fraction < 1.0f, fraction,
+				ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY,
+				"MemoryManager fraction of the free memory must be between 0.0 and 1.0");
+
+			if (memType == MemoryType.HEAP) {
+				long relativeMemSize = (long) (EnvironmentInformation.getSizeOfFreeHeapMemoryWithDefrag() * fraction);
+				if (preAllocateMemory) {
+					LOG.info("Using {} of the currently free heap space for managed heap memory ({} MB)." ,
+						fraction , relativeMemSize >> 20);
+				} else {
+					LOG.info("Limiting managed memory to {} of the currently free heap space ({} MB), " +
+						"memory will be allocated lazily." , fraction , relativeMemSize >> 20);
+				}
+				memorySize = relativeMemSize;
+			} else if (memType == MemoryType.OFF_HEAP) {
+				// The maximum heap memory has been adjusted according to the fraction
+				long maxMemory = EnvironmentInformation.getMaxJvmHeapMemory();
+				long directMemorySize = (long) (maxMemory / (1.0 - fraction) * fraction);
+				if (preAllocateMemory) {
+					LOG.info("Using {} of the maximum memory size for managed off-heap memory ({} MB)." ,
+						fraction, directMemorySize >> 20);
+				} else {
+					LOG.info("Limiting managed memory to {} of the maximum memory size ({} MB)," +
+						" memory will be allocated lazily.", fraction, directMemorySize >> 20);
+				}
+				memorySize = directMemorySize;
+			} else {
+				throw new RuntimeException("No supported memory type detected.");
+			}
+		}
+
+		// now start the memory manager
+		final MemoryManager memoryManager;
+		try {
+			memoryManager = new MemoryManager(
+				memorySize,
+				taskExecutorConfig.getNumberOfSlots(),
+				taskExecutorConfig.getNetworkConfig().networkBufferSize(),
+				memType,
+				preAllocateMemory);
+		} catch (OutOfMemoryError e) {
+			if (memType == MemoryType.HEAP) {
+				throw new Exception("OutOfMemory error (" + e.getMessage() +
+					") while allocating the TaskManager heap memory (" + memorySize + " bytes).", e);
+			} else if (memType == MemoryType.OFF_HEAP) {
+				throw new Exception("OutOfMemory error (" + e.getMessage() +
+					") while allocating the TaskManager off-heap memory (" + memorySize +
+					" bytes).Try increasing the maximum direct memory (-XX:MaxDirectMemorySize)", e);
+			} else {
+				throw e;
+			}
+		}
+
+		// start the I/O manager, it will create some temp directories.
+		final IOManager ioManager = new IOManagerAsync(taskExecutorConfig.getTmpDirPaths());
+
+		return new TaskManagerComponents(taskManagerLocation, memoryManager, ioManager, network);
+	}
+
+	// --------------------------------------------------------------------------
+	//  Parsing and checking the TaskManager Configuration
+	// --------------------------------------------------------------------------
+
+	/**
+	 * Utility method to extract TaskManager config parameters from the configuration and to
+	 * sanity check them.
+	 *
+	 * @param configuration                         The configuration.
+	 * @param taskManagerHostname           The host name under which the TaskManager communicates.
+	 * @param localTaskManagerCommunication             True, to skip initializing the network stack.
+	 *                                      Use only in cases where only one task manager runs.
+	 * @return TaskExecutorConfiguration that wrappers InstanceConnectionInfo, NetworkEnvironmentConfiguration, etc.
+	 */
+	private static TaskExecutorConfiguration parseTaskManagerConfiguration(
+		Configuration configuration,
+		String taskManagerHostname,
+		boolean localTaskManagerCommunication) throws Exception {
+
+		// ------- read values from the config and check them ---------
+		//                      (a lot of them)
+
+		// ----> hosts / ports for communication and data exchange
+
+		int dataport = configuration.getInteger(ConfigConstants.TASK_MANAGER_DATA_PORT_KEY,
+			ConfigConstants.DEFAULT_TASK_MANAGER_DATA_PORT);
+
+		checkConfigParameter(dataport > 0, dataport, ConfigConstants.TASK_MANAGER_DATA_PORT_KEY,
+			"Leave config parameter empty or use 0 to let the system choose a port automatically.");
+
+		InetAddress taskManagerAddress = InetAddress.getByName(taskManagerHostname);
+		final InetSocketAddress taskManagerInetSocketAddress = new InetSocketAddress(taskManagerAddress, dataport);
+
+		// ----> memory / network stack (shuffles/broadcasts), task slots, temp directories
+
+		// we need this because many configs have been written with a "-1" entry
+		int slots = configuration.getInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 1);
+		if (slots == -1) {
+			slots = 1;
+		}
+
+		checkConfigParameter(slots >= 1, slots, ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS,
+			"Number of task slots must be at least one.");
+
+		final int numNetworkBuffers = configuration.getInteger(
+			ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY,
+			ConfigConstants.DEFAULT_TASK_MANAGER_NETWORK_NUM_BUFFERS);
+
+		checkConfigParameter(numNetworkBuffers > 0, numNetworkBuffers,
+			ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY, "");
+
+		final int pageSize = configuration.getInteger(
+			ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY,
+			ConfigConstants.DEFAULT_TASK_MANAGER_MEMORY_SEGMENT_SIZE);
+
+		// check page size of for minimum size
+		checkConfigParameter(pageSize >= MemoryManager.MIN_PAGE_SIZE, pageSize,
+			ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY,
+			"Minimum memory segment size is " + MemoryManager.MIN_PAGE_SIZE);
+
+		// check page size for power of two
+		checkConfigParameter(MathUtils.isPowerOf2(pageSize), pageSize,
+			ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY,
+			"Memory segment size must be a power of 2.");
+
+		// check whether we use heap or off-heap memory
+		final MemoryType memType;
+		if (configuration.getBoolean(ConfigConstants.TASK_MANAGER_MEMORY_OFF_HEAP_KEY, false)) {
+			memType = MemoryType.OFF_HEAP;
+		} else {
+			memType = MemoryType.HEAP;
+		}
+
+		// initialize the memory segment factory accordingly
+		if (memType == MemoryType.HEAP) {
+			if (!MemorySegmentFactory.initializeIfNotInitialized(HeapMemorySegment.FACTORY)) {
+				throw new Exception("Memory type is set to heap memory, but memory segment " +
+					"factory has been initialized for off-heap memory segments");
+			}
+		} else {
+			if (!MemorySegmentFactory.initializeIfNotInitialized(HybridMemorySegment.FACTORY)) {
+				throw new Exception("Memory type is set to off-heap memory, but memory segment " +
+					"factory has been initialized for heap memory segments");
+			}
+		}
+
+		final String[] tmpDirs = configuration.getString(
+			ConfigConstants.TASK_MANAGER_TMP_DIR_KEY,
+			ConfigConstants.DEFAULT_TASK_MANAGER_TMP_PATH).split(",|" + File.pathSeparator);
+
+		final NettyConfig nettyConfig;
+		if (!localTaskManagerCommunication) {
+			nettyConfig = new NettyConfig(taskManagerInetSocketAddress.getAddress(),
+				taskManagerInetSocketAddress.getPort(), pageSize, slots, configuration);
+		} else {
+			nettyConfig = null;
+		}
+
+		// Default spill I/O mode for intermediate results
+		final String syncOrAsync = configuration.getString(
+			ConfigConstants.TASK_MANAGER_NETWORK_DEFAULT_IO_MODE,
+			ConfigConstants.DEFAULT_TASK_MANAGER_NETWORK_DEFAULT_IO_MODE);
+
+		final IOManager.IOMode ioMode;
+		if (syncOrAsync.equals("async")) {
+			ioMode = IOManager.IOMode.ASYNC;
+		} else {
+			ioMode = IOManager.IOMode.SYNC;
+		}
+
+		final int queryServerPort =  configuration.getInteger(
+			ConfigConstants.QUERYABLE_STATE_SERVER_PORT,
+			ConfigConstants.DEFAULT_QUERYABLE_STATE_SERVER_PORT);
+
+		final int queryServerNetworkThreads =  configuration.getInteger(
+			ConfigConstants.QUERYABLE_STATE_SERVER_NETWORK_THREADS,
+			ConfigConstants.DEFAULT_QUERYABLE_STATE_SERVER_NETWORK_THREADS);
+
+		final int queryServerQueryThreads =  configuration.getInteger(
+			ConfigConstants.QUERYABLE_STATE_SERVER_QUERY_THREADS,
+			ConfigConstants.DEFAULT_QUERYABLE_STATE_SERVER_QUERY_THREADS);
+
+		final NetworkEnvironmentConfiguration networkConfig = new NetworkEnvironmentConfiguration(
+			numNetworkBuffers,
+			pageSize,
+			memType,
+			ioMode,
+			queryServerPort,
+			queryServerNetworkThreads,
+			queryServerQueryThreads,
+			Option.apply(nettyConfig),
+			500,
+			3000);
+
+		// ----> timeouts, library caching, profiling
+
+		final FiniteDuration timeout;
+		try {
+			timeout = AkkaUtils.getTimeout(configuration);
+		} catch (Exception e) {
+			throw new IllegalArgumentException(
+				"Invalid format for '" + ConfigConstants.AKKA_ASK_TIMEOUT +
+					"'.Use formats like '50 s' or '1 min' to specify the timeout.");
+		}
+		LOG.info("Messages between TaskManager and JobManager have a max timeout of " + timeout);
+
+		final long cleanupInterval = configuration.getLong(
+			ConfigConstants.LIBRARY_CACHE_MANAGER_CLEANUP_INTERVAL,
+			ConfigConstants.DEFAULT_LIBRARY_CACHE_MANAGER_CLEANUP_INTERVAL) * 1000;
+
+		final FiniteDuration finiteRegistrationDuration;
+		try {
+			Duration maxRegistrationDuration = Duration.create(configuration.getString(
+				ConfigConstants.TASK_MANAGER_MAX_REGISTRATION_DURATION,
+				ConfigConstants.DEFAULT_TASK_MANAGER_MAX_REGISTRATION_DURATION));
+			if (maxRegistrationDuration.isFinite()) {
+				finiteRegistrationDuration = new FiniteDuration(maxRegistrationDuration.toSeconds(), TimeUnit.SECONDS);
+			} else {
+				finiteRegistrationDuration = null;
+			}
+		} catch (NumberFormatException e) {
+			throw new IllegalArgumentException("Invalid format for parameter " +
+				ConfigConstants.TASK_MANAGER_MAX_REGISTRATION_DURATION, e);
+		}
+
+		final FiniteDuration initialRegistrationPause;
+		try {
+			Duration pause = Duration.create(configuration.getString(
+				ConfigConstants.TASK_MANAGER_INITIAL_REGISTRATION_PAUSE,
+				ConfigConstants.DEFAULT_TASK_MANAGER_INITIAL_REGISTRATION_PAUSE));
+			if (pause.isFinite()) {
+				initialRegistrationPause = new FiniteDuration(pause.toSeconds(), TimeUnit.SECONDS);
+			} else {
+				throw new IllegalArgumentException("The initial registration pause must be finite: " + pause);
+			}
+		} catch (NumberFormatException e) {
+			throw new IllegalArgumentException("Invalid format for parameter " +
+				ConfigConstants.TASK_MANAGER_INITIAL_REGISTRATION_PAUSE, e);
+		}
+
+		final FiniteDuration maxRegistrationPause;
+		try {
+			Duration pause = Duration.create(configuration.getString(
+				ConfigConstants.TASK_MANAGER_MAX_REGISTARTION_PAUSE,
+				ConfigConstants.DEFAULT_TASK_MANAGER_MAX_REGISTRATION_PAUSE));
+			if (pause.isFinite()) {
+				maxRegistrationPause = new FiniteDuration(pause.toSeconds(), TimeUnit.SECONDS);
+			} else {
+				throw new IllegalArgumentException("The maximum registration pause must be finite: " + pause);
+			}
+		} catch (NumberFormatException e) {
+			throw new IllegalArgumentException("Invalid format for parameter " +
+				ConfigConstants.TASK_MANAGER_INITIAL_REGISTRATION_PAUSE, e);
+		}
+
+		final FiniteDuration refusedRegistrationPause;
+		try {
+			Duration pause = Duration.create(configuration.getString(
+				ConfigConstants.TASK_MANAGER_REFUSED_REGISTRATION_PAUSE,
+				ConfigConstants.DEFAULT_TASK_MANAGER_REFUSED_REGISTRATION_PAUSE));
+			if (pause.isFinite()) {
+				refusedRegistrationPause = new FiniteDuration(pause.toSeconds(), TimeUnit.SECONDS);
+			} else {
+				throw new IllegalArgumentException("The refused registration pause must be finite: " + pause);
+			}
+		} catch (NumberFormatException e) {
+			throw new IllegalArgumentException("Invalid format for parameter " +
+				ConfigConstants.TASK_MANAGER_INITIAL_REGISTRATION_PAUSE, e);
+		}
+
+		return new TaskExecutorConfiguration(
+			tmpDirs,
+			cleanupInterval,
+			networkConfig,
+			timeout,
+			finiteRegistrationDuration,
+			slots,
+			configuration,
+			initialRegistrationPause,
+			maxRegistrationPause,
+			refusedRegistrationPause);
+	}
+
+	/**
+	 * Validates a condition for a config parameter and displays a standard exception, if the
+	 * the condition does not hold.
+	 *
+	 * @param condition             The condition that must hold. If the condition is false, an exception is thrown.
+	 * @param parameter         The parameter value. Will be shown in the exception message.
+	 * @param name              The name of the config parameter. Will be shown in the exception message.
+	 * @param errorMessage  The optional custom error message to append to the exception message.
+	 */
+	private static void checkConfigParameter(
+		boolean condition,
+		Object parameter,
+		String name,
+		String errorMessage) {
+		if (!condition) {
+			throw new IllegalConfigurationException("Invalid configuration value for " + name + " : " + parameter + " - " + errorMessage);
+		}
+	}
+
+	/**
+	 * Validates that all the directories denoted by the strings do actually exist, are proper
+	 * directories (not files), and are writable.
+	 *
+	 * @param tmpDirs       The array of directory paths to check.
+	 * @throws Exception    Thrown if any of the directories does not exist or is not writable
+	 *                   or is a file, rather than a directory.
+	 */
+	private static void checkTempDirs(String[] tmpDirs) throws IOException {
+		for (String dir : tmpDirs) {
+			if (dir != null && !dir.equals("")) {
+				File file = new File(dir);
+				if (!file.exists()) {
+					throw new IOException("Temporary file directory " + file.getAbsolutePath() + " does not exist.");
+				}
+				if (!file.isDirectory()) {
+					throw new IOException("Temporary file directory " + file.getAbsolutePath() + " is not a directory.");
+				}
+				if (!file.canWrite()) {
+					throw new IOException("Temporary file directory " + file.getAbsolutePath() + " is not writable.");
+				}
+
+				if (LOG.isInfoEnabled()) {
+					long totalSpaceGb = file.getTotalSpace() >> 30;
+					long usableSpaceGb = file.getUsableSpace() >> 30;
+					double usablePercentage = (double)usableSpaceGb / totalSpaceGb * 100;
+					String path = file.getAbsolutePath();
+					LOG.info(String.format("Temporary file directory '%s': total %d GB, " + "usable %d GB (%.2f%% usable)",
+						path, totalSpaceGb, usableSpaceGb, usablePercentage));
+				}
+			} else {
+				throw new IllegalArgumentException("Temporary file directory #$id is null.");
+			}
+		}
+	}
+
+	private static class TaskManagerComponents {
+		private final TaskManagerLocation taskManagerLocation;
+		private final MemoryManager memoryManager;
+		private final IOManager ioManager;
+		private final NetworkEnvironment networkEnvironment;
+
+		private TaskManagerComponents(
+			TaskManagerLocation taskManagerLocation,
+			MemoryManager memoryManager,
+			IOManager ioManager,
+			NetworkEnvironment networkEnvironment) {
+
+			this.taskManagerLocation = Preconditions.checkNotNull(taskManagerLocation);
+			this.memoryManager = Preconditions.checkNotNull(memoryManager);
+			this.ioManager = Preconditions.checkNotNull(ioManager);
+			this.networkEnvironment = Preconditions.checkNotNull(networkEnvironment);
+		}
+
+		public MemoryManager getMemoryManager() {
+			return memoryManager;
+		}
+
+		public IOManager getIOManager() {
+			return ioManager;
+		}
+
+		public NetworkEnvironment getNetworkEnvironment() {
+			return networkEnvironment;
+		}
+
+		public TaskManagerLocation getTaskManagerLocation() {
+			return taskManagerLocation;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/c34f13c2/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 09aab18..26218dd 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
@@ -19,17 +19,22 @@
 package org.apache.flink.runtime.taskexecutor;
 
 import org.apache.flink.api.common.time.Time;
-import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.clusterframework.types.ResourceID;
 import org.apache.flink.runtime.highavailability.NonHaServices;
 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.leaderelection.TestingLeaderRetrievalService;
+import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.runtime.rpc.TestingRpcService;
 import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway;
+import org.apache.flink.runtime.rpc.TestingSerialRpcService;
+import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
 import org.apache.flink.util.TestLogger;
 
 import org.junit.Test;
 
+import org.powermock.api.mockito.PowerMockito;
 import java.util.UUID;
 
 import static org.junit.Assert.*;
@@ -42,19 +47,31 @@ public class TaskExecutorTest extends TestLogger {
 		final ResourceID resourceID = ResourceID.generate();
 		final String resourceManagerAddress = "/resource/manager/address/one";
 
-		final TestingRpcService rpc = new TestingRpcService();
+		final TestingSerialRpcService rpc = new TestingSerialRpcService();
 		try {
 			// register a mock resource manager gateway
 			ResourceManagerGateway rmGateway = mock(ResourceManagerGateway.class);
+			TaskExecutorConfiguration taskExecutorConfiguration = mock(TaskExecutorConfiguration.class);
+			PowerMockito.when(taskExecutorConfiguration.getNumberOfSlots()).thenReturn(1);
 			rpc.registerGateway(resourceManagerAddress, rmGateway);
 
+			TaskManagerLocation taskManagerLocation = mock(TaskManagerLocation.class);
+			when(taskManagerLocation.getResourceID()).thenReturn(resourceID);
+
 			NonHaServices haServices = new NonHaServices(resourceManagerAddress);
-			TaskExecutor taskManager = TaskExecutor.startTaskManagerComponentsAndActor(
-				new Configuration(), resourceID, rpc, "localhost", haServices, true);
-			String taskManagerAddress = taskManager.getAddress();
+
+			TaskExecutor taskManager = new TaskExecutor(
+				taskExecutorConfiguration,
+				taskManagerLocation,
+				rpc, mock(MemoryManager.class),
+				mock(IOManager.class),
+				mock(NetworkEnvironment.class),
+				haServices);
+
 			taskManager.start();
+			String taskManagerAddress = taskManager.getAddress();
 
-			verify(rmGateway, timeout(5000)).registerTaskExecutor(
+			verify(rmGateway).registerTaskExecutor(
 					any(UUID.class), eq(taskManagerAddress), eq(resourceID), any(Time.class));
 		}
 		finally {
@@ -71,7 +88,7 @@ public class TaskExecutorTest extends TestLogger {
 		final UUID leaderId1 = UUID.randomUUID();
 		final UUID leaderId2 = UUID.randomUUID();
 
-		final TestingRpcService rpc = new TestingRpcService();
+		final TestingSerialRpcService rpc = new TestingSerialRpcService();
 		try {
 			// register the mock resource manager gateways
 			ResourceManagerGateway rmGateway1 = mock(ResourceManagerGateway.class);
@@ -84,10 +101,22 @@ public class TaskExecutorTest extends TestLogger {
 			TestingHighAvailabilityServices haServices = new TestingHighAvailabilityServices();
 			haServices.setResourceManagerLeaderRetriever(testLeaderService);
 
-			TaskExecutor taskManager = TaskExecutor.startTaskManagerComponentsAndActor(
-				new Configuration(), resourceID, rpc, "localhost", haServices, true);
-			String taskManagerAddress = taskManager.getAddress();
+			TaskExecutorConfiguration taskExecutorConfiguration = mock(TaskExecutorConfiguration.class);
+			PowerMockito.when(taskExecutorConfiguration.getNumberOfSlots()).thenReturn(1);
+
+			TaskManagerLocation taskManagerLocation = mock(TaskManagerLocation.class);
+			when(taskManagerLocation.getResourceID()).thenReturn(resourceID);
+
+			TaskExecutor taskManager = new TaskExecutor(
+				taskExecutorConfiguration,
+				taskManagerLocation,
+				rpc, mock(MemoryManager.class),
+				mock(IOManager.class),
+				mock(NetworkEnvironment.class),
+				haServices);
+
 			taskManager.start();
+			String taskManagerAddress = taskManager.getAddress();
 
 			// no connection initially, since there is no leader
 			assertNull(taskManager.getResourceManagerConnection());
@@ -95,7 +124,7 @@ public class TaskExecutorTest extends TestLogger {
 			// define a leader and see that a registration happens
 			testLeaderService.notifyListener(address1, leaderId1);
 
-			verify(rmGateway1, timeout(5000)).registerTaskExecutor(
+			verify(rmGateway1).registerTaskExecutor(
 					eq(leaderId1), eq(taskManagerAddress), eq(resourceID), any(Time.class));
 			assertNotNull(taskManager.getResourceManagerConnection());
 
@@ -105,7 +134,7 @@ public class TaskExecutorTest extends TestLogger {
 			// set a new leader, see that a registration happens 
 			testLeaderService.notifyListener(address2, leaderId2);
 
-			verify(rmGateway2, timeout(5000)).registerTaskExecutor(
+			verify(rmGateway2).registerTaskExecutor(
 					eq(leaderId2), eq(taskManagerAddress), eq(resourceID), any(Time.class));
 			assertNotNull(taskManager.getResourceManagerConnection());
 		}


[06/50] [abbrv] flink git commit: [FLINK-4529] [flip-6] Move TaskExecutor, JobMaster and ResourceManager out of the rpc package

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/9c247d1f/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotRequest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotRequest.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotRequest.java
new file mode 100644
index 0000000..896421b
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotRequest.java
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+
+import java.io.Serializable;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * This describes the requirement of the slot, mainly used by JobManager requesting slot from ResourceManager.
+ */
+public class SlotRequest implements Serializable {
+
+	private static final long serialVersionUID = -6586877187990445986L;
+
+	/** The JobID of the slot requested for */
+	private final JobID jobId;
+
+	/** The unique identification of this request */
+	private final AllocationID allocationId;
+
+	/** The resource profile of the required slot */
+	private final ResourceProfile resourceProfile;
+
+	public SlotRequest(JobID jobId, AllocationID allocationId, ResourceProfile resourceProfile) {
+		this.jobId = checkNotNull(jobId);
+		this.allocationId = checkNotNull(allocationId);
+		this.resourceProfile = checkNotNull(resourceProfile);
+	}
+
+	/**
+	 * Get the JobID of the slot requested for.
+	 * @return The job id
+	 */
+	public JobID getJobId() {
+		return jobId;
+	}
+
+	/**
+	 * Get the unique identification of this request
+	 * @return the allocation id
+	 */
+	public AllocationID getAllocationId() {
+		return allocationId;
+	}
+
+	/**
+	 * Get the resource profile of the desired slot
+	 * @return The resource profile
+	 */
+	public ResourceProfile getResourceProfile() {
+		return resourceProfile;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9c247d1f/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/jobmaster/JobMaster.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/jobmaster/JobMaster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/jobmaster/JobMaster.java
deleted file mode 100644
index a046cb8..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/jobmaster/JobMaster.java
+++ /dev/null
@@ -1,244 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.runtime.rpc.jobmaster;
-
-import org.apache.flink.api.common.JobID;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
-import org.apache.flink.runtime.jobgraph.JobGraph;
-import org.apache.flink.runtime.leaderelection.LeaderContender;
-import org.apache.flink.runtime.leaderelection.LeaderElectionService;
-import org.apache.flink.runtime.messages.Acknowledge;
-import org.apache.flink.runtime.rpc.RpcMethod;
-import org.apache.flink.runtime.rpc.resourcemanager.ResourceManagerGateway;
-import org.apache.flink.runtime.rpc.RpcEndpoint;
-import org.apache.flink.runtime.rpc.RpcService;
-import org.apache.flink.runtime.taskmanager.TaskExecutionState;
-import org.apache.flink.util.Preconditions;
-
-import java.util.UUID;
-
-/**
- * JobMaster implementation. The job master is responsible for the execution of a single
- * {@link org.apache.flink.runtime.jobgraph.JobGraph}.
- * <p>
- * 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
- * given task</li>
- * </ul>
- */
-public class JobMaster extends RpcEndpoint<JobMasterGateway> {
-
-	/** Gateway to connected resource manager, null iff not connected */
-	private ResourceManagerGateway resourceManager = null;
-
-	/** Logical representation of the job */
-	private final JobGraph jobGraph;
-	private final JobID jobID;
-
-	/** Configuration of the job */
-	private final Configuration configuration;
-
-	/** Service to contend for and retrieve the leadership of JM and RM */
-	private final HighAvailabilityServices highAvailabilityServices;
-
-	/** Leader Management */
-	private LeaderElectionService leaderElectionService = null;
-	private UUID leaderSessionID;
-
-	/**
-	 * The JM's Constructor
-	 *
-	 * @param jobGraph The representation of the job's execution plan
-	 * @param configuration The job's configuration
-	 * @param rpcService The RPC service at which the JM serves
-	 * @param highAvailabilityService The cluster's HA service from the JM can elect and retrieve leaders.
-	 */
-	public JobMaster(
-		JobGraph jobGraph,
-		Configuration configuration,
-		RpcService rpcService,
-		HighAvailabilityServices highAvailabilityService) {
-
-		super(rpcService);
-
-		this.jobGraph = Preconditions.checkNotNull(jobGraph);
-		this.jobID = Preconditions.checkNotNull(jobGraph.getJobID());
-
-		this.configuration = Preconditions.checkNotNull(configuration);
-
-		this.highAvailabilityServices = Preconditions.checkNotNull(highAvailabilityService);
-	}
-
-	public ResourceManagerGateway getResourceManager() {
-		return resourceManager;
-	}
-
-	//----------------------------------------------------------------------------------------------
-	// Initialization methods
-	//----------------------------------------------------------------------------------------------
-	public void start() {
-		super.start();
-
-		// register at the election once the JM starts
-		registerAtElectionService();
-	}
-
-
-	//----------------------------------------------------------------------------------------------
-	// JobMaster Leadership methods
-	//----------------------------------------------------------------------------------------------
-
-	/**
-	 * Retrieves the election service and contend for the leadership.
-	 */
-	private void registerAtElectionService() {
-		try {
-			leaderElectionService = highAvailabilityServices.getJobMasterLeaderElectionService(jobID);
-			leaderElectionService.start(new JobMasterLeaderContender());
-		} catch (Exception e) {
-			throw new RuntimeException("Fail to register at the election of JobMaster", e);
-		}
-	}
-
-	/**
-	 * Start the execution when the leadership is granted.
-	 *
-	 * @param newLeaderSessionID The identifier of the new leadership session
-	 */
-	public void grantJobMasterLeadership(final UUID newLeaderSessionID) {
-		runAsync(new Runnable() {
-			@Override
-			public void run() {
-				log.info("JobManager {} grants leadership with session id {}.", getAddress(), newLeaderSessionID);
-
-				// The operation may be blocking, but since JM is idle before it grants the leadership, it's okay that
-				// JM waits here for the operation's completeness.
-				leaderSessionID = newLeaderSessionID;
-				leaderElectionService.confirmLeaderSessionID(newLeaderSessionID);
-
-				// TODO:: execute the job when the leadership is granted.
-			}
-		});
-	}
-
-	/**
-	 * Stop the execution when the leadership is revoked.
-	 */
-	public void revokeJobMasterLeadership() {
-		runAsync(new Runnable() {
-			@Override
-			public void run() {
-				log.info("JobManager {} was revoked leadership.", getAddress());
-
-				// TODO:: cancel the job's execution and notify all listeners
-				cancelAndClearEverything(new Exception("JobManager is no longer the leader."));
-
-				leaderSessionID = null;
-			}
-		});
-	}
-
-	/**
-	 * Handles error occurring in the leader election service
-	 *
-	 * @param exception Exception thrown in the leader election service
-	 */
-	public void onJobMasterElectionError(final Exception exception) {
-		runAsync(new Runnable() {
-			@Override
-			public void run() {
-				log.error("Received an error from the LeaderElectionService.", exception);
-
-				// TODO:: cancel the job's execution and shutdown the JM
-				cancelAndClearEverything(exception);
-
-				leaderSessionID = null;
-			}
-		});
-
-	}
-
-	//----------------------------------------------------------------------------------------------
-	// RPC methods
-	//----------------------------------------------------------------------------------------------
-
-	/**
-	 * Updates the task execution state for a given task.
-	 *
-	 * @param taskExecutionState New task execution state for a given task
-	 * @return Acknowledge the task execution state update
-	 */
-	@RpcMethod
-	public Acknowledge updateTaskExecutionState(TaskExecutionState taskExecutionState) {
-		System.out.println("TaskExecutionState: " + taskExecutionState);
-		return Acknowledge.get();
-	}
-
-	/**
-	 * Triggers the registration of the job master at the resource manager.
-	 *
-	 * @param address Address of the resource manager
-	 */
-	@RpcMethod
-	public void registerAtResourceManager(final String address) {
-		//TODO:: register at the RM
-	}
-
-	//----------------------------------------------------------------------------------------------
-	// Helper methods
-	//----------------------------------------------------------------------------------------------
-
-	/**
-	 * Cancel the current job and notify all listeners the job's cancellation.
-	 *
-	 * @param cause Cause for the cancelling.
-	 */
-	private void cancelAndClearEverything(Throwable cause) {
-		// currently, nothing to do here
-	}
-
-	// ------------------------------------------------------------------------
-	//  Utility classes
-	// ------------------------------------------------------------------------
-	private class JobMasterLeaderContender implements LeaderContender {
-
-		@Override
-		public void grantLeadership(UUID leaderSessionID) {
-			JobMaster.this.grantJobMasterLeadership(leaderSessionID);
-		}
-
-		@Override
-		public void revokeLeadership() {
-			JobMaster.this.revokeJobMasterLeadership();
-		}
-
-		@Override
-		public String getAddress() {
-			return JobMaster.this.getAddress();
-		}
-
-		@Override
-		public void handleError(Exception exception) {
-			onJobMasterElectionError(exception);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9c247d1f/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/jobmaster/JobMasterGateway.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/jobmaster/JobMasterGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/jobmaster/JobMasterGateway.java
deleted file mode 100644
index 17a4c3a..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/jobmaster/JobMasterGateway.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.runtime.rpc.jobmaster;
-
-import org.apache.flink.runtime.messages.Acknowledge;
-import org.apache.flink.runtime.rpc.RpcGateway;
-import org.apache.flink.runtime.taskmanager.TaskExecutionState;
-import scala.concurrent.Future;
-
-/**
- * {@link JobMaster} rpc gateway interface
- */
-public interface JobMasterGateway extends RpcGateway {
-
-	/**
-	 * Updates the task execution state for a given task.
-	 *
-	 * @param taskExecutionState New task execution state for a given task
-	 * @return Future acknowledge of the task execution state update
-	 */
-	Future<Acknowledge> updateTaskExecutionState(TaskExecutionState taskExecutionState);
-
-	/**
-	 * Triggers the registration of the job master at the resource manager.
-	 *
-	 * @param address Address of the resource manager
-	 */
-	void registerAtResourceManager(final String address);
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9c247d1f/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/registration/RegistrationResponse.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/registration/RegistrationResponse.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/registration/RegistrationResponse.java
deleted file mode 100644
index 2de560a..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/registration/RegistrationResponse.java
+++ /dev/null
@@ -1,84 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.runtime.rpc.registration;
-
-import java.io.Serializable;
-
-/**
- * Base class for responses given to registration attempts from {@link RetryingRegistration}.
- */
-public abstract class RegistrationResponse implements Serializable {
-
-	private static final long serialVersionUID = 1L;
-
-	// ----------------------------------------------------------------------------
-	
-	/**
-	 * Base class for a successful registration. Concrete registration implementations
-	 * will typically extend this class to attach more information.
-	 */
-	public static class Success extends RegistrationResponse {
-		private static final long serialVersionUID = 1L;
-		
-		@Override
-		public String toString() {
-			return "Registration Successful";
-		}
-	}
-
-	// ----------------------------------------------------------------------------
-
-	/**
-	 * A rejected (declined) registration.
-	 */
-	public static final class Decline extends RegistrationResponse {
-		private static final long serialVersionUID = 1L;
-
-		/** the rejection reason */
-		private final String reason;
-
-		/**
-		 * Creates a new rejection message.
-		 * 
-		 * @param reason The reason for the rejection.
-		 */
-		public Decline(String reason) {
-			this.reason = reason != null ? reason : "(unknown)";
-		}
-
-		/**
-		 * Gets the reason for the rejection.
-		 */
-		public String getReason() {
-			return reason;
-		}
-
-		@Override
-		public String toString() {
-			return "Registration Declined (" + reason + ')';
-		}
-	}
-}
-
-
-
-
-
-
-

http://git-wip-us.apache.org/repos/asf/flink/blob/9c247d1f/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/registration/RetryingRegistration.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/registration/RetryingRegistration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/registration/RetryingRegistration.java
deleted file mode 100644
index dcb5011..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/registration/RetryingRegistration.java
+++ /dev/null
@@ -1,296 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.runtime.rpc.registration;
-
-import akka.dispatch.OnFailure;
-import akka.dispatch.OnSuccess;
-
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.runtime.rpc.RpcGateway;
-import org.apache.flink.runtime.rpc.RpcService;
-
-import org.slf4j.Logger;
-
-import scala.concurrent.Future;
-import scala.concurrent.Promise;
-import scala.concurrent.impl.Promise.DefaultPromise;
-
-import java.util.UUID;
-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;
-
-
-/**
- * This utility class implements the basis of registering one component at another component,
- * for example registering the TaskExecutor at the ResourceManager.
- * This {@code RetryingRegistration} implements both the initial address resolution
- * and the retries-with-backoff strategy.
- * 
- * <p>The registration gives access to a future that is completed upon successful registration.
- * The registration can be canceled, for example when the target where it tries to register
- * at looses leader status.
- * 
- * @param <Gateway> The type of the gateway to connect to.
- * @param <Success> The type of the successful registration responses.
- */
-public abstract class RetryingRegistration<Gateway extends RpcGateway, Success extends RegistrationResponse.Success> {
-
-	// ------------------------------------------------------------------------
-	//  default configuration values
-	// ------------------------------------------------------------------------
-
-	/** default value for the initial registration timeout (milliseconds) */
-	private static final long INITIAL_REGISTRATION_TIMEOUT_MILLIS = 100;
-
-	/** default value for the maximum registration timeout, after exponential back-off (milliseconds) */
-	private static final long MAX_REGISTRATION_TIMEOUT_MILLIS = 30000;
-
-	/** The pause (milliseconds) made after an registration attempt caused an exception (other than timeout) */
-	private static final long ERROR_REGISTRATION_DELAY_MILLIS = 10000;
-
-	/** The pause (milliseconds) made after the registration attempt was refused */
-	private static final long REFUSED_REGISTRATION_DELAY_MILLIS = 30000;
-
-	// ------------------------------------------------------------------------
-	// Fields
-	// ------------------------------------------------------------------------
-
-	private final Logger log;
-
-	private final RpcService rpcService;
-
-	private final String targetName;
-
-	private final Class<Gateway> targetType;
-
-	private final String targetAddress;
-
-	private final UUID leaderId;
-
-	private final Promise<Tuple2<Gateway, Success>> completionPromise;
-
-	private final long initialRegistrationTimeout;
-
-	private final long maxRegistrationTimeout;
-
-	private final long delayOnError;
-
-	private final long delayOnRefusedRegistration;
-
-	private volatile boolean canceled;
-
-	// ------------------------------------------------------------------------
-
-	public RetryingRegistration(
-			Logger log,
-			RpcService rpcService,
-			String targetName,
-			Class<Gateway> targetType,
-			String targetAddress,
-			UUID leaderId) {
-		this(log, rpcService, targetName, targetType, targetAddress, leaderId,
-				INITIAL_REGISTRATION_TIMEOUT_MILLIS, MAX_REGISTRATION_TIMEOUT_MILLIS,
-				ERROR_REGISTRATION_DELAY_MILLIS, REFUSED_REGISTRATION_DELAY_MILLIS);
-	}
-
-	public RetryingRegistration(
-			Logger log,
-			RpcService rpcService,
-			String targetName, 
-			Class<Gateway> targetType,
-			String targetAddress,
-			UUID leaderId,
-			long initialRegistrationTimeout,
-			long maxRegistrationTimeout,
-			long delayOnError,
-			long delayOnRefusedRegistration) {
-
-		checkArgument(initialRegistrationTimeout > 0, "initial registration timeout must be greater than zero");
-		checkArgument(maxRegistrationTimeout > 0, "maximum registration timeout must be greater than zero");
-		checkArgument(delayOnError >= 0, "delay on error must be non-negative");
-		checkArgument(delayOnRefusedRegistration >= 0, "delay on refused registration must be non-negative");
-
-		this.log = checkNotNull(log);
-		this.rpcService = checkNotNull(rpcService);
-		this.targetName = checkNotNull(targetName);
-		this.targetType = checkNotNull(targetType);
-		this.targetAddress = checkNotNull(targetAddress);
-		this.leaderId = checkNotNull(leaderId);
-		this.initialRegistrationTimeout = initialRegistrationTimeout;
-		this.maxRegistrationTimeout = maxRegistrationTimeout;
-		this.delayOnError = delayOnError;
-		this.delayOnRefusedRegistration = delayOnRefusedRegistration;
-
-		this.completionPromise = new DefaultPromise<>();
-	}
-
-	// ------------------------------------------------------------------------
-	//  completion and cancellation
-	// ------------------------------------------------------------------------
-
-	public Future<Tuple2<Gateway, Success>> getFuture() {
-		return completionPromise.future();
-	}
-
-	/**
-	 * Cancels the registration procedure.
-	 */
-	public void cancel() {
-		canceled = true;
-	}
-
-	/**
-	 * Checks if the registration was canceled.
-	 * @return True if the registration was canceled, false otherwise.
-	 */
-	public boolean isCanceled() {
-		return canceled;
-	}
-
-	// ------------------------------------------------------------------------
-	//  registration
-	// ------------------------------------------------------------------------
-
-	protected abstract Future<RegistrationResponse> invokeRegistration(
-			Gateway gateway, UUID leaderId, long timeoutMillis) throws Exception;
-
-	/**
-	 * This method resolves the target address to a callable gateway and starts the
-	 * registration after that.
-	 */
-	@SuppressWarnings("unchecked")
-	public void startRegistration() {
-		try {
-			// trigger resolution of the resource manager address to a callable gateway
-			Future<Gateway> resourceManagerFuture = rpcService.connect(targetAddress, targetType);
-	
-			// upon success, start the registration attempts
-			resourceManagerFuture.onSuccess(new OnSuccess<Gateway>() {
-				@Override
-				public void onSuccess(Gateway result) {
-					log.info("Resolved {} address, beginning registration", targetName);
-					register(result, 1, initialRegistrationTimeout);
-				}
-			}, rpcService.getExecutionContext());
-	
-			// upon failure, retry, unless this is cancelled
-			resourceManagerFuture.onFailure(new OnFailure() {
-				@Override
-				public void onFailure(Throwable failure) {
-					if (!isCanceled()) {
-						log.warn("Could not resolve {} address {}, retrying...", targetName, targetAddress);
-						startRegistration();
-					}
-				}
-			}, rpcService.getExecutionContext());
-		}
-		catch (Throwable t) {
-			cancel();
-			completionPromise.tryFailure(t);
-		}
-	}
-
-	/**
-	 * This method performs a registration attempt and triggers either a success notification or a retry,
-	 * depending on the result.
-	 */
-	@SuppressWarnings("unchecked")
-	private void register(final Gateway gateway, final int attempt, final long timeoutMillis) {
-		// eager check for canceling to avoid some unnecessary work
-		if (canceled) {
-			return;
-		}
-
-		try {
-			log.info("Registration at {} attempt {} (timeout={}ms)", targetName, attempt, timeoutMillis);
-			Future<RegistrationResponse> registrationFuture = invokeRegistration(gateway, leaderId, timeoutMillis);
-	
-			// if the registration was successful, let the TaskExecutor know
-			registrationFuture.onSuccess(new OnSuccess<RegistrationResponse>() {
-				
-				@Override
-				public void onSuccess(RegistrationResponse result) throws Throwable {
-					if (!isCanceled()) {
-						if (result instanceof RegistrationResponse.Success) {
-							// registration successful!
-							Success success = (Success) result;
-							completionPromise.success(new Tuple2<>(gateway, success));
-						}
-						else {
-							// registration refused or unknown
-							if (result instanceof RegistrationResponse.Decline) {
-								RegistrationResponse.Decline decline = (RegistrationResponse.Decline) result;
-								log.info("Registration at {} was declined: {}", targetName, decline.getReason());
-							} else {
-								log.error("Received unknown response to registration attempt: " + result);
-							}
-
-							log.info("Pausing and re-attempting registration in {} ms", delayOnRefusedRegistration);
-							registerLater(gateway, 1, initialRegistrationTimeout, delayOnRefusedRegistration);
-						}
-					}
-				}
-			}, rpcService.getExecutionContext());
-	
-			// upon failure, retry
-			registrationFuture.onFailure(new OnFailure() {
-				@Override
-				public void onFailure(Throwable failure) {
-					if (!isCanceled()) {
-						if (failure instanceof TimeoutException) {
-							// we simply have not received a response in time. maybe the timeout was
-							// very low (initial fast registration attempts), maybe the target endpoint is
-							// currently down.
-							if (log.isDebugEnabled()) {
-								log.debug("Registration at {} ({}) attempt {} timed out after {} ms",
-										targetName, targetAddress, attempt, timeoutMillis);
-							}
-	
-							long newTimeoutMillis = Math.min(2 * timeoutMillis, maxRegistrationTimeout);
-							register(gateway, attempt + 1, newTimeoutMillis);
-						}
-						else {
-							// a serious failure occurred. we still should not give up, but keep trying
-							log.error("Registration at " + targetName + " failed due to an error", failure);
-							log.info("Pausing and re-attempting registration in {} ms", delayOnError);
-	
-							registerLater(gateway, 1, initialRegistrationTimeout, delayOnError);
-						}
-					}
-				}
-			}, rpcService.getExecutionContext());
-		}
-		catch (Throwable t) {
-			cancel();
-			completionPromise.tryFailure(t);
-		}
-	}
-
-	private void registerLater(final Gateway gateway, final int attempt, final long timeoutMillis, long delay) {
-		rpcService.scheduleRunnable(new Runnable() {
-			@Override
-			public void run() {
-				register(gateway, attempt, timeoutMillis);
-			}
-		}, delay, TimeUnit.MILLISECONDS);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9c247d1f/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/JobMasterRegistration.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/JobMasterRegistration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/JobMasterRegistration.java
deleted file mode 100644
index 7a2deae..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/JobMasterRegistration.java
+++ /dev/null
@@ -1,35 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.runtime.rpc.resourcemanager;
-
-import java.io.Serializable;
-
-public class JobMasterRegistration implements Serializable {
-	private static final long serialVersionUID = 8411214999193765202L;
-
-	private final String address;
-
-	public JobMasterRegistration(String address) {
-		this.address = address;
-	}
-
-	public String getAddress() {
-		return address;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9c247d1f/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/RegistrationResponse.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/RegistrationResponse.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/RegistrationResponse.java
deleted file mode 100644
index 8ac9e49..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/RegistrationResponse.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.runtime.rpc.resourcemanager;
-
-import org.apache.flink.runtime.instance.InstanceID;
-
-import java.io.Serializable;
-
-public class RegistrationResponse implements Serializable {
-	private static final long serialVersionUID = -2379003255993119993L;
-
-	private final boolean isSuccess;
-	private final InstanceID instanceID;
-
-	public RegistrationResponse(boolean isSuccess, InstanceID instanceID) {
-		this.isSuccess = isSuccess;
-		this.instanceID = instanceID;
-	}
-
-	public boolean isSuccess() {
-		return isSuccess;
-	}
-
-	public InstanceID getInstanceID() {
-		return instanceID;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9c247d1f/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManager.java
deleted file mode 100644
index f7147c9..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManager.java
+++ /dev/null
@@ -1,214 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.runtime.rpc.resourcemanager;
-
-import akka.dispatch.Mapper;
-
-import org.apache.flink.annotation.VisibleForTesting;
-import org.apache.flink.runtime.clusterframework.types.ResourceID;
-import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
-import org.apache.flink.runtime.instance.InstanceID;
-import org.apache.flink.runtime.leaderelection.LeaderContender;
-import org.apache.flink.runtime.leaderelection.LeaderElectionService;
-import org.apache.flink.runtime.rpc.RpcMethod;
-import org.apache.flink.runtime.rpc.RpcEndpoint;
-import org.apache.flink.runtime.rpc.RpcService;
-import org.apache.flink.runtime.rpc.jobmaster.JobMaster;
-import org.apache.flink.runtime.rpc.jobmaster.JobMasterGateway;
-import org.apache.flink.runtime.rpc.taskexecutor.TaskExecutorRegistrationSuccess;
-
-import scala.concurrent.Future;
-
-import java.util.HashMap;
-import java.util.Map;
-import java.util.UUID;
-
-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:
- * <ul>
- *     <li>{@link #registerJobMaster(JobMasterRegistration)} registers a {@link JobMaster} at the resource manager</li>
- *     <li>{@link #requestSlot(SlotRequest)} requests a slot from the resource manager</li>
- * </ul>
- */
-public class ResourceManager extends RpcEndpoint<ResourceManagerGateway> {
-	private final Map<JobMasterGateway, InstanceID> jobMasterGateways;
-	private final HighAvailabilityServices highAvailabilityServices;
-	private LeaderElectionService leaderElectionService = null;
-	private UUID leaderSessionID = null;
-
-	public ResourceManager(RpcService rpcService, HighAvailabilityServices highAvailabilityServices) {
-		super(rpcService);
-		this.highAvailabilityServices = checkNotNull(highAvailabilityServices);
-		this.jobMasterGateways = new HashMap<>();
-	}
-
-	@Override
-	public void start() {
-		// start a leader
-		try {
-			super.start();
-			leaderElectionService = highAvailabilityServices.getResourceManagerLeaderElectionService();
-			leaderElectionService.start(new ResourceManagerLeaderContender());
-		} 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);
-		}
-	}
-
-	@Override
-	public void shutDown() {
-		try {
-			leaderElectionService.stop();
-			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);
-		}
-	}
-
-	/**
-	 * 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;
-	}
-
-	/**
-	 * Register a {@link JobMaster} at the resource manager.
-	 *
-	 * @param jobMasterRegistration Job master registration information
-	 * @return Future registration response
-	 */
-	@RpcMethod
-	public Future<RegistrationResponse> registerJobMaster(JobMasterRegistration jobMasterRegistration) {
-		Future<JobMasterGateway> jobMasterFuture = getRpcService().connect(jobMasterRegistration.getAddress(), JobMasterGateway.class);
-
-		return jobMasterFuture.map(new Mapper<JobMasterGateway, RegistrationResponse>() {
-			@Override
-			public RegistrationResponse apply(final JobMasterGateway jobMasterGateway) {
-				InstanceID instanceID;
-
-				if (jobMasterGateways.containsKey(jobMasterGateway)) {
-					instanceID = jobMasterGateways.get(jobMasterGateway);
-				} else {
-					instanceID = new InstanceID();
-					jobMasterGateways.put(jobMasterGateway, instanceID);
-				}
-
-				return new RegistrationResponse(true, instanceID);
-			}
-		}, getMainThreadExecutionContext());
-	}
-
-	/**
-	 * Requests a slot from the resource manager.
-	 *
-	 * @param slotRequest Slot request
-	 * @return Slot assignment
-	 */
-	@RpcMethod
-	public SlotAssignment requestSlot(SlotRequest slotRequest) {
-		System.out.println("SlotRequest: " + slotRequest);
-		return new SlotAssignment();
-	}
-
-
-	/**
-	 *
-	 * @param resourceManagerLeaderId  The fencing token for the ResourceManager leader 
-	 * @param taskExecutorAddress      The address of the TaskExecutor that registers
-	 * @param resourceID               The resource ID of the TaskExecutor that registers
-	 *
-	 * @return The response by the ResourceManager.
-	 */
-	@RpcMethod
-	public org.apache.flink.runtime.rpc.registration.RegistrationResponse registerTaskExecutor(
-			UUID resourceManagerLeaderId,
-			String taskExecutorAddress,
-			ResourceID resourceID) {
-
-		return new TaskExecutorRegistrationSuccess(new InstanceID(), 5000);
-	}
-
-	private class ResourceManagerLeaderContender implements LeaderContender {
-
-		/**
-		 * Callback method when current resourceManager is granted leadership
-		 *
-		 * @param leaderSessionID unique leadershipID
-		 */
-		@Override
-		public void grantLeadership(final UUID leaderSessionID) {
-			runAsync(new Runnable() {
-				@Override
-				public void run() {
-					log.info("ResourceManager {} was granted leadership with leader session ID {}", getAddress(), leaderSessionID);
-					ResourceManager.this.leaderSessionID = leaderSessionID;
-					// confirming the leader session ID might be blocking,
-					leaderElectionService.confirmLeaderSessionID(leaderSessionID);
-				}
-			});
-		}
-
-		/**
-		 * Callback method when current resourceManager lose leadership.
-		 */
-		@Override
-		public void revokeLeadership() {
-			runAsync(new Runnable() {
-				@Override
-				public void run() {
-					log.info("ResourceManager {} was revoked leadership.", getAddress());
-					jobMasterGateways.clear();
-					leaderSessionID = null;
-				}
-			});
-		}
-
-		@Override
-		public String getAddress() {
-			return ResourceManager.this.getAddress();
-		}
-
-		/**
-		 * Handles error occurring in the leader election service
-		 *
-		 * @param exception Exception being thrown in the leader election service
-		 */
-		@Override
-		public void handleError(final Exception exception) {
-			runAsync(new Runnable() {
-				@Override
-				public void run() {
-					log.error("ResourceManager received an error from the LeaderElectionService.", exception);
-					// terminate ResourceManager in case of an error
-					shutDown();
-				}
-			});
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9c247d1f/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManagerGateway.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManagerGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManagerGateway.java
deleted file mode 100644
index afddb01..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManagerGateway.java
+++ /dev/null
@@ -1,77 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.runtime.rpc.resourcemanager;
-
-import org.apache.flink.runtime.clusterframework.types.ResourceID;
-import org.apache.flink.runtime.rpc.RpcGateway;
-import org.apache.flink.runtime.rpc.RpcTimeout;
-import org.apache.flink.runtime.rpc.jobmaster.JobMaster;
-
-import scala.concurrent.Future;
-import scala.concurrent.duration.FiniteDuration;
-
-import java.util.UUID;
-
-/**
- * The {@link ResourceManager}'s RPC gateway interface.
- */
-public interface ResourceManagerGateway extends RpcGateway {
-
-	/**
-	 * Register a {@link JobMaster} at the resource manager.
-	 *
-	 * @param jobMasterRegistration Job master registration information
-	 * @param timeout Timeout for the future to complete
-	 * @return Future registration response
-	 */
-	Future<RegistrationResponse> registerJobMaster(
-		JobMasterRegistration jobMasterRegistration,
-		@RpcTimeout FiniteDuration timeout);
-
-	/**
-	 * Register a {@link JobMaster} at the resource manager.
-	 *
-	 * @param jobMasterRegistration Job master registration information
-	 * @return Future registration response
-	 */
-	Future<RegistrationResponse> registerJobMaster(JobMasterRegistration jobMasterRegistration);
-
-	/**
-	 * Requests a slot from the resource manager.
-	 *
-	 * @param slotRequest Slot request
-	 * @return Future slot assignment
-	 */
-	Future<SlotAssignment> requestSlot(SlotRequest slotRequest);
-
-	/**
-	 * 
-	 * @param resourceManagerLeaderId  The fencing token for the ResourceManager leader 
-	 * @param taskExecutorAddress      The address of the TaskExecutor that registers
-	 * @param resourceID               The resource ID of the TaskExecutor that registers
-	 * @param timeout                  The timeout for the response.
-	 * 
-	 * @return The future to the response by the ResourceManager.
-	 */
-	Future<org.apache.flink.runtime.rpc.registration.RegistrationResponse> registerTaskExecutor(
-			UUID resourceManagerLeaderId,
-			String taskExecutorAddress,
-			ResourceID resourceID,
-			@RpcTimeout FiniteDuration timeout);
-}

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

http://git-wip-us.apache.org/repos/asf/flink/blob/9c247d1f/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/SlotRequest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/SlotRequest.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/SlotRequest.java
deleted file mode 100644
index 74c7c39..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/SlotRequest.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.rpc.resourcemanager;
-
-import org.apache.flink.api.common.JobID;
-import org.apache.flink.runtime.clusterframework.types.AllocationID;
-import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
-
-import java.io.Serializable;
-
-import static org.apache.flink.util.Preconditions.checkNotNull;
-
-/**
- * This describes the requirement of the slot, mainly used by JobManager requesting slot from ResourceManager.
- */
-public class SlotRequest implements Serializable {
-
-	private static final long serialVersionUID = -6586877187990445986L;
-
-	/** The JobID of the slot requested for */
-	private final JobID jobId;
-
-	/** The unique identification of this request */
-	private final AllocationID allocationId;
-
-	/** The resource profile of the required slot */
-	private final ResourceProfile resourceProfile;
-
-	public SlotRequest(JobID jobId, AllocationID allocationId, ResourceProfile resourceProfile) {
-		this.jobId = checkNotNull(jobId);
-		this.allocationId = checkNotNull(allocationId);
-		this.resourceProfile = checkNotNull(resourceProfile);
-	}
-
-	/**
-	 * Get the JobID of the slot requested for.
-	 * @return The job id
-	 */
-	public JobID getJobId() {
-		return jobId;
-	}
-
-	/**
-	 * Get the unique identification of this request
-	 * @return the allocation id
-	 */
-	public AllocationID getAllocationId() {
-		return allocationId;
-	}
-
-	/**
-	 * Get the resource profile of the desired slot
-	 * @return The resource profile
-	 */
-	public ResourceProfile getResourceProfile() {
-		return resourceProfile;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9c247d1f/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/SlotReport.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/SlotReport.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/SlotReport.java
deleted file mode 100644
index c372ecb..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/SlotReport.java
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.runtime.rpc.taskexecutor;
-
-import org.apache.flink.runtime.clusterframework.types.ResourceID;
-
-import java.io.Serializable;
-import java.util.List;
-
-import static org.apache.flink.util.Preconditions.checkNotNull;
-
-/**
- * A report about the current status of all slots of the TaskExecutor, describing
- * which slots are available and allocated, and what jobs (JobManagers) the allocated slots
- * have been allocated to.
- */
-public class SlotReport implements Serializable {
-
-	private static final long serialVersionUID = -3150175198722481689L;
-
-	/** The slots status of the TaskManager */
-	private final List<SlotStatus> slotsStatus;
-
-	/** The resource id which identifies the TaskManager */
-	private final ResourceID resourceID;
-
-	public SlotReport(final List<SlotStatus> slotsStatus, final ResourceID resourceID) {
-		this.slotsStatus = checkNotNull(slotsStatus);
-		this.resourceID = checkNotNull(resourceID);
-	}
-
-	public List<SlotStatus> getSlotsStatus() {
-		return slotsStatus;
-	}
-
-	public ResourceID getResourceID() {
-		return resourceID;
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9c247d1f/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/SlotStatus.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/SlotStatus.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/SlotStatus.java
deleted file mode 100644
index e8e2084..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/SlotStatus.java
+++ /dev/null
@@ -1,129 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.runtime.rpc.taskexecutor;
-
-import org.apache.flink.api.common.JobID;
-import org.apache.flink.runtime.clusterframework.types.AllocationID;
-import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
-import org.apache.flink.runtime.clusterframework.types.SlotID;
-
-import java.io.Serializable;
-
-import static org.apache.flink.util.Preconditions.checkNotNull;
-
-/**
- * This describes the slot current status which located in TaskManager.
- */
-public class SlotStatus implements Serializable {
-
-	private static final long serialVersionUID = 5099191707339664493L;
-
-	/** slotID to identify a slot */
-	private final SlotID slotID;
-
-	/** the resource profile of the slot */
-	private final ResourceProfile profiler;
-
-	/** if the slot is allocated, allocationId identify its allocation; else, allocationId is null */
-	private final AllocationID allocationID;
-
-	/** if the slot is allocated, jobId identify which job this slot is allocated to; else, jobId is null */
-	private final JobID jobID;
-
-	public SlotStatus(SlotID slotID, ResourceProfile profiler) {
-		this(slotID, profiler, null, null);
-	}
-
-	public SlotStatus(SlotID slotID, ResourceProfile profiler, AllocationID allocationID, JobID jobID) {
-		this.slotID = checkNotNull(slotID, "slotID cannot be null");
-		this.profiler = checkNotNull(profiler, "profile cannot be null");
-		this.allocationID = allocationID;
-		this.jobID = jobID;
-	}
-
-	/**
-	 * Get the unique identification of this slot
-	 *
-	 * @return The slot id
-	 */
-	public SlotID getSlotID() {
-		return slotID;
-	}
-
-	/**
-	 * Get the resource profile of this slot
-	 *
-	 * @return The resource profile
-	 */
-	public ResourceProfile getProfiler() {
-		return profiler;
-	}
-
-	/**
-	 * Get the allocation id of this slot
-	 *
-	 * @return The allocation id if this slot is allocated, otherwise null
-	 */
-	public AllocationID getAllocationID() {
-		return allocationID;
-	}
-
-	/**
-	 * Get the job id of the slot allocated for
-	 *
-	 * @return The job id if this slot is allocated, otherwise null
-	 */
-	public JobID getJobID() {
-		return jobID;
-	}
-
-	@Override
-	public boolean equals(Object o) {
-		if (this == o) {
-			return true;
-		}
-		if (o == null || getClass() != o.getClass()) {
-			return false;
-		}
-
-		SlotStatus that = (SlotStatus) o;
-
-		if (!slotID.equals(that.slotID)) {
-			return false;
-		}
-		if (!profiler.equals(that.profiler)) {
-			return false;
-		}
-		if (allocationID != null ? !allocationID.equals(that.allocationID) : that.allocationID != null) {
-			return false;
-		}
-		return jobID != null ? jobID.equals(that.jobID) : that.jobID == null;
-
-	}
-
-	@Override
-	public int hashCode() {
-		int result = slotID.hashCode();
-		result = 31 * result + profiler.hashCode();
-		result = 31 * result + (allocationID != null ? allocationID.hashCode() : 0);
-		result = 31 * result + (jobID != null ? jobID.hashCode() : 0);
-		return result;
-	}
-
-}


[03/50] [abbrv] flink git commit: [FLINK-4529] [flip-6] Move TaskExecutor, JobMaster and ResourceManager out of the rpc package

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/9c247d1f/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/SlotManagerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/SlotManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/SlotManagerTest.java
deleted file mode 100644
index 2ee280f..0000000
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/SlotManagerTest.java
+++ /dev/null
@@ -1,540 +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.clusterframework;
-
-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.clusterframework.types.ResourceSlot;
-import org.apache.flink.runtime.clusterframework.types.SlotID;
-import org.apache.flink.runtime.rpc.resourcemanager.ResourceManagerGateway;
-import org.apache.flink.runtime.rpc.resourcemanager.SlotRequest;
-import org.apache.flink.runtime.rpc.taskexecutor.SlotStatus;
-import org.junit.Before;
-import org.junit.Test;
-
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.mockito.Mockito.mock;
-
-public class SlotManagerTest {
-
-	private static final double DEFAULT_TESTING_CPU_CORES = 1.0;
-
-	private static final long DEFAULT_TESTING_MEMORY = 512;
-
-	private static final ResourceProfile DEFAULT_TESTING_PROFILE =
-		new ResourceProfile(DEFAULT_TESTING_CPU_CORES, DEFAULT_TESTING_MEMORY);
-
-	private static final ResourceProfile DEFAULT_TESTING_BIG_PROFILE =
-		new ResourceProfile(DEFAULT_TESTING_CPU_CORES * 2, DEFAULT_TESTING_MEMORY * 2);
-
-	private ResourceManagerGateway resourceManagerGateway;
-
-	@Before
-	public void setUp() {
-		resourceManagerGateway = mock(ResourceManagerGateway.class);
-	}
-
-	/**
-	 * Tests that there are no free slots when we request, need to allocate from cluster manager master
-	 */
-	@Test
-	public void testRequestSlotWithoutFreeSlot() {
-		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
-		slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE));
-
-		assertEquals(0, slotManager.getAllocatedSlotCount());
-		assertEquals(0, slotManager.getFreeSlotCount());
-		assertEquals(1, slotManager.getPendingRequestCount());
-		assertEquals(1, slotManager.getAllocatedContainers().size());
-		assertEquals(DEFAULT_TESTING_PROFILE, slotManager.getAllocatedContainers().get(0));
-	}
-
-	/**
-	 * Tests that there are some free slots when we request, and the request is fulfilled immediately
-	 */
-	@Test
-	public void testRequestSlotWithFreeSlot() {
-		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
-
-		directlyProvideFreeSlots(slotManager, DEFAULT_TESTING_PROFILE, 1);
-		assertEquals(1, slotManager.getFreeSlotCount());
-
-		slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE));
-		assertEquals(1, slotManager.getAllocatedSlotCount());
-		assertEquals(0, slotManager.getFreeSlotCount());
-		assertEquals(0, slotManager.getPendingRequestCount());
-		assertEquals(0, slotManager.getAllocatedContainers().size());
-	}
-
-	/**
-	 * Tests that there are some free slots when we request, but none of them are suitable
-	 */
-	@Test
-	public void testRequestSlotWithoutSuitableSlot() {
-		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
-
-		directlyProvideFreeSlots(slotManager, DEFAULT_TESTING_PROFILE, 2);
-		assertEquals(2, slotManager.getFreeSlotCount());
-
-		slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_BIG_PROFILE));
-		assertEquals(0, slotManager.getAllocatedSlotCount());
-		assertEquals(2, slotManager.getFreeSlotCount());
-		assertEquals(1, slotManager.getPendingRequestCount());
-		assertEquals(1, slotManager.getAllocatedContainers().size());
-		assertEquals(DEFAULT_TESTING_BIG_PROFILE, slotManager.getAllocatedContainers().get(0));
-	}
-
-	/**
-	 * Tests that we send duplicated slot request
-	 */
-	@Test
-	public void testDuplicatedSlotRequest() {
-		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
-		directlyProvideFreeSlots(slotManager, DEFAULT_TESTING_PROFILE, 1);
-
-		SlotRequest request1 = new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE);
-		SlotRequest request2 = new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_BIG_PROFILE);
-
-		slotManager.requestSlot(request1);
-		slotManager.requestSlot(request2);
-		slotManager.requestSlot(request2);
-		slotManager.requestSlot(request1);
-
-		assertEquals(1, slotManager.getAllocatedSlotCount());
-		assertEquals(0, slotManager.getFreeSlotCount());
-		assertEquals(1, slotManager.getPendingRequestCount());
-		assertEquals(1, slotManager.getAllocatedContainers().size());
-		assertEquals(DEFAULT_TESTING_BIG_PROFILE, slotManager.getAllocatedContainers().get(0));
-	}
-
-	/**
-	 * Tests that we send multiple slot requests
-	 */
-	@Test
-	public void testRequestMultipleSlots() {
-		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
-		directlyProvideFreeSlots(slotManager, DEFAULT_TESTING_PROFILE, 5);
-
-		// request 3 normal slots
-		for (int i = 0; i < 3; ++i) {
-			slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE));
-		}
-
-		// request 2 big slots
-		for (int i = 0; i < 2; ++i) {
-			slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_BIG_PROFILE));
-		}
-
-		// request 1 normal slot again
-		slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE));
-
-		assertEquals(4, slotManager.getAllocatedSlotCount());
-		assertEquals(1, slotManager.getFreeSlotCount());
-		assertEquals(2, slotManager.getPendingRequestCount());
-		assertEquals(2, slotManager.getAllocatedContainers().size());
-		assertEquals(DEFAULT_TESTING_BIG_PROFILE, slotManager.getAllocatedContainers().get(0));
-		assertEquals(DEFAULT_TESTING_BIG_PROFILE, slotManager.getAllocatedContainers().get(1));
-	}
-
-	/**
-	 * Tests that a new slot appeared in SlotReport, and we used it to fulfill a pending request
-	 */
-	@Test
-	public void testNewlyAppearedFreeSlotFulfillPendingRequest() {
-		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
-		slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE));
-		assertEquals(1, slotManager.getPendingRequestCount());
-
-		SlotID slotId = SlotID.generate();
-		SlotStatus slotStatus = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE);
-		slotManager.updateSlotStatus(slotStatus);
-
-		assertEquals(1, slotManager.getAllocatedSlotCount());
-		assertEquals(0, slotManager.getFreeSlotCount());
-		assertEquals(0, slotManager.getPendingRequestCount());
-		assertTrue(slotManager.isAllocated(slotId));
-	}
-
-	/**
-	 * Tests that a new slot appeared in SlotReport, but we have no pending request
-	 */
-	@Test
-	public void testNewlyAppearedFreeSlot() {
-		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
-		SlotID slotId = SlotID.generate();
-		SlotStatus slotStatus = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE);
-		slotManager.updateSlotStatus(slotStatus);
-
-		assertEquals(0, slotManager.getAllocatedSlotCount());
-		assertEquals(1, slotManager.getFreeSlotCount());
-	}
-
-	/**
-	 * Tests that a new slot appeared in SlotReport, but it't not suitable for all the pending requests
-	 */
-	@Test
-	public void testNewlyAppearedFreeSlotNotMatchPendingRequests() {
-		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
-		slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_BIG_PROFILE));
-		assertEquals(1, slotManager.getPendingRequestCount());
-
-		SlotID slotId = SlotID.generate();
-		SlotStatus slotStatus = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE);
-		slotManager.updateSlotStatus(slotStatus);
-
-		assertEquals(0, slotManager.getAllocatedSlotCount());
-		assertEquals(1, slotManager.getFreeSlotCount());
-		assertEquals(1, slotManager.getPendingRequestCount());
-		assertFalse(slotManager.isAllocated(slotId));
-	}
-
-	/**
-	 * Tests that a new slot appeared in SlotReport, and it's been reported using by some job
-	 */
-	@Test
-	public void testNewlyAppearedInUseSlot() {
-		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
-
-		SlotID slotId = SlotID.generate();
-		SlotStatus slotStatus = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE, new AllocationID(), new JobID());
-		slotManager.updateSlotStatus(slotStatus);
-
-		assertEquals(1, slotManager.getAllocatedSlotCount());
-		assertEquals(0, slotManager.getFreeSlotCount());
-		assertTrue(slotManager.isAllocated(slotId));
-	}
-
-	/**
-	 * Tests that we had a slot in-use, and it's confirmed by SlotReport
-	 */
-	@Test
-	public void testExistingInUseSlotUpdateStatus() {
-		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
-		SlotRequest request = new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE);
-		slotManager.requestSlot(request);
-
-		// make this slot in use
-		SlotID slotId = SlotID.generate();
-		SlotStatus slotStatus = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE);
-		slotManager.updateSlotStatus(slotStatus);
-
-		assertEquals(1, slotManager.getAllocatedSlotCount());
-		assertEquals(0, slotManager.getFreeSlotCount());
-		assertTrue(slotManager.isAllocated(slotId));
-
-		// slot status is confirmed
-		SlotStatus slotStatus2 = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE,
-			request.getAllocationId(), request.getJobId());
-		slotManager.updateSlotStatus(slotStatus2);
-
-		assertEquals(1, slotManager.getAllocatedSlotCount());
-		assertEquals(0, slotManager.getFreeSlotCount());
-		assertTrue(slotManager.isAllocated(slotId));
-	}
-
-	/**
-	 * Tests that we had a slot in-use, but it's empty according to the SlotReport
-	 */
-	@Test
-	public void testExistingInUseSlotAdjustedToEmpty() {
-		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
-		SlotRequest request1 = new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE);
-		slotManager.requestSlot(request1);
-
-		// make this slot in use
-		SlotID slotId = SlotID.generate();
-		SlotStatus slotStatus = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE);
-		slotManager.updateSlotStatus(slotStatus);
-
-		// another request pending
-		SlotRequest request2 = new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE);
-		slotManager.requestSlot(request2);
-
-		assertEquals(1, slotManager.getAllocatedSlotCount());
-		assertEquals(0, slotManager.getFreeSlotCount());
-		assertEquals(1, slotManager.getPendingRequestCount());
-		assertTrue(slotManager.isAllocated(slotId));
-		assertTrue(slotManager.isAllocated(request1.getAllocationId()));
-
-
-		// but slot is reported empty again, request2 will be fulfilled, request1 will be missing
-		slotManager.updateSlotStatus(slotStatus);
-
-		assertEquals(1, slotManager.getAllocatedSlotCount());
-		assertEquals(0, slotManager.getFreeSlotCount());
-		assertEquals(0, slotManager.getPendingRequestCount());
-		assertTrue(slotManager.isAllocated(slotId));
-		assertTrue(slotManager.isAllocated(request2.getAllocationId()));
-	}
-
-	/**
-	 * Tests that we had a slot in use, and it's also reported in use by TaskManager, but the allocation
-	 * information didn't match.
-	 */
-	@Test
-	public void testExistingInUseSlotWithDifferentAllocationInfo() {
-		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
-		SlotRequest request = new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE);
-		slotManager.requestSlot(request);
-
-		// make this slot in use
-		SlotID slotId = SlotID.generate();
-		SlotStatus slotStatus = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE);
-		slotManager.updateSlotStatus(slotStatus);
-
-		assertEquals(1, slotManager.getAllocatedSlotCount());
-		assertEquals(0, slotManager.getFreeSlotCount());
-		assertEquals(0, slotManager.getPendingRequestCount());
-		assertTrue(slotManager.isAllocated(slotId));
-		assertTrue(slotManager.isAllocated(request.getAllocationId()));
-
-		SlotStatus slotStatus2 = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE, new AllocationID(), new JobID());
-		// update slot status with different allocation info
-		slotManager.updateSlotStatus(slotStatus2);
-
-		// original request is missing and won't be allocated
-		assertEquals(1, slotManager.getAllocatedSlotCount());
-		assertEquals(0, slotManager.getFreeSlotCount());
-		assertEquals(0, slotManager.getPendingRequestCount());
-		assertTrue(slotManager.isAllocated(slotId));
-		assertFalse(slotManager.isAllocated(request.getAllocationId()));
-		assertTrue(slotManager.isAllocated(slotStatus2.getAllocationID()));
-	}
-
-	/**
-	 * Tests that we had a free slot, and it's confirmed by SlotReport
-	 */
-	@Test
-	public void testExistingEmptySlotUpdateStatus() {
-		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
-		ResourceSlot slot = new ResourceSlot(SlotID.generate(), DEFAULT_TESTING_PROFILE);
-		slotManager.addFreeSlot(slot);
-
-		SlotStatus slotStatus = new SlotStatus(slot.getSlotId(), DEFAULT_TESTING_PROFILE);
-		slotManager.updateSlotStatus(slotStatus);
-
-		assertEquals(0, slotManager.getAllocatedSlotCount());
-		assertEquals(1, slotManager.getFreeSlotCount());
-		assertEquals(0, slotManager.getPendingRequestCount());
-	}
-
-	/**
-	 * Tests that we had a free slot, and it's reported in-use by TaskManager
-	 */
-	@Test
-	public void testExistingEmptySlotAdjustedToInUse() {
-		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
-		ResourceSlot slot = new ResourceSlot(SlotID.generate(), DEFAULT_TESTING_PROFILE);
-		slotManager.addFreeSlot(slot);
-
-		SlotStatus slotStatus = new SlotStatus(slot.getSlotId(), DEFAULT_TESTING_PROFILE,
-			new AllocationID(), new JobID());
-		slotManager.updateSlotStatus(slotStatus);
-
-		assertEquals(1, slotManager.getAllocatedSlotCount());
-		assertEquals(0, slotManager.getFreeSlotCount());
-		assertEquals(0, slotManager.getPendingRequestCount());
-		assertTrue(slotManager.isAllocated(slot.getSlotId()));
-	}
-
-	/**
-	 * Tests that we did some allocation but failed / rejected by TaskManager, request will retry
-	 */
-	@Test
-	public void testSlotAllocationFailedAtTaskManager() {
-		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
-		ResourceSlot slot = new ResourceSlot(SlotID.generate(), DEFAULT_TESTING_PROFILE);
-		slotManager.addFreeSlot(slot);
-
-		SlotRequest request = new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE);
-		slotManager.requestSlot(request);
-
-		assertEquals(1, slotManager.getAllocatedSlotCount());
-		assertEquals(0, slotManager.getFreeSlotCount());
-		assertEquals(0, slotManager.getPendingRequestCount());
-		assertTrue(slotManager.isAllocated(slot.getSlotId()));
-
-		slotManager.handleSlotRequestFailedAtTaskManager(request, slot.getSlotId());
-
-		assertEquals(1, slotManager.getAllocatedSlotCount());
-		assertEquals(0, slotManager.getFreeSlotCount());
-		assertEquals(0, slotManager.getPendingRequestCount());
-	}
-
-
-	/**
-	 * Tests that we did some allocation but failed / rejected by TaskManager, and slot is occupied by another request
-	 */
-	@Test
-	public void testSlotAllocationFailedAtTaskManagerOccupiedByOther() {
-		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
-		ResourceSlot slot = new ResourceSlot(SlotID.generate(), DEFAULT_TESTING_PROFILE);
-		slotManager.addFreeSlot(slot);
-
-		SlotRequest request = new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE);
-		slotManager.requestSlot(request);
-
-		// slot is set empty by heartbeat
-		SlotStatus slotStatus = new SlotStatus(slot.getSlotId(), slot.getResourceProfile());
-		slotManager.updateSlotStatus(slotStatus);
-
-		// another request took this slot
-		SlotRequest request2 = new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE);
-		slotManager.requestSlot(request2);
-
-		assertEquals(1, slotManager.getAllocatedSlotCount());
-		assertEquals(0, slotManager.getFreeSlotCount());
-		assertEquals(0, slotManager.getPendingRequestCount());
-		assertFalse(slotManager.isAllocated(request.getAllocationId()));
-		assertTrue(slotManager.isAllocated(request2.getAllocationId()));
-
-		// original request should be pended
-		slotManager.handleSlotRequestFailedAtTaskManager(request, slot.getSlotId());
-
-		assertEquals(1, slotManager.getAllocatedSlotCount());
-		assertEquals(0, slotManager.getFreeSlotCount());
-		assertEquals(1, slotManager.getPendingRequestCount());
-		assertFalse(slotManager.isAllocated(request.getAllocationId()));
-		assertTrue(slotManager.isAllocated(request2.getAllocationId()));
-	}
-
-	@Test
-	public void testNotifyTaskManagerFailure() {
-		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
-
-		ResourceID resource1 = ResourceID.generate();
-		ResourceID resource2 = ResourceID.generate();
-
-		ResourceSlot slot11 = new ResourceSlot(new SlotID(resource1, 1), DEFAULT_TESTING_PROFILE);
-		ResourceSlot slot12 = new ResourceSlot(new SlotID(resource1, 2), DEFAULT_TESTING_PROFILE);
-		ResourceSlot slot21 = new ResourceSlot(new SlotID(resource2, 1), DEFAULT_TESTING_PROFILE);
-		ResourceSlot slot22 = new ResourceSlot(new SlotID(resource2, 2), DEFAULT_TESTING_PROFILE);
-
-		slotManager.addFreeSlot(slot11);
-		slotManager.addFreeSlot(slot21);
-
-		slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE));
-		slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE));
-
-		assertEquals(2, slotManager.getAllocatedSlotCount());
-		assertEquals(0, slotManager.getFreeSlotCount());
-		assertEquals(0, slotManager.getPendingRequestCount());
-
-		slotManager.addFreeSlot(slot12);
-		slotManager.addFreeSlot(slot22);
-
-		assertEquals(2, slotManager.getAllocatedSlotCount());
-		assertEquals(2, slotManager.getFreeSlotCount());
-		assertEquals(0, slotManager.getPendingRequestCount());
-
-		slotManager.notifyTaskManagerFailure(resource2);
-
-		assertEquals(1, slotManager.getAllocatedSlotCount());
-		assertEquals(1, slotManager.getFreeSlotCount());
-		assertEquals(0, slotManager.getPendingRequestCount());
-
-		// notify an not exist resource failure
-		slotManager.notifyTaskManagerFailure(ResourceID.generate());
-
-		assertEquals(1, slotManager.getAllocatedSlotCount());
-		assertEquals(1, slotManager.getFreeSlotCount());
-		assertEquals(0, slotManager.getPendingRequestCount());
-	}
-
-	// ------------------------------------------------------------------------
-	//  testing utilities
-	// ------------------------------------------------------------------------
-
-	private void directlyProvideFreeSlots(
-		final SlotManager slotManager,
-		final ResourceProfile resourceProfile,
-		final int freeSlotNum)
-	{
-		for (int i = 0; i < freeSlotNum; ++i) {
-			slotManager.addFreeSlot(new ResourceSlot(SlotID.generate(), new ResourceProfile(resourceProfile)));
-		}
-	}
-
-	// ------------------------------------------------------------------------
-	//  testing classes
-	// ------------------------------------------------------------------------
-
-	private static class TestingSlotManager extends SlotManager {
-
-		private final List<ResourceProfile> allocatedContainers;
-
-		TestingSlotManager(ResourceManagerGateway resourceManagerGateway) {
-			super(resourceManagerGateway);
-			this.allocatedContainers = new LinkedList<>();
-		}
-
-		/**
-		 * Choose slot randomly if it matches requirement
-		 *
-		 * @param request   The slot request
-		 * @param freeSlots All slots which can be used
-		 * @return The chosen slot or null if cannot find a match
-		 */
-		@Override
-		protected ResourceSlot chooseSlotToUse(SlotRequest request, Map<SlotID, ResourceSlot> freeSlots) {
-			for (ResourceSlot slot : freeSlots.values()) {
-				if (slot.isMatchingRequirement(request.getResourceProfile())) {
-					return slot;
-				}
-			}
-			return null;
-		}
-
-		/**
-		 * Choose request randomly if offered slot can match its requirement
-		 *
-		 * @param offeredSlot     The free slot
-		 * @param pendingRequests All the pending slot requests
-		 * @return The chosen request's AllocationID or null if cannot find a match
-		 */
-		@Override
-		protected SlotRequest chooseRequestToFulfill(ResourceSlot offeredSlot,
-			Map<AllocationID, SlotRequest> pendingRequests)
-		{
-			for (Map.Entry<AllocationID, SlotRequest> pendingRequest : pendingRequests.entrySet()) {
-				if (offeredSlot.isMatchingRequirement(pendingRequest.getValue().getResourceProfile())) {
-					return pendingRequest.getValue();
-				}
-			}
-			return null;
-		}
-
-		@Override
-		protected void allocateContainer(ResourceProfile resourceProfile) {
-			allocatedContainers.add(resourceProfile);
-		}
-
-		List<ResourceProfile> getAllocatedContainers() {
-			return allocatedContainers;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9c247d1f/flink-runtime/src/test/java/org/apache/flink/runtime/registration/RetryingRegistrationTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/registration/RetryingRegistrationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/registration/RetryingRegistrationTest.java
new file mode 100644
index 0000000..80fa19c
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/registration/RetryingRegistrationTest.java
@@ -0,0 +1,336 @@
+/*
+ * 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.registration;
+
+import akka.dispatch.Futures;
+
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.runtime.rpc.RpcService;
+import org.apache.flink.runtime.rpc.TestingRpcService;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Test;
+
+import org.slf4j.LoggerFactory;
+
+import scala.concurrent.Await;
+import scala.concurrent.ExecutionContext$;
+import scala.concurrent.Future;
+import scala.concurrent.Promise;
+import scala.concurrent.duration.FiniteDuration;
+
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeoutException;
+
+import static java.util.concurrent.TimeUnit.SECONDS;
+import static org.junit.Assert.*;
+import static org.mockito.Mockito.*;
+
+/**
+ * Tests for the generic retrying registration class, validating the failure, retry, and back-off behavior.
+ */
+public class RetryingRegistrationTest extends TestLogger {
+
+	@Test
+	public void testSimpleSuccessfulRegistration() throws Exception {
+		final String testId = "laissez les bon temps roulez";
+		final String testEndpointAddress = "<test-address>";
+		final UUID leaderId = UUID.randomUUID();
+
+		// an endpoint that immediately returns success
+		TestRegistrationGateway testGateway = new TestRegistrationGateway(new TestRegistrationSuccess(testId));
+		TestingRpcService rpc = new TestingRpcService();
+
+		try {
+			rpc.registerGateway(testEndpointAddress, testGateway);
+
+			TestRetryingRegistration registration = new TestRetryingRegistration(rpc, testEndpointAddress, leaderId);
+			registration.startRegistration();
+
+			Future<Tuple2<TestRegistrationGateway, TestRegistrationSuccess>> future = registration.getFuture();
+			assertNotNull(future);
+
+			// multiple accesses return the same future
+			assertEquals(future, registration.getFuture());
+
+			Tuple2<TestRegistrationGateway, TestRegistrationSuccess> success = 
+					Await.result(future, new FiniteDuration(10, SECONDS));
+
+			// validate correct invocation and result
+			assertEquals(testId, success.f1.getCorrelationId());
+			assertEquals(leaderId, testGateway.getInvocations().take().leaderId());
+		}
+		finally {
+			testGateway.stop();
+			rpc.stopService();
+		}
+	}
+	
+	@Test
+	public void testPropagateFailures() throws Exception {
+		final String testExceptionMessage = "testExceptionMessage";
+
+		// RPC service that fails with exception upon the connection
+		RpcService rpc = mock(RpcService.class);
+		when(rpc.connect(anyString(), any(Class.class))).thenThrow(new RuntimeException(testExceptionMessage));
+
+		TestRetryingRegistration registration = new TestRetryingRegistration(rpc, "testaddress", UUID.randomUUID());
+		registration.startRegistration();
+
+		Future<?> future = registration.getFuture();
+		assertTrue(future.failed().isCompleted());
+
+		assertEquals(testExceptionMessage, future.failed().value().get().get().getMessage());
+	}
+
+	@Test
+	public void testRetryConnectOnFailure() throws Exception {
+		final String testId = "laissez les bon temps roulez";
+		final UUID leaderId = UUID.randomUUID();
+
+		ExecutorService executor = Executors.newCachedThreadPool();
+		TestRegistrationGateway testGateway = new TestRegistrationGateway(new TestRegistrationSuccess(testId));
+
+		try {
+			// RPC service that fails upon the first connection, but succeeds on the second
+			RpcService rpc = mock(RpcService.class);
+			when(rpc.connect(anyString(), any(Class.class))).thenReturn(
+					Futures.failed(new Exception("test connect failure")),  // first connection attempt fails
+					Futures.successful(testGateway)                         // second connection attempt succeeds
+			);
+			when(rpc.getExecutionContext()).thenReturn(ExecutionContext$.MODULE$.fromExecutor(executor));
+
+			TestRetryingRegistration registration = new TestRetryingRegistration(rpc, "foobar address", leaderId);
+			registration.startRegistration();
+
+			Tuple2<TestRegistrationGateway, TestRegistrationSuccess> success =
+					Await.result(registration.getFuture(), new FiniteDuration(10, SECONDS));
+
+			// validate correct invocation and result
+			assertEquals(testId, success.f1.getCorrelationId());
+			assertEquals(leaderId, testGateway.getInvocations().take().leaderId());
+		}
+		finally {
+			testGateway.stop();
+			executor.shutdown();
+		}
+	}
+
+	@Test
+	public void testRetriesOnTimeouts() throws Exception {
+		final String testId = "rien ne va plus";
+		final String testEndpointAddress = "<test-address>";
+		final UUID leaderId = UUID.randomUUID();
+
+		// an endpoint that immediately returns futures with timeouts before returning a successful future
+		TestRegistrationGateway testGateway = new TestRegistrationGateway(
+				null, // timeout
+				null, // timeout
+				new TestRegistrationSuccess(testId) // success
+		);
+
+		TestingRpcService rpc = new TestingRpcService();
+
+		try {
+			rpc.registerGateway(testEndpointAddress, testGateway);
+	
+			TestRetryingRegistration registration = new TestRetryingRegistration(rpc, testEndpointAddress, leaderId);
+	
+			long started = System.nanoTime();
+			registration.startRegistration();
+	
+			Future<Tuple2<TestRegistrationGateway, TestRegistrationSuccess>> future = registration.getFuture();
+			Tuple2<TestRegistrationGateway, TestRegistrationSuccess> success =
+					Await.result(future, new FiniteDuration(10, SECONDS));
+	
+			long finished = System.nanoTime();
+			long elapsedMillis = (finished - started) / 1000000;
+	
+			// validate correct invocation and result
+			assertEquals(testId, success.f1.getCorrelationId());
+			assertEquals(leaderId, testGateway.getInvocations().take().leaderId());
+	
+			// validate that some retry-delay / back-off behavior happened
+			assertTrue("retries did not properly back off", elapsedMillis >= 3 * TestRetryingRegistration.INITIAL_TIMEOUT);
+		}
+		finally {
+			rpc.stopService();
+			testGateway.stop();
+		}
+	}
+
+	@Test
+	public void testDecline() throws Exception {
+		final String testId = "qui a coupe le fromage";
+		final String testEndpointAddress = "<test-address>";
+		final UUID leaderId = UUID.randomUUID();
+
+		TestingRpcService rpc = new TestingRpcService();
+
+		TestRegistrationGateway testGateway = new TestRegistrationGateway(
+				null, // timeout
+				new RegistrationResponse.Decline("no reason "),
+				null, // timeout
+				new TestRegistrationSuccess(testId) // success
+		);
+
+		try {
+			rpc.registerGateway(testEndpointAddress, testGateway);
+
+			TestRetryingRegistration registration = new TestRetryingRegistration(rpc, testEndpointAddress, leaderId);
+
+			long started = System.nanoTime();
+			registration.startRegistration();
+	
+			Future<Tuple2<TestRegistrationGateway, TestRegistrationSuccess>> future = registration.getFuture();
+			Tuple2<TestRegistrationGateway, TestRegistrationSuccess> success =
+					Await.result(future, new FiniteDuration(10, SECONDS));
+
+			long finished = System.nanoTime();
+			long elapsedMillis = (finished - started) / 1000000;
+
+			// validate correct invocation and result
+			assertEquals(testId, success.f1.getCorrelationId());
+			assertEquals(leaderId, testGateway.getInvocations().take().leaderId());
+
+			// validate that some retry-delay / back-off behavior happened
+			assertTrue("retries did not properly back off", elapsedMillis >= 
+					2 * TestRetryingRegistration.INITIAL_TIMEOUT + TestRetryingRegistration.DELAY_ON_DECLINE);
+		}
+		finally {
+			testGateway.stop();
+			rpc.stopService();
+		}
+	}
+	
+	@Test
+	@SuppressWarnings("unchecked")
+	public void testRetryOnError() throws Exception {
+		final String testId = "Petit a petit, l'oiseau fait son nid";
+		final String testEndpointAddress = "<test-address>";
+		final UUID leaderId = UUID.randomUUID();
+
+		TestingRpcService rpc = new TestingRpcService();
+
+		try {
+			// gateway that upon calls first responds with a failure, then with a success
+			TestRegistrationGateway testGateway = mock(TestRegistrationGateway.class);
+
+			when(testGateway.registrationCall(any(UUID.class), anyLong())).thenReturn(
+					Futures.<RegistrationResponse>failed(new Exception("test exception")),
+					Futures.<RegistrationResponse>successful(new TestRegistrationSuccess(testId)));
+			
+			rpc.registerGateway(testEndpointAddress, testGateway);
+
+			TestRetryingRegistration registration = new TestRetryingRegistration(rpc, testEndpointAddress, leaderId);
+
+			long started = System.nanoTime();
+			registration.startRegistration();
+
+			Future<Tuple2<TestRegistrationGateway, TestRegistrationSuccess>> future = registration.getFuture();
+			Tuple2<TestRegistrationGateway, TestRegistrationSuccess> success =
+					Await.result(future, new FiniteDuration(10, SECONDS));
+
+			long finished = System.nanoTime();
+			long elapsedMillis = (finished - started) / 1000000;
+			
+			assertEquals(testId, success.f1.getCorrelationId());
+
+			// validate that some retry-delay / back-off behavior happened
+			assertTrue("retries did not properly back off",
+					elapsedMillis >= TestRetryingRegistration.DELAY_ON_ERROR);
+		}
+		finally {
+			rpc.stopService();
+		}
+	}
+
+	@Test
+	public void testCancellation() throws Exception {
+		final String testEndpointAddress = "my-test-address";
+		final UUID leaderId = UUID.randomUUID();
+
+		TestingRpcService rpc = new TestingRpcService();
+
+		try {
+			Promise<RegistrationResponse> result = Futures.promise();
+
+			TestRegistrationGateway testGateway = mock(TestRegistrationGateway.class);
+			when(testGateway.registrationCall(any(UUID.class), anyLong())).thenReturn(result.future());
+
+			rpc.registerGateway(testEndpointAddress, testGateway);
+
+			TestRetryingRegistration registration = new TestRetryingRegistration(rpc, testEndpointAddress, leaderId);
+			registration.startRegistration();
+
+			// cancel and fail the current registration attempt
+			registration.cancel();
+			result.failure(new TimeoutException());
+
+			// there should not be a second registration attempt
+			verify(testGateway, atMost(1)).registrationCall(any(UUID.class), anyLong());
+		}
+		finally {
+			rpc.stopService();
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  test registration
+	// ------------------------------------------------------------------------
+
+	private static class TestRegistrationSuccess extends RegistrationResponse.Success {
+		private static final long serialVersionUID = 5542698790917150604L;
+
+		private final String correlationId;
+
+		private TestRegistrationSuccess(String correlationId) {
+			this.correlationId = correlationId;
+		}
+
+		public String getCorrelationId() {
+			return correlationId;
+		}
+	}
+
+	private static class TestRetryingRegistration extends RetryingRegistration<TestRegistrationGateway, TestRegistrationSuccess> {
+
+		// we use shorter timeouts here to speed up the tests
+		static final long INITIAL_TIMEOUT = 20;
+		static final long MAX_TIMEOUT = 200;
+		static final long DELAY_ON_ERROR = 200;
+		static final long DELAY_ON_DECLINE = 200;
+
+		public TestRetryingRegistration(RpcService rpc, String targetAddress, UUID leaderId) {
+			super(LoggerFactory.getLogger(RetryingRegistrationTest.class),
+					rpc, "TestEndpoint",
+					TestRegistrationGateway.class,
+					targetAddress, leaderId,
+					INITIAL_TIMEOUT, MAX_TIMEOUT, DELAY_ON_ERROR, DELAY_ON_DECLINE);
+		}
+
+		@Override
+		protected Future<RegistrationResponse> invokeRegistration(
+				TestRegistrationGateway gateway, UUID leaderId, long timeoutMillis) {
+			return gateway.registrationCall(leaderId, timeoutMillis);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9c247d1f/flink-runtime/src/test/java/org/apache/flink/runtime/registration/TestRegistrationGateway.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/registration/TestRegistrationGateway.java b/flink-runtime/src/test/java/org/apache/flink/runtime/registration/TestRegistrationGateway.java
new file mode 100644
index 0000000..431fbe8
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/registration/TestRegistrationGateway.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.registration;
+
+import akka.dispatch.Futures;
+
+import org.apache.flink.runtime.rpc.TestingGatewayBase;
+import org.apache.flink.util.Preconditions;
+
+import scala.concurrent.Future;
+
+import java.util.UUID;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+
+public class TestRegistrationGateway extends TestingGatewayBase {
+
+	private final BlockingQueue<RegistrationCall> invocations;
+
+	private final RegistrationResponse[] responses;
+
+	private int pos;
+
+	public TestRegistrationGateway(RegistrationResponse... responses) {
+		Preconditions.checkArgument(responses != null && responses.length > 0);
+
+		this.invocations = new LinkedBlockingQueue<>();
+		this.responses = responses;
+		
+	}
+
+	// ------------------------------------------------------------------------
+
+	public Future<RegistrationResponse> registrationCall(UUID leaderId, long timeout) {
+		invocations.add(new RegistrationCall(leaderId, timeout));
+
+		RegistrationResponse response = responses[pos];
+		if (pos < responses.length - 1) {
+			pos++;
+		}
+
+		// return a completed future (for a proper value), or one that never completes and will time out (for null)
+		return response != null ? Futures.successful(response) : this.<RegistrationResponse>futureWithTimeout(timeout);
+	}
+
+	public BlockingQueue<RegistrationCall> getInvocations() {
+		return invocations;
+	}
+
+	// ------------------------------------------------------------------------
+
+	public static class RegistrationCall {
+		private final UUID leaderId;
+		private final long timeout;
+
+		public RegistrationCall(UUID leaderId, long timeout) {
+			this.leaderId = leaderId;
+			this.timeout = timeout;
+		}
+
+		public UUID leaderId() {
+			return leaderId;
+		}
+
+		public long timeout() {
+			return timeout;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9c247d1f/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ClusterShutdownITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ClusterShutdownITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ClusterShutdownITCase.java
deleted file mode 100644
index 32c6cac..0000000
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ClusterShutdownITCase.java
+++ /dev/null
@@ -1,156 +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 akka.actor.ActorSystem;
-import akka.testkit.JavaTestKit;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.akka.AkkaUtils;
-import org.apache.flink.runtime.clusterframework.ApplicationStatus;
-import org.apache.flink.runtime.clusterframework.messages.StopCluster;
-import org.apache.flink.runtime.clusterframework.messages.StopClusterSuccessful;
-import org.apache.flink.runtime.instance.ActorGateway;
-import org.apache.flink.runtime.messages.Messages;
-import org.apache.flink.runtime.testingUtils.TestingMessages;
-import org.apache.flink.runtime.testingUtils.TestingUtils;
-import org.apache.flink.runtime.testutils.TestingResourceManager;
-import org.apache.flink.util.TestLogger;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import scala.Option;
-
-
-/**
- * Runs tests to ensure that a cluster is shutdown properly.
- */
-public class ClusterShutdownITCase extends TestLogger {
-
-	private static ActorSystem system;
-
-	private static Configuration config = new Configuration();
-
-	@BeforeClass
-	public static void setup() {
-		system = AkkaUtils.createActorSystem(AkkaUtils.getDefaultAkkaConfig());
-	}
-
-	@AfterClass
-	public static void teardown() {
-		JavaTestKit.shutdownActorSystem(system);
-	}
-
-	/**
-	 * Tests a faked cluster shutdown procedure without the ResourceManager.
-	 */
-	@Test
-	public void testClusterShutdownWithoutResourceManager() {
-
-		new JavaTestKit(system){{
-		new Within(duration("30 seconds")) {
-		@Override
-		protected void run() {
-
-			ActorGateway me =
-				TestingUtils.createForwardingActor(system, getTestActor(), Option.<String>empty());
-
-			// start job manager which doesn't shutdown the actor system
-			ActorGateway jobManager =
-				TestingUtils.createJobManager(system, config, "jobmanager1");
-
-			// Tell the JobManager to inform us of shutdown actions
-			jobManager.tell(TestingMessages.getNotifyOfComponentShutdown(), me);
-
-			// Register a TaskManager
-			ActorGateway taskManager =
-				TestingUtils.createTaskManager(system, jobManager, config, true, true);
-
-			// Tell the TaskManager to inform us of TaskManager shutdowns
-			taskManager.tell(TestingMessages.getNotifyOfComponentShutdown(), me);
-
-
-			// No resource manager connected
-			jobManager.tell(new StopCluster(ApplicationStatus.SUCCEEDED, "Shutting down."), me);
-
-			expectMsgAllOf(
-				new TestingMessages.ComponentShutdown(taskManager.actor()),
-				new TestingMessages.ComponentShutdown(jobManager.actor()),
-				StopClusterSuccessful.getInstance()
-			);
-
-		}};
-		}};
-	}
-
-	/**
-	 * Tests a faked cluster shutdown procedure with the ResourceManager.
-	 */
-	@Test
-	public void testClusterShutdownWithResourceManager() {
-
-		new JavaTestKit(system){{
-		new Within(duration("30 seconds")) {
-		@Override
-		protected void run() {
-
-			ActorGateway me =
-				TestingUtils.createForwardingActor(system, getTestActor(), Option.<String>empty());
-
-			// start job manager which doesn't shutdown the actor system
-			ActorGateway jobManager =
-				TestingUtils.createJobManager(system, config, "jobmanager2");
-
-			// Tell the JobManager to inform us of shutdown actions
-			jobManager.tell(TestingMessages.getNotifyOfComponentShutdown(), me);
-
-			// Register a TaskManager
-			ActorGateway taskManager =
-				TestingUtils.createTaskManager(system, jobManager, config, true, true);
-
-			// Tell the TaskManager to inform us of TaskManager shutdowns
-			taskManager.tell(TestingMessages.getNotifyOfComponentShutdown(), me);
-
-			// Start resource manager and let it register
-			ActorGateway resourceManager =
-				TestingUtils.createResourceManager(system, jobManager.actor(), config);
-
-			// Tell the ResourceManager to inform us of ResourceManager shutdowns
-			resourceManager.tell(TestingMessages.getNotifyOfComponentShutdown(), me);
-
-			// notify about a resource manager registration at the job manager
-			resourceManager.tell(new TestingResourceManager.NotifyWhenResourceManagerConnected(), me);
-
-			// Wait for resource manager
-			expectMsgEquals(Messages.getAcknowledge());
-
-
-			// Shutdown cluster with resource manager connected
-			jobManager.tell(new StopCluster(ApplicationStatus.SUCCEEDED, "Shutting down."), me);
-
-			expectMsgAllOf(
-				new TestingMessages.ComponentShutdown(taskManager.actor()),
-				new TestingMessages.ComponentShutdown(jobManager.actor()),
-				new TestingMessages.ComponentShutdown(resourceManager.actor()),
-				StopClusterSuccessful.getInstance()
-			);
-
-		}};
-		}};
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9c247d1f/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
new file mode 100644
index 0000000..5799e62
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerHATest.java
@@ -0,0 +1,76 @@
+/*
+ * 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.TestingHighAvailabilityServices;
+import org.apache.flink.runtime.leaderelection.TestingLeaderElectionService;
+import org.apache.flink.runtime.rpc.MainThreadExecutor;
+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.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
+ */
+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 execute runAsync call
+		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);
+
+		TestingLeaderElectionService leaderElectionService = new TestingLeaderElectionService();
+		TestingHighAvailabilityServices highAvailabilityServices = new TestingHighAvailabilityServices();
+		highAvailabilityServices.setResourceManagerLeaderElectionService(leaderElectionService);
+
+		final ResourceManager resourceManager = new ResourceManager(rpcService, highAvailabilityServices);
+		resourceManager.start();
+		// before grant leadership, resourceManager's leaderId is null
+		Assert.assertNull(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.assertNull(resourceManager.getLeaderSessionID());
+	}
+
+	private static abstract class TestingResourceManagerGatewayProxy implements MainThreadExecutor, StartStoppable, RpcGateway {
+		@Override
+		public void runAsync(Runnable runnable) {
+			runnable.run();
+		}
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9c247d1f/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerITCase.java
deleted file mode 100644
index 0c2ca1a..0000000
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerITCase.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.resourcemanager;
-
-import akka.actor.ActorSystem;
-import akka.testkit.JavaTestKit;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.akka.AkkaUtils;
-import org.apache.flink.runtime.clusterframework.types.ResourceID;
-import org.apache.flink.runtime.instance.ActorGateway;
-import org.apache.flink.runtime.instance.HardwareDescription;
-import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
-import org.apache.flink.runtime.messages.Messages;
-import org.apache.flink.runtime.messages.RegistrationMessages;
-import org.apache.flink.runtime.testingUtils.TestingUtils;
-import org.apache.flink.runtime.testutils.TestingResourceManager;
-import org.apache.flink.util.TestLogger;
-
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import scala.Option;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-/**
- * It cases which test the interaction of the resource manager with job manager and task managers.
- * Runs all tests in one Actor system.
- */
-public class ResourceManagerITCase extends TestLogger {
-
-	private static ActorSystem system;
-
-	private static Configuration config = new Configuration();
-
-	@BeforeClass
-	public static void setup() {
-		system = AkkaUtils.createActorSystem(AkkaUtils.getDefaultAkkaConfig());
-	}
-
-	@AfterClass
-	public static void teardown() {
-		JavaTestKit.shutdownActorSystem(system);
-	}
-
-	/**
-	 * Tests whether the resource manager connects and reconciles existing task managers.
-	 */
-	@Test
-	public void testResourceManagerReconciliation() {
-
-		new JavaTestKit(system){{
-		new Within(duration("10 seconds")) {
-		@Override
-		protected void run() {
-
-			ActorGateway jobManager =
-				TestingUtils.createJobManager(system, config, "ReconciliationTest");
-			ActorGateway me =
-				TestingUtils.createForwardingActor(system, getTestActor(), Option.<String>empty());
-
-			// !! no resource manager started !!
-
-			ResourceID resourceID = ResourceID.generate();
-
-			TaskManagerLocation location = mock(TaskManagerLocation.class);
-			when(location.getResourceID()).thenReturn(resourceID);
-
-			HardwareDescription resourceProfile = HardwareDescription.extractFromSystem(1_000_000);
-
-			jobManager.tell(
-				new RegistrationMessages.RegisterTaskManager(resourceID, location, resourceProfile, 1),
-				me);
-
-			expectMsgClass(RegistrationMessages.AcknowledgeRegistration.class);
-
-			// now start the resource manager
-			ActorGateway resourceManager =
-				TestingUtils.createResourceManager(system, jobManager.actor(), config);
-
-			// register at testing job manager to receive a message once a resource manager registers
-			resourceManager.tell(new TestingResourceManager.NotifyWhenResourceManagerConnected(), me);
-
-			// Wait for resource manager
-			expectMsgEquals(Messages.getAcknowledge());
-
-			// check if we registered the task manager resource
-			resourceManager.tell(new TestingResourceManager.GetRegisteredResources(), me);
-
-			TestingResourceManager.GetRegisteredResourcesReply reply =
-				expectMsgClass(TestingResourceManager.GetRegisteredResourcesReply.class);
-
-			assertEquals(1, reply.resources.size());
-			assertTrue(reply.resources.contains(resourceID));
-
-		}};
-		}};
-	}
-
-	/**
-	 * Tests whether the resource manager gets informed upon TaskManager registration.
-	 */
-	@Test
-	public void testResourceManagerTaskManagerRegistration() {
-
-		new JavaTestKit(system){{
-		new Within(duration("30 seconds")) {
-		@Override
-		protected void run() {
-
-			ActorGateway jobManager =
-				TestingUtils.createJobManager(system, config, "RegTest");
-			ActorGateway me =
-				TestingUtils.createForwardingActor(system, getTestActor(), Option.<String>empty());
-
-			// start the resource manager
-			ActorGateway resourceManager =
-				TestingUtils.createResourceManager(system, jobManager.actor(), config);
-
-			// notify about a resource manager registration at the job manager
-			resourceManager.tell(new TestingResourceManager.NotifyWhenResourceManagerConnected(), me);
-
-			// Wait for resource manager
-			expectMsgEquals(Messages.getAcknowledge());
-
-			// start task manager and wait for registration
-			ActorGateway taskManager =
-				TestingUtils.createTaskManager(system, jobManager.actor(), config, true, true);
-
-			// check if we registered the task manager resource
-			resourceManager.tell(new TestingResourceManager.GetRegisteredResources(), me);
-
-			TestingResourceManager.GetRegisteredResourcesReply reply =
-				expectMsgClass(TestingResourceManager.GetRegisteredResourcesReply.class);
-
-			assertEquals(1, reply.resources.size());
-
-		}};
-		}};
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9c247d1f/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTest.java
deleted file mode 100644
index 043c81c..0000000
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTest.java
+++ /dev/null
@@ -1,338 +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 akka.actor.ActorSystem;
-import akka.testkit.JavaTestKit;
-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.messages.NotifyResourceStarted;
-import org.apache.flink.runtime.clusterframework.messages.RegisterResourceManager;
-import org.apache.flink.runtime.clusterframework.messages.RegisterResourceManagerSuccessful;
-import org.apache.flink.runtime.clusterframework.messages.RemoveResource;
-import org.apache.flink.runtime.clusterframework.messages.ResourceRemoved;
-import org.apache.flink.runtime.clusterframework.messages.TriggerRegistrationAtJobManager;
-import org.apache.flink.runtime.clusterframework.types.ResourceID;
-import org.apache.flink.runtime.instance.ActorGateway;
-import org.apache.flink.runtime.messages.Acknowledge;
-import org.apache.flink.runtime.messages.JobManagerMessages;
-import org.apache.flink.runtime.testingUtils.TestingUtils;
-import org.apache.flink.runtime.testutils.TestingResourceManager;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import scala.Option;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-
-import static org.junit.Assert.*;
-
-/**
- * General tests for the resource manager component.
- */
-public class ResourceManagerTest {
-
-	private static ActorSystem system;
-
-	private static ActorGateway fakeJobManager;
-	private static ActorGateway resourceManager;
-
-	private static Configuration config = new Configuration();
-
-	@BeforeClass
-	public static void setup() {
-		system = AkkaUtils.createLocalActorSystem(config);
-	}
-
-	@AfterClass
-	public static void teardown() {
-		JavaTestKit.shutdownActorSystem(system);
-	}
-
-	/**
-	 * Tests the registration and reconciliation of the ResourceManager with the JobManager
-	 */
-	@Test
-	public void testJobManagerRegistrationAndReconciliation() {
-		new JavaTestKit(system){{
-		new Within(duration("10 seconds")) {
-		@Override
-		protected void run() {
-			fakeJobManager = TestingUtils.createForwardingActor(system, getTestActor(), Option.<String>empty());
-			resourceManager = TestingUtils.createResourceManager(system, fakeJobManager.actor(), config);
-
-			expectMsgClass(RegisterResourceManager.class);
-
-			List<ResourceID> resourceList = new ArrayList<>();
-			resourceList.add(ResourceID.generate());
-			resourceList.add(ResourceID.generate());
-			resourceList.add(ResourceID.generate());
-
-			resourceManager.tell(
-				new RegisterResourceManagerSuccessful(fakeJobManager.actor(), resourceList),
-				fakeJobManager);
-
-			resourceManager.tell(new TestingResourceManager.GetRegisteredResources(), fakeJobManager);
-			TestingResourceManager.GetRegisteredResourcesReply reply =
-				expectMsgClass(TestingResourceManager.GetRegisteredResourcesReply.class);
-
-			for (ResourceID id : resourceList) {
-				if (!reply.resources.contains(id)) {
-					fail("Expected to find all resources that were provided during registration.");
-				}
-			}
-		}};
-		}};
-	}
-
-	/**
-	 * Tests delayed or erroneous registration of the ResourceManager with the JobManager
-	 */
-	@Test
-	public void testDelayedJobManagerRegistration() {
-		new JavaTestKit(system){{
-		new Within(duration("10 seconds")) {
-		@Override
-		protected void run() {
-
-			// set a short timeout for lookups
-			Configuration shortTimeoutConfig = config.clone();
-			shortTimeoutConfig.setString(ConfigConstants.AKKA_LOOKUP_TIMEOUT, "1 s");
-
-			fakeJobManager = TestingUtils.createForwardingActor(system, getTestActor(), Option.<String>empty());
-			resourceManager = TestingUtils.createResourceManager(system, fakeJobManager.actor(), shortTimeoutConfig);
-
-			// wait for registration message
-			RegisterResourceManager msg = expectMsgClass(RegisterResourceManager.class);
-			// give wrong response
-			getLastSender().tell(new JobManagerMessages.LeaderSessionMessage(null, new Object()),
-				fakeJobManager.actor());
-
-			// expect another retry and let it time out
-			expectMsgClass(RegisterResourceManager.class);
-
-			// wait for next try after timeout
-			expectMsgClass(RegisterResourceManager.class);
-
-		}};
-		}};
-	}
-
-	@Test
-	public void testTriggerReconnect() {
-		new JavaTestKit(system){{
-		new Within(duration("10 seconds")) {
-		@Override
-		protected void run() {
-
-			// set a long timeout for lookups such that the test fails in case of timeouts
-			Configuration shortTimeoutConfig = config.clone();
-			shortTimeoutConfig.setString(ConfigConstants.AKKA_LOOKUP_TIMEOUT, "99999 s");
-
-			fakeJobManager = TestingUtils.createForwardingActor(system, getTestActor(), Option.<String>empty());
-			resourceManager = TestingUtils.createResourceManager(system, fakeJobManager.actor(), shortTimeoutConfig);
-
-			// wait for registration message
-			RegisterResourceManager msg = expectMsgClass(RegisterResourceManager.class);
-			// all went well
-			resourceManager.tell(
-				new RegisterResourceManagerSuccessful(fakeJobManager.actor(), Collections.<ResourceID>emptyList()),
-				fakeJobManager);
-
-			// force a reconnect
-			resourceManager.tell(
-				new TriggerRegistrationAtJobManager(fakeJobManager.actor()),
-				fakeJobManager);
-
-			// new registration attempt should come in
-			expectMsgClass(RegisterResourceManager.class);
-
-		}};
-		}};
-	}
-
-	/**
-	 * Tests the registration and accounting of resources at the ResourceManager.
-	 */
-	@Test
-	public void testTaskManagerRegistration() {
-		new JavaTestKit(system){{
-		new Within(duration("10 seconds")) {
-		@Override
-		protected void run() {
-
-			fakeJobManager = TestingUtils.createForwardingActor(system, getTestActor(), Option.<String>empty());
-			resourceManager = TestingUtils.createResourceManager(system, fakeJobManager.actor(), config);
-
-			// register with JM
-			expectMsgClass(RegisterResourceManager.class);
-			resourceManager.tell(
-				new RegisterResourceManagerSuccessful(fakeJobManager.actor(), Collections.<ResourceID>emptyList()),
-				fakeJobManager);
-
-			ResourceID resourceID = ResourceID.generate();
-
-			// Send task manager registration
-			resourceManager.tell(new NotifyResourceStarted(resourceID),
-				fakeJobManager);
-
-			expectMsgClass(Acknowledge.class);
-
-			// check for number registration of registered resources
-			resourceManager.tell(new TestingResourceManager.GetRegisteredResources(), fakeJobManager);
-			TestingResourceManager.GetRegisteredResourcesReply reply =
-				expectMsgClass(TestingResourceManager.GetRegisteredResourcesReply.class);
-
-			assertEquals(1, reply.resources.size());
-
-			// Send task manager registration again
-			resourceManager.tell(new NotifyResourceStarted(resourceID),
-				fakeJobManager);
-
-			expectMsgClass(Acknowledge.class);
-
-			// check for number registration of registered resources
-			resourceManager.tell(new TestingResourceManager.GetRegisteredResources(), fakeJobManager);
-			reply = expectMsgClass(TestingResourceManager.GetRegisteredResourcesReply.class);
-
-			assertEquals(1, reply.resources.size());
-
-			// Send invalid null resource id to throw an exception during resource registration
-			resourceManager.tell(new NotifyResourceStarted(null),
-				fakeJobManager);
-
-			expectMsgClass(Acknowledge.class);
-
-			// check for number registration of registered resources
-			resourceManager.tell(new TestingResourceManager.GetRegisteredResources(), fakeJobManager);
-			reply = expectMsgClass(TestingResourceManager.GetRegisteredResourcesReply.class);
-
-			assertEquals(1, reply.resources.size());
-		}};
-		}};
-	}
-
-	@Test
-	public void testResourceRemoval() {
-		new JavaTestKit(system){{
-		new Within(duration("10 seconds")) {
-		@Override
-		protected void run() {
-
-			fakeJobManager = TestingUtils.createForwardingActor(system, getTestActor(), Option.<String>empty());
-			resourceManager = TestingUtils.createResourceManager(system, fakeJobManager.actor(), config);
-
-			// register with JM
-			expectMsgClass(RegisterResourceManager.class);
-			resourceManager.tell(
-				new RegisterResourceManagerSuccessful(fakeJobManager.actor(), Collections.<ResourceID>emptyList()),
-				fakeJobManager);
-
-			ResourceID resourceID = ResourceID.generate();
-
-			// remove unknown resource
-			resourceManager.tell(new RemoveResource(resourceID), fakeJobManager);
-
-			// Send task manager registration
-			resourceManager.tell(new NotifyResourceStarted(resourceID),
-				fakeJobManager);
-
-			expectMsgClass(Acknowledge.class);
-
-			// check for number registration of registered resources
-			resourceManager.tell(new TestingResourceManager.GetRegisteredResources(), fakeJobManager);
-			TestingResourceManager.GetRegisteredResourcesReply reply =
-				expectMsgClass(TestingResourceManager.GetRegisteredResourcesReply.class);
-
-			assertEquals(1, reply.resources.size());
-			assertTrue(reply.resources.contains(resourceID));
-
-			// remove resource
-			resourceManager.tell(new RemoveResource(resourceID), fakeJobManager);
-
-			// check for number registration of registered resources
-			resourceManager.tell(new TestingResourceManager.GetRegisteredResources(), fakeJobManager);
-			reply =	expectMsgClass(TestingResourceManager.GetRegisteredResourcesReply.class);
-
-			assertEquals(0, reply.resources.size());
-
-		}};
-		}};
-	}
-
-	/**
-	 * Tests notification of JobManager about a failed resource.
-	 */
-	@Test
-	public void testResourceFailureNotification() {
-		new JavaTestKit(system){{
-		new Within(duration("10 seconds")) {
-		@Override
-		protected void run() {
-
-			fakeJobManager = TestingUtils.createForwardingActor(system, getTestActor(), Option.<String>empty());
-			resourceManager = TestingUtils.createResourceManager(system, fakeJobManager.actor(), config);
-
-			// register with JM
-			expectMsgClass(RegisterResourceManager.class);
-			resourceManager.tell(
-				new RegisterResourceManagerSuccessful(fakeJobManager.actor(), Collections.<ResourceID>emptyList()),
-				fakeJobManager);
-
-			ResourceID resourceID1 = ResourceID.generate();
-			ResourceID resourceID2 = ResourceID.generate();
-
-			// Send task manager registration
-			resourceManager.tell(new NotifyResourceStarted(resourceID1),
-				fakeJobManager);
-
-			expectMsgClass(Acknowledge.class);
-
-			// Send task manager registration
-			resourceManager.tell(new NotifyResourceStarted(resourceID2),
-				fakeJobManager);
-
-			expectMsgClass(Acknowledge.class);
-
-			// check for number registration of registered resources
-			resourceManager.tell(new TestingResourceManager.GetRegisteredResources(), fakeJobManager);
-			TestingResourceManager.GetRegisteredResourcesReply reply =
-				expectMsgClass(TestingResourceManager.GetRegisteredResourcesReply.class);
-
-			assertEquals(2, reply.resources.size());
-			assertTrue(reply.resources.contains(resourceID1));
-			assertTrue(reply.resources.contains(resourceID2));
-
-			// fail resources
-			resourceManager.tell(new TestingResourceManager.FailResource(resourceID1), fakeJobManager);
-			resourceManager.tell(new TestingResourceManager.FailResource(resourceID2), fakeJobManager);
-
-			ResourceRemoved answer = expectMsgClass(ResourceRemoved.class);
-			ResourceRemoved answer2 = expectMsgClass(ResourceRemoved.class);
-
-			assertEquals(resourceID1, answer.resourceId());
-			assertEquals(resourceID2, answer2.resourceId());
-
-		}};
-		}};
-	}
-}


[45/50] [abbrv] flink git commit: [FLINK-4746] Make TaskManagerRuntimeInfo an interface

Posted by se...@apache.org.
[FLINK-4746] Make TaskManagerRuntimeInfo an interface

Let the TaskManagerConfiguration implement the TaskManagerRuntimeInformation to make some of
the TaskManager's configuration values accessible from different components.

This closes #2599.


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

Branch: refs/heads/flip-6
Commit: b380634965acf1327ac0f6ca0a64e7d5522dbea7
Parents: a00619a
Author: Till Rohrmann <tr...@apache.org>
Authored: Wed Oct 5 14:47:24 2016 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Oct 14 15:14:43 2016 +0200

----------------------------------------------------------------------
 .../runtime/taskexecutor/TaskExecutor.java      | 11 +---
 .../taskexecutor/TaskManagerConfiguration.java  | 22 +++----
 .../taskmanager/TaskManagerRuntimeInfo.java     | 61 ++------------------
 .../flink/runtime/taskmanager/TaskManager.scala | 11 +---
 .../operators/drivers/TestTaskContext.java      |  4 +-
 .../testutils/BinaryOperatorTestBase.java       |  4 +-
 .../operators/testutils/DriverTestBase.java     |  4 +-
 .../operators/testutils/MockEnvironment.java    |  8 +--
 .../testutils/UnaryOperatorTestBase.java        |  4 +-
 .../runtime/taskexecutor/TaskExecutorTest.java  |  8 +--
 .../runtime/taskmanager/TaskAsyncCallTest.java  |  4 +-
 .../flink/runtime/taskmanager/TaskTest.java     |  3 +-
 .../util/TestingTaskManagerRuntimeInfo.java     | 52 +++++++++++++++++
 .../tasks/InterruptSensitiveRestoreTest.java    |  5 +-
 .../runtime/tasks/StreamMockEnvironment.java    |  4 +-
 .../streaming/runtime/tasks/StreamTaskTest.java |  4 +-
 16 files changed, 98 insertions(+), 111 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/b3806349/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 35b639b..a2716e5 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,7 +18,6 @@
 
 package org.apache.flink.runtime.taskexecutor;
 
-import org.apache.flink.configuration.UnmodifiableConfiguration;
 import org.apache.flink.runtime.accumulators.AccumulatorSnapshot;
 import org.apache.flink.runtime.blob.BlobCache;
 import org.apache.flink.runtime.broadcast.BroadcastVariableManager;
@@ -71,7 +70,6 @@ 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.taskmanager.TaskManagerRuntimeInfo;
 import org.apache.flink.util.Preconditions;
 
 import java.util.HashSet;
@@ -127,9 +125,6 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 
 	private final FileCache fileCache;
 
-	// TODO: Try to get rid of it
-	private final TaskManagerRuntimeInfo taskManagerRuntimeInfo;
-
 	// --------- resource manager --------
 
 	private TaskExecutorToResourceManagerConnection resourceManagerConnection;
@@ -177,10 +172,6 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 		this.taskManagerMetricGroup = checkNotNull(taskManagerMetricGroup);
 		this.broadcastVariableManager = checkNotNull(broadcastVariableManager);
 		this.fileCache = checkNotNull(fileCache);
-		this.taskManagerRuntimeInfo = new TaskManagerRuntimeInfo(
-			taskManagerLocation.getHostname(),
-			new UnmodifiableConfiguration(taskManagerConfiguration.getConfiguration()),
-			taskManagerConfiguration.getTmpDirPaths());
 
 		this.jobManagerConnections = new HashMap<>(4);
 
@@ -308,7 +299,7 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 			checkpointResponder,
 			libraryCache,
 			fileCache,
-			taskManagerRuntimeInfo,
+			taskManagerConfiguration,
 			taskMetricGroup,
 			resultPartitionConsumableNotifier,
 			partitionStateChecker,

http://git-wip-us.apache.org/repos/asf/flink/blob/b3806349/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerConfiguration.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerConfiguration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerConfiguration.java
index bce3dc3..1d1e732 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerConfiguration.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerConfiguration.java
@@ -23,6 +23,7 @@ import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.UnmodifiableConfiguration;
 import org.apache.flink.runtime.akka.AkkaUtils;
+import org.apache.flink.runtime.taskmanager.TaskManagerRuntimeInfo;
 import org.apache.flink.util.Preconditions;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -33,13 +34,13 @@ import java.io.File;
 /**
  * Configuration object for {@link TaskExecutor}.
  */
-public class TaskManagerConfiguration {
+public class TaskManagerConfiguration implements TaskManagerRuntimeInfo {
 
 	private static final Logger LOG = LoggerFactory.getLogger(TaskManagerConfiguration.class);
 
 	private final int numberSlots;
 
-	private final String[] tmpDirPaths;
+	private final String[] tmpDirectories;
 
 	private final Time timeout;
 	// null indicates an infinite duration
@@ -50,12 +51,11 @@ public class TaskManagerConfiguration {
 
 	private final long cleanupInterval;
 
-	// TODO: remove necessity for complete configuration object
-	private final Configuration configuration;
+	private final UnmodifiableConfiguration configuration;
 
 	public TaskManagerConfiguration(
 		int numberSlots,
-		String[] tmpDirPaths,
+		String[] tmpDirectories,
 		Time timeout,
 		Time maxRegistrationDuration,
 		Time initialRegistrationPause,
@@ -65,7 +65,7 @@ public class TaskManagerConfiguration {
 		Configuration configuration) {
 
 		this.numberSlots = numberSlots;
-		this.tmpDirPaths = Preconditions.checkNotNull(tmpDirPaths);
+		this.tmpDirectories = Preconditions.checkNotNull(tmpDirectories);
 		this.timeout = Preconditions.checkNotNull(timeout);
 		this.maxRegistrationDuration = maxRegistrationDuration;
 		this.initialRegistrationPause = Preconditions.checkNotNull(initialRegistrationPause);
@@ -79,10 +79,6 @@ public class TaskManagerConfiguration {
 		return numberSlots;
 	}
 
-	public String[] getTmpDirPaths() {
-		return tmpDirPaths;
-	}
-
 	public Time getTimeout() {
 		return timeout;
 	}
@@ -107,10 +103,16 @@ public class TaskManagerConfiguration {
 		return cleanupInterval;
 	}
 
+	@Override
 	public Configuration getConfiguration() {
 		return configuration;
 	}
 
+	@Override
+	public String[] getTmpDirectories() {
+		return tmpDirectories;
+	}
+
 	// --------------------------------------------------------------------------------------------
 	//  Static factory methods
 	// --------------------------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/flink/blob/b3806349/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManagerRuntimeInfo.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManagerRuntimeInfo.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManagerRuntimeInfo.java
index 9ac982e..d1efe34 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManagerRuntimeInfo.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManagerRuntimeInfo.java
@@ -20,71 +20,22 @@ package org.apache.flink.runtime.taskmanager;
 
 import org.apache.flink.configuration.Configuration;
 
-import static org.apache.flink.util.Preconditions.checkNotNull;
-import static org.apache.flink.util.Preconditions.checkArgument;
-
 /**
- * Encapsulation of TaskManager runtime information, like hostname and configuration.
+ * Interface to access {@link TaskManager} information.
  */
-public class TaskManagerRuntimeInfo implements java.io.Serializable {
-
-	private static final long serialVersionUID = 5598219619760274072L;
-	
-	/** host name of the interface that the TaskManager uses to communicate */
-	private final String hostname;
-
-	/** configuration that the TaskManager was started with */
-	private final Configuration configuration;
-
-	/** list of temporary file directories */
-	private final String[] tmpDirectories;
-	
-	/**
-	 * Creates a runtime info.
-	 * 
-	 * @param hostname The host name of the interface that the TaskManager uses to communicate.
-	 * @param configuration The configuration that the TaskManager was started with.
-	 * @param tmpDirectory The temporary file directory.   
-	 */
-	public TaskManagerRuntimeInfo(String hostname, Configuration configuration, String tmpDirectory) {
-		this(hostname, configuration, new String[] { tmpDirectory });
-	}
-	
-	/**
-	 * Creates a runtime info.
-	 * @param hostname The host name of the interface that the TaskManager uses to communicate.
-	 * @param configuration The configuration that the TaskManager was started with.
-	 * @param tmpDirectories The list of temporary file directories.   
-	 */
-	public TaskManagerRuntimeInfo(String hostname, Configuration configuration, String[] tmpDirectories) {
-		checkArgument(tmpDirectories.length > 0);
-		this.hostname = checkNotNull(hostname);
-		this.configuration = checkNotNull(configuration);
-		this.tmpDirectories = tmpDirectories;
-		
-	}
-
-	/**
-	 * Gets host name of the interface that the TaskManager uses to communicate.
-	 * @return The host name of the interface that the TaskManager uses to communicate.
-	 */
-	public String getHostname() {
-		return hostname;
-	}
+public interface TaskManagerRuntimeInfo {
 
 	/**
 	 * Gets the configuration that the TaskManager was started with.
+	 *
 	 * @return The configuration that the TaskManager was started with.
 	 */
-	public Configuration getConfiguration() {
-		return configuration;
-	}
+	Configuration getConfiguration();
 
 	/**
 	 * Gets the list of temporary file directories.
+	 * 
 	 * @return The list of temporary file directories.
 	 */
-	public String[] getTmpDirectories() {
-		return tmpDirectories;
-	}
+	String[] getTmpDirectories();
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/b3806349/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala
index da8c14e..26e13ba 100644
--- a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala
+++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala
@@ -149,7 +149,7 @@ class TaskManager(
   protected val bcVarManager = new BroadcastVariableManager()
 
   /** Handler for distributed files cached by this TaskManager */
-  protected val fileCache = new FileCache(config.getTmpDirPaths())
+  protected val fileCache = new FileCache(config.getTmpDirectories())
 
   /** Registry of metrics periodically transmitted to the JobManager */
   private val metricRegistry = TaskManager.createMetricsRegistry()
@@ -183,11 +183,6 @@ class TaskManager(
 
   var leaderSessionID: Option[UUID] = None
 
-  private val runtimeInfo = new TaskManagerRuntimeInfo(
-       location.getHostname(),
-       new UnmodifiableConfiguration(config.getConfiguration()),
-       config.getTmpDirPaths())
-
   private var scheduledTaskManagerRegistration: Option[Cancellable] = None
   private var currentRegistrationRun: UUID = UUID.randomUUID()
 
@@ -995,7 +990,7 @@ class TaskManager(
     }
     
     taskManagerMetricGroup = 
-      new TaskManagerMetricGroup(metricsRegistry, this.runtimeInfo.getHostname, id.toString)
+      new TaskManagerMetricGroup(metricsRegistry, location.getHostname, id.toString)
     
     TaskExecutorMetricsInitializer.instantiateStatusMetrics(taskManagerMetricGroup, network)
     
@@ -1179,7 +1174,7 @@ class TaskManager(
         checkpointResponder,
         libCache,
         fileCache,
-        runtimeInfo,
+        config,
         taskMetricGroup,
         resultPartitionConsumableNotifier,
         partitionStateChecker,

http://git-wip-us.apache.org/repos/asf/flink/blob/b3806349/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/TestTaskContext.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/TestTaskContext.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/TestTaskContext.java
index 62110a7..d34bb40 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/TestTaskContext.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/TestTaskContext.java
@@ -35,6 +35,7 @@ import org.apache.flink.runtime.operators.TaskContext;
 import org.apache.flink.runtime.operators.testutils.DummyInvokable;
 import org.apache.flink.runtime.operators.util.TaskConfig;
 import org.apache.flink.runtime.taskmanager.TaskManagerRuntimeInfo;
+import org.apache.flink.runtime.util.TestingTaskManagerRuntimeInfo;
 import org.apache.flink.util.Collector;
 import org.apache.flink.util.MutableObjectIterator;
 
@@ -74,8 +75,7 @@ public class TestTaskContext<S, T> implements TaskContext<S, T> {
 	
 	public TestTaskContext(long memoryInBytes) {
 		this.memoryManager = new MemoryManager(memoryInBytes, 1, 32 * 1024, MemoryType.HEAP, true);
-		this.taskManageInfo = new TaskManagerRuntimeInfo(
-				"localhost", new Configuration(), System.getProperty("java.io.tmpdir"));
+		this.taskManageInfo = new TestingTaskManagerRuntimeInfo();
 	}
 	
 	// --------------------------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/flink/blob/b3806349/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/BinaryOperatorTestBase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/BinaryOperatorTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/BinaryOperatorTestBase.java
index 75f960e..3d4c45f 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/BinaryOperatorTestBase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/BinaryOperatorTestBase.java
@@ -38,6 +38,7 @@ import org.apache.flink.runtime.operators.TaskContext;
 import org.apache.flink.runtime.operators.sort.UnilateralSortMerger;
 import org.apache.flink.runtime.operators.util.TaskConfig;
 import org.apache.flink.runtime.taskmanager.TaskManagerRuntimeInfo;
+import org.apache.flink.runtime.util.TestingTaskManagerRuntimeInfo;
 import org.apache.flink.util.Collector;
 import org.apache.flink.util.MutableObjectIterator;
 import org.apache.flink.util.TestLogger;
@@ -110,8 +111,7 @@ public class BinaryOperatorTestBase<S extends Function, IN, OUT> extends TestLog
 		this.owner = new DummyInvokable();
 		this.taskConfig = new TaskConfig(new Configuration());
 		this.executionConfig = executionConfig;
-		this.taskManageInfo = new TaskManagerRuntimeInfo(
-				"localhost", new Configuration(), System.getProperty("java.io.tmpdir"));
+		this.taskManageInfo = new TestingTaskManagerRuntimeInfo();
 	}
 	
 	@Parameterized.Parameters

http://git-wip-us.apache.org/repos/asf/flink/blob/b3806349/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DriverTestBase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DriverTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DriverTestBase.java
index 088435a..f43632c 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DriverTestBase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DriverTestBase.java
@@ -38,6 +38,7 @@ import org.apache.flink.runtime.operators.util.TaskConfig;
 import org.apache.flink.runtime.taskmanager.TaskManagerRuntimeInfo;
 import org.apache.flink.runtime.testutils.recordutils.RecordComparator;
 import org.apache.flink.runtime.testutils.recordutils.RecordSerializerFactory;
+import org.apache.flink.runtime.util.TestingTaskManagerRuntimeInfo;
 import org.apache.flink.types.Record;
 import org.apache.flink.util.Collector;
 import org.apache.flink.util.MutableObjectIterator;
@@ -115,8 +116,7 @@ public class DriverTestBase<S extends Function> extends TestLogger implements Ta
 		this.owner = new DummyInvokable();
 		this.taskConfig = new TaskConfig(new Configuration());
 		this.executionConfig = executionConfig;
-		this.taskManageInfo = new TaskManagerRuntimeInfo(
-				"localhost", new Configuration(), System.getProperty("java.io.tmpdir"));
+		this.taskManageInfo = new TestingTaskManagerRuntimeInfo();
 	}
 
 	@Parameterized.Parameters

http://git-wip-us.apache.org/repos/asf/flink/blob/b3806349/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MockEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MockEnvironment.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MockEnvironment.java
index eb55c4d..ce7fb77 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MockEnvironment.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MockEnvironment.java
@@ -22,7 +22,6 @@ import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.api.common.TaskInfo;
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.configuration.UnmodifiableConfiguration;
 import org.apache.flink.core.fs.Path;
 import org.apache.flink.core.memory.MemorySegmentFactory;
 import org.apache.flink.runtime.accumulators.AccumulatorRegistry;
@@ -48,8 +47,10 @@ import org.apache.flink.runtime.query.KvStateRegistry;
 import org.apache.flink.runtime.query.TaskKvStateRegistry;
 import org.apache.flink.runtime.state.CheckpointStateHandles;
 import org.apache.flink.runtime.taskmanager.TaskManagerRuntimeInfo;
+import org.apache.flink.runtime.util.TestingTaskManagerRuntimeInfo;
 import org.apache.flink.types.Record;
 import org.apache.flink.util.MutableObjectIterator;
+
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 
@@ -233,10 +234,7 @@ public class MockEnvironment implements Environment {
 
 	@Override
 	public TaskManagerRuntimeInfo getTaskManagerInfo() {
-		return new TaskManagerRuntimeInfo(
-				"localhost",
-				new UnmodifiableConfiguration(new Configuration()),
-				System.getProperty("java.io.tmpdir"));
+		return new TestingTaskManagerRuntimeInfo();
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/b3806349/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/UnaryOperatorTestBase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/UnaryOperatorTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/UnaryOperatorTestBase.java
index a94e694..85137cf 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/UnaryOperatorTestBase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/UnaryOperatorTestBase.java
@@ -37,6 +37,7 @@ import org.apache.flink.runtime.operators.ResettableDriver;
 import org.apache.flink.runtime.operators.sort.UnilateralSortMerger;
 import org.apache.flink.runtime.operators.util.TaskConfig;
 import org.apache.flink.runtime.taskmanager.TaskManagerRuntimeInfo;
+import org.apache.flink.runtime.util.TestingTaskManagerRuntimeInfo;
 import org.apache.flink.util.Collector;
 import org.apache.flink.util.MutableObjectIterator;
 
@@ -115,8 +116,7 @@ public class UnaryOperatorTestBase<S extends Function, IN, OUT> extends TestLogg
 		this.executionConfig = executionConfig;
 		this.comparators = new ArrayList<TypeComparator<IN>>(2);
 
-		this.taskManageInfo = new TaskManagerRuntimeInfo(
-				"localhost", new Configuration(), System.getProperty("java.io.tmpdir"));
+		this.taskManageInfo = new TestingTaskManagerRuntimeInfo();
 	}
 
 	@Parameterized.Parameters

http://git-wip-us.apache.org/repos/asf/flink/blob/b3806349/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 f5fe52c..ecbd9b5 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
@@ -63,14 +63,11 @@ public class TaskExecutorTest extends TestLogger {
 			ResourceManagerGateway rmGateway = mock(ResourceManagerGateway.class);
 			TaskManagerConfiguration taskManagerServicesConfiguration = mock(TaskManagerConfiguration.class);
 			PowerMockito.when(taskManagerServicesConfiguration.getNumberSlots()).thenReturn(1);
-			PowerMockito.when(taskManagerServicesConfiguration.getConfiguration()).thenReturn(new Configuration());
-			PowerMockito.when(taskManagerServicesConfiguration.getTmpDirPaths()).thenReturn(new String[1]);
 
 			rpc.registerGateway(resourceManagerAddress, rmGateway);
 
 			TaskManagerLocation taskManagerLocation = mock(TaskManagerLocation.class);
 			when(taskManagerLocation.getResourceID()).thenReturn(resourceID);
-			when(taskManagerLocation.getHostname()).thenReturn("foobar");
 
 			NonHaServices haServices = new NonHaServices(resourceManagerAddress);
 
@@ -124,7 +121,7 @@ public class TaskExecutorTest extends TestLogger {
 			TaskManagerConfiguration taskManagerServicesConfiguration = mock(TaskManagerConfiguration.class);
 			PowerMockito.when(taskManagerServicesConfiguration.getNumberSlots()).thenReturn(1);
 			PowerMockito.when(taskManagerServicesConfiguration.getConfiguration()).thenReturn(new Configuration());
-			PowerMockito.when(taskManagerServicesConfiguration.getTmpDirPaths()).thenReturn(new String[1]);
+			PowerMockito.when(taskManagerServicesConfiguration.getTmpDirectories()).thenReturn(new String[1]);
 
 			TaskManagerLocation taskManagerLocation = mock(TaskManagerLocation.class);
 			when(taskManagerLocation.getResourceID()).thenReturn(resourceID);
@@ -198,12 +195,9 @@ public class TaskExecutorTest extends TestLogger {
 
 			TaskManagerConfiguration taskManagerServicesConfiguration = mock(TaskManagerConfiguration.class);
 			PowerMockito.when(taskManagerServicesConfiguration.getNumberSlots()).thenReturn(1);
-			PowerMockito.when(taskManagerServicesConfiguration.getConfiguration()).thenReturn(new Configuration());
-			PowerMockito.when(taskManagerServicesConfiguration.getTmpDirPaths()).thenReturn(new String[1]);
 
 			TaskManagerLocation taskManagerLocation = mock(TaskManagerLocation.class);
 			when(taskManagerLocation.getResourceID()).thenReturn(resourceID);
-			when(taskManagerLocation.getHostname()).thenReturn("foobar");
 
 			TaskExecutor taskManager = new TaskExecutor(
 				taskManagerServicesConfiguration,

http://git-wip-us.apache.org/repos/asf/flink/blob/b3806349/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskAsyncCallTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskAsyncCallTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskAsyncCallTest.java
index 9a79935..eb7f3c5 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskAsyncCallTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskAsyncCallTest.java
@@ -48,7 +48,9 @@ import org.apache.flink.runtime.state.ChainedStateHandle;
 import org.apache.flink.runtime.state.KeyGroupsStateHandle;
 import org.apache.flink.runtime.state.OperatorStateHandle;
 import org.apache.flink.runtime.state.StreamStateHandle;
+import org.apache.flink.runtime.util.TestingTaskManagerRuntimeInfo;
 import org.apache.flink.util.SerializedValue;
+
 import org.junit.Before;
 import org.junit.Test;
 
@@ -178,7 +180,7 @@ public class TaskAsyncCallTest {
 			mock(CheckpointResponder.class),
 			libCache,
 			mock(FileCache.class),
-			new TaskManagerRuntimeInfo("localhost", new Configuration(), System.getProperty("java.io.tmpdir")),
+			new TestingTaskManagerRuntimeInfo(),
 			mock(TaskMetricGroup.class),
 			consumableNotifier,
 			partitionStateChecker,

http://git-wip-us.apache.org/repos/asf/flink/blob/b3806349/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java
index fe618ff..50fc181 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java
@@ -48,6 +48,7 @@ import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.runtime.messages.TaskMessages;
 import org.apache.flink.runtime.metrics.groups.TaskMetricGroup;
 import org.apache.flink.runtime.query.TaskKvStateRegistry;
+import org.apache.flink.runtime.util.TestingTaskManagerRuntimeInfo;
 import org.apache.flink.util.SerializedValue;
 import org.junit.After;
 import org.junit.Before;
@@ -648,7 +649,7 @@ public class TaskTest {
 			checkpointResponder,
 			libCache,
 			mock(FileCache.class),
-			new TaskManagerRuntimeInfo("localhost", new Configuration(), System.getProperty("java.io.tmpdir")),
+			new TestingTaskManagerRuntimeInfo(),
 			mock(TaskMetricGroup.class),
 			consumableNotifier,
 			partitionStateChecker,

http://git-wip-us.apache.org/repos/asf/flink/blob/b3806349/flink-runtime/src/test/java/org/apache/flink/runtime/util/TestingTaskManagerRuntimeInfo.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/util/TestingTaskManagerRuntimeInfo.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/TestingTaskManagerRuntimeInfo.java
new file mode 100644
index 0000000..e56da97
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/TestingTaskManagerRuntimeInfo.java
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.util;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.taskmanager.TaskManagerRuntimeInfo;
+
+import java.io.File;
+
+/**
+ * TaskManagerRuntimeInfo implementation for testing purposes
+ */
+public class TestingTaskManagerRuntimeInfo implements TaskManagerRuntimeInfo {
+
+	private final Configuration configuration;
+	private final String[] tmpDirectories;
+
+	public TestingTaskManagerRuntimeInfo() {
+		this(new Configuration(), System.getProperty("java.io.tmpdir").split(",|" + File.pathSeparator));
+	}
+
+	public TestingTaskManagerRuntimeInfo(Configuration configuration, String[] tmpDirectories) {
+		this.configuration = configuration;
+		this.tmpDirectories = tmpDirectories;
+	}
+
+	@Override
+	public Configuration getConfiguration() {
+		return configuration;
+	}
+
+	@Override
+	public String[] getTmpDirectories() {
+		return tmpDirectories;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/b3806349/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/InterruptSensitiveRestoreTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/InterruptSensitiveRestoreTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/InterruptSensitiveRestoreTest.java
index ffda126..fb1b3b3 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/InterruptSensitiveRestoreTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/InterruptSensitiveRestoreTest.java
@@ -48,8 +48,8 @@ import org.apache.flink.runtime.state.StreamStateHandle;
 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.TaskManagerRuntimeInfo;
 import org.apache.flink.runtime.util.EnvironmentInformation;
+import org.apache.flink.runtime.util.TestingTaskManagerRuntimeInfo;
 import org.apache.flink.streaming.api.TimeCharacteristic;
 import org.apache.flink.streaming.api.checkpoint.Checkpointed;
 import org.apache.flink.streaming.api.functions.source.SourceFunction;
@@ -167,8 +167,7 @@ public class InterruptSensitiveRestoreTest {
 				mock(CheckpointResponder.class),
 				new FallbackLibraryCacheManager(),
 				new FileCache(tmpDirectories),
-				new TaskManagerRuntimeInfo(
-						"localhost", new Configuration(), EnvironmentInformation.getTemporaryFileDirectory()),
+				new TestingTaskManagerRuntimeInfo(new Configuration(), tmpDirectories),
 				new UnregisteredTaskMetricsGroup(),
 				mock(ResultPartitionConsumableNotifier.class),
 				mock(PartitionStateChecker.class),

http://git-wip-us.apache.org/repos/asf/flink/blob/b3806349/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamMockEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamMockEnvironment.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamMockEnvironment.java
index f852682..2594f71 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamMockEnvironment.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamMockEnvironment.java
@@ -52,6 +52,8 @@ import org.apache.flink.runtime.query.KvStateRegistry;
 import org.apache.flink.runtime.query.TaskKvStateRegistry;
 import org.apache.flink.runtime.state.CheckpointStateHandles;
 import org.apache.flink.runtime.taskmanager.TaskManagerRuntimeInfo;
+import org.apache.flink.runtime.util.TestingTaskManagerRuntimeInfo;
+
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 
@@ -327,7 +329,7 @@ public class StreamMockEnvironment implements Environment {
 
 	@Override
 	public TaskManagerRuntimeInfo getTaskManagerInfo() {
-		return new TaskManagerRuntimeInfo("localhost", new Configuration(), System.getProperty("java.io.tmpdir"));
+		return new TestingTaskManagerRuntimeInfo();
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/b3806349/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java
index 106d3df..9548595 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java
@@ -52,7 +52,7 @@ import org.apache.flink.runtime.taskmanager.Task;
 import org.apache.flink.runtime.taskmanager.TaskExecutionState;
 import org.apache.flink.runtime.taskmanager.TaskExecutionStateListener;
 import org.apache.flink.runtime.taskmanager.TaskManagerActions;
-import org.apache.flink.runtime.taskmanager.TaskManagerRuntimeInfo;
+import org.apache.flink.runtime.util.TestingTaskManagerRuntimeInfo;
 import org.apache.flink.streaming.api.TimeCharacteristic;
 import org.apache.flink.streaming.api.functions.source.SourceFunction;
 import org.apache.flink.streaming.api.graph.StreamConfig;
@@ -243,7 +243,7 @@ public class StreamTaskTest {
 			mock(CheckpointResponder.class),
 			libCache,
 			mock(FileCache.class),
-			new TaskManagerRuntimeInfo("localhost", taskManagerConfig, System.getProperty("java.io.tmpdir")),
+			new TestingTaskManagerRuntimeInfo(taskManagerConfig, new String[] {System.getProperty("java.io.tmpdir")}),
 			new UnregisteredTaskMetricsGroup(),
 			consumableNotifier,
 			partitionStateChecker,


[42/50] [abbrv] flink git commit: [FLINK-4694] [rpc] Add termination futures to RpcEndpoint and RpcService

Posted by se...@apache.org.
[FLINK-4694] [rpc] Add termination futures to RpcEndpoint and RpcService

The termination futures can be used to wait for the termination of the respective component.

This closes #2558.


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

Branch: refs/heads/flip-6
Commit: 9dfaf457fcc282fb01a1ee11950416e6a0b51171
Parents: 5915613
Author: Till Rohrmann <tr...@apache.org>
Authored: Tue Sep 27 18:17:42 2016 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Oct 14 15:14:43 2016 +0200

----------------------------------------------------------------------
 .../concurrent/impl/FlinkCompletableFuture.java | 11 +++---
 .../apache/flink/runtime/rpc/RpcEndpoint.java   |  9 +++++
 .../apache/flink/runtime/rpc/RpcService.java    |  7 ++++
 .../apache/flink/runtime/rpc/SelfGateway.java   | 34 ++++++++++++++++++
 .../runtime/rpc/akka/AkkaInvocationHandler.java | 22 ++++++++++--
 .../flink/runtime/rpc/akka/AkkaRpcActor.java    | 17 ++++++++-
 .../flink/runtime/rpc/akka/AkkaRpcService.java  | 32 +++++++++++++++--
 .../runtime/rpc/TestingSerialRpcService.java    | 10 +++++-
 .../runtime/rpc/akka/AkkaRpcActorTest.java      | 36 ++++++++++++++++++++
 .../runtime/rpc/akka/AkkaRpcServiceTest.java    | 29 ++++++++++++++++
 10 files changed, 193 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/9dfaf457/flink-runtime/src/main/java/org/apache/flink/runtime/concurrent/impl/FlinkCompletableFuture.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/concurrent/impl/FlinkCompletableFuture.java b/flink-runtime/src/main/java/org/apache/flink/runtime/concurrent/impl/FlinkCompletableFuture.java
index e648a71..14686d7 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/concurrent/impl/FlinkCompletableFuture.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/concurrent/impl/FlinkCompletableFuture.java
@@ -20,7 +20,6 @@ package org.apache.flink.runtime.concurrent.impl;
 
 import akka.dispatch.Futures;
 import org.apache.flink.runtime.concurrent.CompletableFuture;
-import org.apache.flink.util.Preconditions;
 import scala.concurrent.Promise;
 import scala.concurrent.Promise$;
 
@@ -52,8 +51,6 @@ public class FlinkCompletableFuture<T> extends FlinkFuture<T> implements Complet
 
 	@Override
 	public boolean complete(T value) {
-		Preconditions.checkNotNull(value);
-
 		try {
 			promise.success(value);
 
@@ -65,10 +62,12 @@ public class FlinkCompletableFuture<T> extends FlinkFuture<T> implements Complet
 
 	@Override
 	public boolean completeExceptionally(Throwable t) {
-		Preconditions.checkNotNull(t);
-
 		try {
-			promise.failure(t);
+			if (t == null) {
+				promise.failure(new NullPointerException("Throwable was null."));
+			} else {
+				promise.failure(t);
+			}
 
 			return true;
 		} catch (IllegalStateException e) {

http://git-wip-us.apache.org/repos/asf/flink/blob/9dfaf457/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 79961f7..f93a2e2 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
@@ -173,6 +173,15 @@ public abstract class RpcEndpoint<C extends RpcGateway> {
 		return rpcService;
 	}
 
+	/**
+	 * Return a future which is completed when the rpc endpoint has been terminated.
+	 *
+	 * @return Future which is completed when the rpc endpoint has been terminated.
+	 */
+	public Future<Void> getTerminationFuture() {
+		return ((SelfGateway)self).getTerminationFuture();
+	}
+
 	// ------------------------------------------------------------------------
 	//  Asynchronous executions
 	// ------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/flink/blob/9dfaf457/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 96844ed..2052f98 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
@@ -77,6 +77,13 @@ public interface RpcService {
 	void stopService();
 
 	/**
+	 * Returns a future indicating when the RPC service has been shut down.
+	 *
+	 * @return Termination future
+	 */
+	Future<Void> getTerminationFuture();
+
+	/**
 	 * Gets the executor, provided by this RPC service. This executor can be used for example for
 	 * the {@code handleAsync(...)} or {@code thenAcceptAsync(...)} methods of futures.
 	 * 

http://git-wip-us.apache.org/repos/asf/flink/blob/9dfaf457/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/SelfGateway.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/SelfGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/SelfGateway.java
new file mode 100644
index 0000000..ed8ef9d
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/SelfGateway.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.rpc;
+
+import org.apache.flink.runtime.concurrent.Future;
+
+/**
+ * Interface for self gateways
+ */
+public interface SelfGateway {
+
+	/**
+	 * Return a future which is completed when the rpc endpoint has been terminated.
+	 *
+	 * @return Future indicating when the rpc endpoint has been terminated
+	 */
+	Future<Void> getTerminationFuture();
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9dfaf457/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaInvocationHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaInvocationHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaInvocationHandler.java
index 8f4deff..709ff92 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaInvocationHandler.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaInvocationHandler.java
@@ -25,6 +25,7 @@ import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.runtime.concurrent.Future;
 import org.apache.flink.runtime.concurrent.impl.FlinkFuture;
 import org.apache.flink.runtime.rpc.MainThreadExecutable;
+import org.apache.flink.runtime.rpc.SelfGateway;
 import org.apache.flink.runtime.rpc.RpcGateway;
 import org.apache.flink.runtime.rpc.RpcTimeout;
 import org.apache.flink.runtime.rpc.StartStoppable;
@@ -52,7 +53,7 @@ import static org.apache.flink.util.Preconditions.checkArgument;
  * rpc in a {@link LocalRpcInvocation} message and then sends it to the {@link AkkaRpcActor} where it is
  * executed.
  */
-class AkkaInvocationHandler implements InvocationHandler, AkkaGateway, MainThreadExecutable, StartStoppable {
+class AkkaInvocationHandler implements InvocationHandler, AkkaGateway, MainThreadExecutable, StartStoppable, SelfGateway {
 	private static final Logger LOG = Logger.getLogger(AkkaInvocationHandler.class);
 
 	private final String address;
@@ -67,12 +68,22 @@ class AkkaInvocationHandler implements InvocationHandler, AkkaGateway, MainThrea
 
 	private final long maximumFramesize;
 
-	AkkaInvocationHandler(String address, ActorRef rpcEndpoint, Time timeout, long maximumFramesize) {
+	// null if gateway; otherwise non-null
+	private final Future<Void> terminationFuture;
+
+	AkkaInvocationHandler(
+			String address,
+			ActorRef rpcEndpoint,
+			Time timeout,
+			long maximumFramesize,
+			Future<Void> terminationFuture) {
+
 		this.address = Preconditions.checkNotNull(address);
 		this.rpcEndpoint = Preconditions.checkNotNull(rpcEndpoint);
 		this.isLocal = this.rpcEndpoint.path().address().hasLocalScope();
 		this.timeout = Preconditions.checkNotNull(timeout);
 		this.maximumFramesize = maximumFramesize;
+		this.terminationFuture = terminationFuture;
 	}
 
 	@Override
@@ -83,7 +94,7 @@ class AkkaInvocationHandler implements InvocationHandler, AkkaGateway, MainThrea
 
 		if (declaringClass.equals(AkkaGateway.class) || declaringClass.equals(MainThreadExecutable.class) ||
 			declaringClass.equals(Object.class) || declaringClass.equals(StartStoppable.class) ||
-			declaringClass.equals(RpcGateway.class)) {
+			declaringClass.equals(RpcGateway.class) || declaringClass.equals(SelfGateway.class)) {
 			result = method.invoke(this, args);
 		} else {
 			String methodName = method.getName();
@@ -300,4 +311,9 @@ class AkkaInvocationHandler implements InvocationHandler, AkkaGateway, MainThrea
 	public String getAddress() {
 		return address;
 	}
+
+	@Override
+	public Future<Void> getTerminationFuture() {
+		return terminationFuture;
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/9dfaf457/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 1b456a7..c21383a 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
@@ -24,6 +24,7 @@ import akka.actor.UntypedActorWithStash;
 import akka.dispatch.Futures;
 import akka.japi.Procedure;
 import akka.pattern.Patterns;
+import org.apache.flink.runtime.concurrent.CompletableFuture;
 import org.apache.flink.runtime.concurrent.Future;
 import org.apache.flink.runtime.concurrent.impl.FlinkFuture;
 import org.apache.flink.runtime.rpc.MainThreadValidatorUtil;
@@ -76,9 +77,23 @@ class AkkaRpcActor<C extends RpcGateway, T extends RpcEndpoint<C>> extends Untyp
 	/** the helper that tracks whether calls come from the main thread */
 	private final MainThreadValidatorUtil mainThreadValidator;
 
-	AkkaRpcActor(final T rpcEndpoint) {
+	private final CompletableFuture<Void> terminationFuture;
+
+	AkkaRpcActor(final T rpcEndpoint, final CompletableFuture<Void> terminationFuture) {
 		this.rpcEndpoint = checkNotNull(rpcEndpoint, "rpc endpoint");
 		this.mainThreadValidator = new MainThreadValidatorUtil(rpcEndpoint);
+		this.terminationFuture = checkNotNull(terminationFuture);
+	}
+
+	@Override
+	public void postStop() {
+		super.postStop();
+
+		// IMPORTANT: This only works if we don't use a restarting supervisor strategy. Otherwise
+		// we would complete the future and let the actor system restart the actor with a completed
+		// future.
+		// Complete the termination future so that others know that we've stopped.
+		terminationFuture.complete(null);
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/9dfaf457/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 fb7896a..44719c8 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
@@ -32,9 +32,12 @@ import akka.dispatch.Mapper;
 import akka.pattern.Patterns;
 import org.apache.flink.api.common.time.Time;
 import org.apache.flink.runtime.akka.AkkaUtils;
+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.concurrent.impl.FlinkFuture;
 import org.apache.flink.runtime.rpc.MainThreadExecutable;
+import org.apache.flink.runtime.rpc.SelfGateway;
 import org.apache.flink.runtime.rpc.RpcGateway;
 import org.apache.flink.runtime.rpc.RpcEndpoint;
 import org.apache.flink.runtime.rpc.RpcService;
@@ -131,7 +134,12 @@ public class AkkaRpcService implements RpcService {
 
 					final String address = AkkaUtils.getAkkaURL(actorSystem, actorRef);
 
-					InvocationHandler akkaInvocationHandler = new AkkaInvocationHandler(address, actorRef, timeout, maximumFramesize);
+					InvocationHandler akkaInvocationHandler = new AkkaInvocationHandler(
+						address,
+						actorRef,
+						timeout,
+						maximumFramesize,
+						null);
 
 					// Rather than using the System ClassLoader directly, we derive the ClassLoader
 					// from this class . That works better in cases where Flink runs embedded and all Flink
@@ -156,7 +164,8 @@ public class AkkaRpcService implements RpcService {
 	public <C extends RpcGateway, S extends RpcEndpoint<C>> C startServer(S rpcEndpoint) {
 		checkNotNull(rpcEndpoint, "rpc endpoint");
 
-		Props akkaRpcActorProps = Props.create(AkkaRpcActor.class, rpcEndpoint);
+		CompletableFuture<Void> terminationFuture = new FlinkCompletableFuture<>();
+		Props akkaRpcActorProps = Props.create(AkkaRpcActor.class, rpcEndpoint, terminationFuture);
 		ActorRef actorRef;
 
 		synchronized (lock) {
@@ -169,7 +178,12 @@ public class AkkaRpcService implements RpcService {
 
 		final String address = AkkaUtils.getAkkaURL(actorSystem, actorRef);
 
-		InvocationHandler akkaInvocationHandler = new AkkaInvocationHandler(address, actorRef, timeout, maximumFramesize);
+		InvocationHandler akkaInvocationHandler = new AkkaInvocationHandler(
+			address,
+			actorRef,
+			timeout,
+			maximumFramesize,
+			terminationFuture);
 
 		// Rather than using the System ClassLoader directly, we derive the ClassLoader
 		// from this class . That works better in cases where Flink runs embedded and all Flink
@@ -181,6 +195,7 @@ public class AkkaRpcService implements RpcService {
 			classLoader,
 			new Class<?>[]{
 				rpcEndpoint.getSelfGatewayType(),
+				SelfGateway.class,
 				MainThreadExecutable.class,
 				StartStoppable.class,
 				AkkaGateway.class},
@@ -231,6 +246,17 @@ public class AkkaRpcService implements RpcService {
 	}
 
 	@Override
+	public Future<Void> getTerminationFuture() {
+		return FlinkFuture.supplyAsync(new Callable<Void>(){
+			@Override
+			public Void call() throws Exception {
+				actorSystem.awaitTermination();
+				return null;
+			}
+		}, getExecutor());
+	}
+
+	@Override
 	public Executor getExecutor() {
 		return actorSystem.dispatcher();
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/9dfaf457/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 2a004c5..88906a7 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
@@ -21,6 +21,7 @@ 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;
 import org.apache.flink.runtime.concurrent.Future;
 import org.apache.flink.runtime.concurrent.impl.FlinkCompletableFuture;
 import org.apache.flink.runtime.util.DirectExecutorService;
@@ -39,7 +40,6 @@ import java.util.concurrent.TimeUnit;
 
 import static org.apache.flink.util.Preconditions.checkNotNull;
 
-
 /**
  * An RPC Service implementation for testing. This RPC service directly executes all asynchronous
  * calls one by one in the calling thread.
@@ -48,10 +48,12 @@ public class TestingSerialRpcService implements RpcService {
 
 	private final DirectExecutorService executorService;
 	private final ConcurrentHashMap<String, RpcGateway> registeredConnections;
+	private final CompletableFuture<Void> terminationFuture;
 
 	public TestingSerialRpcService() {
 		executorService = new DirectExecutorService();
 		this.registeredConnections = new ConcurrentHashMap<>(16);
+		this.terminationFuture = new FlinkCompletableFuture<>();
 	}
 
 	@Override
@@ -89,6 +91,12 @@ public class TestingSerialRpcService implements RpcService {
 	public void stopService() {
 		executorService.shutdown();
 		registeredConnections.clear();
+		terminationFuture.complete(null);
+	}
+
+	@Override
+	public Future<Void> getTerminationFuture() {
+		return terminationFuture;
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/9dfaf457/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 5d76024..ba8eb11 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,8 @@ 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;
 import org.apache.flink.runtime.rpc.RpcMethod;
@@ -32,9 +34,15 @@ import org.hamcrest.core.Is;
 import org.junit.AfterClass;
 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;
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
@@ -148,6 +156,34 @@ public class AkkaRpcActorTest extends TestLogger {
 		}
 	}
 
+	/**
+	 * Tests that we can wait for a RpcEndpoint to terminate.
+	 *
+	 * @throws ExecutionException
+	 * @throws InterruptedException
+	 */
+	@Test(timeout=1000)
+	public void testRpcEndpointTerminationFuture() throws ExecutionException, InterruptedException {
+		final DummyRpcEndpoint rpcEndpoint = new DummyRpcEndpoint(akkaRpcService);
+		rpcEndpoint.start();
+
+		Future<Void> terminationFuture = rpcEndpoint.getTerminationFuture();
+
+		assertFalse(terminationFuture.isDone());
+
+		FlinkFuture.supplyAsync(new Callable<Void>() {
+			@Override
+			public Void call() throws Exception {
+				rpcEndpoint.shutDown();
+
+				return null;
+			}
+		}, actorSystem.dispatcher());
+
+		// wait until the rpc endpoint has terminated
+		terminationFuture.get();
+	}
+
 	private interface DummyRpcGateway extends RpcGateway {
 		Future<Integer> foobar();
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/9dfaf457/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java
index 3388011..7c8defa 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java
@@ -23,6 +23,7 @@ import org.apache.flink.api.common.time.Time;
 import org.apache.flink.core.testutils.OneShotLatch;
 import org.apache.flink.runtime.akka.AkkaUtils;
 import org.apache.flink.runtime.concurrent.Future;
+import org.apache.flink.runtime.concurrent.impl.FlinkFuture;
 import org.apache.flink.util.TestLogger;
 
 import org.junit.AfterClass;
@@ -34,6 +35,7 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
 public class AkkaRpcServiceTest extends TestLogger {
@@ -120,4 +122,31 @@ public class AkkaRpcServiceTest extends TestLogger {
 	public void testGetAddress() {
 		assertEquals(AkkaUtils.getAddress(actorSystem).host().get(), akkaRpcService.getAddress());
 	}
+
+	/**
+	 * Tests that we can wait for the termination of the rpc service
+	 *
+	 * @throws ExecutionException
+	 * @throws InterruptedException
+	 */
+	@Test(timeout = 1000)
+	public void testTerminationFuture() throws ExecutionException, InterruptedException {
+		final ActorSystem actorSystem = AkkaUtils.createDefaultActorSystem();
+		final AkkaRpcService rpcService = new AkkaRpcService(actorSystem, Time.milliseconds(1000));
+
+		Future<Void> terminationFuture = rpcService.getTerminationFuture();
+
+		assertFalse(terminationFuture.isDone());
+
+		FlinkFuture.supplyAsync(new Callable<Void>() {
+			@Override
+			public Void call() throws Exception {
+				rpcService.stopService();
+
+				return null;
+			}
+		}, actorSystem.dispatcher());
+
+		terminationFuture.get();
+	}
 }


[07/50] [abbrv] flink git commit: [FLINK-4529] [flip-6] Move TaskExecutor, JobMaster and ResourceManager out of the rpc package

Posted by se...@apache.org.
[FLINK-4529] [flip-6] Move TaskExecutor, JobMaster and ResourceManager out of the rpc package

The TaskExecutor, the JobMaster and the ResourceManager were still contained in the rpc
package. With this commit, they will be moved out of this package. Now they are contained
in dedicated packages on the o.a.f.runtime level.

This closes #2438.


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

Branch: refs/heads/flip-6
Commit: 9c247d1ffd732d101145ef3ef6e8050151128f5c
Parents: 4b077af
Author: Till Rohrmann <tr...@apache.org>
Authored: Mon Aug 29 16:35:29 2016 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Oct 14 15:14:39 2016 +0200

----------------------------------------------------------------------
 .../runtime/clusterframework/SlotManager.java   | 525 ------------
 .../flink/runtime/jobmaster/JobMaster.java      | 244 ++++++
 .../runtime/jobmaster/JobMasterGateway.java     |  45 +
 .../registration/RegistrationResponse.java      |  84 ++
 .../registration/RetryingRegistration.java      | 296 +++++++
 .../resourcemanager/JobMasterRegistration.java  |  35 +
 .../resourcemanager/RegistrationResponse.java   |  43 +
 .../resourcemanager/ResourceManager.java        | 214 +++++
 .../resourcemanager/ResourceManagerGateway.java |  77 ++
 .../runtime/resourcemanager/SlotAssignment.java |  25 +
 .../runtime/resourcemanager/SlotManager.java    | 523 ++++++++++++
 .../runtime/resourcemanager/SlotRequest.java    |  74 ++
 .../flink/runtime/rpc/jobmaster/JobMaster.java  | 244 ------
 .../runtime/rpc/jobmaster/JobMasterGateway.java |  45 -
 .../rpc/registration/RegistrationResponse.java  |  84 --
 .../rpc/registration/RetryingRegistration.java  | 296 -------
 .../resourcemanager/JobMasterRegistration.java  |  35 -
 .../resourcemanager/RegistrationResponse.java   |  43 -
 .../rpc/resourcemanager/ResourceManager.java    | 214 -----
 .../resourcemanager/ResourceManagerGateway.java |  77 --
 .../rpc/resourcemanager/SlotAssignment.java     |  25 -
 .../rpc/resourcemanager/SlotRequest.java        |  74 --
 .../runtime/rpc/taskexecutor/SlotReport.java    |  56 --
 .../runtime/rpc/taskexecutor/SlotStatus.java    | 129 ---
 .../runtime/rpc/taskexecutor/TaskExecutor.java  | 827 -------------------
 .../taskexecutor/TaskExecutorConfiguration.java | 151 ----
 .../rpc/taskexecutor/TaskExecutorGateway.java   |  35 -
 .../TaskExecutorRegistrationSuccess.java        |  75 --
 ...TaskExecutorToResourceManagerConnection.java | 198 -----
 .../flink/runtime/taskexecutor/SlotReport.java  |  56 ++
 .../flink/runtime/taskexecutor/SlotStatus.java  | 129 +++
 .../runtime/taskexecutor/TaskExecutor.java      | 827 +++++++++++++++++++
 .../taskexecutor/TaskExecutorConfiguration.java | 151 ++++
 .../taskexecutor/TaskExecutorGateway.java       |  35 +
 .../TaskExecutorRegistrationSuccess.java        |  75 ++
 ...TaskExecutorToResourceManagerConnection.java | 198 +++++
 .../clusterframework/ClusterShutdownITCase.java | 156 ++++
 .../clusterframework/ResourceManagerITCase.java | 162 ++++
 .../clusterframework/ResourceManagerTest.java   | 338 ++++++++
 .../clusterframework/SlotManagerTest.java       | 540 ------------
 .../registration/RetryingRegistrationTest.java  | 336 ++++++++
 .../registration/TestRegistrationGateway.java   |  85 ++
 .../resourcemanager/ClusterShutdownITCase.java  | 156 ----
 .../resourcemanager/ResourceManagerHATest.java  |  76 ++
 .../resourcemanager/ResourceManagerITCase.java  | 162 ----
 .../resourcemanager/ResourceManagerTest.java    | 338 --------
 .../resourcemanager/SlotManagerTest.java        | 538 ++++++++++++
 .../runtime/rpc/akka/AkkaRpcServiceTest.java    |  14 -
 .../registration/RetryingRegistrationTest.java  | 336 --------
 .../registration/TestRegistrationGateway.java   |  85 --
 .../resourcemanager/ResourceManagerHATest.java  |  76 --
 .../rpc/taskexecutor/TaskExecutorTest.java      | 117 ---
 .../runtime/taskexecutor/TaskExecutorTest.java  | 117 +++
 53 files changed, 4939 insertions(+), 4957 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/9c247d1f/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/SlotManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/SlotManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/SlotManager.java
deleted file mode 100644
index cc140a1..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/SlotManager.java
+++ /dev/null
@@ -1,525 +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.clusterframework;
-
-import org.apache.flink.annotation.VisibleForTesting;
-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.ResourceSlot;
-import org.apache.flink.runtime.clusterframework.types.SlotID;
-import org.apache.flink.runtime.rpc.resourcemanager.ResourceManagerGateway;
-import org.apache.flink.runtime.rpc.resourcemanager.SlotRequest;
-import org.apache.flink.runtime.rpc.taskexecutor.SlotReport;
-import org.apache.flink.runtime.rpc.taskexecutor.SlotStatus;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.HashMap;
-import java.util.LinkedHashMap;
-import java.util.Map;
-
-import static org.apache.flink.util.Preconditions.checkNotNull;
-
-/**
- * SlotManager is responsible for receiving slot requests and do slot allocations. It allows to request
- * slots from registered TaskManagers and issues container allocation requests in case of there are not
- * enough available slots. Besides, it should sync its slot allocation with TaskManager's heartbeat.
- * <p>
- * The main operation principle of SlotManager is:
- * <ul>
- * <li>1. All slot allocation status should be synced with TaskManager, which is the ground truth.</li>
- * <li>2. All slots that have registered must be tracked, either by free pool or allocated pool.</li>
- * <li>3. All slot requests will be handled by best efforts, there is no guarantee that one request will be
- * fulfilled in time or correctly allocated. Conflicts or timeout or some special error will happen, it should
- * be handled outside SlotManager. SlotManager will make each decision based on the information it currently
- * holds.</li>
- * </ul>
- * <b>IMPORTANT:</b> This class is <b>Not Thread-safe</b>.
- */
-public abstract class SlotManager {
-
-	private static final Logger LOG = LoggerFactory.getLogger(SlotManager.class);
-
-	/** Gateway to communicate with ResourceManager */
-	private final ResourceManagerGateway resourceManagerGateway;
-
-	/** All registered slots, including free and allocated slots */
-	private final Map<ResourceID, Map<SlotID, ResourceSlot>> registeredSlots;
-
-	/** All pending slot requests, waiting available slots to fulfil */
-	private final Map<AllocationID, SlotRequest> pendingSlotRequests;
-
-	/** All free slots that can be used to be allocated */
-	private final Map<SlotID, ResourceSlot> freeSlots;
-
-	/** All allocations, we can lookup allocations either by SlotID or AllocationID */
-	private final AllocationMap allocationMap;
-
-	public SlotManager(ResourceManagerGateway resourceManagerGateway) {
-		this.resourceManagerGateway = checkNotNull(resourceManagerGateway);
-		this.registeredSlots = new HashMap<>(16);
-		this.pendingSlotRequests = new LinkedHashMap<>(16);
-		this.freeSlots = new HashMap<>(16);
-		this.allocationMap = new AllocationMap();
-	}
-
-	// ------------------------------------------------------------------------
-	//  slot managements
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Request a slot with requirements, we may either fulfill the request or pending it. Trigger container
-	 * allocation if we don't have enough resource. If we have free slot which can match the request, record
-	 * this allocation and forward the request to TaskManager through ResourceManager (we want this done by
-	 * RPC's main thread to avoid race condition).
-	 *
-	 * @param request The detailed request of the slot
-	 */
-	public void requestSlot(final SlotRequest request) {
-		if (isRequestDuplicated(request)) {
-			LOG.warn("Duplicated slot request, AllocationID:{}", request.getAllocationId());
-			return;
-		}
-
-		// try to fulfil the request with current free slots
-		ResourceSlot slot = chooseSlotToUse(request, freeSlots);
-		if (slot != null) {
-			LOG.info("Assigning SlotID({}) to AllocationID({}), JobID:{}", slot.getSlotId(),
-				request.getAllocationId(), request.getJobId());
-
-			// record this allocation in bookkeeping
-			allocationMap.addAllocation(slot.getSlotId(), request.getAllocationId());
-
-			// remove selected slot from free pool
-			freeSlots.remove(slot.getSlotId());
-
-			// TODO: send slot request to TaskManager
-		} else {
-			LOG.info("Cannot fulfil slot request, try to allocate a new container for it, " +
-				"AllocationID:{}, JobID:{}", request.getAllocationId(), request.getJobId());
-			allocateContainer(request.getResourceProfile());
-			pendingSlotRequests.put(request.getAllocationId(), request);
-		}
-	}
-
-	/**
-	 * Sync slot status with TaskManager's SlotReport.
-	 */
-	public void updateSlotStatus(final SlotReport slotReport) {
-		for (SlotStatus slotStatus : slotReport.getSlotsStatus()) {
-			updateSlotStatus(slotStatus);
-		}
-	}
-
-	/**
-	 * The slot request to TaskManager may be either failed by rpc communication (timeout, network error, etc.)
-	 * or really rejected by TaskManager. We shall retry this request by:
-	 * <ul>
-	 * <li>1. verify and clear all the previous allocate information for this request
-	 * <li>2. try to request slot again
-	 * </ul>
-	 * <p>
-	 * This may cause some duplicate allocation, e.g. the slot request to TaskManager is successful but the response
-	 * is lost somehow, so we may request a slot in another TaskManager, this causes two slots assigned to one request,
-	 * but it can be taken care of by rejecting registration at JobManager.
-	 *
-	 * @param originalRequest The original slot request
-	 * @param slotId          The target SlotID
-	 */
-	public void handleSlotRequestFailedAtTaskManager(final SlotRequest originalRequest, final SlotID slotId) {
-		final AllocationID originalAllocationId = originalRequest.getAllocationId();
-		LOG.info("Slot request failed at TaskManager, SlotID:{}, AllocationID:{}, JobID:{}",
-			slotId, originalAllocationId, originalRequest.getJobId());
-
-		// verify the allocation info before we do anything
-		if (freeSlots.containsKey(slotId)) {
-			// this slot is currently empty, no need to de-allocate it from our allocations
-			LOG.info("Original slot is somehow empty, retrying this request");
-
-			// before retry, we should double check whether this request was allocated by some other ways
-			if (!allocationMap.isAllocated(originalAllocationId)) {
-				requestSlot(originalRequest);
-			} else {
-				LOG.info("The failed request has somehow been allocated, SlotID:{}",
-					allocationMap.getSlotID(originalAllocationId));
-			}
-		} else if (allocationMap.isAllocated(slotId)) {
-			final AllocationID currentAllocationId = allocationMap.getAllocationID(slotId);
-
-			// check whether we have an agreement on whom this slot belongs to
-			if (originalAllocationId.equals(currentAllocationId)) {
-				LOG.info("De-allocate this request and retry");
-				allocationMap.removeAllocation(currentAllocationId);
-
-				// put this slot back to free pool
-				ResourceSlot slot = checkNotNull(getRegisteredSlot(slotId));
-				freeSlots.put(slotId, slot);
-
-				// retry the request
-				requestSlot(originalRequest);
-			} else {
-				// the slot is taken by someone else, no need to de-allocate it from our allocations
-				LOG.info("Original slot is taken by someone else, current AllocationID:{}", currentAllocationId);
-
-				// before retry, we should double check whether this request was allocated by some other ways
-				if (!allocationMap.isAllocated(originalAllocationId)) {
-					requestSlot(originalRequest);
-				} else {
-					LOG.info("The failed request is somehow been allocated, SlotID:{}",
-						allocationMap.getSlotID(originalAllocationId));
-				}
-			}
-		} else {
-			LOG.error("BUG! {} is neither in free pool nor in allocated pool", slotId);
-		}
-	}
-
-	/**
-	 * Callback for TaskManager failures. In case that a TaskManager fails, we have to clean up all its slots.
-	 *
-	 * @param resourceId The ResourceID of the TaskManager
-	 */
-	public void notifyTaskManagerFailure(final ResourceID resourceId) {
-		LOG.info("Resource:{} been notified failure", resourceId);
-		final Map<SlotID, ResourceSlot> slotIdsToRemove = registeredSlots.remove(resourceId);
-		if (slotIdsToRemove != null) {
-			for (SlotID slotId : slotIdsToRemove.keySet()) {
-				LOG.info("Removing Slot:{} upon resource failure", slotId);
-				if (freeSlots.containsKey(slotId)) {
-					freeSlots.remove(slotId);
-				} else if (allocationMap.isAllocated(slotId)) {
-					allocationMap.removeAllocation(slotId);
-				} else {
-					LOG.error("BUG! {} is neither in free pool nor in allocated pool", slotId);
-				}
-			}
-		}
-	}
-
-	// ------------------------------------------------------------------------
-	//  internal behaviors
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Update slot status based on TaskManager's report. There are mainly two situations when we receive the report:
-	 * <ul>
-	 * <li>1. The slot is newly registered.</li>
-	 * <li>2. The slot has registered, it contains its current status.</li>
-	 * </ul>
-	 * <p>
-	 * Regarding 1: It's fairly simple, we just record this slot's status, and trigger schedule if slot is empty.
-	 * <p>
-	 * Regarding 2: It will cause some weird situation since we may have some time-gap on how the slot's status really
-	 * is. We may have some updates on the slot's allocation, but it doesn't reflected by TaskManager's heartbeat yet,
-	 * and we may make some wrong decision if we cannot guarantee we have the exact status about all the slots. So
-	 * the principle here is: We always trust TaskManager's heartbeat, we will correct our information based on that
-	 * and take next action based on the diff between our information and heartbeat status.
-	 *
-	 * @param reportedStatus Reported slot status
-	 */
-	void updateSlotStatus(final SlotStatus reportedStatus) {
-		final SlotID slotId = reportedStatus.getSlotID();
-		final ResourceSlot slot = new ResourceSlot(slotId, reportedStatus.getProfiler());
-
-		if (registerNewSlot(slot)) {
-			// we have a newly registered slot
-			LOG.info("New slot appeared, SlotID:{}, AllocationID:{}", slotId, reportedStatus.getAllocationID());
-
-			if (reportedStatus.getAllocationID() != null) {
-				// slot in use, record this in bookkeeping
-				allocationMap.addAllocation(slotId, reportedStatus.getAllocationID());
-			} else {
-				handleFreeSlot(new ResourceSlot(slotId, reportedStatus.getProfiler()));
-			}
-		} else {
-			// slot exists, update current information
-			if (reportedStatus.getAllocationID() != null) {
-				// slot is reported in use
-				final AllocationID reportedAllocationId = reportedStatus.getAllocationID();
-
-				// check whether we also thought this slot is in use
-				if (allocationMap.isAllocated(slotId)) {
-					// we also think that slot is in use, check whether the AllocationID matches
-					final AllocationID currentAllocationId = allocationMap.getAllocationID(slotId);
-
-					if (!reportedAllocationId.equals(currentAllocationId)) {
-						LOG.info("Slot allocation info mismatch! SlotID:{}, current:{}, reported:{}",
-							slotId, currentAllocationId, reportedAllocationId);
-
-						// seems we have a disagreement about the slot assignments, need to correct it
-						allocationMap.removeAllocation(slotId);
-						allocationMap.addAllocation(slotId, reportedAllocationId);
-					}
-				} else {
-					LOG.info("Slot allocation info mismatch! SlotID:{}, current:null, reported:{}",
-						slotId, reportedAllocationId);
-
-					// we thought the slot is free, should correct this information
-					allocationMap.addAllocation(slotId, reportedStatus.getAllocationID());
-
-					// remove this slot from free slots pool
-					freeSlots.remove(slotId);
-				}
-			} else {
-				// slot is reported empty
-
-				// check whether we also thought this slot is empty
-				if (allocationMap.isAllocated(slotId)) {
-					LOG.info("Slot allocation info mismatch! SlotID:{}, current:{}, reported:null",
-						slotId, allocationMap.getAllocationID(slotId));
-
-					// we thought the slot is in use, correct it
-					allocationMap.removeAllocation(slotId);
-
-					// we have a free slot!
-					handleFreeSlot(new ResourceSlot(slotId, reportedStatus.getProfiler()));
-				}
-			}
-		}
-	}
-
-	/**
-	 * When we have a free slot, try to fulfill the pending request first. If any request can be fulfilled,
-	 * record this allocation in bookkeeping and send slot request to TaskManager, else we just add this slot
-	 * to the free pool.
-	 *
-	 * @param freeSlot The free slot
-	 */
-	private void handleFreeSlot(final ResourceSlot freeSlot) {
-		SlotRequest chosenRequest = chooseRequestToFulfill(freeSlot, pendingSlotRequests);
-
-		if (chosenRequest != null) {
-			pendingSlotRequests.remove(chosenRequest.getAllocationId());
-
-			LOG.info("Assigning SlotID({}) to AllocationID({}), JobID:{}", freeSlot.getSlotId(),
-				chosenRequest.getAllocationId(), chosenRequest.getJobId());
-			allocationMap.addAllocation(freeSlot.getSlotId(), chosenRequest.getAllocationId());
-
-			// TODO: send slot request to TaskManager
-		} else {
-			freeSlots.put(freeSlot.getSlotId(), freeSlot);
-		}
-	}
-
-	/**
-	 * Check whether the request is duplicated. We use AllocationID to identify slot request, for each
-	 * formerly received slot request, it is either in pending list or already been allocated.
-	 *
-	 * @param request The slot request
-	 * @return <tt>true</tt> if the request is duplicated
-	 */
-	private boolean isRequestDuplicated(final SlotRequest request) {
-		final AllocationID allocationId = request.getAllocationId();
-		return pendingSlotRequests.containsKey(allocationId)
-			|| allocationMap.isAllocated(allocationId);
-	}
-
-	/**
-	 * Try to register slot, and tell if this slot is newly registered.
-	 *
-	 * @param slot The ResourceSlot which will be checked and registered
-	 * @return <tt>true</tt> if we meet a new slot
-	 */
-	private boolean registerNewSlot(final ResourceSlot slot) {
-		final SlotID slotId = slot.getSlotId();
-		final ResourceID resourceId = slotId.getResourceID();
-		if (!registeredSlots.containsKey(resourceId)) {
-			registeredSlots.put(resourceId, new HashMap<SlotID, ResourceSlot>());
-		}
-		return registeredSlots.get(resourceId).put(slotId, slot) == null;
-	}
-
-	private ResourceSlot getRegisteredSlot(final SlotID slotId) {
-		final ResourceID resourceId = slotId.getResourceID();
-		if (!registeredSlots.containsKey(resourceId)) {
-			return null;
-		}
-		return registeredSlots.get(resourceId).get(slotId);
-	}
-
-	// ------------------------------------------------------------------------
-	//  Framework specific behavior
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Choose a slot to use among all free slots, the behavior is framework specified.
-	 *
-	 * @param request   The slot request
-	 * @param freeSlots All slots which can be used
-	 * @return The slot we choose to use, <tt>null</tt> if we did not find a match
-	 */
-	protected abstract ResourceSlot chooseSlotToUse(final SlotRequest request,
-		final Map<SlotID, ResourceSlot> freeSlots);
-
-	/**
-	 * Choose a pending request to fulfill when we have a free slot, the behavior is framework specified.
-	 *
-	 * @param offeredSlot     The free slot
-	 * @param pendingRequests All the pending slot requests
-	 * @return The chosen SlotRequest, <tt>null</tt> if we did not find a match
-	 */
-	protected abstract SlotRequest chooseRequestToFulfill(final ResourceSlot offeredSlot,
-		final Map<AllocationID, SlotRequest> pendingRequests);
-
-	/**
-	 * The framework specific code for allocating a container for specified resource profile.
-	 *
-	 * @param resourceProfile The resource profile
-	 */
-	protected abstract void allocateContainer(final ResourceProfile resourceProfile);
-
-
-	// ------------------------------------------------------------------------
-	//  Helper classes
-	// ------------------------------------------------------------------------
-
-	/**
-	 * We maintain all the allocations with SlotID and AllocationID. We are able to get or remove the allocation info
-	 * either by SlotID or AllocationID.
-	 */
-	private static class AllocationMap {
-
-		/** All allocated slots (by SlotID) */
-		private final Map<SlotID, AllocationID> allocatedSlots;
-
-		/** All allocated slots (by AllocationID), it'a a inverse view of allocatedSlots */
-		private final Map<AllocationID, SlotID> allocatedSlotsByAllocationId;
-
-		AllocationMap() {
-			this.allocatedSlots = new HashMap<>(16);
-			this.allocatedSlotsByAllocationId = new HashMap<>(16);
-		}
-
-		/**
-		 * Add a allocation
-		 *
-		 * @param slotId       The slot id
-		 * @param allocationId The allocation id
-		 */
-		void addAllocation(final SlotID slotId, final AllocationID allocationId) {
-			allocatedSlots.put(slotId, allocationId);
-			allocatedSlotsByAllocationId.put(allocationId, slotId);
-		}
-
-		/**
-		 * De-allocation with slot id
-		 *
-		 * @param slotId The slot id
-		 */
-		void removeAllocation(final SlotID slotId) {
-			if (allocatedSlots.containsKey(slotId)) {
-				final AllocationID allocationId = allocatedSlots.get(slotId);
-				allocatedSlots.remove(slotId);
-				allocatedSlotsByAllocationId.remove(allocationId);
-			}
-		}
-
-		/**
-		 * De-allocation with allocation id
-		 *
-		 * @param allocationId The allocation id
-		 */
-		void removeAllocation(final AllocationID allocationId) {
-			if (allocatedSlotsByAllocationId.containsKey(allocationId)) {
-				SlotID slotId = allocatedSlotsByAllocationId.get(allocationId);
-				allocatedSlotsByAllocationId.remove(allocationId);
-				allocatedSlots.remove(slotId);
-			}
-		}
-
-		/**
-		 * Check whether allocation exists by slot id
-		 *
-		 * @param slotId The slot id
-		 * @return true if the allocation exists
-		 */
-		boolean isAllocated(final SlotID slotId) {
-			return allocatedSlots.containsKey(slotId);
-		}
-
-		/**
-		 * Check whether allocation exists by allocation id
-		 *
-		 * @param allocationId The allocation id
-		 * @return true if the allocation exists
-		 */
-		boolean isAllocated(final AllocationID allocationId) {
-			return allocatedSlotsByAllocationId.containsKey(allocationId);
-		}
-
-		AllocationID getAllocationID(final SlotID slotId) {
-			return allocatedSlots.get(slotId);
-		}
-
-		SlotID getSlotID(final AllocationID allocationId) {
-			return allocatedSlotsByAllocationId.get(allocationId);
-		}
-
-		public int size() {
-			return allocatedSlots.size();
-		}
-	}
-
-	// ------------------------------------------------------------------------
-	//  Testing utilities
-	// ------------------------------------------------------------------------
-
-	@VisibleForTesting
-	boolean isAllocated(final SlotID slotId) {
-		return allocationMap.isAllocated(slotId);
-	}
-
-	@VisibleForTesting
-	boolean isAllocated(final AllocationID allocationId) {
-		return allocationMap.isAllocated(allocationId);
-	}
-
-	/**
-	 * Add free slots directly to the free pool, this will not trigger pending requests allocation
-	 *
-	 * @param slot The resource slot
-	 */
-	@VisibleForTesting
-	void addFreeSlot(final ResourceSlot slot) {
-		final ResourceID resourceId = slot.getResourceID();
-		final SlotID slotId = slot.getSlotId();
-
-		if (!registeredSlots.containsKey(resourceId)) {
-			registeredSlots.put(resourceId, new HashMap<SlotID, ResourceSlot>());
-		}
-		registeredSlots.get(resourceId).put(slot.getSlotId(), slot);
-		freeSlots.put(slotId, slot);
-	}
-
-	@VisibleForTesting
-	int getAllocatedSlotCount() {
-		return allocationMap.size();
-	}
-
-	@VisibleForTesting
-	int getFreeSlotCount() {
-		return freeSlots.size();
-	}
-
-	@VisibleForTesting
-	int getPendingRequestCount() {
-		return pendingSlotRequests.size();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9c247d1f/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
new file mode 100644
index 0000000..0a6a7ef
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java
@@ -0,0 +1,244 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.jobmaster;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.leaderelection.LeaderContender;
+import org.apache.flink.runtime.leaderelection.LeaderElectionService;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.rpc.RpcMethod;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway;
+import org.apache.flink.runtime.rpc.RpcEndpoint;
+import org.apache.flink.runtime.rpc.RpcService;
+import org.apache.flink.runtime.taskmanager.TaskExecutionState;
+import org.apache.flink.util.Preconditions;
+
+import java.util.UUID;
+
+/**
+ * JobMaster implementation. The job master is responsible for the execution of a single
+ * {@link org.apache.flink.runtime.jobgraph.JobGraph}.
+ * <p>
+ * 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
+ * given task</li>
+ * </ul>
+ */
+public class JobMaster extends RpcEndpoint<JobMasterGateway> {
+
+	/** Gateway to connected resource manager, null iff not connected */
+	private ResourceManagerGateway resourceManager = null;
+
+	/** Logical representation of the job */
+	private final JobGraph jobGraph;
+	private final JobID jobID;
+
+	/** Configuration of the job */
+	private final Configuration configuration;
+
+	/** Service to contend for and retrieve the leadership of JM and RM */
+	private final HighAvailabilityServices highAvailabilityServices;
+
+	/** Leader Management */
+	private LeaderElectionService leaderElectionService = null;
+	private UUID leaderSessionID;
+
+	/**
+	 * The JM's Constructor
+	 *
+	 * @param jobGraph The representation of the job's execution plan
+	 * @param configuration The job's configuration
+	 * @param rpcService The RPC service at which the JM serves
+	 * @param highAvailabilityService The cluster's HA service from the JM can elect and retrieve leaders.
+	 */
+	public JobMaster(
+		JobGraph jobGraph,
+		Configuration configuration,
+		RpcService rpcService,
+		HighAvailabilityServices highAvailabilityService) {
+
+		super(rpcService);
+
+		this.jobGraph = Preconditions.checkNotNull(jobGraph);
+		this.jobID = Preconditions.checkNotNull(jobGraph.getJobID());
+
+		this.configuration = Preconditions.checkNotNull(configuration);
+
+		this.highAvailabilityServices = Preconditions.checkNotNull(highAvailabilityService);
+	}
+
+	public ResourceManagerGateway getResourceManager() {
+		return resourceManager;
+	}
+
+	//----------------------------------------------------------------------------------------------
+	// Initialization methods
+	//----------------------------------------------------------------------------------------------
+	public void start() {
+		super.start();
+
+		// register at the election once the JM starts
+		registerAtElectionService();
+	}
+
+
+	//----------------------------------------------------------------------------------------------
+	// JobMaster Leadership methods
+	//----------------------------------------------------------------------------------------------
+
+	/**
+	 * Retrieves the election service and contend for the leadership.
+	 */
+	private void registerAtElectionService() {
+		try {
+			leaderElectionService = highAvailabilityServices.getJobMasterLeaderElectionService(jobID);
+			leaderElectionService.start(new JobMasterLeaderContender());
+		} catch (Exception e) {
+			throw new RuntimeException("Fail to register at the election of JobMaster", e);
+		}
+	}
+
+	/**
+	 * Start the execution when the leadership is granted.
+	 *
+	 * @param newLeaderSessionID The identifier of the new leadership session
+	 */
+	public void grantJobMasterLeadership(final UUID newLeaderSessionID) {
+		runAsync(new Runnable() {
+			@Override
+			public void run() {
+				log.info("JobManager {} grants leadership with session id {}.", getAddress(), newLeaderSessionID);
+
+				// The operation may be blocking, but since JM is idle before it grants the leadership, it's okay that
+				// JM waits here for the operation's completeness.
+				leaderSessionID = newLeaderSessionID;
+				leaderElectionService.confirmLeaderSessionID(newLeaderSessionID);
+
+				// TODO:: execute the job when the leadership is granted.
+			}
+		});
+	}
+
+	/**
+	 * Stop the execution when the leadership is revoked.
+	 */
+	public void revokeJobMasterLeadership() {
+		runAsync(new Runnable() {
+			@Override
+			public void run() {
+				log.info("JobManager {} was revoked leadership.", getAddress());
+
+				// TODO:: cancel the job's execution and notify all listeners
+				cancelAndClearEverything(new Exception("JobManager is no longer the leader."));
+
+				leaderSessionID = null;
+			}
+		});
+	}
+
+	/**
+	 * Handles error occurring in the leader election service
+	 *
+	 * @param exception Exception thrown in the leader election service
+	 */
+	public void onJobMasterElectionError(final Exception exception) {
+		runAsync(new Runnable() {
+			@Override
+			public void run() {
+				log.error("Received an error from the LeaderElectionService.", exception);
+
+				// TODO:: cancel the job's execution and shutdown the JM
+				cancelAndClearEverything(exception);
+
+				leaderSessionID = null;
+			}
+		});
+
+	}
+
+	//----------------------------------------------------------------------------------------------
+	// RPC methods
+	//----------------------------------------------------------------------------------------------
+
+	/**
+	 * Updates the task execution state for a given task.
+	 *
+	 * @param taskExecutionState New task execution state for a given task
+	 * @return Acknowledge the task execution state update
+	 */
+	@RpcMethod
+	public Acknowledge updateTaskExecutionState(TaskExecutionState taskExecutionState) {
+		System.out.println("TaskExecutionState: " + taskExecutionState);
+		return Acknowledge.get();
+	}
+
+	/**
+	 * Triggers the registration of the job master at the resource manager.
+	 *
+	 * @param address Address of the resource manager
+	 */
+	@RpcMethod
+	public void registerAtResourceManager(final String address) {
+		//TODO:: register at the RM
+	}
+
+	//----------------------------------------------------------------------------------------------
+	// Helper methods
+	//----------------------------------------------------------------------------------------------
+
+	/**
+	 * Cancel the current job and notify all listeners the job's cancellation.
+	 *
+	 * @param cause Cause for the cancelling.
+	 */
+	private void cancelAndClearEverything(Throwable cause) {
+		// currently, nothing to do here
+	}
+
+	// ------------------------------------------------------------------------
+	//  Utility classes
+	// ------------------------------------------------------------------------
+	private class JobMasterLeaderContender implements LeaderContender {
+
+		@Override
+		public void grantLeadership(UUID leaderSessionID) {
+			JobMaster.this.grantJobMasterLeadership(leaderSessionID);
+		}
+
+		@Override
+		public void revokeLeadership() {
+			JobMaster.this.revokeJobMasterLeadership();
+		}
+
+		@Override
+		public String getAddress() {
+			return JobMaster.this.getAddress();
+		}
+
+		@Override
+		public void handleError(Exception exception) {
+			onJobMasterElectionError(exception);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9c247d1f/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
new file mode 100644
index 0000000..a53e383
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterGateway.java
@@ -0,0 +1,45 @@
+/*
+ * 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.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.rpc.RpcGateway;
+import org.apache.flink.runtime.taskmanager.TaskExecutionState;
+import scala.concurrent.Future;
+
+/**
+ * {@link JobMaster} rpc gateway interface
+ */
+public interface JobMasterGateway extends RpcGateway {
+
+	/**
+	 * Updates the task execution state for a given task.
+	 *
+	 * @param taskExecutionState New task execution state for a given task
+	 * @return Future acknowledge of the task execution state update
+	 */
+	Future<Acknowledge> updateTaskExecutionState(TaskExecutionState taskExecutionState);
+
+	/**
+	 * Triggers the registration of the job master at the resource manager.
+	 *
+	 * @param address Address of the resource manager
+	 */
+	void registerAtResourceManager(final String address);
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9c247d1f/flink-runtime/src/main/java/org/apache/flink/runtime/registration/RegistrationResponse.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/registration/RegistrationResponse.java b/flink-runtime/src/main/java/org/apache/flink/runtime/registration/RegistrationResponse.java
new file mode 100644
index 0000000..fefcc78
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/registration/RegistrationResponse.java
@@ -0,0 +1,84 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.registration;
+
+import java.io.Serializable;
+
+/**
+ * Base class for responses given to registration attempts from {@link RetryingRegistration}.
+ */
+public abstract class RegistrationResponse implements Serializable {
+
+	private static final long serialVersionUID = 1L;
+
+	// ----------------------------------------------------------------------------
+	
+	/**
+	 * Base class for a successful registration. Concrete registration implementations
+	 * will typically extend this class to attach more information.
+	 */
+	public static class Success extends RegistrationResponse {
+		private static final long serialVersionUID = 1L;
+		
+		@Override
+		public String toString() {
+			return "Registration Successful";
+		}
+	}
+
+	// ----------------------------------------------------------------------------
+
+	/**
+	 * A rejected (declined) registration.
+	 */
+	public static final class Decline extends RegistrationResponse {
+		private static final long serialVersionUID = 1L;
+
+		/** the rejection reason */
+		private final String reason;
+
+		/**
+		 * Creates a new rejection message.
+		 * 
+		 * @param reason The reason for the rejection.
+		 */
+		public Decline(String reason) {
+			this.reason = reason != null ? reason : "(unknown)";
+		}
+
+		/**
+		 * Gets the reason for the rejection.
+		 */
+		public String getReason() {
+			return reason;
+		}
+
+		@Override
+		public String toString() {
+			return "Registration Declined (" + reason + ')';
+		}
+	}
+}
+
+
+
+
+
+
+

http://git-wip-us.apache.org/repos/asf/flink/blob/9c247d1f/flink-runtime/src/main/java/org/apache/flink/runtime/registration/RetryingRegistration.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/registration/RetryingRegistration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/registration/RetryingRegistration.java
new file mode 100644
index 0000000..88fe9b5
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/registration/RetryingRegistration.java
@@ -0,0 +1,296 @@
+/*
+ * 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.registration;
+
+import akka.dispatch.OnFailure;
+import akka.dispatch.OnSuccess;
+
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.runtime.rpc.RpcGateway;
+import org.apache.flink.runtime.rpc.RpcService;
+
+import org.slf4j.Logger;
+
+import scala.concurrent.Future;
+import scala.concurrent.Promise;
+import scala.concurrent.impl.Promise.DefaultPromise;
+
+import java.util.UUID;
+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;
+
+
+/**
+ * This utility class implements the basis of registering one component at another component,
+ * for example registering the TaskExecutor at the ResourceManager.
+ * This {@code RetryingRegistration} implements both the initial address resolution
+ * and the retries-with-backoff strategy.
+ * 
+ * <p>The registration gives access to a future that is completed upon successful registration.
+ * The registration can be canceled, for example when the target where it tries to register
+ * at looses leader status.
+ * 
+ * @param <Gateway> The type of the gateway to connect to.
+ * @param <Success> The type of the successful registration responses.
+ */
+public abstract class RetryingRegistration<Gateway extends RpcGateway, Success extends RegistrationResponse.Success> {
+
+	// ------------------------------------------------------------------------
+	//  default configuration values
+	// ------------------------------------------------------------------------
+
+	/** default value for the initial registration timeout (milliseconds) */
+	private static final long INITIAL_REGISTRATION_TIMEOUT_MILLIS = 100;
+
+	/** default value for the maximum registration timeout, after exponential back-off (milliseconds) */
+	private static final long MAX_REGISTRATION_TIMEOUT_MILLIS = 30000;
+
+	/** The pause (milliseconds) made after an registration attempt caused an exception (other than timeout) */
+	private static final long ERROR_REGISTRATION_DELAY_MILLIS = 10000;
+
+	/** The pause (milliseconds) made after the registration attempt was refused */
+	private static final long REFUSED_REGISTRATION_DELAY_MILLIS = 30000;
+
+	// ------------------------------------------------------------------------
+	// Fields
+	// ------------------------------------------------------------------------
+
+	private final Logger log;
+
+	private final RpcService rpcService;
+
+	private final String targetName;
+
+	private final Class<Gateway> targetType;
+
+	private final String targetAddress;
+
+	private final UUID leaderId;
+
+	private final Promise<Tuple2<Gateway, Success>> completionPromise;
+
+	private final long initialRegistrationTimeout;
+
+	private final long maxRegistrationTimeout;
+
+	private final long delayOnError;
+
+	private final long delayOnRefusedRegistration;
+
+	private volatile boolean canceled;
+
+	// ------------------------------------------------------------------------
+
+	public RetryingRegistration(
+			Logger log,
+			RpcService rpcService,
+			String targetName,
+			Class<Gateway> targetType,
+			String targetAddress,
+			UUID leaderId) {
+		this(log, rpcService, targetName, targetType, targetAddress, leaderId,
+				INITIAL_REGISTRATION_TIMEOUT_MILLIS, MAX_REGISTRATION_TIMEOUT_MILLIS,
+				ERROR_REGISTRATION_DELAY_MILLIS, REFUSED_REGISTRATION_DELAY_MILLIS);
+	}
+
+	public RetryingRegistration(
+			Logger log,
+			RpcService rpcService,
+			String targetName, 
+			Class<Gateway> targetType,
+			String targetAddress,
+			UUID leaderId,
+			long initialRegistrationTimeout,
+			long maxRegistrationTimeout,
+			long delayOnError,
+			long delayOnRefusedRegistration) {
+
+		checkArgument(initialRegistrationTimeout > 0, "initial registration timeout must be greater than zero");
+		checkArgument(maxRegistrationTimeout > 0, "maximum registration timeout must be greater than zero");
+		checkArgument(delayOnError >= 0, "delay on error must be non-negative");
+		checkArgument(delayOnRefusedRegistration >= 0, "delay on refused registration must be non-negative");
+
+		this.log = checkNotNull(log);
+		this.rpcService = checkNotNull(rpcService);
+		this.targetName = checkNotNull(targetName);
+		this.targetType = checkNotNull(targetType);
+		this.targetAddress = checkNotNull(targetAddress);
+		this.leaderId = checkNotNull(leaderId);
+		this.initialRegistrationTimeout = initialRegistrationTimeout;
+		this.maxRegistrationTimeout = maxRegistrationTimeout;
+		this.delayOnError = delayOnError;
+		this.delayOnRefusedRegistration = delayOnRefusedRegistration;
+
+		this.completionPromise = new DefaultPromise<>();
+	}
+
+	// ------------------------------------------------------------------------
+	//  completion and cancellation
+	// ------------------------------------------------------------------------
+
+	public Future<Tuple2<Gateway, Success>> getFuture() {
+		return completionPromise.future();
+	}
+
+	/**
+	 * Cancels the registration procedure.
+	 */
+	public void cancel() {
+		canceled = true;
+	}
+
+	/**
+	 * Checks if the registration was canceled.
+	 * @return True if the registration was canceled, false otherwise.
+	 */
+	public boolean isCanceled() {
+		return canceled;
+	}
+
+	// ------------------------------------------------------------------------
+	//  registration
+	// ------------------------------------------------------------------------
+
+	protected abstract Future<RegistrationResponse> invokeRegistration(
+			Gateway gateway, UUID leaderId, long timeoutMillis) throws Exception;
+
+	/**
+	 * This method resolves the target address to a callable gateway and starts the
+	 * registration after that.
+	 */
+	@SuppressWarnings("unchecked")
+	public void startRegistration() {
+		try {
+			// trigger resolution of the resource manager address to a callable gateway
+			Future<Gateway> resourceManagerFuture = rpcService.connect(targetAddress, targetType);
+	
+			// upon success, start the registration attempts
+			resourceManagerFuture.onSuccess(new OnSuccess<Gateway>() {
+				@Override
+				public void onSuccess(Gateway result) {
+					log.info("Resolved {} address, beginning registration", targetName);
+					register(result, 1, initialRegistrationTimeout);
+				}
+			}, rpcService.getExecutionContext());
+	
+			// upon failure, retry, unless this is cancelled
+			resourceManagerFuture.onFailure(new OnFailure() {
+				@Override
+				public void onFailure(Throwable failure) {
+					if (!isCanceled()) {
+						log.warn("Could not resolve {} address {}, retrying...", targetName, targetAddress);
+						startRegistration();
+					}
+				}
+			}, rpcService.getExecutionContext());
+		}
+		catch (Throwable t) {
+			cancel();
+			completionPromise.tryFailure(t);
+		}
+	}
+
+	/**
+	 * This method performs a registration attempt and triggers either a success notification or a retry,
+	 * depending on the result.
+	 */
+	@SuppressWarnings("unchecked")
+	private void register(final Gateway gateway, final int attempt, final long timeoutMillis) {
+		// eager check for canceling to avoid some unnecessary work
+		if (canceled) {
+			return;
+		}
+
+		try {
+			log.info("Registration at {} attempt {} (timeout={}ms)", targetName, attempt, timeoutMillis);
+			Future<RegistrationResponse> registrationFuture = invokeRegistration(gateway, leaderId, timeoutMillis);
+	
+			// if the registration was successful, let the TaskExecutor know
+			registrationFuture.onSuccess(new OnSuccess<RegistrationResponse>() {
+				
+				@Override
+				public void onSuccess(RegistrationResponse result) throws Throwable {
+					if (!isCanceled()) {
+						if (result instanceof RegistrationResponse.Success) {
+							// registration successful!
+							Success success = (Success) result;
+							completionPromise.success(new Tuple2<>(gateway, success));
+						}
+						else {
+							// registration refused or unknown
+							if (result instanceof RegistrationResponse.Decline) {
+								RegistrationResponse.Decline decline = (RegistrationResponse.Decline) result;
+								log.info("Registration at {} was declined: {}", targetName, decline.getReason());
+							} else {
+								log.error("Received unknown response to registration attempt: " + result);
+							}
+
+							log.info("Pausing and re-attempting registration in {} ms", delayOnRefusedRegistration);
+							registerLater(gateway, 1, initialRegistrationTimeout, delayOnRefusedRegistration);
+						}
+					}
+				}
+			}, rpcService.getExecutionContext());
+	
+			// upon failure, retry
+			registrationFuture.onFailure(new OnFailure() {
+				@Override
+				public void onFailure(Throwable failure) {
+					if (!isCanceled()) {
+						if (failure instanceof TimeoutException) {
+							// we simply have not received a response in time. maybe the timeout was
+							// very low (initial fast registration attempts), maybe the target endpoint is
+							// currently down.
+							if (log.isDebugEnabled()) {
+								log.debug("Registration at {} ({}) attempt {} timed out after {} ms",
+										targetName, targetAddress, attempt, timeoutMillis);
+							}
+	
+							long newTimeoutMillis = Math.min(2 * timeoutMillis, maxRegistrationTimeout);
+							register(gateway, attempt + 1, newTimeoutMillis);
+						}
+						else {
+							// a serious failure occurred. we still should not give up, but keep trying
+							log.error("Registration at " + targetName + " failed due to an error", failure);
+							log.info("Pausing and re-attempting registration in {} ms", delayOnError);
+	
+							registerLater(gateway, 1, initialRegistrationTimeout, delayOnError);
+						}
+					}
+				}
+			}, rpcService.getExecutionContext());
+		}
+		catch (Throwable t) {
+			cancel();
+			completionPromise.tryFailure(t);
+		}
+	}
+
+	private void registerLater(final Gateway gateway, final int attempt, final long timeoutMillis, long delay) {
+		rpcService.scheduleRunnable(new Runnable() {
+			@Override
+			public void run() {
+				register(gateway, attempt, timeoutMillis);
+			}
+		}, delay, TimeUnit.MILLISECONDS);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9c247d1f/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/JobMasterRegistration.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/JobMasterRegistration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/JobMasterRegistration.java
new file mode 100644
index 0000000..309dcc1
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/JobMasterRegistration.java
@@ -0,0 +1,35 @@
+/*
+ * 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 java.io.Serializable;
+
+public class JobMasterRegistration implements Serializable {
+	private static final long serialVersionUID = 8411214999193765202L;
+
+	private final String address;
+
+	public JobMasterRegistration(String address) {
+		this.address = address;
+	}
+
+	public String getAddress() {
+		return address;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9c247d1f/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/RegistrationResponse.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/RegistrationResponse.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/RegistrationResponse.java
new file mode 100644
index 0000000..fb6c401
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/RegistrationResponse.java
@@ -0,0 +1,43 @@
+/*
+ * 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.instance.InstanceID;
+
+import java.io.Serializable;
+
+public class RegistrationResponse implements Serializable {
+	private static final long serialVersionUID = -2379003255993119993L;
+
+	private final boolean isSuccess;
+	private final InstanceID instanceID;
+
+	public RegistrationResponse(boolean isSuccess, InstanceID instanceID) {
+		this.isSuccess = isSuccess;
+		this.instanceID = instanceID;
+	}
+
+	public boolean isSuccess() {
+		return isSuccess;
+	}
+
+	public InstanceID getInstanceID() {
+		return instanceID;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9c247d1f/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
new file mode 100644
index 0000000..44c022b
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java
@@ -0,0 +1,214 @@
+/*
+ * 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 akka.dispatch.Mapper;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.leaderelection.LeaderContender;
+import org.apache.flink.runtime.leaderelection.LeaderElectionService;
+import org.apache.flink.runtime.rpc.RpcMethod;
+import org.apache.flink.runtime.rpc.RpcEndpoint;
+import org.apache.flink.runtime.rpc.RpcService;
+import org.apache.flink.runtime.jobmaster.JobMaster;
+import org.apache.flink.runtime.jobmaster.JobMasterGateway;
+import org.apache.flink.runtime.taskexecutor.TaskExecutorRegistrationSuccess;
+
+import scala.concurrent.Future;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.UUID;
+
+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:
+ * <ul>
+ *     <li>{@link #registerJobMaster(JobMasterRegistration)} registers a {@link JobMaster} at the resource manager</li>
+ *     <li>{@link #requestSlot(SlotRequest)} requests a slot from the resource manager</li>
+ * </ul>
+ */
+public class ResourceManager extends RpcEndpoint<ResourceManagerGateway> {
+	private final Map<JobMasterGateway, InstanceID> jobMasterGateways;
+	private final HighAvailabilityServices highAvailabilityServices;
+	private LeaderElectionService leaderElectionService = null;
+	private UUID leaderSessionID = null;
+
+	public ResourceManager(RpcService rpcService, HighAvailabilityServices highAvailabilityServices) {
+		super(rpcService);
+		this.highAvailabilityServices = checkNotNull(highAvailabilityServices);
+		this.jobMasterGateways = new HashMap<>();
+	}
+
+	@Override
+	public void start() {
+		// start a leader
+		try {
+			super.start();
+			leaderElectionService = highAvailabilityServices.getResourceManagerLeaderElectionService();
+			leaderElectionService.start(new ResourceManagerLeaderContender());
+		} 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);
+		}
+	}
+
+	@Override
+	public void shutDown() {
+		try {
+			leaderElectionService.stop();
+			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);
+		}
+	}
+
+	/**
+	 * 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;
+	}
+
+	/**
+	 * Register a {@link JobMaster} at the resource manager.
+	 *
+	 * @param jobMasterRegistration Job master registration information
+	 * @return Future registration response
+	 */
+	@RpcMethod
+	public Future<RegistrationResponse> registerJobMaster(JobMasterRegistration jobMasterRegistration) {
+		Future<JobMasterGateway> jobMasterFuture = getRpcService().connect(jobMasterRegistration.getAddress(), JobMasterGateway.class);
+
+		return jobMasterFuture.map(new Mapper<JobMasterGateway, RegistrationResponse>() {
+			@Override
+			public RegistrationResponse apply(final JobMasterGateway jobMasterGateway) {
+				InstanceID instanceID;
+
+				if (jobMasterGateways.containsKey(jobMasterGateway)) {
+					instanceID = jobMasterGateways.get(jobMasterGateway);
+				} else {
+					instanceID = new InstanceID();
+					jobMasterGateways.put(jobMasterGateway, instanceID);
+				}
+
+				return new RegistrationResponse(true, instanceID);
+			}
+		}, getMainThreadExecutionContext());
+	}
+
+	/**
+	 * Requests a slot from the resource manager.
+	 *
+	 * @param slotRequest Slot request
+	 * @return Slot assignment
+	 */
+	@RpcMethod
+	public SlotAssignment requestSlot(SlotRequest slotRequest) {
+		System.out.println("SlotRequest: " + slotRequest);
+		return new SlotAssignment();
+	}
+
+
+	/**
+	 *
+	 * @param resourceManagerLeaderId  The fencing token for the ResourceManager leader 
+	 * @param taskExecutorAddress      The address of the TaskExecutor that registers
+	 * @param resourceID               The resource ID of the TaskExecutor that registers
+	 *
+	 * @return The response by the ResourceManager.
+	 */
+	@RpcMethod
+	public org.apache.flink.runtime.registration.RegistrationResponse registerTaskExecutor(
+			UUID resourceManagerLeaderId,
+			String taskExecutorAddress,
+			ResourceID resourceID) {
+
+		return new TaskExecutorRegistrationSuccess(new InstanceID(), 5000);
+	}
+
+	private class ResourceManagerLeaderContender implements LeaderContender {
+
+		/**
+		 * Callback method when current resourceManager is granted leadership
+		 *
+		 * @param leaderSessionID unique leadershipID
+		 */
+		@Override
+		public void grantLeadership(final UUID leaderSessionID) {
+			runAsync(new Runnable() {
+				@Override
+				public void run() {
+					log.info("ResourceManager {} was granted leadership with leader session ID {}", getAddress(), leaderSessionID);
+					ResourceManager.this.leaderSessionID = leaderSessionID;
+					// confirming the leader session ID might be blocking,
+					leaderElectionService.confirmLeaderSessionID(leaderSessionID);
+				}
+			});
+		}
+
+		/**
+		 * Callback method when current resourceManager lose leadership.
+		 */
+		@Override
+		public void revokeLeadership() {
+			runAsync(new Runnable() {
+				@Override
+				public void run() {
+					log.info("ResourceManager {} was revoked leadership.", getAddress());
+					jobMasterGateways.clear();
+					leaderSessionID = null;
+				}
+			});
+		}
+
+		@Override
+		public String getAddress() {
+			return ResourceManager.this.getAddress();
+		}
+
+		/**
+		 * Handles error occurring in the leader election service
+		 *
+		 * @param exception Exception being thrown in the leader election service
+		 */
+		@Override
+		public void handleError(final Exception exception) {
+			runAsync(new Runnable() {
+				@Override
+				public void run() {
+					log.error("ResourceManager received an error from the LeaderElectionService.", exception);
+					// terminate ResourceManager in case of an error
+					shutDown();
+				}
+			});
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9c247d1f/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
new file mode 100644
index 0000000..b5782b0
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java
@@ -0,0 +1,77 @@
+/*
+ * 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.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.rpc.RpcGateway;
+import org.apache.flink.runtime.rpc.RpcTimeout;
+import org.apache.flink.runtime.jobmaster.JobMaster;
+
+import scala.concurrent.Future;
+import scala.concurrent.duration.FiniteDuration;
+
+import java.util.UUID;
+
+/**
+ * The {@link ResourceManager}'s RPC gateway interface.
+ */
+public interface ResourceManagerGateway extends RpcGateway {
+
+	/**
+	 * Register a {@link JobMaster} at the resource manager.
+	 *
+	 * @param jobMasterRegistration Job master registration information
+	 * @param timeout Timeout for the future to complete
+	 * @return Future registration response
+	 */
+	Future<RegistrationResponse> registerJobMaster(
+		JobMasterRegistration jobMasterRegistration,
+		@RpcTimeout FiniteDuration timeout);
+
+	/**
+	 * Register a {@link JobMaster} at the resource manager.
+	 *
+	 * @param jobMasterRegistration Job master registration information
+	 * @return Future registration response
+	 */
+	Future<RegistrationResponse> registerJobMaster(JobMasterRegistration jobMasterRegistration);
+
+	/**
+	 * Requests a slot from the resource manager.
+	 *
+	 * @param slotRequest Slot request
+	 * @return Future slot assignment
+	 */
+	Future<SlotAssignment> requestSlot(SlotRequest slotRequest);
+
+	/**
+	 * 
+	 * @param resourceManagerLeaderId  The fencing token for the ResourceManager leader 
+	 * @param taskExecutorAddress      The address of the TaskExecutor that registers
+	 * @param resourceID               The resource ID of the TaskExecutor that registers
+	 * @param timeout                  The timeout for the response.
+	 * 
+	 * @return The future to the response by the ResourceManager.
+	 */
+	Future<org.apache.flink.runtime.registration.RegistrationResponse> registerTaskExecutor(
+			UUID resourceManagerLeaderId,
+			String taskExecutorAddress,
+			ResourceID resourceID,
+			@RpcTimeout FiniteDuration timeout);
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9c247d1f/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotAssignment.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotAssignment.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotAssignment.java
new file mode 100644
index 0000000..695204d
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotAssignment.java
@@ -0,0 +1,25 @@
+/*
+ * 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 java.io.Serializable;
+
+public class SlotAssignment implements Serializable{
+	private static final long serialVersionUID = -6990813455942742322L;
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9c247d1f/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotManager.java
new file mode 100644
index 0000000..5c06648
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotManager.java
@@ -0,0 +1,523 @@
+/*
+ * 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.annotation.VisibleForTesting;
+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.ResourceSlot;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.taskexecutor.SlotReport;
+import org.apache.flink.runtime.taskexecutor.SlotStatus;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.Map;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * SlotManager is responsible for receiving slot requests and do slot allocations. It allows to request
+ * slots from registered TaskManagers and issues container allocation requests in case of there are not
+ * enough available slots. Besides, it should sync its slot allocation with TaskManager's heartbeat.
+ * <p>
+ * The main operation principle of SlotManager is:
+ * <ul>
+ * <li>1. All slot allocation status should be synced with TaskManager, which is the ground truth.</li>
+ * <li>2. All slots that have registered must be tracked, either by free pool or allocated pool.</li>
+ * <li>3. All slot requests will be handled by best efforts, there is no guarantee that one request will be
+ * fulfilled in time or correctly allocated. Conflicts or timeout or some special error will happen, it should
+ * be handled outside SlotManager. SlotManager will make each decision based on the information it currently
+ * holds.</li>
+ * </ul>
+ * <b>IMPORTANT:</b> This class is <b>Not Thread-safe</b>.
+ */
+public abstract class SlotManager {
+
+	private static final Logger LOG = LoggerFactory.getLogger(SlotManager.class);
+
+	/** Gateway to communicate with ResourceManager */
+	private final ResourceManagerGateway resourceManagerGateway;
+
+	/** All registered slots, including free and allocated slots */
+	private final Map<ResourceID, Map<SlotID, ResourceSlot>> registeredSlots;
+
+	/** All pending slot requests, waiting available slots to fulfil */
+	private final Map<AllocationID, SlotRequest> pendingSlotRequests;
+
+	/** All free slots that can be used to be allocated */
+	private final Map<SlotID, ResourceSlot> freeSlots;
+
+	/** All allocations, we can lookup allocations either by SlotID or AllocationID */
+	private final AllocationMap allocationMap;
+
+	public SlotManager(ResourceManagerGateway resourceManagerGateway) {
+		this.resourceManagerGateway = checkNotNull(resourceManagerGateway);
+		this.registeredSlots = new HashMap<>(16);
+		this.pendingSlotRequests = new LinkedHashMap<>(16);
+		this.freeSlots = new HashMap<>(16);
+		this.allocationMap = new AllocationMap();
+	}
+
+	// ------------------------------------------------------------------------
+	//  slot managements
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Request a slot with requirements, we may either fulfill the request or pending it. Trigger container
+	 * allocation if we don't have enough resource. If we have free slot which can match the request, record
+	 * this allocation and forward the request to TaskManager through ResourceManager (we want this done by
+	 * RPC's main thread to avoid race condition).
+	 *
+	 * @param request The detailed request of the slot
+	 */
+	public void requestSlot(final SlotRequest request) {
+		if (isRequestDuplicated(request)) {
+			LOG.warn("Duplicated slot request, AllocationID:{}", request.getAllocationId());
+			return;
+		}
+
+		// try to fulfil the request with current free slots
+		ResourceSlot slot = chooseSlotToUse(request, freeSlots);
+		if (slot != null) {
+			LOG.info("Assigning SlotID({}) to AllocationID({}), JobID:{}", slot.getSlotId(),
+				request.getAllocationId(), request.getJobId());
+
+			// record this allocation in bookkeeping
+			allocationMap.addAllocation(slot.getSlotId(), request.getAllocationId());
+
+			// remove selected slot from free pool
+			freeSlots.remove(slot.getSlotId());
+
+			// TODO: send slot request to TaskManager
+		} else {
+			LOG.info("Cannot fulfil slot request, try to allocate a new container for it, " +
+				"AllocationID:{}, JobID:{}", request.getAllocationId(), request.getJobId());
+			allocateContainer(request.getResourceProfile());
+			pendingSlotRequests.put(request.getAllocationId(), request);
+		}
+	}
+
+	/**
+	 * Sync slot status with TaskManager's SlotReport.
+	 */
+	public void updateSlotStatus(final SlotReport slotReport) {
+		for (SlotStatus slotStatus : slotReport.getSlotsStatus()) {
+			updateSlotStatus(slotStatus);
+		}
+	}
+
+	/**
+	 * The slot request to TaskManager may be either failed by rpc communication (timeout, network error, etc.)
+	 * or really rejected by TaskManager. We shall retry this request by:
+	 * <ul>
+	 * <li>1. verify and clear all the previous allocate information for this request
+	 * <li>2. try to request slot again
+	 * </ul>
+	 * <p>
+	 * This may cause some duplicate allocation, e.g. the slot request to TaskManager is successful but the response
+	 * is lost somehow, so we may request a slot in another TaskManager, this causes two slots assigned to one request,
+	 * but it can be taken care of by rejecting registration at JobManager.
+	 *
+	 * @param originalRequest The original slot request
+	 * @param slotId          The target SlotID
+	 */
+	public void handleSlotRequestFailedAtTaskManager(final SlotRequest originalRequest, final SlotID slotId) {
+		final AllocationID originalAllocationId = originalRequest.getAllocationId();
+		LOG.info("Slot request failed at TaskManager, SlotID:{}, AllocationID:{}, JobID:{}",
+			slotId, originalAllocationId, originalRequest.getJobId());
+
+		// verify the allocation info before we do anything
+		if (freeSlots.containsKey(slotId)) {
+			// this slot is currently empty, no need to de-allocate it from our allocations
+			LOG.info("Original slot is somehow empty, retrying this request");
+
+			// before retry, we should double check whether this request was allocated by some other ways
+			if (!allocationMap.isAllocated(originalAllocationId)) {
+				requestSlot(originalRequest);
+			} else {
+				LOG.info("The failed request has somehow been allocated, SlotID:{}",
+					allocationMap.getSlotID(originalAllocationId));
+			}
+		} else if (allocationMap.isAllocated(slotId)) {
+			final AllocationID currentAllocationId = allocationMap.getAllocationID(slotId);
+
+			// check whether we have an agreement on whom this slot belongs to
+			if (originalAllocationId.equals(currentAllocationId)) {
+				LOG.info("De-allocate this request and retry");
+				allocationMap.removeAllocation(currentAllocationId);
+
+				// put this slot back to free pool
+				ResourceSlot slot = checkNotNull(getRegisteredSlot(slotId));
+				freeSlots.put(slotId, slot);
+
+				// retry the request
+				requestSlot(originalRequest);
+			} else {
+				// the slot is taken by someone else, no need to de-allocate it from our allocations
+				LOG.info("Original slot is taken by someone else, current AllocationID:{}", currentAllocationId);
+
+				// before retry, we should double check whether this request was allocated by some other ways
+				if (!allocationMap.isAllocated(originalAllocationId)) {
+					requestSlot(originalRequest);
+				} else {
+					LOG.info("The failed request is somehow been allocated, SlotID:{}",
+						allocationMap.getSlotID(originalAllocationId));
+				}
+			}
+		} else {
+			LOG.error("BUG! {} is neither in free pool nor in allocated pool", slotId);
+		}
+	}
+
+	/**
+	 * Callback for TaskManager failures. In case that a TaskManager fails, we have to clean up all its slots.
+	 *
+	 * @param resourceId The ResourceID of the TaskManager
+	 */
+	public void notifyTaskManagerFailure(final ResourceID resourceId) {
+		LOG.info("Resource:{} been notified failure", resourceId);
+		final Map<SlotID, ResourceSlot> slotIdsToRemove = registeredSlots.remove(resourceId);
+		if (slotIdsToRemove != null) {
+			for (SlotID slotId : slotIdsToRemove.keySet()) {
+				LOG.info("Removing Slot:{} upon resource failure", slotId);
+				if (freeSlots.containsKey(slotId)) {
+					freeSlots.remove(slotId);
+				} else if (allocationMap.isAllocated(slotId)) {
+					allocationMap.removeAllocation(slotId);
+				} else {
+					LOG.error("BUG! {} is neither in free pool nor in allocated pool", slotId);
+				}
+			}
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  internal behaviors
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Update slot status based on TaskManager's report. There are mainly two situations when we receive the report:
+	 * <ul>
+	 * <li>1. The slot is newly registered.</li>
+	 * <li>2. The slot has registered, it contains its current status.</li>
+	 * </ul>
+	 * <p>
+	 * Regarding 1: It's fairly simple, we just record this slot's status, and trigger schedule if slot is empty.
+	 * <p>
+	 * Regarding 2: It will cause some weird situation since we may have some time-gap on how the slot's status really
+	 * is. We may have some updates on the slot's allocation, but it doesn't reflected by TaskManager's heartbeat yet,
+	 * and we may make some wrong decision if we cannot guarantee we have the exact status about all the slots. So
+	 * the principle here is: We always trust TaskManager's heartbeat, we will correct our information based on that
+	 * and take next action based on the diff between our information and heartbeat status.
+	 *
+	 * @param reportedStatus Reported slot status
+	 */
+	void updateSlotStatus(final SlotStatus reportedStatus) {
+		final SlotID slotId = reportedStatus.getSlotID();
+		final ResourceSlot slot = new ResourceSlot(slotId, reportedStatus.getProfiler());
+
+		if (registerNewSlot(slot)) {
+			// we have a newly registered slot
+			LOG.info("New slot appeared, SlotID:{}, AllocationID:{}", slotId, reportedStatus.getAllocationID());
+
+			if (reportedStatus.getAllocationID() != null) {
+				// slot in use, record this in bookkeeping
+				allocationMap.addAllocation(slotId, reportedStatus.getAllocationID());
+			} else {
+				handleFreeSlot(new ResourceSlot(slotId, reportedStatus.getProfiler()));
+			}
+		} else {
+			// slot exists, update current information
+			if (reportedStatus.getAllocationID() != null) {
+				// slot is reported in use
+				final AllocationID reportedAllocationId = reportedStatus.getAllocationID();
+
+				// check whether we also thought this slot is in use
+				if (allocationMap.isAllocated(slotId)) {
+					// we also think that slot is in use, check whether the AllocationID matches
+					final AllocationID currentAllocationId = allocationMap.getAllocationID(slotId);
+
+					if (!reportedAllocationId.equals(currentAllocationId)) {
+						LOG.info("Slot allocation info mismatch! SlotID:{}, current:{}, reported:{}",
+							slotId, currentAllocationId, reportedAllocationId);
+
+						// seems we have a disagreement about the slot assignments, need to correct it
+						allocationMap.removeAllocation(slotId);
+						allocationMap.addAllocation(slotId, reportedAllocationId);
+					}
+				} else {
+					LOG.info("Slot allocation info mismatch! SlotID:{}, current:null, reported:{}",
+						slotId, reportedAllocationId);
+
+					// we thought the slot is free, should correct this information
+					allocationMap.addAllocation(slotId, reportedStatus.getAllocationID());
+
+					// remove this slot from free slots pool
+					freeSlots.remove(slotId);
+				}
+			} else {
+				// slot is reported empty
+
+				// check whether we also thought this slot is empty
+				if (allocationMap.isAllocated(slotId)) {
+					LOG.info("Slot allocation info mismatch! SlotID:{}, current:{}, reported:null",
+						slotId, allocationMap.getAllocationID(slotId));
+
+					// we thought the slot is in use, correct it
+					allocationMap.removeAllocation(slotId);
+
+					// we have a free slot!
+					handleFreeSlot(new ResourceSlot(slotId, reportedStatus.getProfiler()));
+				}
+			}
+		}
+	}
+
+	/**
+	 * When we have a free slot, try to fulfill the pending request first. If any request can be fulfilled,
+	 * record this allocation in bookkeeping and send slot request to TaskManager, else we just add this slot
+	 * to the free pool.
+	 *
+	 * @param freeSlot The free slot
+	 */
+	private void handleFreeSlot(final ResourceSlot freeSlot) {
+		SlotRequest chosenRequest = chooseRequestToFulfill(freeSlot, pendingSlotRequests);
+
+		if (chosenRequest != null) {
+			pendingSlotRequests.remove(chosenRequest.getAllocationId());
+
+			LOG.info("Assigning SlotID({}) to AllocationID({}), JobID:{}", freeSlot.getSlotId(),
+				chosenRequest.getAllocationId(), chosenRequest.getJobId());
+			allocationMap.addAllocation(freeSlot.getSlotId(), chosenRequest.getAllocationId());
+
+			// TODO: send slot request to TaskManager
+		} else {
+			freeSlots.put(freeSlot.getSlotId(), freeSlot);
+		}
+	}
+
+	/**
+	 * Check whether the request is duplicated. We use AllocationID to identify slot request, for each
+	 * formerly received slot request, it is either in pending list or already been allocated.
+	 *
+	 * @param request The slot request
+	 * @return <tt>true</tt> if the request is duplicated
+	 */
+	private boolean isRequestDuplicated(final SlotRequest request) {
+		final AllocationID allocationId = request.getAllocationId();
+		return pendingSlotRequests.containsKey(allocationId)
+			|| allocationMap.isAllocated(allocationId);
+	}
+
+	/**
+	 * Try to register slot, and tell if this slot is newly registered.
+	 *
+	 * @param slot The ResourceSlot which will be checked and registered
+	 * @return <tt>true</tt> if we meet a new slot
+	 */
+	private boolean registerNewSlot(final ResourceSlot slot) {
+		final SlotID slotId = slot.getSlotId();
+		final ResourceID resourceId = slotId.getResourceID();
+		if (!registeredSlots.containsKey(resourceId)) {
+			registeredSlots.put(resourceId, new HashMap<SlotID, ResourceSlot>());
+		}
+		return registeredSlots.get(resourceId).put(slotId, slot) == null;
+	}
+
+	private ResourceSlot getRegisteredSlot(final SlotID slotId) {
+		final ResourceID resourceId = slotId.getResourceID();
+		if (!registeredSlots.containsKey(resourceId)) {
+			return null;
+		}
+		return registeredSlots.get(resourceId).get(slotId);
+	}
+
+	// ------------------------------------------------------------------------
+	//  Framework specific behavior
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Choose a slot to use among all free slots, the behavior is framework specified.
+	 *
+	 * @param request   The slot request
+	 * @param freeSlots All slots which can be used
+	 * @return The slot we choose to use, <tt>null</tt> if we did not find a match
+	 */
+	protected abstract ResourceSlot chooseSlotToUse(final SlotRequest request,
+		final Map<SlotID, ResourceSlot> freeSlots);
+
+	/**
+	 * Choose a pending request to fulfill when we have a free slot, the behavior is framework specified.
+	 *
+	 * @param offeredSlot     The free slot
+	 * @param pendingRequests All the pending slot requests
+	 * @return The chosen SlotRequest, <tt>null</tt> if we did not find a match
+	 */
+	protected abstract SlotRequest chooseRequestToFulfill(final ResourceSlot offeredSlot,
+		final Map<AllocationID, SlotRequest> pendingRequests);
+
+	/**
+	 * The framework specific code for allocating a container for specified resource profile.
+	 *
+	 * @param resourceProfile The resource profile
+	 */
+	protected abstract void allocateContainer(final ResourceProfile resourceProfile);
+
+
+	// ------------------------------------------------------------------------
+	//  Helper classes
+	// ------------------------------------------------------------------------
+
+	/**
+	 * We maintain all the allocations with SlotID and AllocationID. We are able to get or remove the allocation info
+	 * either by SlotID or AllocationID.
+	 */
+	private static class AllocationMap {
+
+		/** All allocated slots (by SlotID) */
+		private final Map<SlotID, AllocationID> allocatedSlots;
+
+		/** All allocated slots (by AllocationID), it'a a inverse view of allocatedSlots */
+		private final Map<AllocationID, SlotID> allocatedSlotsByAllocationId;
+
+		AllocationMap() {
+			this.allocatedSlots = new HashMap<>(16);
+			this.allocatedSlotsByAllocationId = new HashMap<>(16);
+		}
+
+		/**
+		 * Add a allocation
+		 *
+		 * @param slotId       The slot id
+		 * @param allocationId The allocation id
+		 */
+		void addAllocation(final SlotID slotId, final AllocationID allocationId) {
+			allocatedSlots.put(slotId, allocationId);
+			allocatedSlotsByAllocationId.put(allocationId, slotId);
+		}
+
+		/**
+		 * De-allocation with slot id
+		 *
+		 * @param slotId The slot id
+		 */
+		void removeAllocation(final SlotID slotId) {
+			if (allocatedSlots.containsKey(slotId)) {
+				final AllocationID allocationId = allocatedSlots.get(slotId);
+				allocatedSlots.remove(slotId);
+				allocatedSlotsByAllocationId.remove(allocationId);
+			}
+		}
+
+		/**
+		 * De-allocation with allocation id
+		 *
+		 * @param allocationId The allocation id
+		 */
+		void removeAllocation(final AllocationID allocationId) {
+			if (allocatedSlotsByAllocationId.containsKey(allocationId)) {
+				SlotID slotId = allocatedSlotsByAllocationId.get(allocationId);
+				allocatedSlotsByAllocationId.remove(allocationId);
+				allocatedSlots.remove(slotId);
+			}
+		}
+
+		/**
+		 * Check whether allocation exists by slot id
+		 *
+		 * @param slotId The slot id
+		 * @return true if the allocation exists
+		 */
+		boolean isAllocated(final SlotID slotId) {
+			return allocatedSlots.containsKey(slotId);
+		}
+
+		/**
+		 * Check whether allocation exists by allocation id
+		 *
+		 * @param allocationId The allocation id
+		 * @return true if the allocation exists
+		 */
+		boolean isAllocated(final AllocationID allocationId) {
+			return allocatedSlotsByAllocationId.containsKey(allocationId);
+		}
+
+		AllocationID getAllocationID(final SlotID slotId) {
+			return allocatedSlots.get(slotId);
+		}
+
+		SlotID getSlotID(final AllocationID allocationId) {
+			return allocatedSlotsByAllocationId.get(allocationId);
+		}
+
+		public int size() {
+			return allocatedSlots.size();
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  Testing utilities
+	// ------------------------------------------------------------------------
+
+	@VisibleForTesting
+	boolean isAllocated(final SlotID slotId) {
+		return allocationMap.isAllocated(slotId);
+	}
+
+	@VisibleForTesting
+	boolean isAllocated(final AllocationID allocationId) {
+		return allocationMap.isAllocated(allocationId);
+	}
+
+	/**
+	 * Add free slots directly to the free pool, this will not trigger pending requests allocation
+	 *
+	 * @param slot The resource slot
+	 */
+	@VisibleForTesting
+	void addFreeSlot(final ResourceSlot slot) {
+		final ResourceID resourceId = slot.getResourceID();
+		final SlotID slotId = slot.getSlotId();
+
+		if (!registeredSlots.containsKey(resourceId)) {
+			registeredSlots.put(resourceId, new HashMap<SlotID, ResourceSlot>());
+		}
+		registeredSlots.get(resourceId).put(slot.getSlotId(), slot);
+		freeSlots.put(slotId, slot);
+	}
+
+	@VisibleForTesting
+	int getAllocatedSlotCount() {
+		return allocationMap.size();
+	}
+
+	@VisibleForTesting
+	int getFreeSlotCount() {
+		return freeSlots.size();
+	}
+
+	@VisibleForTesting
+	int getPendingRequestCount() {
+		return pendingSlotRequests.size();
+	}
+}


[25/50] [abbrv] flink git commit: [hotfix] fix ResourceManagerGateway

Posted by se...@apache.org.
[hotfix] fix ResourceManagerGateway


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

Branch: refs/heads/flip-6
Commit: c6954a948eab9561bce58992159674750b2f53e7
Parents: 485ef00
Author: Maximilian Michels <mx...@apache.org>
Authored: Thu Sep 22 13:56:00 2016 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Oct 14 15:14:41 2016 +0200

----------------------------------------------------------------------
 .../flink/runtime/resourcemanager/ResourceManagerGateway.java     | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/c6954a94/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 d8b8ebe..484cea7 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
@@ -27,7 +27,6 @@ import org.apache.flink.runtime.rpc.RpcTimeout;
 import org.apache.flink.runtime.jobmaster.JobMaster;
 import org.apache.flink.runtime.registration.RegistrationResponse;
 
-import org.apache.flink.runtime.registration.RegistrationResponse;
 import java.util.UUID;
 
 /**
@@ -39,6 +38,7 @@ public interface ResourceManagerGateway extends RpcGateway {
 	 * Register a {@link JobMaster} at the resource manager.
 	 *
 	 * @param resourceManagerLeaderId The fencing token for the ResourceManager leader
+	 * @param jobMasterLeaderId The fencing token for the JobMaster leader
 	 * @param jobMasterAddress        The address of the JobMaster that registers
 	 * @param jobID                   The Job ID of the JobMaster that registers
 	 * @param timeout                 Timeout for the future to complete
@@ -46,6 +46,7 @@ public interface ResourceManagerGateway extends RpcGateway {
 	 */
 	Future<RegistrationResponse> registerJobMaster(
 		UUID resourceManagerLeaderId,
+		UUID jobMasterLeaderId,
 		String jobMasterAddress,
 		JobID jobID,
 		@RpcTimeout Time timeout);


[05/50] [abbrv] flink git commit: [FLINK-4529] [flip-6] Move TaskExecutor, JobMaster and ResourceManager out of the rpc package

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/9c247d1f/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java
deleted file mode 100644
index 36d6310..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java
+++ /dev/null
@@ -1,827 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.runtime.rpc.taskexecutor;
-
-import akka.actor.ActorSystem;
-import akka.dispatch.ExecutionContexts$;
-import akka.util.Timeout;
-import com.typesafe.config.Config;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.flink.api.common.JobID;
-import org.apache.flink.configuration.ConfigConstants;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.configuration.IllegalConfigurationException;
-import org.apache.flink.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.annotation.VisibleForTesting;
-import org.apache.flink.runtime.clusterframework.types.ResourceID;
-import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
-import org.apache.flink.runtime.instance.InstanceConnectionInfo;
-import org.apache.flink.runtime.io.disk.iomanager.IOManager;
-import org.apache.flink.runtime.io.disk.iomanager.IOManager.IOMode;
-import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
-import org.apache.flink.runtime.io.network.NetworkEnvironment;
-import org.apache.flink.runtime.io.network.netty.NettyConfig;
-import org.apache.flink.runtime.leaderelection.LeaderElectionService;
-import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener;
-import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
-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.runtime.rpc.akka.AkkaRpcService;
-import org.apache.flink.runtime.taskmanager.MemoryLogger;
-import org.apache.flink.runtime.util.EnvironmentInformation;
-import org.apache.flink.runtime.util.LeaderRetrievalUtils;
-import org.apache.flink.runtime.taskmanager.NetworkEnvironmentConfiguration;
-import org.apache.flink.util.MathUtils;
-import org.apache.flink.util.NetUtils;
-
-import scala.Tuple2;
-import scala.Option;
-import scala.Some;
-import scala.concurrent.ExecutionContext;
-import scala.concurrent.duration.Duration;
-import scala.concurrent.duration.FiniteDuration;
-
-import java.io.File;
-import java.io.IOException;
-import java.net.InetAddress;
-import java.net.InetSocketAddress;
-import java.util.UUID;
-import java.util.concurrent.ForkJoinPool;
-import java.util.concurrent.TimeUnit;
-
-import static org.apache.flink.util.Preconditions.checkNotNull;
-
-/**
- * TaskExecutor implementation. The task executor is responsible for the execution of multiple
- * {@link org.apache.flink.runtime.taskmanager.Task}.
- */
-public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
-
-	private static final Logger LOG = LoggerFactory.getLogger(TaskExecutor.class);
-
-	/** The unique resource ID of this TaskExecutor */
-	private final ResourceID resourceID;
-
-	/** The access to the leader election and metadata storage services */
-	private final HighAvailabilityServices haServices;
-
-	/** The task manager configuration */
-	private final TaskExecutorConfiguration taskExecutorConfig;
-
-	/** The I/O manager component in the task manager */
-	private final IOManager ioManager;
-
-	/** The memory manager component in the task manager */
-	private final MemoryManager memoryManager;
-
-	/** The network component in the task manager */
-	private final NetworkEnvironment networkEnvironment;
-
-	/** The number of slots in the task manager, should be 1 for YARN */
-	private final int numberOfSlots;
-
-	// --------- resource manager --------
-
-	private TaskExecutorToResourceManagerConnection resourceManagerConnection;
-
-	// ------------------------------------------------------------------------
-
-	public TaskExecutor(
-			TaskExecutorConfiguration taskExecutorConfig,
-			ResourceID resourceID,
-			MemoryManager memoryManager,
-			IOManager ioManager,
-			NetworkEnvironment networkEnvironment,
-			int numberOfSlots,
-			RpcService rpcService,
-			HighAvailabilityServices haServices) {
-
-		super(rpcService);
-
-		this.taskExecutorConfig = checkNotNull(taskExecutorConfig);
-		this.resourceID = checkNotNull(resourceID);
-		this.memoryManager = checkNotNull(memoryManager);
-		this.ioManager = checkNotNull(ioManager);
-		this.networkEnvironment = checkNotNull(networkEnvironment);
-		this.numberOfSlots = checkNotNull(numberOfSlots);
-		this.haServices = checkNotNull(haServices);
-	}
-
-	// ------------------------------------------------------------------------
-	//  Life cycle
-	// ------------------------------------------------------------------------
-
-	@Override
-	public void start() {
-		super.start();
-
-		// start by connecting to the ResourceManager
-		try {
-			haServices.getResourceManagerLeaderRetriever().start(new ResourceManagerLeaderListener());
-		} catch (Exception e) {
-			onFatalErrorAsync(e);
-		}
-	}
-
-	// ------------------------------------------------------------------------
-	//  RPC methods - ResourceManager related
-	// ------------------------------------------------------------------------
-
-	@RpcMethod
-	public void notifyOfNewResourceManagerLeader(String newLeaderAddress, UUID newLeaderId) {
-		if (resourceManagerConnection != null) {
-			if (newLeaderAddress != null) {
-				// the resource manager switched to a new leader
-				log.info("ResourceManager leader changed from {} to {}. Registering at new leader.",
-					resourceManagerConnection.getResourceManagerAddress(), newLeaderAddress);
-			}
-			else {
-				// address null means that the current leader is lost without a new leader being there, yet
-				log.info("Current ResourceManager {} lost leader status. Waiting for new ResourceManager leader.",
-					resourceManagerConnection.getResourceManagerAddress());
-			}
-
-			// drop the current connection or connection attempt
-			if (resourceManagerConnection != null) {
-				resourceManagerConnection.close();
-				resourceManagerConnection = null;
-			}
-		}
-
-		// establish a connection to the new leader
-		if (newLeaderAddress != null) {
-			log.info("Attempting to register at ResourceManager {}", newLeaderAddress);
-			resourceManagerConnection =
-				new TaskExecutorToResourceManagerConnection(log, this, newLeaderAddress, newLeaderId);
-			resourceManagerConnection.start();
-		}
-	}
-
-	/**
-	 * Starts and runs the TaskManager.
-	 * <p/>
-	 * This method first tries to select the network interface to use for the TaskManager
-	 * communication. The network interface is used both for the actor communication
-	 * (coordination) as well as for the data exchange between task managers. Unless
-	 * the hostname/interface is explicitly configured in the configuration, this
-	 * method will try out various interfaces and methods to connect to the JobManager
-	 * and select the one where the connection attempt is successful.
-	 * <p/>
-	 * After selecting the network interface, this method brings up an actor system
-	 * for the TaskManager and its actors, starts the TaskManager's services
-	 * (library cache, shuffle network stack, ...), and starts the TaskManager itself.
-	 *
-	 * @param configuration    The configuration for the TaskManager.
-	 * @param resourceID       The id of the resource which the task manager will run on.
-	 */
-	public static void selectNetworkInterfaceAndRunTaskManager(
-		Configuration configuration,
-		ResourceID resourceID) throws Exception {
-
-		final InetSocketAddress taskManagerAddress = selectNetworkInterfaceAndPort(configuration);
-
-		runTaskManager(taskManagerAddress.getHostName(), resourceID, taskManagerAddress.getPort(), configuration);
-	}
-
-	private static InetSocketAddress selectNetworkInterfaceAndPort(Configuration configuration)
-		throws Exception {
-		String taskManagerHostname = configuration.getString(ConfigConstants.TASK_MANAGER_HOSTNAME_KEY, null);
-		if (taskManagerHostname != null) {
-			LOG.info("Using configured hostname/address for TaskManager: " + taskManagerHostname);
-		} else {
-			LeaderRetrievalService leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(configuration);
-			FiniteDuration lookupTimeout = AkkaUtils.getLookupTimeout(configuration);
-
-			InetAddress taskManagerAddress = LeaderRetrievalUtils.findConnectingAddress(leaderRetrievalService, lookupTimeout);
-			taskManagerHostname = taskManagerAddress.getHostName();
-			LOG.info("TaskManager will use hostname/address '{}' ({}) for communication.",
-				taskManagerHostname, taskManagerAddress.getHostAddress());
-		}
-
-		// if no task manager port has been configured, use 0 (system will pick any free port)
-		final int actorSystemPort = configuration.getInteger(ConfigConstants.TASK_MANAGER_IPC_PORT_KEY, 0);
-		if (actorSystemPort < 0 || actorSystemPort > 65535) {
-			throw new IllegalConfigurationException("Invalid value for '" +
-				ConfigConstants.TASK_MANAGER_IPC_PORT_KEY +
-				"' (port for the TaskManager actor system) : " + actorSystemPort +
-				" - Leave config parameter empty or use 0 to let the system choose a port automatically.");
-		}
-
-		return new InetSocketAddress(taskManagerHostname, actorSystemPort);
-	}
-
-	/**
-	 * Starts and runs the TaskManager. Brings up an actor system for the TaskManager and its
-	 * actors, starts the TaskManager's services (library cache, shuffle network stack, ...),
-	 * and starts the TaskManager itself.
-	 * <p/>
-	 * This method will also spawn a process reaper for the TaskManager (kill the process if
-	 * the actor fails) and optionally start the JVM memory logging thread.
-	 *
-	 * @param taskManagerHostname The hostname/address of the interface where the actor system
-	 *                            will communicate.
-	 * @param resourceID          The id of the resource which the task manager will run on.
-	 * @param actorSystemPort   The port at which the actor system will communicate.
-	 * @param configuration       The configuration for the TaskManager.
-	 */
-	private static void runTaskManager(
-		String taskManagerHostname,
-		ResourceID resourceID,
-		int actorSystemPort,
-		final Configuration configuration) throws Exception {
-
-		LOG.info("Starting TaskManager");
-
-		// Bring up the TaskManager actor system first, bind it to the given address.
-
-		LOG.info("Starting TaskManager actor system at " +
-			NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort));
-
-		final ActorSystem taskManagerSystem;
-		try {
-			Tuple2<String, Object> address = new Tuple2<String, Object>(taskManagerHostname, actorSystemPort);
-			Config akkaConfig = AkkaUtils.getAkkaConfig(configuration, new Some<>(address));
-			LOG.debug("Using akka configuration\n " + akkaConfig);
-			taskManagerSystem = AkkaUtils.createActorSystem(akkaConfig);
-		} catch (Throwable t) {
-			if (t instanceof org.jboss.netty.channel.ChannelException) {
-				Throwable cause = t.getCause();
-				if (cause != null && t.getCause() instanceof java.net.BindException) {
-					String address = NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort);
-					throw new IOException("Unable to bind TaskManager actor system to address " +
-						address + " - " + cause.getMessage(), t);
-				}
-			}
-			throw new Exception("Could not create TaskManager actor system", t);
-		}
-
-		// start akka rpc service based on actor system
-		final Timeout timeout = new Timeout(AkkaUtils.getTimeout(configuration).toMillis(), TimeUnit.MILLISECONDS);
-		final AkkaRpcService akkaRpcService = new AkkaRpcService(taskManagerSystem, timeout);
-
-		// start high availability service to implement getResourceManagerLeaderRetriever method only
-		final HighAvailabilityServices haServices = new HighAvailabilityServices() {
-			@Override
-			public LeaderRetrievalService getResourceManagerLeaderRetriever() throws Exception {
-				return LeaderRetrievalUtils.createLeaderRetrievalService(configuration);
-			}
-
-			@Override
-			public LeaderElectionService getResourceManagerLeaderElectionService() throws Exception {
-				return null;
-			}
-
-			@Override
-			public LeaderElectionService getJobMasterLeaderElectionService(JobID jobID) throws Exception {
-				return null;
-			}
-		};
-
-		// start all the TaskManager services (network stack,  library cache, ...)
-		// and the TaskManager actor
-		try {
-			LOG.info("Starting TaskManager actor");
-			TaskExecutor taskExecutor = startTaskManagerComponentsAndActor(
-				configuration,
-				resourceID,
-				akkaRpcService,
-				taskManagerHostname,
-				haServices,
-				false);
-
-			taskExecutor.start();
-
-			// if desired, start the logging daemon that periodically logs the memory usage information
-			if (LOG.isInfoEnabled() && configuration.getBoolean(
-				ConfigConstants.TASK_MANAGER_DEBUG_MEMORY_USAGE_START_LOG_THREAD,
-				ConfigConstants.DEFAULT_TASK_MANAGER_DEBUG_MEMORY_USAGE_START_LOG_THREAD)) {
-				LOG.info("Starting periodic memory usage logger");
-
-				long interval = configuration.getLong(
-					ConfigConstants.TASK_MANAGER_DEBUG_MEMORY_USAGE_LOG_INTERVAL_MS,
-					ConfigConstants.DEFAULT_TASK_MANAGER_DEBUG_MEMORY_USAGE_LOG_INTERVAL_MS);
-
-				MemoryLogger logger = new MemoryLogger(LOG, interval, taskManagerSystem);
-				logger.start();
-			}
-
-			// block until everything is done
-			taskManagerSystem.awaitTermination();
-		} catch (Throwable t) {
-			LOG.error("Error while starting up taskManager", t);
-			try {
-				taskManagerSystem.shutdown();
-			} catch (Throwable tt) {
-				LOG.warn("Could not cleanly shut down actor system", tt);
-			}
-			throw t;
-		}
-	}
-
-	// --------------------------------------------------------------------------
-	//  Starting and running the TaskManager
-	// --------------------------------------------------------------------------
-
-	/**
-	 * @param configuration                 The configuration for the TaskManager.
-	 * @param resourceID                    The id of the resource which the task manager will run on.
-	 * @param rpcService                  The rpc service which is used to start and connect to the TaskManager RpcEndpoint .
-	 * @param taskManagerHostname       The hostname/address that describes the TaskManager's data location.
-	 * @param haServices        Optionally, a high availability service can be provided. If none is given,
-	 *                                      then a HighAvailabilityServices is constructed from the configuration.
-	 * @param localTaskManagerCommunication     If true, the TaskManager will not initiate the TCP network stack.
-	 * @return An ActorRef to the TaskManager actor.
-	 * @throws org.apache.flink.configuration.IllegalConfigurationException     Thrown, if the given config contains illegal values.
-	 * @throws java.io.IOException      Thrown, if any of the I/O components (such as buffer pools,
-	 *                                       I/O manager, ...) cannot be properly started.
-	 * @throws java.lang.Exception      Thrown is some other error occurs while parsing the configuration
-	 *                                      or starting the TaskManager components.
-	 */
-	public static TaskExecutor startTaskManagerComponentsAndActor(
-		Configuration configuration,
-		ResourceID resourceID,
-		RpcService rpcService,
-		String taskManagerHostname,
-		HighAvailabilityServices haServices,
-		boolean localTaskManagerCommunication) throws Exception {
-
-		final TaskExecutorConfiguration taskExecutorConfig = parseTaskManagerConfiguration(
-			configuration, taskManagerHostname, localTaskManagerCommunication);
-
-		MemoryType memType = taskExecutorConfig.getNetworkConfig().memoryType();
-
-		// pre-start checks
-		checkTempDirs(taskExecutorConfig.getTmpDirPaths());
-
-		ExecutionContext executionContext = ExecutionContexts$.MODULE$.fromExecutor(new ForkJoinPool());
-
-		// we start the network first, to make sure it can allocate its buffers first
-		final NetworkEnvironment network = new NetworkEnvironment(
-			executionContext,
-			taskExecutorConfig.getTimeout(),
-			taskExecutorConfig.getNetworkConfig(),
-			taskExecutorConfig.getConnectionInfo());
-
-		// computing the amount of memory to use depends on how much memory is available
-		// it strictly needs to happen AFTER the network stack has been initialized
-
-		// check if a value has been configured
-		long configuredMemory = configuration.getLong(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, -1L);
-		checkConfigParameter(configuredMemory == -1 || configuredMemory > 0, configuredMemory,
-			ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY,
-			"MemoryManager needs at least one MB of memory. " +
-				"If you leave this config parameter empty, the system automatically " +
-				"pick a fraction of the available memory.");
-
-		final long memorySize;
-		boolean preAllocateMemory = configuration.getBoolean(
-			ConfigConstants.TASK_MANAGER_MEMORY_PRE_ALLOCATE_KEY,
-			ConfigConstants.DEFAULT_TASK_MANAGER_MEMORY_PRE_ALLOCATE);
-		if (configuredMemory > 0) {
-			if (preAllocateMemory) {
-				LOG.info("Using {} MB for managed memory." , configuredMemory);
-			} else {
-				LOG.info("Limiting managed memory to {} MB, memory will be allocated lazily." , configuredMemory);
-			}
-			memorySize = configuredMemory << 20; // megabytes to bytes
-		} else {
-			float fraction = configuration.getFloat(
-				ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY,
-				ConfigConstants.DEFAULT_MEMORY_MANAGER_MEMORY_FRACTION);
-			checkConfigParameter(fraction > 0.0f && fraction < 1.0f, fraction,
-				ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY,
-				"MemoryManager fraction of the free memory must be between 0.0 and 1.0");
-
-			if (memType == MemoryType.HEAP) {
-				long relativeMemSize = (long) (EnvironmentInformation.getSizeOfFreeHeapMemoryWithDefrag() * fraction);
-				if (preAllocateMemory) {
-					LOG.info("Using {} of the currently free heap space for managed heap memory ({} MB)." ,
-						fraction , relativeMemSize >> 20);
-				} else {
-					LOG.info("Limiting managed memory to {} of the currently free heap space ({} MB), " +
-						"memory will be allocated lazily." , fraction , relativeMemSize >> 20);
-				}
-				memorySize = relativeMemSize;
-			} else if (memType == MemoryType.OFF_HEAP) {
-				// The maximum heap memory has been adjusted according to the fraction
-				long maxMemory = EnvironmentInformation.getMaxJvmHeapMemory();
-				long directMemorySize = (long) (maxMemory / (1.0 - fraction) * fraction);
-				if (preAllocateMemory) {
-					LOG.info("Using {} of the maximum memory size for managed off-heap memory ({} MB)." ,
-						fraction, directMemorySize >> 20);
-				} else {
-					LOG.info("Limiting managed memory to {} of the maximum memory size ({} MB)," +
-						" memory will be allocated lazily.", fraction, directMemorySize >> 20);
-				}
-				memorySize = directMemorySize;
-			} else {
-				throw new RuntimeException("No supported memory type detected.");
-			}
-		}
-
-		// now start the memory manager
-		final MemoryManager memoryManager;
-		try {
-			memoryManager = new MemoryManager(
-				memorySize,
-				taskExecutorConfig.getNumberOfSlots(),
-				taskExecutorConfig.getNetworkConfig().networkBufferSize(),
-				memType,
-				preAllocateMemory);
-		} catch (OutOfMemoryError e) {
-			if (memType == MemoryType.HEAP) {
-				throw new Exception("OutOfMemory error (" + e.getMessage() +
-					") while allocating the TaskManager heap memory (" + memorySize + " bytes).", e);
-			} else if (memType == MemoryType.OFF_HEAP) {
-				throw new Exception("OutOfMemory error (" + e.getMessage() +
-					") while allocating the TaskManager off-heap memory (" + memorySize +
-					" bytes).Try increasing the maximum direct memory (-XX:MaxDirectMemorySize)", e);
-			} else {
-				throw e;
-			}
-		}
-
-		// start the I/O manager, it will create some temp directories.
-		final IOManager ioManager = new IOManagerAsync(taskExecutorConfig.getTmpDirPaths());
-
-		final TaskExecutor taskExecutor = new TaskExecutor(
-			taskExecutorConfig,
-			resourceID,
-			memoryManager,
-			ioManager,
-			network,
-			taskExecutorConfig.getNumberOfSlots(),
-			rpcService,
-			haServices);
-
-		return taskExecutor;
-	}
-
-	// --------------------------------------------------------------------------
-	//  Parsing and checking the TaskManager Configuration
-	// --------------------------------------------------------------------------
-
-	/**
-	 * Utility method to extract TaskManager config parameters from the configuration and to
-	 * sanity check them.
-	 *
-	 * @param configuration                 The configuration.
-	 * @param taskManagerHostname           The host name under which the TaskManager communicates.
-	 * @param localTaskManagerCommunication             True, to skip initializing the network stack.
-	 *                                      Use only in cases where only one task manager runs.
-	 * @return TaskExecutorConfiguration that wrappers InstanceConnectionInfo, NetworkEnvironmentConfiguration, etc.
-	 */
-	private static TaskExecutorConfiguration parseTaskManagerConfiguration(
-		Configuration configuration,
-		String taskManagerHostname,
-		boolean localTaskManagerCommunication) throws Exception {
-
-		// ------- read values from the config and check them ---------
-		//                      (a lot of them)
-
-		// ----> hosts / ports for communication and data exchange
-
-		int dataport = configuration.getInteger(ConfigConstants.TASK_MANAGER_DATA_PORT_KEY,
-			ConfigConstants.DEFAULT_TASK_MANAGER_DATA_PORT);
-		if (dataport == 0) {
-			dataport = NetUtils.getAvailablePort();
-		}
-		checkConfigParameter(dataport > 0, dataport, ConfigConstants.TASK_MANAGER_DATA_PORT_KEY,
-			"Leave config parameter empty or use 0 to let the system choose a port automatically.");
-
-		InetAddress taskManagerAddress = InetAddress.getByName(taskManagerHostname);
-		final InstanceConnectionInfo connectionInfo = new InstanceConnectionInfo(taskManagerAddress, dataport);
-
-		// ----> memory / network stack (shuffles/broadcasts), task slots, temp directories
-
-		// we need this because many configs have been written with a "-1" entry
-		int slots = configuration.getInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 1);
-		if (slots == -1) {
-			slots = 1;
-		}
-		checkConfigParameter(slots >= 1, slots, ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS,
-			"Number of task slots must be at least one.");
-
-		final int numNetworkBuffers = configuration.getInteger(
-			ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY,
-			ConfigConstants.DEFAULT_TASK_MANAGER_NETWORK_NUM_BUFFERS);
-		checkConfigParameter(numNetworkBuffers > 0, numNetworkBuffers,
-			ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY, "");
-
-		final int pageSize = configuration.getInteger(
-			ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY,
-			ConfigConstants.DEFAULT_TASK_MANAGER_MEMORY_SEGMENT_SIZE);
-		// check page size of for minimum size
-		checkConfigParameter(pageSize >= MemoryManager.MIN_PAGE_SIZE, pageSize,
-			ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY,
-			"Minimum memory segment size is " + MemoryManager.MIN_PAGE_SIZE);
-		// check page size for power of two
-		checkConfigParameter(MathUtils.isPowerOf2(pageSize), pageSize,
-			ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY,
-			"Memory segment size must be a power of 2.");
-
-		// check whether we use heap or off-heap memory
-		final MemoryType memType;
-		if (configuration.getBoolean(ConfigConstants.TASK_MANAGER_MEMORY_OFF_HEAP_KEY, false)) {
-			memType = MemoryType.OFF_HEAP;
-		} else {
-			memType = MemoryType.HEAP;
-		}
-
-		// initialize the memory segment factory accordingly
-		if (memType == MemoryType.HEAP) {
-			if (!MemorySegmentFactory.initializeIfNotInitialized(HeapMemorySegment.FACTORY)) {
-				throw new Exception("Memory type is set to heap memory, but memory segment " +
-					"factory has been initialized for off-heap memory segments");
-			}
-		} else {
-			if (!MemorySegmentFactory.initializeIfNotInitialized(HybridMemorySegment.FACTORY)) {
-				throw new Exception("Memory type is set to off-heap memory, but memory segment " +
-					"factory has been initialized for heap memory segments");
-			}
-		}
-
-		final String[] tmpDirs = configuration.getString(
-			ConfigConstants.TASK_MANAGER_TMP_DIR_KEY,
-			ConfigConstants.DEFAULT_TASK_MANAGER_TMP_PATH).split(",|" + File.pathSeparator);
-
-		final NettyConfig nettyConfig;
-		if (!localTaskManagerCommunication) {
-			nettyConfig = new NettyConfig(connectionInfo.address(), connectionInfo.dataPort(), pageSize, slots, configuration);
-		} else {
-			nettyConfig = null;
-		}
-
-		// Default spill I/O mode for intermediate results
-		final String syncOrAsync = configuration.getString(
-			ConfigConstants.TASK_MANAGER_NETWORK_DEFAULT_IO_MODE,
-			ConfigConstants.DEFAULT_TASK_MANAGER_NETWORK_DEFAULT_IO_MODE);
-
-		final IOMode ioMode;
-		if (syncOrAsync.equals("async")) {
-			ioMode = IOManager.IOMode.ASYNC;
-		} else {
-			ioMode = IOManager.IOMode.SYNC;
-		}
-
-		final int queryServerPort =  configuration.getInteger(
-			ConfigConstants.QUERYABLE_STATE_SERVER_PORT,
-			ConfigConstants.DEFAULT_QUERYABLE_STATE_SERVER_PORT);
-
-		final int queryServerNetworkThreads =  configuration.getInteger(
-			ConfigConstants.QUERYABLE_STATE_SERVER_NETWORK_THREADS,
-			ConfigConstants.DEFAULT_QUERYABLE_STATE_SERVER_NETWORK_THREADS);
-
-		final int queryServerQueryThreads =  configuration.getInteger(
-			ConfigConstants.QUERYABLE_STATE_SERVER_QUERY_THREADS,
-			ConfigConstants.DEFAULT_QUERYABLE_STATE_SERVER_QUERY_THREADS);
-
-		final NetworkEnvironmentConfiguration networkConfig = new NetworkEnvironmentConfiguration(
-			numNetworkBuffers,
-			pageSize,
-			memType,
-			ioMode,
-			queryServerPort,
-			queryServerNetworkThreads,
-			queryServerQueryThreads,
-			localTaskManagerCommunication ? Option.<NettyConfig>empty() : new Some<>(nettyConfig),
-			new Tuple2<>(500, 3000));
-
-		// ----> timeouts, library caching, profiling
-
-		final FiniteDuration timeout;
-		try {
-			timeout = AkkaUtils.getTimeout(configuration);
-		} catch (Exception e) {
-			throw new IllegalArgumentException(
-				"Invalid format for '" + ConfigConstants.AKKA_ASK_TIMEOUT +
-					"'.Use formats like '50 s' or '1 min' to specify the timeout.");
-		}
-		LOG.info("Messages between TaskManager and JobManager have a max timeout of " + timeout);
-
-		final long cleanupInterval = configuration.getLong(
-			ConfigConstants.LIBRARY_CACHE_MANAGER_CLEANUP_INTERVAL,
-			ConfigConstants.DEFAULT_LIBRARY_CACHE_MANAGER_CLEANUP_INTERVAL) * 1000;
-
-		final FiniteDuration finiteRegistrationDuration;
-		try {
-			Duration maxRegistrationDuration = Duration.create(configuration.getString(
-				ConfigConstants.TASK_MANAGER_MAX_REGISTRATION_DURATION,
-				ConfigConstants.DEFAULT_TASK_MANAGER_MAX_REGISTRATION_DURATION));
-			if (maxRegistrationDuration.isFinite()) {
-				finiteRegistrationDuration = new FiniteDuration(maxRegistrationDuration.toSeconds(), TimeUnit.SECONDS);
-			} else {
-				finiteRegistrationDuration = null;
-			}
-		} catch (NumberFormatException e) {
-			throw new IllegalArgumentException("Invalid format for parameter " +
-				ConfigConstants.TASK_MANAGER_MAX_REGISTRATION_DURATION, e);
-		}
-
-		final FiniteDuration initialRegistrationPause;
-		try {
-			Duration pause = Duration.create(configuration.getString(
-				ConfigConstants.TASK_MANAGER_INITIAL_REGISTRATION_PAUSE,
-				ConfigConstants.DEFAULT_TASK_MANAGER_INITIAL_REGISTRATION_PAUSE));
-			if (pause.isFinite()) {
-				initialRegistrationPause = new FiniteDuration(pause.toSeconds(), TimeUnit.SECONDS);
-			} else {
-				throw new IllegalArgumentException("The initial registration pause must be finite: " + pause);
-			}
-		} catch (NumberFormatException e) {
-			throw new IllegalArgumentException("Invalid format for parameter " +
-				ConfigConstants.TASK_MANAGER_INITIAL_REGISTRATION_PAUSE, e);
-		}
-
-		final FiniteDuration maxRegistrationPause;
-		try {
-			Duration pause = Duration.create(configuration.getString(
-				ConfigConstants.TASK_MANAGER_MAX_REGISTARTION_PAUSE,
-				ConfigConstants.DEFAULT_TASK_MANAGER_MAX_REGISTRATION_PAUSE));
-			if (pause.isFinite()) {
-				maxRegistrationPause = new FiniteDuration(pause.toSeconds(), TimeUnit.SECONDS);
-			} else {
-				throw new IllegalArgumentException("The maximum registration pause must be finite: " + pause);
-			}
-		} catch (NumberFormatException e) {
-			throw new IllegalArgumentException("Invalid format for parameter " +
-				ConfigConstants.TASK_MANAGER_INITIAL_REGISTRATION_PAUSE, e);
-		}
-
-		final FiniteDuration refusedRegistrationPause;
-		try {
-			Duration pause = Duration.create(configuration.getString(
-				ConfigConstants.TASK_MANAGER_REFUSED_REGISTRATION_PAUSE,
-				ConfigConstants.DEFAULT_TASK_MANAGER_REFUSED_REGISTRATION_PAUSE));
-			if (pause.isFinite()) {
-				refusedRegistrationPause = new FiniteDuration(pause.toSeconds(), TimeUnit.SECONDS);
-			} else {
-				throw new IllegalArgumentException("The refused registration pause must be finite: " + pause);
-			}
-		} catch (NumberFormatException e) {
-			throw new IllegalArgumentException("Invalid format for parameter " +
-				ConfigConstants.TASK_MANAGER_INITIAL_REGISTRATION_PAUSE, e);
-		}
-
-		return new TaskExecutorConfiguration(
-			tmpDirs,
-			cleanupInterval,
-			connectionInfo,
-			networkConfig,
-			timeout,
-			finiteRegistrationDuration,
-			slots,
-			configuration,
-			initialRegistrationPause,
-			maxRegistrationPause,
-			refusedRegistrationPause);
-	}
-
-	/**
-	 * Validates a condition for a config parameter and displays a standard exception, if the
-	 * the condition does not hold.
-	 *
-	 * @param condition    The condition that must hold. If the condition is false, an exception is thrown.
-	 * @param parameter    The parameter value. Will be shown in the exception message.
-	 * @param name         The name of the config parameter. Will be shown in the exception message.
-	 * @param errorMessage The optional custom error message to append to the exception message.
-	 */
-	private static void checkConfigParameter(
-		boolean condition,
-		Object parameter,
-		String name,
-		String errorMessage) {
-		if (!condition) {
-			throw new IllegalConfigurationException("Invalid configuration value for " + name + " : " + parameter + " - " + errorMessage);
-		}
-	}
-
-	/**
-	 * Validates that all the directories denoted by the strings do actually exist, are proper
-	 * directories (not files), and are writable.
-	 *
-	 * @param tmpDirs The array of directory paths to check.
-	 * @throws Exception Thrown if any of the directories does not exist or is not writable
-	 *                   or is a file, rather than a directory.
-	 */
-	private static void checkTempDirs(String[] tmpDirs) throws IOException {
-		for (String dir : tmpDirs) {
-			if (dir != null && !dir.equals("")) {
-				File file = new File(dir);
-				if (!file.exists()) {
-					throw new IOException("Temporary file directory " + file.getAbsolutePath() + " does not exist.");
-				}
-				if (!file.isDirectory()) {
-					throw new IOException("Temporary file directory " + file.getAbsolutePath() + " is not a directory.");
-				}
-				if (!file.canWrite()) {
-					throw new IOException("Temporary file directory " + file.getAbsolutePath() + " is not writable.");
-				}
-
-				if (LOG.isInfoEnabled()) {
-					long totalSpaceGb = file.getTotalSpace() >> 30;
-					long usableSpaceGb = file.getUsableSpace() >> 30;
-					double usablePercentage = (double)usableSpaceGb / totalSpaceGb * 100;
-					String path = file.getAbsolutePath();
-					LOG.info(String.format("Temporary file directory '%s': total %d GB, " + "usable %d GB (%.2f%% usable)",
-						path, totalSpaceGb, usableSpaceGb, usablePercentage));
-				}
-			} else {
-				throw new IllegalArgumentException("Temporary file directory #$id is null.");
-			}
-		}
-	}
-
-	// ------------------------------------------------------------------------
-	//  Properties
-	// ------------------------------------------------------------------------
-
-	public ResourceID getResourceID() {
-		return resourceID;
-	}
-
-	// ------------------------------------------------------------------------
-	//  Error Handling
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Notifies the TaskExecutor that a fatal error has occurred and it cannot proceed.
-	 * This method should be used when asynchronous threads want to notify the
-	 * TaskExecutor 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 TaskExecutor that a fatal error has occurred and it cannot proceed.
-	 * This method must only be called from within the TaskExecutor's main thread.
-	 *
-	 * @param t The exception describing the fatal error
-	 */
-	void onFatalError(Throwable t) {
-		// to be determined, probably delegate to a fatal error handler that 
-		// would either log (mini cluster) ot kill the process (yarn, mesos, ...)
-		log.error("FATAL ERROR", t);
-	}
-
-	// ------------------------------------------------------------------------
-	//  Access to fields for testing
-	// ------------------------------------------------------------------------
-
-	@VisibleForTesting
-	TaskExecutorToResourceManagerConnection getResourceManagerConnection() {
-		return resourceManagerConnection;
-	}
-
-	// ------------------------------------------------------------------------
-	//  Utility classes
-	// ------------------------------------------------------------------------
-
-	/**
-	 * The listener for leader changes of the resource manager
-	 */
-	private class ResourceManagerLeaderListener implements LeaderRetrievalListener {
-
-		@Override
-		public void notifyLeaderAddress(String leaderAddress, UUID leaderSessionID) {
-			getSelf().notifyOfNewResourceManagerLeader(leaderAddress, leaderSessionID);
-		}
-
-		@Override
-		public void handleError(Exception exception) {
-			onFatalErrorAsync(exception);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9c247d1f/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorConfiguration.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorConfiguration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorConfiguration.java
deleted file mode 100644
index 32484e1..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorConfiguration.java
+++ /dev/null
@@ -1,151 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.runtime.rpc.taskexecutor;
-
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.instance.InstanceConnectionInfo;
-import org.apache.flink.runtime.taskmanager.NetworkEnvironmentConfiguration;
-
-import scala.concurrent.duration.FiniteDuration;
-
-import java.io.Serializable;
-import java.util.concurrent.TimeUnit;
-
-import static org.apache.flink.util.Preconditions.checkNotNull;
-
-/**
- * {@link TaskExecutor} Configuration
- */
-public class TaskExecutorConfiguration implements Serializable {
-
-	private static final long serialVersionUID = 1L;
-
-	private final String[] tmpDirPaths;
-
-	private final long cleanupInterval;
-
-	private final int numberOfSlots;
-
-	private final Configuration configuration;
-
-	private final FiniteDuration timeout;
-	private final FiniteDuration maxRegistrationDuration;
-	private final FiniteDuration initialRegistrationPause;
-	private final FiniteDuration maxRegistrationPause;
-	private final FiniteDuration refusedRegistrationPause;
-
-	private final NetworkEnvironmentConfiguration networkConfig;
-
-	private final InstanceConnectionInfo connectionInfo;
-
-	public TaskExecutorConfiguration(
-			String[] tmpDirPaths,
-			long cleanupInterval,
-			InstanceConnectionInfo connectionInfo,
-			NetworkEnvironmentConfiguration networkConfig,
-			FiniteDuration timeout,
-			FiniteDuration maxRegistrationDuration,
-			int numberOfSlots,
-			Configuration configuration) {
-
-		this (tmpDirPaths,
-			cleanupInterval,
-			connectionInfo,
-			networkConfig,
-			timeout,
-			maxRegistrationDuration,
-			numberOfSlots,
-			configuration,
-			new FiniteDuration(500, TimeUnit.MILLISECONDS),
-			new FiniteDuration(30, TimeUnit.SECONDS),
-			new FiniteDuration(10, TimeUnit.SECONDS));
-	}
-
-	public TaskExecutorConfiguration(
-			String[] tmpDirPaths,
-			long cleanupInterval,
-			InstanceConnectionInfo connectionInfo,
-			NetworkEnvironmentConfiguration networkConfig,
-			FiniteDuration timeout,
-			FiniteDuration maxRegistrationDuration,
-			int numberOfSlots,
-			Configuration configuration,
-			FiniteDuration initialRegistrationPause,
-			FiniteDuration maxRegistrationPause,
-			FiniteDuration refusedRegistrationPause) {
-
-		this.tmpDirPaths = checkNotNull(tmpDirPaths);
-		this.cleanupInterval = checkNotNull(cleanupInterval);
-		this.connectionInfo = checkNotNull(connectionInfo);
-		this.networkConfig = checkNotNull(networkConfig);
-		this.timeout = checkNotNull(timeout);
-		this.maxRegistrationDuration = maxRegistrationDuration;
-		this.numberOfSlots = checkNotNull(numberOfSlots);
-		this.configuration = checkNotNull(configuration);
-		this.initialRegistrationPause = checkNotNull(initialRegistrationPause);
-		this.maxRegistrationPause = checkNotNull(maxRegistrationPause);
-		this.refusedRegistrationPause = checkNotNull(refusedRegistrationPause);
-	}
-
-	// --------------------------------------------------------------------------------------------
-	//  Properties
-	// --------------------------------------------------------------------------------------------
-
-	public String[] getTmpDirPaths() {
-		return tmpDirPaths;
-	}
-
-	public long getCleanupInterval() {
-		return cleanupInterval;
-	}
-
-	public InstanceConnectionInfo getConnectionInfo() { return connectionInfo; }
-
-	public NetworkEnvironmentConfiguration getNetworkConfig() { return networkConfig; }
-
-	public FiniteDuration getTimeout() {
-		return timeout;
-	}
-
-	public FiniteDuration getMaxRegistrationDuration() {
-		return maxRegistrationDuration;
-	}
-
-	public int getNumberOfSlots() {
-		return numberOfSlots;
-	}
-
-	public Configuration getConfiguration() {
-		return configuration;
-	}
-
-	public FiniteDuration getInitialRegistrationPause() {
-		return initialRegistrationPause;
-	}
-
-	public FiniteDuration getMaxRegistrationPause() {
-		return maxRegistrationPause;
-	}
-
-	public FiniteDuration getRefusedRegistrationPause() {
-		return refusedRegistrationPause;
-	}
-
-}
-

http://git-wip-us.apache.org/repos/asf/flink/blob/9c247d1f/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorGateway.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorGateway.java
deleted file mode 100644
index b0b21b0..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorGateway.java
+++ /dev/null
@@ -1,35 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.runtime.rpc.taskexecutor;
-
-import org.apache.flink.runtime.rpc.RpcGateway;
-
-import java.util.UUID;
-
-/**
- * {@link TaskExecutor} RPC gateway interface
- */
-public interface TaskExecutorGateway extends RpcGateway {
-
-	// ------------------------------------------------------------------------
-	//  ResourceManager handlers
-	// ------------------------------------------------------------------------
-
-	void notifyOfNewResourceManagerLeader(String address, UUID resourceManagerLeaderId);
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9c247d1f/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorRegistrationSuccess.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorRegistrationSuccess.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorRegistrationSuccess.java
deleted file mode 100644
index 641102d..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorRegistrationSuccess.java
+++ /dev/null
@@ -1,75 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.runtime.rpc.taskexecutor;
-
-import org.apache.flink.runtime.instance.InstanceID;
-import org.apache.flink.runtime.rpc.registration.RegistrationResponse;
-
-import java.io.Serializable;
-
-/**
- * Base class for responses from the ResourceManager to a registration attempt by a
- * TaskExecutor.
- */
-public final class TaskExecutorRegistrationSuccess extends RegistrationResponse.Success implements Serializable {
-
-	private static final long serialVersionUID = 1L;
-
-	private final InstanceID registrationId;
-
-	private final long heartbeatInterval;
-
-	/**
-	 * Create a new {@code TaskExecutorRegistrationSuccess} message.
-	 * 
-	 * @param registrationId     The ID that the ResourceManager assigned the registration.
-	 * @param heartbeatInterval  The interval in which the ResourceManager will heartbeat the TaskExecutor.
-	 */
-	public TaskExecutorRegistrationSuccess(InstanceID registrationId, long heartbeatInterval) {
-		this.registrationId = registrationId;
-		this.heartbeatInterval = heartbeatInterval;
-	}
-
-	/**
-	 * Gets the ID that the ResourceManager assigned the registration.
-	 */
-	public InstanceID getRegistrationId() {
-		return registrationId;
-	}
-
-	/**
-	 * Gets the interval in which the ResourceManager will heartbeat the TaskExecutor.
-	 */
-	public long getHeartbeatInterval() {
-		return heartbeatInterval;
-	}
-
-	@Override
-	public String toString() {
-		return "TaskExecutorRegistrationSuccess (" + registrationId + " / " + heartbeatInterval + ')';
-	}
-
-}
-
-
-
-
-
-
-

http://git-wip-us.apache.org/repos/asf/flink/blob/9c247d1f/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorToResourceManagerConnection.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorToResourceManagerConnection.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorToResourceManagerConnection.java
deleted file mode 100644
index 7ccc879..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorToResourceManagerConnection.java
+++ /dev/null
@@ -1,198 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.runtime.rpc.taskexecutor;
-
-import akka.dispatch.OnFailure;
-import akka.dispatch.OnSuccess;
-
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.runtime.clusterframework.types.ResourceID;
-import org.apache.flink.runtime.instance.InstanceID;
-import org.apache.flink.runtime.rpc.RpcService;
-import org.apache.flink.runtime.rpc.registration.RegistrationResponse;
-import org.apache.flink.runtime.rpc.registration.RetryingRegistration;
-import org.apache.flink.runtime.rpc.resourcemanager.ResourceManagerGateway;
-
-import org.slf4j.Logger;
-
-import scala.concurrent.Future;
-import scala.concurrent.duration.FiniteDuration;
-
-import java.util.UUID;
-import java.util.concurrent.TimeUnit;
-
-import static org.apache.flink.util.Preconditions.checkNotNull;
-import static org.apache.flink.util.Preconditions.checkState;
-
-/**
- * The connection between a TaskExecutor and the ResourceManager.
- */
-public class TaskExecutorToResourceManagerConnection {
-
-	/** the logger for all log messages of this class */
-	private final Logger log;
-
-	/** the TaskExecutor whose connection to the ResourceManager this represents */
-	private final TaskExecutor taskExecutor;
-
-	private final UUID resourceManagerLeaderId;
-
-	private final String resourceManagerAddress;
-
-	private TaskExecutorToResourceManagerConnection.ResourceManagerRegistration pendingRegistration;
-
-	private ResourceManagerGateway registeredResourceManager;
-
-	private InstanceID registrationId;
-
-	/** flag indicating that the connection is closed */
-	private volatile boolean closed;
-
-
-	public TaskExecutorToResourceManagerConnection(
-			Logger log,
-			TaskExecutor taskExecutor,
-			String resourceManagerAddress,
-			UUID resourceManagerLeaderId) {
-
-		this.log = checkNotNull(log);
-		this.taskExecutor = checkNotNull(taskExecutor);
-		this.resourceManagerAddress = checkNotNull(resourceManagerAddress);
-		this.resourceManagerLeaderId = checkNotNull(resourceManagerLeaderId);
-	}
-
-	// ------------------------------------------------------------------------
-	//  Life cycle
-	// ------------------------------------------------------------------------
-
-	@SuppressWarnings("unchecked")
-	public void start() {
-		checkState(!closed, "The connection is already closed");
-		checkState(!isRegistered() && pendingRegistration == null, "The connection is already started");
-
-		pendingRegistration = new TaskExecutorToResourceManagerConnection.ResourceManagerRegistration(
-				log, taskExecutor.getRpcService(),
-				resourceManagerAddress, resourceManagerLeaderId,
-				taskExecutor.getAddress(), taskExecutor.getResourceID());
-		pendingRegistration.startRegistration();
-
-		Future<Tuple2<ResourceManagerGateway, TaskExecutorRegistrationSuccess>> future = pendingRegistration.getFuture();
-		
-		future.onSuccess(new OnSuccess<Tuple2<ResourceManagerGateway, TaskExecutorRegistrationSuccess>>() {
-			@Override
-			public void onSuccess(Tuple2<ResourceManagerGateway, TaskExecutorRegistrationSuccess> result) {
-				registeredResourceManager = result.f0;
-				registrationId = result.f1.getRegistrationId();
-			}
-		}, taskExecutor.getMainThreadExecutionContext());
-		
-		// this future should only ever fail if there is a bug, not if the registration is declined
-		future.onFailure(new OnFailure() {
-			@Override
-			public void onFailure(Throwable failure) {
-				taskExecutor.onFatalError(failure);
-			}
-		}, taskExecutor.getMainThreadExecutionContext());
-	}
-
-	public void close() {
-		closed = true;
-
-		// make sure we do not keep re-trying forever
-		if (pendingRegistration != null) {
-			pendingRegistration.cancel();
-		}
-	}
-
-	public boolean isClosed() {
-		return closed;
-	}
-
-	// ------------------------------------------------------------------------
-	//  Properties
-	// ------------------------------------------------------------------------
-
-	public UUID getResourceManagerLeaderId() {
-		return resourceManagerLeaderId;
-	}
-
-	public String getResourceManagerAddress() {
-		return resourceManagerAddress;
-	}
-
-	/**
-	 * Gets the ResourceManagerGateway. This returns null until the registration is completed.
-	 */
-	public ResourceManagerGateway getResourceManager() {
-		return registeredResourceManager;
-	}
-
-	/**
-	 * Gets the ID under which the TaskExecutor is registered at the ResourceManager.
-	 * This returns null until the registration is completed.
-	 */
-	public InstanceID getRegistrationId() {
-		return registrationId;
-	}
-
-	public boolean isRegistered() {
-		return registeredResourceManager != null;
-	}
-
-	// ------------------------------------------------------------------------
-
-	@Override
-	public String toString() {
-		return String.format("Connection to ResourceManager %s (leaderId=%s)",
-				resourceManagerAddress, resourceManagerLeaderId); 
-	}
-
-	// ------------------------------------------------------------------------
-	//  Utilities
-	// ------------------------------------------------------------------------
-
-	private static class ResourceManagerRegistration
-			extends RetryingRegistration<ResourceManagerGateway, TaskExecutorRegistrationSuccess> {
-
-		private final String taskExecutorAddress;
-		
-		private final ResourceID resourceID;
-
-		ResourceManagerRegistration(
-				Logger log,
-				RpcService rpcService,
-				String targetAddress,
-				UUID leaderId,
-				String taskExecutorAddress,
-				ResourceID resourceID) {
-
-			super(log, rpcService, "ResourceManager", ResourceManagerGateway.class, targetAddress, leaderId);
-			this.taskExecutorAddress = checkNotNull(taskExecutorAddress);
-			this.resourceID = checkNotNull(resourceID);
-		}
-
-		@Override
-		protected Future<RegistrationResponse> invokeRegistration(
-				ResourceManagerGateway resourceManager, UUID leaderId, long timeoutMillis) throws Exception {
-
-			FiniteDuration timeout = new FiniteDuration(timeoutMillis, TimeUnit.MILLISECONDS);
-			return resourceManager.registerTaskExecutor(leaderId, taskExecutorAddress, resourceID, timeout);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9c247d1f/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/SlotReport.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/SlotReport.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/SlotReport.java
new file mode 100644
index 0000000..a5de2d5
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/SlotReport.java
@@ -0,0 +1,56 @@
+/*
+ * 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.runtime.clusterframework.types.ResourceID;
+
+import java.io.Serializable;
+import java.util.List;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * A report about the current status of all slots of the TaskExecutor, describing
+ * which slots are available and allocated, and what jobs (JobManagers) the allocated slots
+ * have been allocated to.
+ */
+public class SlotReport implements Serializable {
+
+	private static final long serialVersionUID = -3150175198722481689L;
+
+	/** The slots status of the TaskManager */
+	private final List<SlotStatus> slotsStatus;
+
+	/** The resource id which identifies the TaskManager */
+	private final ResourceID resourceID;
+
+	public SlotReport(final List<SlotStatus> slotsStatus, final ResourceID resourceID) {
+		this.slotsStatus = checkNotNull(slotsStatus);
+		this.resourceID = checkNotNull(resourceID);
+	}
+
+	public List<SlotStatus> getSlotsStatus() {
+		return slotsStatus;
+	}
+
+	public ResourceID getResourceID() {
+		return resourceID;
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9c247d1f/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/SlotStatus.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/SlotStatus.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/SlotStatus.java
new file mode 100644
index 0000000..744b674
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/SlotStatus.java
@@ -0,0 +1,129 @@
+/*
+ * 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.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+
+import java.io.Serializable;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * This describes the slot current status which located in TaskManager.
+ */
+public class SlotStatus implements Serializable {
+
+	private static final long serialVersionUID = 5099191707339664493L;
+
+	/** slotID to identify a slot */
+	private final SlotID slotID;
+
+	/** the resource profile of the slot */
+	private final ResourceProfile profiler;
+
+	/** if the slot is allocated, allocationId identify its allocation; else, allocationId is null */
+	private final AllocationID allocationID;
+
+	/** if the slot is allocated, jobId identify which job this slot is allocated to; else, jobId is null */
+	private final JobID jobID;
+
+	public SlotStatus(SlotID slotID, ResourceProfile profiler) {
+		this(slotID, profiler, null, null);
+	}
+
+	public SlotStatus(SlotID slotID, ResourceProfile profiler, AllocationID allocationID, JobID jobID) {
+		this.slotID = checkNotNull(slotID, "slotID cannot be null");
+		this.profiler = checkNotNull(profiler, "profile cannot be null");
+		this.allocationID = allocationID;
+		this.jobID = jobID;
+	}
+
+	/**
+	 * Get the unique identification of this slot
+	 *
+	 * @return The slot id
+	 */
+	public SlotID getSlotID() {
+		return slotID;
+	}
+
+	/**
+	 * Get the resource profile of this slot
+	 *
+	 * @return The resource profile
+	 */
+	public ResourceProfile getProfiler() {
+		return profiler;
+	}
+
+	/**
+	 * Get the allocation id of this slot
+	 *
+	 * @return The allocation id if this slot is allocated, otherwise null
+	 */
+	public AllocationID getAllocationID() {
+		return allocationID;
+	}
+
+	/**
+	 * Get the job id of the slot allocated for
+	 *
+	 * @return The job id if this slot is allocated, otherwise null
+	 */
+	public JobID getJobID() {
+		return jobID;
+	}
+
+	@Override
+	public boolean equals(Object o) {
+		if (this == o) {
+			return true;
+		}
+		if (o == null || getClass() != o.getClass()) {
+			return false;
+		}
+
+		SlotStatus that = (SlotStatus) o;
+
+		if (!slotID.equals(that.slotID)) {
+			return false;
+		}
+		if (!profiler.equals(that.profiler)) {
+			return false;
+		}
+		if (allocationID != null ? !allocationID.equals(that.allocationID) : that.allocationID != null) {
+			return false;
+		}
+		return jobID != null ? jobID.equals(that.jobID) : that.jobID == null;
+
+	}
+
+	@Override
+	public int hashCode() {
+		int result = slotID.hashCode();
+		result = 31 * result + profiler.hashCode();
+		result = 31 * result + (allocationID != null ? allocationID.hashCode() : 0);
+		result = 31 * result + (jobID != null ? jobID.hashCode() : 0);
+		return result;
+	}
+
+}


[10/50] [abbrv] flink git commit: [FLINK-4656] [rpc] Port the existing code to Flink's own future abstraction

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/507e86cf/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 01776ed..957453a 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,16 +18,13 @@
 
 package org.apache.flink.runtime.rpc;
 
-import akka.dispatch.ExecutionContexts;
 import akka.dispatch.Futures;
-import akka.util.Timeout;
+import org.apache.flink.api.common.time.Time;
 import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.runtime.concurrent.Future;
+import org.apache.flink.runtime.concurrent.impl.FlinkCompletableFuture;
 import org.apache.flink.runtime.util.DirectExecutorService;
 import org.apache.flink.util.Preconditions;
-import scala.concurrent.Await;
-import scala.concurrent.ExecutionContext;
-import scala.concurrent.Future;
-import scala.concurrent.duration.FiniteDuration;
 
 import java.lang.annotation.Annotation;
 import java.lang.reflect.InvocationHandler;
@@ -37,6 +34,7 @@ import java.util.BitSet;
 import java.util.UUID;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Executor;
 import java.util.concurrent.TimeUnit;
 
 import static org.apache.flink.util.Preconditions.checkNotNull;
@@ -67,8 +65,8 @@ public class TestingSerialRpcService implements RpcService {
 	}
 
 	@Override
-	public ExecutionContext getExecutionContext() {
-		return ExecutionContexts.fromExecutorService(executorService);
+	public Executor getExecutor() {
+		return executorService;
 	}
 
 	@Override
@@ -94,7 +92,7 @@ public class TestingSerialRpcService implements RpcService {
 			classLoader,
 			new Class<?>[]{
 				rpcEndpoint.getSelfGatewayType(),
-				MainThreadExecutor.class,
+				MainThreadExecutable.class,
 				StartStoppable.class,
 				RpcGateway.class
 			},
@@ -114,13 +112,13 @@ public class TestingSerialRpcService implements RpcService {
 			if (clazz.isAssignableFrom(gateway.getClass())) {
 				@SuppressWarnings("unchecked")
 				C typedGateway = (C) gateway;
-				return Futures.successful(typedGateway);
+				return FlinkCompletableFuture.completed(typedGateway);
 			} else {
-				return Futures.failed(
+				return FlinkCompletableFuture.completedExceptionally(
 					new Exception("Gateway registered under " + address + " is not of type " + clazz));
 			}
 		} else {
-			return Futures.failed(new Exception("No gateway registered under that name"));
+			return FlinkCompletableFuture.completedExceptionally(new Exception("No gateway registered under that name"));
 		}
 	}
 
@@ -141,20 +139,20 @@ public class TestingSerialRpcService implements RpcService {
 		registeredConnections.clear();
 	}
 
-	private static final class TestingSerialInvocationHandler<C extends RpcGateway, T extends RpcEndpoint<C>> implements InvocationHandler, RpcGateway, MainThreadExecutor, StartStoppable {
+	private static final class TestingSerialInvocationHandler<C extends RpcGateway, T extends RpcEndpoint<C>> implements InvocationHandler, RpcGateway, MainThreadExecutable, StartStoppable {
 
 		private final T rpcEndpoint;
 
 		/** default timeout for asks */
-		private final Timeout timeout;
+		private final Time timeout;
 
 		private final String address;
 
 		private TestingSerialInvocationHandler(String address, T rpcEndpoint) {
-			this(address, rpcEndpoint, new Timeout(new FiniteDuration(10, TimeUnit.SECONDS)));
+			this(address, rpcEndpoint, Time.seconds(10));
 		}
 
-		private TestingSerialInvocationHandler(String address, T rpcEndpoint, Timeout timeout) {
+		private TestingSerialInvocationHandler(String address, T rpcEndpoint, Time timeout) {
 			this.rpcEndpoint = rpcEndpoint;
 			this.timeout = timeout;
 			this.address = address;
@@ -163,7 +161,7 @@ public class TestingSerialRpcService implements RpcService {
 		@Override
 		public Object invoke(Object proxy, Method method, Object[] args) throws Throwable {
 			Class<?> declaringClass = method.getDeclaringClass();
-			if (declaringClass.equals(MainThreadExecutor.class) ||
+			if (declaringClass.equals(MainThreadExecutable.class) ||
 				declaringClass.equals(Object.class) || declaringClass.equals(StartStoppable.class) ||
 				declaringClass.equals(RpcGateway.class)) {
 				return method.invoke(this, args);
@@ -171,7 +169,7 @@ public class TestingSerialRpcService implements RpcService {
 				final String methodName = method.getName();
 				Class<?>[] parameterTypes = method.getParameterTypes();
 				Annotation[][] parameterAnnotations = method.getParameterAnnotations();
-				Timeout futureTimeout = extractRpcTimeout(parameterAnnotations, args, timeout);
+				Time futureTimeout = extractRpcTimeout(parameterAnnotations, args, timeout);
 
 				final Tuple2<Class<?>[], Object[]> filteredArguments = filterArguments(
 					parameterTypes,
@@ -201,13 +199,13 @@ public class TestingSerialRpcService implements RpcService {
 		private Object handleRpcInvocationSync(final String methodName,
 			final Class<?>[] parameterTypes,
 			final Object[] args,
-			final Timeout futureTimeout) throws Exception {
+			final Time futureTimeout) throws Exception {
 			final Method rpcMethod = lookupRpcMethod(methodName, parameterTypes);
 			Object result = rpcMethod.invoke(rpcEndpoint, args);
 
 			if (result instanceof Future) {
 				Future<?> future = (Future<?>) result;
-				return Await.result(future, futureTimeout.duration());
+				return future.get(futureTimeout.getSize(), futureTimeout.getUnit());
 			} else {
 				return result;
 			}
@@ -219,11 +217,11 @@ public class TestingSerialRpcService implements RpcService {
 		}
 
 		@Override
-		public <V> Future<V> callAsync(Callable<V> callable, Timeout callTimeout) {
+		public <V> Future<V> callAsync(Callable<V> callable, Time callTimeout) {
 			try {
-				return Futures.successful(callable.call());
+				return FlinkCompletableFuture.completed(callable.call());
 			} catch (Throwable e) {
-				return Futures.failed(e);
+				return FlinkCompletableFuture.completedExceptionally(e);
 			}
 		}
 
@@ -281,18 +279,18 @@ public class TestingSerialRpcService implements RpcService {
 		 *                             has been found
 		 * @return Timeout extracted from the array of arguments or the default timeout
 		 */
-		private static Timeout extractRpcTimeout(Annotation[][] parameterAnnotations, Object[] args,
-			Timeout defaultTimeout) {
+		private static Time extractRpcTimeout(Annotation[][] parameterAnnotations, Object[] args,
+			Time defaultTimeout) {
 			if (args != null) {
 				Preconditions.checkArgument(parameterAnnotations.length == args.length);
 
 				for (int i = 0; i < parameterAnnotations.length; i++) {
 					if (isRpcTimeout(parameterAnnotations[i])) {
-						if (args[i] instanceof FiniteDuration) {
-							return new Timeout((FiniteDuration) args[i]);
+						if (args[i] instanceof Time) {
+							return (Time) args[i];
 						} else {
 							throw new RuntimeException("The rpc timeout parameter must be of type " +
-								FiniteDuration.class.getName() + ". The type " + args[i].getClass().getName() +
+								Time.class.getName() + ". The type " + args[i].getClass().getName() +
 								" is not supported.");
 						}
 					}

http://git-wip-us.apache.org/repos/asf/flink/blob/507e86cf/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 a6ceb91..5624d12 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
@@ -19,8 +19,9 @@
 package org.apache.flink.runtime.rpc.akka;
 
 import akka.actor.ActorSystem;
-import akka.util.Timeout;
+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.rpc.RpcEndpoint;
 import org.apache.flink.runtime.rpc.RpcGateway;
 import org.apache.flink.runtime.rpc.RpcMethod;
@@ -30,13 +31,12 @@ import org.apache.flink.util.TestLogger;
 import org.hamcrest.core.Is;
 import org.junit.AfterClass;
 import org.junit.Test;
-import scala.concurrent.Await;
-import scala.concurrent.Future;
 
-import java.util.concurrent.TimeUnit;
+import java.util.concurrent.ExecutionException;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 public class AkkaRpcActorTest extends TestLogger {
@@ -47,7 +47,7 @@ public class AkkaRpcActorTest extends TestLogger {
 
 	private static ActorSystem actorSystem = AkkaUtils.createDefaultActorSystem();
 
-	private static Timeout timeout = new Timeout(10000, TimeUnit.MILLISECONDS);
+	private static Time timeout = Time.milliseconds(10000L);
 
 	private static AkkaRpcService akkaRpcService =
 		new AkkaRpcService(actorSystem, timeout);
@@ -69,7 +69,7 @@ public class AkkaRpcActorTest extends TestLogger {
 
 		Future<DummyRpcGateway> futureRpcGateway = akkaRpcService.connect(rpcEndpoint.getAddress(), DummyRpcGateway.class);
 
-		DummyRpcGateway rpcGateway = Await.result(futureRpcGateway, timeout.duration());
+		DummyRpcGateway rpcGateway = futureRpcGateway.get(timeout.getSize(), timeout.getUnit());
 
 		assertEquals(rpcEndpoint.getAddress(), rpcGateway.getAddress());
 	}
@@ -82,11 +82,12 @@ public class AkkaRpcActorTest extends TestLogger {
 		Future<DummyRpcGateway> futureRpcGateway = akkaRpcService.connect("foobar", DummyRpcGateway.class);
 
 		try {
-			DummyRpcGateway gateway = Await.result(futureRpcGateway, timeout.duration());
+			DummyRpcGateway gateway = futureRpcGateway.get(timeout.getSize(), timeout.getUnit());
 
 			fail("The rpc connection resolution should have failed.");
-		} catch (RpcConnectionException exception) {
+		} catch (ExecutionException exception) {
 			// we're expecting a RpcConnectionException
+			assertTrue(exception.getCause() instanceof RpcConnectionException);
 		}
 	}
 
@@ -111,7 +112,7 @@ public class AkkaRpcActorTest extends TestLogger {
 		// now process the rpc
 		rpcEndpoint.start();
 
-		Integer actualValue = Await.result(result, timeout.duration());
+		Integer actualValue = result.get(timeout.getSize(), timeout.getUnit());
 
 		assertThat("The new foobar value should have been returned.", actualValue, Is.is(expectedValue));
 

http://git-wip-us.apache.org/repos/asf/flink/blob/507e86cf/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java
index f55069e..4e9e518 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java
@@ -19,7 +19,7 @@
 package org.apache.flink.runtime.rpc.akka;
 
 import akka.actor.ActorSystem;
-import akka.util.Timeout;
+import org.apache.flink.api.common.time.Time;
 import org.apache.flink.core.testutils.OneShotLatch;
 import org.apache.flink.runtime.akka.AkkaUtils;
 import org.apache.flink.util.TestLogger;
@@ -40,7 +40,7 @@ public class AkkaRpcServiceTest extends TestLogger {
 	private static ActorSystem actorSystem = AkkaUtils.createDefaultActorSystem();
 
 	private static AkkaRpcService akkaRpcService =
-			new AkkaRpcService(actorSystem, new Timeout(10000, TimeUnit.MILLISECONDS));
+			new AkkaRpcService(actorSystem, Time.milliseconds(10000));
 
 	@AfterClass
 	public static void shutdown() {

http://git-wip-us.apache.org/repos/asf/flink/blob/507e86cf/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 9ffafda..9ec1f7e 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
@@ -18,8 +18,7 @@
 
 package org.apache.flink.runtime.rpc.akka;
 
-import akka.util.Timeout;
-
+import org.apache.flink.api.common.time.Time;
 import org.apache.flink.runtime.akka.AkkaUtils;
 
 import org.apache.flink.runtime.rpc.RpcEndpoint;
@@ -30,8 +29,6 @@ import org.apache.flink.runtime.rpc.RpcService;
 import org.apache.flink.util.TestLogger;
 import org.junit.Test;
 
-import java.util.concurrent.TimeUnit;
-
 import static org.junit.Assert.assertTrue;
 
 public class MainThreadValidationTest extends TestLogger {
@@ -48,7 +45,7 @@ public class MainThreadValidationTest extends TestLogger {
 		// actual test
 		AkkaRpcService akkaRpcService = new AkkaRpcService(
 				AkkaUtils.createDefaultActorSystem(),
-				new Timeout(10000, TimeUnit.MILLISECONDS));
+				Time.milliseconds(10000));
 
 		try {
 			TestEndpoint testEndpoint = new TestEndpoint(akkaRpcService);

http://git-wip-us.apache.org/repos/asf/flink/blob/507e86cf/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 9d2ed99..0d5dc28 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
@@ -19,10 +19,11 @@
 package org.apache.flink.runtime.rpc.akka;
 
 import akka.actor.ActorSystem;
-import akka.util.Timeout;
 import com.typesafe.config.Config;
 import com.typesafe.config.ConfigValueFactory;
+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.rpc.RpcEndpoint;
 import org.apache.flink.runtime.rpc.RpcGateway;
 import org.apache.flink.runtime.rpc.RpcMethod;
@@ -32,13 +33,9 @@ import org.hamcrest.core.Is;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
-import scala.concurrent.Await;
-import scala.concurrent.Future;
-import scala.concurrent.duration.FiniteDuration;
 
 import java.io.IOException;
 import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.TimeUnit;
 
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.fail;
@@ -52,7 +49,7 @@ public class MessageSerializationTest extends TestLogger {
 	private static AkkaRpcService akkaRpcService1;
 	private static AkkaRpcService akkaRpcService2;
 
-	private static final FiniteDuration timeout = new FiniteDuration(10L, TimeUnit.SECONDS);
+	private static final Time timeout = Time.seconds(10L);
 	private static final int maxFrameSize = 32000;
 
 	@BeforeClass
@@ -63,8 +60,8 @@ public class MessageSerializationTest extends TestLogger {
 		actorSystem1 = AkkaUtils.createActorSystem(modifiedAkkaConfig);
 		actorSystem2 = AkkaUtils.createActorSystem(modifiedAkkaConfig);
 
-		akkaRpcService1 = new AkkaRpcService(actorSystem1, new Timeout(timeout));
-		akkaRpcService2 = new AkkaRpcService(actorSystem2, new Timeout(timeout));
+		akkaRpcService1 = new AkkaRpcService(actorSystem1, timeout);
+		akkaRpcService2 = new AkkaRpcService(actorSystem2, timeout);
 	}
 
 	@AfterClass
@@ -113,7 +110,7 @@ public class MessageSerializationTest extends TestLogger {
 
 		Future<TestGateway> remoteGatewayFuture = akkaRpcService2.connect(address, TestGateway.class);
 
-		TestGateway remoteGateway = Await.result(remoteGatewayFuture, timeout);
+		TestGateway remoteGateway = remoteGatewayFuture.get(timeout.getSize(), timeout.getUnit());
 
 		remoteGateway.foobar(new Object());
 
@@ -134,7 +131,7 @@ public class MessageSerializationTest extends TestLogger {
 
 		Future<TestGateway> remoteGatewayFuture = akkaRpcService2.connect(address, TestGateway.class);
 
-		TestGateway remoteGateway = Await.result(remoteGatewayFuture, timeout);
+		TestGateway remoteGateway = remoteGatewayFuture.get(timeout.getSize(), timeout.getUnit());
 
 		int expected = 42;
 
@@ -158,7 +155,7 @@ public class MessageSerializationTest extends TestLogger {
 
 		Future<TestGateway> remoteGatewayFuture = akkaRpcService2.connect(address, TestGateway.class);
 
-		TestGateway remoteGateway = Await.result(remoteGatewayFuture, timeout);
+		TestGateway remoteGateway = remoteGatewayFuture.get(timeout.getSize(), timeout.getUnit());
 
 		int bufferSize = maxFrameSize + 1;
 		byte[] buffer = new byte[bufferSize];

http://git-wip-us.apache.org/repos/asf/flink/blob/507e86cf/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 a8d5bd7..09aab18 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
@@ -18,6 +18,7 @@
 
 package org.apache.flink.runtime.taskexecutor;
 
+import org.apache.flink.api.common.time.Time;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.clusterframework.types.ResourceID;
 import org.apache.flink.runtime.highavailability.NonHaServices;
@@ -29,8 +30,6 @@ import org.apache.flink.util.TestLogger;
 
 import org.junit.Test;
 
-import scala.concurrent.duration.FiniteDuration;
-
 import java.util.UUID;
 
 import static org.junit.Assert.*;
@@ -56,7 +55,7 @@ public class TaskExecutorTest extends TestLogger {
 			taskManager.start();
 
 			verify(rmGateway, timeout(5000)).registerTaskExecutor(
-					any(UUID.class), eq(taskManagerAddress), eq(resourceID), any(FiniteDuration.class));
+					any(UUID.class), eq(taskManagerAddress), eq(resourceID), any(Time.class));
 		}
 		finally {
 			rpc.stopService();
@@ -97,7 +96,7 @@ public class TaskExecutorTest extends TestLogger {
 			testLeaderService.notifyListener(address1, leaderId1);
 
 			verify(rmGateway1, timeout(5000)).registerTaskExecutor(
-					eq(leaderId1), eq(taskManagerAddress), eq(resourceID), any(FiniteDuration.class));
+					eq(leaderId1), eq(taskManagerAddress), eq(resourceID), any(Time.class));
 			assertNotNull(taskManager.getResourceManagerConnection());
 
 			// cancel the leader 
@@ -107,7 +106,7 @@ public class TaskExecutorTest extends TestLogger {
 			testLeaderService.notifyListener(address2, leaderId2);
 
 			verify(rmGateway2, timeout(5000)).registerTaskExecutor(
-					eq(leaderId2), eq(taskManagerAddress), eq(resourceID), any(FiniteDuration.class));
+					eq(leaderId2), eq(taskManagerAddress), eq(resourceID), any(Time.class));
 			assertNotNull(taskManager.getResourceManagerConnection());
 		}
 		finally {


[13/50] [abbrv] flink git commit: [FLINK-4537] [cluster management] ResourceManager registration with JobManager

Posted by se...@apache.org.
[FLINK-4537] [cluster management] ResourceManager registration with JobManager


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

Branch: refs/heads/flip-6
Commit: efc7de5bd3bff0512c20485f94d563c9e9cea5ec
Parents: f4dc474
Author: beyond1920 <be...@126.com>
Authored: Thu Sep 1 15:27:20 2016 +0800
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Oct 14 15:14:40 2016 +0200

----------------------------------------------------------------------
 .../HighAvailabilityServices.java               |   9 ++
 .../runtime/highavailability/NonHaServices.java |  19 +++
 .../jobmaster/JobMasterRegistrationSuccess.java |  49 ++++++
 .../resourcemanager/JobMasterRegistration.java  |  39 ++++-
 .../resourcemanager/ResourceManager.java        | 125 +++++++++++++--
 .../resourcemanager/ResourceManagerGateway.java |  34 ++--
 .../exceptions/LeaderSessionIDException.java    |  60 +++++++
 .../runtime/taskexecutor/TaskExecutor.java      |   5 +
 .../TestingHighAvailabilityServices.java        |  17 ++
 .../resourcemanager/ResourceManagerTest.java    | 160 +++++++++++++++++++
 10 files changed, 483 insertions(+), 34 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/efc7de5b/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 298147c..7634176 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
@@ -40,6 +40,15 @@ public interface HighAvailabilityServices {
 	LeaderRetrievalService getResourceManagerLeaderRetriever() throws Exception;
 
 	/**
+	 * 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
+	 */
+	LeaderRetrievalService getJobMasterLeaderRetriever(JobID jobID) throws Exception;
+
+	/**
 	 * Gets the leader election service for the cluster's resource manager.
 	 * @return
 	 * @throws Exception

http://git-wip-us.apache.org/repos/asf/flink/blob/efc7de5b/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 292a404..33dc2d7 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,6 +19,7 @@
 package org.apache.flink.runtime.highavailability;
 
 import org.apache.flink.api.common.JobID;
+import org.apache.flink.hadoop.shaded.org.jboss.netty.util.internal.ConcurrentHashMap;
 import org.apache.flink.runtime.leaderelection.LeaderElectionService;
 import org.apache.flink.runtime.leaderelection.StandaloneLeaderElectionService;
 import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
@@ -42,6 +43,8 @@ public class NonHaServices 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.
 	 * 
@@ -49,6 +52,17 @@ public class NonHaServices implements HighAvailabilityServices {
 	 */
 	public NonHaServices(String resourceManagerAddress) {
 		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);
 	}
 
 	// ------------------------------------------------------------------------
@@ -61,6 +75,11 @@ public class NonHaServices implements HighAvailabilityServices {
 	}
 
 	@Override
+	public LeaderRetrievalService getJobMasterLeaderRetriever(JobID jobID) throws Exception {
+		return new StandaloneLeaderRetrievalService(jobMastersAddress.get(jobID), new UUID(0, 0));
+	}
+
+	@Override
 	public LeaderElectionService getResourceManagerLeaderElectionService() throws Exception {
 		return new StandaloneLeaderElectionService();
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/efc7de5b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterRegistrationSuccess.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterRegistrationSuccess.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterRegistrationSuccess.java
new file mode 100644
index 0000000..031c38e
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterRegistrationSuccess.java
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.jobmaster;
+
+import org.apache.flink.runtime.registration.RegistrationResponse;
+
+/**
+ * Base class for responses from the ResourceManager to a registration attempt by a JobMaster.
+ */
+public class JobMasterRegistrationSuccess extends RegistrationResponse.Success {
+
+	private static final long serialVersionUID = 5577641250204140415L;
+
+	private final long heartbeatInterval;
+
+	public JobMasterRegistrationSuccess(long heartbeatInterval) {
+		this.heartbeatInterval = heartbeatInterval;
+	}
+
+	/**
+	 * Gets the interval in which the ResourceManager will heartbeat the JobMaster.
+	 *
+	 * @return the interval in which the ResourceManager will heartbeat the JobMaster
+	 */
+	public long getHeartbeatInterval() {
+		return heartbeatInterval;
+	}
+
+	@Override
+	public String toString() {
+		return "JobMasterRegistrationSuccess(" + heartbeatInterval + ')';
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/efc7de5b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/JobMasterRegistration.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/JobMasterRegistration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/JobMasterRegistration.java
index 439e56b..7b8ec70 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/JobMasterRegistration.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/JobMasterRegistration.java
@@ -18,23 +18,56 @@
 
 package org.apache.flink.runtime.resourcemanager;
 
+<<<<<<< HEAD
 import org.apache.flink.api.common.JobID;
+=======
+import org.apache.flink.runtime.jobmaster.JobMasterGateway;
+>>>>>>> db98efb... rsourceManager registration with JobManager
 
 import java.io.Serializable;
+import java.util.UUID;
 
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * This class is responsible for group the JobMasterGateway and the LeaderSessionID of a registered job master
+ */
 public class JobMasterRegistration implements Serializable {
-	private static final long serialVersionUID = 8411214999193765202L;
 
+<<<<<<< HEAD
 	private final String address;
 	private final JobID jobID;
 
 	public JobMasterRegistration(String address, JobID jobID) {
 		this.address = address;
 		this.jobID = jobID;
+=======
+	private static final long serialVersionUID = -2316627821716999527L;
+
+	private final JobMasterGateway jobMasterGateway;
+
+	private UUID jobMasterLeaderSessionID;
+
+	public JobMasterRegistration(JobMasterGateway jobMasterGateway) {
+		this.jobMasterGateway = checkNotNull(jobMasterGateway);
+	}
+
+	public JobMasterRegistration(JobMasterGateway jobMasterGateway, UUID jobMasterLeaderSessionID) {
+		this.jobMasterGateway = checkNotNull(jobMasterGateway);
+		this.jobMasterLeaderSessionID = jobMasterLeaderSessionID;
+	}
+
+	public JobMasterGateway getJobMasterGateway() {
+		return jobMasterGateway;
+	}
+
+	public void setJobMasterLeaderSessionID(UUID leaderSessionID) {
+		this.jobMasterLeaderSessionID = jobMasterLeaderSessionID;
+>>>>>>> db98efb... rsourceManager registration with JobManager
 	}
 
-	public String getAddress() {
-		return address;
+	public UUID getJobMasterLeaderSessionID() {
+		return jobMasterLeaderSessionID;
 	}
 
 	public JobID getJobID() {

http://git-wip-us.apache.org/repos/asf/flink/blob/efc7de5b/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 5370710..8be1455 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
@@ -18,6 +18,7 @@
 
 package org.apache.flink.runtime.resourcemanager;
 
+import akka.dispatch.Futures;
 import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.runtime.clusterframework.types.ResourceID;
@@ -25,15 +26,22 @@ import org.apache.flink.runtime.concurrent.ApplyFunction;
 import org.apache.flink.runtime.concurrent.Future;
 import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
 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;
+<<<<<<< HEAD
 import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager;
+=======
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener;
+>>>>>>> db98efb... rsourceManager registration with JobManager
 import org.apache.flink.runtime.rpc.RpcMethod;
 import org.apache.flink.runtime.rpc.RpcEndpoint;
 import org.apache.flink.runtime.rpc.RpcService;
 import org.apache.flink.runtime.jobmaster.JobMaster;
 import org.apache.flink.runtime.jobmaster.JobMasterGateway;
+import org.apache.flink.runtime.rpc.exceptions.LeaderSessionIDException;
 import org.apache.flink.runtime.taskexecutor.TaskExecutorRegistrationSuccess;
+import org.apache.flink.runtime.registration.RegistrationResponse;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -50,15 +58,21 @@ import static org.apache.flink.util.Preconditions.checkNotNull;
  *
  * It offers the following methods as part of its rpc interface to interact with the him remotely:
  * <ul>
- *     <li>{@link #registerJobMaster(JobMasterRegistration)} registers a {@link JobMaster} at the resource manager</li>
+ *     <li>{@link #registerJobMaster(UUID, String, JobID)} registers a {@link JobMaster} at the resource manager</li>
  *     <li>{@link #requestSlot(SlotRequest)} requests a slot from the resource manager</li>
  * </ul>
  */
+<<<<<<< HEAD
 public class ResourceManager extends RpcEndpoint<ResourceManagerGateway> implements LeaderContender {
 
 	private final Logger LOG = LoggerFactory.getLogger(getClass());
 
 	private final Map<JobID, JobMasterGateway> jobMasterGateways;
+=======
+public class ResourceManager extends RpcEndpoint<ResourceManagerGateway> {
+	/** the mapping relationship of JobID and JobMasterGateway */
+	private final Map<JobID, JobMasterRegistration> jobMasters;
+>>>>>>> db98efb... rsourceManager registration with JobManager
 
 	private final HighAvailabilityServices highAvailabilityServices;
 
@@ -74,8 +88,12 @@ public class ResourceManager extends RpcEndpoint<ResourceManagerGateway> impleme
 			SlotManager slotManager) {
 		super(rpcService);
 		this.highAvailabilityServices = checkNotNull(highAvailabilityServices);
+<<<<<<< HEAD
 		this.jobMasterGateways = new HashMap<>();
 		this.slotManager = slotManager;
+=======
+		this.jobMasters = new HashMap<>(16);
+>>>>>>> db98efb... rsourceManager registration with JobManager
 	}
 
 	@Override
@@ -95,8 +113,11 @@ public class ResourceManager extends RpcEndpoint<ResourceManagerGateway> impleme
 	public void shutDown() {
 		try {
 			leaderElectionService.stop();
+			for(JobID jobID : jobMasters.keySet()) {
+				highAvailabilityServices.getJobMasterLeaderRetriever(jobID).stop();
+			}
 			super.shutDown();
-		} catch(Throwable e) {
+		} 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);
 		}
@@ -115,24 +136,58 @@ public class ResourceManager extends RpcEndpoint<ResourceManagerGateway> impleme
 	/**
 	 * Register a {@link JobMaster} at the resource manager.
 	 *
-	 * @param jobMasterRegistration Job master registration information
+	 * @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
+<<<<<<< HEAD
 	public Future<RegistrationResponse> registerJobMaster(JobMasterRegistration jobMasterRegistration) {
 		final Future<JobMasterGateway> jobMasterFuture =
 			getRpcService().connect(jobMasterRegistration.getAddress(), JobMasterGateway.class);
 		final JobID jobID = jobMasterRegistration.getJobID();
+=======
+	public Future<RegistrationResponse> registerJobMaster(UUID resourceManagerLeaderId, final String jobMasterAddress, final JobID jobID) {
+
+		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 Futures.failed(new LeaderSessionIDException(leaderSessionID, resourceManagerLeaderId));
+		}
+
+		Future<JobMasterGateway> jobMasterFuture = getRpcService().connect(jobMasterAddress, JobMasterGateway.class);
+>>>>>>> db98efb... rsourceManager registration with JobManager
 
 		return jobMasterFuture.thenApplyAsync(new ApplyFunction<JobMasterGateway, RegistrationResponse>() {
 			@Override
 			public RegistrationResponse apply(JobMasterGateway jobMasterGateway) {
+<<<<<<< HEAD
 				final JobMasterGateway existingGateway = jobMasterGateways.put(jobID, jobMasterGateway);
 				if (existingGateway != null) {
 					LOG.info("Replacing existing gateway {} for JobID {} with  {}.",
 						existingGateway, jobID, jobMasterGateway);
 				}
 				return new RegistrationResponse(true);
+=======
+				if (jobMasters.containsKey(jobID)) {
+					JobMasterRegistration jobMasterRegistration = new JobMasterRegistration(jobMasterGateway, jobMasters.get(jobID).getJobMasterLeaderSessionID());
+					jobMasters.put(jobID, jobMasterRegistration);
+					log.info("Replacing gateway for registered JobID {}.", jobID);
+				} else {
+					JobMasterRegistration jobMasterRegistration = new JobMasterRegistration(jobMasterGateway);
+					jobMasters.put(jobID, jobMasterRegistration);
+					try {
+						highAvailabilityServices.getJobMasterLeaderRetriever(jobID).start(new JobMasterLeaderListener(jobID));
+					} catch(Throwable e) {
+						log.warn("Decline registration from JobMaster {} at ({}) because fail to get the leader retriever for the given job JobMaster",
+							jobID, jobMasterAddress);
+						return new RegistrationResponse.Decline("Fail to get the leader retriever for the given JobMaster");
+					}
+				}
+
+				return new JobMasterRegistrationSuccess(5000);
+>>>>>>> db98efb... rsourceManager registration with JobManager
 			}
 		}, getMainThreadExecutor());
 	}
@@ -158,26 +213,41 @@ public class ResourceManager extends RpcEndpoint<ResourceManagerGateway> impleme
 
 
 	/**
-	 *
-	 * @param resourceManagerLeaderId  The fencing token for the ResourceManager leader 
-	 * @param taskExecutorAddress      The address of the TaskExecutor that registers
-	 * @param resourceID               The resource ID of the TaskExecutor that registers
-	 *
+	 * @param resourceManagerLeaderId The fencing token for the ResourceManager leader
+	 * @param taskExecutorAddress     The address of the TaskExecutor that registers
+	 * @param resourceID              The resource ID of the TaskExecutor that registers
 	 * @return The response by the ResourceManager.
 	 */
 	@RpcMethod
-	public org.apache.flink.runtime.registration.RegistrationResponse registerTaskExecutor(
-			UUID resourceManagerLeaderId,
-			String taskExecutorAddress,
-			ResourceID resourceID) {
+	public RegistrationResponse registerTaskExecutor(
+		UUID resourceManagerLeaderId,
+		String taskExecutorAddress,
+		ResourceID resourceID) {
 
 		return new TaskExecutorRegistrationSuccess(new InstanceID(), 5000);
 	}
 
 
+<<<<<<< HEAD
 	// ------------------------------------------------------------------------
 	//  Leader Contender
 	// ------------------------------------------------------------------------
+=======
+		/**
+		 * Callback method when current resourceManager lose leadership.
+		 */
+		@Override
+		public void revokeLeadership() {
+			runAsync(new Runnable() {
+				@Override
+				public void run() {
+					log.info("ResourceManager {} was revoked leadership.", getAddress());
+					jobMasters.clear();
+					leaderSessionID = null;
+				}
+			});
+		}
+>>>>>>> db98efb... rsourceManager registration with JobManager
 
 	/**
 	 * Callback method when current resourceManager is granted leadership
@@ -232,4 +302,35 @@ public class ResourceManager extends RpcEndpoint<ResourceManagerGateway> impleme
 			}
 		});
 	}
+
+	private class JobMasterLeaderListener implements LeaderRetrievalListener {
+		private final JobID jobID;
+
+		private JobMasterLeaderListener(JobID jobID) {
+			this.jobID = jobID;
+		}
+
+		@Override
+		public void notifyLeaderAddress(final String leaderAddress, final UUID leaderSessionID) {
+			runAsync(new Runnable() {
+				@Override
+				public void run() {
+					log.info("A new leader for JobMaster {} is elected, address is {}, leaderSessionID is {}", jobID, leaderAddress, leaderSessionID);
+					// update job master leader session id
+					JobMasterRegistration jobMasterRegistration = jobMasters.get(jobID);
+					jobMasterRegistration.setJobMasterLeaderSessionID(leaderSessionID);
+				}
+			});
+		}
+
+		@Override
+		public void handleError(final Exception exception) {
+			runAsync(new Runnable() {
+				@Override
+				public void run() {
+					log.error("JobMasterLeaderListener received an error from the LeaderRetrievalService", exception);
+				}
+			});
+		}
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/efc7de5b/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 5c8786c..1ee11a1 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
@@ -18,12 +18,13 @@
 
 package org.apache.flink.runtime.resourcemanager;
 
-import org.apache.flink.api.common.time.Time;
+import org.apache.flink.api.common.JobID;
 import org.apache.flink.runtime.clusterframework.types.ResourceID;
 import org.apache.flink.runtime.concurrent.Future;
 import org.apache.flink.runtime.rpc.RpcGateway;
 import org.apache.flink.runtime.rpc.RpcTimeout;
 import org.apache.flink.runtime.jobmaster.JobMaster;
+import org.apache.flink.runtime.registration.RegistrationResponse;
 
 import java.util.UUID;
 
@@ -35,21 +36,18 @@ public interface ResourceManagerGateway extends RpcGateway {
 	/**
 	 * Register a {@link JobMaster} at the resource manager.
 	 *
-	 * @param jobMasterRegistration Job master registration information
-	 * @param timeout Timeout for the future to complete
+	 * @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
+	 * @param timeout                 Timeout for the future to complete
 	 * @return Future registration response
 	 */
 	Future<RegistrationResponse> registerJobMaster(
-		JobMasterRegistration jobMasterRegistration,
-		@RpcTimeout Time timeout);
+		UUID resourceManagerLeaderId,
+		String jobMasterAddress,
+		JobID jobID,
+				@RpcTimeout Time timeout);
 
-	/**
-	 * Register a {@link JobMaster} at the resource manager.
-	 *
-	 * @param jobMasterRegistration Job master registration information
-	 * @return Future registration response
-	 */
-	Future<RegistrationResponse> registerJobMaster(JobMasterRegistration jobMasterRegistration);
 
 	/**
 	 * Requests a slot from the resource manager.
@@ -60,15 +58,13 @@ public interface ResourceManagerGateway extends RpcGateway {
 	Future<SlotRequestReply> requestSlot(SlotRequest slotRequest);
 
 	/**
-	 * 
-	 * @param resourceManagerLeaderId  The fencing token for the ResourceManager leader 
-	 * @param taskExecutorAddress      The address of the TaskExecutor that registers
-	 * @param resourceID               The resource ID of the TaskExecutor that registers
-	 * @param timeout                  The timeout for the response.
-	 * 
+	 * @param resourceManagerLeaderId The fencing token for the ResourceManager leader
+	 * @param taskExecutorAddress     The address of the TaskExecutor that registers
+	 * @param resourceID              The resource ID of the TaskExecutor that registers
+	 * @param timeout                 The timeout for the response.
 	 * @return The future to the response by the ResourceManager.
 	 */
-	Future<org.apache.flink.runtime.registration.RegistrationResponse> registerTaskExecutor(
+	Future<RegistrationResponse> registerTaskExecutor(
 			UUID resourceManagerLeaderId,
 			String taskExecutorAddress,
 			ResourceID resourceID,

http://git-wip-us.apache.org/repos/asf/flink/blob/efc7de5b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/exceptions/LeaderSessionIDException.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/exceptions/LeaderSessionIDException.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/exceptions/LeaderSessionIDException.java
new file mode 100644
index 0000000..cd14a0d
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/exceptions/LeaderSessionIDException.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.rpc.exceptions;
+
+import java.util.UUID;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * An exception specifying that the received leader session ID is not the same as expected.
+ */
+public class LeaderSessionIDException extends Exception {
+
+	private static final long serialVersionUID = -3276145308053264636L;
+
+	/** expected leader session id */
+	private final UUID expectedLeaderSessionID;
+
+	/** actual leader session id */
+	private final UUID actualLeaderSessionID;
+
+	public LeaderSessionIDException(UUID expectedLeaderSessionID, UUID actualLeaderSessionID) {
+		super("Unmatched leader session ID : expected " + expectedLeaderSessionID + ", actual " + actualLeaderSessionID);
+		this.expectedLeaderSessionID =  checkNotNull(expectedLeaderSessionID);
+		this.actualLeaderSessionID = checkNotNull(actualLeaderSessionID);
+	}
+
+	/**
+	 * Get expected leader session id
+	 *
+	 * @return expect leader session id
+	 */
+	public UUID getExpectedLeaderSessionID() {
+		return expectedLeaderSessionID;
+	}
+
+	/**
+	 * Get actual leader session id
+	 *
+	 * @return actual leader session id
+	 */
+	public UUID getActualLeaderSessionID() {
+		return actualLeaderSessionID;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/efc7de5b/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 d84a6a9..cf709c8 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
@@ -327,6 +327,11 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 			}
 
 			@Override
+			public LeaderRetrievalService getJobMasterLeaderRetriever(JobID jobID) throws Exception {
+				return null;
+			}
+
+			@Override
 			public LeaderElectionService getResourceManagerLeaderElectionService() throws Exception {
 				return null;
 			}

http://git-wip-us.apache.org/repos/asf/flink/blob/efc7de5b/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 3162f40..2ac43be 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,6 +19,7 @@
 package org.apache.flink.runtime.highavailability;
 
 import org.apache.flink.api.common.JobID;
+import org.apache.flink.hadoop.shaded.org.jboss.netty.util.internal.ConcurrentHashMap;
 import org.apache.flink.runtime.leaderelection.LeaderElectionService;
 import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
 
@@ -30,6 +31,8 @@ public class TestingHighAvailabilityServices implements HighAvailabilityServices
 
 	private volatile LeaderRetrievalService resourceManagerLeaderRetriever;
 
+	private ConcurrentHashMap<JobID, LeaderRetrievalService> jobMasterLeaderRetrievers = new ConcurrentHashMap<>();
+
 	private volatile LeaderElectionService jobMasterLeaderElectionService;
 
 	private volatile LeaderElectionService resourceManagerLeaderElectionService;
@@ -43,6 +46,10 @@ public class TestingHighAvailabilityServices implements HighAvailabilityServices
 		this.resourceManagerLeaderRetriever = resourceManagerLeaderRetriever;
 	}
 
+	public void setJobMasterLeaderRetriever(JobID jobID, LeaderRetrievalService jobMasterLeaderRetriever) {
+		this.jobMasterLeaderRetrievers.put(jobID, jobMasterLeaderRetriever);
+	}
+
 	public void setJobMasterLeaderElectionService(LeaderElectionService leaderElectionService) {
 		this.jobMasterLeaderElectionService = leaderElectionService;
 	}
@@ -66,6 +73,16 @@ public class TestingHighAvailabilityServices implements HighAvailabilityServices
 	}
 
 	@Override
+	public LeaderRetrievalService getJobMasterLeaderRetriever(JobID jobID) throws Exception {
+		LeaderRetrievalService service = this.jobMasterLeaderRetrievers.get(jobID);
+		if (service != null) {
+			return service;
+		} else {
+			throw new IllegalStateException("JobMasterLeaderRetriever has not been set");
+		}
+	}
+
+	@Override
 	public LeaderElectionService getJobMasterLeaderElectionService(JobID jobID) throws Exception {
 		LeaderElectionService service = jobMasterLeaderElectionService;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/efc7de5b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTest.java
new file mode 100644
index 0000000..4d04001
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTest.java
@@ -0,0 +1,160 @@
+/*
+ * 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.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.rpc.TestingSerialRpcService;
+import org.apache.flink.runtime.registration.RegistrationResponse;
+import org.apache.flink.runtime.rpc.exceptions.LeaderSessionIDException;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import scala.concurrent.Await;
+import scala.concurrent.Future;
+import scala.concurrent.duration.FiniteDuration;
+
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.*;
+
+public class ResourceManagerTest {
+
+	private TestingSerialRpcService rpcService;
+
+	@Before
+	public void setup() throws Exception {
+		rpcService = new TestingSerialRpcService();
+	}
+
+	@After
+	public void teardown() throws Exception {
+		rpcService.stopService();
+	}
+
+	/**
+	 * Test receive normal registration from job master and receive duplicate registration from job master
+	 *
+	 * @throws Exception
+	 */
+	@Test
+	public void testRegisterJobMaster() throws Exception {
+		String jobMasterAddress = "/jobMasterAddress1";
+		JobID jobID = mockJobMaster(jobMasterAddress);
+		TestingLeaderElectionService resourceManagerLeaderElectionService = new TestingLeaderElectionService();
+		TestingLeaderRetrievalService jobMasterLeaderRetrievalService = new TestingLeaderRetrievalService();
+		final ResourceManager resourceManager = createAndStartResourceManager(resourceManagerLeaderElectionService, jobID, jobMasterLeaderRetrievalService);
+		final UUID leaderSessionId = grantResourceManagerLeadership(resourceManagerLeaderElectionService);
+
+		// test response successful
+		Future<RegistrationResponse> successfulFuture = resourceManager.registerJobMaster(leaderSessionId, jobMasterAddress, jobID);
+		RegistrationResponse response = Await.result(successfulFuture, new FiniteDuration(0, TimeUnit.SECONDS));
+		assertTrue(response instanceof JobMasterRegistrationSuccess);
+	}
+
+	/**
+	 * Test receive registration with unmatched leadershipId from job master
+	 *
+	 * @throws Exception
+	 */
+	@Test(expected = LeaderSessionIDException.class)
+	public void testRegisterJobMasterWithUnmatchedLeaderSessionId() throws Exception {
+		String jobMasterAddress = "/jobMasterAddress1";
+		JobID jobID = mockJobMaster(jobMasterAddress);
+		TestingLeaderElectionService resourceManagerLeaderElectionService = new TestingLeaderElectionService();
+		TestingLeaderRetrievalService jobMasterLeaderRetrievalService = new TestingLeaderRetrievalService();
+		final ResourceManager resourceManager = createAndStartResourceManager(resourceManagerLeaderElectionService, jobID, jobMasterLeaderRetrievalService);
+		final UUID leaderSessionId = 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, jobMasterAddress, jobID);
+		Await.result(unMatchedLeaderFuture, new FiniteDuration(200, TimeUnit.MILLISECONDS));
+	}
+
+	/**
+	 * Test receive registration with invalid address from job master
+	 *
+	 * @throws Exception
+	 */
+	@Test(expected = Exception.class)
+	public void testRegisterJobMasterFromInvalidAddress() throws Exception {
+		String jobMasterAddress = "/jobMasterAddress1";
+		JobID jobID = mockJobMaster(jobMasterAddress);
+		TestingLeaderElectionService resourceManagerLeaderElectionService = new TestingLeaderElectionService();
+		TestingLeaderRetrievalService jobMasterLeaderRetrievalService = new TestingLeaderRetrievalService();
+		final ResourceManager resourceManager = createAndStartResourceManager(resourceManagerLeaderElectionService, jobID, jobMasterLeaderRetrievalService);
+		final UUID leaderSessionId = grantResourceManagerLeadership(resourceManagerLeaderElectionService);
+
+		// test throw exception when receive a registration from job master which takes invalid address
+		String invalidAddress = "/jobMasterAddress2";
+		Future<RegistrationResponse> invalidAddressFuture = resourceManager.registerJobMaster(leaderSessionId, invalidAddress, jobID);
+		Await.result(invalidAddressFuture, new FiniteDuration(200, TimeUnit.MILLISECONDS));
+	}
+
+	/**
+	 * Check and verify return RegistrationResponse.Decline when failed to start a job master Leader retrieval listener
+	 *
+	 * @throws Exception
+	 */
+	@Test
+	public void testRegisterJobMasterWithFailureLeaderListener() throws Exception {
+		String jobMasterAddress = "/jobMasterAddress1";
+		JobID jobID = mockJobMaster(jobMasterAddress);
+		TestingLeaderElectionService resourceManagerLeaderElectionService = new TestingLeaderElectionService();
+		TestingLeaderRetrievalService jobMasterLeaderRetrievalService = new TestingLeaderRetrievalService();
+		final ResourceManager resourceManager = createAndStartResourceManager(resourceManagerLeaderElectionService, jobID, jobMasterLeaderRetrievalService);
+		final UUID leaderSessionId = grantResourceManagerLeadership(resourceManagerLeaderElectionService);
+
+		JobID unknownJobIDToHAServices = new JobID();
+		// verify return RegistrationResponse.Decline when failed to start a job master Leader retrieval listener
+		Future<RegistrationResponse> declineFuture = resourceManager.registerJobMaster(leaderSessionId, jobMasterAddress, unknownJobIDToHAServices);
+		RegistrationResponse response = Await.result(declineFuture, new FiniteDuration(0, TimeUnit.SECONDS));
+		assertTrue(response instanceof RegistrationResponse.Decline);
+	}
+
+	private JobID mockJobMaster(String jobMasterAddress) {
+		JobID jobID = new JobID();
+		JobMasterGateway jobMasterGateway = mock(JobMasterGateway.class);
+		rpcService.registerGateway(jobMasterAddress, jobMasterGateway);
+		return jobID;
+	}
+
+	private ResourceManager createAndStartResourceManager(TestingLeaderElectionService resourceManagerLeaderElectionService, JobID jobID, TestingLeaderRetrievalService jobMasterLeaderRetrievalService) {
+		TestingHighAvailabilityServices highAvailabilityServices = new TestingHighAvailabilityServices();
+		highAvailabilityServices.setResourceManagerLeaderElectionService(resourceManagerLeaderElectionService);
+		highAvailabilityServices.setJobMasterLeaderRetriever(jobID, jobMasterLeaderRetrievalService);
+		ResourceManager resourceManager = new ResourceManager(rpcService, highAvailabilityServices);
+		resourceManager.start();
+		return resourceManager;
+	}
+
+	private UUID grantResourceManagerLeadership(TestingLeaderElectionService resourceManagerLeaderElectionService) {
+		UUID leaderSessionId = UUID.randomUUID();
+		resourceManagerLeaderElectionService.isLeader(leaderSessionId);
+		return leaderSessionId;
+	}
+
+}


[44/50] [abbrv] flink git commit: [FLINK-4738] [TaskManager] Port TaskManager logic to new Flip-6 TaskManager

Posted by se...@apache.org.
[FLINK-4738] [TaskManager] Port TaskManager logic to new Flip-6 TaskManager

The ported logic contains the task lifecycle management methods, JobManager association and
setup of TaskManager components.

Introduce Rpc implementations for TaskManager components

Implement metrics setup

Move more TaskManager components out of the constructor to make TaskExecutor more testable

Add RpcMethod annotation to TaskExecutor#confirmCheckpoint

This closes #2594.


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

Branch: refs/heads/flip-6
Commit: a00619a4850e1239eb54bcf656bf1802b8403454
Parents: 9dfaf45
Author: Till Rohrmann <tr...@apache.org>
Authored: Wed Sep 28 14:39:51 2016 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Oct 14 15:14:43 2016 +0200

----------------------------------------------------------------------
 .../CheckpointCoordinatorGateway.java           |  38 ++
 .../deployment/TaskDeploymentDescriptor.java    |   9 +
 .../runtime/executiongraph/PartitionInfo.java   |  47 ++
 .../flink/runtime/filecache/FileCache.java      |  17 +-
 .../jobgraph/tasks/InputSplitProvider.java      |   3 +-
 .../tasks/InputSplitProviderException.java      |  36 ++
 .../jobmaster/ExecutionGraphException.java      |  41 ++
 .../runtime/jobmaster/JobManagerException.java  |  39 ++
 .../flink/runtime/jobmaster/JobMaster.java      |  50 +-
 .../runtime/jobmaster/JobMasterGateway.java     |  30 +-
 .../jobmaster/MiniClusterJobDispatcher.java     |   2 +-
 .../runtime/jobmaster/SerializedInputSplit.java |  39 ++
 .../jobmaster/message/NextInputSplit.java       |  39 --
 .../flink/runtime/operators/DataSourceTask.java |  12 +-
 .../runtime/query/KvStateRegistryGateway.java   |  57 ++
 .../taskexecutor/JobManagerConnection.java      |  91 +++
 .../runtime/taskexecutor/TaskExecutor.java      | 627 +++++++++++++++++--
 .../taskexecutor/TaskExecutorGateway.java       |  80 ++-
 .../taskexecutor/TaskManagerConfiguration.java  |   3 +-
 .../runtime/taskexecutor/TaskManagerRunner.java |   3 +
 .../taskexecutor/TaskManagerServices.java       |  51 +-
 .../flink/runtime/taskexecutor/TaskSlot.java    |  73 +++
 .../runtime/taskexecutor/TaskSlotMapping.java   |  44 ++
 .../exceptions/CheckpointException.java         |  41 ++
 .../exceptions/PartitionException.java          |  41 ++
 .../taskexecutor/exceptions/TaskException.java  |  41 ++
 .../exceptions/TaskManagerException.java        |  41 ++
 .../exceptions/TaskSubmissionException.java     |  41 ++
 .../rpc/RpcCheckpointResponder.java             |  56 ++
 .../taskexecutor/rpc/RpcInputSplitProvider.java |  73 +++
 .../rpc/RpcKvStateRegistryListener.java         |  73 +++
 .../rpc/RpcPartitionStateChecker.java           |  48 ++
 .../RpcResultPartitionConsumableNotifier.java   |  67 ++
 .../utils/TaskExecutorMetricsInitializer.java   | 257 ++++++++
 .../ActorGatewayTaskManagerActions.java         |  59 ++
 .../ActorGatewayTaskManagerConnection.java      |  59 --
 .../apache/flink/runtime/taskmanager/Task.java  |  23 +-
 .../runtime/taskmanager/TaskExecutionState.java |   4 +-
 .../taskmanager/TaskInputSplitProvider.java     |  49 +-
 .../runtime/taskmanager/TaskManagerActions.java |  57 ++
 .../taskmanager/TaskManagerConnection.java      |  57 --
 .../flink/runtime/taskmanager/TaskManager.scala | 169 +----
 .../FileCacheDeleteValidationTest.java          |   4 +-
 .../jobmaster/JobManagerRunnerMockTest.java     |   1 -
 .../runtime/taskexecutor/TaskExecutorTest.java  |  24 +-
 .../runtime/taskmanager/TaskAsyncCallTest.java  |   2 +-
 .../taskmanager/TaskInputSplitProviderTest.java |   3 +-
 .../flink/runtime/taskmanager/TaskStopTest.java |   4 +-
 .../flink/runtime/taskmanager/TaskTest.java     |   4 +-
 .../source/InputFormatSourceFunction.java       |   8 +-
 .../tasks/InterruptSensitiveRestoreTest.java    |  11 +-
 .../streaming/runtime/tasks/StreamTaskTest.java |   4 +-
 52 files changed, 2320 insertions(+), 432 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/a00619a4/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
new file mode 100644
index 0000000..5a01e4d
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorGateway.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.checkpoint;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.rpc.RpcGateway;
+import org.apache.flink.runtime.state.CheckpointStateHandles;
+
+public interface CheckpointCoordinatorGateway extends RpcGateway {
+
+	void acknowledgeCheckpoint(
+			final JobID jobID,
+			final ExecutionAttemptID executionAttemptID,
+			final CheckpointMetaData checkpointInfo,
+			final CheckpointStateHandles checkpointStateHandles);
+
+	void declineCheckpoint(
+			final JobID jobID,
+			final ExecutionAttemptID executionAttemptID,
+			final CheckpointMetaData checkpoint);
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/a00619a4/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptor.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptor.java
index 7bbdb2a..b1ac665 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptor.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptor.java
@@ -23,6 +23,7 @@ import org.apache.flink.api.common.JobID;
 import org.apache.flink.api.common.TaskInfo;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.blob.BlobKey;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.state.ChainedStateHandle;
@@ -57,6 +58,9 @@ public final class TaskDeploymentDescriptor implements Serializable {
 	/** The ID referencing the attempt to execute the task. */
 	private final ExecutionAttemptID executionId;
 
+	/** The allocation ID of the slot in which the task shall be run */
+	private final AllocationID allocationID;
+
 	/** The task's name. */
 	private final String taskName;
 
@@ -158,6 +162,7 @@ public final class TaskDeploymentDescriptor implements Serializable {
 		this.operatorState = operatorState;
 		this.keyGroupState = keyGroupState;
 		this.partitionableOperatorState = partitionableOperatorStateHandles;
+		this.allocationID = new AllocationID();
 	}
 
 	public TaskDeploymentDescriptor(
@@ -322,6 +327,10 @@ public final class TaskDeploymentDescriptor implements Serializable {
 		return requiredClasspaths;
 	}
 
+	public AllocationID getAllocationID() {
+		return allocationID;
+	}
+
 	@Override
 	public String toString() {
 		return String.format("TaskDeploymentDescriptor [job id: %s, job vertex id: %s, " +

http://git-wip-us.apache.org/repos/asf/flink/blob/a00619a4/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/PartitionInfo.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/PartitionInfo.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/PartitionInfo.java
new file mode 100644
index 0000000..1a79a99
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/PartitionInfo.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.executiongraph;
+
+import org.apache.flink.runtime.deployment.InputChannelDeploymentDescriptor;
+import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
+import org.apache.flink.util.Preconditions;
+
+/**
+ * Contains information where to find a partition. The partition is defined by the
+ * {@link IntermediateDataSetID} and the partition location is specified by
+ * {@link InputChannelDeploymentDescriptor}.
+ */
+public class PartitionInfo {
+
+	private final IntermediateDataSetID intermediateDataSetID;
+	private final InputChannelDeploymentDescriptor inputChannelDeploymentDescriptor;
+
+	public PartitionInfo(IntermediateDataSetID intermediateResultPartitionID, InputChannelDeploymentDescriptor inputChannelDeploymentDescriptor) {
+		this.intermediateDataSetID = Preconditions.checkNotNull(intermediateResultPartitionID);
+		this.inputChannelDeploymentDescriptor = Preconditions.checkNotNull(inputChannelDeploymentDescriptor);
+	}
+
+	public IntermediateDataSetID getIntermediateDataSetID() {
+		return intermediateDataSetID;
+	}
+
+	public InputChannelDeploymentDescriptor getInputChannelDeploymentDescriptor() {
+		return inputChannelDeploymentDescriptor;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/a00619a4/flink-runtime/src/main/java/org/apache/flink/runtime/filecache/FileCache.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/filecache/FileCache.java b/flink-runtime/src/main/java/org/apache/flink/runtime/filecache/FileCache.java
index b5bdcaf..a07f1a0 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/filecache/FileCache.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/filecache/FileCache.java
@@ -33,8 +33,6 @@ import java.util.concurrent.TimeUnit;
 import org.apache.commons.io.FileUtils;
 import org.apache.flink.api.common.cache.DistributedCache.DistributedCacheEntry;
 import org.apache.flink.api.java.tuple.Tuple4;
-import org.apache.flink.configuration.ConfigConstants;
-import org.apache.flink.configuration.Configuration;
 import org.apache.flink.core.fs.FSDataInputStream;
 import org.apache.flink.core.fs.FSDataOutputStream;
 import org.apache.flink.core.fs.FileStatus;
@@ -44,6 +42,7 @@ import org.apache.flink.api.common.JobID;
 import org.apache.flink.runtime.util.ExecutorThreadFactory;
 import org.apache.flink.util.IOUtils;
 
+import org.apache.flink.util.Preconditions;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -71,17 +70,15 @@ public class FileCache {
 
 	// ------------------------------------------------------------------------
 
-	public FileCache(Configuration config) throws IOException {
-		
-		String tempDirs = config.getString(ConfigConstants.TASK_MANAGER_TMP_DIR_KEY,
-				ConfigConstants.DEFAULT_TASK_MANAGER_TMP_PATH);
+	public FileCache(String[] tempDirectories) throws IOException {
 
-		String[] directories = tempDirs.split(",|" + File.pathSeparator);
-		storageDirectories = new File[directories.length];
+		Preconditions.checkNotNull(tempDirectories);
 
-		for (int i = 0; i < directories.length; i++) {
+		storageDirectories = new File[tempDirectories.length];
+
+		for (int i = 0; i < tempDirectories.length; i++) {
 			String cacheDirName = "flink-dist-cache-" + UUID.randomUUID().toString();
-			storageDirectories[i] = new File(directories[i], cacheDirName);
+			storageDirectories[i] = new File(tempDirectories[i], cacheDirName);
 			String path = storageDirectories[i].getAbsolutePath();
 
 			if (storageDirectories[i].mkdirs()) {

http://git-wip-us.apache.org/repos/asf/flink/blob/a00619a4/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/InputSplitProvider.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/InputSplitProvider.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/InputSplitProvider.java
index e0cde17..464b13f 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/InputSplitProvider.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/InputSplitProvider.java
@@ -34,6 +34,7 @@ public interface InputSplitProvider {
 	 * @param userCodeClassLoader used to deserialize input splits
 	 * @return the next input split to be consumed by the calling task or <code>null</code> if the
 	 *         task shall not consume any further input splits.
+	 * @throws InputSplitProviderException if fetching the next input split fails
 	 */
-	InputSplit getNextInputSplit(ClassLoader userCodeClassLoader);
+	InputSplit getNextInputSplit(ClassLoader userCodeClassLoader) throws InputSplitProviderException;
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/a00619a4/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/InputSplitProviderException.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/InputSplitProviderException.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/InputSplitProviderException.java
new file mode 100644
index 0000000..ac73c6f
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/InputSplitProviderException.java
@@ -0,0 +1,36 @@
+/*
+ * 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.jobgraph.tasks;
+
+public class InputSplitProviderException extends Exception {
+
+	private static final long serialVersionUID = -8043190713983651548L;
+
+	public InputSplitProviderException(String message) {
+		super(message);
+	}
+
+	public InputSplitProviderException(String message, Throwable cause) {
+		super(message, cause);
+	}
+
+	public InputSplitProviderException(Throwable cause) {
+		super(cause);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/a00619a4/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/ExecutionGraphException.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/ExecutionGraphException.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/ExecutionGraphException.java
new file mode 100644
index 0000000..7c35f3d
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/ExecutionGraphException.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.jobmaster;
+
+import org.apache.flink.runtime.executiongraph.ExecutionGraph;
+
+/**
+ * Exceptions thrown by operations on the {@link ExecutionGraph} by the {@link JobMaster}.
+ */
+public class ExecutionGraphException extends JobManagerException {
+
+	private static final long serialVersionUID = -5439002256464886357L;
+
+	public ExecutionGraphException(String message) {
+		super(message);
+	}
+
+	public ExecutionGraphException(String message, Throwable cause) {
+		super(message, cause);
+	}
+
+	public ExecutionGraphException(Throwable cause) {
+		super(cause);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/a00619a4/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerException.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerException.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerException.java
new file mode 100644
index 0000000..bc2759d
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerException.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.jobmaster;
+
+/**
+ * Base exception thrown by the {@link JobMaster}.
+ */
+public class JobManagerException extends Exception {
+
+	private static final long serialVersionUID = -7290962952242188064L;
+
+	public JobManagerException(final String message) {
+		super(message);
+	}
+
+	public JobManagerException(final String message, Throwable cause) {
+		super(message, cause);
+	}
+
+	public JobManagerException(Throwable cause) {
+		super(cause);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/a00619a4/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 e67a167..8f3a342 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
@@ -38,6 +38,7 @@ import org.apache.flink.runtime.checkpoint.stats.SimpleCheckpointStatsTracker;
 import org.apache.flink.runtime.client.JobExecutionException;
 import org.apache.flink.runtime.client.JobSubmissionException;
 import org.apache.flink.runtime.client.SerializedJobExecutionResult;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
 import org.apache.flink.runtime.execution.ExecutionState;
 import org.apache.flink.runtime.concurrent.Future;
 import org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager;
@@ -60,9 +61,9 @@ import org.apache.flink.runtime.jobgraph.jsonplan.JsonPlanGenerator;
 import org.apache.flink.runtime.jobgraph.tasks.JobSnapshottingSettings;
 import org.apache.flink.runtime.jobmanager.OnCompletionActions;
 import org.apache.flink.runtime.jobmanager.scheduler.Scheduler;
-import org.apache.flink.runtime.jobmaster.message.NextInputSplit;
 import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener;
 import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+import org.apache.flink.runtime.messages.Acknowledge;
 import org.apache.flink.runtime.metrics.groups.JobManagerMetricGroup;
 import org.apache.flink.runtime.registration.RegisteredRpcConnection;
 import org.apache.flink.runtime.registration.RegistrationResponse;
@@ -71,6 +72,7 @@ import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway;
 import org.apache.flink.runtime.rpc.RpcEndpoint;
 import org.apache.flink.runtime.rpc.RpcMethod;
 import org.apache.flink.runtime.rpc.RpcService;
+import org.apache.flink.runtime.state.CheckpointStateHandles;
 import org.apache.flink.runtime.taskmanager.TaskExecutionState;
 import org.apache.flink.runtime.util.SerializedThrowable;
 import org.apache.flink.util.InstantiationUtil;
@@ -507,12 +509,18 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 	 * @return Acknowledge the task execution state update
 	 */
 	@RpcMethod
-	public boolean updateTaskExecutionState(final TaskExecutionState taskExecutionState) {
+	public Acknowledge updateTaskExecutionState(final TaskExecutionState taskExecutionState) throws ExecutionGraphException {
 		if (taskExecutionState == null) {
-			return false;
+			throw new NullPointerException("TaskExecutionState must not be null.");
+		}
+
+		if (executionGraph.updateState(taskExecutionState)) {
+			return Acknowledge.get();
 		} else {
-			return executionGraph.updateState(taskExecutionState);
+			throw new ExecutionGraphException("The execution attempt " +
+				taskExecutionState.getID() + " was not found.");
 		}
+
 	}
 
 	//----------------------------------------------------------------------------------------------\u2028
@@ -531,7 +539,7 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 	}
 
 	@RpcMethod
-	public NextInputSplit requestNextInputSplit(
+	public SerializedInputSplit requestNextInputSplit(
 		final JobVertexID vertexID,
 		final ExecutionAttemptID executionAttempt) throws Exception
 	{
@@ -569,7 +577,7 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 
 		try {
 			final byte[] serializedInputSplit = InstantiationUtil.serializeObject(nextInputSplit);
-			return new NextInputSplit(serializedInputSplit);
+			return new SerializedInputSplit(serializedInputSplit);
 		} 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 " +
@@ -591,8 +599,36 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 	}
 
 	@RpcMethod
-	public void scheduleOrUpdateConsumers(final ResultPartitionID partitionID) {
+	public Acknowledge scheduleOrUpdateConsumers(final ResultPartitionID partitionID) {
 		executionGraph.scheduleOrUpdateConsumers(partitionID);
+		return Acknowledge.get();
+	}
+
+	@RpcMethod
+	public void disconnectTaskManager(final ResourceID resourceID) {
+		throw new UnsupportedOperationException();
+	}
+
+	@RpcMethod
+	public void acknowledgeCheckpoint(
+		JobID jobID,
+		ExecutionAttemptID executionAttemptID,
+		long checkpointID,
+		CheckpointStateHandles checkpointStateHandles,
+		long synchronousDurationMillis,
+		long asynchronousDurationMillis,
+		long bytesBufferedInAlignment,
+		long alignmentDurationNanos) {
+		throw new UnsupportedOperationException();
+	}
+
+	@RpcMethod
+	public void declineCheckpoint(
+		JobID jobID,
+		ExecutionAttemptID executionAttemptID,
+		long checkpointID,
+		long checkpointTimestamp) {
+		throw new UnsupportedOperationException();
 	}
 
 	//----------------------------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/flink/blob/a00619a4/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 686a3f3..e3e57d4 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterGateway.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterGateway.java
@@ -18,6 +18,9 @@
 
 package org.apache.flink.runtime.jobmaster;
 
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.checkpoint.CheckpointCoordinatorGateway;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
 import org.apache.flink.runtime.concurrent.Future;
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
 import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
@@ -27,8 +30,8 @@ import org.apache.flink.runtime.io.network.partition.ResultPartition;
 import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
 import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
-import org.apache.flink.runtime.jobmaster.message.NextInputSplit;
-import org.apache.flink.runtime.rpc.RpcGateway;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.rpc.RpcTimeout;
 import org.apache.flink.runtime.taskmanager.TaskExecutionState;
 
 import java.util.UUID;
@@ -36,7 +39,7 @@ import java.util.UUID;
 /**
  * {@link JobMaster} rpc gateway interface
  */
-public interface JobMasterGateway extends RpcGateway {
+public interface JobMasterGateway extends CheckpointCoordinatorGateway {
 
 	/**
 	 * Starting the job under the given leader session ID.
@@ -57,20 +60,19 @@ public interface JobMasterGateway extends RpcGateway {
 	 * @param taskExecutionState New task execution state for a given task
 	 * @return Future flag of the task execution state update result
 	 */
-	Future<Boolean> updateTaskExecutionState(TaskExecutionState taskExecutionState);
+	Future<Acknowledge> updateTaskExecutionState(TaskExecutionState taskExecutionState);
 
 	/**
 	 * Requesting next input split for the {@link ExecutionJobVertex}. The next input split is sent back to the sender
-	 * as a {@link NextInputSplit} message.
+	 * as a {@link SerializedInputSplit} message.
 	 *
 	 * @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.
-	 * @throws Exception if some error occurred or information mismatch.
 	 */
-	Future<NextInputSplit> requestNextInputSplit(
+	Future<SerializedInputSplit> requestNextInputSplit(
 		final JobVertexID vertexID,
-		final ExecutionAttemptID executionAttempt) throws Exception;
+		final ExecutionAttemptID executionAttempt);
 
 	/**
 	 * Requests the current state of the partition.
@@ -96,6 +98,16 @@ public interface JobMasterGateway extends RpcGateway {
 	 * 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
+	 * @return Future acknowledge of the schedule or update operation
 	 */
-	void scheduleOrUpdateConsumers(final ResultPartitionID partitionID);
+	Future<Acknowledge> scheduleOrUpdateConsumers(final ResultPartitionID partitionID, @RpcTimeout final Time timeout);
+
+	/**
+	 * Disconnects the given {@link org.apache.flink.runtime.taskexecutor.TaskExecutor} from the
+	 * {@link JobMaster}.
+	 *
+	 * @param resourceID identifying the TaskManager to disconnect
+	 */
+	void disconnectTaskManager(ResourceID resourceID);
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/a00619a4/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 792bfd5..e8fb5bb 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
@@ -356,7 +356,7 @@ public class MiniClusterJobDispatcher {
 			final Throwable runnerException = this.runnerException;
 			final JobExecutionResult result = this.result;
 
-			// (1) we check if teh job terminated with an exception
+			// (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

http://git-wip-us.apache.org/repos/asf/flink/blob/a00619a4/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/SerializedInputSplit.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/SerializedInputSplit.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/SerializedInputSplit.java
new file mode 100644
index 0000000..bfdc65a
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/SerializedInputSplit.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.jobmaster;
+
+import java.io.Serializable;
+
+public class SerializedInputSplit implements Serializable {
+	private static final long serialVersionUID = -2063021844254152064L;
+
+	private final byte[] inputSplitData;
+
+	public SerializedInputSplit(byte[] inputSplitData) {
+		this.inputSplitData = inputSplitData;
+	}
+
+	public byte[] getInputSplitData() {
+		return inputSplitData;
+	}
+
+	public boolean isEmpty() {
+		return inputSplitData == null || inputSplitData.length == 0;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/a00619a4/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/message/NextInputSplit.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/message/NextInputSplit.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/message/NextInputSplit.java
deleted file mode 100644
index fe511ed..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/message/NextInputSplit.java
+++ /dev/null
@@ -1,39 +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;
-
-/**
- * Contains the next input split for a task.
- */
-public class NextInputSplit implements Serializable {
-
-	private static final long serialVersionUID = -1355784074565856240L;
-
-	private final byte[] splitData;
-
-	public NextInputSplit(final byte[] splitData) {
-		this.splitData = splitData;
-	}
-
-	public byte[] getSplitData() {
-		return splitData;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/a00619a4/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DataSourceTask.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DataSourceTask.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DataSourceTask.java
index c062bf8..1c751fd 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DataSourceTask.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DataSourceTask.java
@@ -33,6 +33,7 @@ import org.apache.flink.runtime.execution.Environment;
 import org.apache.flink.runtime.io.network.api.writer.RecordWriter;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
 import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider;
+import org.apache.flink.runtime.jobgraph.tasks.InputSplitProviderException;
 import org.apache.flink.runtime.operators.chaining.ChainedDriver;
 import org.apache.flink.runtime.operators.chaining.ExceptionInChainedStubException;
 import org.apache.flink.runtime.operators.util.DistributedRuntimeUDFContext;
@@ -332,9 +333,14 @@ public class DataSourceTask<OT> extends AbstractInvokable {
 				if (nextSplit != null) {
 					return true;
 				}
-				
-				InputSplit split = provider.getNextInputSplit(getUserCodeClassLoader());
-				
+
+				final InputSplit split;
+				try {
+					split = provider.getNextInputSplit(getUserCodeClassLoader());
+				} catch (InputSplitProviderException e) {
+					throw new RuntimeException("Could not retrieve next input split.", e);
+				}
+
 				if (split != null) {
 					this.nextSplit = split;
 					return true;

http://git-wip-us.apache.org/repos/asf/flink/blob/a00619a4/flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateRegistryGateway.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateRegistryGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateRegistryGateway.java
new file mode 100644
index 0000000..d285074
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateRegistryGateway.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.query;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.rpc.RpcGateway;
+import org.apache.flink.runtime.state.KeyGroupRange;
+
+public interface KvStateRegistryGateway extends RpcGateway {
+	/**
+	 * Notifies the listener about a registered KvState instance.
+	 *
+	 * @param jobId            Job ID the KvState instance belongs to
+	 * @param jobVertexId      JobVertexID the KvState instance belongs to
+	 * @param keyGroupRange    Key group range the KvState instance belongs to
+	 * @param registrationName Name under which the KvState is registered
+	 * @param kvStateId        ID of the KvState instance
+	 */
+	void notifyKvStateRegistered(
+		JobID jobId,
+		JobVertexID jobVertexId,
+		KeyGroupRange keyGroupRange,
+		String registrationName,
+		KvStateID kvStateId,
+		KvStateServerAddress kvStateServerAddress);
+
+	/**
+	 * Notifies the listener about an unregistered KvState instance.
+	 *
+	 * @param jobId            Job ID the KvState instance belongs to
+	 * @param jobVertexId      JobVertexID the KvState instance belongs to
+	 * @param keyGroupRange    Key group range the KvState instance belongs to
+	 * @param registrationName Name under which the KvState is registered
+	 */
+	void notifyKvStateUnregistered(
+		JobID jobId,
+		JobVertexID jobVertexId,
+		KeyGroupRange keyGroupRange,
+		String registrationName);
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/a00619a4/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
new file mode 100644
index 0000000..ef62ef1
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/JobManagerConnection.java
@@ -0,0 +1,91 @@
+/*
+ * 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.runtime.execution.librarycache.LibraryCacheManager;
+import org.apache.flink.runtime.io.network.netty.PartitionStateChecker;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionConsumableNotifier;
+import org.apache.flink.runtime.jobmaster.JobMasterGateway;
+import org.apache.flink.runtime.taskmanager.CheckpointResponder;
+import org.apache.flink.runtime.taskmanager.TaskManagerActions;
+import org.apache.flink.util.Preconditions;
+
+/**
+ * Container class for JobManager specific communication utils used by the {@link TaskExecutor}.
+ */
+public class JobManagerConnection {
+
+	// Gateway to the job master
+	private final JobMasterGateway jobMasterGateway;
+
+	// Task manager actions with respect to the connected job manager
+	private final TaskManagerActions taskManagerActions;
+
+	// Checkpoint responder for the specific job manager
+	private final CheckpointResponder checkpointResponder;
+
+	// Library cache manager connected to the specific job manager
+	private final LibraryCacheManager libraryCacheManager;
+
+	// Result partition consumable notifier for the specific job manager
+	private final ResultPartitionConsumableNotifier resultPartitionConsumableNotifier;
+
+	// Partition state checker for the specific job manager
+	private final PartitionStateChecker partitionStateChecker;
+
+	public JobManagerConnection(
+		JobMasterGateway jobMasterGateway,
+		TaskManagerActions taskManagerActions,
+		CheckpointResponder checkpointResponder,
+		LibraryCacheManager libraryCacheManager,
+		ResultPartitionConsumableNotifier resultPartitionConsumableNotifier,
+		PartitionStateChecker partitionStateChecker) {
+
+		this.jobMasterGateway = Preconditions.checkNotNull(jobMasterGateway);
+		this.taskManagerActions = Preconditions.checkNotNull(taskManagerActions);
+		this.checkpointResponder = Preconditions.checkNotNull(checkpointResponder);
+		this.libraryCacheManager = Preconditions.checkNotNull(libraryCacheManager);
+		this.resultPartitionConsumableNotifier = Preconditions.checkNotNull(resultPartitionConsumableNotifier);
+		this.partitionStateChecker = Preconditions.checkNotNull(partitionStateChecker);
+	}
+
+	public JobMasterGateway getJobManagerGateway() {
+		return jobMasterGateway;
+	}
+
+	public TaskManagerActions getTaskManagerActions() {
+		return taskManagerActions;
+	}
+
+	public CheckpointResponder getCheckpointResponder() {
+		return checkpointResponder;
+	}
+
+	public LibraryCacheManager getLibraryCacheManager() {
+		return libraryCacheManager;
+	}
+
+	public ResultPartitionConsumableNotifier getResultPartitionConsumableNotifier() {
+		return resultPartitionConsumableNotifier;
+	}
+
+	public PartitionStateChecker getPartitionStateChecker() {
+		return partitionStateChecker;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/a00619a4/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 c0041a3..35b639b 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,15 +18,48 @@
 
 package org.apache.flink.runtime.taskexecutor;
 
+import org.apache.flink.configuration.UnmodifiableConfiguration;
+import org.apache.flink.runtime.accumulators.AccumulatorSnapshot;
+import org.apache.flink.runtime.blob.BlobCache;
+import org.apache.flink.runtime.broadcast.BroadcastVariableManager;
 import org.apache.flink.runtime.clusterframework.types.AllocationID;
 import org.apache.flink.runtime.clusterframework.types.ResourceID;
 import org.apache.flink.runtime.clusterframework.types.SlotID;
 import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.concurrent.ApplyFunction;
+import org.apache.flink.runtime.concurrent.Future;
+import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor;
+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.PartitionInfo;
+import org.apache.flink.runtime.filecache.FileCache;
+import org.apache.flink.runtime.io.network.netty.PartitionStateChecker;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionConsumableNotifier;
+import org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate;
+import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
+import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider;
+import org.apache.flink.runtime.jobmaster.JobMasterGateway;
+import org.apache.flink.runtime.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.TMSlotRequestReply;
 import org.apache.flink.runtime.rpc.FatalErrorHandler;
+import org.apache.flink.runtime.taskexecutor.exceptions.CheckpointException;
+import org.apache.flink.runtime.taskexecutor.exceptions.PartitionException;
+import org.apache.flink.runtime.taskexecutor.exceptions.TaskException;
+import org.apache.flink.runtime.taskexecutor.exceptions.TaskSubmissionException;
+import org.apache.flink.runtime.taskexecutor.rpc.RpcCheckpointResponder;
+import org.apache.flink.runtime.taskexecutor.rpc.RpcInputSplitProvider;
+import org.apache.flink.runtime.taskexecutor.rpc.RpcPartitionStateChecker;
+import org.apache.flink.runtime.taskexecutor.rpc.RpcResultPartitionConsumableNotifier;
+import org.apache.flink.runtime.taskmanager.CheckpointResponder;
+import org.apache.flink.runtime.taskmanager.Task;
+import org.apache.flink.runtime.taskmanager.TaskExecutionState;
+import org.apache.flink.runtime.taskmanager.TaskManagerActions;
 import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
 import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
@@ -38,11 +71,17 @@ import org.apache.flink.runtime.rpc.RpcEndpoint;
 import org.apache.flink.runtime.rpc.RpcMethod;
 import org.apache.flink.runtime.rpc.RpcService;
 
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import org.apache.flink.runtime.taskmanager.TaskManagerRuntimeInfo;
+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.Iterator;
+import java.util.Map;
 import java.util.UUID;
 
 import static org.apache.flink.util.Preconditions.checkArgument;
@@ -50,12 +89,10 @@ import static org.apache.flink.util.Preconditions.checkNotNull;
 
 /**
  * TaskExecutor implementation. The task executor is responsible for the execution of multiple
- * {@link org.apache.flink.runtime.taskmanager.Task}.
+ * {@link Task}.
  */
 public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 
-	private static final Logger LOG = LoggerFactory.getLogger(TaskExecutor.class);
-
 	/** The connection information of this task manager */
 	private final TaskManagerLocation taskManagerLocation;
 
@@ -77,19 +114,38 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 	/** The metric registry in the task manager */
 	private final MetricRegistry metricRegistry;
 
-	/** The number of slots in the task manager, should be 1 for YARN */
-	private final int numberOfSlots;
-
 	/** The fatal error handler to use in case of a fatal error */
 	private final FatalErrorHandler fatalErrorHandler;
 
+	private final TaskManagerMetricGroup taskManagerMetricGroup;
+
+	private final BroadcastVariableManager broadcastVariableManager;
+	
 	/** Slots which have become available but haven't been confirmed by the RM */
 	private final Set<SlotID> unconfirmedFreeSlots;
 
+
+	private final FileCache fileCache;
+
+	// TODO: Try to get rid of it
+	private final TaskManagerRuntimeInfo taskManagerRuntimeInfo;
+
 	// --------- resource manager --------
 
 	private TaskExecutorToResourceManagerConnection resourceManagerConnection;
 
+	// --------- job manager connections -----------
+
+	private Map<ResourceID, JobManagerConnection> jobManagerConnections;
+
+	// --------- Slot allocation table --------
+
+	private Map<AllocationID, TaskSlot> taskSlots;
+
+	// --------- Slot allocation table --------
+
+	private Map<ExecutionAttemptID, TaskSlotMapping> taskSlotMappings;
+
 	// ------------------------------------------------------------------------
 
 	public TaskExecutor(
@@ -101,6 +157,9 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 		NetworkEnvironment networkEnvironment,
 		HighAvailabilityServices haServices,
 		MetricRegistry metricRegistry,
+		TaskManagerMetricGroup taskManagerMetricGroup,
+		BroadcastVariableManager broadcastVariableManager,
+		FileCache fileCache,
 		FatalErrorHandler fatalErrorHandler) {
 
 		super(rpcService);
@@ -115,10 +174,19 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 		this.haServices = checkNotNull(haServices);
 		this.metricRegistry = checkNotNull(metricRegistry);
 		this.fatalErrorHandler = checkNotNull(fatalErrorHandler);
+		this.taskManagerMetricGroup = checkNotNull(taskManagerMetricGroup);
+		this.broadcastVariableManager = checkNotNull(broadcastVariableManager);
+		this.fileCache = checkNotNull(fileCache);
+		this.taskManagerRuntimeInfo = new TaskManagerRuntimeInfo(
+			taskManagerLocation.getHostname(),
+			new UnmodifiableConfiguration(taskManagerConfiguration.getConfiguration()),
+			taskManagerConfiguration.getTmpDirPaths());
 
-		this.numberOfSlots =  taskManagerConfiguration.getNumberSlots();
+		this.jobManagerConnections = new HashMap<>(4);
 
 		this.unconfirmedFreeSlots = new HashSet<>();
+		this.taskSlots = new HashMap<>(taskManagerConfiguration.getNumberSlots());
+		this.taskSlotMappings = new HashMap<>(taskManagerConfiguration.getNumberSlots() * 2);
 	}
 
 	// ------------------------------------------------------------------------
@@ -137,12 +205,436 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 		}
 	}
 
+	/**
+	 * Called to shut down the TaskManager. The method closes all TaskManager services.
+	 */
+	@Override
+	public void shutDown() {
+		log.info("Stopping TaskManager {}.", getAddress());
+
+		if (resourceManagerConnection.isConnected()) {
+			try {
+				resourceManagerConnection.close();
+			} catch (Exception e) {
+				log.error("Could not cleanly close the ResourceManager connection.", e);
+			}
+		}
+
+		try {
+			ioManager.shutdown();
+		} catch (Exception e) {
+			log.error("IOManager did not shut down properly.", e);
+		}
+
+		try {
+			memoryManager.shutdown();
+		} catch (Exception e) {
+			log.error("MemoryManager did not shut down properly.", e);
+		}
+
+		try {
+			networkEnvironment.shutdown();
+		} catch (Exception e) {
+			log.error("Network environment did not shut down properly.", e);
+		}
+
+		try {
+			fileCache.shutdown();
+		} catch (Exception e) {
+			log.error("File cache did not shut down properly.", e);
+		}
+
+		try {
+			metricRegistry.shutdown();
+		} catch (Exception e) {
+			log.error("MetricRegistry did not shut down properly.", e);
+		}
+
+		log.info("Stopped TaskManager {}.", getAddress());
+	}
+
+	// ========================================================================
+	//  RPC methods
+	// ========================================================================
+
+	// ----------------------------------------------------------------------
+	// Task lifecycle RPCs
+	// ----------------------------------------------------------------------
+
+	@RpcMethod
+	public Acknowledge submitTask(TaskDeploymentDescriptor tdd, ResourceID jobManagerID) throws TaskSubmissionException {
+
+		JobManagerConnection jobManagerConnection = getJobManagerConnection(jobManagerID);
+
+		if (jobManagerConnection == null) {
+			final String message = "Could not submit task because JobManager " + jobManagerID +
+				" was not associated.";
+
+			log.debug(message);
+			throw new TaskSubmissionException(message);
+		}
+
+		TaskSlot taskSlot = taskSlots.get(tdd.getAllocationID());
+
+		if (taskSlot == null) {
+			final String message = "No task slot allocated for allocation ID " + tdd.getAllocationID() + '.';
+			log.debug(message);
+			throw new TaskSubmissionException(message);
+		}
+
+		TaskMetricGroup taskMetricGroup = taskManagerMetricGroup.addTaskForJob(tdd);
+
+		InputSplitProvider inputSplitProvider = new RpcInputSplitProvider(
+			jobManagerConnection.getJobManagerGateway(),
+			tdd.getJobID(),
+			tdd.getVertexID(),
+			tdd.getExecutionId(),
+			taskManagerConfiguration.getTimeout());
+
+		TaskManagerActions taskManagerActions = jobManagerConnection.getTaskManagerActions();
+		CheckpointResponder checkpointResponder = jobManagerConnection.getCheckpointResponder();
+		LibraryCacheManager libraryCache = jobManagerConnection.getLibraryCacheManager();
+		ResultPartitionConsumableNotifier resultPartitionConsumableNotifier = jobManagerConnection.getResultPartitionConsumableNotifier();
+		PartitionStateChecker partitionStateChecker = jobManagerConnection.getPartitionStateChecker();
+
+		Task task = new Task(
+			tdd,
+			memoryManager,
+			ioManager,
+			networkEnvironment,
+			broadcastVariableManager,
+			taskManagerActions,
+			inputSplitProvider,
+			checkpointResponder,
+			libraryCache,
+			fileCache,
+			taskManagerRuntimeInfo,
+			taskMetricGroup,
+			resultPartitionConsumableNotifier,
+			partitionStateChecker,
+			getRpcService().getExecutor());
+
+		log.info("Received task {}.", task.getTaskInfo().getTaskNameWithSubtasks());
+
+		if(taskSlot.add(task)) {
+			TaskSlotMapping taskSlotMapping = new TaskSlotMapping(task, taskSlot);
+
+			taskSlotMappings.put(task.getExecutionId(), taskSlotMapping);
+			task.startTaskThread();
+
+			return Acknowledge.get();
+		} else {
+			final String message = "TaskManager already contains a task for id " +
+				task.getExecutionId() + '.';
+
+			log.debug(message);
+			throw new TaskSubmissionException(message);
+		}
+	}
+
+	@RpcMethod
+	public Acknowledge cancelTask(ExecutionAttemptID executionAttemptID) throws TaskException {
+		final Task task = getTask(executionAttemptID);
+
+		if (task != null) {
+			try {
+				task.cancelExecution();
+				return Acknowledge.get();
+			} catch (Throwable t) {
+				throw new TaskException("Cannot cancel task for execution " + executionAttemptID + '.', t);
+			}
+		} else {
+			final String message = "Cannot find task to stop for execution " + executionAttemptID + '.';
+
+			log.debug(message);
+			throw new TaskException(message);
+		}
+	}
+
+	@RpcMethod
+	public Acknowledge stopTask(ExecutionAttemptID executionAttemptID) throws TaskException {
+		final Task task = getTask(executionAttemptID);
+
+		if (task != null) {
+			try {
+				task.stopExecution();
+				return Acknowledge.get();
+			} catch (Throwable t) {
+				throw new TaskException("Cannot stop task for execution " + executionAttemptID + '.', t);
+			}
+		} else {
+			final String message = "Cannot find task to stop for execution " + executionAttemptID + '.';
+
+			log.debug(message);
+			throw new TaskException(message);
+		}
+	}
+
+	// ----------------------------------------------------------------------
+	// Partition lifecycle RPCs
+	// ----------------------------------------------------------------------
+
+	@RpcMethod
+	public Acknowledge updatePartitions(final ExecutionAttemptID executionAttemptID, Collection<PartitionInfo> partitionInfos) throws PartitionException {
+		final Task task = getTask(executionAttemptID);
+
+		if (task != null) {
+			for (final PartitionInfo partitionInfo: partitionInfos) {
+				IntermediateDataSetID intermediateResultPartitionID = partitionInfo.getIntermediateDataSetID();
+
+				final SingleInputGate singleInputGate = task.getInputGateById(intermediateResultPartitionID);
+
+				if (singleInputGate != null) {
+					// Run asynchronously because it might be blocking
+					getRpcService().execute(new Runnable() {
+						@Override
+						public void run() {
+							try {
+								singleInputGate.updateInputChannel(partitionInfo.getInputChannelDeploymentDescriptor());
+							} catch (IOException | InterruptedException e) {
+								log.error("Could not update input data location for task {}. Trying to fail task.", task.getTaskInfo().getTaskName(), e);
+
+								try {
+									task.failExternally(e);
+								} catch (RuntimeException re) {
+									// TODO: Check whether we need this or make exception in failExtenally checked
+									log.error("Failed canceling task with execution ID {} after task update failure.", executionAttemptID, re);
+								}
+							}
+						}
+					});
+				} else {
+					throw new PartitionException("No reader with ID " +
+						intermediateResultPartitionID + " for task " + executionAttemptID +
+						" was found.");
+				}
+			}
+
+			return Acknowledge.get();
+		} else {
+			log.debug("Discard update for input partitions of task {}. Task is no longer running.", executionAttemptID);
+			return Acknowledge.get();
+		}
+	}
+
+	@RpcMethod
+	public void failPartition(ExecutionAttemptID executionAttemptID) {
+		log.info("Discarding the results produced by task execution {}.", executionAttemptID);
+
+		try {
+			networkEnvironment.getResultPartitionManager().releasePartitionsProducedBy(executionAttemptID);
+		} catch (Throwable t) {
+			// TODO: Do we still need this catch branch?
+			onFatalError(t);
+		}
+
+		// TODO: Maybe it's better to return an Acknowledge here to notify the JM about the success/failure with an Exception
+	}
+
+	// ----------------------------------------------------------------------
+	// Checkpointing RPCs
+	// ----------------------------------------------------------------------
+
+	@RpcMethod
+	public Acknowledge triggerCheckpoint(ExecutionAttemptID executionAttemptID, long checkpointId, long checkpointTimestamp) throws CheckpointException {
+		log.debug("Trigger checkpoint {}@{} for {}.", checkpointId, checkpointTimestamp, executionAttemptID);
+
+		final Task task = getTask(executionAttemptID);
+
+		if (task != null) {
+			task.triggerCheckpointBarrier(checkpointId, checkpointTimestamp);
+
+			return Acknowledge.get();
+		} else {
+			final String message = "TaskManager received a checkpoint request for unknown task " + executionAttemptID + '.';
+
+			log.debug(message);
+			throw new CheckpointException(message);
+		}
+	}
+
+	@RpcMethod
+	public Acknowledge confirmCheckpoint(ExecutionAttemptID executionAttemptID, long checkpointId, long checkpointTimestamp) throws CheckpointException {
+		log.debug("Confirm checkpoint {}@{} for {}.", checkpointId, checkpointTimestamp, executionAttemptID);
+
+		final Task task = getTask(executionAttemptID);
+
+		if (task != null) {
+			task.notifyCheckpointComplete(checkpointId);
+
+			return Acknowledge.get();
+		} else {
+			final String message = "TaskManager received a checkpoint confirmation for unknown task " + executionAttemptID + '.';
+
+			log.debug(message);
+			throw new CheckpointException(message);
+		}
+	}
+
+	/**
+	 * Requests a slot from the TaskManager
+	 *
+	 * @param slotID Slot id for the request
+	 * @param allocationID id for the request
+	 * @param resourceManagerLeaderID current leader id of the ResourceManager
+	 * @return answer to the slot request
+	 */
+	@RpcMethod
+	public TMSlotRequestReply requestSlot(SlotID slotID, AllocationID allocationID, UUID resourceManagerLeaderID) {
+		if (!resourceManagerConnection.getTargetLeaderId().equals(resourceManagerLeaderID)) {
+			return new TMSlotRequestRejected(
+				resourceManagerConnection.getRegistrationId(), getResourceID(), allocationID);
+		}
+		if (unconfirmedFreeSlots.contains(slotID)) {
+			// check if request has not been blacklisted because the notification of a free slot
+			// has not been confirmed by the ResourceManager
+			return new TMSlotRequestRejected(
+				resourceManagerConnection.getRegistrationId(), getResourceID(), allocationID);
+		}
+		return new TMSlotRequestRegistered(new InstanceID(), ResourceID.generate(), allocationID);
+
+	}
+
 	// ------------------------------------------------------------------------
-	//  RPC methods - ResourceManager related
+	//  Internal methods
 	// ------------------------------------------------------------------------
 
-	@RpcMethod
-	public void notifyOfNewResourceManagerLeader(String newLeaderAddress, UUID newLeaderId) {
+	private JobManagerConnection getJobManagerConnection(ResourceID jobManagerID) {
+		return jobManagerConnections.get(jobManagerID);
+	}
+
+	private Task getTask(ExecutionAttemptID executionAttemptID) {
+		TaskSlotMapping taskSlotMapping = taskSlotMappings.get(executionAttemptID);
+
+		if (taskSlotMapping != null) {
+			return taskSlotMapping.getTask();
+		} else {
+			return null;
+		}
+	}
+
+	private Task removeTask(ExecutionAttemptID executionAttemptID) {
+		TaskSlotMapping taskSlotMapping = taskSlotMappings.remove(executionAttemptID);
+
+		if (taskSlotMapping != null) {
+			final Task task = taskSlotMapping.getTask();
+			final TaskSlot taskSlot = taskSlotMapping.getTaskSlot();
+
+			taskSlot.remove(task);
+
+			return task;
+		} else {
+			return null;
+		}
+	}
+
+	private Iterable<Task> getAllTasks() {
+		final Iterator<TaskSlotMapping> taskEntryIterator = taskSlotMappings.values().iterator();
+		final Iterator<Task> iterator = new Iterator<Task>() {
+			@Override
+			public boolean hasNext() {
+				return taskEntryIterator.hasNext();
+			}
+
+			@Override
+			public Task next() {
+				return taskEntryIterator.next().getTask();
+			}
+
+			@Override
+			public void remove() {
+				taskEntryIterator.remove();
+			}
+		};
+
+		return new Iterable<Task>() {
+			@Override
+			public Iterator<Task> iterator() {
+				return iterator;
+			}
+		};
+	}
+
+	private void clearTasks() {
+		taskSlotMappings.clear();
+
+		for (TaskSlot taskSlot: taskSlots.values()) {
+			taskSlot.clear();
+		}
+	}
+
+	private void failTask(final ExecutionAttemptID executionAttemptID, final Throwable cause) {
+		final Task task = getTask(executionAttemptID);
+
+		if (task != null) {
+			try {
+				task.failExternally(cause);
+			} catch (Throwable t) {
+				log.error("Could not fail task {}.", executionAttemptID, t);
+			}
+		} else {
+			log.debug("Cannot find task to fail for execution {}.", executionAttemptID);
+		}
+	}
+
+	private void cancelAndClearAllTasks(Throwable cause) {
+		log.info("Cancellaing all computations and discarding all cached data.");
+
+		Iterable<Task> tasks = getAllTasks();
+
+		for (Task task: tasks) {
+			task.failExternally(cause);
+		}
+
+		clearTasks();
+	}
+
+	private void updateTaskExecutionState(final JobMasterGateway jobMasterGateway, final TaskExecutionState taskExecutionState) {
+		final ExecutionAttemptID executionAttemptID = taskExecutionState.getID();
+
+		Future<Acknowledge> futureAcknowledge = jobMasterGateway.updateTaskExecutionState(taskExecutionState);
+
+		futureAcknowledge.exceptionallyAsync(new ApplyFunction<Throwable, Void>() {
+			@Override
+			public Void apply(Throwable value) {
+				failTask(executionAttemptID, value);
+
+				return null;
+			}
+		}, getMainThreadExecutor());
+	}
+
+	private void unregisterTaskAndNotifyFinalState(final JobMasterGateway jobMasterGateway, ExecutionAttemptID executionAttemptID) {
+		Task task = removeTask(executionAttemptID);
+
+		if (task != null) {
+			if (!task.getExecutionState().isTerminal()) {
+				try {
+					task.failExternally(new IllegalStateException("Task is being remove from TaskManager."));
+				} catch (Exception e) {
+					log.error("Could not properly fail task.", e);
+				}
+			}
+
+			log.info("Un-registering task and sending final execution state {} to JobManager for task {} {}.",
+				task.getExecutionState(), task.getTaskInfo().getTaskName(), task.getExecutionId());
+
+			AccumulatorSnapshot accumulatorSnapshot = task.getAccumulatorRegistry().getSnapshot();
+
+			updateTaskExecutionState(
+				jobMasterGateway,
+				new TaskExecutionState(
+					task.getJobID(),
+					task.getExecutionId(),
+					task.getExecutionState(),
+					task.getFailureCause(),
+					accumulatorSnapshot));
+		} else {
+			log.error("Cannot find task with ID {} to unregister.", executionAttemptID);
+		}
+	}
+
+	private void notifyOfNewResourceManagerLeader(String newLeaderAddress, UUID newLeaderId) {
 		if (resourceManagerConnection != null) {
 			if (newLeaderAddress != null) {
 				// the resource manager switched to a new leader
@@ -178,28 +670,46 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 		}
 	}
 
-	/**
-	 * Requests a slot from the TaskManager
-	 *
-	 * @param slotID Slot id for the request
-	 * @param allocationID id for the request
-	 * @param resourceManagerLeaderID current leader id of the ResourceManager
-	 * @return answer to the slot request
-	 */
-	@RpcMethod
-	public TMSlotRequestReply requestSlot(SlotID slotID, AllocationID allocationID, UUID resourceManagerLeaderID) {
-		if (!resourceManagerConnection.getTargetLeaderId().equals(resourceManagerLeaderID)) {
-			return new TMSlotRequestRejected(
-				resourceManagerConnection.getRegistrationId(), getResourceID(), allocationID);
-		}
-		if (unconfirmedFreeSlots.contains(slotID)) {
-			// check if request has not been blacklisted because the notification of a free slot
-			// has not been confirmed by the ResourceManager
-			return new TMSlotRequestRejected(
-				resourceManagerConnection.getRegistrationId(), getResourceID(), allocationID);
-		}
-		return new TMSlotRequestRegistered(new InstanceID(), ResourceID.generate(), allocationID);
+	private JobManagerConnection associateWithJobManager(JobMasterGateway jobMasterGateway, int blobPort) {
+		Preconditions.checkNotNull(jobMasterGateway);
+		Preconditions.checkArgument(blobPort > 0 || blobPort <= 65535, "Blob port is out of range.");
 
+		TaskManagerActions taskManagerActions = new TaskManagerActionsImpl(jobMasterGateway);
+
+		CheckpointResponder checkpointResponder = new RpcCheckpointResponder(jobMasterGateway);
+
+		InetSocketAddress address = new InetSocketAddress(jobMasterGateway.getAddress(), blobPort);
+
+		BlobCache blobCache = new BlobCache(address, taskManagerConfiguration.getConfiguration());
+
+		LibraryCacheManager libraryCacheManager = new BlobLibraryCacheManager(
+			blobCache,
+			taskManagerConfiguration.getCleanupInterval());
+
+		ResultPartitionConsumableNotifier resultPartitionConsumableNotifier = new RpcResultPartitionConsumableNotifier(
+			jobMasterGateway,
+			getRpcService().getExecutor(),
+			taskManagerConfiguration.getTimeout());
+
+		PartitionStateChecker partitionStateChecker = new RpcPartitionStateChecker(jobMasterGateway);
+
+		return new JobManagerConnection(
+			jobMasterGateway,
+			taskManagerActions,
+			checkpointResponder,
+			libraryCacheManager,
+			resultPartitionConsumableNotifier,
+			partitionStateChecker);
+	}
+
+	private void disassociateFromJobManager(JobManagerConnection jobManagerConnection) throws IOException {
+		if (jobManagerConnection != null) {
+			JobMasterGateway jobManagerGateway = jobManagerConnection.getJobManagerGateway();
+
+			jobManagerGateway.disconnectTaskManager(getResourceID());
+
+			jobManagerConnection.getLibraryCacheManager().shutdown();
+		}
 	}
 
 	// ------------------------------------------------------------------------
@@ -237,8 +747,7 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 	 * @param t The exception describing the fatal error
 	 */
 	void onFatalError(Throwable t) {
-		// to be determined, probably delegate to a fatal error handler that 
-		// would either log (mini cluster) ot kill the process (yarn, mesos, ...)
+		log.error("Fatal error occurred.", t);
 		fatalErrorHandler.onFatalError(t);
 	}
 
@@ -266,8 +775,13 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 	private class ResourceManagerLeaderListener implements LeaderRetrievalListener {
 
 		@Override
-		public void notifyLeaderAddress(String leaderAddress, UUID leaderSessionID) {
-			getSelf().notifyOfNewResourceManagerLeader(leaderAddress, leaderSessionID);
+		public void notifyLeaderAddress(final String leaderAddress, final UUID leaderSessionID) {
+			runAsync(new Runnable() {
+				@Override
+				public void run() {
+					notifyOfNewResourceManagerLeader(leaderAddress, leaderSessionID);
+				}
+			});
 		}
 
 		@Override
@@ -276,4 +790,43 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 		}
 	}
 
+	private class TaskManagerActionsImpl implements TaskManagerActions {
+		private final JobMasterGateway jobMasterGateway;
+
+		private TaskManagerActionsImpl(JobMasterGateway jobMasterGateway) {
+			this.jobMasterGateway = Preconditions.checkNotNull(jobMasterGateway);
+		}
+
+		@Override
+		public void notifyFinalState(final ExecutionAttemptID executionAttemptID) {
+			runAsync(new Runnable() {
+				@Override
+				public void run() {
+					unregisterTaskAndNotifyFinalState(jobMasterGateway, executionAttemptID);
+				}
+			});
+		}
+
+		@Override
+		public void notifyFatalError(String message, Throwable cause) {
+			log.error(message, cause);
+			fatalErrorHandler.onFatalError(cause);
+		}
+
+		@Override
+		public void failTask(final ExecutionAttemptID executionAttemptID, final Throwable cause) {
+			runAsync(new Runnable() {
+				@Override
+				public void run() {
+					TaskExecutor.this.failTask(executionAttemptID, cause);
+				}
+			});
+		}
+
+		@Override
+		public void updateTaskExecutionState(final TaskExecutionState taskExecutionState) {
+			TaskExecutor.this.updateTaskExecutionState(jobMasterGateway, taskExecutionState);
+		}
+	}
+
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/a00619a4/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 2360b53..f062b96 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
@@ -21,11 +21,18 @@ package org.apache.flink.runtime.taskexecutor;
 import org.apache.flink.api.common.time.Time;
 import org.apache.flink.runtime.clusterframework.types.AllocationID;
 import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
 import org.apache.flink.runtime.concurrent.Future;
 import org.apache.flink.runtime.resourcemanager.messages.taskexecutor.TMSlotRequestReply;
+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.messages.Acknowledge;
 import org.apache.flink.runtime.rpc.RpcGateway;
 import org.apache.flink.runtime.rpc.RpcTimeout;
+import org.apache.flink.runtime.taskmanager.Task;
 
+import java.util.Collection;
 import java.util.UUID;
 
 /**
@@ -33,12 +40,6 @@ import java.util.UUID;
  */
 public interface TaskExecutorGateway extends RpcGateway {
 
-	// ------------------------------------------------------------------------
-	//  ResourceManager handlers
-	// ------------------------------------------------------------------------
-
-	void notifyOfNewResourceManagerLeader(String address, UUID resourceManagerLeaderId);
-
 	/**
 	 * Requests a slot from the TaskManager
 	 *
@@ -52,4 +53,71 @@ public interface TaskExecutorGateway extends RpcGateway {
 		AllocationID allocationID,
 		UUID resourceManagerLeaderID,
 		@RpcTimeout Time timeout);
+
+	/**
+	 * Submit a {@link Task} to the {@link TaskExecutor}.
+	 *
+	 * @param tdd describing the task to submit
+	 * @param jobManagerID identifying the submitting JobManager
+	 * @param timeout of the submit operation
+	 * @return Future acknowledge of the successful operation
+	 */
+	Future<Acknowledge> submitTask(
+		TaskDeploymentDescriptor tdd,
+		ResourceID jobManagerID,
+		@RpcTimeout Time timeout);
+
+	/**
+	 * Update the task where the given partitions can be found.
+	 *
+	 * @param executionAttemptID identifying the task
+	 * @param partitionInfos telling where the partition can be retrieved from
+	 * @return Future acknowledge if the partitions have been successfully updated
+	 */
+	Future<Acknowledge> updatePartitions(ExecutionAttemptID executionAttemptID, Collection<PartitionInfo> partitionInfos);
+
+	/**
+	 * Fail all intermediate result partitions of the given task.
+	 *
+	 * @param executionAttemptID identifying the task
+	 */
+	void failPartition(ExecutionAttemptID executionAttemptID);
+
+	/**
+	 * Trigger the checkpoint for the given task. The checkpoint is identified by the checkpoint ID
+	 * and the checkpoint timestamp.
+	 *
+	 * @param executionAttemptID identifying the task
+	 * @param checkpointID unique id for the checkpoint
+	 * @param checkpointTimestamp is the timestamp when the checkpoint has been initiated
+	 * @return Future acknowledge if the checkpoint has been successfully triggered
+	 */
+	Future<Acknowledge> triggerCheckpoint(ExecutionAttemptID executionAttemptID, long checkpointID, long checkpointTimestamp);
+
+	/**
+	 * Confirm a checkpoint for the given task. The checkpoint is identified by the checkpoint ID
+	 * and the checkpoint timestamp.
+	 *
+	 * @param executionAttemptID identifying the task
+	 * @param checkpointId unique id for the checkpoint
+	 * @param checkpointTimestamp is the timestamp when the checkpoint has been initiated
+	 * @return Future acknowledge if the checkpoint has been successfully confirmed
+	 */
+	Future<Acknowledge> confirmCheckpoint(ExecutionAttemptID executionAttemptID, long checkpointId, long checkpointTimestamp);
+
+	/**
+	 * Stop the given task.
+	 *
+	 * @param executionAttemptID identifying the task
+	 * @return Future acknowledge if the task is successfully stopped
+	 */
+	Future<Acknowledge> stopTask(ExecutionAttemptID executionAttemptID);
+
+	/**
+	 * Cancel the given task.
+	 *
+	 * @param executionAttemptID identifying the task
+	 * @return Future acknowledge if the task is successfully canceled
+	 */
+	Future<Acknowledge> cancelTask(ExecutionAttemptID executionAttemptID);
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/a00619a4/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerConfiguration.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerConfiguration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerConfiguration.java
index f58af77..bce3dc3 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerConfiguration.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerConfiguration.java
@@ -21,6 +21,7 @@ package org.apache.flink.runtime.taskexecutor;
 import org.apache.flink.api.common.time.Time;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.UnmodifiableConfiguration;
 import org.apache.flink.runtime.akka.AkkaUtils;
 import org.apache.flink.util.Preconditions;
 import org.slf4j.Logger;
@@ -71,7 +72,7 @@ public class TaskManagerConfiguration {
 		this.maxRegistrationPause = Preconditions.checkNotNull(maxRegistrationPause);
 		this.refusedRegistrationPause = Preconditions.checkNotNull(refusedRegistrationPause);
 		this.cleanupInterval = Preconditions.checkNotNull(cleanupInterval);
-		this.configuration = Preconditions.checkNotNull(configuration);
+		this.configuration = new UnmodifiableConfiguration(Preconditions.checkNotNull(configuration));
 	}
 
 	public int getNumberSlots() {

http://git-wip-us.apache.org/repos/asf/flink/blob/a00619a4/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 8ac0ddd..bb66655 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
@@ -95,6 +95,9 @@ public class TaskManagerRunner implements FatalErrorHandler {
 			taskManagerServices.getNetworkEnvironment(),
 			highAvailabilityServices,
 			taskManagerServices.getMetricRegistry(),
+			taskManagerServices.getTaskManagerMetricGroup(),
+			taskManagerServices.getBroadcastVariableManager(),
+			taskManagerServices.getFileCache(),
 			this);
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/a00619a4/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 ff7f7d5..e264a1c 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
@@ -19,7 +19,9 @@
 package org.apache.flink.runtime.taskexecutor;
 
 import org.apache.flink.core.memory.MemoryType;
+import org.apache.flink.runtime.broadcast.BroadcastVariableManager;
 import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.filecache.FileCache;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
 import org.apache.flink.runtime.io.network.ConnectionManager;
@@ -32,9 +34,11 @@ import org.apache.flink.runtime.io.network.netty.NettyConnectionManager;
 import org.apache.flink.runtime.io.network.partition.ResultPartitionManager;
 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.query.KvStateRegistry;
 import org.apache.flink.runtime.query.netty.DisabledKvStateRequestStats;
 import org.apache.flink.runtime.query.netty.KvStateServer;
+import org.apache.flink.runtime.taskexecutor.utils.TaskExecutorMetricsInitializer;
 import org.apache.flink.runtime.taskmanager.NetworkEnvironmentConfiguration;
 import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
 import org.apache.flink.runtime.util.EnvironmentInformation;
@@ -58,19 +62,28 @@ public class TaskManagerServices {
 	private final IOManager ioManager;
 	private final NetworkEnvironment networkEnvironment;
 	private final MetricRegistry metricRegistry;
+	private final TaskManagerMetricGroup taskManagerMetricGroup;
+	private final BroadcastVariableManager broadcastVariableManager;
+	private final FileCache fileCache;
 
 	private TaskManagerServices(
 		TaskManagerLocation taskManagerLocation,
 		MemoryManager memoryManager,
 		IOManager ioManager,
 		NetworkEnvironment networkEnvironment,
-		MetricRegistry metricRegistry) {
+		MetricRegistry metricRegistry,
+		TaskManagerMetricGroup taskManagerMetricGroup,
+		BroadcastVariableManager broadcastVariableManager,
+		FileCache fileCache) {
 
 		this.taskManagerLocation = Preconditions.checkNotNull(taskManagerLocation);
 		this.memoryManager = Preconditions.checkNotNull(memoryManager);
 		this.ioManager = Preconditions.checkNotNull(ioManager);
 		this.networkEnvironment = Preconditions.checkNotNull(networkEnvironment);
 		this.metricRegistry = Preconditions.checkNotNull(metricRegistry);
+		this.taskManagerMetricGroup = Preconditions.checkNotNull(taskManagerMetricGroup);
+		this.broadcastVariableManager = Preconditions.checkNotNull(broadcastVariableManager);
+		this.fileCache = Preconditions.checkNotNull(fileCache);
 	}
 
 	// --------------------------------------------------------------------------------------------
@@ -97,6 +110,18 @@ public class TaskManagerServices {
 		return metricRegistry;
 	}
 
+	public TaskManagerMetricGroup getTaskManagerMetricGroup() {
+		return taskManagerMetricGroup;
+	}
+
+	public BroadcastVariableManager getBroadcastVariableManager() {
+		return broadcastVariableManager;
+	}
+
+	public FileCache getFileCache() {
+		return fileCache;
+	}
+
 	// --------------------------------------------------------------------------------------------
 	//  Static factory methods for task manager services
 	// --------------------------------------------------------------------------------------------
@@ -128,9 +153,29 @@ public class TaskManagerServices {
 		// start the I/O manager, it will create some temp directories.
 		final IOManager ioManager = new IOManagerAsync(taskManagerServicesConfiguration.getTmpDirPaths());
 
-		MetricRegistry metricsRegistry = new MetricRegistry(taskManagerServicesConfiguration.getMetricRegistryConfiguration());
+		final MetricRegistry metricRegistry = new MetricRegistry(taskManagerServicesConfiguration.getMetricRegistryConfiguration());
+
+		final TaskManagerMetricGroup taskManagerMetricGroup = new TaskManagerMetricGroup(
+			metricRegistry,
+			taskManagerLocation.getHostname(),
+			taskManagerLocation.getResourceID().toString());
+
+		// Initialize the TM metrics
+		TaskExecutorMetricsInitializer.instantiateStatusMetrics(taskManagerMetricGroup, network);
+
+		final BroadcastVariableManager broadcastVariableManager = new BroadcastVariableManager();
+
+		final FileCache fileCache = new FileCache(taskManagerServicesConfiguration.getTmpDirPaths());
 
-		return new TaskManagerServices(taskManagerLocation, memoryManager, ioManager, network, metricsRegistry);
+		return new TaskManagerServices(
+			taskManagerLocation,
+			memoryManager,
+			ioManager,
+			network,
+			metricRegistry,
+			taskManagerMetricGroup,
+			broadcastVariableManager,
+			fileCache);
 	}
 
 	/**

http://git-wip-us.apache.org/repos/asf/flink/blob/a00619a4/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskSlot.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskSlot.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskSlot.java
new file mode 100644
index 0000000..4fc1d66
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskSlot.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.taskexecutor;
+
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.taskmanager.Task;
+import org.apache.flink.util.Preconditions;
+
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * Container for multiple {@link Task} belonging to the same slot.
+ */
+public class TaskSlot {
+	private final AllocationID allocationID;
+	private final ResourceID resourceID;
+	private final Map<ExecutionAttemptID, Task> tasks;
+
+	public TaskSlot(AllocationID allocationID, ResourceID resourceID) {
+		this.allocationID = Preconditions.checkNotNull(allocationID);
+		this.resourceID = Preconditions.checkNotNull(resourceID);
+		tasks = new HashMap<>(4);
+	}
+
+	public AllocationID getAllocationID() {
+		return allocationID;
+	}
+
+	public ResourceID getResourceID() {
+		return resourceID;
+	}
+
+	public boolean add(Task task) {
+		// sanity check
+		Preconditions.checkArgument(allocationID.equals(task.getAllocationID()));
+
+		Task oldTask = tasks.put(task.getExecutionId(), task);
+
+		if (oldTask != null) {
+			tasks.put(task.getExecutionId(), oldTask);
+			return false;
+		} else {
+			return true;
+		}
+	}
+
+	public Task remove(Task task) {
+		return tasks.remove(task.getExecutionId());
+	}
+
+	public void clear() {
+		tasks.clear();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/a00619a4/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskSlotMapping.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskSlotMapping.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskSlotMapping.java
new file mode 100644
index 0000000..e67fd52
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskSlotMapping.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.taskexecutor;
+
+import org.apache.flink.runtime.taskmanager.Task;
+import org.apache.flink.util.Preconditions;
+
+/**
+ * Mapping between a {@link Task} and its {@link TaskSlot}.
+ */
+public class TaskSlotMapping {
+
+	private final Task task;
+	private final TaskSlot taskSlot;
+
+	public TaskSlotMapping(Task task, TaskSlot taskSlot) {
+		this.task = Preconditions.checkNotNull(task);
+		this.taskSlot = Preconditions.checkNotNull(taskSlot);
+	}
+
+	public Task getTask() {
+		return task;
+	}
+
+	public TaskSlot getTaskSlot() {
+		return taskSlot;
+	}
+}


[27/50] [abbrv] flink git commit: [FLINK-4408] [JobManager] Introduce JobMasterRunner and implement job submission & setting up the ExecutionGraph

Posted by se...@apache.org.
[FLINK-4408] [JobManager] Introduce JobMasterRunner and implement job submission & setting up the ExecutionGraph

This closes #2480


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

Branch: refs/heads/flip-6
Commit: 360eaf8a764ef981739f56598bbfd742754456f2
Parents: 0e495b7
Author: Kurt Young <yk...@gmail.com>
Authored: Thu Sep 8 12:00:13 2016 +0800
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Oct 14 15:14:41 2016 +0200

----------------------------------------------------------------------
 .../flink/api/common/JobExecutionResult.java    |   2 +-
 .../flink/api/common/JobSubmissionResult.java   |   2 +-
 .../HighAvailabilityServices.java               |  12 +
 .../runtime/highavailability/NonHaServices.java |  16 +-
 .../runtime/jobmanager/OnCompletionActions.java |  31 ++
 .../runtime/jobmanager/scheduler/Scheduler.java |   9 +
 .../runtime/jobmaster/JobManagerRunner.java     | 288 +++++++++++
 .../runtime/jobmaster/JobManagerServices.java   |  73 +++
 .../flink/runtime/jobmaster/JobMaster.java      | 485 ++++++++++++++-----
 .../runtime/jobmaster/JobMasterGateway.java     |  13 +
 .../jobmaster/MiniClusterJobDispatcher.java     | 385 +++++++++++++++
 .../flink/runtime/rpc/FatalErrorHandler.java    |  24 +
 .../runtime/taskexecutor/TaskExecutor.java      |  12 +
 .../TestingHighAvailabilityServices.java        |  39 +-
 .../jobmaster/JobManagerRunnerMockTest.java     | 254 ++++++++++
 .../flink/runtime/rpc/RpcConnectionTest.java    |  17 +-
 16 files changed, 1533 insertions(+), 129 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/360eaf8a/flink-core/src/main/java/org/apache/flink/api/common/JobExecutionResult.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/JobExecutionResult.java b/flink-core/src/main/java/org/apache/flink/api/common/JobExecutionResult.java
index cb4ecc5..7286cc5 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/JobExecutionResult.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/JobExecutionResult.java
@@ -32,7 +32,7 @@ import java.util.concurrent.TimeUnit;
 @Public
 public class JobExecutionResult extends JobSubmissionResult {
 
-	private long netRuntime;
+	private final long netRuntime;
 
 	private final Map<String, Object> accumulatorResults;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/360eaf8a/flink-core/src/main/java/org/apache/flink/api/common/JobSubmissionResult.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/JobSubmissionResult.java b/flink-core/src/main/java/org/apache/flink/api/common/JobSubmissionResult.java
index c5dc869..b0e7e24 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/JobSubmissionResult.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/JobSubmissionResult.java
@@ -26,7 +26,7 @@ import org.apache.flink.annotation.Public;
 @Public
 public class JobSubmissionResult {
 
-	private JobID jobID;
+	private final JobID jobID;
 
 	public JobSubmissionResult(JobID jobID) {
 		this.jobID = jobID;

http://git-wip-us.apache.org/repos/asf/flink/blob/360eaf8a/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 7634176..d67e927 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,6 +19,8 @@
 package org.apache.flink.runtime.highavailability;
 
 import org.apache.flink.api.common.JobID;
+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;
 
@@ -61,4 +63,14 @@ public interface HighAvailabilityServices {
 	 * @param jobID The identifier of the job running the election.
 	 */
 	LeaderElectionService getJobMasterLeaderElectionService(JobID jobID) throws Exception;
+
+	/**
+	 * Gets the checkpoint recovery factory for the job manager
+	 */
+	CheckpointRecoveryFactory getCheckpointRecoveryFactory() throws Exception;
+
+	/**
+	 * Gets the submitted job graph store for the job manager
+	 */
+	SubmittedJobGraphStore getSubmittedJobGraphStore() throws Exception;
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/360eaf8a/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 33dc2d7..a2c9cc4 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,13 +19,17 @@
 package org.apache.flink.runtime.highavailability;
 
 import org.apache.flink.api.common.JobID;
-import org.apache.flink.hadoop.shaded.org.jboss.netty.util.internal.ConcurrentHashMap;
+import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory;
+import org.apache.flink.runtime.checkpoint.StandaloneCheckpointRecoveryFactory;
+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.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;
 
@@ -88,4 +92,14 @@ public class NonHaServices implements HighAvailabilityServices {
 	public LeaderElectionService getJobMasterLeaderElectionService(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();
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/360eaf8a/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
new file mode 100644
index 0000000..6de4253
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/OnCompletionActions.java
@@ -0,0 +1,31 @@
+/*
+ * 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;
+
+import org.apache.flink.api.common.JobExecutionResult;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+
+public interface OnCompletionActions extends FatalErrorHandler {
+
+	void jobFinished(JobExecutionResult result);
+
+	void jobFailed(Throwable cause);
+
+	void jobFinishedByOther();
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/360eaf8a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/Scheduler.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/Scheduler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/Scheduler.java
index b839e0e..aa09314 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/Scheduler.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/Scheduler.java
@@ -31,6 +31,7 @@ import java.util.Queue;
 import java.util.Set;
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorService;
 import java.util.concurrent.LinkedBlockingQueue;
 
 import akka.dispatch.Futures;
@@ -57,6 +58,7 @@ import org.apache.flink.util.ExceptionUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import scala.concurrent.ExecutionContext;
+import scala.concurrent.ExecutionContext$;
 
 /**
  * The scheduler is responsible for distributing the ready-to-run tasks among instances and slots.
@@ -110,6 +112,13 @@ public class Scheduler implements InstanceListener, SlotAvailabilityListener, Sl
 	/**
 	 * Creates a new scheduler.
 	 */
+	public Scheduler(ExecutorService executor) {
+		this(ExecutionContext$.MODULE$.fromExecutor(executor));
+	}
+	
+	/**
+	 * Creates a new scheduler.
+	 */
 	public Scheduler(ExecutionContext executionContext) {
 		this.executionContext = executionContext;
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/360eaf8a/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
new file mode 100644
index 0000000..bc2bf9a
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunner.java
@@ -0,0 +1,288 @@
+/*
+ * 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.annotation.VisibleForTesting;
+import org.apache.flink.api.common.JobExecutionResult;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory;
+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.jobmanager.scheduler.Scheduler;
+import org.apache.flink.runtime.leaderelection.LeaderContender;
+import org.apache.flink.runtime.leaderelection.LeaderElectionService;
+import org.apache.flink.runtime.rpc.RpcService;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.UUID;
+import java.util.concurrent.Executor;
+
+/**
+ * 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 {
+
+	private 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();
+
+	/** The job graph needs to run */
+	private final JobGraph jobGraph;
+
+	private final OnCompletionActions toNotify;
+
+	/** The execution context which is used to execute futures */
+	private final Executor executionContext;
+
+	// TODO: use this to decide whether the job is finished by other
+	private final CheckpointRecoveryFactory checkpointRecoveryFactory;
+
+	/** Leader election for this job */
+	private final LeaderElectionService leaderElectionService;
+
+	private final JobMaster jobManager;
+
+	/** Leader session id when granted leadership */
+	private UUID leaderSessionID;
+
+	/** 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
+	{
+		this(jobGraph, configuration, rpcService, haServices,
+			JobManagerServices.fromConfiguration(configuration), toNotify);
+	}
+
+	public JobManagerRunner(
+		final JobGraph jobGraph,
+		final Configuration configuration,
+		final RpcService rpcService,
+		final HighAvailabilityServices haServices,
+		final JobManagerServices jobManagerServices,
+		final OnCompletionActions toNotify) throws Exception
+	{
+		this.jobGraph = jobGraph;
+		this.toNotify = toNotify;
+		this.executionContext = rpcService.getExecutor();
+		this.checkpointRecoveryFactory = haServices.getCheckpointRecoveryFactory();
+		this.leaderElectionService = haServices.getJobMasterLeaderElectionService(jobGraph.getJobID());
+		this.leaderSessionID = null;
+
+		this.jobManager = new JobMaster(
+			jobGraph, configuration, rpcService, haServices,
+			jobManagerServices.libraryCacheManager,
+			jobManagerServices.restartStrategyFactory,
+			jobManagerServices.savepointStore,
+			jobManagerServices.timeout,
+			new Scheduler(jobManagerServices.executorService),
+			jobManagerServices.jobManagerMetricGroup,
+			this);
+	}
+
+	//----------------------------------------------------------------------------------------------
+	// Lifecycle management
+	//----------------------------------------------------------------------------------------------
+
+	public void start() throws Exception {
+		jobManager.init();
+		jobManager.start();
+
+		try {
+			leaderElectionService.start(this);
+		}
+		catch (Exception e) {
+			log.error("Could not start the JobManager because the leader election service did not start.", e);
+			throw new Exception("Could not start the leader election service.", e);
+		}
+	}
+
+	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();
+	}
+
+	private void shutdownInternally() {
+		synchronized (lock) {
+			shutdown = true;
+
+			if (leaderElectionService != null) {
+				try {
+					leaderElectionService.stop();
+				} catch (Exception e) {
+					log.error("Could not properly shutdown the leader election service.");
+				}
+			}
+
+			jobManager.shutDown();
+		}
+	}
+
+	//----------------------------------------------------------------------------------------------
+	// Result and error handling methods
+	//----------------------------------------------------------------------------------------------
+
+	/**
+	 * Job completion notification triggered by JobManager
+	 */
+	@Override
+	public void jobFinished(JobExecutionResult result) {
+		try {
+			shutdownInternally();
+		}
+		finally {
+			if (toNotify != null) {
+				toNotify.jobFinished(result);
+			}
+		}
+	}
+
+	/**
+	 * Job completion notification triggered by JobManager
+	 */
+	@Override
+	public void jobFailed(Throwable cause) {
+		try {
+			shutdownInternally();
+		}
+		finally {
+			if (toNotify != null) {
+				toNotify.jobFailed(cause);
+			}
+		}
+	}
+
+	/**
+	 * Job completion notification triggered by self
+	 */
+	@Override
+	public void jobFinishedByOther() {
+		try {
+			shutdownInternally();
+		}
+		finally {
+			if (toNotify != null) {
+				toNotify.jobFinishedByOther();
+			}
+		}
+	}
+
+	/**
+	 * Job completion notification triggered by JobManager or self
+	 */
+	@Override
+	public void onFatalError(Throwable exception) {
+		// first and in any case, notify our handler, so it can react fast
+		try {
+			if (toNotify != null) {
+				toNotify.onFatalError(exception);
+			}
+		}
+		finally {
+			log.error("JobManager runner encountered a fatal error.", exception);
+			shutdownInternally();
+		}
+	}
+
+	//----------------------------------------------------------------------------------------------
+	// Leadership methods
+	//----------------------------------------------------------------------------------------------
+
+	@Override
+	public void grantLeadership(final UUID leaderSessionID) {
+		synchronized (lock) {
+			if (shutdown) {
+				log.info("JobManagerRunner already shutdown.");
+				return;
+			}
+
+			log.info("JobManager runner for job {} ({}) was granted leadership with session id {} at {}.",
+				jobGraph.getName(), jobGraph.getJobID(), leaderSessionID, getAddress());
+
+			// The operation may be blocking, but since this runner is idle before it been granted the leadership,
+			// it's okay that job manager wait for the operation complete
+			leaderElectionService.confirmLeaderSessionID(leaderSessionID);
+			this.leaderSessionID = leaderSessionID;
+
+			// 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()) {
+					log.info("Job {} ({}) already finished by others.", jobGraph.getName(), jobGraph.getJobID());
+					jobFinishedByOther();
+				} else {
+					jobManager.getSelf().startJob();
+				}
+			}
+		}
+	}
+
+	@Override
+	public void revokeLeadership() {
+		synchronized (lock) {
+			if (shutdown) {
+				log.info("JobManagerRunner already shutdown.");
+				return;
+			}
+
+			log.info("JobManager for job {} ({}) was revoked leadership at {}.",
+				jobGraph.getName(), jobGraph.getJobID(), getAddress());
+
+			leaderSessionID = null;
+			jobManager.getSelf().suspendJob(new Exception("JobManager is no longer the leader."));
+		}
+	}
+
+	@Override
+	public String getAddress() {
+		return jobManager.getAddress();
+	}
+
+	@Override
+	public void handleError(Exception exception) {
+		log.error("Leader Election Service encountered a fatal error.", exception);
+		onFatalError(exception);
+	}
+
+	@VisibleForTesting
+	boolean isJobFinishedByOthers() {
+		// TODO
+		return false;
+	}
+
+	@VisibleForTesting
+	boolean isShutdown() {
+		return shutdown;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/360eaf8a/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
new file mode 100644
index 0000000..e6beba6
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerServices.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.jobmaster;
+
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.checkpoint.savepoint.SavepointStore;
+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 java.util.concurrent.ExecutorService;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Utility class to hold all auxiliary services used by the {@link JobMaster}.
+ */
+public class JobManagerServices {
+
+	public final ExecutorService executorService;
+
+	public final BlobLibraryCacheManager libraryCacheManager;
+
+	public final RestartStrategyFactory restartStrategyFactory;
+
+	public final SavepointStore savepointStore;
+
+	public final Time timeout;
+
+	public final JobManagerMetricGroup jobManagerMetricGroup;
+
+	public JobManagerServices(
+			ExecutorService executorService,
+			BlobLibraryCacheManager libraryCacheManager,
+			RestartStrategyFactory restartStrategyFactory,
+			SavepointStore savepointStore,
+			Time timeout,
+			JobManagerMetricGroup jobManagerMetricGroup) {
+
+		this.executorService = checkNotNull(executorService);
+		this.libraryCacheManager = checkNotNull(libraryCacheManager);
+		this.restartStrategyFactory = checkNotNull(restartStrategyFactory);
+		this.savepointStore = checkNotNull(savepointStore);
+		this.timeout = checkNotNull(timeout);
+		this.jobManagerMetricGroup = checkNotNull(jobManagerMetricGroup);
+	}
+
+	// ------------------------------------------------------------------------
+	//  Creating the components from a configuration 
+	// ------------------------------------------------------------------------
+	
+	public static JobManagerServices fromConfiguration(Configuration config) throws Exception {
+		// TODO not yet implemented
+		return null;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/360eaf8a/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 1537396..b52a23c 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,21 +18,50 @@
 
 package org.apache.flink.runtime.jobmaster;
 
-import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.ExecutionConfig;
+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.metrics.MetricGroup;
+import org.apache.flink.metrics.groups.UnregisteredMetricsGroup;
+import org.apache.flink.runtime.checkpoint.CheckpointIDCounter;
+import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory;
+import org.apache.flink.runtime.checkpoint.CompletedCheckpointStore;
+import org.apache.flink.runtime.checkpoint.savepoint.SavepointStore;
+import org.apache.flink.runtime.checkpoint.stats.CheckpointStatsTracker;
+import org.apache.flink.runtime.checkpoint.stats.DisabledCheckpointStatsTracker;
+import org.apache.flink.runtime.checkpoint.stats.SimpleCheckpointStatsTracker;
+import org.apache.flink.runtime.client.JobExecutionException;
+import org.apache.flink.runtime.client.JobSubmissionException;
+import org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager;
+import org.apache.flink.runtime.executiongraph.ExecutionGraph;
+import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
+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.jobgraph.JobGraph;
-import org.apache.flink.runtime.leaderelection.LeaderContender;
-import org.apache.flink.runtime.leaderelection.LeaderElectionService;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.jobgraph.jsonplan.JsonPlanGenerator;
+import org.apache.flink.runtime.jobgraph.tasks.JobSnapshottingSettings;
+import org.apache.flink.runtime.jobmanager.OnCompletionActions;
+import org.apache.flink.runtime.jobmanager.scheduler.Scheduler;
 import org.apache.flink.runtime.messages.Acknowledge;
-import org.apache.flink.runtime.rpc.RpcMethod;
+import org.apache.flink.runtime.metrics.groups.JobManagerMetricGroup;
 import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway;
 import org.apache.flink.runtime.rpc.RpcEndpoint;
+import org.apache.flink.runtime.rpc.RpcMethod;
 import org.apache.flink.runtime.rpc.RpcService;
 import org.apache.flink.runtime.taskmanager.TaskExecutionState;
-import org.apache.flink.util.Preconditions;
+import scala.concurrent.ExecutionContext$;
+import scala.concurrent.duration.FiniteDuration;
 
-import java.util.UUID;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.Executor;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
 
 /**
  * JobMaster implementation. The job master is responsible for the execution of a single
@@ -41,7 +70,7 @@ import java.util.UUID;
  * 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(TaskExecutionState)} updates the task execution state for
  * given task</li>
  * </ul>
  */
@@ -52,7 +81,6 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 
 	/** Logical representation of the job */
 	private final JobGraph jobGraph;
-	private final JobID jobID;
 
 	/** Configuration of the job */
 	private final Configuration configuration;
@@ -60,32 +88,67 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 	/** Service to contend for and retrieve the leadership of JM and RM */
 	private final HighAvailabilityServices highAvailabilityServices;
 
-	/** Leader Management */
-	private LeaderElectionService leaderElectionService = null;
-	private UUID leaderSessionID;
+	/** 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 group used across jobs */
+	private final JobManagerMetricGroup jobManagerMetricGroup;
+
+	/** The execution context which is used to execute futures */
+	private final Executor 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 ClassLoader userCodeLoader;
+
+	private RestartStrategy restartStrategy;
+
+	private MetricGroup jobMetrics;
 
-	/**
-	 * The JM's Constructor
-	 *
-	 * @param jobGraph The representation of the job's execution plan
-	 * @param configuration The job's configuration
-	 * @param rpcService The RPC service at which the JM serves
-	 * @param highAvailabilityService The cluster's HA service from the JM can elect and retrieve leaders.
-	 */
 	public JobMaster(
 		JobGraph jobGraph,
 		Configuration configuration,
 		RpcService rpcService,
-		HighAvailabilityServices highAvailabilityService) {
-
+		HighAvailabilityServices highAvailabilityService,
+		BlobLibraryCacheManager libraryCacheManager,
+		RestartStrategyFactory restartStrategyFactory,
+		SavepointStore savepointStore,
+		Time timeout,
+		Scheduler scheduler,
+		JobManagerMetricGroup jobManagerMetricGroup,
+		OnCompletionActions jobCompletionActions)
+	{
 		super(rpcService);
 
-		this.jobGraph = Preconditions.checkNotNull(jobGraph);
-		this.jobID = Preconditions.checkNotNull(jobGraph.getJobID());
-
-		this.configuration = Preconditions.checkNotNull(configuration);
-
-		this.highAvailabilityServices = Preconditions.checkNotNull(highAvailabilityService);
+		this.jobGraph = checkNotNull(jobGraph);
+		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.jobCompletionActions = checkNotNull(jobCompletionActions);
 	}
 
 	public ResourceManagerGateway getResourceManager() {
@@ -93,93 +156,294 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 	}
 
 	//----------------------------------------------------------------------------------------------
-	// Initialization methods
+	// Lifecycle management
 	//----------------------------------------------------------------------------------------------
+
+	/**
+	 * Initializing the job execution environment, should be called before start. Any error occurred during
+	 * initialization will be treated as job submission failure.
+	 *
+	 * @throws JobSubmissionException
+	 */
+	public void init() throws JobSubmissionException {
+		log.info("Initializing job {} ({}).", jobGraph.getName(), jobGraph.getJobID());
+
+		try {
+			// IMPORTANT: We need to make sure that the library registration is the first action,
+			// because this makes sure that the uploaded jar files are removed in case of
+			// unsuccessful
+			try {
+				libraryCacheManager.registerJob(jobGraph.getJobID(), jobGraph.getUserJarBlobKeys(),
+					jobGraph.getClasspaths());
+			} catch (Throwable t) {
+				throw new JobSubmissionException(jobGraph.getJobID(),
+					"Cannot set up the user code libraries: " + t.getMessage(), t);
+			}
+
+			userCodeLoader = libraryCacheManager.getClassLoader(jobGraph.getJobID());
+			if (userCodeLoader == null) {
+				throw new JobSubmissionException(jobGraph.getJobID(),
+					"The user code class loader could not be initialized.");
+			}
+
+			if (jobGraph.getNumberOfVertices() == 0) {
+				throw new JobSubmissionException(jobGraph.getJobID(), "The given job is empty");
+			}
+
+			final RestartStrategies.RestartStrategyConfiguration restartStrategyConfiguration =
+				jobGraph.getSerializedExecutionConfig()
+					.deserializeValue(userCodeLoader)
+					.getRestartStrategy();
+			if (restartStrategyConfiguration != null) {
+				restartStrategy = RestartStrategyFactory.createRestartStrategy(restartStrategyConfiguration);
+			} else {
+				restartStrategy = restartStrategyFactory.createRestartStrategy();
+			}
+
+			log.info("Using restart strategy {} for {} ({}).", restartStrategy, jobGraph.getName(), jobGraph.getJobID());
+
+			if (jobManagerMetricGroup != null) {
+				jobMetrics = jobManagerMetricGroup.addJob(jobGraph);
+			}
+			if (jobMetrics == null) {
+				jobMetrics = new UnregisteredMetricsGroup();
+			}
+
+			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);
+			}
+
+		} 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);
+			}
+		}
+	}
+
+	@Override
 	public void start() {
 		super.start();
-
-		// register at the election once the JM starts
-		registerAtElectionService();
 	}
 
+	@Override
+	public void shutDown() {
+		super.shutDown();
+
+		suspendJob(new Exception("JobManager is shutting down."));
+	}
 
 	//----------------------------------------------------------------------------------------------
-	// JobMaster Leadership methods
+	// RPC methods
 	//----------------------------------------------------------------------------------------------
 
 	/**
-	 * Retrieves the election service and contend for the leadership.
+	 * 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.
 	 */
-	private void registerAtElectionService() {
-		try {
-			leaderElectionService = highAvailabilityServices.getJobMasterLeaderElectionService(jobID);
-			leaderElectionService.start(new JobMasterLeaderContender());
-		} catch (Exception e) {
-			throw new RuntimeException("Fail to register at the election of JobMaster", e);
+	@RpcMethod
+	public void startJob() {
+		log.info("Starting job {} ({}).", jobGraph.getName(), jobGraph.getJobID());
+
+		if (executionGraph != null) {
+			executionGraph = new ExecutionGraph(
+				ExecutionContext$.MODULE$.fromExecutor(executionContext),
+				jobGraph.getJobID(),
+				jobGraph.getName(),
+				jobGraph.getJobConfiguration(),
+				jobGraph.getSerializedExecutionConfig(),
+				new FiniteDuration(timeout.getSize(), timeout.getUnit()),
+				restartStrategy,
+				jobGraph.getUserJarBlobKeys(),
+				jobGraph.getClasspaths(),
+				userCodeLoader,
+				jobMetrics);
+		} else {
+			// TODO: update last active time in JobInfo
 		}
-	}
 
-	/**
-	 * Start the execution when the leadership is granted.
-	 *
-	 * @param newLeaderSessionID The identifier of the new leadership session
-	 */
-	public void grantJobMasterLeadership(final UUID newLeaderSessionID) {
-		runAsync(new Runnable() {
-			@Override
-			public void run() {
-				log.info("JobManager {} grants leadership with session id {}.", getAddress(), newLeaderSessionID);
+		try {
+			executionGraph.setScheduleMode(jobGraph.getScheduleMode());
+			executionGraph.setQueuedSchedulingAllowed(jobGraph.getAllowQueuedScheduling());
+
+			try {
+				executionGraph.setJsonPlan(JsonPlanGenerator.generatePlan(jobGraph));
+			} catch (Exception e) {
+				log.warn("Cannot create JSON plan for job {} ({})", jobGraph.getJobID(), jobGraph.getName(), e);
+				executionGraph.setJsonPlan("{}");
+			}
 
-				// The operation may be blocking, but since JM is idle before it grants the leadership, it's okay that
-				// JM waits here for the operation's completeness.
-				leaderSessionID = newLeaderSessionID;
-				leaderElectionService.confirmLeaderSessionID(newLeaderSessionID);
+			// 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);
+				}
+			}
 
-				// TODO:: execute the job when the leadership is granted.
+			// 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);
 
-	/**
-	 * Stop the execution when the leadership is revoked.
-	 */
-	public void revokeJobMasterLeadership() {
-		runAsync(new Runnable() {
-			@Override
-			public void run() {
-				log.info("JobManager {} was revoked leadership.", getAddress());
+			if (log.isDebugEnabled()) {
+				log.debug("Successfully created execution graph from job graph {} ({}).",
+					jobGraph.getJobID(), jobGraph.getName());
+			}
 
-				// TODO:: cancel the job's execution and notify all listeners
-				cancelAndClearEverything(new Exception("JobManager is no longer the leader."));
+			final JobSnapshottingSettings snapshotSettings = jobGraph.getSnapshotSettings();
+			if (snapshotSettings != null) {
+				List<ExecutionJobVertex> triggerVertices = getExecutionJobVertexWithId(
+					executionGraph, snapshotSettings.getVerticesToTrigger());
+
+				List<ExecutionJobVertex> ackVertices = getExecutionJobVertexWithId(
+					executionGraph, snapshotSettings.getVerticesToAcknowledge());
+
+				List<ExecutionJobVertex> confirmVertices = getExecutionJobVertexWithId(
+					executionGraph, snapshotSettings.getVerticesToConfirm());
+
+				CompletedCheckpointStore completedCheckpoints = checkpointRecoveryFactory.createCheckpointStore(
+					jobGraph.getJobID(), userCodeLoader);
+
+				CheckpointIDCounter checkpointIdCounter = checkpointRecoveryFactory.createCheckpointIDCounter(
+					jobGraph.getJobID());
+
+				// Checkpoint stats tracker
+				boolean isStatsDisabled = configuration.getBoolean(
+					ConfigConstants.JOB_MANAGER_WEB_CHECKPOINTS_DISABLE,
+					ConfigConstants.DEFAULT_JOB_MANAGER_WEB_CHECKPOINTS_DISABLE);
+
+				final CheckpointStatsTracker checkpointStatsTracker;
+				if (isStatsDisabled) {
+					checkpointStatsTracker = new DisabledCheckpointStatsTracker();
+				} else {
+					int historySize = configuration.getInteger(
+						ConfigConstants.JOB_MANAGER_WEB_CHECKPOINTS_HISTORY_SIZE,
+						ConfigConstants.DEFAULT_JOB_MANAGER_WEB_CHECKPOINTS_HISTORY_SIZE);
+					checkpointStatsTracker = new SimpleCheckpointStatsTracker(historySize, ackVertices, jobMetrics);
+				}
+
+				executionGraph.enableSnapshotCheckpointing(
+					snapshotSettings.getCheckpointInterval(),
+					snapshotSettings.getCheckpointTimeout(),
+					snapshotSettings.getMinPauseBetweenCheckpoints(),
+					snapshotSettings.getMaxConcurrentCheckpoints(),
+					triggerVertices,
+					ackVertices,
+					confirmVertices,
+					checkpointIdCounter,
+					completedCheckpoints,
+					savepointStore,
+					checkpointStatsTracker);
+			}
+
+			// TODO: register this class to execution graph as job status change listeners
+
+			// TODO: register client as job / execution status change listeners if they are interested
+
+			/*
+			TODO: decide whether we should take the savepoint before recovery
+
+			if (isRecovery) {
+				// this is a recovery of a master failure (this master takes over)
+				executionGraph.restoreLatestCheckpointedState();
+			} else {
+				if (snapshotSettings != null) {
+					String savepointPath = snapshotSettings.getSavepointPath();
+					if (savepointPath != null) {
+						// got a savepoint
+						log.info("Starting job from savepoint {}.", savepointPath);
+
+						// load the savepoint as a checkpoint into the system
+						final CompletedCheckpoint savepoint = SavepointLoader.loadAndValidateSavepoint(
+							jobGraph.getJobID(), executionGraph.getAllVertices(), savepointStore, savepointPath);
+						executionGraph.getCheckpointCoordinator().getCheckpointStore().addCheckpoint(savepoint);
+
+						// Reset the checkpoint ID counter
+						long nextCheckpointId = savepoint.getCheckpointID() + 1;
+						log.info("Reset the checkpoint ID to " + nextCheckpointId);
+						executionGraph.getCheckpointCoordinator().getCheckpointIdCounter().setCount(nextCheckpointId);
 
-				leaderSessionID = null;
+						executionGraph.restoreLatestCheckpointedState();
+					}
+				}
 			}
-		});
-	}
+			*/
 
-	/**
-	 * Handles error occurring in the leader election service
-	 *
-	 * @param exception Exception thrown in the leader election service
-	 */
-	public void onJobMasterElectionError(final Exception exception) {
-		runAsync(new Runnable() {
-			@Override
-			public void run() {
-				log.error("Received an error from the LeaderElectionService.", exception);
+		} catch (Throwable t) {
+			log.error("Failed to start job {} ({})", jobGraph.getName(), jobGraph.getJobID(), t);
 
-				// TODO:: cancel the job's execution and shutdown the JM
-				cancelAndClearEverything(exception);
+			executionGraph.fail(t);
+			executionGraph = null;
 
-				leaderSessionID = null;
+			final Throwable 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
+
+			jobCompletionActions.jobFailed(rt);
+			return;
+		}
+
+		// start scheduling job in another thread
+		executionContext.execute(new Runnable() {
+			@Override
+			public void run() {
+				if (executionGraph != null) {
+					try {
+						executionGraph.scheduleForExecution(scheduler);
+					} catch (Throwable t) {
+						executionGraph.fail(t);
+					}
+				}
+			}
+		});
 	}
 
-	//----------------------------------------------------------------------------------------------
-	// RPC methods
-	//----------------------------------------------------------------------------------------------
+	/**
+	 * Suspending job, all the running tasks will be cancelled, and runtime data will be cleared.
+	 *
+	 * @param cause The reason of why this job been suspended.
+	 */
+	@RpcMethod
+	public void suspendJob(final Throwable cause) {
+		if (executionGraph != null) {
+			executionGraph.suspend(cause);
+			executionGraph = null;
+		}
+	}
 
 	/**
 	 * Updates the task execution state for a given task.
@@ -208,37 +472,26 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 	//----------------------------------------------------------------------------------------------
 
 	/**
-	 * Cancel the current job and notify all listeners the job's cancellation.
+	 * Converts JobVertexIDs to corresponding ExecutionJobVertexes
 	 *
-	 * @param cause Cause for the cancelling.
+	 * @param executionGraph The execution graph that holds the relationship
+	 * @param vertexIDs      The vertexIDs need to be converted
+	 * @return The corresponding ExecutionJobVertexes
+	 * @throws JobExecutionException
 	 */
-	private void cancelAndClearEverything(Throwable cause) {
-		// currently, nothing to do here
-	}
-
-	// ------------------------------------------------------------------------
-	//  Utility classes
-	// ------------------------------------------------------------------------
-	private class JobMasterLeaderContender implements LeaderContender {
-
-		@Override
-		public void grantLeadership(UUID leaderSessionID) {
-			JobMaster.this.grantJobMasterLeadership(leaderSessionID);
-		}
-
-		@Override
-		public void revokeLeadership() {
-			JobMaster.this.revokeJobMasterLeadership();
-		}
-
-		@Override
-		public String getAddress() {
-			return JobMaster.this.getAddress();
-		}
-
-		@Override
-		public void handleError(Exception exception) {
-			onJobMasterElectionError(exception);
+	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;
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/360eaf8a/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 86bf17c..b281ea8 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
@@ -29,6 +29,19 @@ import org.apache.flink.runtime.taskmanager.TaskExecutionState;
 public interface JobMasterGateway extends RpcGateway {
 
 	/**
+	 * Making this job begins to run.
+	 */
+	void startJob();
+
+	/**
+	 * 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);
+
+	/**
 	 * Updates the task execution state for a given task.
 	 *
 	 * @param taskExecutionState New task execution state for a given task

http://git-wip-us.apache.org/repos/asf/flink/blob/360eaf8a/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
new file mode 100644
index 0000000..792bfd5
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/MiniClusterJobDispatcher.java
@@ -0,0 +1,385 @@
+/*
+ * 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.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;
+
+	/** 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) throws Exception {
+		this(config, rpcService, haServices, 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,
+			int numJobManagers) throws Exception {
+
+		checkArgument(numJobManagers >= 1);
+		this.configuration = checkNotNull(config);
+		this.rpcService = checkNotNull(rpcService);
+		this.haServices = checkNotNull(haServices);
+		this.numJobManagers = numJobManagers;
+
+		LOG.info("Creating JobMaster services");
+		this.jobManagerServices = JobManagerServices.fromConfiguration(config);
+	}
+
+	// ------------------------------------------------------------------------
+	//  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;
+
+					Exception shutdownException = new Exception("The MiniCluster is shutting down");
+					for (JobManagerRunner runner : runners) {
+						runner.shutdown(shutdownException);
+					}
+				}
+			}
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  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");
+
+			OnCompletionActions onJobCompletion = new DetachedFinalizer(numJobManagers);
+
+			this.runners = startJobRunners(job, onJobCompletion);
+		}
+	}
+
+	/**
+	 * 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 onJobCompletion = 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);
+		}
+
+		try {
+			return onJobCompletion.getResult();
+		}
+		finally {
+			// always clear the status for the next job
+			runners = null;
+		}
+	}
+
+	private JobManagerRunner[] startJobRunners(JobGraph job, OnCompletionActions onCompletion) 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);
+				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);
+						}
+					} 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 {
+
+		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 {
+
+		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 teh 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/360eaf8a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/FatalErrorHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/FatalErrorHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/FatalErrorHandler.java
new file mode 100644
index 0000000..7721117
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/FatalErrorHandler.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.rpc;
+
+public interface FatalErrorHandler {
+
+	void onFatalError(Throwable exception);
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/360eaf8a/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 cf709c8..9e3c3b9 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
@@ -21,6 +21,7 @@ package org.apache.flink.runtime.taskexecutor;
 import akka.actor.ActorSystem;
 import com.typesafe.config.Config;
 import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory;
 import org.apache.flink.runtime.clusterframework.types.AllocationID;
 import org.apache.flink.runtime.io.network.ConnectionManager;
 import org.apache.flink.runtime.io.network.LocalConnectionManager;
@@ -28,6 +29,7 @@ import org.apache.flink.runtime.io.network.TaskEventDispatcher;
 import org.apache.flink.runtime.io.network.buffer.NetworkBufferPool;
 import org.apache.flink.runtime.io.network.netty.NettyConnectionManager;
 import org.apache.flink.runtime.io.network.partition.ResultPartitionManager;
+import org.apache.flink.runtime.jobmanager.SubmittedJobGraphStore;
 import org.apache.flink.runtime.query.KvStateRegistry;
 import org.apache.flink.runtime.query.netty.DisabledKvStateRequestStats;
 import org.apache.flink.runtime.query.netty.KvStateServer;
@@ -340,6 +342,16 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 			public LeaderElectionService getJobMasterLeaderElectionService(JobID jobID) throws Exception {
 				return null;
 			}
+
+			@Override
+			public CheckpointRecoveryFactory getCheckpointRecoveryFactory() throws Exception {
+				return null;
+			}
+
+			@Override
+			public SubmittedJobGraphStore getSubmittedJobGraphStore() throws Exception {
+				return null;
+			}
 		};
 
 		// start all the TaskManager services (network stack,  library cache, ...)

http://git-wip-us.apache.org/repos/asf/flink/blob/360eaf8a/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 2ac43be..1a5450d 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,10 +19,13 @@
 package org.apache.flink.runtime.highavailability;
 
 import org.apache.flink.api.common.JobID;
-import org.apache.flink.hadoop.shaded.org.jboss.netty.util.internal.ConcurrentHashMap;
+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.util.concurrent.ConcurrentHashMap;
+
 /**
  * A variant of the HighAvailabilityServices for testing. Each individual service can be set
  * to an arbitrary implementation, such as a mock or default service.
@@ -37,6 +40,9 @@ public class TestingHighAvailabilityServices implements HighAvailabilityServices
 
 	private volatile LeaderElectionService resourceManagerLeaderElectionService;
 
+	private volatile CheckpointRecoveryFactory checkpointRecoveryFactory;
+
+	private volatile SubmittedJobGraphStore submittedJobGraphStore;
 
 	// ------------------------------------------------------------------------
 	//  Setters for mock / testing implementations
@@ -58,6 +64,14 @@ public class TestingHighAvailabilityServices implements HighAvailabilityServices
 		this.resourceManagerLeaderElectionService = leaderElectionService;
 	}
 
+	public void setCheckpointRecoveryFactory(CheckpointRecoveryFactory checkpointRecoveryFactory) {
+		this.checkpointRecoveryFactory = checkpointRecoveryFactory;
+	}
+
+	public void setSubmittedJobGraphStore(SubmittedJobGraphStore submittedJobGraphStore) {
+		this.submittedJobGraphStore = submittedJobGraphStore;
+	}
+
 	// ------------------------------------------------------------------------
 	//  HA Services Methods
 	// ------------------------------------------------------------------------
@@ -103,4 +117,27 @@ public class TestingHighAvailabilityServices implements HighAvailabilityServices
 			throw new IllegalStateException("ResourceManagerLeaderElectionService has not been set");
 		}
 	}
+
+	@Override
+	public CheckpointRecoveryFactory getCheckpointRecoveryFactory() throws Exception {
+		CheckpointRecoveryFactory factory = checkpointRecoveryFactory;
+
+		if (factory != null) {
+			return factory;
+		} else {
+			throw new IllegalStateException("CheckpointRecoveryFactory has not been set");
+		}
+	}
+
+	@Override
+	public SubmittedJobGraphStore getSubmittedJobGraphStore() throws Exception {
+		SubmittedJobGraphStore store = submittedJobGraphStore;
+
+		if (store != null) {
+			return store;
+		} else {
+			throw new IllegalStateException("SubmittedJobGraphStore has not been set");
+
+		}
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/360eaf8a/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
new file mode 100644
index 0000000..dc3b5fd
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobManagerRunnerMockTest.java
@@ -0,0 +1,254 @@
+/*
+ * 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.highavailability.HighAvailabilityServices;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobmanager.OnCompletionActions;
+import org.apache.flink.runtime.leaderelection.LeaderElectionService;
+import org.apache.flink.runtime.rpc.RpcService;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.powermock.api.mockito.PowerMockito;
+import org.powermock.core.classloader.annotations.PrepareForTest;
+import org.powermock.modules.junit4.PowerMockRunner;
+
+import java.util.UUID;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+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;
+
+@RunWith(PowerMockRunner.class)
+@PrepareForTest(JobManagerRunner.class)
+public class JobManagerRunnerMockTest {
+
+	private JobManagerRunner runner;
+
+	private JobMaster jobManager;
+
+	private JobMasterGateway jobManagerGateway;
+
+	private LeaderElectionService leaderElectionService;
+
+	private TestingOnCompletionActions jobCompletion;
+
+	@Before
+	public void setUp() throws Exception {
+		jobManager = mock(JobMaster.class);
+		jobManagerGateway = mock(JobMasterGateway.class);
+		when(jobManager.getSelf()).thenReturn(jobManagerGateway);
+
+		PowerMockito.whenNew(JobMaster.class).withAnyArguments().thenReturn(jobManager);
+
+		jobCompletion = new TestingOnCompletionActions();
+
+		leaderElectionService = mock(LeaderElectionService.class);
+		when(leaderElectionService.hasLeadership()).thenReturn(true);
+
+		HighAvailabilityServices haServices = mock(HighAvailabilityServices.class);
+		when(haServices.getJobMasterLeaderElectionService(any(JobID.class))).thenReturn(leaderElectionService);
+
+		runner = PowerMockito.spy(new JobManagerRunner(
+			new JobGraph("test"),
+			mock(Configuration.class),
+			mock(RpcService.class),
+			haServices,
+			mock(JobManagerServices.class),
+			jobCompletion));
+	}
+
+	@After
+	public void tearDown() throws Exception {
+	}
+
+	@Test
+	public void testStartAndShutdown() throws Exception {
+		runner.start();
+		verify(jobManager).init();
+		verify(jobManager).start();
+		verify(leaderElectionService).start(runner);
+
+		assertTrue(!jobCompletion.isJobFinished());
+		assertTrue(!jobCompletion.isJobFailed());
+
+		runner.shutdown();
+		verify(leaderElectionService).stop();
+		verify(jobManager).shutDown();
+	}
+
+	@Test
+	public void testShutdownBeforeGrantLeadership() throws Exception {
+		runner.start();
+		verify(jobManager).init();
+		verify(jobManager).start();
+		verify(leaderElectionService).start(runner);
+
+		runner.shutdown();
+		verify(leaderElectionService).stop();
+		verify(jobManager).shutDown();
+
+		assertTrue(!jobCompletion.isJobFinished());
+		assertTrue(!jobCompletion.isJobFailed());
+
+		runner.grantLeadership(UUID.randomUUID());
+		assertTrue(!jobCompletion.isJobFinished());
+		assertTrue(!jobCompletion.isJobFailed());
+
+	}
+
+	@Test
+	public void testJobFinishedByOtherBeforeGrantLeadership() throws Exception {
+		runner.start();
+
+		when(runner.isJobFinishedByOthers()).thenReturn(true);
+		runner.grantLeadership(UUID.randomUUID());
+
+		// runner should shutdown automatic and informed the job completion
+		verify(leaderElectionService).stop();
+		verify(jobManager).shutDown();
+
+		assertTrue(jobCompletion.isJobFinished());
+		assertTrue(jobCompletion.isJobFinishedByOther());
+	}
+
+	@Test
+	public void testJobFinished() throws Exception {
+		runner.start();
+
+		runner.grantLeadership(UUID.randomUUID());
+		verify(jobManagerGateway).startJob();
+		assertTrue(!jobCompletion.isJobFinished());
+
+		// runner been told by JobManager that job is finished
+		runner.jobFinished(mock(JobExecutionResult.class));
+
+		assertTrue(jobCompletion.isJobFinished());
+		assertFalse(jobCompletion.isJobFinishedByOther());
+		verify(leaderElectionService).stop();
+		verify(jobManager).shutDown();
+		assertTrue(runner.isShutdown());
+	}
+
+	@Test
+	public void testJobFailed() throws Exception {
+		runner.start();
+
+		runner.grantLeadership(UUID.randomUUID());
+		verify(jobManagerGateway).startJob();
+		assertTrue(!jobCompletion.isJobFinished());
+
+		// runner been told by JobManager that job is failed
+		runner.jobFailed(new Exception("failed manually"));
+
+		assertTrue(jobCompletion.isJobFailed());
+		verify(leaderElectionService).stop();
+		verify(jobManager).shutDown();
+		assertTrue(runner.isShutdown());
+	}
+
+	@Test
+	public void testLeadershipRevoked() throws Exception {
+		runner.start();
+
+		runner.grantLeadership(UUID.randomUUID());
+		verify(jobManagerGateway).startJob();
+		assertTrue(!jobCompletion.isJobFinished());
+
+		runner.revokeLeadership();
+		verify(jobManagerGateway).suspendJob(any(Throwable.class));
+		assertFalse(runner.isShutdown());
+	}
+
+	@Test
+	public void testRegainLeadership() throws Exception {
+		runner.start();
+
+		runner.grantLeadership(UUID.randomUUID());
+		verify(jobManagerGateway).startJob();
+		assertTrue(!jobCompletion.isJobFinished());
+
+		runner.revokeLeadership();
+		verify(jobManagerGateway).suspendJob(any(Throwable.class));
+		assertFalse(runner.isShutdown());
+
+		runner.grantLeadership(UUID.randomUUID());
+		verify(jobManagerGateway, times(2)).startJob();
+	}
+
+	private static class TestingOnCompletionActions implements OnCompletionActions {
+
+		private volatile JobExecutionResult result;
+
+		private volatile Throwable failedCause;
+
+		private volatile boolean finishedByOther;
+
+		@Override
+		public void jobFinished(JobExecutionResult result) {
+			checkArgument(!isJobFinished(), "job finished already");
+			checkArgument(!isJobFailed(), "job failed already");
+
+			this.result = result;
+		}
+
+		@Override
+		public void jobFailed(Throwable cause) {
+			checkArgument(!isJobFinished(), "job finished already");
+			checkArgument(!isJobFailed(), "job failed already");
+
+			this.failedCause = cause;
+		}
+
+		@Override
+		public void jobFinishedByOther() {
+			checkArgument(!isJobFinished(), "job finished already");
+			checkArgument(!isJobFailed(), "job failed already");
+
+			this.finishedByOther = true;
+		}
+
+		@Override
+		public void onFatalError(Throwable exception) {
+			jobFailed(exception);
+		}
+
+		boolean isJobFinished() {
+			return result != null || finishedByOther;
+		}
+
+		boolean isJobFinishedByOther() {
+			return finishedByOther;
+		}
+
+		boolean isJobFailed() {
+			return failedCause != null;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/360eaf8a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcConnectionTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcConnectionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcConnectionTest.java
index 6363662..e05c8d8 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcConnectionTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcConnectionTest.java
@@ -19,23 +19,21 @@
 package org.apache.flink.runtime.rpc;
 
 import akka.actor.ActorSystem;
-import akka.util.Timeout;
 
+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.rpc.akka.AkkaRpcService;
 import org.apache.flink.runtime.rpc.exceptions.RpcConnectionException;
 import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway;
 
-import org.junit.AfterClass;
 import org.junit.Test;
 
 import scala.Option;
 import scala.Tuple2;
-import scala.concurrent.Await;
-import scala.concurrent.Future;
-import scala.concurrent.duration.FiniteDuration;
 
+import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 
@@ -57,19 +55,20 @@ public class RpcConnectionTest {
 
 			// we start the RPC service with a very long timeout to ensure that the test
 			// can only pass if the connection problem is not recognized merely via a timeout
-			rpcService = new AkkaRpcService(actorSystem, new Timeout(10000000, TimeUnit.SECONDS));
+			rpcService = new AkkaRpcService(actorSystem, Time.of(10000000, TimeUnit.SECONDS));
 
 			Future<TaskExecutorGateway> future = rpcService.connect("foo.bar.com.test.invalid", TaskExecutorGateway.class);
 
-			Await.result(future, new FiniteDuration(10000000, TimeUnit.SECONDS));
+			future.get(10000000, TimeUnit.SECONDS);
 			fail("should never complete normally");
 		}
 		catch (TimeoutException e) {
 			fail("should not fail with a generic timeout exception");
 		}
-		catch (RpcConnectionException e) {
+		catch (ExecutionException e) {
 			// that is what we want
-			assertTrue("wrong error message", e.getMessage().contains("foo.bar.com.test.invalid"));
+			assertTrue(e.getCause() instanceof RpcConnectionException);
+			assertTrue("wrong error message", e.getCause().getMessage().contains("foo.bar.com.test.invalid"));
 		}
 		catch (Throwable t) {
 			fail("wrong exception: " + t);


[40/50] [abbrv] flink git commit: [FLINK-4606] Integrate the new ResourceManager with the existed FlinkResourceManager

Posted by se...@apache.org.
[FLINK-4606]  Integrate the new ResourceManager with the existed FlinkResourceManager


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

Branch: refs/heads/flip-6
Commit: 415af17fdf45fe157a6ee5f7187ee63e8845f168
Parents: bb781ae
Author: beyond1920 <be...@126.com>
Authored: Fri Sep 9 09:11:24 2016 +0800
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Oct 14 15:14:42 2016 +0200

----------------------------------------------------------------------
 .../InfoMessageListenerRpcGateway.java          |  35 +++
 .../resourcemanager/ResourceManager.java        | 214 ++++++++++++++++---
 .../resourcemanager/ResourceManagerGateway.java |  23 ++
 .../StandaloneResourceManager.java              |  64 ++++++
 .../resourcemanager/ResourceManagerHATest.java  |   2 +-
 .../ResourceManagerJobMasterTest.java           |   2 +-
 .../ResourceManagerTaskExecutorTest.java        |   2 +-
 .../slotmanager/SlotProtocolTest.java           |   5 +-
 8 files changed, 318 insertions(+), 29 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/415af17f/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/InfoMessageListenerRpcGateway.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/InfoMessageListenerRpcGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/InfoMessageListenerRpcGateway.java
new file mode 100644
index 0000000..c1eeefa
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/InfoMessageListenerRpcGateway.java
@@ -0,0 +1,35 @@
+/*
+ * 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.clusterframework.messages.InfoMessage;
+import org.apache.flink.runtime.jobmaster.JobMaster;
+import org.apache.flink.runtime.rpc.RpcGateway;
+
+/**
+ * A gateway to listen for info messages from {@link ResourceManager}
+ */
+public interface InfoMessageListenerRpcGateway extends RpcGateway {
+
+	/**
+	 * Notifies when resource manager need to notify listener about InfoMessage
+	 * @param infoMessage
+	 */
+	void notifyInfoMessage(InfoMessage infoMessage);
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/415af17f/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 88b8a11..83dc4db 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,19 +20,22 @@ package org.apache.flink.runtime.resourcemanager;
 
 import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.clusterframework.ApplicationStatus;
+import org.apache.flink.runtime.clusterframework.messages.InfoMessage;
 import org.apache.flink.runtime.clusterframework.types.ResourceID;
+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.highavailability.HighAvailabilityServices;
-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.resourcemanager.slotmanager.SlotManager;
-import org.apache.flink.runtime.rpc.RpcMethod;
 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.jobmaster.JobMaster;
 import org.apache.flink.runtime.jobmaster.JobMasterGateway;
@@ -42,8 +45,6 @@ import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway;
 import org.apache.flink.runtime.taskexecutor.TaskExecutorRegistrationSuccess;
 import org.apache.flink.runtime.util.LeaderConnectionInfo;
 import org.apache.flink.runtime.util.LeaderRetrievalUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import scala.concurrent.duration.FiniteDuration;
 
 import java.util.HashMap;
@@ -66,15 +67,16 @@ import static org.apache.flink.util.Preconditions.checkNotNull;
  *     <li>{@link #requestSlot(SlotRequest)} requests a slot from the resource manager</li>
  * </ul>
  */
-public class ResourceManager extends RpcEndpoint<ResourceManagerGateway> implements LeaderContender {
+public abstract class ResourceManager<ResourceManagerGateway, WorkerType extends TaskExecutorRegistration> extends RpcEndpoint implements LeaderContender {
 
-	private final Logger LOG = LoggerFactory.getLogger(getClass());
+	/** The exit code with which the process is stopped in case of a fatal error */
+	protected static final int EXIT_CODE_FATAL_ERROR = -13;
 
 	private final Map<JobID, JobMasterGateway> jobMasterGateways;
 
 	private final Set<LeaderRetrievalListener> jobMasterLeaderRetrievalListeners;
 
-	private final Map<ResourceID, TaskExecutorRegistration> taskExecutorGateways;
+	private final Map<ResourceID, WorkerType> taskExecutorGateways;
 
 	private final HighAvailabilityServices highAvailabilityServices;
 
@@ -84,16 +86,16 @@ public class ResourceManager extends RpcEndpoint<ResourceManagerGateway> impleme
 
 	private UUID leaderSessionID;
 
-	public ResourceManager(
-			RpcService rpcService,
-			HighAvailabilityServices highAvailabilityServices,
-			SlotManager slotManager) {
+	private Map<String, InfoMessageListenerRpcGateway> infoMessageListeners;
+
+	public ResourceManager(RpcService rpcService, HighAvailabilityServices highAvailabilityServices, SlotManager slotManager) {
 		super(rpcService);
 		this.highAvailabilityServices = checkNotNull(highAvailabilityServices);
 		this.jobMasterGateways = new HashMap<>();
-		this.slotManager = slotManager;
+		this.slotManager = checkNotNull(slotManager);
 		this.jobMasterLeaderRetrievalListeners = new HashSet<>();
 		this.taskExecutorGateways = new HashMap<>();
+		infoMessageListeners = new HashMap<>();
 	}
 
 	@Override
@@ -103,6 +105,8 @@ public class ResourceManager extends RpcEndpoint<ResourceManagerGateway> impleme
 			super.start();
 			leaderElectionService = highAvailabilityServices.getResourceManagerLeaderElectionService();
 			leaderElectionService.start(this);
+			// 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);
@@ -166,12 +170,12 @@ public class ResourceManager extends RpcEndpoint<ResourceManagerGateway> impleme
 						jobMasterLeaderInfo = LeaderRetrievalUtils.retrieveLeaderConnectionInfo(
 							highAvailabilityServices.getJobMasterLeaderRetriever(jobID), new FiniteDuration(5, TimeUnit.SECONDS));
 					} catch (Exception e) {
-						LOG.warn("Failed to start JobMasterLeaderRetriever for JobID {}", jobID);
+						log.warn("Failed to start JobMasterLeaderRetriever for JobID {}", jobID);
 						throw new Exception("Failed to retrieve JobMasterLeaderRetriever");
 					}
 
 					if (!jobMasterLeaderId.equals(jobMasterLeaderInfo.getLeaderSessionID())) {
-						LOG.info("Declining registration request from non-leading JobManager {}", jobMasterAddress);
+						log.info("Declining registration request from non-leading JobManager {}", jobMasterAddress);
 						throw new Exception("JobManager is not leading");
 					}
 
@@ -190,7 +194,7 @@ public class ResourceManager extends RpcEndpoint<ResourceManagerGateway> impleme
 							LeaderRetrievalService jobMasterLeaderRetriever = highAvailabilityServices.getJobMasterLeaderRetriever(jobID);
 							jobMasterLeaderRetriever.start(jobMasterLeaderListener);
 						} catch (Exception e) {
-							LOG.warn("Failed to start JobMasterLeaderRetriever for JobID {}", jobID);
+							log.warn("Failed to start JobMasterLeaderRetriever for JobID {}", jobID);
 							return new RegistrationResponse.Decline("Failed to retrieve JobMasterLeaderRetriever");
 						}
 						jobMasterLeaderRetrievalListeners.add(jobMasterLeaderListener);
@@ -237,13 +241,24 @@ public class ResourceManager extends RpcEndpoint<ResourceManagerGateway> impleme
 				if (throwable != null) {
 					return new RegistrationResponse.Decline(throwable.getMessage());
 				} else {
-					InstanceID id = new InstanceID();
-					TaskExecutorRegistration oldTaskExecutor =
-						taskExecutorGateways.put(resourceID, new TaskExecutorRegistration(taskExecutorGateway, id));
-					if (oldTaskExecutor != null) {
-						log.warn("Receive a duplicate registration from TaskExecutor {} at ({})", resourceID, taskExecutorAddress);
+					WorkerType startedWorker = taskExecutorGateways.get(resourceID);
+					if(startedWorker != null) {
+						String oldWorkerAddress = startedWorker.getTaskExecutorGateway().getAddress();
+						if (taskExecutorAddress.equals(oldWorkerAddress)) {
+							log.warn("Receive a duplicate registration from TaskExecutor {} at ({})", resourceID, taskExecutorAddress);
+						} else {
+							log.warn("Receive a duplicate registration from TaskExecutor {} at different address, previous ({}), new ({})",
+								resourceID, oldWorkerAddress, taskExecutorAddress);
+							// TODO :: suggest old taskExecutor to stop itself
+							slotManager.notifyTaskManagerFailure(resourceID);
+							startedWorker = workerStarted(resourceID, taskExecutorGateway);
+							taskExecutorGateways.put(resourceID, startedWorker);
+						}
+					} else {
+						startedWorker = workerStarted(resourceID, taskExecutorGateway);
+						taskExecutorGateways.put(resourceID, startedWorker);
 					}
-					return new TaskExecutorRegistrationSuccess(id, 5000);
+					return new TaskExecutorRegistrationSuccess(startedWorker.getInstanceID(), 5000);
 				}
 			}
 		}, getMainThreadExecutor());
@@ -263,14 +278,12 @@ public class ResourceManager extends RpcEndpoint<ResourceManagerGateway> impleme
 		if (jobMasterGateway != null) {
 			return slotManager.requestSlot(slotRequest);
 		} else {
-			LOG.info("Ignoring slot request for unknown JobMaster with JobID {}", jobId);
+			log.info("Ignoring slot request for unknown JobMaster with JobID {}", jobId);
 			return new SlotRequestRejected(slotRequest.getAllocationId());
 		}
 	}
 
 
-
-
 	// ------------------------------------------------------------------------
 	//  Leader Contender
 	// ------------------------------------------------------------------------
@@ -324,6 +337,158 @@ public class ResourceManager extends RpcEndpoint<ResourceManagerGateway> impleme
 		shutDown();
 	}
 
+	/**
+	 * Registers an infoMessage listener
+	 *
+	 * @param infoMessageListenerAddress address of infoMessage listener to register to this resource manager
+	 */
+	@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());
+
+			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;
+				}
+			}, 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);
+	}
+
+	/**
+	 * Shutdowns 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);
+	}
+
+	/**
+	 * This method should be called by the framework once it detects that a currently registered task executor has failed.
+	 *
+	 * @param resourceID Id of the worker that has failed.
+	 * @param message An informational message that explains why the worker failed.
+	 */
+	public void notifyWorkerFailed(final ResourceID resourceID, String message) {
+		runAsync(new Runnable() {
+			@Override
+			public void run() {
+				WorkerType worker = taskExecutorGateways.remove(resourceID);
+				if (worker != null) {
+					// TODO :: suggest failed task executor to stop itself
+					slotManager.notifyTaskManagerFailure(resourceID);
+				}
+			}
+		});
+	}
+
+	/**
+	 * Gets the number of currently started TaskManagers.
+	 *
+	 * @return The number of currently started TaskManagers.
+	 */
+	public int getNumberOfStartedTaskManagers() {
+		return taskExecutorGateways.size();
+	}
+
+	/**
+	 * Notifies the resource manager 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.
+	 */
+	public void onFatalError(final String message, final Throwable error) {
+		runAsync(new Runnable() {
+			@Override
+			public void run() {
+				fatalError(message, error);
+			}
+		});
+	}
+
+	// ------------------------------------------------------------------------
+	//  Framework specific behavior
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Initializes the framework specific components.
+	 *
+	 * @throws Exception Exceptions during initialization cause the resource manager to fail.
+	 */
+	protected abstract void initialize() throws Exception;
+
+	/**
+	 * Callback when a task executor register.
+	 *
+	 * @param resourceID The worker resource id
+	 * @param taskExecutorGateway the task executor gateway
+	 */
+	protected abstract WorkerType workerStarted(ResourceID resourceID, TaskExecutorGateway taskExecutorGateway);
+
+	/**
+	 * Callback when a resource manager faced a fatal error
+	 * @param message
+	 * @param error
+	 */
+	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.
+	 *
+	 * This method also needs to make sure all pending containers that are not registered
+	 * yet are returned.
+	 *
+	 * @param finalStatus The application status to report.
+	 * @param optionalDiagnostics An optional diagnostics message.
+	 */
+	protected abstract void shutDownApplication(ApplicationStatus finalStatus, String optionalDiagnostics);
+
+	// ------------------------------------------------------------------------
+	//  Info messaging
+	// ------------------------------------------------------------------------
+
+	public void sendInfoMessage(final String message) {
+		runAsync(new Runnable() {
+			@Override
+			public void run() {
+				InfoMessage infoMessage = new InfoMessage(message);
+				for (InfoMessageListenerRpcGateway listenerRpcGateway : infoMessageListeners.values()) {
+					listenerRpcGateway
+						.notifyInfoMessage(infoMessage);
+				}
+			}
+		});
+	}
+
 	private static class JobMasterLeaderListener implements LeaderRetrievalListener {
 
 		private final JobID jobID;
@@ -343,5 +508,6 @@ public class ResourceManager extends RpcEndpoint<ResourceManagerGateway> impleme
 			// TODO
 		}
 	}
+
 }
 

http://git-wip-us.apache.org/repos/asf/flink/blob/415af17f/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 484cea7..7c44006 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
@@ -20,6 +20,7 @@ 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.clusterframework.ApplicationStatus;
 import org.apache.flink.runtime.clusterframework.types.ResourceID;
 import org.apache.flink.runtime.concurrent.Future;
 import org.apache.flink.runtime.rpc.RpcGateway;
@@ -75,4 +76,26 @@ public interface ResourceManagerGateway extends RpcGateway {
 		String taskExecutorAddress,
 		ResourceID resourceID,
 		@RpcTimeout Time timeout);
+
+	/**
+	 * Registers an infoMessage listener
+	 *
+	 * @param infoMessageListenerAddress address of infoMessage listener to register to this resource manager
+	 */
+	void registerInfoMessageListener(String infoMessageListenerAddress);
+
+	/**
+	 * Unregisters an infoMessage listener
+	 *
+	 * @param infoMessageListenerAddress address of infoMessage listener to unregister from this resource manager
+	 *
+	 */
+	void unRegisterInfoMessageListener(String infoMessageListenerAddress);
+
+	/**
+	 * shutdown cluster
+	 * @param finalStatus
+	 * @param optionalDiagnostics
+	 */
+	void shutDownCluster(final ApplicationStatus finalStatus, final String optionalDiagnostics);
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/415af17f/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
new file mode 100644
index 0000000..84db1ee
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/StandaloneResourceManager.java
@@ -0,0 +1,64 @@
+/*
+ * 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.clusterframework.ApplicationStatus;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager;
+import org.apache.flink.runtime.rpc.RpcService;
+import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway;
+
+/**
+ * A standalone implementation of the resource manager. Used when the system is started in
+ * standalone mode (via scripts), rather than via a resource framework like YARN or Mesos.
+ */
+public class StandaloneResourceManager extends ResourceManager<ResourceManagerGateway, TaskExecutorRegistration> {
+
+	public StandaloneResourceManager(RpcService rpcService,
+		HighAvailabilityServices highAvailabilityServices,
+		SlotManager slotManager) {
+		super(rpcService, highAvailabilityServices, slotManager);
+	}
+
+	@Override
+	protected void initialize() throws Exception {
+		// nothing to initialize
+	}
+
+	@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 TaskExecutorRegistration workerStarted(ResourceID resourceID, TaskExecutorGateway taskExecutorGateway) {
+		InstanceID instanceID = new InstanceID();
+		TaskExecutorRegistration taskExecutorRegistration = new TaskExecutorRegistration(taskExecutorGateway, instanceID);
+		return taskExecutorRegistration;
+	}
+
+	@Override
+	protected void shutDownApplication(ApplicationStatus finalStatus, String optionalDiagnostics) {
+
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/415af17f/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 64a1191..fdb83f5 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
@@ -55,7 +55,7 @@ public class ResourceManagerHATest {
 		highAvailabilityServices.setResourceManagerLeaderElectionService(leaderElectionService);
 
 		SlotManager slotManager = mock(SlotManager.class);
-		final ResourceManager resourceManager = new ResourceManager(rpcService, highAvailabilityServices, slotManager);
+		final ResourceManager resourceManager = new StandaloneResourceManager(rpcService, highAvailabilityServices, slotManager);
 		resourceManager.start();
 		// before grant leadership, resourceManager's leaderId is null
 		Assert.assertNull(resourceManager.getLeaderSessionID());

http://git-wip-us.apache.org/repos/asf/flink/blob/415af17f/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 332c093..8f09152 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
@@ -160,7 +160,7 @@ public class ResourceManagerJobMasterTest {
 		TestingHighAvailabilityServices highAvailabilityServices = new TestingHighAvailabilityServices();
 		highAvailabilityServices.setResourceManagerLeaderElectionService(resourceManagerLeaderElectionService);
 		highAvailabilityServices.setJobMasterLeaderRetriever(jobID, jobMasterLeaderRetrievalService);
-		ResourceManager resourceManager = new ResourceManager(rpcService, highAvailabilityServices, new SimpleSlotManager());
+		ResourceManager resourceManager = new StandaloneResourceManager(rpcService, highAvailabilityServices, new SimpleSlotManager());
 		resourceManager.start();
 		return resourceManager;
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/415af17f/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 ed7c7d7..e6d1ed5 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
@@ -121,7 +121,7 @@ public class ResourceManagerTaskExecutorTest {
 	private ResourceManager createAndStartResourceManager(TestingLeaderElectionService rmLeaderElectionService) {
 		TestingHighAvailabilityServices highAvailabilityServices = new TestingHighAvailabilityServices();
 		highAvailabilityServices.setResourceManagerLeaderElectionService(rmLeaderElectionService);
-		ResourceManager resourceManager = new ResourceManager(rpcService, highAvailabilityServices, new SimpleSlotManager());
+		ResourceManager resourceManager = new StandaloneResourceManager(rpcService, highAvailabilityServices, new SimpleSlotManager());
 		resourceManager.start();
 		return resourceManager;
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/415af17f/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 0232fab..ff25897 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
@@ -35,6 +35,7 @@ import org.apache.flink.runtime.registration.RegistrationResponse;
 import org.apache.flink.runtime.resourcemanager.ResourceManager;
 import org.apache.flink.runtime.resourcemanager.SlotRequest;
 import org.apache.flink.runtime.resourcemanager.SlotRequestReply;
+import org.apache.flink.runtime.resourcemanager.StandaloneResourceManager;
 import org.apache.flink.runtime.rpc.TestingSerialRpcService;
 import org.apache.flink.runtime.taskexecutor.SlotReport;
 import org.apache.flink.runtime.taskexecutor.SlotStatus;
@@ -100,7 +101,7 @@ public class SlotProtocolTest extends TestLogger {
 
 		TestingSlotManager slotManager = Mockito.spy(new TestingSlotManager());
 		ResourceManager resourceManager =
-			new ResourceManager(testRpcService, testingHaServices, slotManager);
+			new StandaloneResourceManager(testRpcService, testingHaServices, slotManager);
 		resourceManager.start();
 		rmLeaderElectionService.isLeader(rmLeaderID);
 
@@ -179,7 +180,7 @@ public class SlotProtocolTest extends TestLogger {
 
 		TestingSlotManager slotManager = Mockito.spy(new TestingSlotManager());
 		ResourceManager resourceManager =
-			new ResourceManager(testRpcService, testingHaServices, slotManager);
+			new StandaloneResourceManager(testRpcService, testingHaServices, slotManager);
 		resourceManager.start();
 		rmLeaderElectionService.isLeader(rmLeaderID);
 


[04/50] [abbrv] flink git commit: [FLINK-4529] [flip-6] Move TaskExecutor, JobMaster and ResourceManager out of the rpc package

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/9c247d1f/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
new file mode 100644
index 0000000..4871b96
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java
@@ -0,0 +1,827 @@
+/*
+ * 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 akka.actor.ActorSystem;
+import akka.dispatch.ExecutionContexts$;
+import akka.util.Timeout;
+import com.typesafe.config.Config;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.IllegalConfigurationException;
+import org.apache.flink.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.annotation.VisibleForTesting;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
+import org.apache.flink.runtime.instance.InstanceConnectionInfo;
+import org.apache.flink.runtime.io.disk.iomanager.IOManager;
+import org.apache.flink.runtime.io.disk.iomanager.IOManager.IOMode;
+import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
+import org.apache.flink.runtime.io.network.NetworkEnvironment;
+import org.apache.flink.runtime.io.network.netty.NettyConfig;
+import org.apache.flink.runtime.leaderelection.LeaderElectionService;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+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.runtime.rpc.akka.AkkaRpcService;
+import org.apache.flink.runtime.taskmanager.MemoryLogger;
+import org.apache.flink.runtime.util.EnvironmentInformation;
+import org.apache.flink.runtime.util.LeaderRetrievalUtils;
+import org.apache.flink.runtime.taskmanager.NetworkEnvironmentConfiguration;
+import org.apache.flink.util.MathUtils;
+import org.apache.flink.util.NetUtils;
+
+import scala.Tuple2;
+import scala.Option;
+import scala.Some;
+import scala.concurrent.ExecutionContext;
+import scala.concurrent.duration.Duration;
+import scala.concurrent.duration.FiniteDuration;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.util.UUID;
+import java.util.concurrent.ForkJoinPool;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * TaskExecutor implementation. The task executor is responsible for the execution of multiple
+ * {@link org.apache.flink.runtime.taskmanager.Task}.
+ */
+public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
+
+	private static final Logger LOG = LoggerFactory.getLogger(TaskExecutor.class);
+
+	/** The unique resource ID of this TaskExecutor */
+	private final ResourceID resourceID;
+
+	/** The access to the leader election and metadata storage services */
+	private final HighAvailabilityServices haServices;
+
+	/** The task manager configuration */
+	private final TaskExecutorConfiguration taskExecutorConfig;
+
+	/** The I/O manager component in the task manager */
+	private final IOManager ioManager;
+
+	/** The memory manager component in the task manager */
+	private final MemoryManager memoryManager;
+
+	/** The network component in the task manager */
+	private final NetworkEnvironment networkEnvironment;
+
+	/** The number of slots in the task manager, should be 1 for YARN */
+	private final int numberOfSlots;
+
+	// --------- resource manager --------
+
+	private TaskExecutorToResourceManagerConnection resourceManagerConnection;
+
+	// ------------------------------------------------------------------------
+
+	public TaskExecutor(
+			TaskExecutorConfiguration taskExecutorConfig,
+			ResourceID resourceID,
+			MemoryManager memoryManager,
+			IOManager ioManager,
+			NetworkEnvironment networkEnvironment,
+			int numberOfSlots,
+			RpcService rpcService,
+			HighAvailabilityServices haServices) {
+
+		super(rpcService);
+
+		this.taskExecutorConfig = checkNotNull(taskExecutorConfig);
+		this.resourceID = checkNotNull(resourceID);
+		this.memoryManager = checkNotNull(memoryManager);
+		this.ioManager = checkNotNull(ioManager);
+		this.networkEnvironment = checkNotNull(networkEnvironment);
+		this.numberOfSlots = checkNotNull(numberOfSlots);
+		this.haServices = checkNotNull(haServices);
+	}
+
+	// ------------------------------------------------------------------------
+	//  Life cycle
+	// ------------------------------------------------------------------------
+
+	@Override
+	public void start() {
+		super.start();
+
+		// start by connecting to the ResourceManager
+		try {
+			haServices.getResourceManagerLeaderRetriever().start(new ResourceManagerLeaderListener());
+		} catch (Exception e) {
+			onFatalErrorAsync(e);
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  RPC methods - ResourceManager related
+	// ------------------------------------------------------------------------
+
+	@RpcMethod
+	public void notifyOfNewResourceManagerLeader(String newLeaderAddress, UUID newLeaderId) {
+		if (resourceManagerConnection != null) {
+			if (newLeaderAddress != null) {
+				// the resource manager switched to a new leader
+				log.info("ResourceManager leader changed from {} to {}. Registering at new leader.",
+					resourceManagerConnection.getResourceManagerAddress(), newLeaderAddress);
+			}
+			else {
+				// address null means that the current leader is lost without a new leader being there, yet
+				log.info("Current ResourceManager {} lost leader status. Waiting for new ResourceManager leader.",
+					resourceManagerConnection.getResourceManagerAddress());
+			}
+
+			// drop the current connection or connection attempt
+			if (resourceManagerConnection != null) {
+				resourceManagerConnection.close();
+				resourceManagerConnection = null;
+			}
+		}
+
+		// establish a connection to the new leader
+		if (newLeaderAddress != null) {
+			log.info("Attempting to register at ResourceManager {}", newLeaderAddress);
+			resourceManagerConnection =
+				new TaskExecutorToResourceManagerConnection(log, this, newLeaderAddress, newLeaderId);
+			resourceManagerConnection.start();
+		}
+	}
+
+	/**
+	 * Starts and runs the TaskManager.
+	 * <p/>
+	 * This method first tries to select the network interface to use for the TaskManager
+	 * communication. The network interface is used both for the actor communication
+	 * (coordination) as well as for the data exchange between task managers. Unless
+	 * the hostname/interface is explicitly configured in the configuration, this
+	 * method will try out various interfaces and methods to connect to the JobManager
+	 * and select the one where the connection attempt is successful.
+	 * <p/>
+	 * After selecting the network interface, this method brings up an actor system
+	 * for the TaskManager and its actors, starts the TaskManager's services
+	 * (library cache, shuffle network stack, ...), and starts the TaskManager itself.
+	 *
+	 * @param configuration    The configuration for the TaskManager.
+	 * @param resourceID       The id of the resource which the task manager will run on.
+	 */
+	public static void selectNetworkInterfaceAndRunTaskManager(
+		Configuration configuration,
+		ResourceID resourceID) throws Exception {
+
+		final InetSocketAddress taskManagerAddress = selectNetworkInterfaceAndPort(configuration);
+
+		runTaskManager(taskManagerAddress.getHostName(), resourceID, taskManagerAddress.getPort(), configuration);
+	}
+
+	private static InetSocketAddress selectNetworkInterfaceAndPort(Configuration configuration)
+		throws Exception {
+		String taskManagerHostname = configuration.getString(ConfigConstants.TASK_MANAGER_HOSTNAME_KEY, null);
+		if (taskManagerHostname != null) {
+			LOG.info("Using configured hostname/address for TaskManager: " + taskManagerHostname);
+		} else {
+			LeaderRetrievalService leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(configuration);
+			FiniteDuration lookupTimeout = AkkaUtils.getLookupTimeout(configuration);
+
+			InetAddress taskManagerAddress = LeaderRetrievalUtils.findConnectingAddress(leaderRetrievalService, lookupTimeout);
+			taskManagerHostname = taskManagerAddress.getHostName();
+			LOG.info("TaskManager will use hostname/address '{}' ({}) for communication.",
+				taskManagerHostname, taskManagerAddress.getHostAddress());
+		}
+
+		// if no task manager port has been configured, use 0 (system will pick any free port)
+		final int actorSystemPort = configuration.getInteger(ConfigConstants.TASK_MANAGER_IPC_PORT_KEY, 0);
+		if (actorSystemPort < 0 || actorSystemPort > 65535) {
+			throw new IllegalConfigurationException("Invalid value for '" +
+				ConfigConstants.TASK_MANAGER_IPC_PORT_KEY +
+				"' (port for the TaskManager actor system) : " + actorSystemPort +
+				" - Leave config parameter empty or use 0 to let the system choose a port automatically.");
+		}
+
+		return new InetSocketAddress(taskManagerHostname, actorSystemPort);
+	}
+
+	/**
+	 * Starts and runs the TaskManager. Brings up an actor system for the TaskManager and its
+	 * actors, starts the TaskManager's services (library cache, shuffle network stack, ...),
+	 * and starts the TaskManager itself.
+	 * <p/>
+	 * This method will also spawn a process reaper for the TaskManager (kill the process if
+	 * the actor fails) and optionally start the JVM memory logging thread.
+	 *
+	 * @param taskManagerHostname The hostname/address of the interface where the actor system
+	 *                            will communicate.
+	 * @param resourceID          The id of the resource which the task manager will run on.
+	 * @param actorSystemPort   The port at which the actor system will communicate.
+	 * @param configuration       The configuration for the TaskManager.
+	 */
+	private static void runTaskManager(
+		String taskManagerHostname,
+		ResourceID resourceID,
+		int actorSystemPort,
+		final Configuration configuration) throws Exception {
+
+		LOG.info("Starting TaskManager");
+
+		// Bring up the TaskManager actor system first, bind it to the given address.
+
+		LOG.info("Starting TaskManager actor system at " +
+			NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort));
+
+		final ActorSystem taskManagerSystem;
+		try {
+			Tuple2<String, Object> address = new Tuple2<String, Object>(taskManagerHostname, actorSystemPort);
+			Config akkaConfig = AkkaUtils.getAkkaConfig(configuration, new Some<>(address));
+			LOG.debug("Using akka configuration\n " + akkaConfig);
+			taskManagerSystem = AkkaUtils.createActorSystem(akkaConfig);
+		} catch (Throwable t) {
+			if (t instanceof org.jboss.netty.channel.ChannelException) {
+				Throwable cause = t.getCause();
+				if (cause != null && t.getCause() instanceof java.net.BindException) {
+					String address = NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort);
+					throw new IOException("Unable to bind TaskManager actor system to address " +
+						address + " - " + cause.getMessage(), t);
+				}
+			}
+			throw new Exception("Could not create TaskManager actor system", t);
+		}
+
+		// start akka rpc service based on actor system
+		final Timeout timeout = new Timeout(AkkaUtils.getTimeout(configuration).toMillis(), TimeUnit.MILLISECONDS);
+		final AkkaRpcService akkaRpcService = new AkkaRpcService(taskManagerSystem, timeout);
+
+		// start high availability service to implement getResourceManagerLeaderRetriever method only
+		final HighAvailabilityServices haServices = new HighAvailabilityServices() {
+			@Override
+			public LeaderRetrievalService getResourceManagerLeaderRetriever() throws Exception {
+				return LeaderRetrievalUtils.createLeaderRetrievalService(configuration);
+			}
+
+			@Override
+			public LeaderElectionService getResourceManagerLeaderElectionService() throws Exception {
+				return null;
+			}
+
+			@Override
+			public LeaderElectionService getJobMasterLeaderElectionService(JobID jobID) throws Exception {
+				return null;
+			}
+		};
+
+		// start all the TaskManager services (network stack,  library cache, ...)
+		// and the TaskManager actor
+		try {
+			LOG.info("Starting TaskManager actor");
+			TaskExecutor taskExecutor = startTaskManagerComponentsAndActor(
+				configuration,
+				resourceID,
+				akkaRpcService,
+				taskManagerHostname,
+				haServices,
+				false);
+
+			taskExecutor.start();
+
+			// if desired, start the logging daemon that periodically logs the memory usage information
+			if (LOG.isInfoEnabled() && configuration.getBoolean(
+				ConfigConstants.TASK_MANAGER_DEBUG_MEMORY_USAGE_START_LOG_THREAD,
+				ConfigConstants.DEFAULT_TASK_MANAGER_DEBUG_MEMORY_USAGE_START_LOG_THREAD)) {
+				LOG.info("Starting periodic memory usage logger");
+
+				long interval = configuration.getLong(
+					ConfigConstants.TASK_MANAGER_DEBUG_MEMORY_USAGE_LOG_INTERVAL_MS,
+					ConfigConstants.DEFAULT_TASK_MANAGER_DEBUG_MEMORY_USAGE_LOG_INTERVAL_MS);
+
+				MemoryLogger logger = new MemoryLogger(LOG, interval, taskManagerSystem);
+				logger.start();
+			}
+
+			// block until everything is done
+			taskManagerSystem.awaitTermination();
+		} catch (Throwable t) {
+			LOG.error("Error while starting up taskManager", t);
+			try {
+				taskManagerSystem.shutdown();
+			} catch (Throwable tt) {
+				LOG.warn("Could not cleanly shut down actor system", tt);
+			}
+			throw t;
+		}
+	}
+
+	// --------------------------------------------------------------------------
+	//  Starting and running the TaskManager
+	// --------------------------------------------------------------------------
+
+	/**
+	 * @param configuration                 The configuration for the TaskManager.
+	 * @param resourceID                    The id of the resource which the task manager will run on.
+	 * @param rpcService                  The rpc service which is used to start and connect to the TaskManager RpcEndpoint .
+	 * @param taskManagerHostname       The hostname/address that describes the TaskManager's data location.
+	 * @param haServices        Optionally, a high availability service can be provided. If none is given,
+	 *                                      then a HighAvailabilityServices is constructed from the configuration.
+	 * @param localTaskManagerCommunication     If true, the TaskManager will not initiate the TCP network stack.
+	 * @return An ActorRef to the TaskManager actor.
+	 * @throws org.apache.flink.configuration.IllegalConfigurationException     Thrown, if the given config contains illegal values.
+	 * @throws java.io.IOException      Thrown, if any of the I/O components (such as buffer pools,
+	 *                                       I/O manager, ...) cannot be properly started.
+	 * @throws java.lang.Exception      Thrown is some other error occurs while parsing the configuration
+	 *                                      or starting the TaskManager components.
+	 */
+	public static TaskExecutor startTaskManagerComponentsAndActor(
+		Configuration configuration,
+		ResourceID resourceID,
+		RpcService rpcService,
+		String taskManagerHostname,
+		HighAvailabilityServices haServices,
+		boolean localTaskManagerCommunication) throws Exception {
+
+		final TaskExecutorConfiguration taskExecutorConfig = parseTaskManagerConfiguration(
+			configuration, taskManagerHostname, localTaskManagerCommunication);
+
+		MemoryType memType = taskExecutorConfig.getNetworkConfig().memoryType();
+
+		// pre-start checks
+		checkTempDirs(taskExecutorConfig.getTmpDirPaths());
+
+		ExecutionContext executionContext = ExecutionContexts$.MODULE$.fromExecutor(new ForkJoinPool());
+
+		// we start the network first, to make sure it can allocate its buffers first
+		final NetworkEnvironment network = new NetworkEnvironment(
+			executionContext,
+			taskExecutorConfig.getTimeout(),
+			taskExecutorConfig.getNetworkConfig(),
+			taskExecutorConfig.getConnectionInfo());
+
+		// computing the amount of memory to use depends on how much memory is available
+		// it strictly needs to happen AFTER the network stack has been initialized
+
+		// check if a value has been configured
+		long configuredMemory = configuration.getLong(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, -1L);
+		checkConfigParameter(configuredMemory == -1 || configuredMemory > 0, configuredMemory,
+			ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY,
+			"MemoryManager needs at least one MB of memory. " +
+				"If you leave this config parameter empty, the system automatically " +
+				"pick a fraction of the available memory.");
+
+		final long memorySize;
+		boolean preAllocateMemory = configuration.getBoolean(
+			ConfigConstants.TASK_MANAGER_MEMORY_PRE_ALLOCATE_KEY,
+			ConfigConstants.DEFAULT_TASK_MANAGER_MEMORY_PRE_ALLOCATE);
+		if (configuredMemory > 0) {
+			if (preAllocateMemory) {
+				LOG.info("Using {} MB for managed memory." , configuredMemory);
+			} else {
+				LOG.info("Limiting managed memory to {} MB, memory will be allocated lazily." , configuredMemory);
+			}
+			memorySize = configuredMemory << 20; // megabytes to bytes
+		} else {
+			float fraction = configuration.getFloat(
+				ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY,
+				ConfigConstants.DEFAULT_MEMORY_MANAGER_MEMORY_FRACTION);
+			checkConfigParameter(fraction > 0.0f && fraction < 1.0f, fraction,
+				ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY,
+				"MemoryManager fraction of the free memory must be between 0.0 and 1.0");
+
+			if (memType == MemoryType.HEAP) {
+				long relativeMemSize = (long) (EnvironmentInformation.getSizeOfFreeHeapMemoryWithDefrag() * fraction);
+				if (preAllocateMemory) {
+					LOG.info("Using {} of the currently free heap space for managed heap memory ({} MB)." ,
+						fraction , relativeMemSize >> 20);
+				} else {
+					LOG.info("Limiting managed memory to {} of the currently free heap space ({} MB), " +
+						"memory will be allocated lazily." , fraction , relativeMemSize >> 20);
+				}
+				memorySize = relativeMemSize;
+			} else if (memType == MemoryType.OFF_HEAP) {
+				// The maximum heap memory has been adjusted according to the fraction
+				long maxMemory = EnvironmentInformation.getMaxJvmHeapMemory();
+				long directMemorySize = (long) (maxMemory / (1.0 - fraction) * fraction);
+				if (preAllocateMemory) {
+					LOG.info("Using {} of the maximum memory size for managed off-heap memory ({} MB)." ,
+						fraction, directMemorySize >> 20);
+				} else {
+					LOG.info("Limiting managed memory to {} of the maximum memory size ({} MB)," +
+						" memory will be allocated lazily.", fraction, directMemorySize >> 20);
+				}
+				memorySize = directMemorySize;
+			} else {
+				throw new RuntimeException("No supported memory type detected.");
+			}
+		}
+
+		// now start the memory manager
+		final MemoryManager memoryManager;
+		try {
+			memoryManager = new MemoryManager(
+				memorySize,
+				taskExecutorConfig.getNumberOfSlots(),
+				taskExecutorConfig.getNetworkConfig().networkBufferSize(),
+				memType,
+				preAllocateMemory);
+		} catch (OutOfMemoryError e) {
+			if (memType == MemoryType.HEAP) {
+				throw new Exception("OutOfMemory error (" + e.getMessage() +
+					") while allocating the TaskManager heap memory (" + memorySize + " bytes).", e);
+			} else if (memType == MemoryType.OFF_HEAP) {
+				throw new Exception("OutOfMemory error (" + e.getMessage() +
+					") while allocating the TaskManager off-heap memory (" + memorySize +
+					" bytes).Try increasing the maximum direct memory (-XX:MaxDirectMemorySize)", e);
+			} else {
+				throw e;
+			}
+		}
+
+		// start the I/O manager, it will create some temp directories.
+		final IOManager ioManager = new IOManagerAsync(taskExecutorConfig.getTmpDirPaths());
+
+		final TaskExecutor taskExecutor = new TaskExecutor(
+			taskExecutorConfig,
+			resourceID,
+			memoryManager,
+			ioManager,
+			network,
+			taskExecutorConfig.getNumberOfSlots(),
+			rpcService,
+			haServices);
+
+		return taskExecutor;
+	}
+
+	// --------------------------------------------------------------------------
+	//  Parsing and checking the TaskManager Configuration
+	// --------------------------------------------------------------------------
+
+	/**
+	 * Utility method to extract TaskManager config parameters from the configuration and to
+	 * sanity check them.
+	 *
+	 * @param configuration                 The configuration.
+	 * @param taskManagerHostname           The host name under which the TaskManager communicates.
+	 * @param localTaskManagerCommunication             True, to skip initializing the network stack.
+	 *                                      Use only in cases where only one task manager runs.
+	 * @return TaskExecutorConfiguration that wrappers InstanceConnectionInfo, NetworkEnvironmentConfiguration, etc.
+	 */
+	private static TaskExecutorConfiguration parseTaskManagerConfiguration(
+		Configuration configuration,
+		String taskManagerHostname,
+		boolean localTaskManagerCommunication) throws Exception {
+
+		// ------- read values from the config and check them ---------
+		//                      (a lot of them)
+
+		// ----> hosts / ports for communication and data exchange
+
+		int dataport = configuration.getInteger(ConfigConstants.TASK_MANAGER_DATA_PORT_KEY,
+			ConfigConstants.DEFAULT_TASK_MANAGER_DATA_PORT);
+		if (dataport == 0) {
+			dataport = NetUtils.getAvailablePort();
+		}
+		checkConfigParameter(dataport > 0, dataport, ConfigConstants.TASK_MANAGER_DATA_PORT_KEY,
+			"Leave config parameter empty or use 0 to let the system choose a port automatically.");
+
+		InetAddress taskManagerAddress = InetAddress.getByName(taskManagerHostname);
+		final InstanceConnectionInfo connectionInfo = new InstanceConnectionInfo(taskManagerAddress, dataport);
+
+		// ----> memory / network stack (shuffles/broadcasts), task slots, temp directories
+
+		// we need this because many configs have been written with a "-1" entry
+		int slots = configuration.getInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 1);
+		if (slots == -1) {
+			slots = 1;
+		}
+		checkConfigParameter(slots >= 1, slots, ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS,
+			"Number of task slots must be at least one.");
+
+		final int numNetworkBuffers = configuration.getInteger(
+			ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY,
+			ConfigConstants.DEFAULT_TASK_MANAGER_NETWORK_NUM_BUFFERS);
+		checkConfigParameter(numNetworkBuffers > 0, numNetworkBuffers,
+			ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY, "");
+
+		final int pageSize = configuration.getInteger(
+			ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY,
+			ConfigConstants.DEFAULT_TASK_MANAGER_MEMORY_SEGMENT_SIZE);
+		// check page size of for minimum size
+		checkConfigParameter(pageSize >= MemoryManager.MIN_PAGE_SIZE, pageSize,
+			ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY,
+			"Minimum memory segment size is " + MemoryManager.MIN_PAGE_SIZE);
+		// check page size for power of two
+		checkConfigParameter(MathUtils.isPowerOf2(pageSize), pageSize,
+			ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY,
+			"Memory segment size must be a power of 2.");
+
+		// check whether we use heap or off-heap memory
+		final MemoryType memType;
+		if (configuration.getBoolean(ConfigConstants.TASK_MANAGER_MEMORY_OFF_HEAP_KEY, false)) {
+			memType = MemoryType.OFF_HEAP;
+		} else {
+			memType = MemoryType.HEAP;
+		}
+
+		// initialize the memory segment factory accordingly
+		if (memType == MemoryType.HEAP) {
+			if (!MemorySegmentFactory.initializeIfNotInitialized(HeapMemorySegment.FACTORY)) {
+				throw new Exception("Memory type is set to heap memory, but memory segment " +
+					"factory has been initialized for off-heap memory segments");
+			}
+		} else {
+			if (!MemorySegmentFactory.initializeIfNotInitialized(HybridMemorySegment.FACTORY)) {
+				throw new Exception("Memory type is set to off-heap memory, but memory segment " +
+					"factory has been initialized for heap memory segments");
+			}
+		}
+
+		final String[] tmpDirs = configuration.getString(
+			ConfigConstants.TASK_MANAGER_TMP_DIR_KEY,
+			ConfigConstants.DEFAULT_TASK_MANAGER_TMP_PATH).split(",|" + File.pathSeparator);
+
+		final NettyConfig nettyConfig;
+		if (!localTaskManagerCommunication) {
+			nettyConfig = new NettyConfig(connectionInfo.address(), connectionInfo.dataPort(), pageSize, slots, configuration);
+		} else {
+			nettyConfig = null;
+		}
+
+		// Default spill I/O mode for intermediate results
+		final String syncOrAsync = configuration.getString(
+			ConfigConstants.TASK_MANAGER_NETWORK_DEFAULT_IO_MODE,
+			ConfigConstants.DEFAULT_TASK_MANAGER_NETWORK_DEFAULT_IO_MODE);
+
+		final IOMode ioMode;
+		if (syncOrAsync.equals("async")) {
+			ioMode = IOManager.IOMode.ASYNC;
+		} else {
+			ioMode = IOManager.IOMode.SYNC;
+		}
+
+		final int queryServerPort =  configuration.getInteger(
+			ConfigConstants.QUERYABLE_STATE_SERVER_PORT,
+			ConfigConstants.DEFAULT_QUERYABLE_STATE_SERVER_PORT);
+
+		final int queryServerNetworkThreads =  configuration.getInteger(
+			ConfigConstants.QUERYABLE_STATE_SERVER_NETWORK_THREADS,
+			ConfigConstants.DEFAULT_QUERYABLE_STATE_SERVER_NETWORK_THREADS);
+
+		final int queryServerQueryThreads =  configuration.getInteger(
+			ConfigConstants.QUERYABLE_STATE_SERVER_QUERY_THREADS,
+			ConfigConstants.DEFAULT_QUERYABLE_STATE_SERVER_QUERY_THREADS);
+
+		final NetworkEnvironmentConfiguration networkConfig = new NetworkEnvironmentConfiguration(
+			numNetworkBuffers,
+			pageSize,
+			memType,
+			ioMode,
+			queryServerPort,
+			queryServerNetworkThreads,
+			queryServerQueryThreads,
+			localTaskManagerCommunication ? Option.<NettyConfig>empty() : new Some<>(nettyConfig),
+			new Tuple2<>(500, 3000));
+
+		// ----> timeouts, library caching, profiling
+
+		final FiniteDuration timeout;
+		try {
+			timeout = AkkaUtils.getTimeout(configuration);
+		} catch (Exception e) {
+			throw new IllegalArgumentException(
+				"Invalid format for '" + ConfigConstants.AKKA_ASK_TIMEOUT +
+					"'.Use formats like '50 s' or '1 min' to specify the timeout.");
+		}
+		LOG.info("Messages between TaskManager and JobManager have a max timeout of " + timeout);
+
+		final long cleanupInterval = configuration.getLong(
+			ConfigConstants.LIBRARY_CACHE_MANAGER_CLEANUP_INTERVAL,
+			ConfigConstants.DEFAULT_LIBRARY_CACHE_MANAGER_CLEANUP_INTERVAL) * 1000;
+
+		final FiniteDuration finiteRegistrationDuration;
+		try {
+			Duration maxRegistrationDuration = Duration.create(configuration.getString(
+				ConfigConstants.TASK_MANAGER_MAX_REGISTRATION_DURATION,
+				ConfigConstants.DEFAULT_TASK_MANAGER_MAX_REGISTRATION_DURATION));
+			if (maxRegistrationDuration.isFinite()) {
+				finiteRegistrationDuration = new FiniteDuration(maxRegistrationDuration.toSeconds(), TimeUnit.SECONDS);
+			} else {
+				finiteRegistrationDuration = null;
+			}
+		} catch (NumberFormatException e) {
+			throw new IllegalArgumentException("Invalid format for parameter " +
+				ConfigConstants.TASK_MANAGER_MAX_REGISTRATION_DURATION, e);
+		}
+
+		final FiniteDuration initialRegistrationPause;
+		try {
+			Duration pause = Duration.create(configuration.getString(
+				ConfigConstants.TASK_MANAGER_INITIAL_REGISTRATION_PAUSE,
+				ConfigConstants.DEFAULT_TASK_MANAGER_INITIAL_REGISTRATION_PAUSE));
+			if (pause.isFinite()) {
+				initialRegistrationPause = new FiniteDuration(pause.toSeconds(), TimeUnit.SECONDS);
+			} else {
+				throw new IllegalArgumentException("The initial registration pause must be finite: " + pause);
+			}
+		} catch (NumberFormatException e) {
+			throw new IllegalArgumentException("Invalid format for parameter " +
+				ConfigConstants.TASK_MANAGER_INITIAL_REGISTRATION_PAUSE, e);
+		}
+
+		final FiniteDuration maxRegistrationPause;
+		try {
+			Duration pause = Duration.create(configuration.getString(
+				ConfigConstants.TASK_MANAGER_MAX_REGISTARTION_PAUSE,
+				ConfigConstants.DEFAULT_TASK_MANAGER_MAX_REGISTRATION_PAUSE));
+			if (pause.isFinite()) {
+				maxRegistrationPause = new FiniteDuration(pause.toSeconds(), TimeUnit.SECONDS);
+			} else {
+				throw new IllegalArgumentException("The maximum registration pause must be finite: " + pause);
+			}
+		} catch (NumberFormatException e) {
+			throw new IllegalArgumentException("Invalid format for parameter " +
+				ConfigConstants.TASK_MANAGER_INITIAL_REGISTRATION_PAUSE, e);
+		}
+
+		final FiniteDuration refusedRegistrationPause;
+		try {
+			Duration pause = Duration.create(configuration.getString(
+				ConfigConstants.TASK_MANAGER_REFUSED_REGISTRATION_PAUSE,
+				ConfigConstants.DEFAULT_TASK_MANAGER_REFUSED_REGISTRATION_PAUSE));
+			if (pause.isFinite()) {
+				refusedRegistrationPause = new FiniteDuration(pause.toSeconds(), TimeUnit.SECONDS);
+			} else {
+				throw new IllegalArgumentException("The refused registration pause must be finite: " + pause);
+			}
+		} catch (NumberFormatException e) {
+			throw new IllegalArgumentException("Invalid format for parameter " +
+				ConfigConstants.TASK_MANAGER_INITIAL_REGISTRATION_PAUSE, e);
+		}
+
+		return new TaskExecutorConfiguration(
+			tmpDirs,
+			cleanupInterval,
+			connectionInfo,
+			networkConfig,
+			timeout,
+			finiteRegistrationDuration,
+			slots,
+			configuration,
+			initialRegistrationPause,
+			maxRegistrationPause,
+			refusedRegistrationPause);
+	}
+
+	/**
+	 * Validates a condition for a config parameter and displays a standard exception, if the
+	 * the condition does not hold.
+	 *
+	 * @param condition    The condition that must hold. If the condition is false, an exception is thrown.
+	 * @param parameter    The parameter value. Will be shown in the exception message.
+	 * @param name         The name of the config parameter. Will be shown in the exception message.
+	 * @param errorMessage The optional custom error message to append to the exception message.
+	 */
+	private static void checkConfigParameter(
+		boolean condition,
+		Object parameter,
+		String name,
+		String errorMessage) {
+		if (!condition) {
+			throw new IllegalConfigurationException("Invalid configuration value for " + name + " : " + parameter + " - " + errorMessage);
+		}
+	}
+
+	/**
+	 * Validates that all the directories denoted by the strings do actually exist, are proper
+	 * directories (not files), and are writable.
+	 *
+	 * @param tmpDirs The array of directory paths to check.
+	 * @throws Exception Thrown if any of the directories does not exist or is not writable
+	 *                   or is a file, rather than a directory.
+	 */
+	private static void checkTempDirs(String[] tmpDirs) throws IOException {
+		for (String dir : tmpDirs) {
+			if (dir != null && !dir.equals("")) {
+				File file = new File(dir);
+				if (!file.exists()) {
+					throw new IOException("Temporary file directory " + file.getAbsolutePath() + " does not exist.");
+				}
+				if (!file.isDirectory()) {
+					throw new IOException("Temporary file directory " + file.getAbsolutePath() + " is not a directory.");
+				}
+				if (!file.canWrite()) {
+					throw new IOException("Temporary file directory " + file.getAbsolutePath() + " is not writable.");
+				}
+
+				if (LOG.isInfoEnabled()) {
+					long totalSpaceGb = file.getTotalSpace() >> 30;
+					long usableSpaceGb = file.getUsableSpace() >> 30;
+					double usablePercentage = (double)usableSpaceGb / totalSpaceGb * 100;
+					String path = file.getAbsolutePath();
+					LOG.info(String.format("Temporary file directory '%s': total %d GB, " + "usable %d GB (%.2f%% usable)",
+						path, totalSpaceGb, usableSpaceGb, usablePercentage));
+				}
+			} else {
+				throw new IllegalArgumentException("Temporary file directory #$id is null.");
+			}
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  Properties
+	// ------------------------------------------------------------------------
+
+	public ResourceID getResourceID() {
+		return resourceID;
+	}
+
+	// ------------------------------------------------------------------------
+	//  Error Handling
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Notifies the TaskExecutor that a fatal error has occurred and it cannot proceed.
+	 * This method should be used when asynchronous threads want to notify the
+	 * TaskExecutor 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 TaskExecutor that a fatal error has occurred and it cannot proceed.
+	 * This method must only be called from within the TaskExecutor's main thread.
+	 *
+	 * @param t The exception describing the fatal error
+	 */
+	void onFatalError(Throwable t) {
+		// to be determined, probably delegate to a fatal error handler that 
+		// would either log (mini cluster) ot kill the process (yarn, mesos, ...)
+		log.error("FATAL ERROR", t);
+	}
+
+	// ------------------------------------------------------------------------
+	//  Access to fields for testing
+	// ------------------------------------------------------------------------
+
+	@VisibleForTesting
+	TaskExecutorToResourceManagerConnection getResourceManagerConnection() {
+		return resourceManagerConnection;
+	}
+
+	// ------------------------------------------------------------------------
+	//  Utility classes
+	// ------------------------------------------------------------------------
+
+	/**
+	 * The listener for leader changes of the resource manager
+	 */
+	private class ResourceManagerLeaderListener implements LeaderRetrievalListener {
+
+		@Override
+		public void notifyLeaderAddress(String leaderAddress, UUID leaderSessionID) {
+			getSelf().notifyOfNewResourceManagerLeader(leaderAddress, leaderSessionID);
+		}
+
+		@Override
+		public void handleError(Exception exception) {
+			onFatalErrorAsync(exception);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9c247d1f/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorConfiguration.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorConfiguration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorConfiguration.java
new file mode 100644
index 0000000..3707a47
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorConfiguration.java
@@ -0,0 +1,151 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.taskexecutor;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.instance.InstanceConnectionInfo;
+import org.apache.flink.runtime.taskmanager.NetworkEnvironmentConfiguration;
+
+import scala.concurrent.duration.FiniteDuration;
+
+import java.io.Serializable;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * {@link TaskExecutor} Configuration
+ */
+public class TaskExecutorConfiguration implements Serializable {
+
+	private static final long serialVersionUID = 1L;
+
+	private final String[] tmpDirPaths;
+
+	private final long cleanupInterval;
+
+	private final int numberOfSlots;
+
+	private final Configuration configuration;
+
+	private final FiniteDuration timeout;
+	private final FiniteDuration maxRegistrationDuration;
+	private final FiniteDuration initialRegistrationPause;
+	private final FiniteDuration maxRegistrationPause;
+	private final FiniteDuration refusedRegistrationPause;
+
+	private final NetworkEnvironmentConfiguration networkConfig;
+
+	private final InstanceConnectionInfo connectionInfo;
+
+	public TaskExecutorConfiguration(
+			String[] tmpDirPaths,
+			long cleanupInterval,
+			InstanceConnectionInfo connectionInfo,
+			NetworkEnvironmentConfiguration networkConfig,
+			FiniteDuration timeout,
+			FiniteDuration maxRegistrationDuration,
+			int numberOfSlots,
+			Configuration configuration) {
+
+		this (tmpDirPaths,
+			cleanupInterval,
+			connectionInfo,
+			networkConfig,
+			timeout,
+			maxRegistrationDuration,
+			numberOfSlots,
+			configuration,
+			new FiniteDuration(500, TimeUnit.MILLISECONDS),
+			new FiniteDuration(30, TimeUnit.SECONDS),
+			new FiniteDuration(10, TimeUnit.SECONDS));
+	}
+
+	public TaskExecutorConfiguration(
+			String[] tmpDirPaths,
+			long cleanupInterval,
+			InstanceConnectionInfo connectionInfo,
+			NetworkEnvironmentConfiguration networkConfig,
+			FiniteDuration timeout,
+			FiniteDuration maxRegistrationDuration,
+			int numberOfSlots,
+			Configuration configuration,
+			FiniteDuration initialRegistrationPause,
+			FiniteDuration maxRegistrationPause,
+			FiniteDuration refusedRegistrationPause) {
+
+		this.tmpDirPaths = checkNotNull(tmpDirPaths);
+		this.cleanupInterval = checkNotNull(cleanupInterval);
+		this.connectionInfo = checkNotNull(connectionInfo);
+		this.networkConfig = checkNotNull(networkConfig);
+		this.timeout = checkNotNull(timeout);
+		this.maxRegistrationDuration = maxRegistrationDuration;
+		this.numberOfSlots = checkNotNull(numberOfSlots);
+		this.configuration = checkNotNull(configuration);
+		this.initialRegistrationPause = checkNotNull(initialRegistrationPause);
+		this.maxRegistrationPause = checkNotNull(maxRegistrationPause);
+		this.refusedRegistrationPause = checkNotNull(refusedRegistrationPause);
+	}
+
+	// --------------------------------------------------------------------------------------------
+	//  Properties
+	// --------------------------------------------------------------------------------------------
+
+	public String[] getTmpDirPaths() {
+		return tmpDirPaths;
+	}
+
+	public long getCleanupInterval() {
+		return cleanupInterval;
+	}
+
+	public InstanceConnectionInfo getConnectionInfo() { return connectionInfo; }
+
+	public NetworkEnvironmentConfiguration getNetworkConfig() { return networkConfig; }
+
+	public FiniteDuration getTimeout() {
+		return timeout;
+	}
+
+	public FiniteDuration getMaxRegistrationDuration() {
+		return maxRegistrationDuration;
+	}
+
+	public int getNumberOfSlots() {
+		return numberOfSlots;
+	}
+
+	public Configuration getConfiguration() {
+		return configuration;
+	}
+
+	public FiniteDuration getInitialRegistrationPause() {
+		return initialRegistrationPause;
+	}
+
+	public FiniteDuration getMaxRegistrationPause() {
+		return maxRegistrationPause;
+	}
+
+	public FiniteDuration getRefusedRegistrationPause() {
+		return refusedRegistrationPause;
+	}
+
+}
+

http://git-wip-us.apache.org/repos/asf/flink/blob/9c247d1f/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
new file mode 100644
index 0000000..6c99706
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorGateway.java
@@ -0,0 +1,35 @@
+/*
+ * 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.runtime.rpc.RpcGateway;
+
+import java.util.UUID;
+
+/**
+ * {@link TaskExecutor} RPC gateway interface
+ */
+public interface TaskExecutorGateway extends RpcGateway {
+
+	// ------------------------------------------------------------------------
+	//  ResourceManager handlers
+	// ------------------------------------------------------------------------
+
+	void notifyOfNewResourceManagerLeader(String address, UUID resourceManagerLeaderId);
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9c247d1f/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorRegistrationSuccess.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorRegistrationSuccess.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorRegistrationSuccess.java
new file mode 100644
index 0000000..b357f52
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorRegistrationSuccess.java
@@ -0,0 +1,75 @@
+/*
+ * 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.runtime.instance.InstanceID;
+import org.apache.flink.runtime.registration.RegistrationResponse;
+
+import java.io.Serializable;
+
+/**
+ * Base class for responses from the ResourceManager to a registration attempt by a
+ * TaskExecutor.
+ */
+public final class TaskExecutorRegistrationSuccess extends RegistrationResponse.Success implements Serializable {
+
+	private static final long serialVersionUID = 1L;
+
+	private final InstanceID registrationId;
+
+	private final long heartbeatInterval;
+
+	/**
+	 * Create a new {@code TaskExecutorRegistrationSuccess} message.
+	 * 
+	 * @param registrationId     The ID that the ResourceManager assigned the registration.
+	 * @param heartbeatInterval  The interval in which the ResourceManager will heartbeat the TaskExecutor.
+	 */
+	public TaskExecutorRegistrationSuccess(InstanceID registrationId, long heartbeatInterval) {
+		this.registrationId = registrationId;
+		this.heartbeatInterval = heartbeatInterval;
+	}
+
+	/**
+	 * Gets the ID that the ResourceManager assigned the registration.
+	 */
+	public InstanceID getRegistrationId() {
+		return registrationId;
+	}
+
+	/**
+	 * Gets the interval in which the ResourceManager will heartbeat the TaskExecutor.
+	 */
+	public long getHeartbeatInterval() {
+		return heartbeatInterval;
+	}
+
+	@Override
+	public String toString() {
+		return "TaskExecutorRegistrationSuccess (" + registrationId + " / " + heartbeatInterval + ')';
+	}
+
+}
+
+
+
+
+
+
+

http://git-wip-us.apache.org/repos/asf/flink/blob/9c247d1f/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
new file mode 100644
index 0000000..25332a0
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorToResourceManagerConnection.java
@@ -0,0 +1,198 @@
+/*
+ * 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 akka.dispatch.OnFailure;
+import akka.dispatch.OnSuccess;
+
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.instance.InstanceID;
+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.slf4j.Logger;
+
+import scala.concurrent.Future;
+import scala.concurrent.duration.FiniteDuration;
+
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * The connection between a TaskExecutor and the ResourceManager.
+ */
+public class TaskExecutorToResourceManagerConnection {
+
+	/** the logger for all log messages of this class */
+	private final Logger log;
+
+	/** the TaskExecutor whose connection to the ResourceManager this represents */
+	private final TaskExecutor taskExecutor;
+
+	private final UUID resourceManagerLeaderId;
+
+	private final String resourceManagerAddress;
+
+	private TaskExecutorToResourceManagerConnection.ResourceManagerRegistration pendingRegistration;
+
+	private ResourceManagerGateway registeredResourceManager;
+
+	private InstanceID registrationId;
+
+	/** flag indicating that the connection is closed */
+	private volatile boolean closed;
+
+
+	public TaskExecutorToResourceManagerConnection(
+			Logger log,
+			TaskExecutor taskExecutor,
+			String resourceManagerAddress,
+			UUID resourceManagerLeaderId) {
+
+		this.log = checkNotNull(log);
+		this.taskExecutor = checkNotNull(taskExecutor);
+		this.resourceManagerAddress = checkNotNull(resourceManagerAddress);
+		this.resourceManagerLeaderId = checkNotNull(resourceManagerLeaderId);
+	}
+
+	// ------------------------------------------------------------------------
+	//  Life cycle
+	// ------------------------------------------------------------------------
+
+	@SuppressWarnings("unchecked")
+	public void start() {
+		checkState(!closed, "The connection is already closed");
+		checkState(!isRegistered() && pendingRegistration == null, "The connection is already started");
+
+		pendingRegistration = new TaskExecutorToResourceManagerConnection.ResourceManagerRegistration(
+				log, taskExecutor.getRpcService(),
+				resourceManagerAddress, resourceManagerLeaderId,
+				taskExecutor.getAddress(), taskExecutor.getResourceID());
+		pendingRegistration.startRegistration();
+
+		Future<Tuple2<ResourceManagerGateway, TaskExecutorRegistrationSuccess>> future = pendingRegistration.getFuture();
+		
+		future.onSuccess(new OnSuccess<Tuple2<ResourceManagerGateway, TaskExecutorRegistrationSuccess>>() {
+			@Override
+			public void onSuccess(Tuple2<ResourceManagerGateway, TaskExecutorRegistrationSuccess> result) {
+				registeredResourceManager = result.f0;
+				registrationId = result.f1.getRegistrationId();
+			}
+		}, taskExecutor.getMainThreadExecutionContext());
+		
+		// this future should only ever fail if there is a bug, not if the registration is declined
+		future.onFailure(new OnFailure() {
+			@Override
+			public void onFailure(Throwable failure) {
+				taskExecutor.onFatalError(failure);
+			}
+		}, taskExecutor.getMainThreadExecutionContext());
+	}
+
+	public void close() {
+		closed = true;
+
+		// make sure we do not keep re-trying forever
+		if (pendingRegistration != null) {
+			pendingRegistration.cancel();
+		}
+	}
+
+	public boolean isClosed() {
+		return closed;
+	}
+
+	// ------------------------------------------------------------------------
+	//  Properties
+	// ------------------------------------------------------------------------
+
+	public UUID getResourceManagerLeaderId() {
+		return resourceManagerLeaderId;
+	}
+
+	public String getResourceManagerAddress() {
+		return resourceManagerAddress;
+	}
+
+	/**
+	 * Gets the ResourceManagerGateway. This returns null until the registration is completed.
+	 */
+	public ResourceManagerGateway getResourceManager() {
+		return registeredResourceManager;
+	}
+
+	/**
+	 * Gets the ID under which the TaskExecutor is registered at the ResourceManager.
+	 * This returns null until the registration is completed.
+	 */
+	public InstanceID getRegistrationId() {
+		return registrationId;
+	}
+
+	public boolean isRegistered() {
+		return registeredResourceManager != null;
+	}
+
+	// ------------------------------------------------------------------------
+
+	@Override
+	public String toString() {
+		return String.format("Connection to ResourceManager %s (leaderId=%s)",
+				resourceManagerAddress, resourceManagerLeaderId); 
+	}
+
+	// ------------------------------------------------------------------------
+	//  Utilities
+	// ------------------------------------------------------------------------
+
+	private static class ResourceManagerRegistration
+			extends RetryingRegistration<ResourceManagerGateway, TaskExecutorRegistrationSuccess> {
+
+		private final String taskExecutorAddress;
+		
+		private final ResourceID resourceID;
+
+		ResourceManagerRegistration(
+				Logger log,
+				RpcService rpcService,
+				String targetAddress,
+				UUID leaderId,
+				String taskExecutorAddress,
+				ResourceID resourceID) {
+
+			super(log, rpcService, "ResourceManager", ResourceManagerGateway.class, targetAddress, leaderId);
+			this.taskExecutorAddress = checkNotNull(taskExecutorAddress);
+			this.resourceID = checkNotNull(resourceID);
+		}
+
+		@Override
+		protected Future<RegistrationResponse> invokeRegistration(
+				ResourceManagerGateway resourceManager, UUID leaderId, long timeoutMillis) throws Exception {
+
+			FiniteDuration timeout = new FiniteDuration(timeoutMillis, TimeUnit.MILLISECONDS);
+			return resourceManager.registerTaskExecutor(leaderId, taskExecutorAddress, resourceID, timeout);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9c247d1f/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ClusterShutdownITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ClusterShutdownITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ClusterShutdownITCase.java
new file mode 100644
index 0000000..744308c
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ClusterShutdownITCase.java
@@ -0,0 +1,156 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.clusterframework;
+
+import akka.actor.ActorSystem;
+import akka.testkit.JavaTestKit;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.akka.AkkaUtils;
+import org.apache.flink.runtime.clusterframework.ApplicationStatus;
+import org.apache.flink.runtime.clusterframework.messages.StopCluster;
+import org.apache.flink.runtime.clusterframework.messages.StopClusterSuccessful;
+import org.apache.flink.runtime.instance.ActorGateway;
+import org.apache.flink.runtime.messages.Messages;
+import org.apache.flink.runtime.testingUtils.TestingMessages;
+import org.apache.flink.runtime.testingUtils.TestingUtils;
+import org.apache.flink.runtime.testutils.TestingResourceManager;
+import org.apache.flink.util.TestLogger;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import scala.Option;
+
+
+/**
+ * Runs tests to ensure that a cluster is shutdown properly.
+ */
+public class ClusterShutdownITCase extends TestLogger {
+
+	private static ActorSystem system;
+
+	private static Configuration config = new Configuration();
+
+	@BeforeClass
+	public static void setup() {
+		system = AkkaUtils.createActorSystem(AkkaUtils.getDefaultAkkaConfig());
+	}
+
+	@AfterClass
+	public static void teardown() {
+		JavaTestKit.shutdownActorSystem(system);
+	}
+
+	/**
+	 * Tests a faked cluster shutdown procedure without the ResourceManager.
+	 */
+	@Test
+	public void testClusterShutdownWithoutResourceManager() {
+
+		new JavaTestKit(system){{
+		new Within(duration("30 seconds")) {
+		@Override
+		protected void run() {
+
+			ActorGateway me =
+				TestingUtils.createForwardingActor(system, getTestActor(), Option.<String>empty());
+
+			// start job manager which doesn't shutdown the actor system
+			ActorGateway jobManager =
+				TestingUtils.createJobManager(system, config, "jobmanager1");
+
+			// Tell the JobManager to inform us of shutdown actions
+			jobManager.tell(TestingMessages.getNotifyOfComponentShutdown(), me);
+
+			// Register a TaskManager
+			ActorGateway taskManager =
+				TestingUtils.createTaskManager(system, jobManager, config, true, true);
+
+			// Tell the TaskManager to inform us of TaskManager shutdowns
+			taskManager.tell(TestingMessages.getNotifyOfComponentShutdown(), me);
+
+
+			// No resource manager connected
+			jobManager.tell(new StopCluster(ApplicationStatus.SUCCEEDED, "Shutting down."), me);
+
+			expectMsgAllOf(
+				new TestingMessages.ComponentShutdown(taskManager.actor()),
+				new TestingMessages.ComponentShutdown(jobManager.actor()),
+				StopClusterSuccessful.getInstance()
+			);
+
+		}};
+		}};
+	}
+
+	/**
+	 * Tests a faked cluster shutdown procedure with the ResourceManager.
+	 */
+	@Test
+	public void testClusterShutdownWithResourceManager() {
+
+		new JavaTestKit(system){{
+		new Within(duration("30 seconds")) {
+		@Override
+		protected void run() {
+
+			ActorGateway me =
+				TestingUtils.createForwardingActor(system, getTestActor(), Option.<String>empty());
+
+			// start job manager which doesn't shutdown the actor system
+			ActorGateway jobManager =
+				TestingUtils.createJobManager(system, config, "jobmanager2");
+
+			// Tell the JobManager to inform us of shutdown actions
+			jobManager.tell(TestingMessages.getNotifyOfComponentShutdown(), me);
+
+			// Register a TaskManager
+			ActorGateway taskManager =
+				TestingUtils.createTaskManager(system, jobManager, config, true, true);
+
+			// Tell the TaskManager to inform us of TaskManager shutdowns
+			taskManager.tell(TestingMessages.getNotifyOfComponentShutdown(), me);
+
+			// Start resource manager and let it register
+			ActorGateway resourceManager =
+				TestingUtils.createResourceManager(system, jobManager.actor(), config);
+
+			// Tell the ResourceManager to inform us of ResourceManager shutdowns
+			resourceManager.tell(TestingMessages.getNotifyOfComponentShutdown(), me);
+
+			// notify about a resource manager registration at the job manager
+			resourceManager.tell(new TestingResourceManager.NotifyWhenResourceManagerConnected(), me);
+
+			// Wait for resource manager
+			expectMsgEquals(Messages.getAcknowledge());
+
+
+			// Shutdown cluster with resource manager connected
+			jobManager.tell(new StopCluster(ApplicationStatus.SUCCEEDED, "Shutting down."), me);
+
+			expectMsgAllOf(
+				new TestingMessages.ComponentShutdown(taskManager.actor()),
+				new TestingMessages.ComponentShutdown(jobManager.actor()),
+				new TestingMessages.ComponentShutdown(resourceManager.actor()),
+				StopClusterSuccessful.getInstance()
+			);
+
+		}};
+		}};
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9c247d1f/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ResourceManagerITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ResourceManagerITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ResourceManagerITCase.java
new file mode 100644
index 0000000..1565dc3
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ResourceManagerITCase.java
@@ -0,0 +1,162 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.clusterframework;
+
+import akka.actor.ActorSystem;
+import akka.testkit.JavaTestKit;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.akka.AkkaUtils;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.instance.ActorGateway;
+import org.apache.flink.runtime.instance.HardwareDescription;
+import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
+import org.apache.flink.runtime.messages.Messages;
+import org.apache.flink.runtime.messages.RegistrationMessages;
+import org.apache.flink.runtime.testingUtils.TestingUtils;
+import org.apache.flink.runtime.testutils.TestingResourceManager;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import scala.Option;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+/**
+ * It cases which test the interaction of the resource manager with job manager and task managers.
+ * Runs all tests in one Actor system.
+ */
+public class ResourceManagerITCase extends TestLogger {
+
+	private static ActorSystem system;
+
+	private static Configuration config = new Configuration();
+
+	@BeforeClass
+	public static void setup() {
+		system = AkkaUtils.createActorSystem(AkkaUtils.getDefaultAkkaConfig());
+	}
+
+	@AfterClass
+	public static void teardown() {
+		JavaTestKit.shutdownActorSystem(system);
+	}
+
+	/**
+	 * Tests whether the resource manager connects and reconciles existing task managers.
+	 */
+	@Test
+	public void testResourceManagerReconciliation() {
+
+		new JavaTestKit(system){{
+		new Within(duration("10 seconds")) {
+		@Override
+		protected void run() {
+
+			ActorGateway jobManager =
+				TestingUtils.createJobManager(system, config, "ReconciliationTest");
+			ActorGateway me =
+				TestingUtils.createForwardingActor(system, getTestActor(), Option.<String>empty());
+
+			// !! no resource manager started !!
+
+			ResourceID resourceID = ResourceID.generate();
+
+			TaskManagerLocation location = mock(TaskManagerLocation.class);
+			when(location.getResourceID()).thenReturn(resourceID);
+
+			HardwareDescription resourceProfile = HardwareDescription.extractFromSystem(1_000_000);
+
+			jobManager.tell(
+				new RegistrationMessages.RegisterTaskManager(resourceID, location, resourceProfile, 1),
+				me);
+
+			expectMsgClass(RegistrationMessages.AcknowledgeRegistration.class);
+
+			// now start the resource manager
+			ActorGateway resourceManager =
+				TestingUtils.createResourceManager(system, jobManager.actor(), config);
+
+			// register at testing job manager to receive a message once a resource manager registers
+			resourceManager.tell(new TestingResourceManager.NotifyWhenResourceManagerConnected(), me);
+
+			// Wait for resource manager
+			expectMsgEquals(Messages.getAcknowledge());
+
+			// check if we registered the task manager resource
+			resourceManager.tell(new TestingResourceManager.GetRegisteredResources(), me);
+
+			TestingResourceManager.GetRegisteredResourcesReply reply =
+				expectMsgClass(TestingResourceManager.GetRegisteredResourcesReply.class);
+
+			assertEquals(1, reply.resources.size());
+			assertTrue(reply.resources.contains(resourceID));
+
+		}};
+		}};
+	}
+
+	/**
+	 * Tests whether the resource manager gets informed upon TaskManager registration.
+	 */
+	@Test
+	public void testResourceManagerTaskManagerRegistration() {
+
+		new JavaTestKit(system){{
+		new Within(duration("30 seconds")) {
+		@Override
+		protected void run() {
+
+			ActorGateway jobManager =
+				TestingUtils.createJobManager(system, config, "RegTest");
+			ActorGateway me =
+				TestingUtils.createForwardingActor(system, getTestActor(), Option.<String>empty());
+
+			// start the resource manager
+			ActorGateway resourceManager =
+				TestingUtils.createResourceManager(system, jobManager.actor(), config);
+
+			// notify about a resource manager registration at the job manager
+			resourceManager.tell(new TestingResourceManager.NotifyWhenResourceManagerConnected(), me);
+
+			// Wait for resource manager
+			expectMsgEquals(Messages.getAcknowledge());
+
+			// start task manager and wait for registration
+			ActorGateway taskManager =
+				TestingUtils.createTaskManager(system, jobManager.actor(), config, true, true);
+
+			// check if we registered the task manager resource
+			resourceManager.tell(new TestingResourceManager.GetRegisteredResources(), me);
+
+			TestingResourceManager.GetRegisteredResourcesReply reply =
+				expectMsgClass(TestingResourceManager.GetRegisteredResourcesReply.class);
+
+			assertEquals(1, reply.resources.size());
+
+		}};
+		}};
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9c247d1f/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ResourceManagerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ResourceManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ResourceManagerTest.java
new file mode 100644
index 0000000..ca8a07a
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ResourceManagerTest.java
@@ -0,0 +1,338 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.clusterframework;
+
+import akka.actor.ActorSystem;
+import akka.testkit.JavaTestKit;
+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.messages.NotifyResourceStarted;
+import org.apache.flink.runtime.clusterframework.messages.RegisterResourceManager;
+import org.apache.flink.runtime.clusterframework.messages.RegisterResourceManagerSuccessful;
+import org.apache.flink.runtime.clusterframework.messages.RemoveResource;
+import org.apache.flink.runtime.clusterframework.messages.ResourceRemoved;
+import org.apache.flink.runtime.clusterframework.messages.TriggerRegistrationAtJobManager;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.instance.ActorGateway;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.messages.JobManagerMessages;
+import org.apache.flink.runtime.testingUtils.TestingUtils;
+import org.apache.flink.runtime.testutils.TestingResourceManager;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import scala.Option;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import static org.junit.Assert.*;
+
+/**
+ * General tests for the resource manager component.
+ */
+public class ResourceManagerTest {
+
+	private static ActorSystem system;
+
+	private static ActorGateway fakeJobManager;
+	private static ActorGateway resourceManager;
+
+	private static Configuration config = new Configuration();
+
+	@BeforeClass
+	public static void setup() {
+		system = AkkaUtils.createLocalActorSystem(config);
+	}
+
+	@AfterClass
+	public static void teardown() {
+		JavaTestKit.shutdownActorSystem(system);
+	}
+
+	/**
+	 * Tests the registration and reconciliation of the ResourceManager with the JobManager
+	 */
+	@Test
+	public void testJobManagerRegistrationAndReconciliation() {
+		new JavaTestKit(system){{
+		new Within(duration("10 seconds")) {
+		@Override
+		protected void run() {
+			fakeJobManager = TestingUtils.createForwardingActor(system, getTestActor(), Option.<String>empty());
+			resourceManager = TestingUtils.createResourceManager(system, fakeJobManager.actor(), config);
+
+			expectMsgClass(RegisterResourceManager.class);
+
+			List<ResourceID> resourceList = new ArrayList<>();
+			resourceList.add(ResourceID.generate());
+			resourceList.add(ResourceID.generate());
+			resourceList.add(ResourceID.generate());
+
+			resourceManager.tell(
+				new RegisterResourceManagerSuccessful(fakeJobManager.actor(), resourceList),
+				fakeJobManager);
+
+			resourceManager.tell(new TestingResourceManager.GetRegisteredResources(), fakeJobManager);
+			TestingResourceManager.GetRegisteredResourcesReply reply =
+				expectMsgClass(TestingResourceManager.GetRegisteredResourcesReply.class);
+
+			for (ResourceID id : resourceList) {
+				if (!reply.resources.contains(id)) {
+					fail("Expected to find all resources that were provided during registration.");
+				}
+			}
+		}};
+		}};
+	}
+
+	/**
+	 * Tests delayed or erroneous registration of the ResourceManager with the JobManager
+	 */
+	@Test
+	public void testDelayedJobManagerRegistration() {
+		new JavaTestKit(system){{
+		new Within(duration("10 seconds")) {
+		@Override
+		protected void run() {
+
+			// set a short timeout for lookups
+			Configuration shortTimeoutConfig = config.clone();
+			shortTimeoutConfig.setString(ConfigConstants.AKKA_LOOKUP_TIMEOUT, "1 s");
+
+			fakeJobManager = TestingUtils.createForwardingActor(system, getTestActor(), Option.<String>empty());
+			resourceManager = TestingUtils.createResourceManager(system, fakeJobManager.actor(), shortTimeoutConfig);
+
+			// wait for registration message
+			RegisterResourceManager msg = expectMsgClass(RegisterResourceManager.class);
+			// give wrong response
+			getLastSender().tell(new JobManagerMessages.LeaderSessionMessage(null, new Object()),
+				fakeJobManager.actor());
+
+			// expect another retry and let it time out
+			expectMsgClass(RegisterResourceManager.class);
+
+			// wait for next try after timeout
+			expectMsgClass(RegisterResourceManager.class);
+
+		}};
+		}};
+	}
+
+	@Test
+	public void testTriggerReconnect() {
+		new JavaTestKit(system){{
+		new Within(duration("10 seconds")) {
+		@Override
+		protected void run() {
+
+			// set a long timeout for lookups such that the test fails in case of timeouts
+			Configuration shortTimeoutConfig = config.clone();
+			shortTimeoutConfig.setString(ConfigConstants.AKKA_LOOKUP_TIMEOUT, "99999 s");
+
+			fakeJobManager = TestingUtils.createForwardingActor(system, getTestActor(), Option.<String>empty());
+			resourceManager = TestingUtils.createResourceManager(system, fakeJobManager.actor(), shortTimeoutConfig);
+
+			// wait for registration message
+			RegisterResourceManager msg = expectMsgClass(RegisterResourceManager.class);
+			// all went well
+			resourceManager.tell(
+				new RegisterResourceManagerSuccessful(fakeJobManager.actor(), Collections.<ResourceID>emptyList()),
+				fakeJobManager);
+
+			// force a reconnect
+			resourceManager.tell(
+				new TriggerRegistrationAtJobManager(fakeJobManager.actor()),
+				fakeJobManager);
+
+			// new registration attempt should come in
+			expectMsgClass(RegisterResourceManager.class);
+
+		}};
+		}};
+	}
+
+	/**
+	 * Tests the registration and accounting of resources at the ResourceManager.
+	 */
+	@Test
+	public void testTaskManagerRegistration() {
+		new JavaTestKit(system){{
+		new Within(duration("10 seconds")) {
+		@Override
+		protected void run() {
+
+			fakeJobManager = TestingUtils.createForwardingActor(system, getTestActor(), Option.<String>empty());
+			resourceManager = TestingUtils.createResourceManager(system, fakeJobManager.actor(), config);
+
+			// register with JM
+			expectMsgClass(RegisterResourceManager.class);
+			resourceManager.tell(
+				new RegisterResourceManagerSuccessful(fakeJobManager.actor(), Collections.<ResourceID>emptyList()),
+				fakeJobManager);
+
+			ResourceID resourceID = ResourceID.generate();
+
+			// Send task manager registration
+			resourceManager.tell(new NotifyResourceStarted(resourceID),
+				fakeJobManager);
+
+			expectMsgClass(Acknowledge.class);
+
+			// check for number registration of registered resources
+			resourceManager.tell(new TestingResourceManager.GetRegisteredResources(), fakeJobManager);
+			TestingResourceManager.GetRegisteredResourcesReply reply =
+				expectMsgClass(TestingResourceManager.GetRegisteredResourcesReply.class);
+
+			assertEquals(1, reply.resources.size());
+
+			// Send task manager registration again
+			resourceManager.tell(new NotifyResourceStarted(resourceID),
+				fakeJobManager);
+
+			expectMsgClass(Acknowledge.class);
+
+			// check for number registration of registered resources
+			resourceManager.tell(new TestingResourceManager.GetRegisteredResources(), fakeJobManager);
+			reply = expectMsgClass(TestingResourceManager.GetRegisteredResourcesReply.class);
+
+			assertEquals(1, reply.resources.size());
+
+			// Send invalid null resource id to throw an exception during resource registration
+			resourceManager.tell(new NotifyResourceStarted(null),
+				fakeJobManager);
+
+			expectMsgClass(Acknowledge.class);
+
+			// check for number registration of registered resources
+			resourceManager.tell(new TestingResourceManager.GetRegisteredResources(), fakeJobManager);
+			reply = expectMsgClass(TestingResourceManager.GetRegisteredResourcesReply.class);
+
+			assertEquals(1, reply.resources.size());
+		}};
+		}};
+	}
+
+	@Test
+	public void testResourceRemoval() {
+		new JavaTestKit(system){{
+		new Within(duration("10 seconds")) {
+		@Override
+		protected void run() {
+
+			fakeJobManager = TestingUtils.createForwardingActor(system, getTestActor(), Option.<String>empty());
+			resourceManager = TestingUtils.createResourceManager(system, fakeJobManager.actor(), config);
+
+			// register with JM
+			expectMsgClass(RegisterResourceManager.class);
+			resourceManager.tell(
+				new RegisterResourceManagerSuccessful(fakeJobManager.actor(), Collections.<ResourceID>emptyList()),
+				fakeJobManager);
+
+			ResourceID resourceID = ResourceID.generate();
+
+			// remove unknown resource
+			resourceManager.tell(new RemoveResource(resourceID), fakeJobManager);
+
+			// Send task manager registration
+			resourceManager.tell(new NotifyResourceStarted(resourceID),
+				fakeJobManager);
+
+			expectMsgClass(Acknowledge.class);
+
+			// check for number registration of registered resources
+			resourceManager.tell(new TestingResourceManager.GetRegisteredResources(), fakeJobManager);
+			TestingResourceManager.GetRegisteredResourcesReply reply =
+				expectMsgClass(TestingResourceManager.GetRegisteredResourcesReply.class);
+
+			assertEquals(1, reply.resources.size());
+			assertTrue(reply.resources.contains(resourceID));
+
+			// remove resource
+			resourceManager.tell(new RemoveResource(resourceID), fakeJobManager);
+
+			// check for number registration of registered resources
+			resourceManager.tell(new TestingResourceManager.GetRegisteredResources(), fakeJobManager);
+			reply =	expectMsgClass(TestingResourceManager.GetRegisteredResourcesReply.class);
+
+			assertEquals(0, reply.resources.size());
+
+		}};
+		}};
+	}
+
+	/**
+	 * Tests notification of JobManager about a failed resource.
+	 */
+	@Test
+	public void testResourceFailureNotification() {
+		new JavaTestKit(system){{
+		new Within(duration("10 seconds")) {
+		@Override
+		protected void run() {
+
+			fakeJobManager = TestingUtils.createForwardingActor(system, getTestActor(), Option.<String>empty());
+			resourceManager = TestingUtils.createResourceManager(system, fakeJobManager.actor(), config);
+
+			// register with JM
+			expectMsgClass(RegisterResourceManager.class);
+			resourceManager.tell(
+				new RegisterResourceManagerSuccessful(fakeJobManager.actor(), Collections.<ResourceID>emptyList()),
+				fakeJobManager);
+
+			ResourceID resourceID1 = ResourceID.generate();
+			ResourceID resourceID2 = ResourceID.generate();
+
+			// Send task manager registration
+			resourceManager.tell(new NotifyResourceStarted(resourceID1),
+				fakeJobManager);
+
+			expectMsgClass(Acknowledge.class);
+
+			// Send task manager registration
+			resourceManager.tell(new NotifyResourceStarted(resourceID2),
+				fakeJobManager);
+
+			expectMsgClass(Acknowledge.class);
+
+			// check for number registration of registered resources
+			resourceManager.tell(new TestingResourceManager.GetRegisteredResources(), fakeJobManager);
+			TestingResourceManager.GetRegisteredResourcesReply reply =
+				expectMsgClass(TestingResourceManager.GetRegisteredResourcesReply.class);
+
+			assertEquals(2, reply.resources.size());
+			assertTrue(reply.resources.contains(resourceID1));
+			assertTrue(reply.resources.contains(resourceID2));
+
+			// fail resources
+			resourceManager.tell(new TestingResourceManager.FailResource(resourceID1), fakeJobManager);
+			resourceManager.tell(new TestingResourceManager.FailResource(resourceID2), fakeJobManager);
+
+			ResourceRemoved answer = expectMsgClass(ResourceRemoved.class);
+			ResourceRemoved answer2 = expectMsgClass(ResourceRemoved.class);
+
+			assertEquals(resourceID1, answer.resourceId());
+			assertEquals(resourceID2, answer2.resourceId());
+
+		}};
+		}};
+	}
+}


[11/50] [abbrv] flink git commit: [FLINK-4656] [rpc] Port the existing code to Flink's own future abstraction

Posted by se...@apache.org.
[FLINK-4656] [rpc] Port the existing code to Flink's own future abstraction

This closes #2530.


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

Branch: refs/heads/flip-6
Commit: 507e86cfd7c22a18d0d839d42c1f3e2a72c1ff8a
Parents: fbd3867
Author: Till Rohrmann <tr...@apache.org>
Authored: Wed Sep 21 17:26:21 2016 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Oct 14 15:14:40 2016 +0200

----------------------------------------------------------------------
 .../runtime/concurrent/impl/FlinkFuture.java    |  4 ++
 .../flink/runtime/jobmaster/JobMaster.java      |  2 +-
 .../runtime/jobmaster/JobMasterGateway.java     |  2 +-
 .../registration/RetryingRegistration.java      | 65 ++++++++---------
 .../resourcemanager/ResourceManager.java        | 13 ++--
 .../resourcemanager/ResourceManagerGateway.java |  9 ++-
 .../slotmanager/SlotManager.java                |  9 ++-
 .../flink/runtime/rpc/MainThreadExecutable.java | 64 +++++++++++++++++
 .../flink/runtime/rpc/MainThreadExecutor.java   | 64 -----------------
 .../apache/flink/runtime/rpc/RpcEndpoint.java   | 60 ++++++----------
 .../apache/flink/runtime/rpc/RpcService.java    | 17 +++--
 .../runtime/rpc/akka/AkkaInvocationHandler.java | 42 +++++------
 .../flink/runtime/rpc/akka/AkkaRpcActor.java    | 21 +++++-
 .../flink/runtime/rpc/akka/AkkaRpcService.java  | 28 ++++----
 .../runtime/taskexecutor/TaskExecutor.java      | 12 ++--
 .../taskexecutor/TaskExecutorGateway.java       |  6 +-
 ...TaskExecutorToResourceManagerConnection.java | 34 +++++----
 .../registration/RetryingRegistrationTest.java  | 75 ++++++++++----------
 .../registration/TestRegistrationGateway.java   |  6 +-
 .../resourcemanager/ResourceManagerHATest.java  |  4 +-
 .../slotmanager/SlotProtocolTest.java           | 14 ++--
 .../flink/runtime/rpc/AsyncCallsTest.java       | 13 ++--
 .../flink/runtime/rpc/RpcCompletenessTest.java  |  9 +--
 .../flink/runtime/rpc/TestingGatewayBase.java   | 18 ++---
 .../flink/runtime/rpc/TestingRpcService.java    | 20 +++---
 .../runtime/rpc/TestingSerialRpcService.java    | 54 +++++++-------
 .../runtime/rpc/akka/AkkaRpcActorTest.java      | 19 ++---
 .../runtime/rpc/akka/AkkaRpcServiceTest.java    |  4 +-
 .../rpc/akka/MainThreadValidationTest.java      |  7 +-
 .../rpc/akka/MessageSerializationTest.java      | 19 +++--
 .../runtime/taskexecutor/TaskExecutorTest.java  |  9 ++-
 31 files changed, 355 insertions(+), 368 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/507e86cf/flink-runtime/src/main/java/org/apache/flink/runtime/concurrent/impl/FlinkFuture.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/concurrent/impl/FlinkFuture.java b/flink-runtime/src/main/java/org/apache/flink/runtime/concurrent/impl/FlinkFuture.java
index 277f4fa..004738b 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/concurrent/impl/FlinkFuture.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/concurrent/impl/FlinkFuture.java
@@ -60,6 +60,10 @@ public class FlinkFuture<T> implements Future<T> {
 		this.scalaFuture = Preconditions.checkNotNull(scalaFuture);
 	}
 
+	public scala.concurrent.Future<T> getScalaFuture() {
+		return scalaFuture;
+	}
+
 	//-----------------------------------------------------------------------------------
 	// Future's methods
 	//-----------------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/flink/blob/507e86cf/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 0a6a7ef..1537396 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
@@ -36,7 +36,7 @@ import java.util.UUID;
 
 /**
  * JobMaster implementation. The job master is responsible for the execution of a single
- * {@link org.apache.flink.runtime.jobgraph.JobGraph}.
+ * {@link JobGraph}.
  * <p>
  * It offers the following methods as part of its rpc interface to interact with the JobMaster
  * remotely:

http://git-wip-us.apache.org/repos/asf/flink/blob/507e86cf/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 a53e383..86bf17c 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,10 +18,10 @@
 
 package org.apache.flink.runtime.jobmaster;
 
+import org.apache.flink.runtime.concurrent.Future;
 import org.apache.flink.runtime.messages.Acknowledge;
 import org.apache.flink.runtime.rpc.RpcGateway;
 import org.apache.flink.runtime.taskmanager.TaskExecutionState;
-import scala.concurrent.Future;
 
 /**
  * {@link JobMaster} rpc gateway interface

http://git-wip-us.apache.org/repos/asf/flink/blob/507e86cf/flink-runtime/src/main/java/org/apache/flink/runtime/registration/RetryingRegistration.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/registration/RetryingRegistration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/registration/RetryingRegistration.java
index ea49e42..32dd978 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/registration/RetryingRegistration.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/registration/RetryingRegistration.java
@@ -18,19 +18,17 @@
 
 package org.apache.flink.runtime.registration;
 
-import akka.dispatch.OnFailure;
-import akka.dispatch.OnSuccess;
-
 import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.runtime.concurrent.AcceptFunction;
+import org.apache.flink.runtime.concurrent.ApplyFunction;
+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.rpc.RpcGateway;
 import org.apache.flink.runtime.rpc.RpcService;
 
 import org.slf4j.Logger;
 
-import scala.concurrent.Future;
-import scala.concurrent.Promise;
-import scala.concurrent.impl.Promise.DefaultPromise;
-
 import java.util.UUID;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
@@ -86,7 +84,7 @@ public abstract class RetryingRegistration<Gateway extends RpcGateway, Success e
 
 	private final UUID leaderId;
 
-	private final Promise<Tuple2<Gateway, Success>> completionPromise;
+	private final CompletableFuture<Tuple2<Gateway, Success>> completionFuture;
 
 	private final long initialRegistrationTimeout;
 
@@ -140,7 +138,7 @@ public abstract class RetryingRegistration<Gateway extends RpcGateway, Success e
 		this.delayOnError = delayOnError;
 		this.delayOnRefusedRegistration = delayOnRefusedRegistration;
 
-		this.completionPromise = new DefaultPromise<>();
+		this.completionFuture = new FlinkCompletableFuture<>();
 	}
 
 	// ------------------------------------------------------------------------
@@ -148,7 +146,7 @@ public abstract class RetryingRegistration<Gateway extends RpcGateway, Success e
 	// ------------------------------------------------------------------------
 
 	public Future<Tuple2<Gateway, Success>> getFuture() {
-		return completionPromise.future();
+		return completionFuture;
 	}
 
 	/**
@@ -184,28 +182,30 @@ public abstract class RetryingRegistration<Gateway extends RpcGateway, Success e
 			Future<Gateway> resourceManagerFuture = rpcService.connect(targetAddress, targetType);
 	
 			// upon success, start the registration attempts
-			resourceManagerFuture.onSuccess(new OnSuccess<Gateway>() {
+			resourceManagerFuture.thenAcceptAsync(new AcceptFunction<Gateway>() {
 				@Override
-				public void onSuccess(Gateway result) {
+				public void accept(Gateway result) {
 					log.info("Resolved {} address, beginning registration", targetName);
 					register(result, 1, initialRegistrationTimeout);
 				}
-			}, rpcService.getExecutionContext());
-	
+			}, rpcService.getExecutor());
+
 			// upon failure, retry, unless this is cancelled
-			resourceManagerFuture.onFailure(new OnFailure() {
+			resourceManagerFuture.exceptionallyAsync(new ApplyFunction<Throwable, Void>() {
 				@Override
-				public void onFailure(Throwable failure) {
+				public Void apply(Throwable failure) {
 					if (!isCanceled()) {
 						log.warn("Could not resolve {} address {}, retrying...", targetName, targetAddress, failure);
 						startRegistration();
 					}
+
+					return null;
 				}
-			}, rpcService.getExecutionContext());
+			}, rpcService.getExecutor());
 		}
 		catch (Throwable t) {
 			cancel();
-			completionPromise.tryFailure(t);
+			completionFuture.completeExceptionally(t);
 		}
 	}
 
@@ -225,15 +225,14 @@ public abstract class RetryingRegistration<Gateway extends RpcGateway, Success e
 			Future<RegistrationResponse> registrationFuture = invokeRegistration(gateway, leaderId, timeoutMillis);
 	
 			// if the registration was successful, let the TaskExecutor know
-			registrationFuture.onSuccess(new OnSuccess<RegistrationResponse>() {
-				
+			registrationFuture.thenAcceptAsync(new AcceptFunction<RegistrationResponse>() {
 				@Override
-				public void onSuccess(RegistrationResponse result) throws Throwable {
+				public void accept(RegistrationResponse result) {
 					if (!isCanceled()) {
 						if (result instanceof RegistrationResponse.Success) {
 							// registration successful!
 							Success success = (Success) result;
-							completionPromise.success(new Tuple2<>(gateway, success));
+							completionFuture.complete(Tuple2.of(gateway, success));
 						}
 						else {
 							// registration refused or unknown
@@ -241,7 +240,7 @@ public abstract class RetryingRegistration<Gateway extends RpcGateway, Success e
 								RegistrationResponse.Decline decline = (RegistrationResponse.Decline) result;
 								log.info("Registration at {} was declined: {}", targetName, decline.getReason());
 							} else {
-								log.error("Received unknown response to registration attempt: " + result);
+								log.error("Received unknown response to registration attempt: {}", result);
 							}
 
 							log.info("Pausing and re-attempting registration in {} ms", delayOnRefusedRegistration);
@@ -249,12 +248,12 @@ public abstract class RetryingRegistration<Gateway extends RpcGateway, Success e
 						}
 					}
 				}
-			}, rpcService.getExecutionContext());
+			}, rpcService.getExecutor());
 	
 			// upon failure, retry
-			registrationFuture.onFailure(new OnFailure() {
+			registrationFuture.exceptionallyAsync(new ApplyFunction<Throwable, Void>() {
 				@Override
-				public void onFailure(Throwable failure) {
+				public Void apply(Throwable failure) {
 					if (!isCanceled()) {
 						if (failure instanceof TimeoutException) {
 							// we simply have not received a response in time. maybe the timeout was
@@ -262,26 +261,28 @@ public abstract class RetryingRegistration<Gateway extends RpcGateway, Success e
 							// currently down.
 							if (log.isDebugEnabled()) {
 								log.debug("Registration at {} ({}) attempt {} timed out after {} ms",
-										targetName, targetAddress, attempt, timeoutMillis);
+									targetName, targetAddress, attempt, timeoutMillis);
 							}
-	
+
 							long newTimeoutMillis = Math.min(2 * timeoutMillis, maxRegistrationTimeout);
 							register(gateway, attempt + 1, newTimeoutMillis);
 						}
 						else {
 							// a serious failure occurred. we still should not give up, but keep trying
-							log.error("Registration at " + targetName + " failed due to an error", failure);
+							log.error("Registration at {} failed due to an error", targetName, failure);
 							log.info("Pausing and re-attempting registration in {} ms", delayOnError);
-	
+
 							registerLater(gateway, 1, initialRegistrationTimeout, delayOnError);
 						}
 					}
+
+					return null;
 				}
-			}, rpcService.getExecutionContext());
+			}, rpcService.getExecutor());
 		}
 		catch (Throwable t) {
 			cancel();
-			completionPromise.tryFailure(t);
+			completionFuture.completeExceptionally(t);
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/507e86cf/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 d9a7134..5370710 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
@@ -18,12 +18,11 @@
 
 package org.apache.flink.runtime.resourcemanager;
 
-import akka.dispatch.Futures;
-import akka.dispatch.Mapper;
-
 import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.concurrent.ApplyFunction;
+import org.apache.flink.runtime.concurrent.Future;
 import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
 import org.apache.flink.runtime.instance.InstanceID;
 import org.apache.flink.runtime.leaderelection.LeaderContender;
@@ -38,7 +37,6 @@ import org.apache.flink.runtime.taskexecutor.TaskExecutorRegistrationSuccess;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import scala.concurrent.Future;
 
 import java.util.HashMap;
 import java.util.Map;
@@ -126,10 +124,9 @@ public class ResourceManager extends RpcEndpoint<ResourceManagerGateway> impleme
 			getRpcService().connect(jobMasterRegistration.getAddress(), JobMasterGateway.class);
 		final JobID jobID = jobMasterRegistration.getJobID();
 
-		return jobMasterFuture.map(new Mapper<JobMasterGateway, RegistrationResponse>() {
+		return jobMasterFuture.thenApplyAsync(new ApplyFunction<JobMasterGateway, RegistrationResponse>() {
 			@Override
-			public RegistrationResponse apply(final JobMasterGateway jobMasterGateway) {
-
+			public RegistrationResponse apply(JobMasterGateway jobMasterGateway) {
 				final JobMasterGateway existingGateway = jobMasterGateways.put(jobID, jobMasterGateway);
 				if (existingGateway != null) {
 					LOG.info("Replacing existing gateway {} for JobID {} with  {}.",
@@ -137,7 +134,7 @@ public class ResourceManager extends RpcEndpoint<ResourceManagerGateway> impleme
 				}
 				return new RegistrationResponse(true);
 			}
-		}, getMainThreadExecutionContext());
+		}, getMainThreadExecutor());
 	}
 
 	/**

http://git-wip-us.apache.org/repos/asf/flink/blob/507e86cf/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 c8e3488..5c8786c 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
@@ -18,14 +18,13 @@
 
 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.rpc.RpcGateway;
 import org.apache.flink.runtime.rpc.RpcTimeout;
 import org.apache.flink.runtime.jobmaster.JobMaster;
 
-import scala.concurrent.Future;
-import scala.concurrent.duration.FiniteDuration;
-
 import java.util.UUID;
 
 /**
@@ -42,7 +41,7 @@ public interface ResourceManagerGateway extends RpcGateway {
 	 */
 	Future<RegistrationResponse> registerJobMaster(
 		JobMasterRegistration jobMasterRegistration,
-		@RpcTimeout FiniteDuration timeout);
+		@RpcTimeout Time timeout);
 
 	/**
 	 * Register a {@link JobMaster} at the resource manager.
@@ -73,5 +72,5 @@ public interface ResourceManagerGateway extends RpcGateway {
 			UUID resourceManagerLeaderId,
 			String taskExecutorAddress,
 			ResourceID resourceID,
-			@RpcTimeout FiniteDuration timeout);
+			@RpcTimeout Time timeout);
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/507e86cf/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManager.java
index 96fde7d..97176b2 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManager.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManager.java
@@ -19,11 +19,13 @@
 package org.apache.flink.runtime.resourcemanager.slotmanager;
 
 import org.apache.flink.annotation.VisibleForTesting;
+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.ResourceSlot;
 import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.concurrent.Future;
 import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener;
 import org.apache.flink.runtime.resourcemanager.SlotRequest;
 import org.apache.flink.runtime.resourcemanager.SlotRequestRegistered;
@@ -33,14 +35,11 @@ import org.apache.flink.runtime.taskexecutor.SlotStatus;
 import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import scala.concurrent.Future;
-import scala.concurrent.duration.FiniteDuration;
 
 import java.util.HashMap;
 import java.util.LinkedHashMap;
 import java.util.Map;
 import java.util.UUID;
-import java.util.concurrent.TimeUnit;
 
 import static org.apache.flink.util.Preconditions.checkNotNull;
 
@@ -79,7 +78,7 @@ public abstract class SlotManager implements LeaderRetrievalListener {
 	/** All allocations, we can lookup allocations either by SlotID or AllocationID */
 	private final AllocationMap allocationMap;
 
-	private final FiniteDuration timeout;
+	private final Time timeout;
 
 	/** The current leader id set by the ResourceManager */
 	private UUID leaderID;
@@ -90,7 +89,7 @@ public abstract class SlotManager implements LeaderRetrievalListener {
 		this.freeSlots = new HashMap<>(16);
 		this.allocationMap = new AllocationMap();
 		this.taskManagerGateways = new HashMap<>();
-		this.timeout = new FiniteDuration(10, TimeUnit.SECONDS);
+		this.timeout = Time.seconds(10);
 	}
 
 

http://git-wip-us.apache.org/repos/asf/flink/blob/507e86cf/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/MainThreadExecutable.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/MainThreadExecutable.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/MainThreadExecutable.java
new file mode 100644
index 0000000..ec1c984
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/MainThreadExecutable.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.rpc;
+
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.concurrent.Future;
+
+import java.util.concurrent.Callable;
+import java.util.concurrent.TimeoutException;
+
+/**
+ * Interface to execute {@link Runnable} and {@link Callable} in the main thread of the underlying
+ * RPC endpoint.
+ *
+ * <p>This interface is intended to be implemented by the self gateway in a {@link RpcEndpoint}
+ * implementation which allows to dispatch local procedures to the main thread of the underlying
+ * RPC endpoint.
+ */
+public interface MainThreadExecutable {
+
+	/**
+	 * Execute the runnable in the main thread of the underlying RPC endpoint.
+	 *
+	 * @param runnable Runnable to be executed
+	 */
+	void runAsync(Runnable runnable);
+
+	/**
+	 * Execute the callable in the main thread of the underlying RPC endpoint and return a future for
+	 * the callable result. If the future is not completed within the given timeout, the returned
+	 * future will throw a {@link TimeoutException}.
+	 *
+	 * @param callable Callable to be executed
+	 * @param callTimeout Timeout for the future to complete
+	 * @param <V> Return value of the callable
+	 * @return Future of the callable result
+	 */
+	<V> Future<V> callAsync(Callable<V> callable, Time callTimeout);
+
+	/**
+	 * 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, in milliseconds, after which the runnable will be executed
+	 */
+	void scheduleRunAsync(Runnable runnable, long delay);
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/507e86cf/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/MainThreadExecutor.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/MainThreadExecutor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/MainThreadExecutor.java
deleted file mode 100644
index 5e4fead..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/MainThreadExecutor.java
+++ /dev/null
@@ -1,64 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.runtime.rpc;
-
-import akka.util.Timeout;
-import scala.concurrent.Future;
-
-import java.util.concurrent.Callable;
-import java.util.concurrent.TimeoutException;
-
-/**
- * Interface to execute {@link Runnable} and {@link Callable} in the main thread of the underlying
- * RPC endpoint.
- *
- * <p>This interface is intended to be implemented by the self gateway in a {@link RpcEndpoint}
- * implementation which allows to dispatch local procedures to the main thread of the underlying
- * RPC endpoint.
- */
-public interface MainThreadExecutor {
-
-	/**
-	 * Execute the runnable in the main thread of the underlying RPC endpoint.
-	 *
-	 * @param runnable Runnable to be executed
-	 */
-	void runAsync(Runnable runnable);
-
-	/**
-	 * Execute the callable in the main thread of the underlying RPC endpoint and return a future for
-	 * the callable result. If the future is not completed within the given timeout, the returned
-	 * future will throw a {@link TimeoutException}.
-	 *
-	 * @param callable Callable to be executed
-	 * @param callTimeout Timeout for the future to complete
-	 * @param <V> Return value of the callable
-	 * @return Future of the callable result
-	 */
-	<V> Future<V> callAsync(Callable<V> callable, Timeout callTimeout);
-
-	/**
-	 * 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, in milliseconds, after which the runnable will be executed
-	 */
-	void scheduleRunAsync(Runnable runnable, long delay);
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/507e86cf/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 e9e2b2c..4e5e49a 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
@@ -18,16 +18,15 @@
 
 package org.apache.flink.runtime.rpc;
 
-import akka.util.Timeout;
-
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.concurrent.Future;
+import org.apache.flink.util.Preconditions;
 import org.apache.flink.util.ReflectionUtil;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import scala.concurrent.ExecutionContext;
-import scala.concurrent.Future;
-
 import java.util.concurrent.Callable;
+import java.util.concurrent.Executor;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicReference;
 
@@ -49,8 +48,8 @@ import static org.apache.flink.util.Preconditions.checkNotNull;
  * thread, we don't have to reason about concurrent accesses, in the same way in the Actor Model
  * of Erlang or Akka.
  *
- * <p>The RPC endpoint provides provides {@link #runAsync(Runnable)}, {@link #callAsync(Callable, Timeout)}
-  * and the {@link #getMainThreadExecutionContext()} to execute code in the RPC endoint's main thread.
+ * <p>The RPC endpoint provides provides {@link #runAsync(Runnable)}, {@link #callAsync(Callable, Time)}
+  * and the {@link #getMainThreadExecutor()} to execute code in the RPC endoint's main thread.
  *
  * @param <C> The RPC gateway counterpart for the implementing RPC endpoint
  */
@@ -69,9 +68,9 @@ public abstract class RpcEndpoint<C extends RpcGateway> {
 	/** Self gateway which can be used to schedule asynchronous calls on yourself */
 	private final C self;
 
-	/** The main thread execution context to be used to execute future callbacks in the main thread
+	/** The main thread executor to be used to execute future callbacks in the main thread
 	 * of the executing rpc server. */
-	private final ExecutionContext mainThreadExecutionContext;
+	private final Executor mainThreadExecutor;
 
 	/** A reference to the endpoint's main thread, if the current method is called by the main thread */
 	final AtomicReference<Thread> currentMainThread = new AtomicReference<>(null); 
@@ -89,7 +88,7 @@ public abstract class RpcEndpoint<C extends RpcGateway> {
 		this.selfGatewayType = ReflectionUtil.getTemplateType1(getClass());
 		this.self = rpcService.startServer(this);
 		
-		this.mainThreadExecutionContext = new MainThreadExecutionContext((MainThreadExecutor) self);
+		this.mainThreadExecutor = new MainThreadExecutor((MainThreadExecutable) self);
 	}
 
 	/**
@@ -120,7 +119,7 @@ public abstract class RpcEndpoint<C extends RpcGateway> {
 	 * Shuts down the underlying RPC endpoint via the RPC service.
 	 * After this method was called, the RPC endpoint will no longer be reachable, neither remotely,
 	 * not via its {@link #getSelf() self gateway}. It will also not accepts executions in main thread
-	 * any more (via {@link #callAsync(Callable, Timeout)} and {@link #runAsync(Runnable)}).
+	 * any more (via {@link #callAsync(Callable, Time)} and {@link #runAsync(Runnable)}).
 	 * 
 	 * <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.
@@ -161,8 +160,8 @@ public abstract class RpcEndpoint<C extends RpcGateway> {
 	 *
 	 * @return Main thread execution context
 	 */
-	protected ExecutionContext getMainThreadExecutionContext() {
-		return mainThreadExecutionContext;
+	protected Executor getMainThreadExecutor() {
+		return mainThreadExecutor;
 	}
 
 	/**
@@ -185,7 +184,7 @@ public abstract class RpcEndpoint<C extends RpcGateway> {
 	 * @param runnable Runnable to be executed in the main thread of the underlying RPC endpoint
 	 */
 	protected void runAsync(Runnable runnable) {
-		((MainThreadExecutor) self).runAsync(runnable);
+		((MainThreadExecutable) self).runAsync(runnable);
 	}
 
 	/**
@@ -196,7 +195,7 @@ public abstract class RpcEndpoint<C extends RpcGateway> {
 	 * @param delay    The delay after which the runnable will be executed
 	 */
 	protected void scheduleRunAsync(Runnable runnable, long delay, TimeUnit unit) {
-		((MainThreadExecutor) self).scheduleRunAsync(runnable, unit.toMillis(delay));
+		((MainThreadExecutable) self).scheduleRunAsync(runnable, unit.toMillis(delay));
 	}
 
 	/**
@@ -209,8 +208,8 @@ public abstract class RpcEndpoint<C extends RpcGateway> {
 	 * @param <V> Return type of the callable
 	 * @return Future for the result of the callable.
 	 */
-	protected <V> Future<V> callAsync(Callable<V> callable, Timeout timeout) {
-		return ((MainThreadExecutor) self).callAsync(callable, timeout);
+	protected <V> Future<V> callAsync(Callable<V> callable, Time timeout) {
+		return ((MainThreadExecutable) self).callAsync(callable, timeout);
 	}
 
 	// ------------------------------------------------------------------------
@@ -241,36 +240,19 @@ public abstract class RpcEndpoint<C extends RpcGateway> {
 	// ------------------------------------------------------------------------
 	
 	/**
-	 * Execution context which executes runnables in the main thread context. A reported failure
-	 * will cause the underlying rpc server to shut down.
+	 * Executor which executes runnables in the main thread context.
 	 */
-	private class MainThreadExecutionContext implements ExecutionContext {
+	private class MainThreadExecutor implements Executor {
 
-		private final MainThreadExecutor gateway;
+		private final MainThreadExecutable gateway;
 
-		MainThreadExecutionContext(MainThreadExecutor gateway) {
-			this.gateway = gateway;
+		MainThreadExecutor(MainThreadExecutable gateway) {
+			this.gateway = Preconditions.checkNotNull(gateway);
 		}
 
 		@Override
 		public void execute(Runnable runnable) {
 			gateway.runAsync(runnable);
 		}
-
-		@Override
-		public void reportFailure(final Throwable t) {
-			gateway.runAsync(new Runnable() {
-				@Override
-				public void run() {
-					log.error("Encountered failure in the main thread execution context.", t);
-					shutDown();
-				}
-			});
-		}
-
-		@Override
-		public ExecutionContext prepare() {
-			return this;
-		}
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/507e86cf/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 78c1cec..a367ff2 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
@@ -18,10 +18,10 @@
 
 package org.apache.flink.runtime.rpc;
 
+import org.apache.flink.runtime.concurrent.Future;
 import org.apache.flink.runtime.rpc.exceptions.RpcConnectionException;
-import scala.concurrent.ExecutionContext;
-import scala.concurrent.Future;
 
+import java.util.concurrent.Executor;
 import java.util.concurrent.TimeUnit;
 
 /**
@@ -68,23 +68,22 @@ public interface RpcService {
 	void stopService();
 
 	/**
-	 * Gets the execution context, provided by this RPC service. This execution
-	 * context can be used for example for the {@code onComplete(...)} or {@code onSuccess(...)}
-	 * methods of Futures.
+	 * Gets the executor, provided by this RPC service. This executor can be used for example for
+	 * the {@code handleAsync(...)} or {@code thenAcceptAsync(...)} methods of futures.
 	 * 
-	 * <p><b>IMPORTANT:</b> This execution context does not isolate the method invocations against
+	 * <p><b>IMPORTANT:</b> This executor does not isolate the method invocations against
 	 * any concurrent invocations and is therefore not suitable to run completion methods of futures
 	 * that modify state of an {@link RpcEndpoint}. For such operations, one needs to use the
-	 * {@link RpcEndpoint#getMainThreadExecutionContext() MainThreadExecutionContext} of that
+	 * {@link RpcEndpoint#getMainThreadExecutor() MainThreadExecutionContext} of that
 	 * {@code RpcEndpoint}.
 	 * 
 	 * @return The execution context provided by the RPC service
 	 */
-	ExecutionContext getExecutionContext();
+	Executor getExecutor();
 
 	/**
 	 * Execute the runnable in the execution context of this RPC Service, as returned by
-	 * {@link #getExecutionContext()}, after a scheduled delay.
+	 * {@link #getExecutor()}, after a scheduled delay.
 	 *
 	 * @param runnable Runnable to be executed
 	 * @param delay    The delay after which the runnable will be executed

http://git-wip-us.apache.org/repos/asf/flink/blob/507e86cf/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaInvocationHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaInvocationHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaInvocationHandler.java
index bfa04f6..8f4deff 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaInvocationHandler.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaInvocationHandler.java
@@ -20,9 +20,11 @@ package org.apache.flink.runtime.rpc.akka;
 
 import akka.actor.ActorRef;
 import akka.pattern.Patterns;
-import akka.util.Timeout;
+import org.apache.flink.api.common.time.Time;
 import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.runtime.rpc.MainThreadExecutor;
+import org.apache.flink.runtime.concurrent.Future;
+import org.apache.flink.runtime.concurrent.impl.FlinkFuture;
+import org.apache.flink.runtime.rpc.MainThreadExecutable;
 import org.apache.flink.runtime.rpc.RpcGateway;
 import org.apache.flink.runtime.rpc.RpcTimeout;
 import org.apache.flink.runtime.rpc.StartStoppable;
@@ -34,9 +36,6 @@ import org.apache.flink.runtime.rpc.akka.messages.RpcInvocation;
 import org.apache.flink.runtime.rpc.akka.messages.RunAsync;
 import org.apache.flink.util.Preconditions;
 import org.apache.log4j.Logger;
-import scala.concurrent.Await;
-import scala.concurrent.Future;
-import scala.concurrent.duration.FiniteDuration;
 
 import java.io.IOException;
 import java.lang.annotation.Annotation;
@@ -53,7 +52,7 @@ import static org.apache.flink.util.Preconditions.checkArgument;
  * rpc in a {@link LocalRpcInvocation} message and then sends it to the {@link AkkaRpcActor} where it is
  * executed.
  */
-class AkkaInvocationHandler implements InvocationHandler, AkkaGateway, MainThreadExecutor, StartStoppable {
+class AkkaInvocationHandler implements InvocationHandler, AkkaGateway, MainThreadExecutable, StartStoppable {
 	private static final Logger LOG = Logger.getLogger(AkkaInvocationHandler.class);
 
 	private final String address;
@@ -64,11 +63,11 @@ class AkkaInvocationHandler implements InvocationHandler, AkkaGateway, MainThrea
 	private final boolean isLocal;
 
 	// default timeout for asks
-	private final Timeout timeout;
+	private final Time timeout;
 
 	private final long maximumFramesize;
 
-	AkkaInvocationHandler(String address, ActorRef rpcEndpoint, Timeout timeout, long maximumFramesize) {
+	AkkaInvocationHandler(String address, ActorRef rpcEndpoint, Time timeout, long maximumFramesize) {
 		this.address = Preconditions.checkNotNull(address);
 		this.rpcEndpoint = Preconditions.checkNotNull(rpcEndpoint);
 		this.isLocal = this.rpcEndpoint.path().address().hasLocalScope();
@@ -82,7 +81,7 @@ class AkkaInvocationHandler implements InvocationHandler, AkkaGateway, MainThrea
 
 		Object result;
 
-		if (declaringClass.equals(AkkaGateway.class) || declaringClass.equals(MainThreadExecutor.class) ||
+		if (declaringClass.equals(AkkaGateway.class) || declaringClass.equals(MainThreadExecutable.class) ||
 			declaringClass.equals(Object.class) || declaringClass.equals(StartStoppable.class) ||
 			declaringClass.equals(RpcGateway.class)) {
 			result = method.invoke(this, args);
@@ -90,7 +89,7 @@ class AkkaInvocationHandler implements InvocationHandler, AkkaGateway, MainThrea
 			String methodName = method.getName();
 			Class<?>[] parameterTypes = method.getParameterTypes();
 			Annotation[][] parameterAnnotations = method.getParameterAnnotations();
-			Timeout futureTimeout = extractRpcTimeout(parameterAnnotations, args, timeout);
+			Time futureTimeout = extractRpcTimeout(parameterAnnotations, args, timeout);
 
 			Tuple2<Class<?>[], Object[]> filteredArguments = filterArguments(
 				parameterTypes,
@@ -130,13 +129,14 @@ class AkkaInvocationHandler implements InvocationHandler, AkkaGateway, MainThrea
 				result = null;
 			} else if (returnType.equals(Future.class)) {
 				// execute an asynchronous call
-				result = Patterns.ask(rpcEndpoint, rpcInvocation, futureTimeout);
+				result = new FlinkFuture<>(Patterns.ask(rpcEndpoint, rpcInvocation, futureTimeout.toMilliseconds()));
 			} else {
 				// execute a synchronous call
-				Future<?> futureResult = Patterns.ask(rpcEndpoint, rpcInvocation, futureTimeout);
-				FiniteDuration duration = timeout.duration();
+				scala.concurrent.Future<?> scalaFuture = Patterns.ask(rpcEndpoint, rpcInvocation, futureTimeout.toMilliseconds());
 
-				result = Await.result(futureResult, duration);
+				Future<?> futureResult = new FlinkFuture<>(scalaFuture);
+
+				return futureResult.get(futureTimeout.getSize(), futureTimeout.getUnit());
 			}
 		}
 
@@ -167,12 +167,12 @@ class AkkaInvocationHandler implements InvocationHandler, AkkaGateway, MainThrea
 	}
 
 	@Override
-	public <V> Future<V> callAsync(Callable<V> callable, Timeout callTimeout) {
+	public <V> Future<V> callAsync(Callable<V> callable, Time callTimeout) {
 		if(isLocal) {
 			@SuppressWarnings("unchecked")
-			Future<V> result = (Future<V>) Patterns.ask(rpcEndpoint, new CallAsync(callable), callTimeout);
+			scala.concurrent.Future<V> result = (scala.concurrent.Future<V>) Patterns.ask(rpcEndpoint, new CallAsync(callable), callTimeout.toMilliseconds());
 
-			return result;
+			return new FlinkFuture<>(result);
 		} else {
 			throw new RuntimeException("Trying to send a Callable to a remote actor at " +
 				rpcEndpoint.path() + ". This is not supported.");
@@ -204,17 +204,17 @@ class AkkaInvocationHandler implements InvocationHandler, AkkaGateway, MainThrea
 	 *                       has been found
 	 * @return Timeout extracted from the array of arguments or the default timeout
 	 */
-	private static Timeout extractRpcTimeout(Annotation[][] parameterAnnotations, Object[] args, Timeout defaultTimeout) {
+	private static Time extractRpcTimeout(Annotation[][] parameterAnnotations, Object[] args, Time defaultTimeout) {
 		if (args != null) {
 			Preconditions.checkArgument(parameterAnnotations.length == args.length);
 
 			for (int i = 0; i < parameterAnnotations.length; i++) {
 				if (isRpcTimeout(parameterAnnotations[i])) {
-					if (args[i] instanceof FiniteDuration) {
-						return new Timeout((FiniteDuration) args[i]);
+					if (args[i] instanceof Time) {
+						return (Time) args[i];
 					} else {
 						throw new RuntimeException("The rpc timeout parameter must be of type " +
-							FiniteDuration.class.getName() + ". The type " + args[i].getClass().getName() +
+							Time.class.getName() + ". The type " + args[i].getClass().getName() +
 							" is not supported.");
 					}
 				}

http://git-wip-us.apache.org/repos/asf/flink/blob/507e86cf/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 2373be9..59daa46 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
@@ -21,8 +21,11 @@ package org.apache.flink.runtime.rpc.akka;
 import akka.actor.ActorRef;
 import akka.actor.Status;
 import akka.actor.UntypedActorWithStash;
+import akka.dispatch.Futures;
 import akka.japi.Procedure;
 import akka.pattern.Patterns;
+import org.apache.flink.runtime.concurrent.Future;
+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;
@@ -35,7 +38,6 @@ import org.apache.flink.runtime.rpc.akka.messages.RunAsync;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import scala.concurrent.Future;
 import scala.concurrent.duration.FiniteDuration;
 
 import java.io.IOException;
@@ -146,8 +148,23 @@ class AkkaRpcActor<C extends RpcGateway, T extends RpcEndpoint<C>> extends Untyp
 					Object result = rpcMethod.invoke(rpcEndpoint, rpcInvocation.getArgs());
 
 					if (result instanceof Future) {
+						final Future<?> future = (Future<?>) result;
+
 						// pipe result to sender
-						Patterns.pipe((Future<?>) result, getContext().dispatcher()).to(getSender());
+						if (future instanceof FlinkFuture) {
+							// FlinkFutures are currently backed by Scala's futures
+							FlinkFuture<?> flinkFuture = (FlinkFuture<?>) future;
+
+							Patterns.pipe(flinkFuture.getScalaFuture(), getContext().dispatcher()).to(getSender());
+						} else {
+							// We have to unpack the Flink future and pack it into a Scala future
+							Patterns.pipe(Futures.future(new Callable<Object>() {
+								@Override
+								public Object call() throws Exception {
+									return future.get();
+								}
+							}, getContext().dispatcher()), getContext().dispatcher());
+						}
 					} else {
 						// tell the sender the result of the computation
 						getSender().tell(new Status.Success(result), getSelf());

http://git-wip-us.apache.org/repos/asf/flink/blob/507e86cf/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 060a1ef..36f1115 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
@@ -26,11 +26,13 @@ import akka.actor.Identify;
 import akka.actor.PoisonPill;
 import akka.actor.Props;
 import akka.dispatch.Mapper;
-import akka.pattern.AskableActorSelection;
-import akka.util.Timeout;
 
+import akka.pattern.Patterns;
+import org.apache.flink.api.common.time.Time;
 import org.apache.flink.runtime.akka.AkkaUtils;
-import org.apache.flink.runtime.rpc.MainThreadExecutor;
+import org.apache.flink.runtime.concurrent.Future;
+import org.apache.flink.runtime.concurrent.impl.FlinkFuture;
+import org.apache.flink.runtime.rpc.MainThreadExecutable;
 import org.apache.flink.runtime.rpc.RpcGateway;
 import org.apache.flink.runtime.rpc.RpcEndpoint;
 import org.apache.flink.runtime.rpc.RpcService;
@@ -39,8 +41,6 @@ import org.apache.flink.runtime.rpc.exceptions.RpcConnectionException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import scala.concurrent.ExecutionContext;
-import scala.concurrent.Future;
 import scala.concurrent.duration.FiniteDuration;
 
 import javax.annotation.concurrent.ThreadSafe;
@@ -48,6 +48,7 @@ import java.lang.reflect.InvocationHandler;
 import java.lang.reflect.Proxy;
 import java.util.HashSet;
 import java.util.Set;
+import java.util.concurrent.Executor;
 import java.util.concurrent.TimeUnit;
 
 import static org.apache.flink.util.Preconditions.checkArgument;
@@ -68,13 +69,13 @@ public class AkkaRpcService implements RpcService {
 	private final Object lock = new Object();
 
 	private final ActorSystem actorSystem;
-	private final Timeout timeout;
+	private final Time timeout;
 	private final Set<ActorRef> actors = new HashSet<>(4);
 	private final long maximumFramesize;
 
 	private volatile boolean stopped;
 
-	public AkkaRpcService(final ActorSystem actorSystem, final Timeout timeout) {
+	public AkkaRpcService(final ActorSystem actorSystem, final Time timeout) {
 		this.actorSystem = checkNotNull(actorSystem, "actor system");
 		this.timeout = checkNotNull(timeout, "timeout");
 
@@ -95,10 +96,9 @@ public class AkkaRpcService implements RpcService {
 				address, clazz.getName());
 
 		final ActorSelection actorSel = actorSystem.actorSelection(address);
-		final AskableActorSelection asker = new AskableActorSelection(actorSel);
 
-		final Future<Object> identify = asker.ask(new Identify(42), timeout);
-		return identify.map(new Mapper<Object, C>(){
+		final scala.concurrent.Future<Object> identify = Patterns.ask(actorSel, new Identify(42), timeout.toMilliseconds());
+		final scala.concurrent.Future<C> resultFuture = identify.map(new Mapper<Object, C>(){
 			@Override
 			public C checkedApply(Object obj) throws Exception {
 
@@ -128,6 +128,8 @@ public class AkkaRpcService implements RpcService {
 				}
 			}
 		}, actorSystem.dispatcher());
+
+		return new FlinkFuture<>(resultFuture);
 	}
 
 	@Override
@@ -159,7 +161,7 @@ public class AkkaRpcService implements RpcService {
 			classLoader,
 			new Class<?>[]{
 				rpcEndpoint.getSelfGatewayType(),
-				MainThreadExecutor.class,
+				MainThreadExecutable.class,
 				StartStoppable.class,
 				AkkaGateway.class},
 			akkaInvocationHandler);
@@ -209,7 +211,7 @@ public class AkkaRpcService implements RpcService {
 	}
 
 	@Override
-	public ExecutionContext getExecutionContext() {
+	public Executor getExecutor() {
 		return actorSystem.dispatcher();
 	}
 
@@ -219,6 +221,6 @@ public class AkkaRpcService implements RpcService {
 		checkNotNull(unit, "unit");
 		checkArgument(delay >= 0, "delay must be zero or larger");
 
-		actorSystem.scheduler().scheduleOnce(new FiniteDuration(delay, unit), runnable, getExecutionContext());
+		actorSystem.scheduler().scheduleOnce(new FiniteDuration(delay, unit), runnable, actorSystem.dispatcher());
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/507e86cf/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 fadae5f..d84a6a9 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
@@ -19,8 +19,8 @@
 package org.apache.flink.runtime.taskexecutor;
 
 import akka.actor.ActorSystem;
-import akka.util.Timeout;
 import com.typesafe.config.Config;
+import org.apache.flink.api.common.time.Time;
 import org.apache.flink.runtime.clusterframework.types.AllocationID;
 import org.apache.flink.runtime.io.network.ConnectionManager;
 import org.apache.flink.runtime.io.network.LocalConnectionManager;
@@ -35,6 +35,7 @@ import org.apache.flink.runtime.resourcemanager.SlotRequestRegistered;
 import org.apache.flink.runtime.resourcemanager.SlotRequestReply;
 import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
 import org.apache.flink.util.Preconditions;
+import org.jboss.netty.channel.ChannelException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -78,6 +79,7 @@ import scala.concurrent.duration.FiniteDuration;
 
 import java.io.File;
 import java.io.IOException;
+import java.net.BindException;
 import java.net.InetAddress;
 import java.net.InetSocketAddress;
 import java.util.UUID;
@@ -198,7 +200,7 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 					this,
 					newLeaderAddress,
 					newLeaderId,
-					getMainThreadExecutionContext());
+					getMainThreadExecutor());
 			resourceManagerConnection.start();
 		}
 	}
@@ -302,9 +304,9 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 			LOG.debug("Using akka configuration\n " + akkaConfig);
 			taskManagerSystem = AkkaUtils.createActorSystem(akkaConfig);
 		} catch (Throwable t) {
-			if (t instanceof org.jboss.netty.channel.ChannelException) {
+			if (t instanceof ChannelException) {
 				Throwable cause = t.getCause();
-				if (cause != null && t.getCause() instanceof java.net.BindException) {
+				if (cause != null && t.getCause() instanceof BindException) {
 					String address = NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort);
 					throw new IOException("Unable to bind TaskManager actor system to address " +
 						address + " - " + cause.getMessage(), t);
@@ -314,7 +316,7 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 		}
 
 		// start akka rpc service based on actor system
-		final Timeout timeout = new Timeout(AkkaUtils.getTimeout(configuration).toMillis(), TimeUnit.MILLISECONDS);
+		final Time timeout = Time.milliseconds(AkkaUtils.getTimeout(configuration).toMillis());
 		final AkkaRpcService akkaRpcService = new AkkaRpcService(taskManagerSystem, timeout);
 
 		// start high availability service to implement getResourceManagerLeaderRetriever method only

http://git-wip-us.apache.org/repos/asf/flink/blob/507e86cf/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 65323a8..0962802 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
@@ -18,12 +18,12 @@
 
 package org.apache.flink.runtime.taskexecutor;
 
+import org.apache.flink.api.common.time.Time;
 import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.concurrent.Future;
 import org.apache.flink.runtime.resourcemanager.SlotRequestReply;
 import org.apache.flink.runtime.rpc.RpcGateway;
 import org.apache.flink.runtime.rpc.RpcTimeout;
-import scala.concurrent.Future;
-import scala.concurrent.duration.FiniteDuration;
 
 import java.util.UUID;
 
@@ -48,5 +48,5 @@ public interface TaskExecutorGateway extends RpcGateway {
 	Future<SlotRequestReply> requestSlot(
 		AllocationID allocationID,
 		UUID resourceManagerLeaderID,
-		@RpcTimeout FiniteDuration timeout);
+		@RpcTimeout Time timeout);
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/507e86cf/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 28062b6..647359d 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
@@ -18,11 +18,12 @@
 
 package org.apache.flink.runtime.taskexecutor;
 
-import akka.dispatch.OnFailure;
-import akka.dispatch.OnSuccess;
-
+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.AcceptFunction;
+import org.apache.flink.runtime.concurrent.ApplyFunction;
+import org.apache.flink.runtime.concurrent.Future;
 import org.apache.flink.runtime.instance.InstanceID;
 import org.apache.flink.runtime.rpc.RpcService;
 import org.apache.flink.runtime.registration.RegistrationResponse;
@@ -31,12 +32,8 @@ import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway;
 
 import org.slf4j.Logger;
 
-import scala.concurrent.ExecutionContext;
-import scala.concurrent.Future;
-import scala.concurrent.duration.FiniteDuration;
-
 import java.util.UUID;
-import java.util.concurrent.TimeUnit;
+import java.util.concurrent.Executor;
 
 import static org.apache.flink.util.Preconditions.checkNotNull;
 import static org.apache.flink.util.Preconditions.checkState;
@@ -57,7 +54,7 @@ public class TaskExecutorToResourceManagerConnection {
 	private final String resourceManagerAddress;
 
 	/** Execution context to be used to execute the on complete action of the ResourceManagerRegistration */
-	private final ExecutionContext executionContext;
+	private final Executor executor;
 
 	private TaskExecutorToResourceManagerConnection.ResourceManagerRegistration pendingRegistration;
 
@@ -74,13 +71,13 @@ public class TaskExecutorToResourceManagerConnection {
 		TaskExecutor taskExecutor,
 		String resourceManagerAddress,
 		UUID resourceManagerLeaderId,
-		ExecutionContext executionContext) {
+		Executor executor) {
 
 		this.log = checkNotNull(log);
 		this.taskExecutor = checkNotNull(taskExecutor);
 		this.resourceManagerAddress = checkNotNull(resourceManagerAddress);
 		this.resourceManagerLeaderId = checkNotNull(resourceManagerLeaderId);
-		this.executionContext = checkNotNull(executionContext);
+		this.executor = checkNotNull(executor);
 	}
 
 	// ------------------------------------------------------------------------
@@ -100,21 +97,22 @@ public class TaskExecutorToResourceManagerConnection {
 
 		Future<Tuple2<ResourceManagerGateway, TaskExecutorRegistrationSuccess>> future = pendingRegistration.getFuture();
 
-		future.onSuccess(new OnSuccess<Tuple2<ResourceManagerGateway, TaskExecutorRegistrationSuccess>>() {
+		future.thenAcceptAsync(new AcceptFunction<Tuple2<ResourceManagerGateway, TaskExecutorRegistrationSuccess>>() {
 			@Override
-			public void onSuccess(Tuple2<ResourceManagerGateway, TaskExecutorRegistrationSuccess> result) {
+			public void accept(Tuple2<ResourceManagerGateway, TaskExecutorRegistrationSuccess> result) {
 				registrationId = result.f1.getRegistrationId();
 				registeredResourceManager = result.f0;
 			}
-		}, executionContext);
+		}, executor);
 		
 		// this future should only ever fail if there is a bug, not if the registration is declined
-		future.onFailure(new OnFailure() {
+		future.exceptionallyAsync(new ApplyFunction<Throwable, Void>() {
 			@Override
-			public void onFailure(Throwable failure) {
+			public Void apply(Throwable failure) {
 				taskExecutor.onFatalErrorAsync(failure);
+				return null;
 			}
-		}, executionContext);
+		}, executor);
 	}
 
 	public void close() {
@@ -197,7 +195,7 @@ public class TaskExecutorToResourceManagerConnection {
 		protected Future<RegistrationResponse> invokeRegistration(
 				ResourceManagerGateway resourceManager, UUID leaderId, long timeoutMillis) throws Exception {
 
-			FiniteDuration timeout = new FiniteDuration(timeoutMillis, TimeUnit.MILLISECONDS);
+			Time timeout = Time.milliseconds(timeoutMillis);
 			return resourceManager.registerTaskExecutor(leaderId, taskExecutorAddress, resourceID, timeout);
 		}
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/507e86cf/flink-runtime/src/test/java/org/apache/flink/runtime/registration/RetryingRegistrationTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/registration/RetryingRegistrationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/registration/RetryingRegistrationTest.java
index 80fa19c..e56a9ec 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/registration/RetryingRegistrationTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/registration/RetryingRegistrationTest.java
@@ -18,9 +18,9 @@
 
 package org.apache.flink.runtime.registration;
 
-import akka.dispatch.Futures;
-
 import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.runtime.concurrent.Future;
+import org.apache.flink.runtime.concurrent.impl.FlinkCompletableFuture;
 import org.apache.flink.runtime.rpc.RpcService;
 import org.apache.flink.runtime.rpc.TestingRpcService;
 import org.apache.flink.util.TestLogger;
@@ -29,18 +29,13 @@ import org.junit.Test;
 
 import org.slf4j.LoggerFactory;
 
-import scala.concurrent.Await;
-import scala.concurrent.ExecutionContext$;
-import scala.concurrent.Future;
-import scala.concurrent.Promise;
-import scala.concurrent.duration.FiniteDuration;
-
 import java.util.UUID;
+import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 
-import static java.util.concurrent.TimeUnit.SECONDS;
 import static org.junit.Assert.*;
 import static org.mockito.Mockito.*;
 
@@ -71,8 +66,8 @@ public class RetryingRegistrationTest extends TestLogger {
 			// multiple accesses return the same future
 			assertEquals(future, registration.getFuture());
 
-			Tuple2<TestRegistrationGateway, TestRegistrationSuccess> success = 
-					Await.result(future, new FiniteDuration(10, SECONDS));
+			Tuple2<TestRegistrationGateway, TestRegistrationSuccess> success =
+					future.get(10L, TimeUnit.SECONDS);
 
 			// validate correct invocation and result
 			assertEquals(testId, success.f1.getCorrelationId());
@@ -83,7 +78,7 @@ public class RetryingRegistrationTest extends TestLogger {
 			rpc.stopService();
 		}
 	}
-	
+
 	@Test
 	public void testPropagateFailures() throws Exception {
 		final String testExceptionMessage = "testExceptionMessage";
@@ -96,9 +91,15 @@ public class RetryingRegistrationTest extends TestLogger {
 		registration.startRegistration();
 
 		Future<?> future = registration.getFuture();
-		assertTrue(future.failed().isCompleted());
+		assertTrue(future.isDone());
 
-		assertEquals(testExceptionMessage, future.failed().value().get().get().getMessage());
+		try {
+			future.get();
+
+			fail("We expected an ExecutionException.");
+		} catch (ExecutionException e) {
+			assertEquals(testExceptionMessage, e.getCause().getMessage());
+		}
 	}
 
 	@Test
@@ -113,16 +114,16 @@ public class RetryingRegistrationTest extends TestLogger {
 			// RPC service that fails upon the first connection, but succeeds on the second
 			RpcService rpc = mock(RpcService.class);
 			when(rpc.connect(anyString(), any(Class.class))).thenReturn(
-					Futures.failed(new Exception("test connect failure")),  // first connection attempt fails
-					Futures.successful(testGateway)                         // second connection attempt succeeds
+					FlinkCompletableFuture.completedExceptionally(new Exception("test connect failure")),  // first connection attempt fails
+					FlinkCompletableFuture.completed(testGateway)                         // second connection attempt succeeds
 			);
-			when(rpc.getExecutionContext()).thenReturn(ExecutionContext$.MODULE$.fromExecutor(executor));
+			when(rpc.getExecutor()).thenReturn(executor);
 
 			TestRetryingRegistration registration = new TestRetryingRegistration(rpc, "foobar address", leaderId);
 			registration.startRegistration();
 
 			Tuple2<TestRegistrationGateway, TestRegistrationSuccess> success =
-					Await.result(registration.getFuture(), new FiniteDuration(10, SECONDS));
+				registration.getFuture().get(10L, TimeUnit.SECONDS);
 
 			// validate correct invocation and result
 			assertEquals(testId, success.f1.getCorrelationId());
@@ -151,23 +152,23 @@ public class RetryingRegistrationTest extends TestLogger {
 
 		try {
 			rpc.registerGateway(testEndpointAddress, testGateway);
-	
+
 			TestRetryingRegistration registration = new TestRetryingRegistration(rpc, testEndpointAddress, leaderId);
-	
+
 			long started = System.nanoTime();
 			registration.startRegistration();
-	
+
 			Future<Tuple2<TestRegistrationGateway, TestRegistrationSuccess>> future = registration.getFuture();
 			Tuple2<TestRegistrationGateway, TestRegistrationSuccess> success =
-					Await.result(future, new FiniteDuration(10, SECONDS));
-	
+					future.get(10L, TimeUnit.SECONDS);
+
 			long finished = System.nanoTime();
 			long elapsedMillis = (finished - started) / 1000000;
-	
+
 			// validate correct invocation and result
 			assertEquals(testId, success.f1.getCorrelationId());
 			assertEquals(leaderId, testGateway.getInvocations().take().leaderId());
-	
+
 			// validate that some retry-delay / back-off behavior happened
 			assertTrue("retries did not properly back off", elapsedMillis >= 3 * TestRetryingRegistration.INITIAL_TIMEOUT);
 		}
@@ -199,10 +200,10 @@ public class RetryingRegistrationTest extends TestLogger {
 
 			long started = System.nanoTime();
 			registration.startRegistration();
-	
+
 			Future<Tuple2<TestRegistrationGateway, TestRegistrationSuccess>> future = registration.getFuture();
 			Tuple2<TestRegistrationGateway, TestRegistrationSuccess> success =
-					Await.result(future, new FiniteDuration(10, SECONDS));
+					future.get(10L, TimeUnit.SECONDS);
 
 			long finished = System.nanoTime();
 			long elapsedMillis = (finished - started) / 1000000;
@@ -212,7 +213,7 @@ public class RetryingRegistrationTest extends TestLogger {
 			assertEquals(leaderId, testGateway.getInvocations().take().leaderId());
 
 			// validate that some retry-delay / back-off behavior happened
-			assertTrue("retries did not properly back off", elapsedMillis >= 
+			assertTrue("retries did not properly back off", elapsedMillis >=
 					2 * TestRetryingRegistration.INITIAL_TIMEOUT + TestRetryingRegistration.DELAY_ON_DECLINE);
 		}
 		finally {
@@ -220,7 +221,7 @@ public class RetryingRegistrationTest extends TestLogger {
 			rpc.stopService();
 		}
 	}
-	
+
 	@Test
 	@SuppressWarnings("unchecked")
 	public void testRetryOnError() throws Exception {
@@ -235,9 +236,9 @@ public class RetryingRegistrationTest extends TestLogger {
 			TestRegistrationGateway testGateway = mock(TestRegistrationGateway.class);
 
 			when(testGateway.registrationCall(any(UUID.class), anyLong())).thenReturn(
-					Futures.<RegistrationResponse>failed(new Exception("test exception")),
-					Futures.<RegistrationResponse>successful(new TestRegistrationSuccess(testId)));
-			
+					FlinkCompletableFuture.<RegistrationResponse>completedExceptionally(new Exception("test exception")),
+					FlinkCompletableFuture.<RegistrationResponse>completed(new TestRegistrationSuccess(testId)));
+
 			rpc.registerGateway(testEndpointAddress, testGateway);
 
 			TestRetryingRegistration registration = new TestRetryingRegistration(rpc, testEndpointAddress, leaderId);
@@ -247,11 +248,11 @@ public class RetryingRegistrationTest extends TestLogger {
 
 			Future<Tuple2<TestRegistrationGateway, TestRegistrationSuccess>> future = registration.getFuture();
 			Tuple2<TestRegistrationGateway, TestRegistrationSuccess> success =
-					Await.result(future, new FiniteDuration(10, SECONDS));
+					future.get(10, TimeUnit.SECONDS);
 
 			long finished = System.nanoTime();
 			long elapsedMillis = (finished - started) / 1000000;
-			
+
 			assertEquals(testId, success.f1.getCorrelationId());
 
 			// validate that some retry-delay / back-off behavior happened
@@ -271,10 +272,10 @@ public class RetryingRegistrationTest extends TestLogger {
 		TestingRpcService rpc = new TestingRpcService();
 
 		try {
-			Promise<RegistrationResponse> result = Futures.promise();
+			FlinkCompletableFuture<RegistrationResponse> result = new FlinkCompletableFuture<>();
 
 			TestRegistrationGateway testGateway = mock(TestRegistrationGateway.class);
-			when(testGateway.registrationCall(any(UUID.class), anyLong())).thenReturn(result.future());
+			when(testGateway.registrationCall(any(UUID.class), anyLong())).thenReturn(result);
 
 			rpc.registerGateway(testEndpointAddress, testGateway);
 
@@ -283,7 +284,7 @@ public class RetryingRegistrationTest extends TestLogger {
 
 			// cancel and fail the current registration attempt
 			registration.cancel();
-			result.failure(new TimeoutException());
+			result.completeExceptionally(new TimeoutException());
 
 			// there should not be a second registration attempt
 			verify(testGateway, atMost(1)).registrationCall(any(UUID.class), anyLong());

http://git-wip-us.apache.org/repos/asf/flink/blob/507e86cf/flink-runtime/src/test/java/org/apache/flink/runtime/registration/TestRegistrationGateway.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/registration/TestRegistrationGateway.java b/flink-runtime/src/test/java/org/apache/flink/runtime/registration/TestRegistrationGateway.java
index 431fbe8..2843aeb 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/registration/TestRegistrationGateway.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/registration/TestRegistrationGateway.java
@@ -20,11 +20,11 @@ package org.apache.flink.runtime.registration;
 
 import akka.dispatch.Futures;
 
+import org.apache.flink.runtime.concurrent.Future;
+import org.apache.flink.runtime.concurrent.impl.FlinkCompletableFuture;
 import org.apache.flink.runtime.rpc.TestingGatewayBase;
 import org.apache.flink.util.Preconditions;
 
-import scala.concurrent.Future;
-
 import java.util.UUID;
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.LinkedBlockingQueue;
@@ -56,7 +56,7 @@ public class TestRegistrationGateway extends TestingGatewayBase {
 		}
 
 		// return a completed future (for a proper value), or one that never completes and will time out (for null)
-		return response != null ? Futures.successful(response) : this.<RegistrationResponse>futureWithTimeout(timeout);
+		return response != null ? FlinkCompletableFuture.completed(response) : this.<RegistrationResponse>futureWithTimeout(timeout);
 	}
 
 	public BlockingQueue<RegistrationCall> getInvocations() {

http://git-wip-us.apache.org/repos/asf/flink/blob/507e86cf/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 8183c0a..64a1191 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
@@ -21,7 +21,7 @@ package org.apache.flink.runtime.resourcemanager;
 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.rpc.MainThreadExecutor;
+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;
@@ -68,7 +68,7 @@ public class ResourceManagerHATest {
 		Assert.assertNull(resourceManager.getLeaderSessionID());
 	}
 
-	private static abstract class TestingResourceManagerGatewayProxy implements MainThreadExecutor, StartStoppable, RpcGateway {
+	private static abstract class TestingResourceManagerGatewayProxy implements MainThreadExecutable, StartStoppable, RpcGateway {
 		@Override
 		public void runAsync(Runnable runnable) {
 			runnable.run();

http://git-wip-us.apache.org/repos/asf/flink/blob/507e86cf/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 85d2880..1f9e7e8 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
@@ -18,10 +18,12 @@
 package org.apache.flink.runtime.resourcemanager.slotmanager;
 
 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.concurrent.Future;
 import org.apache.flink.runtime.highavailability.NonHaServices;
 import org.apache.flink.runtime.jobmaster.JobMasterGateway;
 import org.apache.flink.runtime.resourcemanager.JobMasterRegistration;
@@ -40,10 +42,6 @@ import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.mockito.Mockito;
-import scala.concurrent.Await;
-import scala.concurrent.Future;
-import scala.concurrent.duration.Duration;
-import scala.concurrent.duration.FiniteDuration;
 
 import java.util.Collections;
 import java.util.UUID;
@@ -99,7 +97,7 @@ public class SlotProtocolTest extends TestLogger {
 		Future<RegistrationResponse> registrationFuture =
 			resourceManager.registerJobMaster(new JobMasterRegistration(jmAddress, jobID));
 		try {
-			Await.ready(registrationFuture, Duration.create(5, TimeUnit.SECONDS));
+			registrationFuture.get(5, TimeUnit.SECONDS);
 		} catch (Exception e) {
 			Assert.fail("JobManager registration Future didn't become ready.");
 		}
@@ -141,7 +139,7 @@ public class SlotProtocolTest extends TestLogger {
 		slotManager.updateSlotStatus(slotReport);
 
 		// 4) Slot becomes available and TaskExecutor gets a SlotRequest
-		verify(taskExecutorGateway, timeout(5000)).requestSlot(eq(allocationID), any(UUID.class), any(FiniteDuration.class));
+		verify(taskExecutorGateway, timeout(5000)).requestSlot(eq(allocationID), any(UUID.class), any(Time.class));
 	}
 
 	/**
@@ -171,7 +169,7 @@ public class SlotProtocolTest extends TestLogger {
 		Future<RegistrationResponse> registrationFuture =
 			resourceManager.registerJobMaster(new JobMasterRegistration(jmAddress, jobID));
 		try {
-			Await.ready(registrationFuture, Duration.create(5, TimeUnit.SECONDS));
+			registrationFuture.get(5, TimeUnit.SECONDS);
 		} catch (Exception e) {
 			Assert.fail("JobManager registration Future didn't become ready.");
 		}
@@ -207,7 +205,7 @@ public class SlotProtocolTest extends TestLogger {
 
 
 		// 4) a SlotRequest is routed to the TaskExecutor
-		verify(taskExecutorGateway, timeout(5000)).requestSlot(eq(allocationID), any(UUID.class), any(FiniteDuration.class));
+		verify(taskExecutorGateway, timeout(5000)).requestSlot(eq(allocationID), any(UUID.class), any(Time.class));
 	}
 
 

http://git-wip-us.apache.org/repos/asf/flink/blob/507e86cf/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/AsyncCallsTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/AsyncCallsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/AsyncCallsTest.java
index 1791056..7c6b0ee 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/AsyncCallsTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/AsyncCallsTest.java
@@ -21,18 +21,16 @@ package org.apache.flink.runtime.rpc;
 import akka.actor.ActorSystem;
 import akka.util.Timeout;
 
+import org.apache.flink.api.common.time.Time;
 import org.apache.flink.core.testutils.OneShotLatch;
 import org.apache.flink.runtime.akka.AkkaUtils;
 
+import org.apache.flink.runtime.concurrent.Future;
 import org.apache.flink.runtime.rpc.akka.AkkaRpcService;
 import org.apache.flink.util.TestLogger;
 import org.junit.AfterClass;
 import org.junit.Test;
 
-import scala.concurrent.Await;
-import scala.concurrent.Future;
-import scala.concurrent.duration.FiniteDuration;
-
 import java.util.concurrent.Callable;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
@@ -49,7 +47,7 @@ public class AsyncCallsTest extends TestLogger {
 	private static ActorSystem actorSystem = AkkaUtils.createDefaultActorSystem();
 
 	private static AkkaRpcService akkaRpcService =
-			new AkkaRpcService(actorSystem, new Timeout(10000, TimeUnit.MILLISECONDS));
+			new AkkaRpcService(actorSystem, Time.milliseconds(10000L));
 
 	@AfterClass
 	public static void shutdown() {
@@ -104,8 +102,9 @@ public class AsyncCallsTest extends TestLogger {
 				}
 				return "test";
 			}
-		}, new Timeout(30, TimeUnit.SECONDS));
-		String str = Await.result(result, new FiniteDuration(30, TimeUnit.SECONDS));
+		}, Time.seconds(30L));
+
+		String str = result.get(30, TimeUnit.SECONDS);
 		assertEquals("test", str);
 
 		// validate that no concurrent access happened

http://git-wip-us.apache.org/repos/asf/flink/blob/507e86cf/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcCompletenessTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcCompletenessTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcCompletenessTest.java
index b431eb9..ee3f784 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcCompletenessTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcCompletenessTest.java
@@ -18,14 +18,14 @@
 
 package org.apache.flink.runtime.rpc;
 
+import org.apache.flink.api.common.time.Time;
 import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.runtime.concurrent.Future;
 import org.apache.flink.util.ReflectionUtil;
 import org.apache.flink.util.TestLogger;
 import org.junit.Test;
 import org.reflections.Reflections;
-import scala.concurrent.Future;
-import scala.concurrent.duration.FiniteDuration;
 
 import java.lang.annotation.Annotation;
 import java.lang.reflect.Method;
@@ -43,6 +43,7 @@ import static org.junit.Assert.fail;
 public class RpcCompletenessTest extends TestLogger {
 
 	private static final Class<?> futureClass = Future.class;
+	private static final Class<?> timeoutClass = Time.class;
 
 	@Test
 	@SuppressWarnings({"rawtypes", "unchecked"})
@@ -147,8 +148,8 @@ public class RpcCompletenessTest extends TestLogger {
 		for (int i = 0; i < parameterAnnotations.length; i++) {
 			if (RpcCompletenessTest.isRpcTimeout(parameterAnnotations[i])) {
 				assertTrue(
-					"The rpc timeout has to be of type " + FiniteDuration.class.getName() + ".",
-					parameterTypes[i].equals(FiniteDuration.class));
+					"The rpc timeout has to be of type " + timeoutClass.getName() + ".",
+					parameterTypes[i].equals(timeoutClass));
 
 				rpcTimeoutParameters++;
 			}

http://git-wip-us.apache.org/repos/asf/flink/blob/507e86cf/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingGatewayBase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingGatewayBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingGatewayBase.java
index 8133a87..caf5e81 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingGatewayBase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingGatewayBase.java
@@ -18,9 +18,9 @@
 
 package org.apache.flink.runtime.rpc;
 
-import akka.dispatch.Futures;
-import scala.concurrent.Future;
-import scala.concurrent.Promise;
+import org.apache.flink.runtime.concurrent.CompletableFuture;
+import org.apache.flink.runtime.concurrent.Future;
+import org.apache.flink.runtime.concurrent.impl.FlinkCompletableFuture;
 
 import java.util.concurrent.Executors;
 import java.util.concurrent.ScheduledExecutorService;
@@ -73,25 +73,25 @@ public abstract class TestingGatewayBase implements RpcGateway {
 	// ------------------------------------------------------------------------
 
 	public <T> Future<T> futureWithTimeout(long timeoutMillis) {
-		Promise<T> promise = Futures.<T>promise();
-		executor.schedule(new FutureTimeout(promise), timeoutMillis, TimeUnit.MILLISECONDS);
-		return promise.future();
+		FlinkCompletableFuture<T> future = new FlinkCompletableFuture<>();
+		executor.schedule(new FutureTimeout(future), timeoutMillis, TimeUnit.MILLISECONDS);
+		return future;
 	}
 
 	// ------------------------------------------------------------------------
 	
 	private static final class FutureTimeout implements Runnable {
 
-		private final Promise<?> promise;
+		private final CompletableFuture<?> promise;
 
-		private FutureTimeout(Promise<?> promise) {
+		private FutureTimeout(CompletableFuture<?> promise) {
 			this.promise = promise;
 		}
 
 		@Override
 		public void run() {
 			try {
-				promise.failure(new TimeoutException());
+				promise.completeExceptionally(new TimeoutException());
 			} catch (Throwable t) {
 				System.err.println("CAUGHT AN ERROR IN THE TEST: " + t.getMessage());
 				t.printStackTrace();

http://git-wip-us.apache.org/repos/asf/flink/blob/507e86cf/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingRpcService.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingRpcService.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingRpcService.java
index 2212680..f164056 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingRpcService.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingRpcService.java
@@ -18,18 +18,14 @@
 
 package org.apache.flink.runtime.rpc;
 
-import akka.dispatch.Futures;
-import akka.util.Timeout;
-
+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.concurrent.impl.FlinkCompletableFuture;
 import org.apache.flink.runtime.rpc.akka.AkkaRpcService;
 
-import scala.concurrent.Future;
-import scala.concurrent.duration.FiniteDuration;
-
 import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.TimeUnit;
 
 import static org.apache.flink.util.Preconditions.checkNotNull;
 
@@ -69,7 +65,7 @@ public class TestingRpcService extends AkkaRpcService {
 	 * Creates a new {@code TestingRpcService}, using the given configuration. 
 	 */
 	public TestingRpcService(Configuration configuration) {
-		super(AkkaUtils.createLocalActorSystem(configuration), new Timeout(new FiniteDuration(10, TimeUnit.SECONDS)));
+		super(AkkaUtils.createLocalActorSystem(configuration), Time.seconds(10));
 
 		this.registeredConnections = new ConcurrentHashMap<>();
 	}
@@ -103,13 +99,13 @@ public class TestingRpcService extends AkkaRpcService {
 			if (clazz.isAssignableFrom(gateway.getClass())) {
 				@SuppressWarnings("unchecked")
 				C typedGateway = (C) gateway;
-				return Futures.successful(typedGateway);
+				return FlinkCompletableFuture.completed(typedGateway);
 			} else {
-				return Futures.failed(
-						new Exception("Gateway registered under " + address + " is not of type " + clazz));
+				return FlinkCompletableFuture.completedExceptionally(
+					new Exception("Gateway registered under " + address + " is not of type " + clazz));
 			}
 		} else {
-			return Futures.failed(new Exception("No gateway registered under that name"));
+			return FlinkCompletableFuture.completedExceptionally(new Exception("No gateway registered under that name"));
 		}
 	}
 


[37/50] [abbrv] flink git commit: [hotfix] Fix failing JobManagerRunnerMockTest

Posted by se...@apache.org.
[hotfix] Fix failing JobManagerRunnerMockTest


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

Branch: refs/heads/flip-6
Commit: 30517da90f947cc3622d969dc788797bff7abe3c
Parents: 3e4eb4f
Author: Till Rohrmann <tr...@apache.org>
Authored: Tue Oct 4 14:27:37 2016 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Oct 14 15:14:42 2016 +0200

----------------------------------------------------------------------
 .../jobmaster/JobManagerRunnerMockTest.java        | 17 +----------------
 1 file changed, 1 insertion(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/30517da9/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 3a769bb..f8a0b6a 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
@@ -131,21 +131,6 @@ public class JobManagerRunnerMockTest {
 	}
 
 	@Test
-	public void testJobFinishedByOtherBeforeGrantLeadership() throws Exception {
-		runner.start();
-
-		when(submittedJobGraphStore.contains(any(JobID.class))).thenReturn(false);
-		runner.grantLeadership(UUID.randomUUID());
-
-		// runner should shutdown automatic and informed the job completion
-		verify(leaderElectionService).stop();
-		verify(jobManager).shutDown();
-
-		assertTrue(jobCompletion.isJobFinished());
-		assertTrue(jobCompletion.isJobFinishedByOther());
-	}
-
-	@Test
 	public void testJobFinished() throws Exception {
 		runner.start();
 
@@ -211,7 +196,7 @@ public class JobManagerRunnerMockTest {
 
 		UUID leaderSessionID2 = UUID.randomUUID();
 		runner.grantLeadership(leaderSessionID2);
-		verify(jobManagerGateway, times(2)).startJob(leaderSessionID2);
+		verify(jobManagerGateway).startJob(leaderSessionID2);
 	}
 
 	private static class TestingOnCompletionActions implements OnCompletionActions {


[31/50] [abbrv] flink git commit: [FLINK-4703] RpcCompletenessTest: Add support for type arguments and subclasses

Posted by se...@apache.org.
[FLINK-4703] RpcCompletenessTest: Add support for type arguments and subclasses

This closes #2561


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

Branch: refs/heads/flip-6
Commit: 6e58ebf22cb11631438ea51118615053e11cbcdb
Parents: 415af17
Author: Maximilian Michels <mx...@apache.org>
Authored: Wed Sep 28 12:39:30 2016 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Oct 14 15:14:42 2016 +0200

----------------------------------------------------------------------
 .../apache/flink/runtime/rpc/RpcEndpoint.java   | 23 +++++-
 .../flink/runtime/rpc/RpcCompletenessTest.java  | 80 ++++++++++++++++++--
 2 files changed, 94 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/6e58ebf2/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 4e5e49a..79961f7 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
@@ -85,9 +85,9 @@ public abstract class RpcEndpoint<C extends RpcGateway> {
 
 		// IMPORTANT: Don't change order of selfGatewayType and self because rpcService.startServer
 		// requires that selfGatewayType has been initialized
-		this.selfGatewayType = ReflectionUtil.getTemplateType1(getClass());
+		this.selfGatewayType = determineSelfGatewayType();
 		this.self = rpcService.startServer(this);
-		
+
 		this.mainThreadExecutor = new MainThreadExecutor((MainThreadExecutable) self);
 	}
 
@@ -255,4 +255,23 @@ public abstract class RpcEndpoint<C extends RpcGateway> {
 			gateway.runAsync(runnable);
 		}
 	}
+
+	/**
+	 * Determines the self gateway type specified in one of the subclasses which extend this class.
+	 * May traverse multiple class hierarchies until a Gateway type is found as a first type argument.
+	 * @return Class<C> The determined self gateway type
+	 */
+	private Class<C> determineSelfGatewayType() {
+
+		// determine self gateway type
+		Class c = getClass();
+		Class<C> determinedSelfGatewayType;
+		do {
+			determinedSelfGatewayType = ReflectionUtil.getTemplateType1(c);
+			// check if super class contains self gateway type in next loop
+			c = c.getSuperclass();
+		} while (!RpcGateway.class.isAssignableFrom(determinedSelfGatewayType));
+
+		return determinedSelfGatewayType;
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/6e58ebf2/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcCompletenessTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcCompletenessTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcCompletenessTest.java
index 53355e8..e7143ae 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcCompletenessTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcCompletenessTest.java
@@ -26,9 +26,14 @@ import org.apache.flink.util.ReflectionUtil;
 import org.apache.flink.util.TestLogger;
 import org.junit.Test;
 import org.reflections.Reflections;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.lang.annotation.Annotation;
 import java.lang.reflect.Method;
+import java.lang.reflect.Modifier;
+import java.lang.reflect.Type;
+import java.lang.reflect.TypeVariable;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
@@ -41,8 +46,33 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
+/**
+ * Test which ensures that all classes of subtype {@link RpcEndpoint} implement
+ * the methods specified in the generic gateway type argument.
+ *
+ * {@code
+ * 	    RpcEndpoint<GatewayTypeParameter extends RpcGateway>
+ * }
+ *
+ * Note, that the class hierarchy can also be nested. In this case the type argument
+ * always has to be the first argument, e.g. {@code
+ *
+ * 	    // RpcClass needs to implement RpcGatewayClass' methods
+ * 	    RpcClass extends RpcEndpoint<RpcGatewayClass>
+ *
+ * 	    // RpcClass2 or its subclass needs to implement RpcGatewayClass' methods
+ *      RpcClass<GatewayTypeParameter extends RpcGateway,...> extends RpcEndpoint<GatewayTypeParameter>
+ *      RpcClass2 extends RpcClass<RpcGatewayClass,...>
+ *
+ *      // needless to say, this can even be nested further
+ *      ...
+ * }
+ *
+ */
 public class RpcCompletenessTest extends TestLogger {
 
+	private static Logger LOG = LoggerFactory.getLogger(RpcCompletenessTest.class);
+
 	private static final Class<?> futureClass = Future.class;
 	private static final Class<?> timeoutClass = Time.class;
 
@@ -55,16 +85,52 @@ public class RpcCompletenessTest extends TestLogger {
 
 		Class<? extends RpcEndpoint> c;
 
-		for (Class<? extends RpcEndpoint> rpcEndpoint :classes){
+		mainloop:
+		for (Class<? extends RpcEndpoint> rpcEndpoint : classes) {
 			c = rpcEndpoint;
 
-			Class<?> rpcGatewayType = ReflectionUtil.getTemplateType1(c);
+			LOG.debug("-------------");
+			LOG.debug("c: {}", c);
 
-			if (rpcGatewayType != null) {
-				checkCompleteness(rpcEndpoint, (Class<? extends RpcGateway>) rpcGatewayType);
-			} else {
-				fail("Could not retrieve the rpc gateway class for the given rpc endpoint class " + rpcEndpoint.getName());
+			// skip abstract classes
+			if (Modifier.isAbstract(c.getModifiers())) {
+				LOG.debug("Skipping abstract class");
+				continue;
 			}
+
+			// check for type parameter bound to RpcGateway
+			// skip if one is found because a subclass will provide the concrete argument
+			TypeVariable<? extends Class<? extends RpcEndpoint>>[] typeParameters = c.getTypeParameters();
+			LOG.debug("Checking {} parameters.", typeParameters.length);
+			for (int i = 0; i < typeParameters.length; i++) {
+				for (Type bound : typeParameters[i].getBounds()) {
+					LOG.debug("checking bound {} of type parameter {}", bound, typeParameters[i]);
+					if (bound.toString().equals("interface " + RpcGateway.class.getName())) {
+						if (i > 0) {
+							fail("Type parameter for RpcGateway should come first in " + c);
+						}
+						LOG.debug("Skipping class with type parameter bound to RpcGateway.");
+						// Type parameter is bound to RpcGateway which a subclass will provide
+						continue mainloop;
+					}
+				}
+			}
+
+			// check if this class or any super class contains the RpcGateway argument
+			Class<?> rpcGatewayType;
+			do {
+				LOG.debug("checking type argument of class: {}", c);
+				rpcGatewayType = ReflectionUtil.getTemplateType1(c);
+				LOG.debug("type argument is: {}", rpcGatewayType);
+
+				c = (Class<? extends RpcEndpoint>) c.getSuperclass();
+
+			} while (!RpcGateway.class.isAssignableFrom(rpcGatewayType));
+
+			LOG.debug("Checking RRC completeness of endpoint '{}' with gateway '{}'",
+				rpcEndpoint.getSimpleName(), rpcGatewayType.getSimpleName());
+
+			checkCompleteness(rpcEndpoint, (Class<? extends RpcGateway>) rpcGatewayType);
 		}
 	}
 
@@ -352,7 +418,7 @@ public class RpcCompletenessTest extends TestLogger {
 	 */
 	private List<Method> getRpcMethodsFromGateway(Class<? extends RpcGateway> interfaceClass) {
 		if(!interfaceClass.isInterface()) {
-			fail(interfaceClass.getName() + "is not a interface");
+			fail(interfaceClass.getName() + " is not a interface");
 		}
 
 		ArrayList<Method> allMethods = new ArrayList<>();


[24/50] [abbrv] flink git commit: [FLINK-4535] [cluster management] resourceManager process the registration from TaskExecutor

Posted by se...@apache.org.
[FLINK-4535] [cluster management] resourceManager process the registration from TaskExecutor


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

Branch: refs/heads/flip-6
Commit: c9764c8f62e08a1fbf929ee2b718b52b8ae4fff2
Parents: 34a6854
Author: beyond1920 <be...@126.com>
Authored: Thu Sep 1 11:14:00 2016 +0800
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Oct 14 15:14:41 2016 +0200

----------------------------------------------------------------------
 .../resourcemanager/RegistrationResponse.java   |  36 ---
 .../resourcemanager/ResourceManager.java        | 288 ++++++++-----------
 .../resourcemanager/ResourceManagerGateway.java |  45 +--
 .../TaskExecutorRegistration.java               |  51 ++++
 .../exceptions/LeaderSessionIDException.java    |   1 +
 .../resourcemanager/ResourceManagerTest.java    | 119 ++++----
 6 files changed, 241 insertions(+), 299 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/c9764c8f/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/RegistrationResponse.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/RegistrationResponse.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/RegistrationResponse.java
deleted file mode 100644
index 796e634..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/RegistrationResponse.java
+++ /dev/null
@@ -1,36 +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 java.io.Serializable;
-
-public class RegistrationResponse implements Serializable {
-	private static final long serialVersionUID = -2379003255993119993L;
-
-	private final boolean isSuccess;
-
-	public RegistrationResponse(boolean isSuccess) {
-		this.isSuccess = isSuccess;
-	}
-
-	public boolean isSuccess() {
-		return isSuccess;
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/c9764c8f/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 aae4874..15692b6 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
@@ -18,41 +18,29 @@
 
 package org.apache.flink.runtime.resourcemanager;
 
+import akka.dispatch.Futures;
+import akka.dispatch.Mapper;
+
 import org.apache.flink.annotation.VisibleForTesting;
-import org.apache.flink.api.common.JobID;
 import org.apache.flink.runtime.clusterframework.types.ResourceID;
-import org.apache.flink.runtime.concurrent.ApplyFunction;
-import org.apache.flink.runtime.concurrent.Future;
 import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
 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.resourcemanager.slotmanager.SlotManager;
 import org.apache.flink.runtime.rpc.RpcMethod;
 import org.apache.flink.runtime.rpc.RpcEndpoint;
 import org.apache.flink.runtime.rpc.RpcService;
 import org.apache.flink.runtime.jobmaster.JobMaster;
 import org.apache.flink.runtime.jobmaster.JobMasterGateway;
+import org.apache.flink.runtime.rpc.exceptions.LeaderSessionIDException;
+import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway;
 import org.apache.flink.runtime.taskexecutor.TaskExecutorRegistrationSuccess;
 import org.apache.flink.runtime.registration.RegistrationResponse;
-
-import org.apache.flink.runtime.concurrent.Future;
-
-import org.apache.flink.runtime.util.LeaderConnectionInfo;
-import org.apache.flink.runtime.util.LeaderRetrievalUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import scala.concurrent.duration.FiniteDuration;
+import scala.concurrent.Future;
 
 import java.util.HashMap;
-import java.util.HashSet;
 import java.util.Map;
-import java.util.Set;
 import java.util.UUID;
-import java.util.concurrent.TimeUnit;
 
 import static org.apache.flink.util.Preconditions.checkNotNull;
 
@@ -62,35 +50,25 @@ import static org.apache.flink.util.Preconditions.checkNotNull;
  *
  * It offers the following methods as part of its rpc interface to interact with the him remotely:
  * <ul>
- *     <li>{@link #registerJobMaster(UUID, String, JobID)} registers a {@link JobMaster} at the resource manager</li>
+ *     <li>{@link #registerJobMaster(JobMasterRegistration)} registers a {@link JobMaster} at the resource manager</li>
  *     <li>{@link #requestSlot(SlotRequest)} requests a slot from the resource manager</li>
  * </ul>
  */
-public class ResourceManager extends RpcEndpoint<ResourceManagerGateway> implements LeaderContender {
-
-	private final Logger LOG = LoggerFactory.getLogger(getClass());
+public class ResourceManager extends RpcEndpoint<ResourceManagerGateway> {
+	private final Map<JobMasterGateway, InstanceID> jobMasterGateways;
 
-	private final Map<JobID, JobMasterGateway> jobMasterGateways;
-
-	private final Set<LeaderRetrievalListener> jobMasterLeaderRetrievalListeners;
+	/** ResourceID and TaskExecutorRegistration mapping relationship of registered taskExecutors */
+	private final Map<ResourceID, TaskExecutorRegistration>  startedTaskExecutorGateways;
 
 	private final HighAvailabilityServices highAvailabilityServices;
+	private LeaderElectionService leaderElectionService = null;
+	private UUID leaderSessionID = null;
 
-	private LeaderElectionService leaderElectionService;
-
-	private final SlotManager slotManager;
-
-	private UUID leaderSessionID;
-
-	public ResourceManager(
-			RpcService rpcService,
-			HighAvailabilityServices highAvailabilityServices,
-			SlotManager slotManager) {
+	public ResourceManager(RpcService rpcService, HighAvailabilityServices highAvailabilityServices) {
 		super(rpcService);
 		this.highAvailabilityServices = checkNotNull(highAvailabilityServices);
-		this.jobMasterGateways = new HashMap<>();
-		this.slotManager = slotManager;
-		this.jobMasterLeaderRetrievalListeners = new HashSet<>();
+		this.jobMasterGateways = new HashMap<>(16);
+		this.startedTaskExecutorGateways = new HashMap<>(16);
 	}
 
 	@Override
@@ -99,7 +77,7 @@ public class ResourceManager extends RpcEndpoint<ResourceManagerGateway> impleme
 		try {
 			super.start();
 			leaderElectionService = highAvailabilityServices.getResourceManagerLeaderElectionService();
-			leaderElectionService.start(this);
+			leaderElectionService.start(new ResourceManagerLeaderContender());
 		} 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);
@@ -110,11 +88,8 @@ public class ResourceManager extends RpcEndpoint<ResourceManagerGateway> impleme
 	public void shutDown() {
 		try {
 			leaderElectionService.stop();
-			for(JobID jobID : jobMasterGateways.keySet()) {
-				highAvailabilityServices.getJobMasterLeaderRetriever(jobID).stop();
-			}
 			super.shutDown();
-		} catch (Throwable e) {
+		} 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);
 		}
@@ -127,78 +102,34 @@ public class ResourceManager extends RpcEndpoint<ResourceManagerGateway> impleme
 	 */
 	@VisibleForTesting
 	UUID getLeaderSessionID() {
-		return this.leaderSessionID;
+		return leaderSessionID;
 	}
 
 	/**
 	 * 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
+	 * @param jobMasterRegistration Job master registration information
 	 * @return Future registration response
 	 */
 	@RpcMethod
-	public Future<RegistrationResponse> registerJobMaster(
-		final UUID resourceManagerLeaderId, final UUID jobMasterLeaderId,
-		final String jobMasterAddress, final JobID jobID) {
-
-		checkNotNull(resourceManagerLeaderId);
-		checkNotNull(jobMasterAddress);
-		checkNotNull(jobID);
+	public Future<RegistrationResponse> registerJobMaster(JobMasterRegistration jobMasterRegistration) {
+		Future<JobMasterGateway> jobMasterFuture = getRpcService().connect(jobMasterRegistration.getAddress(), JobMasterGateway.class);
 
-		// TODO mxm The leader retrieval needs to be split up in an async part which runs outside the main execution thread
-		// The state updates should be performed inside the main thread
-
-		final FlinkCompletableFuture<RegistrationResponse> future = new FlinkCompletableFuture<>();
-
-		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);
-			future.complete(new RegistrationResponse.Decline("Invalid leader session id"));
-			return future;
-		}
-
-		final LeaderConnectionInfo jobMasterLeaderInfo;
-		try {
-			jobMasterLeaderInfo = LeaderRetrievalUtils.retrieveLeaderConnectionInfo(
-				highAvailabilityServices.getJobMasterLeaderRetriever(jobID), new FiniteDuration(5, TimeUnit.SECONDS));
-		} catch (Exception e) {
-			LOG.warn("Failed to start JobMasterLeaderRetriever for JobID {}", jobID);
-			future.complete(new RegistrationResponse.Decline("Failed to retrieve JobMasterLeaderRetriever"));
-			return future;
-		}
-
-		if (!jobMasterLeaderId.equals(jobMasterLeaderInfo.getLeaderSessionID())) {
-			LOG.info("Declining registration request from non-leading JobManager {}", jobMasterAddress);
-			future.complete(new RegistrationResponse.Decline("JobManager is not leading"));
-			return future;
-		}
-
-		Future<JobMasterGateway> jobMasterGatewayFuture =
-			getRpcService().connect(jobMasterAddress, JobMasterGateway.class);
-
-		return jobMasterGatewayFuture.thenApplyAsync(new ApplyFunction<JobMasterGateway, RegistrationResponse>() {
+		return jobMasterFuture.map(new Mapper<JobMasterGateway, RegistrationResponse>() {
 			@Override
-			public RegistrationResponse apply(JobMasterGateway jobMasterGateway) {
-
-				final JobMasterLeaderListener jobMasterLeaderListener = new JobMasterLeaderListener(jobID);
-				try {
-					LeaderRetrievalService jobMasterLeaderRetriever = highAvailabilityServices.getJobMasterLeaderRetriever(jobID);
-					jobMasterLeaderRetriever.start(jobMasterLeaderListener);
-				} catch (Exception e) {
-					LOG.warn("Failed to start JobMasterLeaderRetriever for JobID {}", jobID);
-					return new RegistrationResponse.Decline("Failed to retrieve JobMasterLeaderRetriever");
-				}
-				jobMasterLeaderRetrievalListeners.add(jobMasterLeaderListener);
-				final JobMasterGateway existingGateway = jobMasterGateways.put(jobID, jobMasterGateway);
-				if (existingGateway != null) {
-					log.info("Replacing gateway for registered JobID {}.", jobID);
+			public RegistrationResponse apply(final JobMasterGateway jobMasterGateway) {
+				InstanceID instanceID;
+
+				if (jobMasterGateways.containsKey(jobMasterGateway)) {
+					instanceID = jobMasterGateways.get(jobMasterGateway);
+				} else {
+					instanceID = new InstanceID();
+					jobMasterGateways.put(jobMasterGateway, instanceID);
 				}
-				return new JobMasterRegistrationSuccess(5000);
+
+				return new TaskExecutorRegistrationSuccess(instanceID, 5000);
 			}
-		}, getMainThreadExecutor());
+		}, getMainThreadExecutionContext());
 	}
 
 	/**
@@ -208,104 +139,111 @@ public class ResourceManager extends RpcEndpoint<ResourceManagerGateway> impleme
 	 * @return Slot assignment
 	 */
 	@RpcMethod
-	public SlotRequestReply requestSlot(SlotRequest slotRequest) {
-		final JobID jobId = slotRequest.getJobId();
-		final JobMasterGateway jobMasterGateway = jobMasterGateways.get(jobId);
-
-		if (jobMasterGateway != null) {
-			return slotManager.requestSlot(slotRequest);
-		} else {
-			LOG.info("Ignoring slot request for unknown JobMaster with JobID {}", jobId);
-			return new SlotRequestRejected(slotRequest.getAllocationId());
-		}
+	public SlotAssignment requestSlot(SlotRequest slotRequest) {
+		System.out.println("SlotRequest: " + slotRequest);
+		return new SlotAssignment();
 	}
 
 
 	/**
-	 * @param resourceManagerLeaderId The fencing token for the ResourceManager leader
-	 * @param taskExecutorAddress     The address of the TaskExecutor that registers
-	 * @param resourceID              The resource ID of the TaskExecutor that registers
+	 * Register a {@link org.apache.flink.runtime.taskexecutor.TaskExecutor} at the resource manager
+	 *
+	 * @param resourceManagerLeaderId  The fencing token for the ResourceManager leader
+	 * @param taskExecutorAddress      The address of the TaskExecutor that registers
+	 * @param resourceID               The resource ID of the TaskExecutor that registers
+	 *
 	 * @return The response by the ResourceManager.
 	 */
 	@RpcMethod
-	public RegistrationResponse registerTaskExecutor(
-		UUID resourceManagerLeaderId,
-		String taskExecutorAddress,
-		ResourceID resourceID) {
+	public Future<RegistrationResponse> registerTaskExecutor(
+		final UUID resourceManagerLeaderId,
+		final String taskExecutorAddress,
+		final ResourceID resourceID) {
 
-		return new TaskExecutorRegistrationSuccess(new InstanceID(), 5000);
-	}
+		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);
+			return Futures.failed(new LeaderSessionIDException(leaderSessionID, resourceManagerLeaderId));
+		}
 
+		Future<TaskExecutorGateway> taskExecutorGatewayFuture = getRpcService().connect(taskExecutorAddress, TaskExecutorGateway.class);
 
-	// ------------------------------------------------------------------------
-	//  Leader Contender
-	// ------------------------------------------------------------------------
+		return taskExecutorGatewayFuture.map(new Mapper<TaskExecutorGateway, RegistrationResponse>() {
 
-	/**
-	 * Callback method when current resourceManager is granted leadership
-	 *
-	 * @param leaderSessionID unique leadershipID
-	 */
-	@Override
-	public void grantLeadership(final UUID leaderSessionID) {
-		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);
-				// notify SlotManager
-				slotManager.setLeaderUUID(leaderSessionID);
-				ResourceManager.this.leaderSessionID = leaderSessionID;
-			}
-		});
-	}
+			public RegistrationResponse apply(final TaskExecutorGateway taskExecutorGateway) {
+				InstanceID instanceID = null;
+				TaskExecutorRegistration taskExecutorRegistration = startedTaskExecutorGateways.get(resourceID);
+				if(taskExecutorRegistration != null) {
+					log.warn("Receive a duplicate registration from TaskExecutor {} at ({})", resourceID, taskExecutorAddress);
+					instanceID = taskExecutorRegistration.getInstanceID();
+				} else {
+					instanceID = new InstanceID();
+					startedTaskExecutorGateways.put(resourceID, new TaskExecutorRegistration(taskExecutorGateway, instanceID));
+				}
 
-	/**
-	 * Callback method when current resourceManager lose leadership.
-	 */
-	@Override
-	public void revokeLeadership() {
-		runAsync(new Runnable() {
-			@Override
-			public void run() {
-				log.info("ResourceManager {} was revoked leadership.", getAddress());
-				jobMasterGateways.clear();
-				slotManager.clearState();
-				leaderSessionID = null;
+				return new TaskExecutorRegistrationSuccess(instanceID, 5000);
 			}
-		});
+		}, getMainThreadExecutionContext());
 	}
 
-	/**
-	 * Handles error occurring in the leader election service
-	 *
-	 * @param exception Exception being thrown in the leader election service
-	 */
-	@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();
-	}
 
-	private static class JobMasterLeaderListener implements LeaderRetrievalListener {
+	private class ResourceManagerLeaderContender implements LeaderContender {
 
-		private final JobID jobID;
-		private UUID leaderID;
+		/**
+		 * Callback method when current resourceManager is granted leadership
+		 *
+		 * @param leaderSessionID unique leadershipID
+		 */
+		@Override
+		public void grantLeadership(final UUID leaderSessionID) {
+			runAsync(new Runnable() {
+				@Override
+				public void run() {
+					log.info("ResourceManager {} was granted leadership with leader session ID {}", getAddress(), leaderSessionID);
+					ResourceManager.this.leaderSessionID = leaderSessionID;
+					// confirming the leader session ID might be blocking,
+					leaderElectionService.confirmLeaderSessionID(leaderSessionID);
+				}
+			});
+		}
 
-		private JobMasterLeaderListener(JobID jobID) {
-			this.jobID = jobID;
+		/**
+		 * Callback method when current resourceManager lose leadership.
+		 */
+		@Override
+		public void revokeLeadership() {
+			runAsync(new Runnable() {
+				@Override
+				public void run() {
+					log.info("ResourceManager {} was revoked leadership.", getAddress());
+					jobMasterGateways.clear();
+					startedTaskExecutorGateways.clear();
+					leaderSessionID = null;
+				}
+			});
 		}
 
 		@Override
-		public void notifyLeaderAddress(final String leaderAddress, final UUID leaderSessionID) {
-			this.leaderID = leaderSessionID;
+		public String getAddress() {
+			return ResourceManager.this.getAddress();
 		}
 
+		/**
+		 * Handles error occurring in the leader election service
+		 *
+		 * @param exception Exception being thrown in the leader election service
+		 */
 		@Override
 		public void handleError(final Exception exception) {
-			// TODO
+			runAsync(new Runnable() {
+				@Override
+				public void run() {
+					log.error("ResourceManager received an error from the LeaderElectionService.", exception);
+					// terminate ResourceManager in case of an error
+					shutDown();
+				}
+			});
 		}
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/c9764c8f/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 1ee11a1..30a096f 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
@@ -18,13 +18,14 @@
 
 package org.apache.flink.runtime.resourcemanager;
 
-import org.apache.flink.api.common.JobID;
 import org.apache.flink.runtime.clusterframework.types.ResourceID;
-import org.apache.flink.runtime.concurrent.Future;
 import org.apache.flink.runtime.rpc.RpcGateway;
 import org.apache.flink.runtime.rpc.RpcTimeout;
 import org.apache.flink.runtime.jobmaster.JobMaster;
+
 import org.apache.flink.runtime.registration.RegistrationResponse;
+import scala.concurrent.Future;
+import scala.concurrent.duration.FiniteDuration;
 
 import java.util.UUID;
 
@@ -36,18 +37,21 @@ public interface ResourceManagerGateway extends RpcGateway {
 	/**
 	 * 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
-	 * @param timeout                 Timeout for the future to complete
+	 * @param jobMasterRegistration Job master registration information
+	 * @param timeout Timeout for the future to complete
 	 * @return Future registration response
 	 */
 	Future<RegistrationResponse> registerJobMaster(
-		UUID resourceManagerLeaderId,
-		String jobMasterAddress,
-		JobID jobID,
-				@RpcTimeout Time timeout);
+		JobMasterRegistration jobMasterRegistration,
+		@RpcTimeout FiniteDuration timeout);
 
+	/**
+	 * Register a {@link JobMaster} at the resource manager.
+	 *
+	 * @param jobMasterRegistration Job master registration information
+	 * @return Future registration response
+	 */
+	Future<RegistrationResponse> registerJobMaster(JobMasterRegistration jobMasterRegistration);
 
 	/**
 	 * Requests a slot from the resource manager.
@@ -55,18 +59,21 @@ public interface ResourceManagerGateway extends RpcGateway {
 	 * @param slotRequest Slot request
 	 * @return Future slot assignment
 	 */
-	Future<SlotRequestReply> requestSlot(SlotRequest slotRequest);
+	Future<SlotAssignment> requestSlot(SlotRequest slotRequest);
 
 	/**
-	 * @param resourceManagerLeaderId The fencing token for the ResourceManager leader
-	 * @param taskExecutorAddress     The address of the TaskExecutor that registers
-	 * @param resourceID              The resource ID of the TaskExecutor that registers
-	 * @param timeout                 The timeout for the response.
+	 * Register a {@link org.apache.flink.runtime.taskexecutor.TaskExecutor} at the resource manager.
+	 *
+	 * @param resourceManagerLeaderId  The fencing token for the ResourceManager leader
+	 * @param taskExecutorAddress      The address of the TaskExecutor that registers
+	 * @param resourceID               The resource ID of the TaskExecutor that registers
+	 * @param timeout                  The timeout for the response.
+	 *
 	 * @return The future to the response by the ResourceManager.
 	 */
 	Future<RegistrationResponse> registerTaskExecutor(
-			UUID resourceManagerLeaderId,
-			String taskExecutorAddress,
-			ResourceID resourceID,
-			@RpcTimeout Time timeout);
+		UUID resourceManagerLeaderId,
+		String taskExecutorAddress,
+		ResourceID resourceID,
+		@RpcTimeout FiniteDuration timeout);
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/c9764c8f/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/TaskExecutorRegistration.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/TaskExecutorRegistration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/TaskExecutorRegistration.java
new file mode 100644
index 0000000..bd78a47
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/TaskExecutorRegistration.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.runtime.instance.InstanceID;
+import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway;
+
+import java.io.Serializable;
+
+/**
+ * This class is responsible for group the TaskExecutorGateway and the InstanceID of a registered task executor.
+ */
+public class TaskExecutorRegistration implements Serializable {
+
+	private static final long serialVersionUID = -2062957799469434614L;
+
+	private TaskExecutorGateway taskExecutorGateway;
+
+	private InstanceID instanceID;
+
+	public TaskExecutorRegistration(TaskExecutorGateway taskExecutorGateway,
+		InstanceID instanceID) {
+		this.taskExecutorGateway = taskExecutorGateway;
+		this.instanceID = instanceID;
+	}
+
+	public InstanceID getInstanceID() {
+		return instanceID;
+	}
+
+	public TaskExecutorGateway getTaskExecutorGateway() {
+		return taskExecutorGateway;
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/c9764c8f/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/exceptions/LeaderSessionIDException.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/exceptions/LeaderSessionIDException.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/exceptions/LeaderSessionIDException.java
index cd14a0d..d3ba9a9 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/exceptions/LeaderSessionIDException.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/exceptions/LeaderSessionIDException.java
@@ -19,6 +19,7 @@
 package org.apache.flink.runtime.rpc.exceptions;
 
 import java.util.UUID;
+
 import static org.apache.flink.util.Preconditions.checkNotNull;
 
 /**

http://git-wip-us.apache.org/repos/asf/flink/blob/c9764c8f/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTest.java
index 4d04001..b75d9b8 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTest.java
@@ -18,15 +18,14 @@
 
 package org.apache.flink.runtime.resourcemanager;
 
-import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
 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.rpc.TestingSerialRpcService;
 import org.apache.flink.runtime.registration.RegistrationResponse;
 import org.apache.flink.runtime.rpc.exceptions.LeaderSessionIDException;
+import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway;
+import org.apache.flink.runtime.taskexecutor.TaskExecutorRegistrationSuccess;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -37,8 +36,9 @@ import scala.concurrent.duration.FiniteDuration;
 import java.util.UUID;
 import java.util.concurrent.TimeUnit;
 
+import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
-import static org.mockito.Mockito.*;
+import static org.mockito.Mockito.mock;
 
 public class ResourceManagerTest {
 
@@ -55,105 +55,86 @@ public class ResourceManagerTest {
 	}
 
 	/**
-	 * Test receive normal registration from job master and receive duplicate registration from job master
+	 * Test receive normal registration from task executor and receive duplicate registration from task executor
 	 *
 	 * @throws Exception
 	 */
 	@Test
-	public void testRegisterJobMaster() throws Exception {
-		String jobMasterAddress = "/jobMasterAddress1";
-		JobID jobID = mockJobMaster(jobMasterAddress);
-		TestingLeaderElectionService resourceManagerLeaderElectionService = new TestingLeaderElectionService();
-		TestingLeaderRetrievalService jobMasterLeaderRetrievalService = new TestingLeaderRetrievalService();
-		final ResourceManager resourceManager = createAndStartResourceManager(resourceManagerLeaderElectionService, jobID, jobMasterLeaderRetrievalService);
-		final UUID leaderSessionId = grantResourceManagerLeadership(resourceManagerLeaderElectionService);
+	public void testRegisterTaskExecutor() throws Exception {
+		String taskExecutorAddress = "/taskExecutor1";
+		ResourceID taskExecutorResourceID = mockTaskExecutor(taskExecutorAddress);
+		TestingLeaderElectionService leaderElectionService = new TestingLeaderElectionService();
+		final ResourceManager resourceManager = createAndStartResourceManager(leaderElectionService);
+		final UUID leaderSessionId = grantResourceManagerLeadership(leaderElectionService);
 
 		// test response successful
-		Future<RegistrationResponse> successfulFuture = resourceManager.registerJobMaster(leaderSessionId, jobMasterAddress, jobID);
+		Future<RegistrationResponse> successfulFuture = resourceManager.registerTaskExecutor(leaderSessionId, taskExecutorAddress, taskExecutorResourceID);
 		RegistrationResponse response = Await.result(successfulFuture, new FiniteDuration(0, TimeUnit.SECONDS));
-		assertTrue(response instanceof JobMasterRegistrationSuccess);
+		assertTrue(response instanceof TaskExecutorRegistrationSuccess);
+
+		// test response successful with previous instanceID when receive duplicate registration from taskExecutor
+		Future<RegistrationResponse> duplicateFuture = resourceManager.registerTaskExecutor(leaderSessionId, taskExecutorAddress, taskExecutorResourceID);
+		RegistrationResponse duplicateResponse = Await.result(duplicateFuture, new FiniteDuration(0, TimeUnit.SECONDS));
+		assertTrue(duplicateResponse instanceof TaskExecutorRegistrationSuccess);
+		assertEquals(((TaskExecutorRegistrationSuccess) response).getRegistrationId(), ((TaskExecutorRegistrationSuccess) duplicateResponse).getRegistrationId());
 	}
 
 	/**
-	 * Test receive registration with unmatched leadershipId from job master
+	 * Test receive registration with unmatched leadershipId from task executor
 	 *
 	 * @throws Exception
 	 */
 	@Test(expected = LeaderSessionIDException.class)
-	public void testRegisterJobMasterWithUnmatchedLeaderSessionId() throws Exception {
-		String jobMasterAddress = "/jobMasterAddress1";
-		JobID jobID = mockJobMaster(jobMasterAddress);
-		TestingLeaderElectionService resourceManagerLeaderElectionService = new TestingLeaderElectionService();
-		TestingLeaderRetrievalService jobMasterLeaderRetrievalService = new TestingLeaderRetrievalService();
-		final ResourceManager resourceManager = createAndStartResourceManager(resourceManagerLeaderElectionService, jobID, jobMasterLeaderRetrievalService);
-		final UUID leaderSessionId = grantResourceManagerLeadership(resourceManagerLeaderElectionService);
-
-		// test throw exception when receive a registration from job master which takes unmatched leaderSessionId
+	public void testRegisterTaskExecutorWithUnmatchedLeaderSessionId() throws Exception {
+		String taskExecutorAddress = "/taskExecutor1";
+		ResourceID taskExecutorResourceID = mockTaskExecutor(taskExecutorAddress);
+		TestingLeaderElectionService leaderElectionService = new TestingLeaderElectionService();
+		final ResourceManager resourceManager = createAndStartResourceManager(leaderElectionService);
+		final UUID leaderSessionId = grantResourceManagerLeadership(leaderElectionService);
+
+		// test throw exception when receive a registration from taskExecutor which takes unmatched leaderSessionId
 		UUID differentLeaderSessionID = UUID.randomUUID();
-		Future<RegistrationResponse> unMatchedLeaderFuture = resourceManager.registerJobMaster(differentLeaderSessionID, jobMasterAddress, jobID);
+		Future<RegistrationResponse> unMatchedLeaderFuture = resourceManager.registerTaskExecutor(differentLeaderSessionID, taskExecutorAddress, taskExecutorResourceID);
 		Await.result(unMatchedLeaderFuture, new FiniteDuration(200, TimeUnit.MILLISECONDS));
 	}
 
 	/**
-	 * Test receive registration with invalid address from job master
+	 * Test receive registration with invalid address from task executor
 	 *
 	 * @throws Exception
 	 */
 	@Test(expected = Exception.class)
-	public void testRegisterJobMasterFromInvalidAddress() throws Exception {
-		String jobMasterAddress = "/jobMasterAddress1";
-		JobID jobID = mockJobMaster(jobMasterAddress);
-		TestingLeaderElectionService resourceManagerLeaderElectionService = new TestingLeaderElectionService();
-		TestingLeaderRetrievalService jobMasterLeaderRetrievalService = new TestingLeaderRetrievalService();
-		final ResourceManager resourceManager = createAndStartResourceManager(resourceManagerLeaderElectionService, jobID, jobMasterLeaderRetrievalService);
-		final UUID leaderSessionId = grantResourceManagerLeadership(resourceManagerLeaderElectionService);
-
-		// test throw exception when receive a registration from job master which takes invalid address
-		String invalidAddress = "/jobMasterAddress2";
-		Future<RegistrationResponse> invalidAddressFuture = resourceManager.registerJobMaster(leaderSessionId, invalidAddress, jobID);
+	public void testRegisterTaskExecutorFromInvalidAddress() throws Exception {
+		String taskExecutorAddress = "/taskExecutor1";
+		ResourceID taskExecutorResourceID = mockTaskExecutor(taskExecutorAddress);
+		TestingLeaderElectionService leaderElectionService = new TestingLeaderElectionService();
+		final ResourceManager resourceManager = createAndStartResourceManager(leaderElectionService);
+		final UUID leaderSessionId = grantResourceManagerLeadership(leaderElectionService);
+
+		// test throw exception when receive a registration from taskExecutor which takes invalid address
+		String invalidAddress = "/taskExecutor2";
+		Future<RegistrationResponse> invalidAddressFuture = resourceManager.registerTaskExecutor(leaderSessionId, invalidAddress, taskExecutorResourceID);
 		Await.result(invalidAddressFuture, new FiniteDuration(200, TimeUnit.MILLISECONDS));
 	}
 
-	/**
-	 * Check and verify return RegistrationResponse.Decline when failed to start a job master Leader retrieval listener
-	 *
-	 * @throws Exception
-	 */
-	@Test
-	public void testRegisterJobMasterWithFailureLeaderListener() throws Exception {
-		String jobMasterAddress = "/jobMasterAddress1";
-		JobID jobID = mockJobMaster(jobMasterAddress);
-		TestingLeaderElectionService resourceManagerLeaderElectionService = new TestingLeaderElectionService();
-		TestingLeaderRetrievalService jobMasterLeaderRetrievalService = new TestingLeaderRetrievalService();
-		final ResourceManager resourceManager = createAndStartResourceManager(resourceManagerLeaderElectionService, jobID, jobMasterLeaderRetrievalService);
-		final UUID leaderSessionId = grantResourceManagerLeadership(resourceManagerLeaderElectionService);
-
-		JobID unknownJobIDToHAServices = new JobID();
-		// verify return RegistrationResponse.Decline when failed to start a job master Leader retrieval listener
-		Future<RegistrationResponse> declineFuture = resourceManager.registerJobMaster(leaderSessionId, jobMasterAddress, unknownJobIDToHAServices);
-		RegistrationResponse response = Await.result(declineFuture, new FiniteDuration(0, TimeUnit.SECONDS));
-		assertTrue(response instanceof RegistrationResponse.Decline);
-	}
-
-	private JobID mockJobMaster(String jobMasterAddress) {
-		JobID jobID = new JobID();
-		JobMasterGateway jobMasterGateway = mock(JobMasterGateway.class);
-		rpcService.registerGateway(jobMasterAddress, jobMasterGateway);
-		return jobID;
+	private ResourceID mockTaskExecutor(String taskExecutorAddress) {
+		TaskExecutorGateway taskExecutorGateway = mock(TaskExecutorGateway.class);
+		ResourceID taskExecutorResourceID = ResourceID.generate();
+		rpcService.registerGateway(taskExecutorAddress, taskExecutorGateway);
+		return taskExecutorResourceID;
 	}
 
-	private ResourceManager createAndStartResourceManager(TestingLeaderElectionService resourceManagerLeaderElectionService, JobID jobID, TestingLeaderRetrievalService jobMasterLeaderRetrievalService) {
+	private ResourceManager createAndStartResourceManager(TestingLeaderElectionService leaderElectionService) {
 		TestingHighAvailabilityServices highAvailabilityServices = new TestingHighAvailabilityServices();
-		highAvailabilityServices.setResourceManagerLeaderElectionService(resourceManagerLeaderElectionService);
-		highAvailabilityServices.setJobMasterLeaderRetriever(jobID, jobMasterLeaderRetrievalService);
+		highAvailabilityServices.setResourceManagerLeaderElectionService(leaderElectionService);
 		ResourceManager resourceManager = new ResourceManager(rpcService, highAvailabilityServices);
 		resourceManager.start();
 		return resourceManager;
 	}
 
-	private UUID grantResourceManagerLeadership(TestingLeaderElectionService resourceManagerLeaderElectionService) {
+	private UUID grantResourceManagerLeadership(TestingLeaderElectionService leaderElectionService) {
 		UUID leaderSessionId = UUID.randomUUID();
-		resourceManagerLeaderElectionService.isLeader(leaderSessionId);
+		leaderElectionService.isLeader(leaderSessionId);
 		return leaderSessionId;
 	}
 


[14/50] [abbrv] flink git commit: [FLINK-4537] rebase and refine

Posted by se...@apache.org.
[FLINK-4537] rebase and refine


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

Branch: refs/heads/flip-6
Commit: 34a6854b44db8e949a02700deff18475d023b7bc
Parents: efc7de5
Author: Maximilian Michels <mx...@apache.org>
Authored: Wed Sep 21 14:13:12 2016 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Oct 14 15:14:40 2016 +0200

----------------------------------------------------------------------
 .../resourcemanager/JobMasterRegistration.java  |  52 +++---
 .../resourcemanager/ResourceManager.java        | 165 ++++++++-----------
 .../slotmanager/SlotManager.java                |  29 +++-
 3 files changed, 110 insertions(+), 136 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/34a6854b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/JobMasterRegistration.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/JobMasterRegistration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/JobMasterRegistration.java
index 7b8ec70..981441f 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/JobMasterRegistration.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/JobMasterRegistration.java
@@ -18,59 +18,47 @@
 
 package org.apache.flink.runtime.resourcemanager;
 
-<<<<<<< HEAD
 import org.apache.flink.api.common.JobID;
-=======
 import org.apache.flink.runtime.jobmaster.JobMasterGateway;
->>>>>>> db98efb... rsourceManager registration with JobManager
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener;
 
-import java.io.Serializable;
 import java.util.UUID;
 
-import static org.apache.flink.util.Preconditions.checkNotNull;
-
 /**
  * This class is responsible for group the JobMasterGateway and the LeaderSessionID of a registered job master
  */
-public class JobMasterRegistration implements Serializable {
+public class JobMasterRegistration implements LeaderRetrievalListener {
 
-<<<<<<< HEAD
-	private final String address;
+	private final JobMasterGateway gateway;
 	private final JobID jobID;
+	private final UUID leaderSessionID;
+	private LeaderRetrievalListener retriever;
 
-	public JobMasterRegistration(String address, JobID jobID) {
-		this.address = address;
+	public JobMasterRegistration(JobMasterGateway gateway, JobID jobID, UUID leaderSessionID) {
+		this.gateway = gateway;
 		this.jobID = jobID;
-=======
-	private static final long serialVersionUID = -2316627821716999527L;
-
-	private final JobMasterGateway jobMasterGateway;
-
-	private UUID jobMasterLeaderSessionID;
-
-	public JobMasterRegistration(JobMasterGateway jobMasterGateway) {
-		this.jobMasterGateway = checkNotNull(jobMasterGateway);
+		this.leaderSessionID = leaderSessionID;
 	}
 
-	public JobMasterRegistration(JobMasterGateway jobMasterGateway, UUID jobMasterLeaderSessionID) {
-		this.jobMasterGateway = checkNotNull(jobMasterGateway);
-		this.jobMasterLeaderSessionID = jobMasterLeaderSessionID;
+	public JobMasterGateway getGateway() {
+		return gateway;
 	}
 
-	public JobMasterGateway getJobMasterGateway() {
-		return jobMasterGateway;
+	public UUID getLeaderSessionID() {
+		return leaderSessionID;
 	}
 
-	public void setJobMasterLeaderSessionID(UUID leaderSessionID) {
-		this.jobMasterLeaderSessionID = jobMasterLeaderSessionID;
->>>>>>> db98efb... rsourceManager registration with JobManager
+	public JobID getJobID() {
+		return jobID;
 	}
 
-	public UUID getJobMasterLeaderSessionID() {
-		return jobMasterLeaderSessionID;
+	@Override
+	public void notifyLeaderAddress(String leaderAddress, UUID leaderSessionID) {
+		
 	}
 
-	public JobID getJobID() {
-		return jobID;
+	@Override
+	public void handleError(Exception exception) {
+
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/34a6854b/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 8be1455..aae4874 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
@@ -18,7 +18,6 @@
 
 package org.apache.flink.runtime.resourcemanager;
 
-import akka.dispatch.Futures;
 import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.runtime.clusterframework.types.ResourceID;
@@ -29,26 +28,31 @@ 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;
-<<<<<<< HEAD
-import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager;
-=======
 import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener;
->>>>>>> db98efb... rsourceManager registration with JobManager
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager;
 import org.apache.flink.runtime.rpc.RpcMethod;
 import org.apache.flink.runtime.rpc.RpcEndpoint;
 import org.apache.flink.runtime.rpc.RpcService;
 import org.apache.flink.runtime.jobmaster.JobMaster;
 import org.apache.flink.runtime.jobmaster.JobMasterGateway;
-import org.apache.flink.runtime.rpc.exceptions.LeaderSessionIDException;
 import org.apache.flink.runtime.taskexecutor.TaskExecutorRegistrationSuccess;
 import org.apache.flink.runtime.registration.RegistrationResponse;
 
+import org.apache.flink.runtime.concurrent.Future;
+
+import org.apache.flink.runtime.util.LeaderConnectionInfo;
+import org.apache.flink.runtime.util.LeaderRetrievalUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import scala.concurrent.duration.FiniteDuration;
 
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.Map;
+import java.util.Set;
 import java.util.UUID;
+import java.util.concurrent.TimeUnit;
 
 import static org.apache.flink.util.Preconditions.checkNotNull;
 
@@ -62,17 +66,13 @@ import static org.apache.flink.util.Preconditions.checkNotNull;
  *     <li>{@link #requestSlot(SlotRequest)} requests a slot from the resource manager</li>
  * </ul>
  */
-<<<<<<< HEAD
 public class ResourceManager extends RpcEndpoint<ResourceManagerGateway> implements LeaderContender {
 
 	private final Logger LOG = LoggerFactory.getLogger(getClass());
 
 	private final Map<JobID, JobMasterGateway> jobMasterGateways;
-=======
-public class ResourceManager extends RpcEndpoint<ResourceManagerGateway> {
-	/** the mapping relationship of JobID and JobMasterGateway */
-	private final Map<JobID, JobMasterRegistration> jobMasters;
->>>>>>> db98efb... rsourceManager registration with JobManager
+
+	private final Set<LeaderRetrievalListener> jobMasterLeaderRetrievalListeners;
 
 	private final HighAvailabilityServices highAvailabilityServices;
 
@@ -88,12 +88,9 @@ public class ResourceManager extends RpcEndpoint<ResourceManagerGateway> {
 			SlotManager slotManager) {
 		super(rpcService);
 		this.highAvailabilityServices = checkNotNull(highAvailabilityServices);
-<<<<<<< HEAD
 		this.jobMasterGateways = new HashMap<>();
 		this.slotManager = slotManager;
-=======
-		this.jobMasters = new HashMap<>(16);
->>>>>>> db98efb... rsourceManager registration with JobManager
+		this.jobMasterLeaderRetrievalListeners = new HashSet<>();
 	}
 
 	@Override
@@ -113,7 +110,7 @@ public class ResourceManager extends RpcEndpoint<ResourceManagerGateway> {
 	public void shutDown() {
 		try {
 			leaderElectionService.stop();
-			for(JobID jobID : jobMasters.keySet()) {
+			for(JobID jobID : jobMasterGateways.keySet()) {
 				highAvailabilityServices.getJobMasterLeaderRetriever(jobID).stop();
 			}
 			super.shutDown();
@@ -142,52 +139,64 @@ public class ResourceManager extends RpcEndpoint<ResourceManagerGateway> {
 	 * @return Future registration response
 	 */
 	@RpcMethod
-<<<<<<< HEAD
-	public Future<RegistrationResponse> registerJobMaster(JobMasterRegistration jobMasterRegistration) {
-		final Future<JobMasterGateway> jobMasterFuture =
-			getRpcService().connect(jobMasterRegistration.getAddress(), JobMasterGateway.class);
-		final JobID jobID = jobMasterRegistration.getJobID();
-=======
-	public Future<RegistrationResponse> registerJobMaster(UUID resourceManagerLeaderId, final String jobMasterAddress, final JobID jobID) {
+	public Future<RegistrationResponse> registerJobMaster(
+		final UUID resourceManagerLeaderId, final UUID jobMasterLeaderId,
+		final String jobMasterAddress, final JobID jobID) {
+
+		checkNotNull(resourceManagerLeaderId);
+		checkNotNull(jobMasterAddress);
+		checkNotNull(jobID);
+
+		// TODO mxm The leader retrieval needs to be split up in an async part which runs outside the main execution thread
+		// The state updates should be performed inside the main thread
+
+		final FlinkCompletableFuture<RegistrationResponse> future = new FlinkCompletableFuture<>();
 
 		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  {}",
+			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 Futures.failed(new LeaderSessionIDException(leaderSessionID, resourceManagerLeaderId));
+			future.complete(new RegistrationResponse.Decline("Invalid leader session id"));
+			return future;
 		}
 
-		Future<JobMasterGateway> jobMasterFuture = getRpcService().connect(jobMasterAddress, JobMasterGateway.class);
->>>>>>> db98efb... rsourceManager registration with JobManager
+		final LeaderConnectionInfo jobMasterLeaderInfo;
+		try {
+			jobMasterLeaderInfo = LeaderRetrievalUtils.retrieveLeaderConnectionInfo(
+				highAvailabilityServices.getJobMasterLeaderRetriever(jobID), new FiniteDuration(5, TimeUnit.SECONDS));
+		} catch (Exception e) {
+			LOG.warn("Failed to start JobMasterLeaderRetriever for JobID {}", jobID);
+			future.complete(new RegistrationResponse.Decline("Failed to retrieve JobMasterLeaderRetriever"));
+			return future;
+		}
+
+		if (!jobMasterLeaderId.equals(jobMasterLeaderInfo.getLeaderSessionID())) {
+			LOG.info("Declining registration request from non-leading JobManager {}", jobMasterAddress);
+			future.complete(new RegistrationResponse.Decline("JobManager is not leading"));
+			return future;
+		}
 
-		return jobMasterFuture.thenApplyAsync(new ApplyFunction<JobMasterGateway, RegistrationResponse>() {
+		Future<JobMasterGateway> jobMasterGatewayFuture =
+			getRpcService().connect(jobMasterAddress, JobMasterGateway.class);
+
+		return jobMasterGatewayFuture.thenApplyAsync(new ApplyFunction<JobMasterGateway, RegistrationResponse>() {
 			@Override
 			public RegistrationResponse apply(JobMasterGateway jobMasterGateway) {
-<<<<<<< HEAD
+
+				final JobMasterLeaderListener jobMasterLeaderListener = new JobMasterLeaderListener(jobID);
+				try {
+					LeaderRetrievalService jobMasterLeaderRetriever = highAvailabilityServices.getJobMasterLeaderRetriever(jobID);
+					jobMasterLeaderRetriever.start(jobMasterLeaderListener);
+				} catch (Exception e) {
+					LOG.warn("Failed to start JobMasterLeaderRetriever for JobID {}", jobID);
+					return new RegistrationResponse.Decline("Failed to retrieve JobMasterLeaderRetriever");
+				}
+				jobMasterLeaderRetrievalListeners.add(jobMasterLeaderListener);
 				final JobMasterGateway existingGateway = jobMasterGateways.put(jobID, jobMasterGateway);
 				if (existingGateway != null) {
-					LOG.info("Replacing existing gateway {} for JobID {} with  {}.",
-						existingGateway, jobID, jobMasterGateway);
-				}
-				return new RegistrationResponse(true);
-=======
-				if (jobMasters.containsKey(jobID)) {
-					JobMasterRegistration jobMasterRegistration = new JobMasterRegistration(jobMasterGateway, jobMasters.get(jobID).getJobMasterLeaderSessionID());
-					jobMasters.put(jobID, jobMasterRegistration);
 					log.info("Replacing gateway for registered JobID {}.", jobID);
-				} else {
-					JobMasterRegistration jobMasterRegistration = new JobMasterRegistration(jobMasterGateway);
-					jobMasters.put(jobID, jobMasterRegistration);
-					try {
-						highAvailabilityServices.getJobMasterLeaderRetriever(jobID).start(new JobMasterLeaderListener(jobID));
-					} catch(Throwable e) {
-						log.warn("Decline registration from JobMaster {} at ({}) because fail to get the leader retriever for the given job JobMaster",
-							jobID, jobMasterAddress);
-						return new RegistrationResponse.Decline("Fail to get the leader retriever for the given JobMaster");
-					}
 				}
-
 				return new JobMasterRegistrationSuccess(5000);
->>>>>>> db98efb... rsourceManager registration with JobManager
 			}
 		}, getMainThreadExecutor());
 	}
@@ -228,26 +237,9 @@ public class ResourceManager extends RpcEndpoint<ResourceManagerGateway> {
 	}
 
 
-<<<<<<< HEAD
 	// ------------------------------------------------------------------------
 	//  Leader Contender
 	// ------------------------------------------------------------------------
-=======
-		/**
-		 * Callback method when current resourceManager lose leadership.
-		 */
-		@Override
-		public void revokeLeadership() {
-			runAsync(new Runnable() {
-				@Override
-				public void run() {
-					log.info("ResourceManager {} was revoked leadership.", getAddress());
-					jobMasters.clear();
-					leaderSessionID = null;
-				}
-			});
-		}
->>>>>>> db98efb... rsourceManager registration with JobManager
 
 	/**
 	 * Callback method when current resourceManager is granted leadership
@@ -263,7 +255,7 @@ public class ResourceManager extends RpcEndpoint<ResourceManagerGateway> {
 				// confirming the leader session ID might be blocking,
 				leaderElectionService.confirmLeaderSessionID(leaderSessionID);
 				// notify SlotManager
-				slotManager.notifyLeaderAddress(getAddress(), leaderSessionID);
+				slotManager.setLeaderUUID(leaderSessionID);
 				ResourceManager.this.leaderSessionID = leaderSessionID;
 			}
 		});
@@ -279,7 +271,8 @@ public class ResourceManager extends RpcEndpoint<ResourceManagerGateway> {
 			public void run() {
 				log.info("ResourceManager {} was revoked leadership.", getAddress());
 				jobMasterGateways.clear();
-				ResourceManager.this.leaderSessionID = null;
+				slotManager.clearState();
+				leaderSessionID = null;
 			}
 		});
 	}
@@ -291,20 +284,15 @@ public class ResourceManager extends RpcEndpoint<ResourceManagerGateway> {
 	 */
 	@Override
 	public void handleError(final Exception exception) {
-		runAsync(new Runnable() {
-			@Override
-			public void run() {
-				log.error("ResourceManager received an error from the LeaderElectionService.", exception);
-				// notify SlotManager
-				slotManager.handleError(exception);
-				// terminate ResourceManager in case of an error
-				shutDown();
-			}
-		});
+		log.error("ResourceManager received an error from the LeaderElectionService.", exception);
+		// terminate ResourceManager in case of an error
+		shutDown();
 	}
 
-	private class JobMasterLeaderListener implements LeaderRetrievalListener {
+	private static class JobMasterLeaderListener implements LeaderRetrievalListener {
+
 		private final JobID jobID;
+		private UUID leaderID;
 
 		private JobMasterLeaderListener(JobID jobID) {
 			this.jobID = jobID;
@@ -312,25 +300,12 @@ public class ResourceManager extends RpcEndpoint<ResourceManagerGateway> {
 
 		@Override
 		public void notifyLeaderAddress(final String leaderAddress, final UUID leaderSessionID) {
-			runAsync(new Runnable() {
-				@Override
-				public void run() {
-					log.info("A new leader for JobMaster {} is elected, address is {}, leaderSessionID is {}", jobID, leaderAddress, leaderSessionID);
-					// update job master leader session id
-					JobMasterRegistration jobMasterRegistration = jobMasters.get(jobID);
-					jobMasterRegistration.setJobMasterLeaderSessionID(leaderSessionID);
-				}
-			});
+			this.leaderID = leaderSessionID;
 		}
 
 		@Override
 		public void handleError(final Exception exception) {
-			runAsync(new Runnable() {
-				@Override
-				public void run() {
-					log.error("JobMasterLeaderListener received an error from the LeaderRetrievalService", exception);
-				}
-			});
+			// TODO
 		}
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/34a6854b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManager.java
index 97176b2..5d0013c 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManager.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManager.java
@@ -59,7 +59,7 @@ import static org.apache.flink.util.Preconditions.checkNotNull;
  * </ul>
  * <b>IMPORTANT:</b> This class is <b>Not Thread-safe</b>.
  */
-public abstract class SlotManager implements LeaderRetrievalListener {
+public abstract class SlotManager {
 
 	protected final Logger LOG = LoggerFactory.getLogger(getClass());
 
@@ -514,22 +514,33 @@ public abstract class SlotManager implements LeaderRetrievalListener {
 		public int size() {
 			return allocatedSlots.size();
 		}
+
+		public void clear() {
+			allocatedSlots.clear();
+			allocatedSlotsByAllocationId.clear();
+		}
+	}
+
+	/**
+	 * Clears the state of the SlotManager after leadership revokal
+	 */
+	public void clearState() {
+		taskManagerGateways.clear();
+		registeredSlots.clear();
+		pendingSlotRequests.clear();
+		freeSlots.clear();
+		allocationMap.clear();
+		leaderID = null;
 	}
 
 	// ------------------------------------------------------------------------
-	//  High availability
+	//  High availability (called by the ResourceManager)
 	// ------------------------------------------------------------------------
 
-	@Override
-	public void notifyLeaderAddress(String leaderAddress, UUID leaderSessionID) {
+	public void setLeaderUUID(UUID leaderSessionID) {
 		this.leaderID = leaderSessionID;
 	}
 
-	@Override
-	public void handleError(Exception exception) {
-		LOG.error("Slot Manager received an error from the leader service", exception);
-	}
-
 	// ------------------------------------------------------------------------
 	//  Testing utilities
 	// ------------------------------------------------------------------------


[16/50] [abbrv] flink git commit: [FLINK-4538][FLINK-4348] ResourceManager slot allocation protcol

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/74570d45/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/SlotManagerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/SlotManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/SlotManagerTest.java
deleted file mode 100644
index 52d9d06..0000000
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/SlotManagerTest.java
+++ /dev/null
@@ -1,538 +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.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.clusterframework.types.ResourceSlot;
-import org.apache.flink.runtime.clusterframework.types.SlotID;
-import org.apache.flink.runtime.taskexecutor.SlotStatus;
-import org.junit.Before;
-import org.junit.Test;
-
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.mockito.Mockito.mock;
-
-public class SlotManagerTest {
-
-	private static final double DEFAULT_TESTING_CPU_CORES = 1.0;
-
-	private static final long DEFAULT_TESTING_MEMORY = 512;
-
-	private static final ResourceProfile DEFAULT_TESTING_PROFILE =
-		new ResourceProfile(DEFAULT_TESTING_CPU_CORES, DEFAULT_TESTING_MEMORY);
-
-	private static final ResourceProfile DEFAULT_TESTING_BIG_PROFILE =
-		new ResourceProfile(DEFAULT_TESTING_CPU_CORES * 2, DEFAULT_TESTING_MEMORY * 2);
-
-	private ResourceManagerGateway resourceManagerGateway;
-
-	@Before
-	public void setUp() {
-		resourceManagerGateway = mock(ResourceManagerGateway.class);
-	}
-
-	/**
-	 * Tests that there are no free slots when we request, need to allocate from cluster manager master
-	 */
-	@Test
-	public void testRequestSlotWithoutFreeSlot() {
-		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
-		slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE));
-
-		assertEquals(0, slotManager.getAllocatedSlotCount());
-		assertEquals(0, slotManager.getFreeSlotCount());
-		assertEquals(1, slotManager.getPendingRequestCount());
-		assertEquals(1, slotManager.getAllocatedContainers().size());
-		assertEquals(DEFAULT_TESTING_PROFILE, slotManager.getAllocatedContainers().get(0));
-	}
-
-	/**
-	 * Tests that there are some free slots when we request, and the request is fulfilled immediately
-	 */
-	@Test
-	public void testRequestSlotWithFreeSlot() {
-		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
-
-		directlyProvideFreeSlots(slotManager, DEFAULT_TESTING_PROFILE, 1);
-		assertEquals(1, slotManager.getFreeSlotCount());
-
-		slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE));
-		assertEquals(1, slotManager.getAllocatedSlotCount());
-		assertEquals(0, slotManager.getFreeSlotCount());
-		assertEquals(0, slotManager.getPendingRequestCount());
-		assertEquals(0, slotManager.getAllocatedContainers().size());
-	}
-
-	/**
-	 * Tests that there are some free slots when we request, but none of them are suitable
-	 */
-	@Test
-	public void testRequestSlotWithoutSuitableSlot() {
-		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
-
-		directlyProvideFreeSlots(slotManager, DEFAULT_TESTING_PROFILE, 2);
-		assertEquals(2, slotManager.getFreeSlotCount());
-
-		slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_BIG_PROFILE));
-		assertEquals(0, slotManager.getAllocatedSlotCount());
-		assertEquals(2, slotManager.getFreeSlotCount());
-		assertEquals(1, slotManager.getPendingRequestCount());
-		assertEquals(1, slotManager.getAllocatedContainers().size());
-		assertEquals(DEFAULT_TESTING_BIG_PROFILE, slotManager.getAllocatedContainers().get(0));
-	}
-
-	/**
-	 * Tests that we send duplicated slot request
-	 */
-	@Test
-	public void testDuplicatedSlotRequest() {
-		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
-		directlyProvideFreeSlots(slotManager, DEFAULT_TESTING_PROFILE, 1);
-
-		SlotRequest request1 = new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE);
-		SlotRequest request2 = new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_BIG_PROFILE);
-
-		slotManager.requestSlot(request1);
-		slotManager.requestSlot(request2);
-		slotManager.requestSlot(request2);
-		slotManager.requestSlot(request1);
-
-		assertEquals(1, slotManager.getAllocatedSlotCount());
-		assertEquals(0, slotManager.getFreeSlotCount());
-		assertEquals(1, slotManager.getPendingRequestCount());
-		assertEquals(1, slotManager.getAllocatedContainers().size());
-		assertEquals(DEFAULT_TESTING_BIG_PROFILE, slotManager.getAllocatedContainers().get(0));
-	}
-
-	/**
-	 * Tests that we send multiple slot requests
-	 */
-	@Test
-	public void testRequestMultipleSlots() {
-		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
-		directlyProvideFreeSlots(slotManager, DEFAULT_TESTING_PROFILE, 5);
-
-		// request 3 normal slots
-		for (int i = 0; i < 3; ++i) {
-			slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE));
-		}
-
-		// request 2 big slots
-		for (int i = 0; i < 2; ++i) {
-			slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_BIG_PROFILE));
-		}
-
-		// request 1 normal slot again
-		slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE));
-
-		assertEquals(4, slotManager.getAllocatedSlotCount());
-		assertEquals(1, slotManager.getFreeSlotCount());
-		assertEquals(2, slotManager.getPendingRequestCount());
-		assertEquals(2, slotManager.getAllocatedContainers().size());
-		assertEquals(DEFAULT_TESTING_BIG_PROFILE, slotManager.getAllocatedContainers().get(0));
-		assertEquals(DEFAULT_TESTING_BIG_PROFILE, slotManager.getAllocatedContainers().get(1));
-	}
-
-	/**
-	 * Tests that a new slot appeared in SlotReport, and we used it to fulfill a pending request
-	 */
-	@Test
-	public void testNewlyAppearedFreeSlotFulfillPendingRequest() {
-		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
-		slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE));
-		assertEquals(1, slotManager.getPendingRequestCount());
-
-		SlotID slotId = SlotID.generate();
-		SlotStatus slotStatus = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE);
-		slotManager.updateSlotStatus(slotStatus);
-
-		assertEquals(1, slotManager.getAllocatedSlotCount());
-		assertEquals(0, slotManager.getFreeSlotCount());
-		assertEquals(0, slotManager.getPendingRequestCount());
-		assertTrue(slotManager.isAllocated(slotId));
-	}
-
-	/**
-	 * Tests that a new slot appeared in SlotReport, but we have no pending request
-	 */
-	@Test
-	public void testNewlyAppearedFreeSlot() {
-		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
-		SlotID slotId = SlotID.generate();
-		SlotStatus slotStatus = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE);
-		slotManager.updateSlotStatus(slotStatus);
-
-		assertEquals(0, slotManager.getAllocatedSlotCount());
-		assertEquals(1, slotManager.getFreeSlotCount());
-	}
-
-	/**
-	 * Tests that a new slot appeared in SlotReport, but it't not suitable for all the pending requests
-	 */
-	@Test
-	public void testNewlyAppearedFreeSlotNotMatchPendingRequests() {
-		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
-		slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_BIG_PROFILE));
-		assertEquals(1, slotManager.getPendingRequestCount());
-
-		SlotID slotId = SlotID.generate();
-		SlotStatus slotStatus = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE);
-		slotManager.updateSlotStatus(slotStatus);
-
-		assertEquals(0, slotManager.getAllocatedSlotCount());
-		assertEquals(1, slotManager.getFreeSlotCount());
-		assertEquals(1, slotManager.getPendingRequestCount());
-		assertFalse(slotManager.isAllocated(slotId));
-	}
-
-	/**
-	 * Tests that a new slot appeared in SlotReport, and it's been reported using by some job
-	 */
-	@Test
-	public void testNewlyAppearedInUseSlot() {
-		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
-
-		SlotID slotId = SlotID.generate();
-		SlotStatus slotStatus = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE, new AllocationID(), new JobID());
-		slotManager.updateSlotStatus(slotStatus);
-
-		assertEquals(1, slotManager.getAllocatedSlotCount());
-		assertEquals(0, slotManager.getFreeSlotCount());
-		assertTrue(slotManager.isAllocated(slotId));
-	}
-
-	/**
-	 * Tests that we had a slot in-use, and it's confirmed by SlotReport
-	 */
-	@Test
-	public void testExistingInUseSlotUpdateStatus() {
-		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
-		SlotRequest request = new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE);
-		slotManager.requestSlot(request);
-
-		// make this slot in use
-		SlotID slotId = SlotID.generate();
-		SlotStatus slotStatus = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE);
-		slotManager.updateSlotStatus(slotStatus);
-
-		assertEquals(1, slotManager.getAllocatedSlotCount());
-		assertEquals(0, slotManager.getFreeSlotCount());
-		assertTrue(slotManager.isAllocated(slotId));
-
-		// slot status is confirmed
-		SlotStatus slotStatus2 = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE,
-			request.getAllocationId(), request.getJobId());
-		slotManager.updateSlotStatus(slotStatus2);
-
-		assertEquals(1, slotManager.getAllocatedSlotCount());
-		assertEquals(0, slotManager.getFreeSlotCount());
-		assertTrue(slotManager.isAllocated(slotId));
-	}
-
-	/**
-	 * Tests that we had a slot in-use, but it's empty according to the SlotReport
-	 */
-	@Test
-	public void testExistingInUseSlotAdjustedToEmpty() {
-		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
-		SlotRequest request1 = new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE);
-		slotManager.requestSlot(request1);
-
-		// make this slot in use
-		SlotID slotId = SlotID.generate();
-		SlotStatus slotStatus = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE);
-		slotManager.updateSlotStatus(slotStatus);
-
-		// another request pending
-		SlotRequest request2 = new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE);
-		slotManager.requestSlot(request2);
-
-		assertEquals(1, slotManager.getAllocatedSlotCount());
-		assertEquals(0, slotManager.getFreeSlotCount());
-		assertEquals(1, slotManager.getPendingRequestCount());
-		assertTrue(slotManager.isAllocated(slotId));
-		assertTrue(slotManager.isAllocated(request1.getAllocationId()));
-
-
-		// but slot is reported empty again, request2 will be fulfilled, request1 will be missing
-		slotManager.updateSlotStatus(slotStatus);
-
-		assertEquals(1, slotManager.getAllocatedSlotCount());
-		assertEquals(0, slotManager.getFreeSlotCount());
-		assertEquals(0, slotManager.getPendingRequestCount());
-		assertTrue(slotManager.isAllocated(slotId));
-		assertTrue(slotManager.isAllocated(request2.getAllocationId()));
-	}
-
-	/**
-	 * Tests that we had a slot in use, and it's also reported in use by TaskManager, but the allocation
-	 * information didn't match.
-	 */
-	@Test
-	public void testExistingInUseSlotWithDifferentAllocationInfo() {
-		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
-		SlotRequest request = new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE);
-		slotManager.requestSlot(request);
-
-		// make this slot in use
-		SlotID slotId = SlotID.generate();
-		SlotStatus slotStatus = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE);
-		slotManager.updateSlotStatus(slotStatus);
-
-		assertEquals(1, slotManager.getAllocatedSlotCount());
-		assertEquals(0, slotManager.getFreeSlotCount());
-		assertEquals(0, slotManager.getPendingRequestCount());
-		assertTrue(slotManager.isAllocated(slotId));
-		assertTrue(slotManager.isAllocated(request.getAllocationId()));
-
-		SlotStatus slotStatus2 = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE, new AllocationID(), new JobID());
-		// update slot status with different allocation info
-		slotManager.updateSlotStatus(slotStatus2);
-
-		// original request is missing and won't be allocated
-		assertEquals(1, slotManager.getAllocatedSlotCount());
-		assertEquals(0, slotManager.getFreeSlotCount());
-		assertEquals(0, slotManager.getPendingRequestCount());
-		assertTrue(slotManager.isAllocated(slotId));
-		assertFalse(slotManager.isAllocated(request.getAllocationId()));
-		assertTrue(slotManager.isAllocated(slotStatus2.getAllocationID()));
-	}
-
-	/**
-	 * Tests that we had a free slot, and it's confirmed by SlotReport
-	 */
-	@Test
-	public void testExistingEmptySlotUpdateStatus() {
-		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
-		ResourceSlot slot = new ResourceSlot(SlotID.generate(), DEFAULT_TESTING_PROFILE);
-		slotManager.addFreeSlot(slot);
-
-		SlotStatus slotStatus = new SlotStatus(slot.getSlotId(), DEFAULT_TESTING_PROFILE);
-		slotManager.updateSlotStatus(slotStatus);
-
-		assertEquals(0, slotManager.getAllocatedSlotCount());
-		assertEquals(1, slotManager.getFreeSlotCount());
-		assertEquals(0, slotManager.getPendingRequestCount());
-	}
-
-	/**
-	 * Tests that we had a free slot, and it's reported in-use by TaskManager
-	 */
-	@Test
-	public void testExistingEmptySlotAdjustedToInUse() {
-		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
-		ResourceSlot slot = new ResourceSlot(SlotID.generate(), DEFAULT_TESTING_PROFILE);
-		slotManager.addFreeSlot(slot);
-
-		SlotStatus slotStatus = new SlotStatus(slot.getSlotId(), DEFAULT_TESTING_PROFILE,
-			new AllocationID(), new JobID());
-		slotManager.updateSlotStatus(slotStatus);
-
-		assertEquals(1, slotManager.getAllocatedSlotCount());
-		assertEquals(0, slotManager.getFreeSlotCount());
-		assertEquals(0, slotManager.getPendingRequestCount());
-		assertTrue(slotManager.isAllocated(slot.getSlotId()));
-	}
-
-	/**
-	 * Tests that we did some allocation but failed / rejected by TaskManager, request will retry
-	 */
-	@Test
-	public void testSlotAllocationFailedAtTaskManager() {
-		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
-		ResourceSlot slot = new ResourceSlot(SlotID.generate(), DEFAULT_TESTING_PROFILE);
-		slotManager.addFreeSlot(slot);
-
-		SlotRequest request = new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE);
-		slotManager.requestSlot(request);
-
-		assertEquals(1, slotManager.getAllocatedSlotCount());
-		assertEquals(0, slotManager.getFreeSlotCount());
-		assertEquals(0, slotManager.getPendingRequestCount());
-		assertTrue(slotManager.isAllocated(slot.getSlotId()));
-
-		slotManager.handleSlotRequestFailedAtTaskManager(request, slot.getSlotId());
-
-		assertEquals(1, slotManager.getAllocatedSlotCount());
-		assertEquals(0, slotManager.getFreeSlotCount());
-		assertEquals(0, slotManager.getPendingRequestCount());
-	}
-
-
-	/**
-	 * Tests that we did some allocation but failed / rejected by TaskManager, and slot is occupied by another request
-	 */
-	@Test
-	public void testSlotAllocationFailedAtTaskManagerOccupiedByOther() {
-		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
-		ResourceSlot slot = new ResourceSlot(SlotID.generate(), DEFAULT_TESTING_PROFILE);
-		slotManager.addFreeSlot(slot);
-
-		SlotRequest request = new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE);
-		slotManager.requestSlot(request);
-
-		// slot is set empty by heartbeat
-		SlotStatus slotStatus = new SlotStatus(slot.getSlotId(), slot.getResourceProfile());
-		slotManager.updateSlotStatus(slotStatus);
-
-		// another request took this slot
-		SlotRequest request2 = new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE);
-		slotManager.requestSlot(request2);
-
-		assertEquals(1, slotManager.getAllocatedSlotCount());
-		assertEquals(0, slotManager.getFreeSlotCount());
-		assertEquals(0, slotManager.getPendingRequestCount());
-		assertFalse(slotManager.isAllocated(request.getAllocationId()));
-		assertTrue(slotManager.isAllocated(request2.getAllocationId()));
-
-		// original request should be pended
-		slotManager.handleSlotRequestFailedAtTaskManager(request, slot.getSlotId());
-
-		assertEquals(1, slotManager.getAllocatedSlotCount());
-		assertEquals(0, slotManager.getFreeSlotCount());
-		assertEquals(1, slotManager.getPendingRequestCount());
-		assertFalse(slotManager.isAllocated(request.getAllocationId()));
-		assertTrue(slotManager.isAllocated(request2.getAllocationId()));
-	}
-
-	@Test
-	public void testNotifyTaskManagerFailure() {
-		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
-
-		ResourceID resource1 = ResourceID.generate();
-		ResourceID resource2 = ResourceID.generate();
-
-		ResourceSlot slot11 = new ResourceSlot(new SlotID(resource1, 1), DEFAULT_TESTING_PROFILE);
-		ResourceSlot slot12 = new ResourceSlot(new SlotID(resource1, 2), DEFAULT_TESTING_PROFILE);
-		ResourceSlot slot21 = new ResourceSlot(new SlotID(resource2, 1), DEFAULT_TESTING_PROFILE);
-		ResourceSlot slot22 = new ResourceSlot(new SlotID(resource2, 2), DEFAULT_TESTING_PROFILE);
-
-		slotManager.addFreeSlot(slot11);
-		slotManager.addFreeSlot(slot21);
-
-		slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE));
-		slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE));
-
-		assertEquals(2, slotManager.getAllocatedSlotCount());
-		assertEquals(0, slotManager.getFreeSlotCount());
-		assertEquals(0, slotManager.getPendingRequestCount());
-
-		slotManager.addFreeSlot(slot12);
-		slotManager.addFreeSlot(slot22);
-
-		assertEquals(2, slotManager.getAllocatedSlotCount());
-		assertEquals(2, slotManager.getFreeSlotCount());
-		assertEquals(0, slotManager.getPendingRequestCount());
-
-		slotManager.notifyTaskManagerFailure(resource2);
-
-		assertEquals(1, slotManager.getAllocatedSlotCount());
-		assertEquals(1, slotManager.getFreeSlotCount());
-		assertEquals(0, slotManager.getPendingRequestCount());
-
-		// notify an not exist resource failure
-		slotManager.notifyTaskManagerFailure(ResourceID.generate());
-
-		assertEquals(1, slotManager.getAllocatedSlotCount());
-		assertEquals(1, slotManager.getFreeSlotCount());
-		assertEquals(0, slotManager.getPendingRequestCount());
-	}
-
-	// ------------------------------------------------------------------------
-	//  testing utilities
-	// ------------------------------------------------------------------------
-
-	private void directlyProvideFreeSlots(
-		final SlotManager slotManager,
-		final ResourceProfile resourceProfile,
-		final int freeSlotNum)
-	{
-		for (int i = 0; i < freeSlotNum; ++i) {
-			slotManager.addFreeSlot(new ResourceSlot(SlotID.generate(), new ResourceProfile(resourceProfile)));
-		}
-	}
-
-	// ------------------------------------------------------------------------
-	//  testing classes
-	// ------------------------------------------------------------------------
-
-	private static class TestingSlotManager extends SlotManager {
-
-		private final List<ResourceProfile> allocatedContainers;
-
-		TestingSlotManager(ResourceManagerGateway resourceManagerGateway) {
-			super(resourceManagerGateway);
-			this.allocatedContainers = new LinkedList<>();
-		}
-
-		/**
-		 * Choose slot randomly if it matches requirement
-		 *
-		 * @param request   The slot request
-		 * @param freeSlots All slots which can be used
-		 * @return The chosen slot or null if cannot find a match
-		 */
-		@Override
-		protected ResourceSlot chooseSlotToUse(SlotRequest request, Map<SlotID, ResourceSlot> freeSlots) {
-			for (ResourceSlot slot : freeSlots.values()) {
-				if (slot.isMatchingRequirement(request.getResourceProfile())) {
-					return slot;
-				}
-			}
-			return null;
-		}
-
-		/**
-		 * Choose request randomly if offered slot can match its requirement
-		 *
-		 * @param offeredSlot     The free slot
-		 * @param pendingRequests All the pending slot requests
-		 * @return The chosen request's AllocationID or null if cannot find a match
-		 */
-		@Override
-		protected SlotRequest chooseRequestToFulfill(ResourceSlot offeredSlot,
-			Map<AllocationID, SlotRequest> pendingRequests)
-		{
-			for (Map.Entry<AllocationID, SlotRequest> pendingRequest : pendingRequests.entrySet()) {
-				if (offeredSlot.isMatchingRequirement(pendingRequest.getValue().getResourceProfile())) {
-					return pendingRequest.getValue();
-				}
-			}
-			return null;
-		}
-
-		@Override
-		protected void allocateContainer(ResourceProfile resourceProfile) {
-			allocatedContainers.add(resourceProfile);
-		}
-
-		List<ResourceProfile> getAllocatedContainers() {
-			return allocatedContainers;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/74570d45/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerTest.java
new file mode 100644
index 0000000..9ee9690
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerTest.java
@@ -0,0 +1,554 @@
+/*
+ * 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.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.clusterframework.types.ResourceSlot;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.resourcemanager.SlotRequest;
+import org.apache.flink.runtime.taskexecutor.SlotStatus;
+import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+
+public class SlotManagerTest {
+
+	private static final double DEFAULT_TESTING_CPU_CORES = 1.0;
+
+	private static final long DEFAULT_TESTING_MEMORY = 512;
+
+	private static final ResourceProfile DEFAULT_TESTING_PROFILE =
+		new ResourceProfile(DEFAULT_TESTING_CPU_CORES, DEFAULT_TESTING_MEMORY);
+
+	private static final ResourceProfile DEFAULT_TESTING_BIG_PROFILE =
+		new ResourceProfile(DEFAULT_TESTING_CPU_CORES * 2, DEFAULT_TESTING_MEMORY * 2);
+
+	private static TaskExecutorGateway taskExecutorGateway;
+
+	@BeforeClass
+	public static void setUp() {
+		taskExecutorGateway = Mockito.mock(TaskExecutorGateway.class);
+	}
+
+	/**
+	 * Tests that there are no free slots when we request, need to allocate from cluster manager master
+	 */
+	@Test
+	public void testRequestSlotWithoutFreeSlot() {
+		TestingSlotManager slotManager = new TestingSlotManager();
+		slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE));
+
+		assertEquals(0, slotManager.getAllocatedSlotCount());
+		assertEquals(0, slotManager.getFreeSlotCount());
+		assertEquals(1, slotManager.getPendingRequestCount());
+		assertEquals(1, slotManager.getAllocatedContainers().size());
+		assertEquals(DEFAULT_TESTING_PROFILE, slotManager.getAllocatedContainers().get(0));
+	}
+
+	/**
+	 * Tests that there are some free slots when we request, and the request is fulfilled immediately
+	 */
+	@Test
+	public void testRequestSlotWithFreeSlot() {
+		TestingSlotManager slotManager = new TestingSlotManager();
+
+		directlyProvideFreeSlots(slotManager, DEFAULT_TESTING_PROFILE, 1);
+		assertEquals(1, slotManager.getFreeSlotCount());
+
+		slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE));
+		assertEquals(1, slotManager.getAllocatedSlotCount());
+		assertEquals(0, slotManager.getFreeSlotCount());
+		assertEquals(0, slotManager.getPendingRequestCount());
+		assertEquals(0, slotManager.getAllocatedContainers().size());
+	}
+
+	/**
+	 * Tests that there are some free slots when we request, but none of them are suitable
+	 */
+	@Test
+	public void testRequestSlotWithoutSuitableSlot() {
+		TestingSlotManager slotManager = new TestingSlotManager();
+
+		directlyProvideFreeSlots(slotManager, DEFAULT_TESTING_PROFILE, 2);
+		assertEquals(2, slotManager.getFreeSlotCount());
+
+		slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_BIG_PROFILE));
+		assertEquals(0, slotManager.getAllocatedSlotCount());
+		assertEquals(2, slotManager.getFreeSlotCount());
+		assertEquals(1, slotManager.getPendingRequestCount());
+		assertEquals(1, slotManager.getAllocatedContainers().size());
+		assertEquals(DEFAULT_TESTING_BIG_PROFILE, slotManager.getAllocatedContainers().get(0));
+	}
+
+	/**
+	 * Tests that we send duplicated slot request
+	 */
+	@Test
+	public void testDuplicatedSlotRequest() {
+		TestingSlotManager slotManager = new TestingSlotManager();
+		directlyProvideFreeSlots(slotManager, DEFAULT_TESTING_PROFILE, 1);
+
+		SlotRequest request1 = new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE);
+		SlotRequest request2 = new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_BIG_PROFILE);
+
+		slotManager.requestSlot(request1);
+		slotManager.requestSlot(request2);
+		slotManager.requestSlot(request2);
+		slotManager.requestSlot(request1);
+
+		assertEquals(1, slotManager.getAllocatedSlotCount());
+		assertEquals(0, slotManager.getFreeSlotCount());
+		assertEquals(1, slotManager.getPendingRequestCount());
+		assertEquals(1, slotManager.getAllocatedContainers().size());
+		assertEquals(DEFAULT_TESTING_BIG_PROFILE, slotManager.getAllocatedContainers().get(0));
+	}
+
+	/**
+	 * Tests that we send multiple slot requests
+	 */
+	@Test
+	public void testRequestMultipleSlots() {
+		TestingSlotManager slotManager = new TestingSlotManager();
+		directlyProvideFreeSlots(slotManager, DEFAULT_TESTING_PROFILE, 5);
+
+		// request 3 normal slots
+		for (int i = 0; i < 3; ++i) {
+			slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE));
+		}
+
+		// request 2 big slots
+		for (int i = 0; i < 2; ++i) {
+			slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_BIG_PROFILE));
+		}
+
+		// request 1 normal slot again
+		slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE));
+
+		assertEquals(4, slotManager.getAllocatedSlotCount());
+		assertEquals(1, slotManager.getFreeSlotCount());
+		assertEquals(2, slotManager.getPendingRequestCount());
+		assertEquals(2, slotManager.getAllocatedContainers().size());
+		assertEquals(DEFAULT_TESTING_BIG_PROFILE, slotManager.getAllocatedContainers().get(0));
+		assertEquals(DEFAULT_TESTING_BIG_PROFILE, slotManager.getAllocatedContainers().get(1));
+	}
+
+	/**
+	 * Tests that a new slot appeared in SlotReport, and we used it to fulfill a pending request
+	 */
+	@Test
+	public void testNewlyAppearedFreeSlotFulfillPendingRequest() {
+		TestingSlotManager slotManager = new TestingSlotManager();
+		slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE));
+		assertEquals(1, slotManager.getPendingRequestCount());
+
+		SlotID slotId = SlotID.generate();
+		slotManager.registerTaskExecutor(slotId.getResourceID(), taskExecutorGateway);
+		SlotStatus slotStatus = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE);
+		slotManager.updateSlotStatus(slotStatus);
+
+		assertEquals(1, slotManager.getAllocatedSlotCount());
+		assertEquals(0, slotManager.getFreeSlotCount());
+		assertEquals(0, slotManager.getPendingRequestCount());
+		assertTrue(slotManager.isAllocated(slotId));
+	}
+
+	/**
+	 * Tests that a new slot appeared in SlotReport, but we have no pending request
+	 */
+	@Test
+	public void testNewlyAppearedFreeSlot() {
+		TestingSlotManager slotManager = new TestingSlotManager();
+
+		SlotID slotId = SlotID.generate();
+		slotManager.registerTaskExecutor(slotId.getResourceID(), taskExecutorGateway);
+		SlotStatus slotStatus = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE);
+		slotManager.updateSlotStatus(slotStatus);
+
+		assertEquals(0, slotManager.getAllocatedSlotCount());
+		assertEquals(1, slotManager.getFreeSlotCount());
+	}
+
+	/**
+	 * Tests that a new slot appeared in SlotReport, but it't not suitable for all the pending requests
+	 */
+	@Test
+	public void testNewlyAppearedFreeSlotNotMatchPendingRequests() {
+		TestingSlotManager slotManager = new TestingSlotManager();
+		slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_BIG_PROFILE));
+		assertEquals(1, slotManager.getPendingRequestCount());
+
+		SlotID slotId = SlotID.generate();
+		slotManager.registerTaskExecutor(slotId.getResourceID(), taskExecutorGateway);
+		SlotStatus slotStatus = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE);
+		slotManager.updateSlotStatus(slotStatus);
+
+		assertEquals(0, slotManager.getAllocatedSlotCount());
+		assertEquals(1, slotManager.getFreeSlotCount());
+		assertEquals(1, slotManager.getPendingRequestCount());
+		assertFalse(slotManager.isAllocated(slotId));
+	}
+
+	/**
+	 * Tests that a new slot appeared in SlotReport, and it's been reported using by some job
+	 */
+	@Test
+	public void testNewlyAppearedInUseSlot() {
+		TestingSlotManager slotManager = new TestingSlotManager();
+
+		SlotID slotId = SlotID.generate();
+		slotManager.registerTaskExecutor(slotId.getResourceID(), taskExecutorGateway);
+		SlotStatus slotStatus = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE, new JobID(), new AllocationID());
+		slotManager.updateSlotStatus(slotStatus);
+
+		assertEquals(1, slotManager.getAllocatedSlotCount());
+		assertEquals(0, slotManager.getFreeSlotCount());
+		assertTrue(slotManager.isAllocated(slotId));
+	}
+
+	/**
+	 * Tests that we had a slot in-use, and it's confirmed by SlotReport
+	 */
+	@Test
+	public void testExistingInUseSlotUpdateStatus() {
+		TestingSlotManager slotManager = new TestingSlotManager();
+		SlotRequest request = new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE);
+		slotManager.requestSlot(request);
+
+		// make this slot in use
+		SlotID slotId = SlotID.generate();
+		slotManager.registerTaskExecutor(slotId.getResourceID(), taskExecutorGateway);
+		SlotStatus slotStatus = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE);
+		slotManager.updateSlotStatus(slotStatus);
+
+		assertEquals(1, slotManager.getAllocatedSlotCount());
+		assertEquals(0, slotManager.getFreeSlotCount());
+		assertTrue(slotManager.isAllocated(slotId));
+
+		// slot status is confirmed
+		SlotStatus slotStatus2 = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE,
+			request.getJobId(), request.getAllocationId());
+		slotManager.updateSlotStatus(slotStatus2);
+
+		assertEquals(1, slotManager.getAllocatedSlotCount());
+		assertEquals(0, slotManager.getFreeSlotCount());
+		assertTrue(slotManager.isAllocated(slotId));
+	}
+
+	/**
+	 * Tests that we had a slot in-use, but it's empty according to the SlotReport
+	 */
+	@Test
+	public void testExistingInUseSlotAdjustedToEmpty() {
+		TestingSlotManager slotManager = new TestingSlotManager();
+		SlotRequest request1 = new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE);
+		slotManager.requestSlot(request1);
+
+		// make this slot in use
+		SlotID slotId = SlotID.generate();
+		slotManager.registerTaskExecutor(slotId.getResourceID(), taskExecutorGateway);
+		SlotStatus slotStatus = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE);
+		slotManager.updateSlotStatus(slotStatus);
+
+		// another request pending
+		SlotRequest request2 = new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE);
+		slotManager.requestSlot(request2);
+
+		assertEquals(1, slotManager.getAllocatedSlotCount());
+		assertEquals(0, slotManager.getFreeSlotCount());
+		assertEquals(1, slotManager.getPendingRequestCount());
+		assertTrue(slotManager.isAllocated(slotId));
+		assertTrue(slotManager.isAllocated(request1.getAllocationId()));
+
+
+		// but slot is reported empty again, request2 will be fulfilled, request1 will be missing
+		slotManager.updateSlotStatus(slotStatus);
+
+		assertEquals(1, slotManager.getAllocatedSlotCount());
+		assertEquals(0, slotManager.getFreeSlotCount());
+		assertEquals(0, slotManager.getPendingRequestCount());
+		assertTrue(slotManager.isAllocated(slotId));
+		assertTrue(slotManager.isAllocated(request2.getAllocationId()));
+	}
+
+	/**
+	 * Tests that we had a slot in use, and it's also reported in use by TaskManager, but the allocation
+	 * information didn't match.
+	 */
+	@Test
+	public void testExistingInUseSlotWithDifferentAllocationInfo() {
+		TestingSlotManager slotManager = new TestingSlotManager();
+		SlotRequest request = new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE);
+		slotManager.requestSlot(request);
+
+		// make this slot in use
+		SlotID slotId = SlotID.generate();
+		slotManager.registerTaskExecutor(slotId.getResourceID(), taskExecutorGateway);
+		SlotStatus slotStatus = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE);
+		slotManager.updateSlotStatus(slotStatus);
+
+		assertEquals(1, slotManager.getAllocatedSlotCount());
+		assertEquals(0, slotManager.getFreeSlotCount());
+		assertEquals(0, slotManager.getPendingRequestCount());
+		assertTrue(slotManager.isAllocated(slotId));
+		assertTrue(slotManager.isAllocated(request.getAllocationId()));
+
+		SlotStatus slotStatus2 = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE, new JobID(), new AllocationID());
+		// update slot status with different allocation info
+		slotManager.updateSlotStatus(slotStatus2);
+
+		// original request is missing and won't be allocated
+		assertEquals(1, slotManager.getAllocatedSlotCount());
+		assertEquals(0, slotManager.getFreeSlotCount());
+		assertEquals(0, slotManager.getPendingRequestCount());
+		assertTrue(slotManager.isAllocated(slotId));
+		assertFalse(slotManager.isAllocated(request.getAllocationId()));
+		assertTrue(slotManager.isAllocated(slotStatus2.getAllocationID()));
+	}
+
+	/**
+	 * Tests that we had a free slot, and it's confirmed by SlotReport
+	 */
+	@Test
+	public void testExistingEmptySlotUpdateStatus() {
+		TestingSlotManager slotManager = new TestingSlotManager();
+		ResourceSlot slot = new ResourceSlot(SlotID.generate(), DEFAULT_TESTING_PROFILE, taskExecutorGateway);
+		slotManager.addFreeSlot(slot);
+
+		SlotStatus slotStatus = new SlotStatus(slot.getSlotId(), DEFAULT_TESTING_PROFILE);
+		slotManager.updateSlotStatus(slotStatus);
+
+		assertEquals(0, slotManager.getAllocatedSlotCount());
+		assertEquals(1, slotManager.getFreeSlotCount());
+		assertEquals(0, slotManager.getPendingRequestCount());
+	}
+
+	/**
+	 * Tests that we had a free slot, and it's reported in-use by TaskManager
+	 */
+	@Test
+	public void testExistingEmptySlotAdjustedToInUse() {
+		TestingSlotManager slotManager = new TestingSlotManager();
+		final SlotID slotID = SlotID.generate();
+		slotManager.registerTaskExecutor(slotID.getResourceID(), taskExecutorGateway);
+
+		ResourceSlot slot = new ResourceSlot(slotID, DEFAULT_TESTING_PROFILE, taskExecutorGateway);
+		slotManager.addFreeSlot(slot);
+
+		SlotStatus slotStatus = new SlotStatus(slot.getSlotId(), DEFAULT_TESTING_PROFILE,
+			new JobID(), new AllocationID());
+		slotManager.updateSlotStatus(slotStatus);
+
+		assertEquals(1, slotManager.getAllocatedSlotCount());
+		assertEquals(0, slotManager.getFreeSlotCount());
+		assertEquals(0, slotManager.getPendingRequestCount());
+		assertTrue(slotManager.isAllocated(slot.getSlotId()));
+	}
+
+	/**
+	 * Tests that we did some allocation but failed / rejected by TaskManager, request will retry
+	 */
+	@Test
+	public void testSlotAllocationFailedAtTaskManager() {
+		TestingSlotManager slotManager = new TestingSlotManager();
+		ResourceSlot slot = new ResourceSlot(SlotID.generate(), DEFAULT_TESTING_PROFILE, taskExecutorGateway);
+		slotManager.addFreeSlot(slot);
+
+		SlotRequest request = new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE);
+		slotManager.requestSlot(request);
+
+		assertEquals(1, slotManager.getAllocatedSlotCount());
+		assertEquals(0, slotManager.getFreeSlotCount());
+		assertEquals(0, slotManager.getPendingRequestCount());
+		assertTrue(slotManager.isAllocated(slot.getSlotId()));
+
+		slotManager.handleSlotRequestFailedAtTaskManager(request, slot.getSlotId());
+
+		assertEquals(1, slotManager.getAllocatedSlotCount());
+		assertEquals(0, slotManager.getFreeSlotCount());
+		assertEquals(0, slotManager.getPendingRequestCount());
+	}
+
+
+	/**
+	 * Tests that we did some allocation but failed / rejected by TaskManager, and slot is occupied by another request
+	 */
+	@Test
+	public void testSlotAllocationFailedAtTaskManagerOccupiedByOther() {
+		TestingSlotManager slotManager = new TestingSlotManager();
+		final SlotID slotID = SlotID.generate();
+		slotManager.registerTaskExecutor(slotID.getResourceID(), taskExecutorGateway);
+
+		ResourceSlot slot = new ResourceSlot(slotID, DEFAULT_TESTING_PROFILE, taskExecutorGateway);
+		slotManager.addFreeSlot(slot);
+
+		SlotRequest request = new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE);
+		slotManager.requestSlot(request);
+
+		// slot is set empty by heartbeat
+		SlotStatus slotStatus = new SlotStatus(slot.getSlotId(), slot.getResourceProfile());
+		slotManager.updateSlotStatus(slotStatus);
+
+		// another request took this slot
+		SlotRequest request2 = new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE);
+		slotManager.requestSlot(request2);
+
+		assertEquals(1, slotManager.getAllocatedSlotCount());
+		assertEquals(0, slotManager.getFreeSlotCount());
+		assertEquals(0, slotManager.getPendingRequestCount());
+		assertFalse(slotManager.isAllocated(request.getAllocationId()));
+		assertTrue(slotManager.isAllocated(request2.getAllocationId()));
+
+		// original request should be pended
+		slotManager.handleSlotRequestFailedAtTaskManager(request, slot.getSlotId());
+
+		assertEquals(1, slotManager.getAllocatedSlotCount());
+		assertEquals(0, slotManager.getFreeSlotCount());
+		assertEquals(1, slotManager.getPendingRequestCount());
+		assertFalse(slotManager.isAllocated(request.getAllocationId()));
+		assertTrue(slotManager.isAllocated(request2.getAllocationId()));
+	}
+
+	@Test
+	public void testNotifyTaskManagerFailure() {
+		TestingSlotManager slotManager = new TestingSlotManager();
+
+		ResourceID resource1 = ResourceID.generate();
+		ResourceID resource2 = ResourceID.generate();
+
+		ResourceSlot slot11 = new ResourceSlot(new SlotID(resource1, 1), DEFAULT_TESTING_PROFILE, taskExecutorGateway);
+		ResourceSlot slot12 = new ResourceSlot(new SlotID(resource1, 2), DEFAULT_TESTING_PROFILE, taskExecutorGateway);
+		ResourceSlot slot21 = new ResourceSlot(new SlotID(resource2, 1), DEFAULT_TESTING_PROFILE, taskExecutorGateway);
+		ResourceSlot slot22 = new ResourceSlot(new SlotID(resource2, 2), DEFAULT_TESTING_PROFILE, taskExecutorGateway);
+
+		slotManager.addFreeSlot(slot11);
+		slotManager.addFreeSlot(slot21);
+
+		slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE));
+		slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE));
+
+		assertEquals(2, slotManager.getAllocatedSlotCount());
+		assertEquals(0, slotManager.getFreeSlotCount());
+		assertEquals(0, slotManager.getPendingRequestCount());
+
+		slotManager.addFreeSlot(slot12);
+		slotManager.addFreeSlot(slot22);
+
+		assertEquals(2, slotManager.getAllocatedSlotCount());
+		assertEquals(2, slotManager.getFreeSlotCount());
+		assertEquals(0, slotManager.getPendingRequestCount());
+
+		slotManager.notifyTaskManagerFailure(resource2);
+
+		assertEquals(1, slotManager.getAllocatedSlotCount());
+		assertEquals(1, slotManager.getFreeSlotCount());
+		assertEquals(0, slotManager.getPendingRequestCount());
+
+		// notify an not exist resource failure
+		slotManager.notifyTaskManagerFailure(ResourceID.generate());
+
+		assertEquals(1, slotManager.getAllocatedSlotCount());
+		assertEquals(1, slotManager.getFreeSlotCount());
+		assertEquals(0, slotManager.getPendingRequestCount());
+	}
+
+	// ------------------------------------------------------------------------
+	//  testing utilities
+	// ------------------------------------------------------------------------
+
+	private void directlyProvideFreeSlots(
+		final SlotManager slotManager,
+		final ResourceProfile resourceProfile,
+		final int freeSlotNum)
+	{
+		for (int i = 0; i < freeSlotNum; ++i) {
+			slotManager.addFreeSlot(new ResourceSlot(SlotID.generate(), new ResourceProfile(resourceProfile), taskExecutorGateway));
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  testing classes
+	// ------------------------------------------------------------------------
+
+	private static class TestingSlotManager extends SlotManager {
+
+		private final List<ResourceProfile> allocatedContainers;
+
+		TestingSlotManager() {
+			this.allocatedContainers = new LinkedList<>();
+		}
+
+		/**
+		 * Choose slot randomly if it matches requirement
+		 *
+		 * @param request   The slot request
+		 * @param freeSlots All slots which can be used
+		 * @return The chosen slot or null if cannot find a match
+		 */
+		@Override
+		protected ResourceSlot chooseSlotToUse(SlotRequest request, Map<SlotID, ResourceSlot> freeSlots) {
+			for (ResourceSlot slot : freeSlots.values()) {
+				if (slot.isMatchingRequirement(request.getResourceProfile())) {
+					return slot;
+				}
+			}
+			return null;
+		}
+
+		/**
+		 * Choose request randomly if offered slot can match its requirement
+		 *
+		 * @param offeredSlot     The free slot
+		 * @param pendingRequests All the pending slot requests
+		 * @return The chosen request's AllocationID or null if cannot find a match
+		 */
+		@Override
+		protected SlotRequest chooseRequestToFulfill(ResourceSlot offeredSlot,
+			Map<AllocationID, SlotRequest> pendingRequests)
+		{
+			for (Map.Entry<AllocationID, SlotRequest> pendingRequest : pendingRequests.entrySet()) {
+				if (offeredSlot.isMatchingRequirement(pendingRequest.getValue().getResourceProfile())) {
+					return pendingRequest.getValue();
+				}
+			}
+			return null;
+		}
+
+		@Override
+		protected void allocateContainer(ResourceProfile resourceProfile) {
+			allocatedContainers.add(resourceProfile);
+		}
+
+		List<ResourceProfile> getAllocatedContainers() {
+			return allocatedContainers;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/74570d45/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
new file mode 100644
index 0000000..85d2880
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotProtocolTest.java
@@ -0,0 +1,225 @@
+/*
+ * 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.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.clusterframework.types.SlotID;
+import org.apache.flink.runtime.highavailability.NonHaServices;
+import org.apache.flink.runtime.jobmaster.JobMasterGateway;
+import org.apache.flink.runtime.resourcemanager.JobMasterRegistration;
+import org.apache.flink.runtime.resourcemanager.RegistrationResponse;
+import org.apache.flink.runtime.resourcemanager.ResourceManager;
+import org.apache.flink.runtime.resourcemanager.SlotRequest;
+import org.apache.flink.runtime.resourcemanager.SlotRequestReply;
+import org.apache.flink.runtime.rpc.TestingSerialRpcService;
+import org.apache.flink.runtime.taskexecutor.SlotReport;
+import org.apache.flink.runtime.taskexecutor.SlotStatus;
+import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway;
+import org.apache.flink.util.TestLogger;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.mockito.Mockito;
+import scala.concurrent.Await;
+import scala.concurrent.Future;
+import scala.concurrent.duration.Duration;
+import scala.concurrent.duration.FiniteDuration;
+
+import java.util.Collections;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.timeout;
+import static org.mockito.Mockito.verify;
+
+public class SlotProtocolTest extends TestLogger {
+
+	private static TestingSerialRpcService testRpcService;
+
+	@BeforeClass
+	public static void beforeClass() {
+		testRpcService = new TestingSerialRpcService();
+	}
+
+	@AfterClass
+	public static void afterClass() {
+		testRpcService.stopService();
+		testRpcService = null;
+	}
+
+	@Before
+	public void beforeTest(){
+		testRpcService.clearGateways();
+	}
+
+	/**
+	 * Tests whether
+	 * 1) SlotRequest is routed to the SlotManager
+	 * 2) SlotRequest is confirmed
+	 * 3) SlotRequest leads to a container allocation
+	 * 4) Slot becomes available and TaskExecutor gets a SlotRequest
+	 */
+	@Test
+	public void testSlotsUnavailableRequest() throws Exception {
+		final String rmAddress = "/rm1";
+		final String jmAddress = "/jm1";
+		final JobID jobID = new JobID();
+
+		testRpcService.registerGateway(jmAddress, mock(JobMasterGateway.class));
+
+
+		TestingSlotManager slotManager = Mockito.spy(new TestingSlotManager());
+		ResourceManager resourceManager =
+			new ResourceManager(testRpcService, new NonHaServices(rmAddress), slotManager);
+		resourceManager.start();
+
+		Future<RegistrationResponse> registrationFuture =
+			resourceManager.registerJobMaster(new JobMasterRegistration(jmAddress, jobID));
+		try {
+			Await.ready(registrationFuture, Duration.create(5, TimeUnit.SECONDS));
+		} catch (Exception e) {
+			Assert.fail("JobManager registration Future didn't become ready.");
+		}
+
+		final AllocationID allocationID = new AllocationID();
+		final ResourceProfile resourceProfile = new ResourceProfile(1.0, 100);
+
+		SlotRequest slotRequest = new SlotRequest(jobID, allocationID, resourceProfile);
+		SlotRequestReply slotRequestReply =
+			resourceManager.requestSlot(slotRequest);
+
+		// 1) SlotRequest is routed to the SlotManager
+		verify(slotManager).requestSlot(slotRequest);
+
+		// 2) SlotRequest is confirmed
+		Assert.assertEquals(
+			slotRequestReply.getAllocationID(),
+			allocationID);
+
+		// 3) SlotRequest leads to a container allocation
+		verify(slotManager, timeout(5000)).allocateContainer(resourceProfile);
+
+		Assert.assertFalse(slotManager.isAllocated(allocationID));
+
+		// slot becomes available
+		final String tmAddress = "/tm1";
+		TaskExecutorGateway taskExecutorGateway = mock(TaskExecutorGateway.class);
+		testRpcService.registerGateway(tmAddress, taskExecutorGateway);
+
+		final ResourceID resourceID = ResourceID.generate();
+		final SlotID slotID = new SlotID(resourceID, 0);
+
+		final SlotStatus slotStatus =
+			new SlotStatus(slotID, resourceProfile);
+		final SlotReport slotReport =
+			new SlotReport(Collections.singletonList(slotStatus), resourceID);
+		// register slot at SlotManager
+		slotManager.registerTaskExecutor(resourceID, taskExecutorGateway);
+		slotManager.updateSlotStatus(slotReport);
+
+		// 4) Slot becomes available and TaskExecutor gets a SlotRequest
+		verify(taskExecutorGateway, timeout(5000)).requestSlot(eq(allocationID), any(UUID.class), any(FiniteDuration.class));
+	}
+
+	/**
+	 * Tests whether
+	 * 1) a SlotRequest is routed to the SlotManager
+	 * 2) a SlotRequest is confirmed
+	 * 3) a SlotRequest leads to an allocation of a registered slot
+	 * 4) a SlotRequest is routed to the TaskExecutor
+	 */
+	@Test
+	public void testSlotAvailableRequest() throws Exception {
+		final String rmAddress = "/rm1";
+		final String jmAddress = "/jm1";
+		final String tmAddress = "/tm1";
+		final JobID jobID = new JobID();
+
+		testRpcService.registerGateway(jmAddress, mock(JobMasterGateway.class));
+
+		TaskExecutorGateway taskExecutorGateway = mock(TaskExecutorGateway.class);
+		testRpcService.registerGateway(tmAddress, taskExecutorGateway);
+
+		TestingSlotManager slotManager = Mockito.spy(new TestingSlotManager());
+		ResourceManager resourceManager =
+			new ResourceManager(testRpcService, new NonHaServices(rmAddress), slotManager);
+		resourceManager.start();
+
+		Future<RegistrationResponse> registrationFuture =
+			resourceManager.registerJobMaster(new JobMasterRegistration(jmAddress, jobID));
+		try {
+			Await.ready(registrationFuture, Duration.create(5, TimeUnit.SECONDS));
+		} catch (Exception e) {
+			Assert.fail("JobManager registration Future didn't become ready.");
+		}
+
+		final ResourceID resourceID = ResourceID.generate();
+		final AllocationID allocationID = new AllocationID();
+		final ResourceProfile resourceProfile = new ResourceProfile(1.0, 100);
+		final SlotID slotID = new SlotID(resourceID, 0);
+
+		final SlotStatus slotStatus =
+			new SlotStatus(slotID, resourceProfile);
+		final SlotReport slotReport =
+			new SlotReport(Collections.singletonList(slotStatus), resourceID);
+		// register slot at SlotManager
+		slotManager.registerTaskExecutor(resourceID, taskExecutorGateway);
+		slotManager.updateSlotStatus(slotReport);
+
+		SlotRequest slotRequest = new SlotRequest(jobID, allocationID, resourceProfile);
+		SlotRequestReply slotRequestReply =
+			resourceManager.requestSlot(slotRequest);
+
+		// 1) a SlotRequest is routed to the SlotManager
+		verify(slotManager).requestSlot(slotRequest);
+
+		// 2) a SlotRequest is confirmed
+		Assert.assertEquals(
+			slotRequestReply.getAllocationID(),
+			allocationID);
+
+		// 3) a SlotRequest leads to an allocation of a registered slot
+		Assert.assertTrue(slotManager.isAllocated(slotID));
+		Assert.assertTrue(slotManager.isAllocated(allocationID));
+
+
+		// 4) a SlotRequest is routed to the TaskExecutor
+		verify(taskExecutorGateway, timeout(5000)).requestSlot(eq(allocationID), any(UUID.class), any(FiniteDuration.class));
+	}
+
+
+	private static class TestingSlotManager extends SimpleSlotManager {
+
+		// change visibility of function to public for testing
+		@Override
+		public void allocateContainer(ResourceProfile resourceProfile) {
+			super.allocateContainer(resourceProfile);
+		}
+
+
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/74570d45/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingRpcService.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingRpcService.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingRpcService.java
index 7e92e8d..2212680 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingRpcService.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingRpcService.java
@@ -112,4 +112,8 @@ public class TestingRpcService extends AkkaRpcService {
 			return Futures.failed(new Exception("No gateway registered under that name"));
 		}
 	}
-}
\ No newline at end of file
+
+	public void clearGateways() {
+		registeredConnections.clear();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/74570d45/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 955edcc..01776ed 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
@@ -137,6 +137,10 @@ public class TestingSerialRpcService implements RpcService {
 		}
 	}
 
+	public void clearGateways() {
+		registeredConnections.clear();
+	}
+
 	private static final class TestingSerialInvocationHandler<C extends RpcGateway, T extends RpcEndpoint<C>> implements InvocationHandler, RpcGateway, MainThreadExecutor, StartStoppable {
 
 		private final T rpcEndpoint;


[46/50] [abbrv] flink git commit: [FLINK-4516] update leadership information in ResourceManager

Posted by se...@apache.org.
[FLINK-4516] update leadership information in ResourceManager

The leadership information remained static for connected
JobMasters. This updates it to remove stale JobMasters when they lose
leadership status.

This closes #2624


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

Branch: refs/heads/flip-6
Commit: cef319126ced676b5c6d08e6a963986f1dd6c5ee
Parents: b380634
Author: Maximilian Michels <mx...@apache.org>
Authored: Mon Oct 10 17:36:10 2016 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Oct 14 15:14:43 2016 +0200

----------------------------------------------------------------------
 .../resourcemanager/ResourceManager.java        | 196 +++++++++++++------
 .../resourcemanager/ResourceManagerGateway.java |   4 +-
 .../ResourceManagerServices.java                |   6 +
 .../registration/JobMasterRegistration.java     |  62 ++++++
 .../slotmanager/SlotManager.java                |  16 +-
 .../resourcemanager/TestingSlotManager.java     |   8 +
 .../slotmanager/SlotManagerTest.java            |  10 +-
 7 files changed, 224 insertions(+), 78 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/cef31912/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 d2d00cf..8fbb34b 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
@@ -30,6 +30,7 @@ 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.highavailability.HighAvailabilityServices;
 import org.apache.flink.runtime.instance.InstanceID;
 import org.apache.flink.runtime.jobmaster.JobMasterRegistrationSuccess;
@@ -40,6 +41,7 @@ import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
 import org.apache.flink.runtime.resourcemanager.messages.jobmanager.RMSlotRequestRejected;
 import org.apache.flink.runtime.resourcemanager.messages.jobmanager.RMSlotRequestReply;
 import org.apache.flink.runtime.resourcemanager.messages.taskexecutor.SlotAvailableReply;
+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;
@@ -53,17 +55,14 @@ 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.LeaderConnectionInfo;
-import org.apache.flink.runtime.util.LeaderRetrievalUtils;
-import scala.concurrent.duration.FiniteDuration;
 
 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.Executor;
-import java.util.concurrent.TimeUnit;
 
 import static org.apache.flink.util.Preconditions.checkNotNull;
 
@@ -85,10 +84,10 @@ public abstract class ResourceManager<WorkerType extends Serializable>
 	protected static final int EXIT_CODE_FATAL_ERROR = -13;
 
 	/** All currently registered JobMasterGateways scoped by JobID. */
-	private final Map<JobID, JobMasterGateway> jobMasterGateways;
+	private final Map<JobID, JobMasterRegistration> jobMasters;
 
-	/** LeaderListeners for all registered JobMasters. */
-	private final Map<JobID, JobMasterLeaderListener> jobMasterLeaderRetrievalListeners;
+	/** LeaderListeners for all registered JobIDs. */
+	private final Map<JobID, JobIdLeaderListener> leaderListeners;
 
 	/** All currently registered TaskExecutors with there framework specific worker information. */
 	private final Map<ResourceID, WorkerRegistration<WorkerType>> taskExecutors;
@@ -106,7 +105,7 @@ public abstract class ResourceManager<WorkerType extends Serializable>
 	private LeaderElectionService leaderElectionService;
 
 	/** ResourceManager's leader session id which is updated on leader election. */
-	private UUID leaderSessionID;
+	private volatile UUID leaderSessionID;
 
 	/** All registered listeners for status updates of the ResourceManager. */
 	private Map<String, InfoMessageListenerRpcGateway> infoMessageListeners;
@@ -121,8 +120,8 @@ public abstract class ResourceManager<WorkerType extends Serializable>
 		super(rpcService);
 		this.highAvailabilityServices = checkNotNull(highAvailabilityServices);
 		this.slotManagerFactory = checkNotNull(slotManagerFactory);
-		this.jobMasterGateways = new HashMap<>();
-		this.jobMasterLeaderRetrievalListeners = new HashMap<>();
+		this.jobMasters = new HashMap<>();
+		this.leaderListeners = new HashMap<>();
 		this.taskExecutors = new HashMap<>();
 		this.leaderSessionID = new UUID(0, 0);
 		infoMessageListeners = new HashMap<>();
@@ -149,9 +148,7 @@ public abstract class ResourceManager<WorkerType extends Serializable>
 	public void shutDown() {
 		try {
 			leaderElectionService.stop();
-			for (JobID jobID : jobMasterGateways.keySet()) {
-				highAvailabilityServices.getJobManagerLeaderRetriever(jobID).stop();
-			}
+			clearState();
 			super.shutDown();
 		} catch (Throwable e) {
 			log.error("A fatal error happened when shutdown the ResourceManager", e);
@@ -185,6 +182,24 @@ public abstract class ResourceManager<WorkerType extends Serializable>
 		checkNotNull(jobMasterAddress);
 		checkNotNull(jobID);
 
+		// 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);
+			} 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;
+			}
+			leaderListeners.put(jobID, jobIdLeaderListener);
+		}
+
 		return getRpcService()
 			.execute(new Callable<JobMasterGateway>() {
 				@Override
@@ -197,21 +212,13 @@ public abstract class ResourceManager<WorkerType extends Serializable>
 						throw new Exception("Invalid leader session id");
 					}
 
-					final LeaderConnectionInfo jobMasterLeaderInfo;
-					try {
-						jobMasterLeaderInfo = LeaderRetrievalUtils.retrieveLeaderConnectionInfo(
-							highAvailabilityServices.getJobManagerLeaderRetriever(jobID), new FiniteDuration(5, TimeUnit.SECONDS));
-					} catch (Exception e) {
-						log.warn("Failed to start JobMasterLeaderRetriever for JobID {}", jobID);
-						throw new Exception("Failed to retrieve JobMasterLeaderRetriever");
-					}
-
-					if (!jobMasterLeaderId.equals(jobMasterLeaderInfo.getLeaderSessionID())) {
-						log.info("Declining registration request from non-leading JobManager {}", jobMasterAddress);
-						throw new Exception("JobManager is not leading");
+					if (!jobIdLeaderListener.getLeaderID().get(timeout.getSize(), timeout.getUnit())
+							.equals(jobMasterLeaderId)) {
+						throw new Exception("Leader Id did not match");
 					}
 
-					return getRpcService().connect(jobMasterAddress, JobMasterGateway.class).get(5, TimeUnit.SECONDS);
+					return getRpcService().connect(jobMasterAddress, JobMasterGateway.class)
+						.get(timeout.getSize(), timeout.getUnit());
 				}
 			})
 			.handleAsync(new BiFunction<JobMasterGateway, Throwable, RegistrationResponse>() {
@@ -220,24 +227,34 @@ public abstract class ResourceManager<WorkerType extends Serializable>
 
 					if (throwable != null) {
 						return new RegistrationResponse.Decline(throwable.getMessage());
-					} else {
-						if (!jobMasterLeaderRetrievalListeners.containsKey(jobID)) {
-							JobMasterLeaderListener jobMasterLeaderListener = new JobMasterLeaderListener(jobID);
-							try {
-								LeaderRetrievalService jobMasterLeaderRetriever = highAvailabilityServices.getJobManagerLeaderRetriever(jobID);
-								jobMasterLeaderRetriever.start(jobMasterLeaderListener);
-							} catch (Exception e) {
-								log.warn("Failed to start JobMasterLeaderRetriever for JobID {}", jobID);
-								return new RegistrationResponse.Decline("Failed to retrieve JobMasterLeaderRetriever");
-							}
-							jobMasterLeaderRetrievalListeners.put(jobID, jobMasterLeaderListener);
-						}
-						final JobMasterGateway existingGateway = jobMasterGateways.put(jobID, jobMasterGateway);
-						if (existingGateway != null) {
-							log.info("Replacing gateway for registered JobID {}.", jobID);
+					}
+
+					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");
+					}
+
+					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");
 						}
-						return new JobMasterRegistrationSuccess(5000, resourceManagerLeaderId);
+					} 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 JobMasterRegistrationSuccess(5000, resourceManagerLeaderId);
+
 				}
 			}, getMainThreadExecutor());
 	}
@@ -305,13 +322,10 @@ public abstract class ResourceManager<WorkerType extends Serializable>
 			SlotRequest slotRequest) {
 
 		JobID jobId = slotRequest.getJobId();
-		JobMasterGateway jobMasterGateway = jobMasterGateways.get(jobId);
-		JobMasterLeaderListener jobMasterLeaderListener = jobMasterLeaderRetrievalListeners.get(jobId);
+		JobMasterRegistration jobMasterRegistration = jobMasters.get(jobId);
 
-		UUID leaderID = jobMasterLeaderListener.getLeaderID();
-
-		if (jobMasterGateway != null
-				&& jobMasterLeaderID.equals(leaderID)
+		if (jobMasterRegistration != null
+				&& jobMasterLeaderID.equals(jobMasterRegistration.getLeaderID())
 				&& resourceManagerLeaderID.equals(leaderSessionID)) {
 			return slotManager.requestSlot(slotRequest);
 		} else {
@@ -371,8 +385,6 @@ public abstract class ResourceManager<WorkerType extends Serializable>
 				log.info("ResourceManager {} was granted leadership with leader session ID {}", getAddress(), leaderSessionID);
 				// confirming the leader session ID might be blocking,
 				leaderElectionService.confirmLeaderSessionID(leaderSessionID);
-				// notify SlotManager
-				slotManager.setLeaderUUID(leaderSessionID);
 				ResourceManager.this.leaderSessionID = leaderSessionID;
 			}
 		});
@@ -387,10 +399,7 @@ public abstract class ResourceManager<WorkerType extends Serializable>
 			@Override
 			public void run() {
 				log.info("ResourceManager {} was revoked leadership.", getAddress());
-				jobMasterGateways.clear();
-				taskExecutors.clear();
-				slotManager.clearState();
-				leaderSessionID = new UUID(0, 0);
+				clearState();
 			}
 		});
 	}
@@ -577,6 +586,11 @@ public abstract class ResourceManager<WorkerType extends Serializable>
 	private class DefaultResourceManagerServices implements ResourceManagerServices {
 
 		@Override
+		public UUID getLeaderID() {
+			return ResourceManager.this.leaderSessionID;
+		}
+
+		@Override
 		public void allocateResource(ResourceProfile resourceProfile) {
 			ResourceManager.this.startNewWorker(resourceProfile);
 		}
@@ -592,33 +606,95 @@ public abstract class ResourceManager<WorkerType extends Serializable>
 		}
 	}
 
-	private static class JobMasterLeaderListener implements LeaderRetrievalListener {
+	/**
+	 * Leader instantiated for each connected JobMaster
+	 */
+	private class JobIdLeaderListener implements LeaderRetrievalListener {
 
 		private final JobID jobID;
-		private UUID leaderID;
+		private final LeaderRetrievalService retrievalService;
 
-		private JobMasterLeaderListener(JobID jobID) {
+		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);
+		}
+
+		public Future<UUID> getLeaderID() {
+			if (!initialLeaderIdFuture.isDone()) {
+				return initialLeaderIdFuture;
+			} else {
+				return FlinkCompletableFuture.completed(leaderID);
+			}
 		}
 
 		public JobID getJobID() {
 			return jobID;
 		}
 
-		public UUID getLeaderID() {
-			return leaderID;
+
+		public void stopService() throws Exception {
+			retrievalService.stop();
 		}
 
 		@Override
 		public void notifyLeaderAddress(final String leaderAddress, final UUID leaderSessionID) {
 			this.leaderID = leaderSessionID;
+
+			if (!initialLeaderIdFuture.isDone()) {
+				initialLeaderIdFuture.complete(leaderSessionID);
+			}
+
+			ResourceManager.this.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);
+							}
+						}
+					}
+				}
+			});
 		}
 
 		@Override
 		public void handleError(final Exception exception) {
-			// TODO
+			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/cef31912/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 3c81227..07e9e43 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
@@ -61,14 +61,14 @@ public interface ResourceManagerGateway extends RpcGateway {
 	/**
 	 * Requests a slot from the resource manager.
 	 *
-	 * @param jobMasterLeaderID leader id of the JobMaster
 	 * @param resourceManagerLeaderID leader if of the ResourceMaster
+	 * @param jobMasterLeaderID leader if of the JobMaster
 	 * @param slotRequest The slot to request
 	 * @return The confirmation that the slot gets allocated
 	 */
 	Future<RMSlotRequestReply> requestSlot(
-		UUID jobMasterLeaderID,
 		UUID resourceManagerLeaderID,
+		UUID jobMasterLeaderID,
 		SlotRequest slotRequest,
 		@RpcTimeout Time timeout);
 

http://git-wip-us.apache.org/repos/asf/flink/blob/cef31912/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 b997a3a..16d0a7d 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
@@ -19,6 +19,7 @@ package org.apache.flink.runtime.resourcemanager;
 
 import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
 
+import java.util.UUID;
 import java.util.concurrent.Executor;
 
 /**
@@ -27,6 +28,11 @@ import java.util.concurrent.Executor;
 public interface ResourceManagerServices {
 
 	/**
+	 * Gets the current leader id assigned at the ResourceManager.
+	 */
+	UUID getLeaderID();
+
+	/**
 	 * Allocates a resource according to the resource profile.
 	 */
 	void allocateResource(ResourceProfile resourceProfile);

http://git-wip-us.apache.org/repos/asf/flink/blob/cef31912/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
new file mode 100644
index 0000000..f417935
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/registration/JobMasterRegistration.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.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/cef31912/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManager.java
index 7eb2d78..e312ea2 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManager.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManager.java
@@ -41,7 +41,6 @@ import org.slf4j.LoggerFactory;
 import java.util.HashMap;
 import java.util.LinkedHashMap;
 import java.util.Map;
-import java.util.UUID;
 
 import static org.apache.flink.util.Preconditions.checkNotNull;
 
@@ -85,9 +84,6 @@ public abstract class SlotManager {
 
 	private final Time timeout;
 
-	/** The current leader id set by the ResourceManager */
-	private UUID leaderID;
-
 	public SlotManager(ResourceManagerServices rmServices) {
 		this.rmServices = checkNotNull(rmServices);
 		this.registeredSlots = new HashMap<>(16);
@@ -96,7 +92,6 @@ public abstract class SlotManager {
 		this.allocationMap = new AllocationMap();
 		this.taskManagers = new HashMap<>();
 		this.timeout = Time.seconds(10);
-		this.leaderID = new UUID(0, 0);
 	}
 
 	// ------------------------------------------------------------------------
@@ -303,7 +298,7 @@ public abstract class SlotManager {
 		final TaskExecutorRegistration registration = freeSlot.getTaskExecutorRegistration();
 		final Future<TMSlotRequestReply> slotRequestReplyFuture =
 			registration.getTaskExecutorGateway()
-				.requestSlot(freeSlot.getSlotId(), allocationID, leaderID, timeout);
+				.requestSlot(freeSlot.getSlotId(), allocationID, rmServices.getLeaderID(), timeout);
 
 		slotRequestReplyFuture.handleAsync(new BiFunction<TMSlotRequestReply, Throwable, Void>() {
 			@Override
@@ -488,15 +483,6 @@ public abstract class SlotManager {
 		pendingSlotRequests.clear();
 		freeSlots.clear();
 		allocationMap.clear();
-		leaderID = new UUID(0, 0);
-	}
-
-	// ------------------------------------------------------------------------
-	//  High availability (called by the ResourceManager)
-	// ------------------------------------------------------------------------
-
-	public void setLeaderUUID(UUID leaderSessionID) {
-		this.leaderID = leaderSessionID;
 	}
 
 	// ------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/flink/blob/cef31912/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/TestingSlotManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/TestingSlotManager.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/TestingSlotManager.java
index 0b2c42b..67b208d 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/TestingSlotManager.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/TestingSlotManager.java
@@ -26,6 +26,7 @@ import org.mockito.Mockito;
 
 import java.util.Iterator;
 import java.util.Map;
+import java.util.UUID;
 import java.util.concurrent.Executor;
 
 public class TestingSlotManager extends SlotManager {
@@ -60,6 +61,13 @@ public class TestingSlotManager extends SlotManager {
 
 	private static class TestingResourceManagerServices implements ResourceManagerServices {
 
+		private final UUID leaderID = UUID.randomUUID();
+
+		@Override
+		public UUID getLeaderID() {
+			return leaderID;
+		}
+
 		@Override
 		public void allocateResource(ResourceProfile resourceProfile) {
 

http://git-wip-us.apache.org/repos/asf/flink/blob/cef31912/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerTest.java
index 0d2b40d..558d3c2 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerTest.java
@@ -498,13 +498,21 @@ public class SlotManagerTest {
 
 		private static class TestingRmServices implements ResourceManagerServices {
 
-			private List<ResourceProfile> allocatedContainers;
+			private final UUID leaderID;
+
+			private final List<ResourceProfile> allocatedContainers;
 
 			public TestingRmServices() {
+				this.leaderID = UUID.randomUUID();
 				this.allocatedContainers = new LinkedList<>();
 			}
 
 			@Override
+			public UUID getLeaderID() {
+				return leaderID;
+			}
+
+			@Override
 			public void allocateResource(ResourceProfile resourceProfile) {
 				allocatedContainers.add(resourceProfile);
 			}


[48/50] [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/0615b62f/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java
index 3b8fc97..e11f3a1 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java
@@ -18,38 +18,32 @@
 
 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.CheckpointMetaData;
 import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory;
-import org.apache.flink.runtime.checkpoint.CompletedCheckpointStore;
-import org.apache.flink.runtime.checkpoint.savepoint.SavepointStore;
-import org.apache.flink.runtime.checkpoint.stats.CheckpointStatsTracker;
-import org.apache.flink.runtime.checkpoint.stats.DisabledCheckpointStatsTracker;
-import org.apache.flink.runtime.checkpoint.stats.SimpleCheckpointStatsTracker;
+import org.apache.flink.runtime.checkpoint.CompletedCheckpoint;
 import org.apache.flink.runtime.client.JobExecutionException;
 import org.apache.flink.runtime.client.JobSubmissionException;
 import org.apache.flink.runtime.client.SerializedJobExecutionResult;
 import org.apache.flink.runtime.clusterframework.types.ResourceID;
-import org.apache.flink.runtime.concurrent.BiFunction;
+import org.apache.flink.runtime.concurrent.ApplyFunction;
 import org.apache.flink.runtime.concurrent.Future;
-import org.apache.flink.runtime.concurrent.impl.FlinkCompletableFuture;
-import org.apache.flink.runtime.concurrent.impl.FlinkFuture;
 import org.apache.flink.runtime.execution.ExecutionState;
 import org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager;
 import org.apache.flink.runtime.executiongraph.Execution;
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
 import org.apache.flink.runtime.executiongraph.ExecutionGraph;
+import org.apache.flink.runtime.executiongraph.ExecutionGraphBuilder;
 import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
+import org.apache.flink.runtime.executiongraph.JobStatusListener;
 import org.apache.flink.runtime.executiongraph.restart.RestartStrategy;
 import org.apache.flink.runtime.executiongraph.restart.RestartStrategyFactory;
 import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
@@ -59,18 +53,13 @@ import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
 import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
 import org.apache.flink.runtime.jobgraph.JobGraph;
 import org.apache.flink.runtime.jobgraph.JobStatus;
-import org.apache.flink.runtime.jobgraph.JobVertex;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
-import org.apache.flink.runtime.jobgraph.jsonplan.JsonPlanGenerator;
-import org.apache.flink.runtime.jobgraph.tasks.JobSnapshottingSettings;
 import org.apache.flink.runtime.jobmanager.OnCompletionActions;
 import org.apache.flink.runtime.jobmanager.scheduler.Scheduler;
 import org.apache.flink.runtime.jobmaster.message.ClassloadingProps;
-import org.apache.flink.runtime.jobmaster.message.DisposeSavepointResponse;
-import org.apache.flink.runtime.jobmaster.message.NextInputSplit;
-import org.apache.flink.runtime.jobmaster.message.TriggerSavepointResponse;
 import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener;
 import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+import org.apache.flink.runtime.messages.Acknowledge;
 import org.apache.flink.runtime.messages.checkpoint.AcknowledgeCheckpoint;
 import org.apache.flink.runtime.messages.checkpoint.DeclineCheckpoint;
 import org.apache.flink.runtime.metrics.groups.JobManagerMetricGroup;
@@ -83,25 +72,27 @@ import org.apache.flink.runtime.registration.RegisteredRpcConnection;
 import org.apache.flink.runtime.registration.RegistrationResponse;
 import org.apache.flink.runtime.registration.RetryingRegistration;
 import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
 import org.apache.flink.runtime.rpc.RpcEndpoint;
 import org.apache.flink.runtime.rpc.RpcMethod;
 import org.apache.flink.runtime.rpc.RpcService;
+import org.apache.flink.runtime.rpc.StartStoppable;
+import org.apache.flink.runtime.state.CheckpointStateHandles;
 import org.apache.flink.runtime.state.KeyGroupRange;
 import org.apache.flink.runtime.taskmanager.TaskExecutionState;
 import org.apache.flink.runtime.util.SerializedThrowable;
 import org.apache.flink.util.InstantiationUtil;
 import org.apache.flink.util.SerializedValue;
-import org.slf4j.Logger;
 
-import scala.concurrent.ExecutionContext$;
-import scala.concurrent.duration.FiniteDuration;
+import 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;
 
@@ -112,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 */
@@ -130,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;
 
@@ -170,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);
 
@@ -193,289 +180,149 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 		this.configuration = checkNotNull(configuration);
 		this.highAvailabilityServices = checkNotNull(highAvailabilityService);
 		this.libraryCacheManager = checkNotNull(libraryCacheManager);
-		this.restartStrategyFactory = checkNotNull(restartStrategyFactory);
-		this.savepointStore = checkNotNull(savepointStore);
-		this.timeout = checkNotNull(timeout);
-		this.scheduler = checkNotNull(scheduler);
-		this.jobManagerMetricGroup = checkNotNull(jobManagerMetricGroup);
-		this.executionContext = checkNotNull(rpcService.getExecutor());
+		this.executionContext = checkNotNull(executorService);
 		this.jobCompletionActions = checkNotNull(jobCompletionActions);
-	}
+		this.errorHandler = checkNotNull(errorHandler);
+		this.userCodeLoader = checkNotNull(userCodeLoader);
 
-	//----------------------------------------------------------------------------------------------
-	// Lifecycle management
-	//----------------------------------------------------------------------------------------------
-
-	/**
-	 * Initializing the job execution environment, should be called before start. Any error occurred during
-	 * initialization will be treated as job submission failure.
-	 *
-	 * @throws JobSubmissionException
-	 */
-	public void init() throws JobSubmissionException {
-		log.info("Initializing job {} ({}).", jobGraph.getName(), jobGraph.getJobID());
+		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());
+		this.executionGraph = ExecutionGraphBuilder.buildGraph(
+				null,
+				jobGraph,
+				configuration,
+				executorService,
+				userCodeLoader,
+				checkpointRecoveryFactory,
+				rpcAskTimeout,
+				restartStrategy,
+				jobMetricGroup,
+				-1,
+				log);
 
-			if (t instanceof JobSubmissionException) {
-				throw (JobSubmissionException) t;
-			}
-			else {
-				throw new JobSubmissionException(jobGraph.getJobID(), "Failed to initialize job " +
-					jobGraph.getName() + " (" + jobGraph.getJobID() + ")", t);
-			}
-		}
+		// TODO - temp fix
+		this.scheduler = new Scheduler(executorService);
 	}
 
+	//----------------------------------------------------------------------------------------------
+	// Lifecycle management
+	//----------------------------------------------------------------------------------------------
+
+
 	@Override
 	public void start() {
-		super.start();
+		throw new UnsupportedOperationException("Should never call start() without leader ID");
 	}
 
+	/**
+	 * Start the rpc service and begin to run the job.
+	 *
+	 * @param leaderSessionID The necessary leader id for running the job.
+	 */
+	public void start(final UUID leaderSessionID) {
+		if (LEADER_ID_UPDATER.compareAndSet(this, null, leaderSessionID)) {
+			super.start();
+
+			log.info("Starting JobManager for job {} ({})", jobGraph.getName(), jobGraph.getJobID());
+			getSelf().startJobExecution();
+		} else {
+			log.warn("Job already started with leaderId {}, ignoring this start request.", leaderSessionID);
+		}
+	}
+
+	/**
+	 * Suspend the job and shutdown all other services including rpc.
+	 */
 	@Override
 	public void shutDown() {
+		// make sure there is a graceful exit
+		getSelf().suspendExecution(new Exception("JobManager is shutting down."));
 		super.shutDown();
-
-		suspendJob(new Exception("JobManager is shutting down."));
-
-		disposeCommunicationWithResourceManager();
 	}
 
-
-
 	//----------------------------------------------------------------------------------------------
 	// RPC methods
 	//----------------------------------------------------------------------------------------------
 
-	/**
-	 * Start to run the job, runtime data structures like ExecutionGraph will be constructed now and checkpoint
-	 * being recovered. After this, we will begin to schedule the job.
-	 */
+	//-- job starting and stopping  -----------------------------------------------------------------
+
 	@RpcMethod
-	public void startJob(final UUID leaderSessionID) {
-		log.info("Starting job {} ({}) with leaderId {}.", jobGraph.getName(), jobGraph.getJobID(), leaderSessionID);
-
-		this.leaderSessionID = leaderSessionID;
-
-		if (executionGraph != null) {
-			executionGraph = new ExecutionGraph(
-				ExecutionContext$.MODULE$.fromExecutor(executionContext),
-				jobGraph.getJobID(),
-				jobGraph.getName(),
-				jobGraph.getJobConfiguration(),
-				jobGraph.getSerializedExecutionConfig(),
-				new FiniteDuration(timeout.getSize(), timeout.getUnit()),
-				restartStrategy,
-				jobGraph.getUserJarBlobKeys(),
-				jobGraph.getClasspaths(),
-				userCodeLoader,
-				jobMetrics);
-		}
-		else {
-			// TODO: update last active time in JobInfo
-		}
+	public void startJobExecution() {
+		log.info("Starting execution of job {} ({}) with leaderId {}.",
+				jobGraph.getName(), jobGraph.getJobID(), leaderSessionID);
 
 		try {
-			executionGraph.setScheduleMode(jobGraph.getScheduleMode());
-			executionGraph.setQueuedSchedulingAllowed(jobGraph.getAllowQueuedScheduling());
-
-			try {
-				executionGraph.setJsonPlan(JsonPlanGenerator.generatePlan(jobGraph));
-			} catch (Exception e) {
-				log.warn("Cannot create JSON plan for job {} ({})", jobGraph.getJobID(), jobGraph.getName(), e);
-				executionGraph.setJsonPlan("{}");
-			}
-
-			// initialize the vertices that have a master initialization hook
-			// file output formats create directories here, input formats create splits
-			if (log.isDebugEnabled()) {
-				log.debug("Running initialization on master for job {} ({}).", jobGraph.getJobID(), jobGraph.getName());
-			}
-			for (JobVertex vertex : jobGraph.getVertices()) {
-				final String executableClass = vertex.getInvokableClassName();
-				if (executableClass == null || executableClass.length() == 0) {
-					throw new JobExecutionException(jobGraph.getJobID(),
-						"The vertex " + vertex.getID() + " (" + vertex.getName() + ") has no invokable class.");
-				}
-				if (vertex.getParallelism() == ExecutionConfig.PARALLELISM_AUTO_MAX) {
-					vertex.setParallelism(scheduler.getTotalNumberOfSlots());
-				}
-
-				try {
-					vertex.initializeOnMaster(userCodeLoader);
-				} catch (Throwable t) {
-					throw new JobExecutionException(jobGraph.getJobID(),
-						"Cannot initialize task '" + vertex.getName() + "': " + t.getMessage(), t);
-				}
-			}
-
-			// topologically sort the job vertices and attach the graph to the existing one
-			final List<JobVertex> sortedTopology = jobGraph.getVerticesSortedTopologicallyFromSources();
-			if (log.isDebugEnabled()) {
-				log.debug("Adding {} vertices from job graph {} ({}).", sortedTopology.size(),
-					jobGraph.getJobID(), jobGraph.getName());
-			}
-			executionGraph.attachJobGraph(sortedTopology);
-
-			if (log.isDebugEnabled()) {
-				log.debug("Successfully created execution graph from job graph {} ({}).",
-					jobGraph.getJobID(), jobGraph.getName());
-			}
-
-			final JobSnapshottingSettings snapshotSettings = jobGraph.getSnapshotSettings();
-			if (snapshotSettings != null) {
-				List<ExecutionJobVertex> triggerVertices = getExecutionJobVertexWithId(
-					executionGraph, snapshotSettings.getVerticesToTrigger());
-
-				List<ExecutionJobVertex> ackVertices = getExecutionJobVertexWithId(
-					executionGraph, snapshotSettings.getVerticesToAcknowledge());
-
-				List<ExecutionJobVertex> confirmVertices = getExecutionJobVertexWithId(
-					executionGraph, snapshotSettings.getVerticesToConfirm());
-
-				CompletedCheckpointStore completedCheckpoints = checkpointRecoveryFactory.createCheckpointStore(
-					jobGraph.getJobID(), userCodeLoader);
-
-				CheckpointIDCounter checkpointIdCounter = checkpointRecoveryFactory.createCheckpointIDCounter(
-					jobGraph.getJobID());
-
-				// Checkpoint stats tracker
-				boolean isStatsDisabled = configuration.getBoolean(
-					ConfigConstants.JOB_MANAGER_WEB_CHECKPOINTS_DISABLE,
-					ConfigConstants.DEFAULT_JOB_MANAGER_WEB_CHECKPOINTS_DISABLE);
-
-				final CheckpointStatsTracker checkpointStatsTracker;
-				if (isStatsDisabled) {
-					checkpointStatsTracker = new DisabledCheckpointStatsTracker();
-				}
-				else {
-					int historySize = configuration.getInteger(
-						ConfigConstants.JOB_MANAGER_WEB_CHECKPOINTS_HISTORY_SIZE,
-						ConfigConstants.DEFAULT_JOB_MANAGER_WEB_CHECKPOINTS_HISTORY_SIZE);
-					checkpointStatsTracker = new SimpleCheckpointStatsTracker(historySize, ackVertices, jobMetrics);
-				}
-
-				executionGraph.enableSnapshotCheckpointing(
-					snapshotSettings.getCheckpointInterval(),
-					snapshotSettings.getCheckpointTimeout(),
-					snapshotSettings.getMinPauseBetweenCheckpoints(),
-					snapshotSettings.getMaxConcurrentCheckpoints(),
-					triggerVertices,
-					ackVertices,
-					confirmVertices,
-					checkpointIdCounter,
-					completedCheckpoints,
-					savepointStore,
-					checkpointStatsTracker);
-			}
-
-			// TODO: register this class to execution graph as job status change listeners
-
-			// TODO: register client as job / execution status change listeners if they are interested
-
-			/*
-			TODO: decide whether we should take the savepoint before recovery
-
-			if (isRecovery) {
-				// this is a recovery of a master failure (this master takes over)
-				executionGraph.restoreLatestCheckpointedState();
-			} else {
-				if (snapshotSettings != null) {
-					String savepointPath = snapshotSettings.getSavepointPath();
-					if (savepointPath != null) {
-						// got a savepoint
-						log.info("Starting job from savepoint {}.", savepointPath);
-
-						// load the savepoint as a checkpoint into the system
-						final CompletedCheckpoint savepoint = SavepointLoader.loadAndValidateSavepoint(
-							jobGraph.getJobID(), executionGraph.getAllVertices(), savepointStore, savepointPath);
-						executionGraph.getCheckpointCoordinator().getCheckpointStore().addCheckpoint(savepoint);
-
-						// Reset the checkpoint ID counter
-						long nextCheckpointId = savepoint.getCheckpointID() + 1;
-						log.info("Reset the checkpoint ID to " + nextCheckpointId);
-						executionGraph.getCheckpointCoordinator().getCheckpointIdCounter().setCount(nextCheckpointId);
-
-						executionGraph.restoreLatestCheckpointedState();
-					}
+			// register self as job status change listener
+			executionGraph.registerJobStatusListener(new JobStatusListener() {
+				@Override
+				public void jobStatusChanges(
+						final JobID jobId, final JobStatus newJobStatus, final long timestamp, final Throwable error)
+				{
+					// run in rpc thread to avoid concurrency
+					runAsync(new Runnable() {
+						@Override
+						public void run() {
+							jobStatusChanged(newJobStatus, timestamp, error);
+						}
+					});
 				}
-			}
-			*/
+			});
 
-			// job is good to go, try to locate resource manager's address
+			// job is ready to go, try to establish connection with resource manager
 			resourceManagerLeaderRetriever.start(new ResourceManagerLeaderListener());
 		} catch (Throwable t) {
+
+			// TODO - this should not result in a job failure, but another leader should take over
+			// TODO - either this master should retry the execution, or it should relinquish leadership / terminate
+
 			log.error("Failed to start job {} ({})", jobGraph.getName(), jobGraph.getJobID(), t);
 
 			executionGraph.fail(t);
-			executionGraph = null;
 
-			final Throwable rt;
+			final JobExecutionException rt;
 			if (t instanceof JobExecutionException) {
 				rt = (JobExecutionException) t;
-			}
-			else {
+			} else {
 				rt = new JobExecutionException(jobGraph.getJobID(),
-					"Failed to start job " + jobGraph.getJobID() + " (" + jobGraph.getName() + ")", t);
+						"Failed to start job " + jobGraph.getJobID() + " (" + jobGraph.getName() + ")", t);
 			}
 
 			// TODO: notify client about this failure
@@ -488,34 +335,51 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 		executionContext.execute(new Runnable() {
 			@Override
 			public void run() {
-				if (executionGraph != null) {
-					try {
-						executionGraph.scheduleForExecution(scheduler);
-					} catch (Throwable t) {
-						executionGraph.fail(t);
-					}
+				try {
+					executionGraph.scheduleForExecution(scheduler);
+				} catch (Throwable t) {
+					executionGraph.fail(t);
 				}
 			}
 		});
 	}
 
 	/**
-	 * Suspending job, all the running tasks will be cancelled, and runtime data will be cleared.
+	 * Suspending job, all the running tasks will be cancelled, and communication with other components
+	 * will be disposed.
+	 *
+	 * <p>Mostly job is suspended because of the leadership has been revoked, one can be restart this job by
+	 * calling the {@link #start(UUID)} method once we take the leadership back again.
 	 *
 	 * @param cause The reason of why this job been suspended.
 	 */
 	@RpcMethod
-	public void suspendJob(final Throwable cause) {
+	public void suspendExecution(final Throwable cause) {
+		if (leaderSessionID == null) {
+			log.debug("Job has already been suspended or shutdown.");
+			return;
+		}
+
+		// receive no more messages until started again, should be called before we clear self leader id
+		((StartStoppable) getSelf()).stop();
+
 		leaderSessionID = null;
+		executionGraph.suspend(cause);
 
-		if (executionGraph != null) {
-			executionGraph.suspend(cause);
-			executionGraph = null;
+		// disconnect from resource manager:
+		try {
+			resourceManagerLeaderRetriever.stop();
+		} catch (Exception e) {
+			log.warn("Failed to stop resource manager leader retriever when suspending.");
 		}
+		closeResourceManagerConnection();
+
+		// TODO: disconnect from all registered task managers
 
-		disposeCommunicationWithResourceManager();
 	}
 
+	//----------------------------------------------------------------------------------------------
+
 	/**
 	 * Updates the task execution state for a given task.
 	 *
@@ -523,24 +387,38 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 	 * @return Acknowledge the task execution state update
 	 */
 	@RpcMethod
-	public Acknowledge updateTaskExecutionState(final TaskExecutionState taskExecutionState) throws ExecutionGraphException {
+	public Acknowledge updateTaskExecutionState(
+			final UUID leaderSessionID,
+			final TaskExecutionState taskExecutionState) throws Exception
+	{
 		if (taskExecutionState == null) {
 			throw new NullPointerException("TaskExecutionState must not be null.");
 		}
 
+		if (!this.leaderSessionID.equals(leaderSessionID)) {
+			throw new Exception("Leader id not match, expected: " + this.leaderSessionID
+					+ ", actual: " + leaderSessionID);
+		}
+
 		if (executionGraph.updateState(taskExecutionState)) {
 			return Acknowledge.get();
 		} else {
 			throw new ExecutionGraphException("The execution attempt " +
-				taskExecutionState.getID() + " was not found.");
+					taskExecutionState.getID() + " was not found.");
 		}
 	}
 
 	@RpcMethod
 	public SerializedInputSplit requestNextInputSplit(
-		final JobVertexID vertexID,
-		final ExecutionAttemptID executionAttempt) throws Exception
+			final UUID leaderSessionID,
+			final JobVertexID vertexID,
+			final ExecutionAttemptID executionAttempt) throws Exception
 	{
+		if (!this.leaderSessionID.equals(leaderSessionID)) {
+			throw new Exception("Leader id not match, expected: " + this.leaderSessionID
+					+ ", actual: " + leaderSessionID);
+		}
+
 		final Execution execution = executionGraph.getRegisteredExecutions().get(executionAttempt);
 		if (execution == null) {
 			// can happen when JobManager had already unregistered this execution upon on task failure,
@@ -579,7 +457,7 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 		} catch (Exception ex) {
 			log.error("Could not serialize the next input split of class {}.", nextInputSplit.getClass(), ex);
 			IOException reason = new IOException("Could not serialize the next input split of class " +
-				nextInputSplit.getClass() + ".", ex);
+					nextInputSplit.getClass() + ".", ex);
 			vertex.fail(reason);
 			throw reason;
 		}
@@ -587,17 +465,31 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 
 	@RpcMethod
 	public PartitionState requestPartitionState(
-		final ResultPartitionID partitionId,
-		final ExecutionAttemptID taskExecutionId,
-		final IntermediateDataSetID taskResultId)
+			final UUID leaderSessionID,
+			final ResultPartitionID partitionId,
+			final ExecutionAttemptID taskExecutionId,
+			final IntermediateDataSetID taskResultId) throws Exception
 	{
+		if (!this.leaderSessionID.equals(leaderSessionID)) {
+			throw new Exception("Leader id not match, expected: " + this.leaderSessionID
+					+ ", actual: " + leaderSessionID);
+		}
+
 		final Execution execution = executionGraph.getRegisteredExecutions().get(partitionId.getProducerId());
 		final ExecutionState state = execution != null ? execution.getState() : null;
 		return new PartitionState(taskResultId, partitionId.getPartitionId(), state);
 	}
 
 	@RpcMethod
-	public Acknowledge scheduleOrUpdateConsumers(final ResultPartitionID partitionID) {
+	public Acknowledge scheduleOrUpdateConsumers(
+			final UUID leaderSessionID,
+			final ResultPartitionID partitionID) throws Exception
+	{
+		if (!this.leaderSessionID.equals(leaderSessionID)) {
+			throw new Exception("Leader id not match, expected: " + this.leaderSessionID
+					+ ", actual: " + leaderSessionID);
+		}
+
 		executionGraph.scheduleOrUpdateConsumers(partitionID);
 		return Acknowledge.get();
 	}
@@ -609,223 +501,149 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 
 	@RpcMethod
 	public void acknowledgeCheckpoint(
-		JobID jobID,
-		ExecutionAttemptID executionAttemptID,
-		long checkpointID,
-		CheckpointStateHandles checkpointStateHandles,
-		long synchronousDurationMillis,
-		long asynchronousDurationMillis,
-		long bytesBufferedInAlignment,
-		long alignmentDurationNanos) {
-		throw new UnsupportedOperationException();
-	}
-
-	@RpcMethod
-	public void declineCheckpoint(
-		JobID jobID,
-		ExecutionAttemptID executionAttemptID,
-		long checkpointID,
-		long checkpointTimestamp) {
-		throw new UnsupportedOperationException();
-	}
-
-	@RpcMethod
-	public void resourceRemoved(final ResourceID resourceId, final String message) {
-		// TODO: remove resource from slot pool
-	}
+			final JobID jobID,
+			final ExecutionAttemptID executionAttemptID,
+			final CheckpointMetaData checkpointInfo,
+			final CheckpointStateHandles checkpointState)
+	{
+		final CheckpointCoordinator checkpointCoordinator = executionGraph.getCheckpointCoordinator();
+		final AcknowledgeCheckpoint ackMessage = 
+				new AcknowledgeCheckpoint(jobID, executionAttemptID, checkpointInfo, checkpointState);
 
-	@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 {
+						if (!checkpointCoordinator.receiveAcknowledgeMessage(ackMessage)) {
+							log.info("Received message for non-existing checkpoint {}.",
+									checkpointInfo.getCheckpointId());
 						}
+					} catch (Exception e) {
+						log.error("Error in CheckpointCoordinator while processing {}", checkpointInfo, e);
 					}
-				});
-			}
-			else {
-				log.error("Received AcknowledgeCheckpoint message for job {} with no CheckpointCoordinator",
-					jobGraph.getJobID());
-			}
+				}
+			});
 		} else {
-			log.error("Received AcknowledgeCheckpoint for unavailable job {}", jobGraph.getJobID());
+			log.error("Received AcknowledgeCheckpoint message for job {} with no CheckpointCoordinator",
+					jobGraph.getJobID());
 		}
 	}
 
 	@RpcMethod
-	public void declineCheckpoint(final DeclineCheckpoint decline) {
-		if (executionGraph != null) {
-			final CheckpointCoordinator checkpointCoordinator = executionGraph.getCheckpointCoordinator();
-			if (checkpointCoordinator != null) {
-				getRpcService().execute(new Runnable() {
-					@Override
-					public void run() {
-						try {
-							if (!checkpointCoordinator.receiveDeclineMessage(decline)) {
-								log.info("Received message for non-existing checkpoint {}.", decline.getCheckpointId());
-							}
-						} catch (Exception e) {
-							log.error("Error in CheckpointCoordinator while processing {}", decline, e);
+	public void declineCheckpoint(
+			final JobID jobID,
+			final ExecutionAttemptID executionAttemptID,
+			final long checkpointID)
+	{
+		final DeclineCheckpoint decline = new DeclineCheckpoint(
+				jobID, executionAttemptID, checkpointID, 0L);
+		final CheckpointCoordinator checkpointCoordinator = executionGraph.getCheckpointCoordinator();
+
+		if (checkpointCoordinator != null) {
+			getRpcService().execute(new Runnable() {
+				@Override
+				public void run() {
+					try {
+						if (!checkpointCoordinator.receiveDeclineMessage(decline)) {
+							log.info("Received message for non-existing checkpoint {}.", decline.getCheckpointId());
 						}
+					} catch (Exception e) {
+						log.error("Error in CheckpointCoordinator while processing {}", decline, e);
 					}
-				});
-			} else {
-				log.error("Received DeclineCheckpoint message for job {} with no CheckpointCoordinator",
-					jobGraph.getJobID());
-			}
+				}
+			});
 		} else {
-			log.error("Received AcknowledgeCheckpoint for unavailable job {}", jobGraph.getJobID());
+			log.error("Received DeclineCheckpoint message for job {} with no CheckpointCoordinator",
+					jobGraph.getJobID());
 		}
 	}
 
 	@RpcMethod
 	public KvStateLocation lookupKvStateLocation(final String registrationName) throws Exception {
-		if (executionGraph != null) {
-			if (log.isDebugEnabled()) {
-				log.debug("Lookup key-value state for job {} with registration " +
+		if (log.isDebugEnabled()) {
+			log.debug("Lookup key-value state for job {} with registration " +
 					"name {}.", jobGraph.getJobID(), registrationName);
-			}
+		}
 
-			final KvStateLocationRegistry registry = executionGraph.getKvStateLocationRegistry();
-			final KvStateLocation location = registry.getKvStateLocation(registrationName);
-			if (location != null) {
-				return location;
-			} else {
-				throw new UnknownKvStateLocation(registrationName);
-			}
+		final KvStateLocationRegistry registry = executionGraph.getKvStateLocationRegistry();
+		final KvStateLocation location = registry.getKvStateLocation(registrationName);
+		if (location != null) {
+			return location;
 		} else {
-			throw new IllegalStateException("Received lookup KvState location request for unavailable job " +
-				jobGraph.getJobID());
+			throw new UnknownKvStateLocation(registrationName);
 		}
 	}
 
 	@RpcMethod
 	public void notifyKvStateRegistered(
-		final JobVertexID jobVertexId,
-		final KeyGroupRange keyGroupRange,
-		final String registrationName,
-		final KvStateID kvStateId,
-		final KvStateServerAddress kvStateServerAddress)
+			final JobVertexID jobVertexId,
+			final KeyGroupRange keyGroupRange,
+			final String registrationName,
+			final KvStateID kvStateId,
+			final KvStateServerAddress kvStateServerAddress)
 	{
-		if (executionGraph != null) {
-			if (log.isDebugEnabled()) {
-				log.debug("Key value state registered for job {} under name {}.",
+		if (log.isDebugEnabled()) {
+			log.debug("Key value state registered for job {} under name {}.",
 					jobGraph.getJobID(), registrationName);
-			}
-			try {
-				executionGraph.getKvStateLocationRegistry().notifyKvStateRegistered(
-					jobVertexId, keyGroupRange, registrationName, kvStateId, kvStateServerAddress
-				);
-			} catch (Exception e) {
-				log.error("Failed to notify KvStateRegistry about registration {}.", registrationName);
-			}
-		} else {
-			log.error("Received notify KvState registered request for unavailable job " + jobGraph.getJobID());
+		}
+
+		try {
+			executionGraph.getKvStateLocationRegistry().notifyKvStateRegistered(
+					jobVertexId, keyGroupRange, registrationName, kvStateId, kvStateServerAddress);
+		} catch (Exception e) {
+			log.error("Failed to notify KvStateRegistry about registration {}.", registrationName);
 		}
 	}
 
 	@RpcMethod
 	public void notifyKvStateUnregistered(
-		JobVertexID jobVertexId,
-		KeyGroupRange keyGroupRange,
-		String registrationName)
+			JobVertexID jobVertexId,
+			KeyGroupRange keyGroupRange,
+			String registrationName)
 	{
-		if (executionGraph != null) {
-			if (log.isDebugEnabled()) {
-				log.debug("Key value state unregistered for job {} under name {}.",
+		if (log.isDebugEnabled()) {
+			log.debug("Key value state unregistered for job {} under name {}.",
 					jobGraph.getJobID(), registrationName);
-			}
-			try {
-				executionGraph.getKvStateLocationRegistry().notifyKvStateUnregistered(
-					jobVertexId, keyGroupRange, registrationName
-				);
-			} catch (Exception e) {
-				log.error("Failed to notify KvStateRegistry about registration {}.", registrationName);
-			}
-		} else {
-			log.error("Received notify KvState unregistered request for unavailable job " + jobGraph.getJobID());
 		}
-	}
-
-	@RpcMethod
-	public Future<TriggerSavepointResponse> triggerSavepoint() throws Exception {
-		if (executionGraph != null) {
-			final CheckpointCoordinator checkpointCoordinator = executionGraph.getCheckpointCoordinator();
-			if (checkpointCoordinator != null) {
-				try {
-					Future<String> savepointFuture = new FlinkFuture<>(
-						checkpointCoordinator.triggerSavepoint(System.currentTimeMillis()));
-
-					return savepointFuture.handleAsync(new BiFunction<String, Throwable, TriggerSavepointResponse>() {
-						@Override
-						public TriggerSavepointResponse apply(String savepointPath, Throwable throwable) {
-							if (throwable == null) {
-								return new TriggerSavepointResponse.Success(jobGraph.getJobID(), savepointPath);
-							}
-							else {
-								return new TriggerSavepointResponse.Failure(jobGraph.getJobID(),
-									new Exception("Failed to complete savepoint", throwable));
-							}
-						}
-					}, getMainThreadExecutor());
 
-				} catch (Exception e) {
-					FlinkCompletableFuture<TriggerSavepointResponse> future = new FlinkCompletableFuture<>();
-					future.complete(new TriggerSavepointResponse.Failure(jobGraph.getJobID(),
-						new Exception("Failed to trigger savepoint", e)));
-					return future;
-				}
-			} else {
-				FlinkCompletableFuture<TriggerSavepointResponse> future = new FlinkCompletableFuture<>();
-				future.complete(new TriggerSavepointResponse.Failure(jobGraph.getJobID(),
-					new IllegalStateException("Checkpointing disabled. You can enable it via the execution " +
-						"environment of your job.")));
-				return future;
-			}
-		} else {
-			FlinkCompletableFuture<TriggerSavepointResponse> future = new FlinkCompletableFuture<>();
-			future.complete(new TriggerSavepointResponse.Failure(jobGraph.getJobID(),
-				new IllegalArgumentException("Received trigger savepoint request for unavailable job " +
-					jobGraph.getJobID())));
-			return future;
+		try {
+			executionGraph.getKvStateLocationRegistry().notifyKvStateUnregistered(
+					jobVertexId, keyGroupRange, registrationName);
+		} catch (Exception e) {
+			log.error("Failed to notify KvStateRegistry about registration {}.", registrationName);
 		}
 	}
 
 	@RpcMethod
-	public DisposeSavepointResponse disposeSavepoint(final String savepointPath) {
-		try {
-			log.info("Disposing savepoint at {}.", savepointPath);
+	public Future<String> triggerSavepoint(final UUID leaderSessionID, final String targetDirectory) throws Exception {
+		if (!this.leaderSessionID.equals(leaderSessionID)) {
+			throw new Exception("Leader id not match, expected: " + this.leaderSessionID
+					+ ", actual: " + leaderSessionID);
+		}
 
-			// check whether the savepoint exists
-			savepointStore.loadSavepoint(savepointPath);
+		final CheckpointCoordinator checkpointCoordinator = executionGraph.getCheckpointCoordinator();
+		if (checkpointCoordinator != null) {
+			Future<CompletedCheckpoint> completedCheckpointFuture = 
+					checkpointCoordinator.triggerSavepoint(System.currentTimeMillis(), targetDirectory);
 
-			savepointStore.disposeSavepoint(savepointPath);
-			return new DisposeSavepointResponse.Success();
-		} catch (Exception e) {
-			log.error("Failed to dispose savepoint at {}.", savepointPath, e);
-			return new DisposeSavepointResponse.Failure(e);
+			return completedCheckpointFuture.thenApplyAsync(new ApplyFunction<CompletedCheckpoint, String>() {
+				@Override
+				public String apply(CompletedCheckpoint checkpoint) {
+					return checkpoint.getExternalPath();
+				}
+			}, executionContext);
+		} else {
+			throw new IllegalStateException("Checkpointing disabled. You can enable it via the execution " +
+					"environment of your job.");
 		}
 	}
 
 	@RpcMethod
 	public ClassloadingProps requestClassloadingProps() throws Exception {
-		if (executionGraph != null) {
-			return new ClassloadingProps(libraryCacheManager.getBlobServerPort(),
+		return new ClassloadingProps(libraryCacheManager.getBlobServerPort(),
 				executionGraph.getRequiredJarFiles(),
 				executionGraph.getRequiredClasspaths());
-		} else {
-			throw new Exception("Received classloading props request for unavailable job " + jobGraph.getJobID());
-		}
 	}
 
 	//----------------------------------------------------------------------------------------------
@@ -838,12 +656,11 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 			public void run() {
 				log.error("Fatal error occurred on JobManager, cause: {}", cause.getMessage(), cause);
 				shutDown();
-				jobCompletionActions.onFatalError(cause);
+				errorHandler.onFatalError(cause);
 			}
 		});
 	}
 
-	// TODO - wrap this as StatusListenerMessenger's callback with rpc main thread
 	private void jobStatusChanged(final JobStatus newJobStatus, long timestamp, final Throwable error) {
 		final JobID jobID = executionGraph.getJobID();
 		final String jobName = executionGraph.getJobName();
@@ -871,36 +688,33 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 			if (newJobStatus == JobStatus.FINISHED) {
 				try {
 					final Map<String, SerializedValue<Object>> accumulatorResults =
-						executionGraph.getAccumulatorsSerialized();
+							executionGraph.getAccumulatorsSerialized();
 					final SerializedJobExecutionResult result = new SerializedJobExecutionResult(
-						jobID, 0, accumulatorResults // TODO get correct job duration
+							jobID, 0, accumulatorResults // TODO get correct job duration
 					);
 					jobCompletionActions.jobFinished(result.toJobExecutionResult(userCodeLoader));
 				} catch (Exception e) {
 					log.error("Cannot fetch final accumulators for job {} ({})", jobName, jobID, e);
 					final JobExecutionException exception = new JobExecutionException(
-						jobID, "Failed to retrieve accumulator results.", e);
+							jobID, "Failed to retrieve accumulator results.", e);
 					// TODO should we also notify client?
 					jobCompletionActions.jobFailed(exception);
 				}
-			}
-			else if (newJobStatus == JobStatus.CANCELED) {
+			} else if (newJobStatus == JobStatus.CANCELED) {
 				final Throwable unpackedError = SerializedThrowable.get(error, userCodeLoader);
 				final JobExecutionException exception = new JobExecutionException(
-					jobID, "Job was cancelled.", unpackedError);
+						jobID, "Job was cancelled.", unpackedError);
 				// TODO should we also notify client?
 				jobCompletionActions.jobFailed(exception);
-			}
-			else if (newJobStatus == JobStatus.FAILED) {
+			} else if (newJobStatus == JobStatus.FAILED) {
 				final Throwable unpackedError = SerializedThrowable.get(error, userCodeLoader);
 				final JobExecutionException exception = new JobExecutionException(
-					jobID, "Job execution failed.", unpackedError);
+						jobID, "Job execution failed.", unpackedError);
 				// TODO should we also notify client?
 				jobCompletionActions.jobFailed(exception);
-			}
-			else {
+			} else {
 				final JobExecutionException exception = new JobExecutionException(
-					jobID, newJobStatus + " is not a terminal state.");
+						jobID, newJobStatus + " is not a terminal state.");
 				// TODO should we also notify client?
 				jobCompletionActions.jobFailed(exception);
 				throw new RuntimeException(exception);
@@ -909,7 +723,7 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 	}
 
 	private void notifyOfNewResourceManagerLeader(
-		final String resourceManagerAddress, final UUID resourceManagerLeaderId)
+			final String resourceManagerAddress, final UUID resourceManagerLeaderId)
 	{
 		// IMPORTANT: executed by main thread to avoid concurrence
 		runAsync(new Runnable() {
@@ -918,17 +732,15 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 				if (resourceManagerConnection != null) {
 					if (resourceManagerAddress != null) {
 						if (resourceManagerAddress.equals(resourceManagerConnection.getTargetAddress())
-							&& resourceManagerLeaderId.equals(resourceManagerConnection.getTargetLeaderId()))
-						{
+								&& resourceManagerLeaderId.equals(resourceManagerConnection.getTargetLeaderId())) {
 							// both address and leader id are not changed, we can keep the old connection
 							return;
 						}
 						log.info("ResourceManager leader changed from {} to {}. Registering at new leader.",
-							resourceManagerConnection.getTargetAddress(), resourceManagerAddress);
-					}
-					else {
+								resourceManagerConnection.getTargetAddress(), resourceManagerAddress);
+					} else {
 						log.info("Current ResourceManager {} lost leader status. Waiting for new ResourceManager leader.",
-							resourceManagerConnection.getTargetAddress());
+								resourceManagerConnection.getTargetAddress());
 					}
 				}
 
@@ -937,8 +749,8 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 				if (resourceManagerAddress != null) {
 					log.info("Attempting to register at ResourceManager {}", resourceManagerAddress);
 					resourceManagerConnection = new ResourceManagerConnection(
-						log, jobGraph.getJobID(), leaderSessionID,
-						resourceManagerAddress, resourceManagerLeaderId, executionContext);
+							log, jobGraph.getJobID(), leaderSessionID,
+							resourceManagerAddress, resourceManagerLeaderId, executionContext);
 					resourceManagerConnection.start();
 				}
 			}
@@ -952,26 +764,14 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 				// TODO - add tests for comment in https://github.com/apache/flink/pull/2565
 				// verify the response with current connection
 				if (resourceManagerConnection != null
-					&& resourceManagerConnection.getTargetLeaderId().equals(success.getResourceManagerLeaderId())) {
+						&& resourceManagerConnection.getTargetLeaderId().equals(success.getResourceManagerLeaderId())) {
 					log.info("JobManager successfully registered at ResourceManager, leader id: {}.",
-						success.getResourceManagerLeaderId());
+							success.getResourceManagerLeaderId());
 				}
 			}
 		});
 	}
 
-	private void disposeCommunicationWithResourceManager() {
-		// 1. stop the leader retriever so we will not receiving updates anymore
-		try {
-			resourceManagerLeaderRetriever.stop();
-		} catch (Exception e) {
-			log.warn("Failed to stop resource manager leader retriever.");
-		}
-
-		// 2. close current connection with ResourceManager if exists
-		closeResourceManagerConnection();
-	}
-
 	private void closeResourceManagerConnection() {
 		if (resourceManagerConnection != null) {
 			resourceManagerConnection.close();
@@ -980,34 +780,6 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 	}
 
 	//----------------------------------------------------------------------------------------------
-	// Helper methods
-	//----------------------------------------------------------------------------------------------
-
-	/**
-	 * Converts JobVertexIDs to corresponding ExecutionJobVertexes
-	 *
-	 * @param executionGraph The execution graph that holds the relationship
-	 * @param vertexIDs      The vertexIDs need to be converted
-	 * @return The corresponding ExecutionJobVertexes
-	 * @throws JobExecutionException
-	 */
-	private static List<ExecutionJobVertex> getExecutionJobVertexWithId(
-		final ExecutionGraph executionGraph, final List<JobVertexID> vertexIDs)
-		throws JobExecutionException
-	{
-		final List<ExecutionJobVertex> ret = new ArrayList<>(vertexIDs.size());
-		for (JobVertexID vertexID : vertexIDs) {
-			final ExecutionJobVertex executionJobVertex = executionGraph.getJobVertex(vertexID);
-			if (executionJobVertex == null) {
-				throw new JobExecutionException(executionGraph.getJobID(),
-					"The snapshot checkpointing settings refer to non-existent vertex " + vertexID);
-			}
-			ret.add(executionJobVertex);
-		}
-		return ret;
-	}
-
-	//----------------------------------------------------------------------------------------------
 	// Utility classes
 	//----------------------------------------------------------------------------------------------
 
@@ -1024,19 +796,19 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 	}
 
 	private class ResourceManagerConnection
-		extends RegisteredRpcConnection<ResourceManagerGateway, JobMasterRegistrationSuccess>
+			extends RegisteredRpcConnection<ResourceManagerGateway, JobMasterRegistrationSuccess>
 	{
 		private final JobID jobID;
 
 		private final UUID jobManagerLeaderID;
 
 		ResourceManagerConnection(
-			final Logger log,
-			final JobID jobID,
-			final UUID jobManagerLeaderID,
-			final String resourceManagerAddress,
-			final UUID resourceManagerLeaderID,
-			final Executor executor)
+				final Logger log,
+				final JobID jobID,
+				final UUID jobManagerLeaderID,
+				final String resourceManagerAddress,
+				final UUID resourceManagerLeaderID,
+				final Executor executor)
 		{
 			super(log, resourceManagerAddress, resourceManagerLeaderID, executor);
 			this.jobID = checkNotNull(jobID);
@@ -1046,12 +818,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/0615b62f/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterGateway.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterGateway.java
index 4b51258..b27b41c 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterGateway.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterGateway.java
@@ -18,6 +18,8 @@
 
 package org.apache.flink.runtime.jobmaster;
 
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.checkpoint.CheckpointCoordinatorGateway;
 import org.apache.flink.runtime.clusterframework.types.ResourceID;
 import org.apache.flink.runtime.concurrent.Future;
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
@@ -29,15 +31,11 @@ import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
 import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.jobmaster.message.ClassloadingProps;
-import org.apache.flink.runtime.jobmaster.message.DisposeSavepointResponse;
-import org.apache.flink.runtime.jobmaster.message.NextInputSplit;
-import org.apache.flink.runtime.jobmaster.message.TriggerSavepointResponse;
-import org.apache.flink.runtime.messages.checkpoint.AcknowledgeCheckpoint;
-import org.apache.flink.runtime.messages.checkpoint.DeclineCheckpoint;
+import org.apache.flink.runtime.messages.Acknowledge;
 import org.apache.flink.runtime.query.KvStateID;
 import org.apache.flink.runtime.query.KvStateLocation;
 import org.apache.flink.runtime.query.KvStateServerAddress;
-import org.apache.flink.runtime.rpc.RpcGateway;
+import org.apache.flink.runtime.rpc.RpcTimeout;
 import org.apache.flink.runtime.state.KeyGroupRange;
 import org.apache.flink.runtime.state.KvState;
 import org.apache.flink.runtime.taskmanager.TaskExecutionState;
@@ -49,52 +47,56 @@ import java.util.UUID;
  */
 public interface JobMasterGateway extends CheckpointCoordinatorGateway {
 
-	/**
-	 * Starting the job under the given leader session ID.
-	 */
-	void startJob(final UUID leaderSessionID);
+	// ------------------------------------------------------------------------
+	//  Job start and stop methods
+	// ------------------------------------------------------------------------
 
-	/**
-	 * Suspending job, all the running tasks will be cancelled, and runtime status will be cleared.
-	 * Should re-submit the job before restarting it.
-	 *
-	 * @param cause The reason of why this job been suspended.
-	 */
-	void suspendJob(final Throwable cause);
+	void startJobExecution();
+
+	void suspendExecution(Throwable cause);
+
+	// ------------------------------------------------------------------------
 
 	/**
 	 * Updates the task execution state for a given task.
 	 *
+	 * @param leaderSessionID    The leader id of JobManager
 	 * @param taskExecutionState New task execution state for a given task
 	 * @return Future flag of the task execution state update result
 	 */
-	Future<Acknowledge> updateTaskExecutionState(TaskExecutionState taskExecutionState);
+	Future<Acknowledge> updateTaskExecutionState(
+			final UUID leaderSessionID,
+			final TaskExecutionState taskExecutionState);
 
 	/**
 	 * Requesting next input split for the {@link ExecutionJobVertex}. The next input split is sent back to the sender
 	 * as a {@link SerializedInputSplit} message.
 	 *
+	 * @param leaderSessionID  The leader id of JobManager
 	 * @param vertexID         The job vertex id
 	 * @param executionAttempt The execution attempt id
 	 * @return The future of the input split. If there is no further input split, will return an empty object.
 	 */
 	Future<SerializedInputSplit> requestNextInputSplit(
-		final JobVertexID vertexID,
-		final ExecutionAttemptID executionAttempt);
+			final UUID leaderSessionID,
+			final JobVertexID vertexID,
+			final ExecutionAttemptID executionAttempt);
 
 	/**
 	 * Requests the current state of the partition.
 	 * The state of a partition is currently bound to the state of the producing execution.
 	 *
+	 * @param leaderSessionID The leader id of JobManager
 	 * @param partitionId     The partition ID of the partition to request the state of.
 	 * @param taskExecutionId The execution attempt ID of the task requesting the partition state.
 	 * @param taskResultId    The input gate ID of the task requesting the partition state.
 	 * @return The future of the partition state
 	 */
 	Future<PartitionState> requestPartitionState(
-		final ResultPartitionID partitionId,
-		final ExecutionAttemptID taskExecutionId,
-		final IntermediateDataSetID taskResultId);
+			final UUID leaderSessionID,
+			final ResultPartitionID partitionId,
+			final ExecutionAttemptID taskExecutionId,
+			final IntermediateDataSetID taskResultId);
 
 	/**
 	 * Notifies the JobManager about available data for a produced partition.
@@ -105,11 +107,15 @@ public interface JobMasterGateway extends CheckpointCoordinatorGateway {
 	 * <p>
 	 * The JobManager then can decide when to schedule the partition consumers of the given session.
 	 *
-	 * @param partitionID The partition which has already produced data
-	 * @param timeout before the rpc call fails
+	 * @param leaderSessionID The leader id of JobManager
+	 * @param partitionID     The partition which has already produced data
+	 * @param timeout         before the rpc call fails
 	 * @return Future acknowledge of the schedule or update operation
 	 */
-	Future<Acknowledge> scheduleOrUpdateConsumers(final ResultPartitionID partitionID, @RpcTimeout final Time timeout);
+	Future<Acknowledge> scheduleOrUpdateConsumers(
+			final UUID leaderSessionID,
+			final ResultPartitionID partitionID,
+			@RpcTimeout final Time timeout);
 
 	/**
 	 * Disconnects the given {@link org.apache.flink.runtime.taskexecutor.TaskExecutor} from the
@@ -118,31 +124,6 @@ public interface JobMasterGateway extends CheckpointCoordinatorGateway {
 	 * @param resourceID identifying the TaskManager to disconnect
 	 */
 	void disconnectTaskManager(ResourceID resourceID);
-	void scheduleOrUpdateConsumers(final ResultPartitionID partitionID);
-
-	/**
-	 * Notifies the JobManager about the removal of a resource.
-	 *
-	 * @param resourceId The ID under which the resource is registered.
-	 * @param message    Optional message with details, for logging and debugging.
-	 */
-
-	void resourceRemoved(final ResourceID resourceId, final String message);
-
-	/**
-	 * Notifies the JobManager that the checkpoint of an individual task is completed.
-	 *
-	 * @param acknowledge The acknowledge message of the checkpoint
-	 */
-	void acknowledgeCheckpoint(final AcknowledgeCheckpoint acknowledge);
-
-	/**
-	 * Notifies the JobManager that a checkpoint request could not be heeded.
-	 * This can happen if a Task is already in RUNNING state but is internally not yet ready to perform checkpoints.
-	 *
-	 * @param decline The decline message of the checkpoint
-	 */
-	void declineCheckpoint(final DeclineCheckpoint decline);
 
 	/**
 	 * Requests a {@link KvStateLocation} for the specified {@link KvState} registration name.
@@ -150,7 +131,7 @@ public interface JobMasterGateway extends CheckpointCoordinatorGateway {
 	 * @param registrationName Name under which the KvState has been registered.
 	 * @return Future of the requested {@link KvState} location
 	 */
-	Future<KvStateLocation> lookupKvStateLocation(final String registrationName) throws Exception;
+	Future<KvStateLocation> lookupKvStateLocation(final String registrationName);
 
 	/**
 	 * @param jobVertexId          JobVertexID the KvState instance belongs to.
@@ -160,11 +141,11 @@ public interface JobMasterGateway extends CheckpointCoordinatorGateway {
 	 * @param kvStateServerAddress Server address where to find the KvState instance.
 	 */
 	void notifyKvStateRegistered(
-		final JobVertexID jobVertexId,
-		final KeyGroupRange keyGroupRange,
-		final String registrationName,
-		final KvStateID kvStateId,
-		final KvStateServerAddress kvStateServerAddress);
+			final JobVertexID jobVertexId,
+			final KeyGroupRange keyGroupRange,
+			final String registrationName,
+			final KvStateID kvStateId,
+			final KvStateServerAddress kvStateServerAddress);
 
 	/**
 	 * @param jobVertexId      JobVertexID the KvState instance belongs to.
@@ -172,24 +153,18 @@ public interface JobMasterGateway extends CheckpointCoordinatorGateway {
 	 * @param registrationName Name under which the KvState has been registered.
 	 */
 	void notifyKvStateUnregistered(
-		JobVertexID jobVertexId,
-		KeyGroupRange keyGroupRange,
-		String registrationName);
+			JobVertexID jobVertexId,
+			KeyGroupRange keyGroupRange,
+			String registrationName);
 
 	/**
 	 * Notifies the JobManager to trigger a savepoint for this job.
 	 *
-	 * @return Future of the savepoint trigger response.
-	 */
-	Future<TriggerSavepointResponse> triggerSavepoint();
-
-	/**
-	 * Notifies the Jobmanager to dispose specified savepoint.
-	 *
-	 * @param savepointPath The path of the savepoint.
-	 * @return The future of the savepoint disponse response.
+	 * @param leaderSessionID The leader id of JobManager
+	 * @param targetDirectory The directory where the savepoint should be stored
+	 * @return The savepoint path
 	 */
-	Future<DisposeSavepointResponse> disposeSavepoint(final String savepointPath);
+	Future<String> triggerSavepoint(UUID leaderSessionID, String targetDirectory);
 
 	/**
 	 * Request the classloading props of this job.

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


[17/50] [abbrv] flink git commit: [FLINK-4538][FLINK-4348] ResourceManager slot allocation protcol

Posted by se...@apache.org.
[FLINK-4538][FLINK-4348] ResourceManager slot allocation protcol

- associates JobMasters with JobID instead of InstanceID
- adds TaskExecutorGateway to slot
- adds SlotManager as RM constructor parameter
- adds LeaderRetrievalListener to SlotManager to keep track of the leader id

- tests the interaction JM->RM requestSlot
- tests the interaction RM->TM requestSlot

This closes #2463


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

Branch: refs/heads/flip-6
Commit: 74570d454e61a890db4e830b4b71caa178b31bed
Parents: 9c07278
Author: Maximilian Michels <mx...@apache.org>
Authored: Thu Sep 1 16:53:31 2016 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Oct 14 15:14:40 2016 +0200

----------------------------------------------------------------------
 .../clusterframework/types/ResourceProfile.java |   2 +-
 .../clusterframework/types/ResourceSlot.java    |  14 +-
 .../resourcemanager/JobMasterRegistration.java  |  10 +-
 .../resourcemanager/RegistrationResponse.java   |   9 +-
 .../resourcemanager/ResourceManager.java        | 167 +++---
 .../resourcemanager/ResourceManagerGateway.java |   2 +-
 .../runtime/resourcemanager/SlotAssignment.java |  25 -
 .../runtime/resourcemanager/SlotManager.java    | 523 -----------------
 .../resourcemanager/SlotRequestRegistered.java  |  33 ++
 .../resourcemanager/SlotRequestRejected.java    |  34 ++
 .../resourcemanager/SlotRequestReply.java       |  41 ++
 .../slotmanager/SimpleSlotManager.java          |  59 ++
 .../slotmanager/SlotManager.java                | 579 +++++++++++++++++++
 .../flink/runtime/taskexecutor/SlotStatus.java  |   5 +-
 .../taskexecutor/TaskExecutorGateway.java       |  17 +
 .../resourcemanager/ResourceManagerHATest.java  |   4 +-
 .../resourcemanager/SlotManagerTest.java        | 538 -----------------
 .../slotmanager/SlotManagerTest.java            | 554 ++++++++++++++++++
 .../slotmanager/SlotProtocolTest.java           | 225 +++++++
 .../flink/runtime/rpc/TestingRpcService.java    |   6 +-
 .../runtime/rpc/TestingSerialRpcService.java    |   4 +
 21 files changed, 1677 insertions(+), 1174 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/74570d45/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 4e78721..7a25de1 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
@@ -87,7 +87,7 @@ public class ResourceProfile implements Serializable {
 	 * @return true if the requirement is matched, otherwise false
 	 */
 	public boolean isMatching(ResourceProfile required) {
-		return Double.compare(cpuCores, required.getCpuCores()) >= 0 && memoryInMB >= required.getMemoryInMB();
+		return cpuCores >= required.getCpuCores() && memoryInMB >= required.getMemoryInMB();
 	}
 
 	// ------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/flink/blob/74570d45/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceSlot.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceSlot.java b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceSlot.java
index 8a6db5f..5fb8aee 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceSlot.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceSlot.java
@@ -18,6 +18,8 @@
 
 package org.apache.flink.runtime.clusterframework.types;
 
+import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway;
+
 import java.io.Serializable;
 
 import static org.apache.flink.util.Preconditions.checkNotNull;
@@ -26,7 +28,7 @@ import static org.apache.flink.util.Preconditions.checkNotNull;
  * A ResourceSlot represents a slot located in TaskManager from ResourceManager's view. It has a unique
  * identification and resource profile which we can compare to the resource request.
  */
-public class ResourceSlot implements ResourceIDRetrievable, Serializable {
+public class ResourceSlot implements ResourceIDRetrievable {
 
 	private static final long serialVersionUID = -5853720153136840674L;
 
@@ -36,9 +38,13 @@ public class ResourceSlot implements ResourceIDRetrievable, Serializable {
 	/** The resource profile of this slot */
 	private final ResourceProfile resourceProfile;
 
-	public ResourceSlot(SlotID slotId, ResourceProfile resourceProfile) {
+	/** Gateway to the TaskExecutor which owns the slot */
+	private final TaskExecutorGateway taskExecutorGateway;
+
+	public ResourceSlot(SlotID slotId, ResourceProfile resourceProfile, TaskExecutorGateway taskExecutorGateway) {
 		this.slotId = checkNotNull(slotId);
 		this.resourceProfile = checkNotNull(resourceProfile);
+		this.taskExecutorGateway = taskExecutorGateway;
 	}
 
 	@Override
@@ -54,6 +60,10 @@ public class ResourceSlot implements ResourceIDRetrievable, Serializable {
 		return resourceProfile;
 	}
 
+	public TaskExecutorGateway getTaskExecutorGateway() {
+		return taskExecutorGateway;
+	}
+
 	/**
 	 * Check whether required resource profile can be matched by this slot.
 	 *

http://git-wip-us.apache.org/repos/asf/flink/blob/74570d45/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/JobMasterRegistration.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/JobMasterRegistration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/JobMasterRegistration.java
index 309dcc1..439e56b 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/JobMasterRegistration.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/JobMasterRegistration.java
@@ -18,18 +18,26 @@
 
 package org.apache.flink.runtime.resourcemanager;
 
+import org.apache.flink.api.common.JobID;
+
 import java.io.Serializable;
 
 public class JobMasterRegistration implements Serializable {
 	private static final long serialVersionUID = 8411214999193765202L;
 
 	private final String address;
+	private final JobID jobID;
 
-	public JobMasterRegistration(String address) {
+	public JobMasterRegistration(String address, JobID jobID) {
 		this.address = address;
+		this.jobID = jobID;
 	}
 
 	public String getAddress() {
 		return address;
 	}
+
+	public JobID getJobID() {
+		return jobID;
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/74570d45/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/RegistrationResponse.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/RegistrationResponse.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/RegistrationResponse.java
index fb6c401..796e634 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/RegistrationResponse.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/RegistrationResponse.java
@@ -18,26 +18,19 @@
 
 package org.apache.flink.runtime.resourcemanager;
 
-import org.apache.flink.runtime.instance.InstanceID;
-
 import java.io.Serializable;
 
 public class RegistrationResponse implements Serializable {
 	private static final long serialVersionUID = -2379003255993119993L;
 
 	private final boolean isSuccess;
-	private final InstanceID instanceID;
 
-	public RegistrationResponse(boolean isSuccess, InstanceID instanceID) {
+	public RegistrationResponse(boolean isSuccess) {
 		this.isSuccess = isSuccess;
-		this.instanceID = instanceID;
 	}
 
 	public boolean isSuccess() {
 		return isSuccess;
 	}
 
-	public InstanceID getInstanceID() {
-		return instanceID;
-	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/74570d45/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 44c022b..29aba1a 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
@@ -21,11 +21,13 @@ package org.apache.flink.runtime.resourcemanager;
 import akka.dispatch.Mapper;
 
 import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.JobID;
 import org.apache.flink.runtime.clusterframework.types.ResourceID;
 import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
 import org.apache.flink.runtime.instance.InstanceID;
 import org.apache.flink.runtime.leaderelection.LeaderContender;
 import org.apache.flink.runtime.leaderelection.LeaderElectionService;
+import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager;
 import org.apache.flink.runtime.rpc.RpcMethod;
 import org.apache.flink.runtime.rpc.RpcEndpoint;
 import org.apache.flink.runtime.rpc.RpcService;
@@ -33,6 +35,8 @@ import org.apache.flink.runtime.jobmaster.JobMaster;
 import org.apache.flink.runtime.jobmaster.JobMasterGateway;
 import org.apache.flink.runtime.taskexecutor.TaskExecutorRegistrationSuccess;
 
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 import scala.concurrent.Future;
 
 import java.util.HashMap;
@@ -51,16 +55,28 @@ import static org.apache.flink.util.Preconditions.checkNotNull;
  *     <li>{@link #requestSlot(SlotRequest)} requests a slot from the resource manager</li>
  * </ul>
  */
-public class ResourceManager extends RpcEndpoint<ResourceManagerGateway> {
-	private final Map<JobMasterGateway, InstanceID> jobMasterGateways;
+public class ResourceManager extends RpcEndpoint<ResourceManagerGateway> implements LeaderContender {
+
+	private final Logger LOG = LoggerFactory.getLogger(getClass());
+
+	private final Map<JobID, JobMasterGateway> jobMasterGateways;
+
 	private final HighAvailabilityServices highAvailabilityServices;
-	private LeaderElectionService leaderElectionService = null;
-	private UUID leaderSessionID = null;
 
-	public ResourceManager(RpcService rpcService, HighAvailabilityServices highAvailabilityServices) {
+	private LeaderElectionService leaderElectionService;
+
+	private final SlotManager slotManager;
+
+	private UUID leaderSessionID;
+
+	public ResourceManager(
+			RpcService rpcService,
+			HighAvailabilityServices highAvailabilityServices,
+			SlotManager slotManager) {
 		super(rpcService);
 		this.highAvailabilityServices = checkNotNull(highAvailabilityServices);
 		this.jobMasterGateways = new HashMap<>();
+		this.slotManager = slotManager;
 	}
 
 	@Override
@@ -69,7 +85,7 @@ public class ResourceManager extends RpcEndpoint<ResourceManagerGateway> {
 		try {
 			super.start();
 			leaderElectionService = highAvailabilityServices.getResourceManagerLeaderElectionService();
-			leaderElectionService.start(new ResourceManagerLeaderContender());
+			leaderElectionService.start(this);
 		} 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);
@@ -94,7 +110,7 @@ public class ResourceManager extends RpcEndpoint<ResourceManagerGateway> {
 	 */
 	@VisibleForTesting
 	UUID getLeaderSessionID() {
-		return leaderSessionID;
+		return this.leaderSessionID;
 	}
 
 	/**
@@ -105,21 +121,20 @@ public class ResourceManager extends RpcEndpoint<ResourceManagerGateway> {
 	 */
 	@RpcMethod
 	public Future<RegistrationResponse> registerJobMaster(JobMasterRegistration jobMasterRegistration) {
-		Future<JobMasterGateway> jobMasterFuture = getRpcService().connect(jobMasterRegistration.getAddress(), JobMasterGateway.class);
+		final Future<JobMasterGateway> jobMasterFuture =
+			getRpcService().connect(jobMasterRegistration.getAddress(), JobMasterGateway.class);
+		final JobID jobID = jobMasterRegistration.getJobID();
 
 		return jobMasterFuture.map(new Mapper<JobMasterGateway, RegistrationResponse>() {
 			@Override
 			public RegistrationResponse apply(final JobMasterGateway jobMasterGateway) {
-				InstanceID instanceID;
 
-				if (jobMasterGateways.containsKey(jobMasterGateway)) {
-					instanceID = jobMasterGateways.get(jobMasterGateway);
-				} else {
-					instanceID = new InstanceID();
-					jobMasterGateways.put(jobMasterGateway, instanceID);
+				final JobMasterGateway existingGateway = jobMasterGateways.put(jobID, jobMasterGateway);
+				if (existingGateway != null) {
+					LOG.info("Replacing existing gateway {} for JobID {} with  {}.",
+						existingGateway, jobID, jobMasterGateway);
 				}
-
-				return new RegistrationResponse(true, instanceID);
+				return new RegistrationResponse(true);
 			}
 		}, getMainThreadExecutionContext());
 	}
@@ -131,9 +146,16 @@ public class ResourceManager extends RpcEndpoint<ResourceManagerGateway> {
 	 * @return Slot assignment
 	 */
 	@RpcMethod
-	public SlotAssignment requestSlot(SlotRequest slotRequest) {
-		System.out.println("SlotRequest: " + slotRequest);
-		return new SlotAssignment();
+	public SlotRequestReply requestSlot(SlotRequest slotRequest) {
+		final JobID jobId = slotRequest.getJobId();
+		final JobMasterGateway jobMasterGateway = jobMasterGateways.get(jobId);
+
+		if (jobMasterGateway != null) {
+			return slotManager.requestSlot(slotRequest);
+		} else {
+			LOG.info("Ignoring slot request for unknown JobMaster with JobID {}", jobId);
+			return new SlotRequestRejected(slotRequest.getAllocationId());
+		}
 	}
 
 
@@ -154,61 +176,62 @@ public class ResourceManager extends RpcEndpoint<ResourceManagerGateway> {
 		return new TaskExecutorRegistrationSuccess(new InstanceID(), 5000);
 	}
 
-	private class ResourceManagerLeaderContender implements LeaderContender {
-
-		/**
-		 * Callback method when current resourceManager is granted leadership
-		 *
-		 * @param leaderSessionID unique leadershipID
-		 */
-		@Override
-		public void grantLeadership(final UUID leaderSessionID) {
-			runAsync(new Runnable() {
-				@Override
-				public void run() {
-					log.info("ResourceManager {} was granted leadership with leader session ID {}", getAddress(), leaderSessionID);
-					ResourceManager.this.leaderSessionID = leaderSessionID;
-					// confirming the leader session ID might be blocking,
-					leaderElectionService.confirmLeaderSessionID(leaderSessionID);
-				}
-			});
-		}
 
-		/**
-		 * Callback method when current resourceManager lose leadership.
-		 */
-		@Override
-		public void revokeLeadership() {
-			runAsync(new Runnable() {
-				@Override
-				public void run() {
-					log.info("ResourceManager {} was revoked leadership.", getAddress());
-					jobMasterGateways.clear();
-					leaderSessionID = null;
-				}
-			});
-		}
+	// ------------------------------------------------------------------------
+	//  Leader Contender
+	// ------------------------------------------------------------------------
 
-		@Override
-		public String getAddress() {
-			return ResourceManager.this.getAddress();
-		}
+	/**
+	 * Callback method when current resourceManager is granted leadership
+	 *
+	 * @param leaderSessionID unique leadershipID
+	 */
+	@Override
+	public void grantLeadership(final UUID leaderSessionID) {
+		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);
+				// notify SlotManager
+				slotManager.notifyLeaderAddress(getAddress(), leaderSessionID);
+				ResourceManager.this.leaderSessionID = leaderSessionID;
+			}
+		});
+	}
 
-		/**
-		 * Handles error occurring in the leader election service
-		 *
-		 * @param exception Exception being thrown in the leader election service
-		 */
-		@Override
-		public void handleError(final Exception exception) {
-			runAsync(new Runnable() {
-				@Override
-				public void run() {
-					log.error("ResourceManager received an error from the LeaderElectionService.", exception);
-					// terminate ResourceManager in case of an error
-					shutDown();
-				}
-			});
-		}
+	/**
+	 * Callback method when current resourceManager lose leadership.
+	 */
+	@Override
+	public void revokeLeadership() {
+		runAsync(new Runnable() {
+			@Override
+			public void run() {
+				log.info("ResourceManager {} was revoked leadership.", getAddress());
+				jobMasterGateways.clear();
+				ResourceManager.this.leaderSessionID = null;
+			}
+		});
+	}
+
+	/**
+	 * Handles error occurring in the leader election service
+	 *
+	 * @param exception Exception being thrown in the leader election service
+	 */
+	@Override
+	public void handleError(final Exception exception) {
+		runAsync(new Runnable() {
+			@Override
+			public void run() {
+				log.error("ResourceManager received an error from the LeaderElectionService.", exception);
+				// notify SlotManager
+				slotManager.handleError(exception);
+				// terminate ResourceManager in case of an error
+				shutDown();
+			}
+		});
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/74570d45/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 b5782b0..e5c8b64 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
@@ -58,7 +58,7 @@ public interface ResourceManagerGateway extends RpcGateway {
 	 * @param slotRequest Slot request
 	 * @return Future slot assignment
 	 */
-	Future<SlotAssignment> requestSlot(SlotRequest slotRequest);
+	Future<SlotRequestRegistered> requestSlot(SlotRequest slotRequest);
 
 	/**
 	 * 

http://git-wip-us.apache.org/repos/asf/flink/blob/74570d45/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotAssignment.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotAssignment.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotAssignment.java
deleted file mode 100644
index 695204d..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotAssignment.java
+++ /dev/null
@@ -1,25 +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 java.io.Serializable;
-
-public class SlotAssignment implements Serializable{
-	private static final long serialVersionUID = -6990813455942742322L;
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/74570d45/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotManager.java
deleted file mode 100644
index 5c06648..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotManager.java
+++ /dev/null
@@ -1,523 +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.annotation.VisibleForTesting;
-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.ResourceSlot;
-import org.apache.flink.runtime.clusterframework.types.SlotID;
-import org.apache.flink.runtime.taskexecutor.SlotReport;
-import org.apache.flink.runtime.taskexecutor.SlotStatus;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.HashMap;
-import java.util.LinkedHashMap;
-import java.util.Map;
-
-import static org.apache.flink.util.Preconditions.checkNotNull;
-
-/**
- * SlotManager is responsible for receiving slot requests and do slot allocations. It allows to request
- * slots from registered TaskManagers and issues container allocation requests in case of there are not
- * enough available slots. Besides, it should sync its slot allocation with TaskManager's heartbeat.
- * <p>
- * The main operation principle of SlotManager is:
- * <ul>
- * <li>1. All slot allocation status should be synced with TaskManager, which is the ground truth.</li>
- * <li>2. All slots that have registered must be tracked, either by free pool or allocated pool.</li>
- * <li>3. All slot requests will be handled by best efforts, there is no guarantee that one request will be
- * fulfilled in time or correctly allocated. Conflicts or timeout or some special error will happen, it should
- * be handled outside SlotManager. SlotManager will make each decision based on the information it currently
- * holds.</li>
- * </ul>
- * <b>IMPORTANT:</b> This class is <b>Not Thread-safe</b>.
- */
-public abstract class SlotManager {
-
-	private static final Logger LOG = LoggerFactory.getLogger(SlotManager.class);
-
-	/** Gateway to communicate with ResourceManager */
-	private final ResourceManagerGateway resourceManagerGateway;
-
-	/** All registered slots, including free and allocated slots */
-	private final Map<ResourceID, Map<SlotID, ResourceSlot>> registeredSlots;
-
-	/** All pending slot requests, waiting available slots to fulfil */
-	private final Map<AllocationID, SlotRequest> pendingSlotRequests;
-
-	/** All free slots that can be used to be allocated */
-	private final Map<SlotID, ResourceSlot> freeSlots;
-
-	/** All allocations, we can lookup allocations either by SlotID or AllocationID */
-	private final AllocationMap allocationMap;
-
-	public SlotManager(ResourceManagerGateway resourceManagerGateway) {
-		this.resourceManagerGateway = checkNotNull(resourceManagerGateway);
-		this.registeredSlots = new HashMap<>(16);
-		this.pendingSlotRequests = new LinkedHashMap<>(16);
-		this.freeSlots = new HashMap<>(16);
-		this.allocationMap = new AllocationMap();
-	}
-
-	// ------------------------------------------------------------------------
-	//  slot managements
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Request a slot with requirements, we may either fulfill the request or pending it. Trigger container
-	 * allocation if we don't have enough resource. If we have free slot which can match the request, record
-	 * this allocation and forward the request to TaskManager through ResourceManager (we want this done by
-	 * RPC's main thread to avoid race condition).
-	 *
-	 * @param request The detailed request of the slot
-	 */
-	public void requestSlot(final SlotRequest request) {
-		if (isRequestDuplicated(request)) {
-			LOG.warn("Duplicated slot request, AllocationID:{}", request.getAllocationId());
-			return;
-		}
-
-		// try to fulfil the request with current free slots
-		ResourceSlot slot = chooseSlotToUse(request, freeSlots);
-		if (slot != null) {
-			LOG.info("Assigning SlotID({}) to AllocationID({}), JobID:{}", slot.getSlotId(),
-				request.getAllocationId(), request.getJobId());
-
-			// record this allocation in bookkeeping
-			allocationMap.addAllocation(slot.getSlotId(), request.getAllocationId());
-
-			// remove selected slot from free pool
-			freeSlots.remove(slot.getSlotId());
-
-			// TODO: send slot request to TaskManager
-		} else {
-			LOG.info("Cannot fulfil slot request, try to allocate a new container for it, " +
-				"AllocationID:{}, JobID:{}", request.getAllocationId(), request.getJobId());
-			allocateContainer(request.getResourceProfile());
-			pendingSlotRequests.put(request.getAllocationId(), request);
-		}
-	}
-
-	/**
-	 * Sync slot status with TaskManager's SlotReport.
-	 */
-	public void updateSlotStatus(final SlotReport slotReport) {
-		for (SlotStatus slotStatus : slotReport.getSlotsStatus()) {
-			updateSlotStatus(slotStatus);
-		}
-	}
-
-	/**
-	 * The slot request to TaskManager may be either failed by rpc communication (timeout, network error, etc.)
-	 * or really rejected by TaskManager. We shall retry this request by:
-	 * <ul>
-	 * <li>1. verify and clear all the previous allocate information for this request
-	 * <li>2. try to request slot again
-	 * </ul>
-	 * <p>
-	 * This may cause some duplicate allocation, e.g. the slot request to TaskManager is successful but the response
-	 * is lost somehow, so we may request a slot in another TaskManager, this causes two slots assigned to one request,
-	 * but it can be taken care of by rejecting registration at JobManager.
-	 *
-	 * @param originalRequest The original slot request
-	 * @param slotId          The target SlotID
-	 */
-	public void handleSlotRequestFailedAtTaskManager(final SlotRequest originalRequest, final SlotID slotId) {
-		final AllocationID originalAllocationId = originalRequest.getAllocationId();
-		LOG.info("Slot request failed at TaskManager, SlotID:{}, AllocationID:{}, JobID:{}",
-			slotId, originalAllocationId, originalRequest.getJobId());
-
-		// verify the allocation info before we do anything
-		if (freeSlots.containsKey(slotId)) {
-			// this slot is currently empty, no need to de-allocate it from our allocations
-			LOG.info("Original slot is somehow empty, retrying this request");
-
-			// before retry, we should double check whether this request was allocated by some other ways
-			if (!allocationMap.isAllocated(originalAllocationId)) {
-				requestSlot(originalRequest);
-			} else {
-				LOG.info("The failed request has somehow been allocated, SlotID:{}",
-					allocationMap.getSlotID(originalAllocationId));
-			}
-		} else if (allocationMap.isAllocated(slotId)) {
-			final AllocationID currentAllocationId = allocationMap.getAllocationID(slotId);
-
-			// check whether we have an agreement on whom this slot belongs to
-			if (originalAllocationId.equals(currentAllocationId)) {
-				LOG.info("De-allocate this request and retry");
-				allocationMap.removeAllocation(currentAllocationId);
-
-				// put this slot back to free pool
-				ResourceSlot slot = checkNotNull(getRegisteredSlot(slotId));
-				freeSlots.put(slotId, slot);
-
-				// retry the request
-				requestSlot(originalRequest);
-			} else {
-				// the slot is taken by someone else, no need to de-allocate it from our allocations
-				LOG.info("Original slot is taken by someone else, current AllocationID:{}", currentAllocationId);
-
-				// before retry, we should double check whether this request was allocated by some other ways
-				if (!allocationMap.isAllocated(originalAllocationId)) {
-					requestSlot(originalRequest);
-				} else {
-					LOG.info("The failed request is somehow been allocated, SlotID:{}",
-						allocationMap.getSlotID(originalAllocationId));
-				}
-			}
-		} else {
-			LOG.error("BUG! {} is neither in free pool nor in allocated pool", slotId);
-		}
-	}
-
-	/**
-	 * Callback for TaskManager failures. In case that a TaskManager fails, we have to clean up all its slots.
-	 *
-	 * @param resourceId The ResourceID of the TaskManager
-	 */
-	public void notifyTaskManagerFailure(final ResourceID resourceId) {
-		LOG.info("Resource:{} been notified failure", resourceId);
-		final Map<SlotID, ResourceSlot> slotIdsToRemove = registeredSlots.remove(resourceId);
-		if (slotIdsToRemove != null) {
-			for (SlotID slotId : slotIdsToRemove.keySet()) {
-				LOG.info("Removing Slot:{} upon resource failure", slotId);
-				if (freeSlots.containsKey(slotId)) {
-					freeSlots.remove(slotId);
-				} else if (allocationMap.isAllocated(slotId)) {
-					allocationMap.removeAllocation(slotId);
-				} else {
-					LOG.error("BUG! {} is neither in free pool nor in allocated pool", slotId);
-				}
-			}
-		}
-	}
-
-	// ------------------------------------------------------------------------
-	//  internal behaviors
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Update slot status based on TaskManager's report. There are mainly two situations when we receive the report:
-	 * <ul>
-	 * <li>1. The slot is newly registered.</li>
-	 * <li>2. The slot has registered, it contains its current status.</li>
-	 * </ul>
-	 * <p>
-	 * Regarding 1: It's fairly simple, we just record this slot's status, and trigger schedule if slot is empty.
-	 * <p>
-	 * Regarding 2: It will cause some weird situation since we may have some time-gap on how the slot's status really
-	 * is. We may have some updates on the slot's allocation, but it doesn't reflected by TaskManager's heartbeat yet,
-	 * and we may make some wrong decision if we cannot guarantee we have the exact status about all the slots. So
-	 * the principle here is: We always trust TaskManager's heartbeat, we will correct our information based on that
-	 * and take next action based on the diff between our information and heartbeat status.
-	 *
-	 * @param reportedStatus Reported slot status
-	 */
-	void updateSlotStatus(final SlotStatus reportedStatus) {
-		final SlotID slotId = reportedStatus.getSlotID();
-		final ResourceSlot slot = new ResourceSlot(slotId, reportedStatus.getProfiler());
-
-		if (registerNewSlot(slot)) {
-			// we have a newly registered slot
-			LOG.info("New slot appeared, SlotID:{}, AllocationID:{}", slotId, reportedStatus.getAllocationID());
-
-			if (reportedStatus.getAllocationID() != null) {
-				// slot in use, record this in bookkeeping
-				allocationMap.addAllocation(slotId, reportedStatus.getAllocationID());
-			} else {
-				handleFreeSlot(new ResourceSlot(slotId, reportedStatus.getProfiler()));
-			}
-		} else {
-			// slot exists, update current information
-			if (reportedStatus.getAllocationID() != null) {
-				// slot is reported in use
-				final AllocationID reportedAllocationId = reportedStatus.getAllocationID();
-
-				// check whether we also thought this slot is in use
-				if (allocationMap.isAllocated(slotId)) {
-					// we also think that slot is in use, check whether the AllocationID matches
-					final AllocationID currentAllocationId = allocationMap.getAllocationID(slotId);
-
-					if (!reportedAllocationId.equals(currentAllocationId)) {
-						LOG.info("Slot allocation info mismatch! SlotID:{}, current:{}, reported:{}",
-							slotId, currentAllocationId, reportedAllocationId);
-
-						// seems we have a disagreement about the slot assignments, need to correct it
-						allocationMap.removeAllocation(slotId);
-						allocationMap.addAllocation(slotId, reportedAllocationId);
-					}
-				} else {
-					LOG.info("Slot allocation info mismatch! SlotID:{}, current:null, reported:{}",
-						slotId, reportedAllocationId);
-
-					// we thought the slot is free, should correct this information
-					allocationMap.addAllocation(slotId, reportedStatus.getAllocationID());
-
-					// remove this slot from free slots pool
-					freeSlots.remove(slotId);
-				}
-			} else {
-				// slot is reported empty
-
-				// check whether we also thought this slot is empty
-				if (allocationMap.isAllocated(slotId)) {
-					LOG.info("Slot allocation info mismatch! SlotID:{}, current:{}, reported:null",
-						slotId, allocationMap.getAllocationID(slotId));
-
-					// we thought the slot is in use, correct it
-					allocationMap.removeAllocation(slotId);
-
-					// we have a free slot!
-					handleFreeSlot(new ResourceSlot(slotId, reportedStatus.getProfiler()));
-				}
-			}
-		}
-	}
-
-	/**
-	 * When we have a free slot, try to fulfill the pending request first. If any request can be fulfilled,
-	 * record this allocation in bookkeeping and send slot request to TaskManager, else we just add this slot
-	 * to the free pool.
-	 *
-	 * @param freeSlot The free slot
-	 */
-	private void handleFreeSlot(final ResourceSlot freeSlot) {
-		SlotRequest chosenRequest = chooseRequestToFulfill(freeSlot, pendingSlotRequests);
-
-		if (chosenRequest != null) {
-			pendingSlotRequests.remove(chosenRequest.getAllocationId());
-
-			LOG.info("Assigning SlotID({}) to AllocationID({}), JobID:{}", freeSlot.getSlotId(),
-				chosenRequest.getAllocationId(), chosenRequest.getJobId());
-			allocationMap.addAllocation(freeSlot.getSlotId(), chosenRequest.getAllocationId());
-
-			// TODO: send slot request to TaskManager
-		} else {
-			freeSlots.put(freeSlot.getSlotId(), freeSlot);
-		}
-	}
-
-	/**
-	 * Check whether the request is duplicated. We use AllocationID to identify slot request, for each
-	 * formerly received slot request, it is either in pending list or already been allocated.
-	 *
-	 * @param request The slot request
-	 * @return <tt>true</tt> if the request is duplicated
-	 */
-	private boolean isRequestDuplicated(final SlotRequest request) {
-		final AllocationID allocationId = request.getAllocationId();
-		return pendingSlotRequests.containsKey(allocationId)
-			|| allocationMap.isAllocated(allocationId);
-	}
-
-	/**
-	 * Try to register slot, and tell if this slot is newly registered.
-	 *
-	 * @param slot The ResourceSlot which will be checked and registered
-	 * @return <tt>true</tt> if we meet a new slot
-	 */
-	private boolean registerNewSlot(final ResourceSlot slot) {
-		final SlotID slotId = slot.getSlotId();
-		final ResourceID resourceId = slotId.getResourceID();
-		if (!registeredSlots.containsKey(resourceId)) {
-			registeredSlots.put(resourceId, new HashMap<SlotID, ResourceSlot>());
-		}
-		return registeredSlots.get(resourceId).put(slotId, slot) == null;
-	}
-
-	private ResourceSlot getRegisteredSlot(final SlotID slotId) {
-		final ResourceID resourceId = slotId.getResourceID();
-		if (!registeredSlots.containsKey(resourceId)) {
-			return null;
-		}
-		return registeredSlots.get(resourceId).get(slotId);
-	}
-
-	// ------------------------------------------------------------------------
-	//  Framework specific behavior
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Choose a slot to use among all free slots, the behavior is framework specified.
-	 *
-	 * @param request   The slot request
-	 * @param freeSlots All slots which can be used
-	 * @return The slot we choose to use, <tt>null</tt> if we did not find a match
-	 */
-	protected abstract ResourceSlot chooseSlotToUse(final SlotRequest request,
-		final Map<SlotID, ResourceSlot> freeSlots);
-
-	/**
-	 * Choose a pending request to fulfill when we have a free slot, the behavior is framework specified.
-	 *
-	 * @param offeredSlot     The free slot
-	 * @param pendingRequests All the pending slot requests
-	 * @return The chosen SlotRequest, <tt>null</tt> if we did not find a match
-	 */
-	protected abstract SlotRequest chooseRequestToFulfill(final ResourceSlot offeredSlot,
-		final Map<AllocationID, SlotRequest> pendingRequests);
-
-	/**
-	 * The framework specific code for allocating a container for specified resource profile.
-	 *
-	 * @param resourceProfile The resource profile
-	 */
-	protected abstract void allocateContainer(final ResourceProfile resourceProfile);
-
-
-	// ------------------------------------------------------------------------
-	//  Helper classes
-	// ------------------------------------------------------------------------
-
-	/**
-	 * We maintain all the allocations with SlotID and AllocationID. We are able to get or remove the allocation info
-	 * either by SlotID or AllocationID.
-	 */
-	private static class AllocationMap {
-
-		/** All allocated slots (by SlotID) */
-		private final Map<SlotID, AllocationID> allocatedSlots;
-
-		/** All allocated slots (by AllocationID), it'a a inverse view of allocatedSlots */
-		private final Map<AllocationID, SlotID> allocatedSlotsByAllocationId;
-
-		AllocationMap() {
-			this.allocatedSlots = new HashMap<>(16);
-			this.allocatedSlotsByAllocationId = new HashMap<>(16);
-		}
-
-		/**
-		 * Add a allocation
-		 *
-		 * @param slotId       The slot id
-		 * @param allocationId The allocation id
-		 */
-		void addAllocation(final SlotID slotId, final AllocationID allocationId) {
-			allocatedSlots.put(slotId, allocationId);
-			allocatedSlotsByAllocationId.put(allocationId, slotId);
-		}
-
-		/**
-		 * De-allocation with slot id
-		 *
-		 * @param slotId The slot id
-		 */
-		void removeAllocation(final SlotID slotId) {
-			if (allocatedSlots.containsKey(slotId)) {
-				final AllocationID allocationId = allocatedSlots.get(slotId);
-				allocatedSlots.remove(slotId);
-				allocatedSlotsByAllocationId.remove(allocationId);
-			}
-		}
-
-		/**
-		 * De-allocation with allocation id
-		 *
-		 * @param allocationId The allocation id
-		 */
-		void removeAllocation(final AllocationID allocationId) {
-			if (allocatedSlotsByAllocationId.containsKey(allocationId)) {
-				SlotID slotId = allocatedSlotsByAllocationId.get(allocationId);
-				allocatedSlotsByAllocationId.remove(allocationId);
-				allocatedSlots.remove(slotId);
-			}
-		}
-
-		/**
-		 * Check whether allocation exists by slot id
-		 *
-		 * @param slotId The slot id
-		 * @return true if the allocation exists
-		 */
-		boolean isAllocated(final SlotID slotId) {
-			return allocatedSlots.containsKey(slotId);
-		}
-
-		/**
-		 * Check whether allocation exists by allocation id
-		 *
-		 * @param allocationId The allocation id
-		 * @return true if the allocation exists
-		 */
-		boolean isAllocated(final AllocationID allocationId) {
-			return allocatedSlotsByAllocationId.containsKey(allocationId);
-		}
-
-		AllocationID getAllocationID(final SlotID slotId) {
-			return allocatedSlots.get(slotId);
-		}
-
-		SlotID getSlotID(final AllocationID allocationId) {
-			return allocatedSlotsByAllocationId.get(allocationId);
-		}
-
-		public int size() {
-			return allocatedSlots.size();
-		}
-	}
-
-	// ------------------------------------------------------------------------
-	//  Testing utilities
-	// ------------------------------------------------------------------------
-
-	@VisibleForTesting
-	boolean isAllocated(final SlotID slotId) {
-		return allocationMap.isAllocated(slotId);
-	}
-
-	@VisibleForTesting
-	boolean isAllocated(final AllocationID allocationId) {
-		return allocationMap.isAllocated(allocationId);
-	}
-
-	/**
-	 * Add free slots directly to the free pool, this will not trigger pending requests allocation
-	 *
-	 * @param slot The resource slot
-	 */
-	@VisibleForTesting
-	void addFreeSlot(final ResourceSlot slot) {
-		final ResourceID resourceId = slot.getResourceID();
-		final SlotID slotId = slot.getSlotId();
-
-		if (!registeredSlots.containsKey(resourceId)) {
-			registeredSlots.put(resourceId, new HashMap<SlotID, ResourceSlot>());
-		}
-		registeredSlots.get(resourceId).put(slot.getSlotId(), slot);
-		freeSlots.put(slotId, slot);
-	}
-
-	@VisibleForTesting
-	int getAllocatedSlotCount() {
-		return allocationMap.size();
-	}
-
-	@VisibleForTesting
-	int getFreeSlotCount() {
-		return freeSlots.size();
-	}
-
-	@VisibleForTesting
-	int getPendingRequestCount() {
-		return pendingSlotRequests.size();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/74570d45/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotRequestRegistered.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotRequestRegistered.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotRequestRegistered.java
new file mode 100644
index 0000000..6b7f6dc
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotRequestRegistered.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager;
+
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+
+import java.io.Serializable;
+
+/**
+ * Acknowledgment by the ResourceManager for a SlotRequest from the JobManager
+ */
+public class SlotRequestRegistered extends SlotRequestReply {
+
+	public SlotRequestRegistered(AllocationID allocationID) {
+		super(allocationID);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/74570d45/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotRequestRejected.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotRequestRejected.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotRequestRejected.java
new file mode 100644
index 0000000..cb3ec72
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotRequestRejected.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.resourcemanager;
+
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+
+import java.io.Serializable;
+
+/**
+ * Rejection message by the ResourceManager for a SlotRequest from the JobManager
+ */
+public class SlotRequestRejected extends SlotRequestReply {
+
+	public SlotRequestRejected(AllocationID allocationID) {
+		super(allocationID);
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/74570d45/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotRequestReply.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotRequestReply.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotRequestReply.java
new file mode 100644
index 0000000..1b85d0c
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotRequestReply.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.resourcemanager;
+
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+
+import java.io.Serializable;
+
+/**
+ * Acknowledgment by the ResourceManager for a SlotRequest from the JobManager
+ */
+public abstract class SlotRequestReply implements Serializable {
+
+	private static final long serialVersionUID = 42;
+
+	private final AllocationID allocationID;
+
+	public SlotRequestReply(AllocationID allocationID) {
+		this.allocationID = allocationID;
+	}
+
+	public AllocationID getAllocationID() {
+		return allocationID;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/74570d45/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SimpleSlotManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SimpleSlotManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SimpleSlotManager.java
new file mode 100644
index 0000000..ef5ce31
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SimpleSlotManager.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.ResourceSlot;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.resourcemanager.SlotRequest;
+
+import java.util.Iterator;
+import java.util.Map;
+
+/**
+ * A simple SlotManager which ignores resource profiles.
+ */
+public class SimpleSlotManager extends SlotManager {
+
+	@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;
+		}
+	}
+
+	@Override
+	protected void allocateContainer(ResourceProfile resourceProfile) {
+		// TODO
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/74570d45/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManager.java
new file mode 100644
index 0000000..96fde7d
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManager.java
@@ -0,0 +1,579 @@
+/*
+ * 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.annotation.VisibleForTesting;
+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.ResourceSlot;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener;
+import org.apache.flink.runtime.resourcemanager.SlotRequest;
+import org.apache.flink.runtime.resourcemanager.SlotRequestRegistered;
+import org.apache.flink.runtime.resourcemanager.SlotRequestReply;
+import org.apache.flink.runtime.taskexecutor.SlotReport;
+import org.apache.flink.runtime.taskexecutor.SlotStatus;
+import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.concurrent.Future;
+import scala.concurrent.duration.FiniteDuration;
+
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * SlotManager is responsible for receiving slot requests and do slot allocations. It allows to request
+ * slots from registered TaskManagers and issues container allocation requests in case of there are not
+ * enough available slots. Besides, it should sync its slot allocation with TaskManager's heartbeat.
+ * <p>
+ * The main operation principle of SlotManager is:
+ * <ul>
+ * <li>1. All slot allocation status should be synced with TaskManager, which is the ground truth.</li>
+ * <li>2. All slots that have registered must be tracked, either by free pool or allocated pool.</li>
+ * <li>3. All slot requests will be handled by best efforts, there is no guarantee that one request will be
+ * fulfilled in time or correctly allocated. Conflicts or timeout or some special error will happen, it should
+ * be handled outside SlotManager. SlotManager will make each decision based on the information it currently
+ * holds.</li>
+ * </ul>
+ * <b>IMPORTANT:</b> This class is <b>Not Thread-safe</b>.
+ */
+public abstract class SlotManager implements LeaderRetrievalListener {
+
+	protected final Logger LOG = LoggerFactory.getLogger(getClass());
+
+	/** All registered task managers with ResourceID and gateway. */
+	private final Map<ResourceID, TaskExecutorGateway> taskManagerGateways;
+
+	/** All registered slots, including free and allocated slots */
+	private final Map<ResourceID, Map<SlotID, ResourceSlot>> registeredSlots;
+
+	/** All pending slot requests, waiting available slots to fulfil */
+	private final Map<AllocationID, SlotRequest> pendingSlotRequests;
+
+	/** All free slots that can be used to be allocated */
+	private final Map<SlotID, ResourceSlot> freeSlots;
+
+	/** All allocations, we can lookup allocations either by SlotID or AllocationID */
+	private final AllocationMap allocationMap;
+
+	private final FiniteDuration timeout;
+
+	/** The current leader id set by the ResourceManager */
+	private UUID leaderID;
+
+	public SlotManager() {
+		this.registeredSlots = new HashMap<>(16);
+		this.pendingSlotRequests = new LinkedHashMap<>(16);
+		this.freeSlots = new HashMap<>(16);
+		this.allocationMap = new AllocationMap();
+		this.taskManagerGateways = new HashMap<>();
+		this.timeout = new FiniteDuration(10, TimeUnit.SECONDS);
+	}
+
+
+	// ------------------------------------------------------------------------
+	//  slot managements
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Request a slot with requirements, we may either fulfill the request or pending it. Trigger container
+	 * allocation if we don't have enough resource. If we have free slot which can match the request, record
+	 * this allocation and forward the request to TaskManager through ResourceManager (we want this done by
+	 * RPC's main thread to avoid race condition).
+	 *
+	 * @param request The detailed request of the slot
+	 * @return SlotRequestRegistered The confirmation message to be send to the caller
+	 */
+	public SlotRequestRegistered requestSlot(final SlotRequest request) {
+		final AllocationID allocationId = request.getAllocationId();
+		if (isRequestDuplicated(request)) {
+			LOG.warn("Duplicated slot request, AllocationID:{}", allocationId);
+			return null;
+		}
+
+		// try to fulfil the request with current free slots
+		final ResourceSlot slot = chooseSlotToUse(request, freeSlots);
+		if (slot != null) {
+			LOG.info("Assigning SlotID({}) to AllocationID({}), JobID:{}", slot.getSlotId(),
+				allocationId, request.getJobId());
+
+			// record this allocation in bookkeeping
+			allocationMap.addAllocation(slot.getSlotId(), allocationId);
+
+			// remove selected slot from free pool
+			freeSlots.remove(slot.getSlotId());
+
+			final Future<SlotRequestReply> slotRequestReplyFuture =
+				slot.getTaskExecutorGateway().requestSlot(allocationId, leaderID, timeout);
+			// TODO handle timeouts and response
+		} else {
+			LOG.info("Cannot fulfil slot request, try to allocate a new container for it, " +
+				"AllocationID:{}, JobID:{}", allocationId, request.getJobId());
+			allocateContainer(request.getResourceProfile());
+			pendingSlotRequests.put(allocationId, request);
+		}
+
+		return new SlotRequestRegistered(allocationId);
+	}
+
+	/**
+	 * Sync slot status with TaskManager's SlotReport.
+	 */
+	public void updateSlotStatus(final SlotReport slotReport) {
+		for (SlotStatus slotStatus : slotReport.getSlotsStatus()) {
+			updateSlotStatus(slotStatus);
+		}
+	}
+
+	/**
+	 * Registers a TaskExecutor
+	 * @param resourceID TaskExecutor's ResourceID
+	 * @param gateway TaskExcutor's gateway
+	 */
+	public void registerTaskExecutor(ResourceID resourceID, TaskExecutorGateway gateway) {
+		this.taskManagerGateways.put(resourceID, gateway);
+	}
+
+	/**
+	 * The slot request to TaskManager may be either failed by rpc communication (timeout, network error, etc.)
+	 * or really rejected by TaskManager. We shall retry this request by:
+	 * <ul>
+	 * <li>1. verify and clear all the previous allocate information for this request
+	 * <li>2. try to request slot again
+	 * </ul>
+	 * <p>
+	 * This may cause some duplicate allocation, e.g. the slot request to TaskManager is successful but the response
+	 * is lost somehow, so we may request a slot in another TaskManager, this causes two slots assigned to one request,
+	 * but it can be taken care of by rejecting registration at JobManager.
+	 *
+	 * @param originalRequest The original slot request
+	 * @param slotId          The target SlotID
+	 */
+	public void handleSlotRequestFailedAtTaskManager(final SlotRequest originalRequest, final SlotID slotId) {
+		final AllocationID originalAllocationId = originalRequest.getAllocationId();
+		LOG.info("Slot request failed at TaskManager, SlotID:{}, AllocationID:{}, JobID:{}",
+			slotId, originalAllocationId, originalRequest.getJobId());
+
+		// verify the allocation info before we do anything
+		if (freeSlots.containsKey(slotId)) {
+			// this slot is currently empty, no need to de-allocate it from our allocations
+			LOG.info("Original slot is somehow empty, retrying this request");
+
+			// before retry, we should double check whether this request was allocated by some other ways
+			if (!allocationMap.isAllocated(originalAllocationId)) {
+				requestSlot(originalRequest);
+			} else {
+				LOG.info("The failed request has somehow been allocated, SlotID:{}",
+					allocationMap.getSlotID(originalAllocationId));
+			}
+		} else if (allocationMap.isAllocated(slotId)) {
+			final AllocationID currentAllocationId = allocationMap.getAllocationID(slotId);
+
+			// check whether we have an agreement on whom this slot belongs to
+			if (originalAllocationId.equals(currentAllocationId)) {
+				LOG.info("De-allocate this request and retry");
+				allocationMap.removeAllocation(currentAllocationId);
+
+				// put this slot back to free pool
+				ResourceSlot slot = checkNotNull(getRegisteredSlot(slotId));
+				freeSlots.put(slotId, slot);
+
+				// retry the request
+				requestSlot(originalRequest);
+			} else {
+				// the slot is taken by someone else, no need to de-allocate it from our allocations
+				LOG.info("Original slot is taken by someone else, current AllocationID:{}", currentAllocationId);
+
+				// before retry, we should double check whether this request was allocated by some other ways
+				if (!allocationMap.isAllocated(originalAllocationId)) {
+					requestSlot(originalRequest);
+				} else {
+					LOG.info("The failed request is somehow been allocated, SlotID:{}",
+						allocationMap.getSlotID(originalAllocationId));
+				}
+			}
+		} else {
+			LOG.error("BUG! {} is neither in free pool nor in allocated pool", slotId);
+		}
+	}
+
+	/**
+	 * Callback for TaskManager failures. In case that a TaskManager fails, we have to clean up all its slots.
+	 *
+	 * @param resourceId The ResourceID of the TaskManager
+	 */
+	public void notifyTaskManagerFailure(final ResourceID resourceId) {
+		LOG.info("Resource:{} been notified failure", resourceId);
+		taskManagerGateways.remove(resourceId);
+		final Map<SlotID, ResourceSlot> slotIdsToRemove = registeredSlots.remove(resourceId);
+		if (slotIdsToRemove != null) {
+			for (SlotID slotId : slotIdsToRemove.keySet()) {
+				LOG.info("Removing Slot: {} upon resource failure", slotId);
+				if (freeSlots.containsKey(slotId)) {
+					freeSlots.remove(slotId);
+				} else if (allocationMap.isAllocated(slotId)) {
+					allocationMap.removeAllocation(slotId);
+				} else {
+					LOG.error("BUG! {} is neither in free pool nor in allocated pool", slotId);
+				}
+			}
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  internal behaviors
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Update slot status based on TaskManager's report. There are mainly two situations when we receive the report:
+	 * <ul>
+	 * <li>1. The slot is newly registered.</li>
+	 * <li>2. The slot has registered, it contains its current status.</li>
+	 * </ul>
+	 * <p>
+	 * Regarding 1: It's fairly simple, we just record this slot's status, and trigger schedule if slot is empty.
+	 * <p>
+	 * Regarding 2: It will cause some weird situation since we may have some time-gap on how the slot's status really
+	 * is. We may have some updates on the slot's allocation, but it doesn't reflected by TaskManager's heartbeat yet,
+	 * and we may make some wrong decision if we cannot guarantee we have the exact status about all the slots. So
+	 * the principle here is: We always trust TaskManager's heartbeat, we will correct our information based on that
+	 * and take next action based on the diff between our information and heartbeat status.
+	 *
+	 * @param reportedStatus Reported slot status
+	 */
+	void updateSlotStatus(final SlotStatus reportedStatus) {
+		final SlotID slotId = reportedStatus.getSlotID();
+
+		final TaskExecutorGateway taskExecutorGateway = taskManagerGateways.get(slotId.getResourceID());
+		if (taskExecutorGateway == null) {
+			LOG.info("Received SlotStatus but ResourceID {} is unknown to the SlotManager",
+				slotId.getResourceID());
+			return;
+		}
+
+		final ResourceSlot slot = new ResourceSlot(slotId, reportedStatus.getProfiler(), taskExecutorGateway);
+
+		if (registerNewSlot(slot)) {
+			// we have a newly registered slot
+			LOG.info("New slot appeared, SlotID:{}, AllocationID:{}", slotId, reportedStatus.getAllocationID());
+
+			if (reportedStatus.getAllocationID() != null) {
+				// slot in use, record this in bookkeeping
+				allocationMap.addAllocation(slotId, reportedStatus.getAllocationID());
+			} else {
+				handleFreeSlot(slot);
+			}
+		} else {
+			// slot exists, update current information
+			if (reportedStatus.getAllocationID() != null) {
+				// slot is reported in use
+				final AllocationID reportedAllocationId = reportedStatus.getAllocationID();
+
+				// check whether we also thought this slot is in use
+				if (allocationMap.isAllocated(slotId)) {
+					// we also think that slot is in use, check whether the AllocationID matches
+					final AllocationID currentAllocationId = allocationMap.getAllocationID(slotId);
+
+					if (!reportedAllocationId.equals(currentAllocationId)) {
+						LOG.info("Slot allocation info mismatch! SlotID:{}, current:{}, reported:{}",
+							slotId, currentAllocationId, reportedAllocationId);
+
+						// seems we have a disagreement about the slot assignments, need to correct it
+						allocationMap.removeAllocation(slotId);
+						allocationMap.addAllocation(slotId, reportedAllocationId);
+					}
+				} else {
+					LOG.info("Slot allocation info mismatch! SlotID:{}, current:null, reported:{}",
+						slotId, reportedAllocationId);
+
+					// we thought the slot is free, should correct this information
+					allocationMap.addAllocation(slotId, reportedStatus.getAllocationID());
+
+					// remove this slot from free slots pool
+					freeSlots.remove(slotId);
+				}
+			} else {
+				// slot is reported empty
+
+				// check whether we also thought this slot is empty
+				if (allocationMap.isAllocated(slotId)) {
+					LOG.info("Slot allocation info mismatch! SlotID:{}, current:{}, reported:null",
+						slotId, allocationMap.getAllocationID(slotId));
+
+					// we thought the slot is in use, correct it
+					allocationMap.removeAllocation(slotId);
+
+					// we have a free slot!
+					handleFreeSlot(slot);
+				}
+			}
+		}
+	}
+
+	/**
+	 * When we have a free slot, try to fulfill the pending request first. If any request can be fulfilled,
+	 * record this allocation in bookkeeping and send slot request to TaskManager, else we just add this slot
+	 * to the free pool.
+	 *
+	 * @param freeSlot The free slot
+	 */
+	private void handleFreeSlot(final ResourceSlot freeSlot) {
+		SlotRequest chosenRequest = chooseRequestToFulfill(freeSlot, pendingSlotRequests);
+
+		if (chosenRequest != null) {
+			final AllocationID allocationId = chosenRequest.getAllocationId();
+			pendingSlotRequests.remove(allocationId);
+
+			LOG.info("Assigning SlotID({}) to AllocationID({}), JobID:{}", freeSlot.getSlotId(),
+				allocationId, chosenRequest.getJobId());
+			allocationMap.addAllocation(freeSlot.getSlotId(), allocationId);
+
+			final Future<SlotRequestReply> slotRequestReplyFuture =
+				freeSlot.getTaskExecutorGateway().requestSlot(allocationId, leaderID, timeout);
+			// TODO handle timeouts and response
+		} else {
+			freeSlots.put(freeSlot.getSlotId(), freeSlot);
+		}
+	}
+
+	/**
+	 * Check whether the request is duplicated. We use AllocationID to identify slot request, for each
+	 * formerly received slot request, it is either in pending list or already been allocated.
+	 *
+	 * @param request The slot request
+	 * @return <tt>true</tt> if the request is duplicated
+	 */
+	private boolean isRequestDuplicated(final SlotRequest request) {
+		final AllocationID allocationId = request.getAllocationId();
+		return pendingSlotRequests.containsKey(allocationId)
+			|| allocationMap.isAllocated(allocationId);
+	}
+
+	/**
+	 * Try to register slot, and tell if this slot is newly registered.
+	 *
+	 * @param slot The ResourceSlot which will be checked and registered
+	 * @return <tt>true</tt> if we meet a new slot
+	 */
+	private boolean registerNewSlot(final ResourceSlot slot) {
+		final SlotID slotId = slot.getSlotId();
+		final ResourceID resourceId = slotId.getResourceID();
+		if (!registeredSlots.containsKey(resourceId)) {
+			registeredSlots.put(resourceId, new HashMap<SlotID, ResourceSlot>());
+		}
+		return registeredSlots.get(resourceId).put(slotId, slot) == null;
+	}
+
+	private ResourceSlot getRegisteredSlot(final SlotID slotId) {
+		final ResourceID resourceId = slotId.getResourceID();
+		if (!registeredSlots.containsKey(resourceId)) {
+			return null;
+		}
+		return registeredSlots.get(resourceId).get(slotId);
+	}
+
+	// ------------------------------------------------------------------------
+	//  Framework specific behavior
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Choose a slot to use among all free slots, the behavior is framework specified.
+	 *
+	 * @param request   The slot request
+	 * @param freeSlots All slots which can be used
+	 * @return The slot we choose to use, <tt>null</tt> if we did not find a match
+	 */
+	protected abstract ResourceSlot chooseSlotToUse(final SlotRequest request,
+		final Map<SlotID, ResourceSlot> freeSlots);
+
+	/**
+	 * Choose a pending request to fulfill when we have a free slot, the behavior is framework specified.
+	 *
+	 * @param offeredSlot     The free slot
+	 * @param pendingRequests All the pending slot requests
+	 * @return The chosen SlotRequest, <tt>null</tt> if we did not find a match
+	 */
+	protected abstract SlotRequest chooseRequestToFulfill(final ResourceSlot offeredSlot,
+		final Map<AllocationID, SlotRequest> pendingRequests);
+
+	/**
+	 * The framework specific code for allocating a container for specified resource profile.
+	 *
+	 * @param resourceProfile The resource profile
+	 */
+	protected abstract void allocateContainer(final ResourceProfile resourceProfile);
+
+	// ------------------------------------------------------------------------
+	//  Helper classes
+	// ------------------------------------------------------------------------
+
+	/**
+	 * We maintain all the allocations with SlotID and AllocationID. We are able to get or remove the allocation info
+	 * either by SlotID or AllocationID.
+	 */
+	private static class AllocationMap {
+
+		/** All allocated slots (by SlotID) */
+		private final Map<SlotID, AllocationID> allocatedSlots;
+
+		/** All allocated slots (by AllocationID), it'a a inverse view of allocatedSlots */
+		private final Map<AllocationID, SlotID> allocatedSlotsByAllocationId;
+
+		AllocationMap() {
+			this.allocatedSlots = new HashMap<>(16);
+			this.allocatedSlotsByAllocationId = new HashMap<>(16);
+		}
+
+		/**
+		 * Add a allocation
+		 *
+		 * @param slotId       The slot id
+		 * @param allocationId The allocation id
+		 */
+		void addAllocation(final SlotID slotId, final AllocationID allocationId) {
+			allocatedSlots.put(slotId, allocationId);
+			allocatedSlotsByAllocationId.put(allocationId, slotId);
+		}
+
+		/**
+		 * De-allocation with slot id
+		 *
+		 * @param slotId The slot id
+		 */
+		void removeAllocation(final SlotID slotId) {
+			if (allocatedSlots.containsKey(slotId)) {
+				final AllocationID allocationId = allocatedSlots.get(slotId);
+				allocatedSlots.remove(slotId);
+				allocatedSlotsByAllocationId.remove(allocationId);
+			}
+		}
+
+		/**
+		 * De-allocation with allocation id
+		 *
+		 * @param allocationId The allocation id
+		 */
+		void removeAllocation(final AllocationID allocationId) {
+			if (allocatedSlotsByAllocationId.containsKey(allocationId)) {
+				SlotID slotId = allocatedSlotsByAllocationId.get(allocationId);
+				allocatedSlotsByAllocationId.remove(allocationId);
+				allocatedSlots.remove(slotId);
+			}
+		}
+
+		/**
+		 * Check whether allocation exists by slot id
+		 *
+		 * @param slotId The slot id
+		 * @return true if the allocation exists
+		 */
+		boolean isAllocated(final SlotID slotId) {
+			return allocatedSlots.containsKey(slotId);
+		}
+
+		/**
+		 * Check whether allocation exists by allocation id
+		 *
+		 * @param allocationId The allocation id
+		 * @return true if the allocation exists
+		 */
+		boolean isAllocated(final AllocationID allocationId) {
+			return allocatedSlotsByAllocationId.containsKey(allocationId);
+		}
+
+		AllocationID getAllocationID(final SlotID slotId) {
+			return allocatedSlots.get(slotId);
+		}
+
+		SlotID getSlotID(final AllocationID allocationId) {
+			return allocatedSlotsByAllocationId.get(allocationId);
+		}
+
+		public int size() {
+			return allocatedSlots.size();
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  High availability
+	// ------------------------------------------------------------------------
+
+	@Override
+	public void notifyLeaderAddress(String leaderAddress, UUID leaderSessionID) {
+		this.leaderID = leaderSessionID;
+	}
+
+	@Override
+	public void handleError(Exception exception) {
+		LOG.error("Slot Manager received an error from the leader service", exception);
+	}
+
+	// ------------------------------------------------------------------------
+	//  Testing utilities
+	// ------------------------------------------------------------------------
+
+	@VisibleForTesting
+	boolean isAllocated(final SlotID slotId) {
+		return allocationMap.isAllocated(slotId);
+	}
+
+	@VisibleForTesting
+	boolean isAllocated(final AllocationID allocationId) {
+		return allocationMap.isAllocated(allocationId);
+	}
+
+	/**
+	 * Add free slots directly to the free pool, this will not trigger pending requests allocation
+	 *
+	 * @param slot The resource slot
+	 */
+	@VisibleForTesting
+	void addFreeSlot(final ResourceSlot slot) {
+		final ResourceID resourceId = slot.getResourceID();
+		final SlotID slotId = slot.getSlotId();
+
+		if (!registeredSlots.containsKey(resourceId)) {
+			registeredSlots.put(resourceId, new HashMap<SlotID, ResourceSlot>());
+		}
+		registeredSlots.get(resourceId).put(slot.getSlotId(), slot);
+		freeSlots.put(slotId, slot);
+	}
+
+	@VisibleForTesting
+	int getAllocatedSlotCount() {
+		return allocationMap.size();
+	}
+
+	@VisibleForTesting
+	int getFreeSlotCount() {
+		return freeSlots.size();
+	}
+
+	@VisibleForTesting
+	int getPendingRequestCount() {
+		return pendingSlotRequests.size();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/74570d45/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/SlotStatus.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/SlotStatus.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/SlotStatus.java
index 744b674..0f57bb1 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/SlotStatus.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/SlotStatus.java
@@ -50,7 +50,10 @@ public class SlotStatus implements Serializable {
 		this(slotID, profiler, null, null);
 	}
 
-	public SlotStatus(SlotID slotID, ResourceProfile profiler, AllocationID allocationID, JobID jobID) {
+	public SlotStatus(
+			SlotID slotID, ResourceProfile profiler,
+			JobID jobID,
+			AllocationID allocationID) {
 		this.slotID = checkNotNull(slotID, "slotID cannot be null");
 		this.profiler = checkNotNull(profiler, "profile cannot be null");
 		this.allocationID = allocationID;

http://git-wip-us.apache.org/repos/asf/flink/blob/74570d45/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 6c99706..7257436 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
@@ -18,7 +18,12 @@
 
 package org.apache.flink.runtime.taskexecutor;
 
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.resourcemanager.SlotRequestReply;
 import org.apache.flink.runtime.rpc.RpcGateway;
+import org.apache.flink.runtime.rpc.RpcTimeout;
+import scala.concurrent.Future;
+import scala.concurrent.duration.FiniteDuration;
 
 import java.util.UUID;
 
@@ -32,4 +37,16 @@ public interface TaskExecutorGateway extends RpcGateway {
 	// ------------------------------------------------------------------------
 
 	void notifyOfNewResourceManagerLeader(String address, UUID resourceManagerLeaderId);
+
+	/**
+	 * Send by the ResourceManager to the TaskExecutor
+	 * @param allocationID id for the request
+	 * @param resourceManagerLeaderID current leader id of the ResourceManager
+	 * @return SlotRequestReply Answer to the request
+	 */
+
+	Future<SlotRequestReply> requestSlot(
+		AllocationID allocationID,
+		UUID resourceManagerLeaderID,
+		@RpcTimeout FiniteDuration timeout);
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/74570d45/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 5799e62..8183c0a 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
@@ -20,6 +20,7 @@ package org.apache.flink.runtime.resourcemanager;
 
 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.rpc.MainThreadExecutor;
 import org.apache.flink.runtime.rpc.RpcEndpoint;
 import org.apache.flink.runtime.rpc.RpcGateway;
@@ -53,7 +54,8 @@ public class ResourceManagerHATest {
 		TestingHighAvailabilityServices highAvailabilityServices = new TestingHighAvailabilityServices();
 		highAvailabilityServices.setResourceManagerLeaderElectionService(leaderElectionService);
 
-		final ResourceManager resourceManager = new ResourceManager(rpcService, highAvailabilityServices);
+		SlotManager slotManager = mock(SlotManager.class);
+		final ResourceManager resourceManager = new ResourceManager(rpcService, highAvailabilityServices, slotManager);
 		resourceManager.start();
 		// before grant leadership, resourceManager's leaderId is null
 		Assert.assertNull(resourceManager.getLeaderSessionID());


[19/50] [abbrv] flink git commit: [FLINK-4451] [rpc] Throw RpcConnectionException when rpc endpoint is not reachable

Posted by se...@apache.org.
[FLINK-4451] [rpc] Throw RpcConnectionException when rpc endpoint is not reachable

This PR introduces a RpcConnectionException which is thrown if the rpc endpoint
is not reachable when calling RpcService.connect.

This closes #2405.


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

Branch: refs/heads/flip-6
Commit: 3be561f57dab448536e41636997506f5f12aea18
Parents: 6f9936b
Author: Till Rohrmann <tr...@apache.org>
Authored: Tue Aug 23 17:59:54 2016 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Oct 14 15:14:40 2016 +0200

----------------------------------------------------------------------
 .../registration/RetryingRegistration.java      |  2 +-
 .../apache/flink/runtime/rpc/RpcService.java    |  7 +++-
 .../flink/runtime/rpc/akka/AkkaRpcService.java  | 38 +++++++++++-------
 .../rpc/exceptions/RpcConnectionException.java  | 41 ++++++++++++++++++++
 .../runtime/rpc/akka/AkkaRpcActorTest.java      | 18 +++++++++
 5 files changed, 88 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/3be561f5/flink-runtime/src/main/java/org/apache/flink/runtime/registration/RetryingRegistration.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/registration/RetryingRegistration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/registration/RetryingRegistration.java
index 88fe9b5..ea49e42 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/registration/RetryingRegistration.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/registration/RetryingRegistration.java
@@ -197,7 +197,7 @@ public abstract class RetryingRegistration<Gateway extends RpcGateway, Success e
 				@Override
 				public void onFailure(Throwable failure) {
 					if (!isCanceled()) {
-						log.warn("Could not resolve {} address {}, retrying...", targetName, targetAddress);
+						log.warn("Could not resolve {} address {}, retrying...", targetName, targetAddress, failure);
 						startRegistration();
 					}
 				}

http://git-wip-us.apache.org/repos/asf/flink/blob/3be561f5/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 bc0f5cb..78c1cec 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
@@ -18,6 +18,7 @@
 
 package org.apache.flink.runtime.rpc;
 
+import org.apache.flink.runtime.rpc.exceptions.RpcConnectionException;
 import scala.concurrent.ExecutionContext;
 import scala.concurrent.Future;
 
@@ -32,12 +33,14 @@ public interface RpcService {
 
 	/**
 	 * Connect to a remote rpc server under the provided address. Returns a rpc gateway which can
-	 * be used to communicate with the rpc server.
+	 * be used to communicate with the rpc server. If the connection failed, then the returned
+	 * future is failed with a {@link RpcConnectionException}.
 	 *
 	 * @param address Address of the remote rpc server
 	 * @param clazz Class of the rpc gateway to return
 	 * @param <C> Type of the rpc gateway to return
-	 * @return Future containing the rpc gateway
+	 * @return Future containing the rpc gateway or an {@link RpcConnectionException} if the
+	 * connection attempt failed
 	 */
 	<C extends RpcGateway> Future<C> connect(String address, Class<C> clazz);
 

http://git-wip-us.apache.org/repos/asf/flink/blob/3be561f5/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 00a6932..060a1ef 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
@@ -35,6 +35,7 @@ import org.apache.flink.runtime.rpc.RpcGateway;
 import org.apache.flink.runtime.rpc.RpcEndpoint;
 import org.apache.flink.runtime.rpc.RpcService;
 import org.apache.flink.runtime.rpc.StartStoppable;
+import org.apache.flink.runtime.rpc.exceptions.RpcConnectionException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -99,25 +100,32 @@ public class AkkaRpcService implements RpcService {
 		final Future<Object> identify = asker.ask(new Identify(42), timeout);
 		return identify.map(new Mapper<Object, C>(){
 			@Override
-			public C apply(Object obj) {
-				ActorRef actorRef = ((ActorIdentity) obj).getRef();
+			public C checkedApply(Object obj) throws Exception {
 
-				final String address = AkkaUtils.getAkkaURL(actorSystem, actorRef);
+				ActorIdentity actorIdentity = (ActorIdentity) obj;
 
-				InvocationHandler akkaInvocationHandler = new AkkaInvocationHandler(address, actorRef, timeout, maximumFramesize);
+				if (actorIdentity.getRef() == null) {
+					throw new RpcConnectionException("Could not connect to rpc endpoint under address " + address + '.');
+				} else {
+					ActorRef actorRef = actorIdentity.getRef();
+
+					final String address = AkkaUtils.getAkkaURL(actorSystem, actorRef);
+
+					InvocationHandler akkaInvocationHandler = new AkkaInvocationHandler(address, actorRef, timeout, maximumFramesize);
 
-				// Rather than using the System ClassLoader directly, we derive the ClassLoader
-				// from this class . That works better in cases where Flink runs embedded and all Flink
-				// code is loaded dynamically (for example from an OSGI bundle) through a custom ClassLoader
-				ClassLoader classLoader = AkkaRpcService.this.getClass().getClassLoader();
-				
-				@SuppressWarnings("unchecked")
-				C proxy = (C) Proxy.newProxyInstance(
-					classLoader,
-					new Class<?>[] {clazz},
-					akkaInvocationHandler);
+					// Rather than using the System ClassLoader directly, we derive the ClassLoader
+					// from this class . That works better in cases where Flink runs embedded and all Flink
+					// code is loaded dynamically (for example from an OSGI bundle) through a custom ClassLoader
+					ClassLoader classLoader = AkkaRpcService.this.getClass().getClassLoader();
 
-				return proxy;
+					@SuppressWarnings("unchecked")
+					C proxy = (C) Proxy.newProxyInstance(
+						classLoader,
+						new Class<?>[]{clazz},
+						akkaInvocationHandler);
+
+					return proxy;
+				}
 			}
 		}, actorSystem.dispatcher());
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/3be561f5/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
new file mode 100644
index 0000000..a22ebe7
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/exceptions/RpcConnectionException.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.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 {
+	private static final long serialVersionUID = -5500560405481142472L;
+
+	public RpcConnectionException(String message) {
+		super(message);
+	}
+
+	public RpcConnectionException(String message, Throwable cause) {
+		super(message, cause);
+	}
+
+	public RpcConnectionException(Throwable cause) {
+		super(cause);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/3be561f5/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 82d13f0..a6ceb91 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
@@ -25,6 +25,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.exceptions.RpcConnectionException;
 import org.apache.flink.util.TestLogger;
 import org.hamcrest.core.Is;
 import org.junit.AfterClass;
@@ -36,6 +37,7 @@ import java.util.concurrent.TimeUnit;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertThat;
+import static org.junit.Assert.fail;
 
 public class AkkaRpcActorTest extends TestLogger {
 
@@ -73,6 +75,22 @@ public class AkkaRpcActorTest extends TestLogger {
 	}
 
 	/**
+	 * Tests that a {@link RpcConnectionException} is thrown if the rpc endpoint cannot be connected to.
+	 */
+	@Test
+	public void testFailingAddressResolution() throws Exception {
+		Future<DummyRpcGateway> futureRpcGateway = akkaRpcService.connect("foobar", DummyRpcGateway.class);
+
+		try {
+			DummyRpcGateway gateway = Await.result(futureRpcGateway, timeout.duration());
+
+			fail("The rpc connection resolution should have failed.");
+		} catch (RpcConnectionException exception) {
+			// we're expecting a RpcConnectionException
+		}
+	}
+
+	/**
 	 * Tests that the {@link AkkaRpcActor} stashes messages until the corresponding
 	 * {@link RpcEndpoint} has been started.
 	 */


[38/50] [abbrv] flink git commit: [FLINK-4347][FLINK-4348] simplify SlotManager and integrate it with ResourceManager

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/5915613d/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 8f09152..14afd0e 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
@@ -25,10 +25,8 @@ 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.resourcemanager.slotmanager.SimpleSlotManager;
 import org.apache.flink.runtime.rpc.TestingSerialRpcService;
 import org.apache.flink.runtime.registration.RegistrationResponse;
-import org.apache.flink.runtime.rpc.exceptions.LeaderSessionIDException;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -160,7 +158,7 @@ public class ResourceManagerJobMasterTest {
 		TestingHighAvailabilityServices highAvailabilityServices = new TestingHighAvailabilityServices();
 		highAvailabilityServices.setResourceManagerLeaderElectionService(resourceManagerLeaderElectionService);
 		highAvailabilityServices.setJobMasterLeaderRetriever(jobID, jobMasterLeaderRetrievalService);
-		ResourceManager resourceManager = new StandaloneResourceManager(rpcService, highAvailabilityServices, new SimpleSlotManager());
+		ResourceManager resourceManager = new TestingResourceManager(rpcService, highAvailabilityServices);
 		resourceManager.start();
 		return resourceManager;
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/5915613d/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 e6d1ed5..a577c26 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
@@ -22,10 +22,9 @@ 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.resourcemanager.slotmanager.SimpleSlotManager;
 import org.apache.flink.runtime.rpc.TestingSerialRpcService;
 import org.apache.flink.runtime.registration.RegistrationResponse;
-import org.apache.flink.runtime.rpc.exceptions.LeaderSessionIDException;
+import org.apache.flink.runtime.taskexecutor.SlotReport;
 import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway;
 import org.apache.flink.runtime.taskexecutor.TaskExecutorRegistrationSuccess;
 import org.junit.After;
@@ -44,9 +43,24 @@ public class ResourceManagerTaskExecutorTest {
 
 	private TestingSerialRpcService rpcService;
 
+	private SlotReport slotReport = new SlotReport();
+
+	private static String taskExecutorAddress = "/taskExecutor1";
+
+	private ResourceID taskExecutorResourceID;
+
+	private StandaloneResourceManager resourceManager;
+
+	private UUID leaderSessionId;
+
 	@Before
 	public void setup() throws Exception {
 		rpcService = new TestingSerialRpcService();
+
+		taskExecutorResourceID = mockTaskExecutor(taskExecutorAddress);
+		TestingLeaderElectionService rmLeaderElectionService = new TestingLeaderElectionService();
+		resourceManager = createAndStartResourceManager(rmLeaderElectionService);
+		leaderSessionId = grantLeadership(rmLeaderElectionService);
 	}
 
 	@After
@@ -59,19 +73,15 @@ public class ResourceManagerTaskExecutorTest {
 	 */
 	@Test
 	public void testRegisterTaskExecutor() throws Exception {
-		String taskExecutorAddress = "/taskExecutor1";
-		ResourceID taskExecutorResourceID = mockTaskExecutor(taskExecutorAddress);
-		TestingLeaderElectionService rmLeaderElectionService = new TestingLeaderElectionService();
-		final ResourceManager resourceManager = createAndStartResourceManager(rmLeaderElectionService);
-		final UUID leaderSessionId = grantLeadership(rmLeaderElectionService);
-
 		// test response successful
-		Future<RegistrationResponse> successfulFuture = resourceManager.registerTaskExecutor(leaderSessionId, taskExecutorAddress, taskExecutorResourceID);
+		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);
+		Future<RegistrationResponse> duplicateFuture =
+			resourceManager.registerTaskExecutor(leaderSessionId, taskExecutorAddress, taskExecutorResourceID, slotReport);
 		RegistrationResponse duplicateResponse = duplicateFuture.get();
 		assertTrue(duplicateResponse instanceof TaskExecutorRegistrationSuccess);
 		assertNotEquals(((TaskExecutorRegistrationSuccess) response).getRegistrationId(), ((TaskExecutorRegistrationSuccess) duplicateResponse).getRegistrationId());
@@ -82,15 +92,10 @@ public class ResourceManagerTaskExecutorTest {
 	 */
 	@Test
 	public void testRegisterTaskExecutorWithUnmatchedLeaderSessionId() throws Exception {
-		String taskExecutorAddress = "/taskExecutor1";
-		ResourceID taskExecutorResourceID = mockTaskExecutor(taskExecutorAddress);
-		TestingLeaderElectionService rmLeaderElectionService = new TestingLeaderElectionService();
-		final ResourceManager resourceManager = createAndStartResourceManager(rmLeaderElectionService);
-		final UUID leaderSessionId = grantLeadership(rmLeaderElectionService);
-
 		// 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);
+		Future<RegistrationResponse> unMatchedLeaderFuture =
+			resourceManager.registerTaskExecutor(differentLeaderSessionID, taskExecutorAddress, taskExecutorResourceID, slotReport);
 		assertTrue(unMatchedLeaderFuture.get(5, TimeUnit.SECONDS) instanceof RegistrationResponse.Decline);
 	}
 
@@ -99,15 +104,10 @@ public class ResourceManagerTaskExecutorTest {
 	 */
 	@Test
 	public void testRegisterTaskExecutorFromInvalidAddress() throws Exception {
-		String taskExecutorAddress = "/taskExecutor1";
-		ResourceID taskExecutorResourceID = mockTaskExecutor(taskExecutorAddress);
-		TestingLeaderElectionService leaderElectionService = new TestingLeaderElectionService();
-		final ResourceManager resourceManager = createAndStartResourceManager(leaderElectionService);
-		final UUID leaderSessionId = grantLeadership(leaderElectionService);
-
 		// test throw exception when receive a registration from taskExecutor which takes invalid address
 		String invalidAddress = "/taskExecutor2";
-		Future<RegistrationResponse> invalidAddressFuture = resourceManager.registerTaskExecutor(leaderSessionId, invalidAddress, taskExecutorResourceID);
+		Future<RegistrationResponse> invalidAddressFuture =
+			resourceManager.registerTaskExecutor(leaderSessionId, invalidAddress, taskExecutorResourceID, slotReport);
 		assertTrue(invalidAddressFuture.get(5, TimeUnit.SECONDS) instanceof RegistrationResponse.Decline);
 	}
 
@@ -118,10 +118,11 @@ public class ResourceManagerTaskExecutorTest {
 		return taskExecutorResourceID;
 	}
 
-	private ResourceManager createAndStartResourceManager(TestingLeaderElectionService rmLeaderElectionService) {
+	private StandaloneResourceManager createAndStartResourceManager(TestingLeaderElectionService rmLeaderElectionService) {
 		TestingHighAvailabilityServices highAvailabilityServices = new TestingHighAvailabilityServices();
 		highAvailabilityServices.setResourceManagerLeaderElectionService(rmLeaderElectionService);
-		ResourceManager resourceManager = new StandaloneResourceManager(rpcService, highAvailabilityServices, new SimpleSlotManager());
+		StandaloneResourceManager resourceManager =
+			new TestingResourceManager(rpcService, highAvailabilityServices);
 		resourceManager.start();
 		return resourceManager;
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/5915613d/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
new file mode 100644
index 0000000..6b4ca14
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/TestingResourceManager.java
@@ -0,0 +1,53 @@
+/*
+ * 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/5915613d/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/TestingSlotManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/TestingSlotManager.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/TestingSlotManager.java
new file mode 100644
index 0000000..0b2c42b
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/TestingSlotManager.java
@@ -0,0 +1,78 @@
+/*
+ * 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.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.ResourceSlot;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager;
+import org.mockito.Mockito;
+
+import java.util.Iterator;
+import java.util.Map;
+import java.util.concurrent.Executor;
+
+public class TestingSlotManager extends SlotManager {
+
+	public TestingSlotManager() {
+		this(new TestingResourceManagerServices());
+	}
+
+	public TestingSlotManager(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;
+		}
+	}
+
+	private static class TestingResourceManagerServices implements ResourceManagerServices {
+
+		@Override
+		public void allocateResource(ResourceProfile resourceProfile) {
+
+		}
+
+		@Override
+		public Executor getAsyncExecutor() {
+			return Mockito.mock(Executor.class);
+		}
+
+		@Override
+		public Executor getMainThreadExecutor() {
+			return Mockito.mock(Executor.class);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/5915613d/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerTest.java
index 0fed79e..0d2b40d 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerTest.java
@@ -28,13 +28,16 @@ import org.apache.flink.runtime.clusterframework.types.SlotID;
 import org.apache.flink.runtime.concurrent.impl.FlinkCompletableFuture;
 import org.apache.flink.runtime.resourcemanager.ResourceManagerServices;
 import org.apache.flink.runtime.resourcemanager.SlotRequest;
-import org.apache.flink.runtime.resourcemanager.SlotRequestReply;
+import org.apache.flink.runtime.resourcemanager.messages.taskexecutor.TMSlotRequestReply;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorRegistration;
+import org.apache.flink.runtime.taskexecutor.SlotReport;
 import org.apache.flink.runtime.taskexecutor.SlotStatus;
 import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.mockito.Mockito;
 
+import java.util.Collections;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
@@ -45,7 +48,6 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 import static org.mockito.Matchers.any;
-import static org.mockito.Mockito.mock;
 
 public class SlotManagerTest {
 
@@ -59,13 +61,15 @@ public class SlotManagerTest {
 	private static final ResourceProfile DEFAULT_TESTING_BIG_PROFILE =
 		new ResourceProfile(DEFAULT_TESTING_CPU_CORES * 2, DEFAULT_TESTING_MEMORY * 2);
 
-	private static TaskExecutorGateway taskExecutorGateway;
+	private static TaskExecutorRegistration taskExecutorRegistration;
 
 	@BeforeClass
 	public static void setUp() {
-		taskExecutorGateway = Mockito.mock(TaskExecutorGateway.class);
-		Mockito.when(taskExecutorGateway.requestSlot(any(AllocationID.class), any(UUID.class), any(Time.class)))
-			.thenReturn(new FlinkCompletableFuture<SlotRequestReply>());
+		taskExecutorRegistration = Mockito.mock(TaskExecutorRegistration.class);
+		TaskExecutorGateway gateway = Mockito.mock(TaskExecutorGateway.class);
+		Mockito.when(taskExecutorRegistration.getTaskExecutorGateway()).thenReturn(gateway);
+		Mockito.when(gateway.requestSlot(any(SlotID.class), any(AllocationID.class), any(UUID.class), any(Time.class)))
+			.thenReturn(new FlinkCompletableFuture<TMSlotRequestReply>());
 	}
 
 	/**
@@ -180,9 +184,9 @@ public class SlotManagerTest {
 		assertEquals(1, slotManager.getPendingRequestCount());
 
 		SlotID slotId = SlotID.generate();
-		slotManager.registerTaskExecutor(slotId.getResourceID(), taskExecutorGateway);
 		SlotStatus slotStatus = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE);
-		slotManager.updateSlotStatus(slotStatus);
+		SlotReport slotReport = new SlotReport(Collections.singletonList(slotStatus));
+		slotManager.registerTaskExecutor(slotId.getResourceID(), taskExecutorRegistration, slotReport);
 
 		assertEquals(1, slotManager.getAllocatedSlotCount());
 		assertEquals(0, slotManager.getFreeSlotCount());
@@ -198,9 +202,9 @@ public class SlotManagerTest {
 		TestingSlotManager slotManager = new TestingSlotManager();
 
 		SlotID slotId = SlotID.generate();
-		slotManager.registerTaskExecutor(slotId.getResourceID(), taskExecutorGateway);
 		SlotStatus slotStatus = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE);
-		slotManager.updateSlotStatus(slotStatus);
+		SlotReport slotReport = new SlotReport(Collections.singletonList(slotStatus));
+		slotManager.registerTaskExecutor(slotId.getResourceID(), taskExecutorRegistration, slotReport);
 
 		assertEquals(0, slotManager.getAllocatedSlotCount());
 		assertEquals(1, slotManager.getFreeSlotCount());
@@ -216,9 +220,9 @@ public class SlotManagerTest {
 		assertEquals(1, slotManager.getPendingRequestCount());
 
 		SlotID slotId = SlotID.generate();
-		slotManager.registerTaskExecutor(slotId.getResourceID(), taskExecutorGateway);
 		SlotStatus slotStatus = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE);
-		slotManager.updateSlotStatus(slotStatus);
+		SlotReport slotReport = new SlotReport(Collections.singletonList(slotStatus));
+		slotManager.registerTaskExecutor(slotId.getResourceID(), taskExecutorRegistration, slotReport);
 
 		assertEquals(0, slotManager.getAllocatedSlotCount());
 		assertEquals(1, slotManager.getFreeSlotCount());
@@ -234,9 +238,9 @@ public class SlotManagerTest {
 		TestingSlotManager slotManager = new TestingSlotManager();
 
 		SlotID slotId = SlotID.generate();
-		slotManager.registerTaskExecutor(slotId.getResourceID(), taskExecutorGateway);
 		SlotStatus slotStatus = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE, new JobID(), new AllocationID());
-		slotManager.updateSlotStatus(slotStatus);
+		SlotReport slotReport = new SlotReport(Collections.singletonList(slotStatus));
+		slotManager.registerTaskExecutor(slotId.getResourceID(), taskExecutorRegistration, slotReport);
 
 		assertEquals(1, slotManager.getAllocatedSlotCount());
 		assertEquals(0, slotManager.getFreeSlotCount());
@@ -244,48 +248,44 @@ public class SlotManagerTest {
 	}
 
 	/**
-	 * Tests that we had a slot in-use, and it's confirmed by SlotReport
+	 * Tests that we had a slot in-use and is freed again subsequently.
 	 */
 	@Test
 	public void testExistingInUseSlotUpdateStatus() {
 		TestingSlotManager slotManager = new TestingSlotManager();
-		SlotRequest request = new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE);
-		slotManager.requestSlot(request);
 
-		// make this slot in use
 		SlotID slotId = SlotID.generate();
-		slotManager.registerTaskExecutor(slotId.getResourceID(), taskExecutorGateway);
-		SlotStatus slotStatus = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE);
-		slotManager.updateSlotStatus(slotStatus);
+		SlotStatus slotStatus = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE, new JobID(), new AllocationID());
+		SlotReport slotReport = new SlotReport(Collections.singletonList(slotStatus));
+		slotManager.registerTaskExecutor(slotId.getResourceID(), taskExecutorRegistration, slotReport);
 
 		assertEquals(1, slotManager.getAllocatedSlotCount());
 		assertEquals(0, slotManager.getFreeSlotCount());
 		assertTrue(slotManager.isAllocated(slotId));
 
-		// slot status is confirmed
-		SlotStatus slotStatus2 = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE,
-			request.getJobId(), request.getAllocationId());
-		slotManager.updateSlotStatus(slotStatus2);
+		// slot is freed again
+		slotManager.notifySlotAvailable(slotId.getResourceID(), slotId);
 
-		assertEquals(1, slotManager.getAllocatedSlotCount());
-		assertEquals(0, slotManager.getFreeSlotCount());
-		assertTrue(slotManager.isAllocated(slotId));
+		assertEquals(0, slotManager.getAllocatedSlotCount());
+		assertEquals(1, slotManager.getFreeSlotCount());
+		assertFalse(slotManager.isAllocated(slotId));
 	}
 
 	/**
-	 * Tests that we had a slot in-use, but it's empty according to the SlotReport
+	 * Tests multiple slot requests with one slots.
 	 */
 	@Test
-	public void testExistingInUseSlotAdjustedToEmpty() {
+	public void testMultipleSlotRequestsWithOneSlot() {
 		TestingSlotManager slotManager = new TestingSlotManager();
-		SlotRequest request1 = new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE);
+		final AllocationID allocationID = new AllocationID();
+
+		SlotRequest request1 = new SlotRequest(new JobID(), allocationID, DEFAULT_TESTING_PROFILE);
 		slotManager.requestSlot(request1);
 
-		// make this slot in use
-		SlotID slotId = SlotID.generate();
-		slotManager.registerTaskExecutor(slotId.getResourceID(), taskExecutorGateway);
-		SlotStatus slotStatus = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE);
-		slotManager.updateSlotStatus(slotStatus);
+		final ResourceID resourceID = ResourceID.generate();
+		final SlotStatus slotStatus = new SlotStatus(new SlotID(resourceID, 0), DEFAULT_TESTING_PROFILE);
+		final SlotReport slotReport = new SlotReport(slotStatus);
+		slotManager.registerTaskExecutor(resourceID, taskExecutorRegistration, slotReport);
 
 		// another request pending
 		SlotRequest request2 = new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE);
@@ -294,66 +294,20 @@ public class SlotManagerTest {
 		assertEquals(1, slotManager.getAllocatedSlotCount());
 		assertEquals(0, slotManager.getFreeSlotCount());
 		assertEquals(1, slotManager.getPendingRequestCount());
-		assertTrue(slotManager.isAllocated(slotId));
+		assertTrue(slotManager.isAllocated(allocationID));
 		assertTrue(slotManager.isAllocated(request1.getAllocationId()));
 
-
-		// but slot is reported empty again, request2 will be fulfilled, request1 will be missing
-		slotManager.updateSlotStatus(slotStatus);
+		// but slot is reported empty in a report in the meantime which shouldn't affect the state
+		slotManager.notifySlotAvailable(resourceID, slotStatus.getSlotID());
 
 		assertEquals(1, slotManager.getAllocatedSlotCount());
 		assertEquals(0, slotManager.getFreeSlotCount());
 		assertEquals(0, slotManager.getPendingRequestCount());
-		assertTrue(slotManager.isAllocated(slotId));
+		assertTrue(slotManager.isAllocated(slotStatus.getSlotID()));
 		assertTrue(slotManager.isAllocated(request2.getAllocationId()));
-	}
-
-	/**
-	 * Tests that we had a slot in use, and it's also reported in use by TaskManager, but the allocation
-	 * information didn't match.
-	 */
-	@Test
-	public void testExistingInUseSlotWithDifferentAllocationInfo() {
-		TestingSlotManager slotManager = new TestingSlotManager();
-		SlotRequest request = new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE);
-		slotManager.requestSlot(request);
-
-		// make this slot in use
-		SlotID slotId = SlotID.generate();
-		slotManager.registerTaskExecutor(slotId.getResourceID(), taskExecutorGateway);
-		SlotStatus slotStatus = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE);
-		slotManager.updateSlotStatus(slotStatus);
-
-		assertEquals(1, slotManager.getAllocatedSlotCount());
-		assertEquals(0, slotManager.getFreeSlotCount());
-		assertEquals(0, slotManager.getPendingRequestCount());
-		assertTrue(slotManager.isAllocated(slotId));
-		assertTrue(slotManager.isAllocated(request.getAllocationId()));
-
-		SlotStatus slotStatus2 = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE, new JobID(), new AllocationID());
-		// update slot status with different allocation info
-		slotManager.updateSlotStatus(slotStatus2);
-
-		// original request is missing and won't be allocated
-		assertEquals(1, slotManager.getAllocatedSlotCount());
-		assertEquals(0, slotManager.getFreeSlotCount());
-		assertEquals(0, slotManager.getPendingRequestCount());
-		assertTrue(slotManager.isAllocated(slotId));
-		assertFalse(slotManager.isAllocated(request.getAllocationId()));
-		assertTrue(slotManager.isAllocated(slotStatus2.getAllocationID()));
-	}
-
-	/**
-	 * Tests that we had a free slot, and it's confirmed by SlotReport
-	 */
-	@Test
-	public void testExistingEmptySlotUpdateStatus() {
-		TestingSlotManager slotManager = new TestingSlotManager();
-		ResourceSlot slot = new ResourceSlot(SlotID.generate(), DEFAULT_TESTING_PROFILE, taskExecutorGateway);
-		slotManager.addFreeSlot(slot);
 
-		SlotStatus slotStatus = new SlotStatus(slot.getSlotId(), DEFAULT_TESTING_PROFILE);
-		slotManager.updateSlotStatus(slotStatus);
+		// but slot is reported empty in a report in the meantime which shouldn't affect the state
+		slotManager.notifySlotAvailable(resourceID, slotStatus.getSlotID());
 
 		assertEquals(0, slotManager.getAllocatedSlotCount());
 		assertEquals(1, slotManager.getFreeSlotCount());
@@ -361,34 +315,12 @@ public class SlotManagerTest {
 	}
 
 	/**
-	 * Tests that we had a free slot, and it's reported in-use by TaskManager
-	 */
-	@Test
-	public void testExistingEmptySlotAdjustedToInUse() {
-		TestingSlotManager slotManager = new TestingSlotManager();
-		final SlotID slotID = SlotID.generate();
-		slotManager.registerTaskExecutor(slotID.getResourceID(), taskExecutorGateway);
-
-		ResourceSlot slot = new ResourceSlot(slotID, DEFAULT_TESTING_PROFILE, taskExecutorGateway);
-		slotManager.addFreeSlot(slot);
-
-		SlotStatus slotStatus = new SlotStatus(slot.getSlotId(), DEFAULT_TESTING_PROFILE,
-			new JobID(), new AllocationID());
-		slotManager.updateSlotStatus(slotStatus);
-
-		assertEquals(1, slotManager.getAllocatedSlotCount());
-		assertEquals(0, slotManager.getFreeSlotCount());
-		assertEquals(0, slotManager.getPendingRequestCount());
-		assertTrue(slotManager.isAllocated(slot.getSlotId()));
-	}
-
-	/**
 	 * Tests that we did some allocation but failed / rejected by TaskManager, request will retry
 	 */
 	@Test
 	public void testSlotAllocationFailedAtTaskManager() {
 		TestingSlotManager slotManager = new TestingSlotManager();
-		ResourceSlot slot = new ResourceSlot(SlotID.generate(), DEFAULT_TESTING_PROFILE, taskExecutorGateway);
+		ResourceSlot slot = new ResourceSlot(SlotID.generate(), DEFAULT_TESTING_PROFILE, taskExecutorRegistration);
 		slotManager.addFreeSlot(slot);
 
 		SlotRequest request = new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE);
@@ -409,24 +341,31 @@ public class SlotManagerTest {
 
 	/**
 	 * Tests that we did some allocation but failed / rejected by TaskManager, and slot is occupied by another request
+	 * This can only occur after reconnect of the TaskExecutor.
 	 */
 	@Test
 	public void testSlotAllocationFailedAtTaskManagerOccupiedByOther() {
 		TestingSlotManager slotManager = new TestingSlotManager();
 		final SlotID slotID = SlotID.generate();
-		slotManager.registerTaskExecutor(slotID.getResourceID(), taskExecutorGateway);
-
-		ResourceSlot slot = new ResourceSlot(slotID, DEFAULT_TESTING_PROFILE, taskExecutorGateway);
-		slotManager.addFreeSlot(slot);
+		SlotStatus slot = new SlotStatus(slotID, DEFAULT_TESTING_PROFILE);
+		SlotReport slotReport = new SlotReport(slot);
+		slotManager.registerTaskExecutor(slotID.getResourceID(), taskExecutorRegistration, slotReport);
 
 		SlotRequest request = new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE);
 		slotManager.requestSlot(request);
 
-		// slot is set empty by heartbeat
-		SlotStatus slotStatus = new SlotStatus(slot.getSlotId(), slot.getResourceProfile());
-		slotManager.updateSlotStatus(slotStatus);
+		assertEquals(1, slotManager.getAllocatedSlotCount());
+		assertEquals(0, slotManager.getFreeSlotCount());
+		assertEquals(0, slotManager.getPendingRequestCount());
+
+		// slot is set empty by a reconnect of the TaskExecutor
+		slotManager.registerTaskExecutor(slotID.getResourceID(), taskExecutorRegistration, slotReport);
+
+		assertEquals(0, slotManager.getAllocatedSlotCount());
+		assertEquals(1, slotManager.getFreeSlotCount());
+		assertEquals(0, slotManager.getPendingRequestCount());
 
-		// another request took this slot
+		// another request takes the slot
 		SlotRequest request2 = new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE);
 		slotManager.requestSlot(request2);
 
@@ -436,12 +375,12 @@ public class SlotManagerTest {
 		assertFalse(slotManager.isAllocated(request.getAllocationId()));
 		assertTrue(slotManager.isAllocated(request2.getAllocationId()));
 
-		// original request should be pended
-		slotManager.handleSlotRequestFailedAtTaskManager(request, slot.getSlotId());
+		// original request should be retried
+		slotManager.handleSlotRequestFailedAtTaskManager(request, slotID);
 
 		assertEquals(1, slotManager.getAllocatedSlotCount());
 		assertEquals(0, slotManager.getFreeSlotCount());
-		assertEquals(1, slotManager.getPendingRequestCount());
+		assertEquals(0, slotManager.getPendingRequestCount());
 		assertFalse(slotManager.isAllocated(request.getAllocationId()));
 		assertTrue(slotManager.isAllocated(request2.getAllocationId()));
 	}
@@ -453,10 +392,10 @@ public class SlotManagerTest {
 		ResourceID resource1 = ResourceID.generate();
 		ResourceID resource2 = ResourceID.generate();
 
-		ResourceSlot slot11 = new ResourceSlot(new SlotID(resource1, 1), DEFAULT_TESTING_PROFILE, taskExecutorGateway);
-		ResourceSlot slot12 = new ResourceSlot(new SlotID(resource1, 2), DEFAULT_TESTING_PROFILE, taskExecutorGateway);
-		ResourceSlot slot21 = new ResourceSlot(new SlotID(resource2, 1), DEFAULT_TESTING_PROFILE, taskExecutorGateway);
-		ResourceSlot slot22 = new ResourceSlot(new SlotID(resource2, 2), DEFAULT_TESTING_PROFILE, taskExecutorGateway);
+		ResourceSlot slot11 = new ResourceSlot(new SlotID(resource1, 1), DEFAULT_TESTING_PROFILE, taskExecutorRegistration);
+		ResourceSlot slot12 = new ResourceSlot(new SlotID(resource1, 2), DEFAULT_TESTING_PROFILE, taskExecutorRegistration);
+		ResourceSlot slot21 = new ResourceSlot(new SlotID(resource2, 1), DEFAULT_TESTING_PROFILE, taskExecutorRegistration);
+		ResourceSlot slot22 = new ResourceSlot(new SlotID(resource2, 2), DEFAULT_TESTING_PROFILE, taskExecutorRegistration);
 
 		slotManager.addFreeSlot(slot11);
 		slotManager.addFreeSlot(slot21);
@@ -499,7 +438,7 @@ public class SlotManagerTest {
 		final int freeSlotNum)
 	{
 		for (int i = 0; i < freeSlotNum; ++i) {
-			slotManager.addFreeSlot(new ResourceSlot(SlotID.generate(), new ResourceProfile(resourceProfile), taskExecutorGateway));
+			slotManager.addFreeSlot(new ResourceSlot(SlotID.generate(), new ResourceProfile(resourceProfile), taskExecutorRegistration));
 		}
 	}
 
@@ -507,13 +446,13 @@ public class SlotManagerTest {
 	//  testing classes
 	// ------------------------------------------------------------------------
 
-	private static class TestingSlotManager extends SlotManager implements ResourceManagerServices {
+	private static class TestingSlotManager extends SlotManager {
 
-		private final List<ResourceProfile> allocatedContainers;
+		private static TestingRmServices testingRmServices = new TestingRmServices();
 
 		TestingSlotManager() {
-			this.allocatedContainers = new LinkedList<>();
-			setupResourceManagerServices(this);
+			super(testingRmServices);
+			testingRmServices.allocatedContainers.clear();
 		}
 
 		/**
@@ -552,24 +491,34 @@ public class SlotManagerTest {
 			return null;
 		}
 
-		@Override
-		public void allocateResource(ResourceProfile resourceProfile) {
-			allocatedContainers.add(resourceProfile);
+		List<ResourceProfile> getAllocatedContainers() {
+			return testingRmServices.allocatedContainers;
 		}
 
-		@Override
-		public Executor getAsyncExecutor() {
-			return Mockito.mock(Executor.class);
-		}
 
-		@Override
-		public Executor getExecutor() {
-			return Mockito.mock(Executor.class);
-		}
+		private static class TestingRmServices implements ResourceManagerServices {
 
-		List<ResourceProfile> getAllocatedContainers() {
-			return allocatedContainers;
-		}
+			private List<ResourceProfile> allocatedContainers;
+
+			public TestingRmServices() {
+				this.allocatedContainers = new LinkedList<>();
+			}
+
+			@Override
+			public void allocateResource(ResourceProfile resourceProfile) {
+				allocatedContainers.add(resourceProfile);
+			}
 
+			@Override
+			public Executor getAsyncExecutor() {
+				return Mockito.mock(Executor.class);
+			}
+
+			@Override
+			public Executor getMainThreadExecutor() {
+				return Mockito.mock(Executor.class);
+			}
+
+		}
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/5915613d/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 a87fe42..24d959e 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
@@ -25,12 +25,20 @@ import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
 import org.apache.flink.runtime.clusterframework.types.SlotID;
 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.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.registration.RegistrationResponse;
-import org.apache.flink.runtime.resourcemanager.*;
+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.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.RpcService;
 import org.apache.flink.runtime.rpc.TestingSerialRpcService;
 import org.apache.flink.runtime.taskexecutor.SlotReport;
 import org.apache.flink.runtime.taskexecutor.SlotStatus;
@@ -94,9 +102,9 @@ public class SlotProtocolTest extends TestLogger {
 		TestingLeaderElectionService rmLeaderElectionService =
 			configureHA(testingHaServices, jobID, rmAddress, rmLeaderID, jmAddress, jmLeaderID);
 
-		SlotManager slotManager = Mockito.spy(new SimpleSlotManager());
-		ResourceManager resourceManager =
-			Mockito.spy(new StandaloneResourceManager(testRpcService, testingHaServices, slotManager));
+		final TestingSlotManagerFactory slotManagerFactory = new TestingSlotManagerFactory();
+		SpiedResourceManager resourceManager =
+			new SpiedResourceManager(testRpcService, testingHaServices, slotManagerFactory);
 		resourceManager.start();
 		rmLeaderElectionService.isLeader(rmLeaderID);
 
@@ -108,11 +116,13 @@ public class SlotProtocolTest extends TestLogger {
 			Assert.fail("JobManager registration Future didn't become ready.");
 		}
 
+		final SlotManager slotManager = slotManagerFactory.slotManager;
+
 		final AllocationID allocationID = new AllocationID();
 		final ResourceProfile resourceProfile = new ResourceProfile(1.0, 100);
 
 		SlotRequest slotRequest = new SlotRequest(jobID, allocationID, resourceProfile);
-		SlotRequestReply slotRequestReply =
+		RMSlotRequestReply slotRequestReply =
 			resourceManager.requestSlot(jmLeaderID, rmLeaderID, slotRequest);
 
 		// 1) SlotRequest is routed to the SlotManager
@@ -124,15 +134,18 @@ public class SlotProtocolTest extends TestLogger {
 			allocationID);
 
 		// 3) SlotRequest leads to a container allocation
-		verify(resourceManager, timeout(5000)).startNewWorker(resourceProfile);
+		Assert.assertEquals(1, resourceManager.startNewWorkerCalled);
 
 		Assert.assertFalse(slotManager.isAllocated(allocationID));
 
 		// slot becomes available
 		final String tmAddress = "/tm1";
 		TaskExecutorGateway taskExecutorGateway = mock(TaskExecutorGateway.class);
-		Mockito.when(taskExecutorGateway.requestSlot(any(AllocationID.class), any(UUID.class), any(Time.class)))
-			.thenReturn(new FlinkCompletableFuture<SlotRequestReply>());
+		Mockito
+			.when(
+				taskExecutorGateway
+					.requestSlot(any(SlotID.class), any(AllocationID.class), any(UUID.class), any(Time.class)))
+			.thenReturn(new FlinkCompletableFuture<TMSlotRequestReply>());
 		testRpcService.registerGateway(tmAddress, taskExecutorGateway);
 
 		final ResourceID resourceID = ResourceID.generate();
@@ -141,13 +154,14 @@ public class SlotProtocolTest extends TestLogger {
 		final SlotStatus slotStatus =
 			new SlotStatus(slotID, resourceProfile);
 		final SlotReport slotReport =
-			new SlotReport(Collections.singletonList(slotStatus), resourceID);
+			new SlotReport(Collections.singletonList(slotStatus));
 		// register slot at SlotManager
-		slotManager.registerTaskExecutor(resourceID, taskExecutorGateway);
-		slotManager.updateSlotStatus(slotReport);
+		slotManager.registerTaskExecutor(
+			resourceID, new TaskExecutorRegistration(taskExecutorGateway), slotReport);
 
 		// 4) Slot becomes available and TaskExecutor gets a SlotRequest
-		verify(taskExecutorGateway, timeout(5000)).requestSlot(eq(allocationID), any(UUID.class), any(Time.class));
+		verify(taskExecutorGateway, timeout(5000))
+			.requestSlot(eq(slotID), eq(allocationID), any(UUID.class), any(Time.class));
 	}
 
 	/**
@@ -173,13 +187,15 @@ public class SlotProtocolTest extends TestLogger {
 			configureHA(testingHaServices, jobID, rmAddress, rmLeaderID, jmAddress, jmLeaderID);
 
 		TaskExecutorGateway taskExecutorGateway = mock(TaskExecutorGateway.class);
-		Mockito.when(taskExecutorGateway.requestSlot(any(AllocationID.class), any(UUID.class), any(Time.class)))
-			.thenReturn(new FlinkCompletableFuture<SlotRequestReply>());
+		Mockito.when(
+			taskExecutorGateway
+				.requestSlot(any(SlotID.class), any(AllocationID.class), any(UUID.class), any(Time.class)))
+			.thenReturn(new FlinkCompletableFuture<TMSlotRequestReply>());
 		testRpcService.registerGateway(tmAddress, taskExecutorGateway);
 
-		SlotManager slotManager = Mockito.spy(new SimpleSlotManager());
-		ResourceManager resourceManager =
-			Mockito.spy(new StandaloneResourceManager(testRpcService, testingHaServices, slotManager));
+		TestingSlotManagerFactory slotManagerFactory = new TestingSlotManagerFactory();
+		TestingResourceManager resourceManager =
+			Mockito.spy(new TestingResourceManager(testRpcService, testingHaServices, slotManagerFactory));
 		resourceManager.start();
 		rmLeaderElectionService.isLeader(rmLeaderID);
 
@@ -191,6 +207,8 @@ public class SlotProtocolTest extends TestLogger {
 			Assert.fail("JobManager registration Future didn't become ready.");
 		}
 
+		final SlotManager slotManager = slotManagerFactory.slotManager;
+
 		final ResourceID resourceID = ResourceID.generate();
 		final AllocationID allocationID = new AllocationID();
 		final ResourceProfile resourceProfile = new ResourceProfile(1.0, 100);
@@ -199,13 +217,13 @@ public class SlotProtocolTest extends TestLogger {
 		final SlotStatus slotStatus =
 			new SlotStatus(slotID, resourceProfile);
 		final SlotReport slotReport =
-			new SlotReport(Collections.singletonList(slotStatus), resourceID);
+			new SlotReport(Collections.singletonList(slotStatus));
 		// register slot at SlotManager
-		slotManager.registerTaskExecutor(resourceID, taskExecutorGateway);
-		slotManager.updateSlotStatus(slotReport);
+		slotManager.registerTaskExecutor(
+			resourceID, new TaskExecutorRegistration(taskExecutorGateway), slotReport);
 
 		SlotRequest slotRequest = new SlotRequest(jobID, allocationID, resourceProfile);
-		SlotRequestReply slotRequestReply =
+		RMSlotRequestReply slotRequestReply =
 			resourceManager.requestSlot(jmLeaderID, rmLeaderID, slotRequest);
 
 		// 1) a SlotRequest is routed to the SlotManager
@@ -220,9 +238,9 @@ public class SlotProtocolTest extends TestLogger {
 		Assert.assertTrue(slotManager.isAllocated(slotID));
 		Assert.assertTrue(slotManager.isAllocated(allocationID));
 
-
 		// 4) a SlotRequest is routed to the TaskExecutor
-		verify(taskExecutorGateway, timeout(5000)).requestSlot(eq(allocationID), any(UUID.class), any(Time.class));
+		verify(taskExecutorGateway, timeout(5000))
+			.requestSlot(eq(slotID), eq(allocationID), any(UUID.class), any(Time.class));
 	}
 
 	private static TestingLeaderElectionService configureHA(
@@ -240,4 +258,32 @@ public class SlotProtocolTest extends TestLogger {
 		return rmLeaderElectionService;
 	}
 
+	private static class SpiedResourceManager extends TestingResourceManager {
+
+		private int startNewWorkerCalled = 0;
+
+		public SpiedResourceManager(
+				RpcService rpcService,
+				HighAvailabilityServices highAvailabilityServices,
+				SlotManagerFactory slotManagerFactory) {
+			super(rpcService, highAvailabilityServices, slotManagerFactory);
+		}
+
+
+		@Override
+		public void startNewWorker(ResourceProfile resourceProfile) {
+			startNewWorkerCalled++;
+		}
+	}
+
+	private static class TestingSlotManagerFactory implements SlotManagerFactory {
+
+		private SlotManager slotManager;
+
+		@Override
+		public SlotManager create(ResourceManagerServices rmServices) {
+			this.slotManager = Mockito.spy(new TestingSlotManager(rmServices));
+			return this.slotManager;
+		}
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/5915613d/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 9c1f288..7710fa9 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
@@ -19,7 +19,9 @@
 package org.apache.flink.runtime.taskexecutor;
 
 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.SlotID;
 import org.apache.flink.runtime.highavailability.NonHaServices;
 import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
@@ -28,11 +30,15 @@ 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.resourcemanager.ResourceManagerGateway;
+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.TestingSerialRpcService;
 import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
 import org.apache.flink.util.TestLogger;
 
+import org.hamcrest.Matchers;
 import org.junit.Test;
 
 import org.powermock.api.mockito.PowerMockito;
@@ -76,7 +82,7 @@ public class TaskExecutorTest extends TestLogger {
 			String taskManagerAddress = taskManager.getAddress();
 
 			verify(rmGateway).registerTaskExecutor(
-					any(UUID.class), eq(taskManagerAddress), eq(resourceID), any(Time.class));
+					any(UUID.class), eq(taskManagerAddress), eq(resourceID), any(SlotReport.class), any(Time.class));
 		}
 		finally {
 			rpc.stopService();
@@ -132,7 +138,7 @@ public class TaskExecutorTest extends TestLogger {
 			testLeaderService.notifyListener(address1, leaderId1);
 
 			verify(rmGateway1).registerTaskExecutor(
-					eq(leaderId1), eq(taskManagerAddress), eq(resourceID), any(Time.class));
+					eq(leaderId1), eq(taskManagerAddress), eq(resourceID), any(SlotReport.class), any(Time.class));
 			assertNotNull(taskManager.getResourceManagerConnection());
 
 			// cancel the leader 
@@ -142,11 +148,95 @@ public class TaskExecutorTest extends TestLogger {
 			testLeaderService.notifyListener(address2, leaderId2);
 
 			verify(rmGateway2).registerTaskExecutor(
-					eq(leaderId2), eq(taskManagerAddress), eq(resourceID), any(Time.class));
+					eq(leaderId2), eq(taskManagerAddress), eq(resourceID), any(SlotReport.class), any(Time.class));
 			assertNotNull(taskManager.getResourceManagerConnection());
 		}
 		finally {
 			rpc.stopService();
 		}
 	}
+
+	/**
+	 * 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.
+	 *
+	 * This is essential for the correctness of the state of the ResourceManager.
+	 */
+	@Test
+	public void testRejectAllocationRequestsForOutOfSyncSlots() {
+		final ResourceID resourceID = ResourceID.generate();
+
+		final String address1 = "/resource/manager/address/one";
+		final UUID leaderId = UUID.randomUUID();
+
+		final TestingSerialRpcService rpc = new TestingSerialRpcService();
+		try {
+			// register the mock resource manager gateways
+			ResourceManagerGateway rmGateway1 = mock(ResourceManagerGateway.class);
+			rpc.registerGateway(address1, rmGateway1);
+
+			TestingLeaderRetrievalService testLeaderService = new TestingLeaderRetrievalService();
+
+			TestingHighAvailabilityServices haServices = new TestingHighAvailabilityServices();
+			haServices.setResourceManagerLeaderRetriever(testLeaderService);
+
+			TaskManagerConfiguration taskManagerServicesConfiguration = mock(TaskManagerConfiguration.class);
+			PowerMockito.when(taskManagerServicesConfiguration.getNumberSlots()).thenReturn(1);
+
+			TaskManagerLocation taskManagerLocation = mock(TaskManagerLocation.class);
+			when(taskManagerLocation.getResourceID()).thenReturn(resourceID);
+
+			TaskExecutor taskManager = new TaskExecutor(
+				taskManagerServicesConfiguration,
+				taskManagerLocation,
+				rpc,
+				mock(MemoryManager.class),
+				mock(IOManager.class),
+				mock(NetworkEnvironment.class),
+				haServices,
+				mock(MetricRegistry.class),
+				mock(FatalErrorHandler.class));
+
+			taskManager.start();
+			String taskManagerAddress = taskManager.getAddress();
+
+			// no connection initially, since there is no leader
+			assertNull(taskManager.getResourceManagerConnection());
+
+			// define a leader and see that a registration happens
+			testLeaderService.notifyListener(address1, leaderId);
+
+			verify(rmGateway1).registerTaskExecutor(
+				eq(leaderId), eq(taskManagerAddress), eq(resourceID), any(SlotReport.class), any(Time.class));
+			assertNotNull(taskManager.getResourceManagerConnection());
+
+			// test that allocating a slot works
+			final SlotID slotID = new SlotID(resourceID, 0);
+			TMSlotRequestReply tmSlotRequestReply = taskManager.requestSlot(slotID, new AllocationID(), leaderId);
+			assertTrue(tmSlotRequestReply instanceof TMSlotRequestRegistered);
+
+			// test that we can't allocate slots which are blacklisted due to pending confirmation of the RM
+			final SlotID unconfirmedFreeSlotID = new SlotID(resourceID, 1);
+			taskManager.addUnconfirmedFreeSlotNotification(unconfirmedFreeSlotID);
+			TMSlotRequestReply tmSlotRequestReply2 =
+				taskManager.requestSlot(unconfirmedFreeSlotID, new AllocationID(), leaderId);
+			assertTrue(tmSlotRequestReply2 instanceof TMSlotRequestRejected);
+
+			// re-register
+			verify(rmGateway1).registerTaskExecutor(
+				eq(leaderId), eq(taskManagerAddress), eq(resourceID), any(SlotReport.class), any(Time.class));
+			testLeaderService.notifyListener(address1, leaderId);
+
+			// now we should be successful because the slots status has been synced
+			// test that we can't allocate slots which are blacklisted due to pending confirmation of the RM
+			TMSlotRequestReply tmSlotRequestReply3 =
+				taskManager.requestSlot(unconfirmedFreeSlotID, new AllocationID(), leaderId);
+			assertTrue(tmSlotRequestReply3 instanceof TMSlotRequestRegistered);
+
+		}
+		finally {
+			rpc.stopService();
+		}
+
+	}
 }


[26/50] [abbrv] flink git commit: [hotfix] [rpc] Add RpcConnectionTest to validate that connection buildup fails fast when endpoint is unreachable.

Posted by se...@apache.org.
[hotfix] [rpc] Add RpcConnectionTest to validate that connection buildup fails fast when endpoint is unreachable.


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

Branch: refs/heads/flip-6
Commit: 0e495b7b29c978b02957ab149e9ae0ba9c7d634f
Parents: c6954a9
Author: Stephan Ewen <se...@apache.org>
Authored: Wed Sep 21 13:03:17 2016 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Oct 14 15:14:41 2016 +0200

----------------------------------------------------------------------
 .../flink/runtime/rpc/AsyncCallsTest.java       |  4 +-
 .../flink/runtime/rpc/RpcCompletenessTest.java  | 14 ++--
 .../flink/runtime/rpc/RpcConnectionTest.java    | 86 ++++++++++++++++++++
 3 files changed, 96 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/0e495b7b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/AsyncCallsTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/AsyncCallsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/AsyncCallsTest.java
index e8255d4..7affdb9 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/AsyncCallsTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/AsyncCallsTest.java
@@ -43,9 +43,9 @@ public class AsyncCallsTest extends TestLogger {
 	//  shared test members
 	// ------------------------------------------------------------------------
 
-	private static ActorSystem actorSystem = AkkaUtils.createDefaultActorSystem();
+	private static final ActorSystem actorSystem = AkkaUtils.createDefaultActorSystem();
 
-	private static AkkaRpcService akkaRpcService =
+	private static final AkkaRpcService akkaRpcService =
 			new AkkaRpcService(actorSystem, Time.milliseconds(10000L));
 
 	@AfterClass

http://git-wip-us.apache.org/repos/asf/flink/blob/0e495b7b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcCompletenessTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcCompletenessTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcCompletenessTest.java
index ee3f784..53355e8 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcCompletenessTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcCompletenessTest.java
@@ -30,6 +30,7 @@ import org.reflections.Reflections;
 import java.lang.annotation.Annotation;
 import java.lang.reflect.Method;
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
@@ -69,7 +70,8 @@ public class RpcCompletenessTest extends TestLogger {
 
 	@SuppressWarnings("rawtypes")
 	private void checkCompleteness(Class<? extends RpcEndpoint> rpcEndpoint, Class<? extends RpcGateway> rpcGateway) {
-		Method[] gatewayMethods = getRpcMethodsFromGateway(rpcGateway).toArray(new Method[0]);
+		List<Method> rpcMethodsFromGateway = getRpcMethodsFromGateway(rpcGateway);
+		Method[] gatewayMethods = rpcMethodsFromGateway.toArray(new Method[rpcMethodsFromGateway.size()]);
 		Method[] serverMethods = rpcEndpoint.getMethods();
 
 		Map<String, Set<Method>> rpcMethods = new HashMap<>();
@@ -360,13 +362,13 @@ public class RpcCompletenessTest extends TestLogger {
 		}
 
 		// Get all methods declared in current interface
-		for(Method method : interfaceClass.getDeclaredMethods()) {
-			allMethods.add(method);
-		}
+		Collections.addAll(allMethods, interfaceClass.getDeclaredMethods());
 
 		// Get all method inherited from super interface
-		for(Class superClass : interfaceClass.getInterfaces()) {
-			allMethods.addAll(getRpcMethodsFromGateway(superClass));
+		for (Class<?> superClass : interfaceClass.getInterfaces()) {
+			@SuppressWarnings("unchecked")
+			Class<? extends RpcGateway> gatewayClass = (Class<? extends RpcGateway>) superClass;
+			allMethods.addAll(getRpcMethodsFromGateway(gatewayClass));
 		}
 		return allMethods;
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/0e495b7b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcConnectionTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcConnectionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcConnectionTest.java
new file mode 100644
index 0000000..6363662
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcConnectionTest.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.rpc;
+
+import akka.actor.ActorSystem;
+import akka.util.Timeout;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.akka.AkkaUtils;
+import org.apache.flink.runtime.rpc.akka.AkkaRpcService;
+import org.apache.flink.runtime.rpc.exceptions.RpcConnectionException;
+import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway;
+
+import org.junit.AfterClass;
+import org.junit.Test;
+
+import scala.Option;
+import scala.Tuple2;
+import scala.concurrent.Await;
+import scala.concurrent.Future;
+import scala.concurrent.duration.FiniteDuration;
+
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+import static org.junit.Assert.*;
+
+/**
+ * This test validates that the RPC service gives a good message when it cannot
+ * connect to an RpcEndpoint.
+ */
+public class RpcConnectionTest {
+
+	@Test
+	public void testConnectFailure() {
+		ActorSystem actorSystem = null;
+		RpcService rpcService = null;
+		try {
+			actorSystem = AkkaUtils.createActorSystem(
+					new Configuration(), Option.apply(new Tuple2<String, Object>("localhost", 0)));
+
+			// we start the RPC service with a very long timeout to ensure that the test
+			// can only pass if the connection problem is not recognized merely via a timeout
+			rpcService = new AkkaRpcService(actorSystem, new Timeout(10000000, TimeUnit.SECONDS));
+
+			Future<TaskExecutorGateway> future = rpcService.connect("foo.bar.com.test.invalid", TaskExecutorGateway.class);
+
+			Await.result(future, new FiniteDuration(10000000, TimeUnit.SECONDS));
+			fail("should never complete normally");
+		}
+		catch (TimeoutException e) {
+			fail("should not fail with a generic timeout exception");
+		}
+		catch (RpcConnectionException e) {
+			// that is what we want
+			assertTrue("wrong error message", e.getMessage().contains("foo.bar.com.test.invalid"));
+		}
+		catch (Throwable t) {
+			fail("wrong exception: " + t);
+		}
+		finally {
+			if (rpcService != null) {
+				rpcService.stopService();
+			}
+			if (actorSystem != null) {
+				actorSystem.shutdown();
+			}
+		}
+	}
+}


[39/50] [abbrv] flink git commit: [FLINK-4347][FLINK-4348] simplify SlotManager and integrate it with ResourceManager

Posted by se...@apache.org.
[FLINK-4347][FLINK-4348] simplify SlotManager and integrate it with ResourceManager

Instead of relying on a full synchronization of all slots information on
every heartbeat, the SlotManager is now responsible for updating its
state. It initially syncs all slots upon registration of the
TaskExecutor. After that, it only receives notifications from the
TaskExecutor when slots become available again. This simplifies the
logic of the SlotManager and makes the slot allocation more predictable
in case of message loss.

Additional changes:

- Move the slot registration and allocation report to the registration
  of the TaskExecutor

- Let the TaskExecutor immediately notify the ResourceManager once a
  slot becomes free. The ResourceManager has to confirm this
  notification. Otherwise, the slot will be blocked because the
  ResourceManager's state is not in sync.

- Integrate with handleSlotRequestFailedAtTaskManager and introduce
  fencing to protect against TaskExecutors which are not registered
  anymore.

- introduce RPC call to notify ResourceManager about free slots

- ignore out-of-date slot requests from ResourceManager at TaskExecutor

- let the ResourceManager update its state instead of relying on heartbeats

- provide ResourceManagerServices to SlotManager

- introduce factory for SlotManager

- keep task gateways and worker information in ResourceManager and
  inform SlotManager

- add TaskExecutor test to ensure that a free slot which hasn't been
  confirmed by the task executor is correctly blacklisted as long as the
  ResourceManager has not confirmed the allocation removal.

- adapt tests

- update javadocs

This closes #2571.


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

Branch: refs/heads/flip-6
Commit: 5915613df00f46fa493fd7d3ff63c836d9fd6146
Parents: 30517da
Author: Maximilian Michels <mx...@apache.org>
Authored: Thu Sep 29 15:08:32 2016 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Oct 14 15:14:42 2016 +0200

----------------------------------------------------------------------
 .../clusterframework/types/ResourceSlot.java    |  12 +-
 .../resourcemanager/ResourceManager.java        | 115 +++++--
 .../resourcemanager/ResourceManagerGateway.java |  31 +-
 .../ResourceManagerServices.java                |   2 +-
 .../resourcemanager/SlotRequestRegistered.java  |  33 --
 .../resourcemanager/SlotRequestRejected.java    |  34 --
 .../resourcemanager/SlotRequestReply.java       |  41 ---
 .../StandaloneResourceManager.java              |   8 +-
 .../jobmanager/RMSlotRequestRegistered.java     |  33 ++
 .../jobmanager/RMSlotRequestRejected.java       |  34 ++
 .../messages/jobmanager/RMSlotRequestReply.java |  41 +++
 .../taskexecutor/SlotAvailableReply.java        |  47 +++
 .../taskexecutor/TMSlotRequestRegistered.java   |  35 ++
 .../taskexecutor/TMSlotRequestRejected.java     |  35 ++
 .../taskexecutor/TMSlotRequestReply.java        |  58 ++++
 .../registration/TaskExecutorRegistration.java  |  12 +-
 .../registration/WorkerRegistration.java        |  42 +++
 .../slotmanager/SimpleSlotManager.java          |  53 ---
 .../slotmanager/SlotManager.java                | 326 +++++++------------
 .../slotmanager/SlotManagerFactory.java         |  31 ++
 .../flink/runtime/taskexecutor/SlotReport.java  |  19 +-
 .../runtime/taskexecutor/TaskExecutor.java      |  37 ++-
 .../taskexecutor/TaskExecutorGateway.java       |   7 +-
 ...TaskExecutorToResourceManagerConnection.java |   2 +-
 .../resourcemanager/ResourceManagerHATest.java  |  12 +-
 .../ResourceManagerJobMasterTest.java           |   4 +-
 .../ResourceManagerTaskExecutorTest.java        |  53 +--
 .../resourcemanager/TestingResourceManager.java |  53 +++
 .../resourcemanager/TestingSlotManager.java     |  78 +++++
 .../slotmanager/SlotManagerTest.java            | 239 ++++++--------
 .../slotmanager/SlotProtocolTest.java           |  92 ++++--
 .../runtime/taskexecutor/TaskExecutorTest.java  |  96 +++++-
 32 files changed, 1087 insertions(+), 628 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/5915613d/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceSlot.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceSlot.java b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceSlot.java
index 4a91a79..0b9367d 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceSlot.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceSlot.java
@@ -18,7 +18,7 @@
 
 package org.apache.flink.runtime.clusterframework.types;
 
-import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorRegistration;
 
 import static org.apache.flink.util.Preconditions.checkNotNull;
 
@@ -35,12 +35,12 @@ public class ResourceSlot implements ResourceIDRetrievable {
 	private final ResourceProfile resourceProfile;
 
 	/** Gateway to the TaskExecutor which owns the slot */
-	private final TaskExecutorGateway taskExecutorGateway;
+	private final TaskExecutorRegistration taskExecutorRegistration;
 
-	public ResourceSlot(SlotID slotId, ResourceProfile resourceProfile, TaskExecutorGateway taskExecutorGateway) {
+	public ResourceSlot(SlotID slotId, ResourceProfile resourceProfile, TaskExecutorRegistration taskExecutorRegistration) {
 		this.slotId = checkNotNull(slotId);
 		this.resourceProfile = checkNotNull(resourceProfile);
-		this.taskExecutorGateway = taskExecutorGateway;
+		this.taskExecutorRegistration = checkNotNull(taskExecutorRegistration);
 	}
 
 	@Override
@@ -56,8 +56,8 @@ public class ResourceSlot implements ResourceIDRetrievable {
 		return resourceProfile;
 	}
 
-	public TaskExecutorGateway getTaskExecutorGateway() {
-		return taskExecutorGateway;
+	public TaskExecutorRegistration getTaskExecutorRegistration() {
+		return taskExecutorRegistration;
 	}
 
 	/**

http://git-wip-us.apache.org/repos/asf/flink/blob/5915613d/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 f45afa3..d2d00cf 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
@@ -24,8 +24,8 @@ 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;
-import org.apache.flink.runtime.clusterframework.types.ResourceIDRetrievable;
 import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
 import org.apache.flink.runtime.concurrent.AcceptFunction;
 import org.apache.flink.runtime.concurrent.ApplyFunction;
 import org.apache.flink.runtime.concurrent.BiFunction;
@@ -37,7 +37,12 @@ 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.resourcemanager.messages.jobmanager.RMSlotRequestRejected;
+import org.apache.flink.runtime.resourcemanager.messages.jobmanager.RMSlotRequestReply;
+import org.apache.flink.runtime.resourcemanager.messages.taskexecutor.SlotAvailableReply;
+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.RpcEndpoint;
 import org.apache.flink.runtime.rpc.RpcMethod;
 import org.apache.flink.runtime.rpc.RpcService;
@@ -45,12 +50,14 @@ import org.apache.flink.runtime.jobmaster.JobMaster;
 import org.apache.flink.runtime.jobmaster.JobMasterGateway;
 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.LeaderConnectionInfo;
 import org.apache.flink.runtime.util.LeaderRetrievalUtils;
 import scala.concurrent.duration.FiniteDuration;
 
+import java.io.Serializable;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.UUID;
@@ -70,41 +77,54 @@ import static org.apache.flink.util.Preconditions.checkNotNull;
  *     <li>{@link #requestSlot(UUID, UUID, SlotRequest)} requests a slot from the resource manager</li>
  * </ul>
  */
-public abstract class ResourceManager<WorkerType extends ResourceIDRetrievable>
+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 */
+	/** The exit code with which the process is stopped in case of a fatal error. */
 	protected static final int EXIT_CODE_FATAL_ERROR = -13;
 
+	/** All currently registered JobMasterGateways scoped by JobID. */
 	private final Map<JobID, JobMasterGateway> jobMasterGateways;
 
+	/** LeaderListeners for all registered JobMasters. */
 	private final Map<JobID, JobMasterLeaderListener> jobMasterLeaderRetrievalListeners;
 
-	private final Map<ResourceID, WorkerType> taskExecutorGateways;
+	/** All currently registered TaskExecutors with there framework specific worker information. */
+	private final Map<ResourceID, WorkerRegistration<WorkerType>> taskExecutors;
 
+	/** High availability services for leader retrieval and election. */
 	private final HighAvailabilityServices highAvailabilityServices;
 
-	private final SlotManager slotManager;
+	/** The factory to construct the SlotManager. */
+	private final SlotManagerFactory slotManagerFactory;
 
+	/** The SlotManager created by the slotManagerFactory when the ResourceManager is started. */
+	private SlotManager slotManager;
+
+	/** The service to elect a ResourceManager leader. */
 	private LeaderElectionService leaderElectionService;
 
+	/** ResourceManager's leader session id which is updated on leader election. */
 	private 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);
 
 	public ResourceManager(
 			RpcService rpcService,
 			HighAvailabilityServices highAvailabilityServices,
-			SlotManager slotManager) {
+			SlotManagerFactory slotManagerFactory) {
 		super(rpcService);
 		this.highAvailabilityServices = checkNotNull(highAvailabilityServices);
+		this.slotManagerFactory = checkNotNull(slotManagerFactory);
 		this.jobMasterGateways = new HashMap<>();
-		this.slotManager = checkNotNull(slotManager);
 		this.jobMasterLeaderRetrievalListeners = new HashMap<>();
-		this.taskExecutorGateways = new HashMap<>();
+		this.taskExecutors = new HashMap<>();
+		this.leaderSessionID = new UUID(0, 0);
 		infoMessageListeners = new HashMap<>();
 	}
 
@@ -113,9 +133,10 @@ public abstract class ResourceManager<WorkerType extends ResourceIDRetrievable>
 		// start a leader
 		try {
 			super.start();
+			// SlotManager should start first
+			slotManager = slotManagerFactory.create(createResourceManagerServices());
 			leaderElectionService = highAvailabilityServices.getResourceManagerLeaderElectionService();
 			leaderElectionService.start(this);
-			slotManager.setupResourceManagerServices(new DefaultResourceManagerServices());
 			// framework specific initialization
 			initialize();
 		} catch (Throwable e) {
@@ -196,7 +217,7 @@ public abstract class ResourceManager<WorkerType extends ResourceIDRetrievable>
 			.handleAsync(new BiFunction<JobMasterGateway, Throwable, RegistrationResponse>() {
 				@Override
 				public RegistrationResponse apply(JobMasterGateway jobMasterGateway, Throwable throwable) {
-					
+
 					if (throwable != null) {
 						return new RegistrationResponse.Decline(throwable.getMessage());
 					} else {
@@ -234,7 +255,8 @@ public abstract class ResourceManager<WorkerType extends ResourceIDRetrievable>
 	public Future<RegistrationResponse> registerTaskExecutor(
 		final UUID resourceManagerLeaderId,
 		final String taskExecutorAddress,
-		final ResourceID resourceID) {
+		final ResourceID resourceID,
+		final SlotReport slotReport) {
 
 		return getRpcService().execute(new Callable<TaskExecutorGateway>() {
 			@Override
@@ -245,7 +267,8 @@ public abstract class ResourceManager<WorkerType extends ResourceIDRetrievable>
 						resourceID, taskExecutorAddress, leaderSessionID, resourceManagerLeaderId);
 					throw new Exception("Invalid leader session id");
 				}
-				return getRpcService().connect(taskExecutorAddress, TaskExecutorGateway.class).get(5, TimeUnit.SECONDS);
+				return getRpcService().connect(taskExecutorAddress, TaskExecutorGateway.class)
+					.get(timeout.toMilliseconds(), timeout.getUnit());
 			}
 		}).handleAsync(new BiFunction<TaskExecutorGateway, Throwable, RegistrationResponse>() {
 			@Override
@@ -253,14 +276,17 @@ public abstract class ResourceManager<WorkerType extends ResourceIDRetrievable>
 				if (throwable != null) {
 					return new RegistrationResponse.Decline(throwable.getMessage());
 				} else {
-					WorkerType oldWorker = taskExecutorGateways.remove(resourceID);
-					if (oldWorker != null) {
+					WorkerRegistration oldRegistration = taskExecutors.remove(resourceID);
+					if (oldRegistration != null) {
 						// TODO :: suggest old taskExecutor to stop itself
-						slotManager.notifyTaskManagerFailure(resourceID);
+						log.info("Replacing old instance of worker for ResourceID {}", resourceID);
 					}
 					WorkerType newWorker = workerStarted(resourceID);
-					taskExecutorGateways.put(resourceID, newWorker);
-					return new TaskExecutorRegistrationSuccess(new InstanceID(), 5000);
+					WorkerRegistration<WorkerType> registration =
+						new WorkerRegistration<>(taskExecutorGateway, newWorker);
+					taskExecutors.put(resourceID, registration);
+					slotManager.registerTaskExecutor(resourceID, registration, slotReport);
+					return new TaskExecutorRegistrationSuccess(registration.getInstanceID(), 5000);
 				}
 			}
 		}, getMainThreadExecutor());
@@ -273,7 +299,7 @@ public abstract class ResourceManager<WorkerType extends ResourceIDRetrievable>
 	 * @return Slot assignment
 	 */
 	@RpcMethod
-	public SlotRequestReply requestSlot(
+	public RMSlotRequestReply requestSlot(
 			UUID jobMasterLeaderID,
 			UUID resourceManagerLeaderID,
 			SlotRequest slotRequest) {
@@ -290,8 +316,41 @@ public abstract class ResourceManager<WorkerType extends ResourceIDRetrievable>
 			return slotManager.requestSlot(slotRequest);
 		} else {
 			log.info("Ignoring slot request for unknown JobMaster with JobID {}", jobId);
-			return new SlotRequestRejected(slotRequest.getAllocationId());
+			return new RMSlotRequestRejected(slotRequest.getAllocationId());
+		}
+	}
+
+	/**
+	 * Notification from a TaskExecutor that a slot has become available
+	 * @param resourceManagerLeaderId TaskExecutor's resource manager leader id
+	 * @param resourceID TaskExecutor's resource id
+	 * @param instanceID TaskExecutor's instance id
+	 * @param slotID The slot id of the available slot
+	 * @return SlotAvailableReply
+	 */
+	@RpcMethod
+	public SlotAvailableReply notifySlotAvailable(
+			final UUID resourceManagerLeaderId,
+			final ResourceID resourceID,
+			final InstanceID instanceID,
+			final SlotID slotID) {
+
+		if (resourceManagerLeaderId.equals(leaderSessionID)) {
+			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);
+						}
+					});
+					return new SlotAvailableReply(leaderSessionID, slotID);
+				}
+			}
 		}
+		return null;
 	}
 
 
@@ -329,9 +388,9 @@ public abstract class ResourceManager<WorkerType extends ResourceIDRetrievable>
 			public void run() {
 				log.info("ResourceManager {} was revoked leadership.", getAddress());
 				jobMasterGateways.clear();
-				taskExecutorGateways.clear();
+				taskExecutors.clear();
 				slotManager.clearState();
-				leaderSessionID = null;
+				leaderSessionID = new UUID(0, 0);
 			}
 		});
 	}
@@ -411,7 +470,7 @@ public abstract class ResourceManager<WorkerType extends ResourceIDRetrievable>
 		runAsync(new Runnable() {
 			@Override
 			public void run() {
-				WorkerType worker = taskExecutorGateways.remove(resourceID);
+				WorkerType worker = taskExecutors.remove(resourceID).getWorker();
 				if (worker != null) {
 					// TODO :: suggest failed task executor to stop itself
 					slotManager.notifyTaskManagerFailure(resourceID);
@@ -426,7 +485,7 @@ public abstract class ResourceManager<WorkerType extends ResourceIDRetrievable>
 	 * @return The number of currently started TaskManagers.
 	 */
 	public int getNumberOfStartedTaskManagers() {
-		return taskExecutorGateways.size();
+		return taskExecutors.size();
 	}
 
 	/**
@@ -507,6 +566,14 @@ public abstract class ResourceManager<WorkerType extends ResourceIDRetrievable>
 		});
 	}
 
+	// ------------------------------------------------------------------------
+	//  Resource Manager Services
+	// ------------------------------------------------------------------------
+
+	protected ResourceManagerServices createResourceManagerServices() {
+		return new DefaultResourceManagerServices();
+	}
+
 	private class DefaultResourceManagerServices implements ResourceManagerServices {
 
 		@Override
@@ -520,7 +587,7 @@ public abstract class ResourceManager<WorkerType extends ResourceIDRetrievable>
 		}
 
 		@Override
-		public Executor getExecutor() {
+		public Executor getMainThreadExecutor() {
 			return ResourceManager.this.getMainThreadExecutor();
 		}
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/5915613d/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 87303a1..3c81227 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
@@ -22,11 +22,16 @@ 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.types.ResourceID;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
 import org.apache.flink.runtime.concurrent.Future;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.resourcemanager.messages.jobmanager.RMSlotRequestReply;
+import org.apache.flink.runtime.resourcemanager.messages.taskexecutor.SlotAvailableReply;
 import org.apache.flink.runtime.rpc.RpcGateway;
 import org.apache.flink.runtime.rpc.RpcTimeout;
 import org.apache.flink.runtime.jobmaster.JobMaster;
 import org.apache.flink.runtime.registration.RegistrationResponse;
+import org.apache.flink.runtime.taskexecutor.SlotReport;
 
 import java.util.UUID;
 
@@ -56,10 +61,12 @@ public interface ResourceManagerGateway extends RpcGateway {
 	/**
 	 * Requests a slot from the resource manager.
 	 *
-	 * @param slotRequest Slot request
-	 * @return Future slot assignment
+	 * @param jobMasterLeaderID leader id of the JobMaster
+	 * @param resourceManagerLeaderID leader if of the ResourceMaster
+	 * @param slotRequest The slot to request
+	 * @return The confirmation that the slot gets allocated
 	 */
-	Future<SlotRequestReply> requestSlot(
+	Future<RMSlotRequestReply> requestSlot(
 		UUID jobMasterLeaderID,
 		UUID resourceManagerLeaderID,
 		SlotRequest slotRequest,
@@ -71,6 +78,7 @@ public interface ResourceManagerGateway extends RpcGateway {
 	 * @param resourceManagerLeaderId  The fencing token for the ResourceManager leader
 	 * @param taskExecutorAddress     The address of the TaskExecutor that registers
 	 * @param resourceID              The resource ID of the TaskExecutor that registers
+	 * @param slotReport              The slot report containing free and allocated task slots
 	 * @param timeout                 The timeout for the response.
 	 *
 	 * @return The future to the response by the ResourceManager.
@@ -79,6 +87,23 @@ public interface ResourceManagerGateway extends RpcGateway {
 		UUID resourceManagerLeaderId,
 		String taskExecutorAddress,
 		ResourceID resourceID,
+		SlotReport slotReport,
+		@RpcTimeout Time timeout);
+
+	/**
+	 * Sent by the TaskExecutor to notify the ResourceManager that a slot has become available.
+	 *
+	 * @param resourceManagerLeaderId The ResourceManager leader id
+	 * @param resourceID The ResourceID of the TaskExecutor
+	 * @param instanceID The InstanceID of the TaskExecutor
+	 * @param slotID The SlotID of the freed slot
+	 * @return The confirmation by the ResourceManager
+	 */
+	Future<SlotAvailableReply> notifySlotAvailable(
+		UUID resourceManagerLeaderId,
+		ResourceID resourceID,
+		InstanceID instanceID,
+		SlotID slotID,
 		@RpcTimeout Time timeout);
 
 	/**

http://git-wip-us.apache.org/repos/asf/flink/blob/5915613d/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 30994dc..b997a3a 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
@@ -39,6 +39,6 @@ public interface ResourceManagerServices {
 	/**
 	 * Gets the executor which executes in the main thread of the ResourceManager
 	 */
-	Executor getExecutor();
+	Executor getMainThreadExecutor();
 
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/5915613d/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotRequestRegistered.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotRequestRegistered.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotRequestRegistered.java
deleted file mode 100644
index f719dce..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotRequestRegistered.java
+++ /dev/null
@@ -1,33 +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.clusterframework.types.AllocationID;
-
-/**
- * Acknowledgment by the ResourceManager for a SlotRequest from the JobManager
- */
-public class SlotRequestRegistered extends SlotRequestReply {
-
-	private static final long serialVersionUID = 4760320859275256855L;
-
-	public SlotRequestRegistered(AllocationID allocationID) {
-		super(allocationID);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/5915613d/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotRequestRejected.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotRequestRejected.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotRequestRejected.java
deleted file mode 100644
index 282a7d5..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotRequestRejected.java
+++ /dev/null
@@ -1,34 +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.clusterframework.types.AllocationID;
-
-/**
- * Rejection message by the ResourceManager for a SlotRequest from the JobManager
- */
-public class SlotRequestRejected extends SlotRequestReply {
-
-	private static final long serialVersionUID = 9049346740895325144L;
-
-	public SlotRequestRejected(AllocationID allocationID) {
-		super(allocationID);
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/5915613d/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotRequestReply.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotRequestReply.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotRequestReply.java
deleted file mode 100644
index 1b85d0c..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotRequestReply.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.runtime.resourcemanager;
-
-import org.apache.flink.runtime.clusterframework.types.AllocationID;
-
-import java.io.Serializable;
-
-/**
- * Acknowledgment by the ResourceManager for a SlotRequest from the JobManager
- */
-public abstract class SlotRequestReply implements Serializable {
-
-	private static final long serialVersionUID = 42;
-
-	private final AllocationID allocationID;
-
-	public SlotRequestReply(AllocationID allocationID) {
-		this.allocationID = allocationID;
-	}
-
-	public AllocationID getAllocationID() {
-		return allocationID;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/5915613d/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 deca8d3..f9f55f8 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,7 @@ 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.resourcemanager.slotmanager.SlotManager;
+import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManagerFactory;
 import org.apache.flink.runtime.rpc.RpcService;
 
 /**
@@ -34,9 +34,9 @@ import org.apache.flink.runtime.rpc.RpcService;
 public class StandaloneResourceManager extends ResourceManager<ResourceID> {
 
 	public StandaloneResourceManager(RpcService rpcService,
-		HighAvailabilityServices highAvailabilityServices,
-		SlotManager slotManager) {
-		super(rpcService, highAvailabilityServices, slotManager);
+			HighAvailabilityServices highAvailabilityServices,
+			SlotManagerFactory slotManagerFactory) {
+		super(rpcService, highAvailabilityServices, slotManagerFactory);
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/5915613d/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/messages/jobmanager/RMSlotRequestRegistered.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/messages/jobmanager/RMSlotRequestRegistered.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/messages/jobmanager/RMSlotRequestRegistered.java
new file mode 100644
index 0000000..01bc532
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/messages/jobmanager/RMSlotRequestRegistered.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.messages.jobmanager;
+
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+
+/**
+ * Acknowledgment by the ResourceManager for a SlotRequest from the JobManager
+ */
+public class RMSlotRequestRegistered extends RMSlotRequestReply {
+
+	private static final long serialVersionUID = 4760320859275256855L;
+
+	public RMSlotRequestRegistered(AllocationID allocationID) {
+		super(allocationID);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/5915613d/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/messages/jobmanager/RMSlotRequestRejected.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/messages/jobmanager/RMSlotRequestRejected.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/messages/jobmanager/RMSlotRequestRejected.java
new file mode 100644
index 0000000..649d61c
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/messages/jobmanager/RMSlotRequestRejected.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.resourcemanager.messages.jobmanager;
+
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+
+/**
+ * Rejection message by the ResourceManager for a SlotRequest from the JobManager
+ */
+public class RMSlotRequestRejected extends RMSlotRequestReply {
+
+	private static final long serialVersionUID = 9049346740895325144L;
+
+	public RMSlotRequestRejected(AllocationID allocationID) {
+		super(allocationID);
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/5915613d/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/messages/jobmanager/RMSlotRequestReply.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/messages/jobmanager/RMSlotRequestReply.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/messages/jobmanager/RMSlotRequestReply.java
new file mode 100644
index 0000000..66e1911
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/messages/jobmanager/RMSlotRequestReply.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.resourcemanager.messages.jobmanager;
+
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+
+import java.io.Serializable;
+
+/**
+ * Acknowledgment by the ResourceManager for a SlotRequest from the JobManager
+ */
+public abstract class RMSlotRequestReply implements Serializable {
+
+	private static final long serialVersionUID = 42;
+
+	private final AllocationID allocationID;
+
+	public RMSlotRequestReply(AllocationID allocationID) {
+		this.allocationID = allocationID;
+	}
+
+	public AllocationID getAllocationID() {
+		return allocationID;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/5915613d/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/messages/taskexecutor/SlotAvailableReply.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/messages/taskexecutor/SlotAvailableReply.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/messages/taskexecutor/SlotAvailableReply.java
new file mode 100644
index 0000000..f2e0105
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/messages/taskexecutor/SlotAvailableReply.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.resourcemanager.messages.taskexecutor;
+
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+
+import java.io.Serializable;
+import java.util.UUID;
+
+/**
+ * Sent by the ResourceManager to the TaskExecutor confirm receipt of
+ * {@code org.apache.flink.runtime.resourcemanager.ResourceManagerGateway.notifySlotAvailable}.
+ */
+public class SlotAvailableReply implements Serializable {
+
+	private final UUID resourceManagerLeaderID;
+
+	private final SlotID slotID;
+
+	public SlotAvailableReply(UUID resourceManagerLeaderID, SlotID slotID) {
+		this.resourceManagerLeaderID = resourceManagerLeaderID;
+		this.slotID = slotID;
+	}
+
+	public UUID getResourceManagerLeaderID() {
+		return resourceManagerLeaderID;
+	}
+
+	public SlotID getSlotID() {
+		return slotID;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/5915613d/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/messages/taskexecutor/TMSlotRequestRegistered.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/messages/taskexecutor/TMSlotRequestRegistered.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/messages/taskexecutor/TMSlotRequestRegistered.java
new file mode 100644
index 0000000..c0f0f49
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/messages/taskexecutor/TMSlotRequestRegistered.java
@@ -0,0 +1,35 @@
+/*
+ * 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.messages.taskexecutor;
+
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.instance.InstanceID;
+
+/**
+ * Acknowledgment by the TaskExecutor for a SlotRequest from the ResourceManager
+ */
+public class TMSlotRequestRegistered extends TMSlotRequestReply {
+
+	private static final long serialVersionUID = 4760320859275256855L;
+
+	public TMSlotRequestRegistered(InstanceID instanceID, ResourceID resourceID, AllocationID allocationID) {
+		super(instanceID, resourceID, allocationID);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/5915613d/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/messages/taskexecutor/TMSlotRequestRejected.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/messages/taskexecutor/TMSlotRequestRejected.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/messages/taskexecutor/TMSlotRequestRejected.java
new file mode 100644
index 0000000..9b10a35
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/messages/taskexecutor/TMSlotRequestRejected.java
@@ -0,0 +1,35 @@
+/*
+ * 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.messages.taskexecutor;
+
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.instance.InstanceID;
+
+/**
+ * Rejection by the TaskExecutor for a SlotRequest from the ResourceManager
+ */
+public class TMSlotRequestRejected extends TMSlotRequestReply {
+
+	private static final long serialVersionUID = 9049346740895325144L;
+
+	public TMSlotRequestRejected(InstanceID instanceID, ResourceID resourceID, AllocationID allocationID) {
+		super(instanceID, resourceID, allocationID);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/5915613d/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/messages/taskexecutor/TMSlotRequestReply.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/messages/taskexecutor/TMSlotRequestReply.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/messages/taskexecutor/TMSlotRequestReply.java
new file mode 100644
index 0000000..b23b6e9
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/messages/taskexecutor/TMSlotRequestReply.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.messages.taskexecutor;
+
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.instance.InstanceID;
+
+import java.io.Serializable;
+
+/**
+ * Acknowledgment by the TaskExecutor for a SlotRequest from the ResourceManager
+ */
+public abstract class TMSlotRequestReply implements Serializable {
+
+	private static final long serialVersionUID = 42;
+
+	private final InstanceID instanceID;
+
+	private final ResourceID resourceID;
+
+	private final AllocationID allocationID;
+
+	protected TMSlotRequestReply(InstanceID instanceID, ResourceID resourceID, AllocationID allocationID) {
+		this.instanceID = instanceID;
+		this.resourceID = resourceID;
+		this.allocationID = allocationID;
+	}
+
+	public InstanceID getInstanceID() {
+		return instanceID;
+	}
+
+	public ResourceID getResourceID() {
+		return resourceID;
+	}
+
+	public AllocationID getAllocationID() {
+		return allocationID;
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/5915613d/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/registration/TaskExecutorRegistration.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/registration/TaskExecutorRegistration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/registration/TaskExecutorRegistration.java
index 6b21f5c..bfa9c00 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/registration/TaskExecutorRegistration.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/registration/TaskExecutorRegistration.java
@@ -24,20 +24,20 @@ import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway;
 import java.io.Serializable;
 
 /**
- * This class is responsible for group the TaskExecutorGateway and the InstanceID of a registered task executor.
+ * This class is responsible for grouping the TaskExecutorGateway and the InstanceID
+ * of a registered task executor.
  */
 public class TaskExecutorRegistration implements Serializable {
 
 	private static final long serialVersionUID = -2062957799469434614L;
 
-	private TaskExecutorGateway taskExecutorGateway;
+	private final InstanceID instanceID;
 
-	private InstanceID instanceID;
+	private TaskExecutorGateway taskExecutorGateway;
 
-	public TaskExecutorRegistration(TaskExecutorGateway taskExecutorGateway,
-									InstanceID instanceID) {
+	public TaskExecutorRegistration(TaskExecutorGateway taskExecutorGateway) {
+		this.instanceID = new InstanceID();
 		this.taskExecutorGateway = taskExecutorGateway;
-		this.instanceID = instanceID;
 	}
 
 	public InstanceID getInstanceID() {

http://git-wip-us.apache.org/repos/asf/flink/blob/5915613d/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
new file mode 100644
index 0000000..ff28f94
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/registration/WorkerRegistration.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.registration;
+
+import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway;
+
+import java.io.Serializable;
+
+/**
+ * This class extends the {@link TaskExecutorRegistration}, adding the worker information.
+ */
+public class WorkerRegistration<WorkerType extends Serializable> extends TaskExecutorRegistration {
+
+	private static final long serialVersionUID = -2062957799469434614L;
+
+	private WorkerType worker;
+
+	public WorkerRegistration(TaskExecutorGateway taskExecutorGateway, WorkerType worker) {
+		super(taskExecutorGateway);
+		this.worker = worker;
+	}
+
+	public WorkerType getWorker() {
+		return worker;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/5915613d/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SimpleSlotManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SimpleSlotManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SimpleSlotManager.java
deleted file mode 100644
index ae1de5a..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SimpleSlotManager.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.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.SlotRequest;
-
-import java.util.Iterator;
-import java.util.Map;
-
-/**
- * A simple SlotManager which ignores resource profiles.
- */
-public class SimpleSlotManager extends SlotManager {
-
-	@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;
-		}
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/5915613d/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManager.java
index a56b2f6..7eb2d78 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManager.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManager.java
@@ -28,11 +28,12 @@ import org.apache.flink.runtime.concurrent.BiFunction;
 import org.apache.flink.runtime.concurrent.Future;
 import org.apache.flink.runtime.resourcemanager.ResourceManagerServices;
 import org.apache.flink.runtime.resourcemanager.SlotRequest;
-import org.apache.flink.runtime.resourcemanager.SlotRequestRegistered;
-import org.apache.flink.runtime.resourcemanager.SlotRequestReply;
+import org.apache.flink.runtime.resourcemanager.messages.jobmanager.RMSlotRequestRegistered;
+import org.apache.flink.runtime.resourcemanager.messages.taskexecutor.TMSlotRequestRejected;
+import org.apache.flink.runtime.resourcemanager.messages.taskexecutor.TMSlotRequestReply;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorRegistration;
 import org.apache.flink.runtime.taskexecutor.SlotReport;
 import org.apache.flink.runtime.taskexecutor.SlotStatus;
-import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway;
 import org.apache.flink.util.Preconditions;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -64,8 +65,11 @@ public abstract class SlotManager {
 
 	protected final Logger LOG = LoggerFactory.getLogger(getClass());
 
+	/** The Resource allocation provider */
+	protected final ResourceManagerServices rmServices;
+
 	/** All registered task managers with ResourceID and gateway. */
-	private final Map<ResourceID, TaskExecutorGateway> taskManagerGateways;
+	private final Map<ResourceID, TaskExecutorRegistration> taskManagers;
 
 	/** All registered slots, including free and allocated slots */
 	private final Map<ResourceID, Map<SlotID, ResourceSlot>> registeredSlots;
@@ -84,29 +88,17 @@ public abstract class SlotManager {
 	/** The current leader id set by the ResourceManager */
 	private UUID leaderID;
 
-	/** The Resource allocation provider */
-	private ResourceManagerServices resourceManagerServices;
-
-	public SlotManager() {
+	public SlotManager(ResourceManagerServices rmServices) {
+		this.rmServices = checkNotNull(rmServices);
 		this.registeredSlots = new HashMap<>(16);
 		this.pendingSlotRequests = new LinkedHashMap<>(16);
 		this.freeSlots = new HashMap<>(16);
 		this.allocationMap = new AllocationMap();
-		this.taskManagerGateways = new HashMap<>();
+		this.taskManagers = new HashMap<>();
 		this.timeout = Time.seconds(10);
+		this.leaderID = new UUID(0, 0);
 	}
 
-	/**
-	 * Initializes the resource supplier which is needed to request new resources.
-	 */
-	public void setupResourceManagerServices(ResourceManagerServices resourceManagerServices) {
-		if (this.resourceManagerServices != null) {
-			throw new IllegalStateException("ResourceManagerServices may only be set once.");
-		}
-		this.resourceManagerServices = resourceManagerServices;
-	}
-
-
 	// ------------------------------------------------------------------------
 	//  slot managements
 	// ------------------------------------------------------------------------
@@ -118,13 +110,13 @@ public abstract class SlotManager {
 	 * RPC's main thread to avoid race condition).
 	 *
 	 * @param request The detailed request of the slot
-	 * @return SlotRequestRegistered The confirmation message to be send to the caller
+	 * @return RMSlotRequestRegistered The confirmation message to be send to the caller
 	 */
-	public SlotRequestRegistered requestSlot(final SlotRequest request) {
+	public RMSlotRequestRegistered requestSlot(final SlotRequest request) {
 		final AllocationID allocationId = request.getAllocationId();
 		if (isRequestDuplicated(request)) {
 			LOG.warn("Duplicated slot request, AllocationID:{}", allocationId);
-			return null;
+			return new RMSlotRequestRegistered(allocationId);
 		}
 
 		// try to fulfil the request with current free slots
@@ -136,53 +128,38 @@ public abstract class SlotManager {
 			// record this allocation in bookkeeping
 			allocationMap.addAllocation(slot.getSlotId(), allocationId);
 			// remove selected slot from free pool
-			final ResourceSlot removedSlot = freeSlots.remove(slot.getSlotId());
-
-			final Future<SlotRequestReply> slotRequestReplyFuture =
-				slot.getTaskExecutorGateway().requestSlot(allocationId, leaderID, timeout);
-
-			slotRequestReplyFuture.handleAsync(new BiFunction<SlotRequestReply, Throwable, Object>() {
-				@Override
-				public Object apply(SlotRequestReply slotRequestReply, Throwable throwable) {
-					if (throwable != null) {
-						// we failed, put the slot and the request back again
-						if (allocationMap.isAllocated(slot.getSlotId())) {
-							// only re-add if the slot hasn't been removed in the meantime
-							freeSlots.put(slot.getSlotId(), removedSlot);
-						}
-						pendingSlotRequests.put(allocationId, request);
-					}
-					return null;
-				}
-			}, resourceManagerServices.getExecutor());
+			freeSlots.remove(slot.getSlotId());
+
+			sendSlotRequest(slot, request);
 		} else {
 			LOG.info("Cannot fulfil slot request, try to allocate a new container for it, " +
 				"AllocationID:{}, JobID:{}", allocationId, request.getJobId());
-			Preconditions.checkState(resourceManagerServices != null,
+			Preconditions.checkState(rmServices != null,
 				"Attempted to allocate resources but no ResourceManagerServices set.");
-			resourceManagerServices.allocateResource(request.getResourceProfile());
+			rmServices.allocateResource(request.getResourceProfile());
 			pendingSlotRequests.put(allocationId, request);
 		}
 
-		return new SlotRequestRegistered(allocationId);
+		return new RMSlotRequestRegistered(allocationId);
 	}
 
 	/**
-	 * Sync slot status with TaskManager's SlotReport.
+	 * Notifies the SlotManager that a slot is available again after being allocated.
+	 * @param slotID slot id of available slot
 	 */
-	public void updateSlotStatus(final SlotReport slotReport) {
-		for (SlotStatus slotStatus : slotReport.getSlotsStatus()) {
-			updateSlotStatus(slotStatus);
+	public void notifySlotAvailable(ResourceID resourceID, SlotID slotID) {
+		if (!allocationMap.isAllocated(slotID)) {
+			throw new IllegalStateException("Slot was not previously allocated but " +
+				"TaskManager reports it as available again");
 		}
-	}
-
-	/**
-	 * Registers a TaskExecutor
-	 * @param resourceID TaskExecutor's ResourceID
-	 * @param gateway TaskExcutor's gateway
-	 */
-	public void registerTaskExecutor(ResourceID resourceID, TaskExecutorGateway gateway) {
-		this.taskManagerGateways.put(resourceID, gateway);
+		allocationMap.removeAllocation(slotID);
+		final Map<SlotID, ResourceSlot> slots = registeredSlots.get(resourceID);
+		ResourceSlot freeSlot = slots.get(slotID);
+		if (freeSlot == null) {
+			throw new IllegalStateException("Slot was not registered with SlotManager but " +
+				"TaskManager reported it to be available.");
+		}
+		handleFreeSlot(freeSlot);
 	}
 
 	/**
@@ -200,51 +177,72 @@ public abstract class SlotManager {
 	 * @param originalRequest The original slot request
 	 * @param slotId          The target SlotID
 	 */
-	public void handleSlotRequestFailedAtTaskManager(final SlotRequest originalRequest, final SlotID slotId) {
+	void handleSlotRequestFailedAtTaskManager(final SlotRequest originalRequest, final SlotID slotId) {
 		final AllocationID originalAllocationId = originalRequest.getAllocationId();
 		LOG.info("Slot request failed at TaskManager, SlotID:{}, AllocationID:{}, JobID:{}",
 			slotId, originalAllocationId, originalRequest.getJobId());
 
-		// verify the allocation info before we do anything
-		if (freeSlots.containsKey(slotId)) {
-			// this slot is currently empty, no need to de-allocate it from our allocations
-			LOG.info("Original slot is somehow empty, retrying this request");
+		if (allocationMap.isAllocated(slotId)) {
+			final AllocationID expectedAllocationId = allocationMap.getAllocationID(slotId);
 
-			// before retry, we should double check whether this request was allocated by some other ways
-			if (!allocationMap.isAllocated(originalAllocationId)) {
-				requestSlot(originalRequest);
+			// check whether we have an agreement on whom this slot belongs to
+			if (originalAllocationId.equals(expectedAllocationId)) {
+				LOG.info("De-allocate this request and retry");
+				allocationMap.removeAllocation(expectedAllocationId);
+				pendingSlotRequests.put(originalRequest.getAllocationId(), originalRequest);
+				ResourceSlot slot = checkNotNull(getRegisteredSlot(slotId));
+				// treat this slot as empty and retry with a different request
+				handleFreeSlot(slot);
 			} else {
-				LOG.info("The failed request has somehow been allocated, SlotID:{}",
-					allocationMap.getSlotID(originalAllocationId));
+				LOG.error("Slot request failed for slot {} with allocation id {}:" +
+						" Allocation id did not match the expected allocation id {}.",
+					slotId, originalAllocationId, expectedAllocationId);
 			}
-		} else if (allocationMap.isAllocated(slotId)) {
-			final AllocationID currentAllocationId = allocationMap.getAllocationID(slotId);
+		} else {
+			LOG.error("Slot request failed for slot {} with allocation id {}: " +
+					"Slot was not previously registered.",
+				slotId, originalAllocationId);
+		}
+	}
 
-			// check whether we have an agreement on whom this slot belongs to
-			if (originalAllocationId.equals(currentAllocationId)) {
-				LOG.info("De-allocate this request and retry");
-				allocationMap.removeAllocation(currentAllocationId);
+	/**
+	 * Registers a TaskExecutor
+	 * @param resourceID TaskExecutor's ResourceID
+	 * @param registration TaskExecutor's registration
+	 * @param slotReport TaskExecutor's free and allocated slots
+	 */
+	public void registerTaskExecutor(
+			ResourceID resourceID,
+			TaskExecutorRegistration registration,
+			SlotReport slotReport) {
 
-				// put this slot back to free pool
-				ResourceSlot slot = checkNotNull(getRegisteredSlot(slotId));
-				freeSlots.put(slotId, slot);
+		if (taskManagers.get(resourceID) != null) {
+			notifyTaskManagerFailure(resourceID);
+		}
 
-				// retry the request
-				requestSlot(originalRequest);
-			} else {
-				// the slot is taken by someone else, no need to de-allocate it from our allocations
-				LOG.info("Original slot is taken by someone else, current AllocationID:{}", currentAllocationId);
+		this.taskManagers.put(resourceID, registration);
 
-				// before retry, we should double check whether this request was allocated by some other ways
-				if (!allocationMap.isAllocated(originalAllocationId)) {
-					requestSlot(originalRequest);
-				} else {
-					LOG.info("The failed request is somehow been allocated, SlotID:{}",
-						allocationMap.getSlotID(originalAllocationId));
-				}
+		for (SlotStatus slotStatus : slotReport.getSlotsStatus()) {
+			final SlotID slotId = slotStatus.getSlotID();
+
+			final TaskExecutorRegistration taskExecutorRegistration = taskManagers.get(slotId.getResourceID());
+			if (taskExecutorRegistration == null) {
+				LOG.info("Received SlotStatus but ResourceID {} is unknown to the SlotManager",
+					slotId.getResourceID());
+				return;
+			}
+
+			final ResourceSlot slot = new ResourceSlot(slotId, slotStatus.getProfiler(), taskExecutorRegistration);
+
+			registerNewSlot(slot);
+			LOG.info("New slot appeared, SlotID:{}, AllocationID:{}", slotId, slotStatus.getAllocationID());
+
+			if (slotStatus.getAllocationID() != null) {
+				// slot in use, record this in bookkeeping
+				allocationMap.addAllocation(slotId, slotStatus.getAllocationID());
+			} else {
+				handleFreeSlot(slot);
 			}
-		} else {
-			LOG.error("BUG! {} is neither in free pool nor in allocated pool", slotId);
 		}
 	}
 
@@ -255,7 +253,7 @@ public abstract class SlotManager {
 	 */
 	public void notifyTaskManagerFailure(final ResourceID resourceId) {
 		LOG.info("Resource:{} been notified failure", resourceId);
-		taskManagerGateways.remove(resourceId);
+		taskManagers.remove(resourceId);
 		final Map<SlotID, ResourceSlot> slotIdsToRemove = registeredSlots.remove(resourceId);
 		if (slotIdsToRemove != null) {
 			for (SlotID slotId : slotIdsToRemove.keySet()) {
@@ -276,92 +274,6 @@ public abstract class SlotManager {
 	// ------------------------------------------------------------------------
 
 	/**
-	 * Update slot status based on TaskManager's report. There are mainly two situations when we receive the report:
-	 * <ul>
-	 * <li>1. The slot is newly registered.</li>
-	 * <li>2. The slot has registered, it contains its current status.</li>
-	 * </ul>
-	 * <p>
-	 * Regarding 1: It's fairly simple, we just record this slot's status, and trigger schedule if slot is empty.
-	 * <p>
-	 * Regarding 2: It will cause some weird situation since we may have some time-gap on how the slot's status really
-	 * is. We may have some updates on the slot's allocation, but it doesn't reflected by TaskManager's heartbeat yet,
-	 * and we may make some wrong decision if we cannot guarantee we have the exact status about all the slots. So
-	 * the principle here is: We always trust TaskManager's heartbeat, we will correct our information based on that
-	 * and take next action based on the diff between our information and heartbeat status.
-	 *
-	 * @param reportedStatus Reported slot status
-	 */
-	void updateSlotStatus(final SlotStatus reportedStatus) {
-		final SlotID slotId = reportedStatus.getSlotID();
-
-		final TaskExecutorGateway taskExecutorGateway = taskManagerGateways.get(slotId.getResourceID());
-		if (taskExecutorGateway == null) {
-			LOG.info("Received SlotStatus but ResourceID {} is unknown to the SlotManager",
-				slotId.getResourceID());
-			return;
-		}
-
-		final ResourceSlot slot = new ResourceSlot(slotId, reportedStatus.getProfiler(), taskExecutorGateway);
-
-		if (registerNewSlot(slot)) {
-			// we have a newly registered slot
-			LOG.info("New slot appeared, SlotID:{}, AllocationID:{}", slotId, reportedStatus.getAllocationID());
-
-			if (reportedStatus.getAllocationID() != null) {
-				// slot in use, record this in bookkeeping
-				allocationMap.addAllocation(slotId, reportedStatus.getAllocationID());
-			} else {
-				handleFreeSlot(slot);
-			}
-		} else {
-			// slot exists, update current information
-			if (reportedStatus.getAllocationID() != null) {
-				// slot is reported in use
-				final AllocationID reportedAllocationId = reportedStatus.getAllocationID();
-
-				// check whether we also thought this slot is in use
-				if (allocationMap.isAllocated(slotId)) {
-					// we also think that slot is in use, check whether the AllocationID matches
-					final AllocationID currentAllocationId = allocationMap.getAllocationID(slotId);
-
-					if (!reportedAllocationId.equals(currentAllocationId)) {
-						LOG.info("Slot allocation info mismatch! SlotID:{}, current:{}, reported:{}",
-							slotId, currentAllocationId, reportedAllocationId);
-
-						// seems we have a disagreement about the slot assignments, need to correct it
-						allocationMap.removeAllocation(slotId);
-						allocationMap.addAllocation(slotId, reportedAllocationId);
-					}
-				} else {
-					LOG.info("Slot allocation info mismatch! SlotID:{}, current:null, reported:{}",
-						slotId, reportedAllocationId);
-
-					// we thought the slot is free, should correct this information
-					allocationMap.addAllocation(slotId, reportedStatus.getAllocationID());
-
-					// remove this slot from free slots pool
-					freeSlots.remove(slotId);
-				}
-			} else {
-				// slot is reported empty
-
-				// check whether we also thought this slot is empty
-				if (allocationMap.isAllocated(slotId)) {
-					LOG.info("Slot allocation info mismatch! SlotID:{}, current:{}, reported:null",
-						slotId, allocationMap.getAllocationID(slotId));
-
-					// we thought the slot is in use, correct it
-					allocationMap.removeAllocation(slotId);
-
-					// we have a free slot!
-					handleFreeSlot(slot);
-				}
-			}
-		}
-	}
-
-	/**
 	 * When we have a free slot, try to fulfill the pending request first. If any request can be fulfilled,
 	 * record this allocation in bookkeeping and send slot request to TaskManager, else we just add this slot
 	 * to the free pool.
@@ -373,32 +285,45 @@ public abstract class SlotManager {
 
 		if (chosenRequest != null) {
 			final AllocationID allocationId = chosenRequest.getAllocationId();
-			final SlotRequest removedSlotRequest = pendingSlotRequests.remove(allocationId);
+			final SlotRequest slotRequest = pendingSlotRequests.remove(allocationId);
 
 			LOG.info("Assigning SlotID({}) to AllocationID({}), JobID:{}", freeSlot.getSlotId(),
 				allocationId, chosenRequest.getJobId());
 			allocationMap.addAllocation(freeSlot.getSlotId(), allocationId);
 
-			final Future<SlotRequestReply> slotRequestReplyFuture =
-				freeSlot.getTaskExecutorGateway().requestSlot(allocationId, leaderID, timeout);
-
-			slotRequestReplyFuture.handleAsync(new BiFunction<SlotRequestReply, Throwable, Object>() {
-				@Override
-				public Object apply(SlotRequestReply slotRequestReply, Throwable throwable) {
-					if (throwable != null) {
-						// we failed, add the request back again
-						if (allocationMap.isAllocated(freeSlot.getSlotId())) {
-							pendingSlotRequests.put(allocationId, removedSlotRequest);
-						}
-					}
-					return null;
-				}
-			}, resourceManagerServices.getExecutor());
+			sendSlotRequest(freeSlot, slotRequest);
 		} else {
 			freeSlots.put(freeSlot.getSlotId(), freeSlot);
 		}
 	}
 
+	private void sendSlotRequest(final ResourceSlot freeSlot, final SlotRequest slotRequest) {
+
+		final AllocationID allocationID = slotRequest.getAllocationId();
+		final TaskExecutorRegistration registration = freeSlot.getTaskExecutorRegistration();
+		final Future<TMSlotRequestReply> slotRequestReplyFuture =
+			registration.getTaskExecutorGateway()
+				.requestSlot(freeSlot.getSlotId(), allocationID, leaderID, timeout);
+
+		slotRequestReplyFuture.handleAsync(new BiFunction<TMSlotRequestReply, Throwable, Void>() {
+			@Override
+			public Void apply(TMSlotRequestReply slotRequestReply, Throwable throwable) {
+				TaskExecutorRegistration current = taskManagers.get(slotRequestReply.getResourceID());
+				if (current != null && current.getInstanceID().equals(slotRequestReply.getInstanceID())) {
+					if (throwable != null || slotRequestReply instanceof TMSlotRequestRejected) {
+						handleSlotRequestFailedAtTaskManager(slotRequest, freeSlot.getSlotId());
+					} else {
+						LOG.debug("Successfully registered slot {} ", freeSlot.getSlotId());
+					}
+				} else {
+					LOG.debug("Discarding message from obsolete TaskExecutor with InstanceID {}",
+						slotRequestReply.getInstanceID());
+				}
+				return null;
+			}
+		}, rmServices.getMainThreadExecutor());
+	}
+
 	/**
 	 * Check whether the request is duplicated. We use AllocationID to identify slot request, for each
 	 * formerly received slot request, it is either in pending list or already been allocated.
@@ -413,18 +338,17 @@ public abstract class SlotManager {
 	}
 
 	/**
-	 * Try to register slot, and tell if this slot is newly registered.
+	 * Registers a new slot with the SlotManager.
 	 *
-	 * @param slot The ResourceSlot which will be checked and registered
-	 * @return <tt>true</tt> if we meet a new slot
+	 * @param slot The ResourceSlot which will be registered
 	 */
-	private boolean registerNewSlot(final ResourceSlot slot) {
+	private void registerNewSlot(final ResourceSlot slot) {
 		final SlotID slotId = slot.getSlotId();
 		final ResourceID resourceId = slotId.getResourceID();
 		if (!registeredSlots.containsKey(resourceId)) {
 			registeredSlots.put(resourceId, new HashMap<SlotID, ResourceSlot>());
 		}
-		return registeredSlots.get(resourceId).put(slotId, slot) == null;
+		registeredSlots.get(resourceId).put(slotId, slot);
 	}
 
 	private ResourceSlot getRegisteredSlot(final SlotID slotId) {
@@ -559,12 +483,12 @@ public abstract class SlotManager {
 	 * Clears the state of the SlotManager after leadership revokal
 	 */
 	public void clearState() {
-		taskManagerGateways.clear();
+		taskManagers.clear();
 		registeredSlots.clear();
 		pendingSlotRequests.clear();
 		freeSlots.clear();
 		allocationMap.clear();
-		leaderID = null;
+		leaderID = new UUID(0, 0);
 	}
 
 	// ------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/flink/blob/5915613d/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerFactory.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerFactory.java
new file mode 100644
index 0000000..b4e9c99
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerFactory.java
@@ -0,0 +1,31 @@
+/*
+ * 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.resourcemanager.ResourceManagerServices;
+
+/**
+ * Factory to create a SlotManager and provide it with dependencies.
+ */
+public interface SlotManagerFactory {
+
+	/**
+	 * Creates a SlotManager and provides it with ResourceManager services.
+	 */
+	SlotManager create(ResourceManagerServices rmServices);
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/5915613d/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/SlotReport.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/SlotReport.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/SlotReport.java
index a5de2d5..54adce6 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/SlotReport.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/SlotReport.java
@@ -18,9 +18,8 @@
 
 package org.apache.flink.runtime.taskexecutor;
 
-import org.apache.flink.runtime.clusterframework.types.ResourceID;
-
 import java.io.Serializable;
+import java.util.Collections;
 import java.util.List;
 
 import static org.apache.flink.util.Preconditions.checkNotNull;
@@ -37,20 +36,20 @@ public class SlotReport implements Serializable {
 	/** The slots status of the TaskManager */
 	private final List<SlotStatus> slotsStatus;
 
-	/** The resource id which identifies the TaskManager */
-	private final ResourceID resourceID;
+	public SlotReport() {
+		this(Collections.<SlotStatus>emptyList());
+	}
+
+	public SlotReport(SlotStatus slotStatus) {
+		this(Collections.singletonList(slotStatus));
+	}
 
-	public SlotReport(final List<SlotStatus> slotsStatus, final ResourceID resourceID) {
+	public SlotReport(final List<SlotStatus> slotsStatus) {
 		this.slotsStatus = checkNotNull(slotsStatus);
-		this.resourceID = checkNotNull(resourceID);
 	}
 
 	public List<SlotStatus> getSlotsStatus() {
 		return slotsStatus;
 	}
 
-	public ResourceID getResourceID() {
-		return resourceID;
-	}
-
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/5915613d/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 7df0a91..c0041a3 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
@@ -20,9 +20,12 @@ package org.apache.flink.runtime.taskexecutor;
 
 import org.apache.flink.runtime.clusterframework.types.AllocationID;
 import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.instance.InstanceID;
 import org.apache.flink.runtime.metrics.MetricRegistry;
-import org.apache.flink.runtime.resourcemanager.SlotRequestRegistered;
-import org.apache.flink.runtime.resourcemanager.SlotRequestReply;
+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.taskmanager.TaskManagerLocation;
 import org.apache.flink.annotation.VisibleForTesting;
@@ -38,6 +41,8 @@ import org.apache.flink.runtime.rpc.RpcService;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.util.HashSet;
+import java.util.Set;
 import java.util.UUID;
 
 import static org.apache.flink.util.Preconditions.checkArgument;
@@ -78,6 +83,9 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 	/** The fatal error handler to use in case of a fatal error */
 	private final FatalErrorHandler fatalErrorHandler;
 
+	/** Slots which have become available but haven't been confirmed by the RM */
+	private final Set<SlotID> unconfirmedFreeSlots;
+
 	// --------- resource manager --------
 
 	private TaskExecutorToResourceManagerConnection resourceManagerConnection;
@@ -109,6 +117,8 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 		this.fatalErrorHandler = checkNotNull(fatalErrorHandler);
 
 		this.numberOfSlots =  taskManagerConfiguration.getNumberSlots();
+
+		this.unconfirmedFreeSlots = new HashSet<>();
 	}
 
 	// ------------------------------------------------------------------------
@@ -152,6 +162,8 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 			}
 		}
 
+		unconfirmedFreeSlots.clear();
+
 		// establish a connection to the new leader
 		if (newLeaderAddress != null) {
 			log.info("Attempting to register at ResourceManager {}", newLeaderAddress);
@@ -169,13 +181,25 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 	/**
 	 * Requests a slot from the TaskManager
 	 *
+	 * @param slotID Slot id for the request
 	 * @param allocationID id for the request
 	 * @param resourceManagerLeaderID current leader id of the ResourceManager
 	 * @return answer to the slot request
 	 */
 	@RpcMethod
-	public SlotRequestReply requestSlot(AllocationID allocationID, UUID resourceManagerLeaderID) {
-		return new SlotRequestRegistered(allocationID);
+	public TMSlotRequestReply requestSlot(SlotID slotID, AllocationID allocationID, UUID resourceManagerLeaderID) {
+		if (!resourceManagerConnection.getTargetLeaderId().equals(resourceManagerLeaderID)) {
+			return new TMSlotRequestRejected(
+				resourceManagerConnection.getRegistrationId(), getResourceID(), allocationID);
+		}
+		if (unconfirmedFreeSlots.contains(slotID)) {
+			// check if request has not been blacklisted because the notification of a free slot
+			// has not been confirmed by the ResourceManager
+			return new TMSlotRequestRejected(
+				resourceManagerConnection.getRegistrationId(), getResourceID(), allocationID);
+		}
+		return new TMSlotRequestRegistered(new InstanceID(), ResourceID.generate(), allocationID);
+
 	}
 
 	// ------------------------------------------------------------------------
@@ -227,6 +251,11 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 		return resourceManagerConnection;
 	}
 
+	@VisibleForTesting
+	public void addUnconfirmedFreeSlotNotification(SlotID slotID) {
+		unconfirmedFreeSlots.add(slotID);
+	}
+
 	// ------------------------------------------------------------------------
 	//  Utility classes
 	// ------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/flink/blob/5915613d/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 0962802..2360b53 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
@@ -20,8 +20,9 @@ package org.apache.flink.runtime.taskexecutor;
 
 import org.apache.flink.api.common.time.Time;
 import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
 import org.apache.flink.runtime.concurrent.Future;
-import org.apache.flink.runtime.resourcemanager.SlotRequestReply;
+import org.apache.flink.runtime.resourcemanager.messages.taskexecutor.TMSlotRequestReply;
 import org.apache.flink.runtime.rpc.RpcGateway;
 import org.apache.flink.runtime.rpc.RpcTimeout;
 
@@ -41,11 +42,13 @@ public interface TaskExecutorGateway extends RpcGateway {
 	/**
 	 * Requests a slot from the TaskManager
 	 *
+	 * @param slotID slot id for the request
 	 * @param allocationID id for the request
 	 * @param resourceManagerLeaderID current leader id of the ResourceManager
 	 * @return answer to the slot request
 	 */
-	Future<SlotRequestReply> requestSlot(
+	Future<TMSlotRequestReply> requestSlot(
+		SlotID slotID,
 		AllocationID allocationID,
 		UUID resourceManagerLeaderID,
 		@RpcTimeout Time timeout);

http://git-wip-us.apache.org/repos/asf/flink/blob/5915613d/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 b4b3bae..2dbd9eb 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
@@ -113,7 +113,7 @@ public class TaskExecutorToResourceManagerConnection
 				ResourceManagerGateway resourceManager, UUID leaderId, long timeoutMillis) throws Exception {
 
 			Time timeout = Time.milliseconds(timeoutMillis);
-			return resourceManager.registerTaskExecutor(leaderId, taskExecutorAddress, resourceID, timeout);
+			return resourceManager.registerTaskExecutor(leaderId, taskExecutorAddress, resourceID, new SlotReport(), timeout);
 		}
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/5915613d/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 fdb83f5..ce1fdca 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
@@ -21,6 +21,7 @@ package org.apache.flink.runtime.resourcemanager;
 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.resourcemanager.slotmanager.SlotManagerFactory;
 import org.apache.flink.runtime.rpc.MainThreadExecutable;
 import org.apache.flink.runtime.rpc.RpcEndpoint;
 import org.apache.flink.runtime.rpc.RpcGateway;
@@ -43,7 +44,8 @@ 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 execute runAsync call
+		// 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));
 
@@ -54,18 +56,18 @@ public class ResourceManagerHATest {
 		TestingHighAvailabilityServices highAvailabilityServices = new TestingHighAvailabilityServices();
 		highAvailabilityServices.setResourceManagerLeaderElectionService(leaderElectionService);
 
-		SlotManager slotManager = mock(SlotManager.class);
-		final ResourceManager resourceManager = new StandaloneResourceManager(rpcService, highAvailabilityServices, slotManager);
+		final ResourceManager resourceManager =
+			new TestingResourceManager(rpcService, highAvailabilityServices);
 		resourceManager.start();
 		// before grant leadership, resourceManager's leaderId is null
-		Assert.assertNull(resourceManager.getLeaderSessionID());
+		Assert.assertEquals(new UUID(0,0), 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.assertNull(resourceManager.getLeaderSessionID());
+		Assert.assertEquals(new UUID(0,0), resourceManager.getLeaderSessionID());
 	}
 
 	private static abstract class TestingResourceManagerGatewayProxy implements MainThreadExecutable, StartStoppable, RpcGateway {


[33/50] [abbrv] flink git commit: [FLINK-4406] [cluster management] Implement job master registration at resource manager

Posted by se...@apache.org.
[FLINK-4406] [cluster management] Implement job master registration at resource manager

[FLINK-4406] [cluster management] Skip new connection if new resource manager's address and leader id are both not changing

[FLINK-4406] [cluster management] Verify registration response with leader id

This closes #2565.


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

Branch: refs/heads/flip-6
Commit: 291daf6ae4903b525722849eff4560daacb703c3
Parents: bce292a
Author: Kurt Young <yk...@gmail.com>
Authored: Thu Sep 29 08:56:27 2016 +0800
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Oct 14 15:14:42 2016 +0200

----------------------------------------------------------------------
 .../runtime/jobmaster/JobManagerRunner.java     |   8 +-
 .../flink/runtime/jobmaster/JobMaster.java      | 222 +++++++++++++++++--
 .../runtime/jobmaster/JobMasterGateway.java     |  17 +-
 .../jobmaster/JobMasterRegistrationSuccess.java |  18 +-
 .../JobMasterToResourceManagerConnection.java   | 117 ----------
 .../resourcemanager/ResourceManager.java        |   2 +-
 .../jobmaster/JobManagerRunnerMockTest.java     |  25 ++-
 7 files changed, 239 insertions(+), 170 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/291daf6a/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 bc2bf9a..6944d85 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
@@ -63,9 +63,6 @@ public class JobManagerRunner implements LeaderContender, OnCompletionActions {
 
 	private final JobMaster jobManager;
 
-	/** Leader session id when granted leadership */
-	private UUID leaderSessionID;
-
 	/** flag marking the runner as shut down */
 	private volatile boolean shutdown;
 
@@ -93,7 +90,6 @@ public class JobManagerRunner implements LeaderContender, OnCompletionActions {
 		this.executionContext = rpcService.getExecutor();
 		this.checkpointRecoveryFactory = haServices.getCheckpointRecoveryFactory();
 		this.leaderElectionService = haServices.getJobMasterLeaderElectionService(jobGraph.getJobID());
-		this.leaderSessionID = null;
 
 		this.jobManager = new JobMaster(
 			jobGraph, configuration, rpcService, haServices,
@@ -232,7 +228,6 @@ public class JobManagerRunner implements LeaderContender, OnCompletionActions {
 			// The operation may be blocking, but since this runner is idle before it been granted the leadership,
 			// it's okay that job manager wait for the operation complete
 			leaderElectionService.confirmLeaderSessionID(leaderSessionID);
-			this.leaderSessionID = leaderSessionID;
 
 			// 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.
@@ -242,7 +237,7 @@ public class JobManagerRunner implements LeaderContender, OnCompletionActions {
 					log.info("Job {} ({}) already finished by others.", jobGraph.getName(), jobGraph.getJobID());
 					jobFinishedByOther();
 				} else {
-					jobManager.getSelf().startJob();
+					jobManager.getSelf().startJob(leaderSessionID);
 				}
 			}
 		}
@@ -259,7 +254,6 @@ public class JobManagerRunner implements LeaderContender, OnCompletionActions {
 			log.info("JobManager for job {} ({}) was revoked leadership at {}.",
 				jobGraph.getName(), jobGraph.getJobID(), getAddress());
 
-			leaderSessionID = null;
 			jobManager.getSelf().suspendJob(new Exception("JobManager is no longer the leader."));
 		}
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/291daf6a/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 b52a23c..1e01c55 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
@@ -19,6 +19,7 @@
 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;
@@ -34,6 +35,7 @@ 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.concurrent.Future;
 import org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager;
 import org.apache.flink.runtime.executiongraph.ExecutionGraph;
 import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
@@ -47,18 +49,26 @@ import org.apache.flink.runtime.jobgraph.jsonplan.JsonPlanGenerator;
 import org.apache.flink.runtime.jobgraph.tasks.JobSnapshottingSettings;
 import org.apache.flink.runtime.jobmanager.OnCompletionActions;
 import org.apache.flink.runtime.jobmanager.scheduler.Scheduler;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
 import org.apache.flink.runtime.messages.Acknowledge;
 import org.apache.flink.runtime.metrics.groups.JobManagerMetricGroup;
+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.RpcEndpoint;
 import org.apache.flink.runtime.rpc.RpcMethod;
 import org.apache.flink.runtime.rpc.RpcService;
 import org.apache.flink.runtime.taskmanager.TaskExecutionState;
+import org.slf4j.Logger;
+
 import scala.concurrent.ExecutionContext$;
 import scala.concurrent.duration.FiniteDuration;
 
 import java.util.ArrayList;
 import java.util.List;
+import java.util.UUID;
 import java.util.concurrent.Executor;
 
 import static org.apache.flink.util.Preconditions.checkNotNull;
@@ -76,9 +86,6 @@ import static org.apache.flink.util.Preconditions.checkNotNull;
  */
 public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 
-	/** Gateway to connected resource manager, null iff not connected */
-	private ResourceManagerGateway resourceManager = null;
-
 	/** Logical representation of the job */
 	private final JobGraph jobGraph;
 
@@ -123,6 +130,18 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 
 	private MetricGroup jobMetrics;
 
+	private volatile UUID leaderSessionID;
+
+	// --------- resource manager --------
+
+	/** Leader retriever service used to locate ResourceManager's address */
+	private LeaderRetrievalService resourceManagerLeaderRetriever;
+
+	/** Connection with ResourceManager, null if not located address yet or we close it initiative */
+	private volatile ResourceManagerConnection resourceManagerConnection;
+
+	// ------------------------------------------------------------------------
+
 	public JobMaster(
 		JobGraph jobGraph,
 		Configuration configuration,
@@ -151,10 +170,6 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 		this.jobCompletionActions = checkNotNull(jobCompletionActions);
 	}
 
-	public ResourceManagerGateway getResourceManager() {
-		return resourceManager;
-	}
-
 	//----------------------------------------------------------------------------------------------
 	// Lifecycle management
 	//----------------------------------------------------------------------------------------------
@@ -196,7 +211,8 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 					.getRestartStrategy();
 			if (restartStrategyConfiguration != null) {
 				restartStrategy = RestartStrategyFactory.createRestartStrategy(restartStrategyConfiguration);
-			} else {
+			}
+			else {
 				restartStrategy = restartStrategyFactory.createRestartStrategy();
 			}
 
@@ -216,6 +232,13 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 				throw new JobSubmissionException(jobGraph.getJobID(), "Could not get the checkpoint recovery factory.", e);
 			}
 
+			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);
 
@@ -223,7 +246,8 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 
 			if (t instanceof JobSubmissionException) {
 				throw (JobSubmissionException) t;
-			} else {
+			}
+			else {
 				throw new JobSubmissionException(jobGraph.getJobID(), "Failed to initialize job " +
 					jobGraph.getName() + " (" + jobGraph.getJobID() + ")", t);
 			}
@@ -240,8 +264,12 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 		super.shutDown();
 
 		suspendJob(new Exception("JobManager is shutting down."));
+
+		disposeCommunicationWithResourceManager();
 	}
 
+
+
 	//----------------------------------------------------------------------------------------------
 	// RPC methods
 	//----------------------------------------------------------------------------------------------
@@ -251,8 +279,10 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 	 * being recovered. After this, we will begin to schedule the job.
 	 */
 	@RpcMethod
-	public void startJob() {
-		log.info("Starting job {} ({}).", jobGraph.getName(), jobGraph.getJobID());
+	public void startJob(final UUID leaderSessionID) {
+		log.info("Starting job {} ({}) with leaderId {}.", jobGraph.getName(), jobGraph.getJobID(), leaderSessionID);
+
+		this.leaderSessionID = leaderSessionID;
 
 		if (executionGraph != null) {
 			executionGraph = new ExecutionGraph(
@@ -267,7 +297,8 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 				jobGraph.getClasspaths(),
 				userCodeLoader,
 				jobMetrics);
-		} else {
+		}
+		else {
 			// TODO: update last active time in JobInfo
 		}
 
@@ -343,7 +374,8 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 				final CheckpointStatsTracker checkpointStatsTracker;
 				if (isStatsDisabled) {
 					checkpointStatsTracker = new DisabledCheckpointStatsTracker();
-				} else {
+				}
+				else {
 					int historySize = configuration.getInteger(
 						ConfigConstants.JOB_MANAGER_WEB_CHECKPOINTS_HISTORY_SIZE,
 						ConfigConstants.DEFAULT_JOB_MANAGER_WEB_CHECKPOINTS_HISTORY_SIZE);
@@ -397,6 +429,8 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 			}
 			*/
 
+			// job is good to go, try to locate resource manager's address
+			resourceManagerLeaderRetriever.start(new ResourceManagerLeaderListener());
 		} catch (Throwable t) {
 			log.error("Failed to start job {} ({})", jobGraph.getName(), jobGraph.getJobID(), t);
 
@@ -406,7 +440,8 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 			final Throwable rt;
 			if (t instanceof JobExecutionException) {
 				rt = (JobExecutionException) t;
-			} else {
+			}
+			else {
 				rt = new JobExecutionException(jobGraph.getJobID(),
 					"Failed to start job " + jobGraph.getJobID() + " (" + jobGraph.getName() + ")", t);
 			}
@@ -439,10 +474,14 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 	 */
 	@RpcMethod
 	public void suspendJob(final Throwable cause) {
+		leaderSessionID = null;
+
 		if (executionGraph != null) {
 			executionGraph.suspend(cause);
 			executionGraph = null;
 		}
+
+		disposeCommunicationWithResourceManager();
 	}
 
 	/**
@@ -457,14 +496,90 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 		return Acknowledge.get();
 	}
 
-	/**
-	 * Triggers the registration of the job master at the resource manager.
-	 *
-	 * @param address Address of the resource manager
-	 */
-	@RpcMethod
-	public void registerAtResourceManager(final String address) {
-		//TODO:: register at the RM
+	//----------------------------------------------------------------------------------------------\u2028
+	// Internal methods\u2028
+	// ----------------------------------------------------------------------------------------------\u2028\u2028
+
+	private void handleFatalError(final Throwable cause) {
+		runAsync(new Runnable() {
+			@Override
+			public void run() {
+				log.error("Fatal error occurred on JobManager, cause: {}", cause.getMessage(), cause);
+				shutDown();
+				jobCompletionActions.onFatalError(cause);
+			}
+		});
+	}
+
+	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());
+					}
+				}
+
+				closeResourceManagerConnection();
+
+				if (resourceManagerAddress != null) {
+					log.info("Attempting to register at ResourceManager {}", resourceManagerAddress);
+					resourceManagerConnection = new ResourceManagerConnection(
+						log, jobGraph.getJobID(), 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());
+				}
+			}
+		});
+	}
+
+	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();
+			resourceManagerConnection = null;
+		}
 	}
 
 	//----------------------------------------------------------------------------------------------
@@ -494,4 +609,67 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 		}
 		return ret;
 	}
+
+	//----------------------------------------------------------------------------------------------
+	// Utility classes
+	//----------------------------------------------------------------------------------------------
+
+	private class ResourceManagerLeaderListener implements LeaderRetrievalListener {
+		@Override
+		public void notifyLeaderAddress(final String leaderAddress, final UUID leaderSessionID) {
+			notifyOfNewResourceManagerLeader(leaderAddress, leaderSessionID);
+		}
+
+		@Override
+		public void handleError(final Exception exception) {
+			handleFatalError(exception);
+		}
+	}
+
+	private class ResourceManagerConnection
+		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)
+		{
+			super(log, resourceManagerAddress, resourceManagerLeaderID, executor);
+			this.jobID = checkNotNull(jobID);
+			this.jobManagerLeaderID = checkNotNull(jobManagerLeaderID);
+		}
+
+		@Override
+		protected RetryingRegistration<ResourceManagerGateway, JobMasterRegistrationSuccess> generateRegistration() {
+			return new RetryingRegistration<ResourceManagerGateway, JobMasterRegistrationSuccess>(
+				log, getRpcService(), "ResourceManager", ResourceManagerGateway.class,
+				getTargetAddress(), getTargetLeaderId())
+			{
+				@Override
+				protected Future<RegistrationResponse> invokeRegistration(ResourceManagerGateway gateway, UUID leaderId,
+					long timeoutMillis) throws Exception
+				{
+					Time timeout = Time.milliseconds(timeoutMillis);
+					return gateway.registerJobMaster(leaderId, jobManagerLeaderID, getAddress(), jobID, timeout);
+				}
+			};
+		}
+
+		@Override
+		protected void onRegistrationSuccess(final JobMasterRegistrationSuccess success) {
+			onResourceManagerRegistrationSuccess(success);
+		}
+
+		@Override
+		protected void onRegistrationFailure(final Throwable failure) {
+			handleFatalError(failure);
+		}
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/291daf6a/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 b281ea8..6587ccb 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
@@ -23,19 +23,21 @@ import org.apache.flink.runtime.messages.Acknowledge;
 import org.apache.flink.runtime.rpc.RpcGateway;
 import org.apache.flink.runtime.taskmanager.TaskExecutionState;
 
+import java.util.UUID;
+
 /**
  * {@link JobMaster} rpc gateway interface
  */
 public interface JobMasterGateway extends RpcGateway {
 
 	/**
-	 * Making this job begins to run.
+	 * Starting the job under the given leader session ID.
 	 */
-	void startJob();
+	void startJob(final UUID leaderSessionID);
 
 	/**
-	 * Suspending job, all the running tasks will be cancelled, and runtime status will be cleared. Should re-submit
-	 * the job before restarting it.
+	 * 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.
 	 */
@@ -48,11 +50,4 @@ public interface JobMasterGateway extends RpcGateway {
 	 * @return Future acknowledge of the task execution state update
 	 */
 	Future<Acknowledge> updateTaskExecutionState(TaskExecutionState taskExecutionState);
-
-	/**
-	 * Triggers the registration of the job master at the resource manager.
-	 *
-	 * @param address Address of the resource manager
-	 */
-	void registerAtResourceManager(final String address);
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/291daf6a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterRegistrationSuccess.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterRegistrationSuccess.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterRegistrationSuccess.java
index 031c38e..4058452 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterRegistrationSuccess.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterRegistrationSuccess.java
@@ -20,6 +20,10 @@ package org.apache.flink.runtime.jobmaster;
 
 import org.apache.flink.runtime.registration.RegistrationResponse;
 
+import java.util.UUID;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
 /**
  * Base class for responses from the ResourceManager to a registration attempt by a JobMaster.
  */
@@ -29,8 +33,11 @@ public class JobMasterRegistrationSuccess extends RegistrationResponse.Success {
 
 	private final long heartbeatInterval;
 
-	public JobMasterRegistrationSuccess(long heartbeatInterval) {
+	private final UUID resourceManagerLeaderId;
+
+	public JobMasterRegistrationSuccess(final long heartbeatInterval, final UUID resourceManagerLeaderId) {
 		this.heartbeatInterval = heartbeatInterval;
+		this.resourceManagerLeaderId = checkNotNull(resourceManagerLeaderId);
 	}
 
 	/**
@@ -42,8 +49,15 @@ public class JobMasterRegistrationSuccess extends RegistrationResponse.Success {
 		return heartbeatInterval;
 	}
 
+	public UUID getResourceManagerLeaderId() {
+		return resourceManagerLeaderId;
+	}
+
 	@Override
 	public String toString() {
-		return "JobMasterRegistrationSuccess(" + heartbeatInterval + ')';
+		return "JobMasterRegistrationSuccess{" +
+			"heartbeatInterval=" + heartbeatInterval +
+			", resourceManagerLeaderId=" + resourceManagerLeaderId +
+			'}';
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/291daf6a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterToResourceManagerConnection.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterToResourceManagerConnection.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterToResourceManagerConnection.java
deleted file mode 100644
index 71fce8c..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterToResourceManagerConnection.java
+++ /dev/null
@@ -1,117 +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.JobID;
-import org.apache.flink.api.common.time.Time;
-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.RpcService;
-import org.apache.flink.runtime.concurrent.Future;
-
-import org.slf4j.Logger;
-
-import java.util.UUID;
-import java.util.concurrent.Executor;
-
-import static org.apache.flink.util.Preconditions.checkNotNull;
-
-/**
- * The connection between a JobMaster and the ResourceManager.
- */
-public class JobMasterToResourceManagerConnection 
-		extends RegisteredRpcConnection<ResourceManagerGateway, JobMasterRegistrationSuccess> {
-
-	/** the JobMaster whose connection to the ResourceManager this represents */
-	private final JobMaster jobMaster;
-
-	private final JobID jobID;
-
-	private final UUID jobMasterLeaderId;
-
-	public JobMasterToResourceManagerConnection(
-			Logger log,
-			JobID jobID,
-			JobMaster jobMaster,
-			UUID jobMasterLeaderId,
-			String resourceManagerAddress,
-			UUID resourceManagerLeaderId,
-			Executor executor) {
-
-		super(log, resourceManagerAddress, resourceManagerLeaderId, executor);
-		this.jobMaster = checkNotNull(jobMaster);
-		this.jobID = checkNotNull(jobID);
-		this.jobMasterLeaderId = checkNotNull(jobMasterLeaderId);
-	}
-
-	@Override
-	protected RetryingRegistration<ResourceManagerGateway, JobMasterRegistrationSuccess> generateRegistration() {
-		return new JobMasterToResourceManagerConnection.ResourceManagerRegistration(
-			log, jobMaster.getRpcService(),
-			getTargetAddress(), getTargetLeaderId(),
-			jobMaster.getAddress(),jobID, jobMasterLeaderId);
-	}
-
-	@Override
-	protected void onRegistrationSuccess(JobMasterRegistrationSuccess success) {
-	}
-
-	@Override
-	protected void onRegistrationFailure(Throwable failure) {
-	}
-
-	// ------------------------------------------------------------------------
-	//  Utilities
-	// ------------------------------------------------------------------------
-
-	private static class ResourceManagerRegistration
-		extends RetryingRegistration<ResourceManagerGateway, JobMasterRegistrationSuccess> {
-
-		private final String jobMasterAddress;
-
-		private final JobID jobID;
-
-		private final UUID jobMasterLeaderId;
-
-		ResourceManagerRegistration(
-			Logger log,
-			RpcService rpcService,
-			String targetAddress,
-			UUID leaderId,
-			String jobMasterAddress,
-			JobID jobID,
-			UUID jobMasterLeaderId) {
-
-			super(log, rpcService, "ResourceManager", ResourceManagerGateway.class, targetAddress, leaderId);
-			this.jobMasterAddress = checkNotNull(jobMasterAddress);
-			this.jobID = checkNotNull(jobID);
-			this.jobMasterLeaderId = checkNotNull(jobMasterLeaderId);
-		}
-
-		@Override
-		protected Future<RegistrationResponse> invokeRegistration(
-			ResourceManagerGateway gateway, UUID leaderId, long timeoutMillis) throws Exception {
-
-			Time timeout = Time.milliseconds(timeoutMillis);
-			return gateway.registerJobMaster(leaderId, jobMasterLeaderId,jobMasterAddress, jobID, timeout);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/291daf6a/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 190a4de..f695de4 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
@@ -215,7 +215,7 @@ public abstract class ResourceManager<WorkerType extends ResourceIDRetrievable>
 						if (existingGateway != null) {
 							log.info("Replacing gateway for registered JobID {}.", jobID);
 						}
-						return new JobMasterRegistrationSuccess(5000);
+						return new JobMasterRegistrationSuccess(5000, resourceManagerLeaderId);
 					}
 				}
 			}, getMainThreadExecutor());

http://git-wip-us.apache.org/repos/asf/flink/blob/291daf6a/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 dc3b5fd..bfe5f55 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
@@ -142,8 +142,9 @@ public class JobManagerRunnerMockTest {
 	public void testJobFinished() throws Exception {
 		runner.start();
 
-		runner.grantLeadership(UUID.randomUUID());
-		verify(jobManagerGateway).startJob();
+		UUID leaderSessionID = UUID.randomUUID();
+		runner.grantLeadership(leaderSessionID);
+		verify(jobManagerGateway).startJob(leaderSessionID);
 		assertTrue(!jobCompletion.isJobFinished());
 
 		// runner been told by JobManager that job is finished
@@ -160,8 +161,9 @@ public class JobManagerRunnerMockTest {
 	public void testJobFailed() throws Exception {
 		runner.start();
 
-		runner.grantLeadership(UUID.randomUUID());
-		verify(jobManagerGateway).startJob();
+		UUID leaderSessionID = UUID.randomUUID();
+		runner.grantLeadership(leaderSessionID);
+		verify(jobManagerGateway).startJob(leaderSessionID);
 		assertTrue(!jobCompletion.isJobFinished());
 
 		// runner been told by JobManager that job is failed
@@ -177,8 +179,9 @@ public class JobManagerRunnerMockTest {
 	public void testLeadershipRevoked() throws Exception {
 		runner.start();
 
-		runner.grantLeadership(UUID.randomUUID());
-		verify(jobManagerGateway).startJob();
+		UUID leaderSessionID = UUID.randomUUID();
+		runner.grantLeadership(leaderSessionID);
+		verify(jobManagerGateway).startJob(leaderSessionID);
 		assertTrue(!jobCompletion.isJobFinished());
 
 		runner.revokeLeadership();
@@ -190,16 +193,18 @@ public class JobManagerRunnerMockTest {
 	public void testRegainLeadership() throws Exception {
 		runner.start();
 
-		runner.grantLeadership(UUID.randomUUID());
-		verify(jobManagerGateway).startJob();
+		UUID leaderSessionID = UUID.randomUUID();
+		runner.grantLeadership(leaderSessionID);
+		verify(jobManagerGateway).startJob(leaderSessionID);
 		assertTrue(!jobCompletion.isJobFinished());
 
 		runner.revokeLeadership();
 		verify(jobManagerGateway).suspendJob(any(Throwable.class));
 		assertFalse(runner.isShutdown());
 
-		runner.grantLeadership(UUID.randomUUID());
-		verify(jobManagerGateway, times(2)).startJob();
+		UUID leaderSessionID2 = UUID.randomUUID();
+		runner.grantLeadership(leaderSessionID2);
+		verify(jobManagerGateway, times(2)).startJob(leaderSessionID2);
 	}
 
 	private static class TestingOnCompletionActions implements OnCompletionActions {


[34/50] [abbrv] flink git commit: [hotfix] Replace TaskManager.createTaskManagerComponents by TaskManagerServices

Posted by se...@apache.org.
[hotfix] Replace TaskManager.createTaskManagerComponents by TaskManagerServices


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

Branch: refs/heads/flip-6
Commit: bce292ae9eb15ba07598e26c96634c7eee45db9d
Parents: 1f198d8
Author: Till Rohrmann <tr...@apache.org>
Authored: Wed Sep 28 14:04:54 2016 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Oct 14 15:14:42 2016 +0200

----------------------------------------------------------------------
 .../clusterframework/MesosTaskManager.scala     |   3 +-
 .../taskexecutor/TaskManagerConfiguration.java  |  25 +-
 .../TaskManagerServicesConfiguration.java       |   2 +-
 .../minicluster/LocalFlinkMiniCluster.scala     |  47 +-
 .../flink/runtime/taskmanager/TaskManager.scala | 605 ++-----------------
 .../taskmanager/TaskManagerConfiguration.scala  |  56 --
 ...askManagerComponentsStartupShutdownTest.java |  24 +-
 .../testingUtils/TestingTaskManager.scala       |   3 +-
 .../runtime/testingUtils/TestingUtils.scala     |   1 -
 .../flink/yarn/TestingYarnTaskManager.scala     |   3 +-
 .../org/apache/flink/yarn/YarnTaskManager.scala |   3 +-
 11 files changed, 126 insertions(+), 646 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/bce292ae/flink-mesos/src/main/scala/org/apache/flink/mesos/runtime/clusterframework/MesosTaskManager.scala
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/scala/org/apache/flink/mesos/runtime/clusterframework/MesosTaskManager.scala b/flink-mesos/src/main/scala/org/apache/flink/mesos/runtime/clusterframework/MesosTaskManager.scala
index 3972a57..e8d6a58 100644
--- a/flink-mesos/src/main/scala/org/apache/flink/mesos/runtime/clusterframework/MesosTaskManager.scala
+++ b/flink-mesos/src/main/scala/org/apache/flink/mesos/runtime/clusterframework/MesosTaskManager.scala
@@ -24,7 +24,8 @@ import org.apache.flink.runtime.io.network.NetworkEnvironment
 import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService
 import org.apache.flink.runtime.memory.MemoryManager
 import org.apache.flink.runtime.metrics.MetricRegistry
-import org.apache.flink.runtime.taskmanager.{TaskManager, TaskManagerConfiguration, TaskManagerLocation}
+import org.apache.flink.runtime.taskexecutor.TaskManagerConfiguration
+import org.apache.flink.runtime.taskmanager.{TaskManager, TaskManagerLocation}
 
 /** An extension of the TaskManager that listens for additional Mesos-related
   * messages.

http://git-wip-us.apache.org/repos/asf/flink/blob/bce292ae/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerConfiguration.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerConfiguration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerConfiguration.java
index 32eb8c1..f58af77 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerConfiguration.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerConfiguration.java
@@ -41,6 +41,7 @@ public class TaskManagerConfiguration {
 	private final String[] tmpDirPaths;
 
 	private final Time timeout;
+	// null indicates an infinite duration
 	private final Time maxRegistrationDuration;
 	private final Time initialRegistrationPause;
 	private final Time maxRegistrationPause;
@@ -48,6 +49,9 @@ public class TaskManagerConfiguration {
 
 	private final long cleanupInterval;
 
+	// TODO: remove necessity for complete configuration object
+	private final Configuration configuration;
+
 	public TaskManagerConfiguration(
 		int numberSlots,
 		String[] tmpDirPaths,
@@ -56,16 +60,18 @@ public class TaskManagerConfiguration {
 		Time initialRegistrationPause,
 		Time maxRegistrationPause,
 		Time refusedRegistrationPause,
-		long cleanupInterval) {
+		long cleanupInterval,
+		Configuration configuration) {
 
 		this.numberSlots = numberSlots;
 		this.tmpDirPaths = Preconditions.checkNotNull(tmpDirPaths);
 		this.timeout = Preconditions.checkNotNull(timeout);
-		this.maxRegistrationDuration = Preconditions.checkNotNull(maxRegistrationDuration);
+		this.maxRegistrationDuration = maxRegistrationDuration;
 		this.initialRegistrationPause = Preconditions.checkNotNull(initialRegistrationPause);
 		this.maxRegistrationPause = Preconditions.checkNotNull(maxRegistrationPause);
 		this.refusedRegistrationPause = Preconditions.checkNotNull(refusedRegistrationPause);
 		this.cleanupInterval = Preconditions.checkNotNull(cleanupInterval);
+		this.configuration = Preconditions.checkNotNull(configuration);
 	}
 
 	public int getNumberSlots() {
@@ -100,6 +106,10 @@ public class TaskManagerConfiguration {
 		return cleanupInterval;
 	}
 
+	public Configuration getConfiguration() {
+		return configuration;
+	}
+
 	// --------------------------------------------------------------------------------------------
 	//  Static factory methods
 	// --------------------------------------------------------------------------------------------
@@ -138,7 +148,7 @@ public class TaskManagerConfiguration {
 				ConfigConstants.TASK_MANAGER_MAX_REGISTRATION_DURATION,
 				ConfigConstants.DEFAULT_TASK_MANAGER_MAX_REGISTRATION_DURATION));
 			if (maxRegistrationDuration.isFinite()) {
-				finiteRegistrationDuration = Time.seconds(maxRegistrationDuration.toSeconds());
+				finiteRegistrationDuration = Time.milliseconds(maxRegistrationDuration.toMillis());
 			} else {
 				finiteRegistrationDuration = null;
 			}
@@ -153,7 +163,7 @@ public class TaskManagerConfiguration {
 				ConfigConstants.TASK_MANAGER_INITIAL_REGISTRATION_PAUSE,
 				ConfigConstants.DEFAULT_TASK_MANAGER_INITIAL_REGISTRATION_PAUSE));
 			if (pause.isFinite()) {
-				initialRegistrationPause = Time.seconds(pause.toSeconds());
+				initialRegistrationPause = Time.milliseconds(pause.toMillis());
 			} else {
 				throw new IllegalArgumentException("The initial registration pause must be finite: " + pause);
 			}
@@ -168,7 +178,7 @@ public class TaskManagerConfiguration {
 				ConfigConstants.TASK_MANAGER_MAX_REGISTARTION_PAUSE,
 				ConfigConstants.DEFAULT_TASK_MANAGER_MAX_REGISTRATION_PAUSE));
 			if (pause.isFinite()) {
-				maxRegistrationPause = Time.seconds(pause.toSeconds());
+				maxRegistrationPause = Time.milliseconds(pause.toMillis());
 			} else {
 				throw new IllegalArgumentException("The maximum registration pause must be finite: " + pause);
 			}
@@ -183,7 +193,7 @@ public class TaskManagerConfiguration {
 				ConfigConstants.TASK_MANAGER_REFUSED_REGISTRATION_PAUSE,
 				ConfigConstants.DEFAULT_TASK_MANAGER_REFUSED_REGISTRATION_PAUSE));
 			if (pause.isFinite()) {
-				refusedRegistrationPause = Time.seconds(pause.toSeconds());
+				refusedRegistrationPause = Time.milliseconds(pause.toMillis());
 			} else {
 				throw new IllegalArgumentException("The refused registration pause must be finite: " + pause);
 			}
@@ -200,6 +210,7 @@ public class TaskManagerConfiguration {
 			initialRegistrationPause,
 			maxRegistrationPause,
 			refusedRegistrationPause,
-			cleanupInterval);
+			cleanupInterval,
+			configuration);
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/bce292ae/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 66d969a..80dfc09 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
@@ -208,7 +208,7 @@ public class TaskManagerServicesConfiguration {
 		int dataport = configuration.getInteger(ConfigConstants.TASK_MANAGER_DATA_PORT_KEY,
 			ConfigConstants.DEFAULT_TASK_MANAGER_DATA_PORT);
 
-		checkConfigParameter(dataport > 0, dataport, ConfigConstants.TASK_MANAGER_DATA_PORT_KEY,
+		checkConfigParameter(dataport >= 0, dataport, ConfigConstants.TASK_MANAGER_DATA_PORT_KEY,
 			"Leave config parameter empty or use 0 to let the system choose a port automatically.");
 
 		checkConfigParameter(slots >= 1, slots, ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS,

http://git-wip-us.apache.org/repos/asf/flink/blob/bce292ae/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/LocalFlinkMiniCluster.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/LocalFlinkMiniCluster.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/LocalFlinkMiniCluster.scala
index 2f453a3..a5e229b 100644
--- a/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/LocalFlinkMiniCluster.scala
+++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/LocalFlinkMiniCluster.scala
@@ -18,6 +18,7 @@
 
 package org.apache.flink.runtime.minicluster
 
+import java.net.InetAddress
 import java.util.concurrent.ExecutorService
 
 import akka.actor.{ActorRef, ActorSystem, Props}
@@ -42,8 +43,9 @@ import org.apache.flink.runtime.memory.MemoryManager
 import org.apache.flink.runtime.messages.JobManagerMessages
 import org.apache.flink.runtime.messages.JobManagerMessages.{RunningJobsStatus, StoppingFailure, StoppingResponse}
 import org.apache.flink.runtime.metrics.MetricRegistry
-import org.apache.flink.runtime.taskmanager.{TaskManager, TaskManagerConfiguration, TaskManagerLocation}
-import org.apache.flink.runtime.util.EnvironmentInformation
+import org.apache.flink.runtime.taskexecutor.{TaskManagerConfiguration, TaskManagerServices, TaskManagerServicesConfiguration}
+import org.apache.flink.runtime.taskmanager.{TaskManager, TaskManagerLocation}
+import org.apache.flink.runtime.util.{EnvironmentInformation, LeaderRetrievalUtils}
 
 import scala.concurrent.Await
 import scala.concurrent.duration.FiniteDuration
@@ -195,31 +197,32 @@ class LocalFlinkMiniCluster(
 
     val resourceID = ResourceID.generate() // generate random resource id
 
-    val (taskManagerConfig,
-    taskManagerLocation,
-    memoryManager,
-    ioManager,
-    network,
-    leaderRetrievalService,
-    metricsRegistry) = TaskManager.createTaskManagerComponents(
+    val taskManagerAddress = InetAddress.getByName(hostname)
+
+    val taskManagerConfiguration = TaskManagerConfiguration.fromConfiguration(config)
+    val taskManagerServicesConfiguration = TaskManagerServicesConfiguration.fromConfiguration(
       config,
-      resourceID,
-      hostname, // network interface to bind to
-      localExecution, // start network stack?
-      Some(createLeaderRetrievalService()))
+      taskManagerAddress,
+      localExecution)
+
+    val taskManagerServices = TaskManagerServices.fromConfiguration(
+      taskManagerServicesConfiguration,
+      resourceID)
+
+    val metricRegistry = taskManagerServices.getMetricRegistry()
 
     val props = getTaskManagerProps(
       taskManagerClass,
-      taskManagerConfig,
+      taskManagerConfiguration,
       resourceID,
-      taskManagerLocation,
-      memoryManager,
-      ioManager,
-      network,
-      leaderRetrievalService,
-      metricsRegistry)
-
-    metricsRegistry.startQueryService(system)
+      taskManagerServices.getTaskManagerLocation(),
+      taskManagerServices.getMemoryManager(),
+      taskManagerServices.getIOManager(),
+      taskManagerServices.getNetworkEnvironment,
+      createLeaderRetrievalService(),
+      metricRegistry)
+
+    metricRegistry.startQueryService(system)
 
     system.actorOf(props, taskManagerActorName)
   }

http://git-wip-us.apache.org/repos/asf/flink/blob/bce292ae/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala
index 79670a4..d16c1b0 100644
--- a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala
+++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala
@@ -37,7 +37,6 @@ import com.fasterxml.jackson.databind.ObjectMapper
 import grizzled.slf4j.Logger
 import org.apache.flink.configuration._
 import org.apache.flink.core.fs.FileSystem
-import org.apache.flink.core.memory.{HeapMemorySegment, HybridMemorySegment, MemorySegmentFactory, MemoryType}
 import org.apache.flink.metrics.{MetricGroup, Gauge => FlinkGauge}
 import org.apache.flink.runtime.accumulators.AccumulatorSnapshot
 import org.apache.flink.runtime.clusterframework.messages.StopCluster
@@ -51,12 +50,8 @@ import org.apache.flink.runtime.execution.librarycache.{BlobLibraryCacheManager,
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID
 import org.apache.flink.runtime.filecache.FileCache
 import org.apache.flink.runtime.instance.{AkkaActorGateway, HardwareDescription, InstanceID}
-import org.apache.flink.runtime.io.disk.iomanager.IOManager.IOMode
-import org.apache.flink.runtime.io.disk.iomanager.{IOManager, IOManagerAsync}
-import org.apache.flink.runtime.io.network.buffer.NetworkBufferPool
-import org.apache.flink.runtime.io.network.{LocalConnectionManager, NetworkEnvironment, TaskEventDispatcher}
-import org.apache.flink.runtime.io.network.netty.{NettyConfig, NettyConnectionManager, PartitionStateChecker}
-import org.apache.flink.runtime.io.network.partition.{ResultPartitionConsumableNotifier, ResultPartitionManager}
+import org.apache.flink.runtime.io.disk.iomanager.IOManager
+import org.apache.flink.runtime.io.network.NetworkEnvironment
 import org.apache.flink.runtime.jobgraph.IntermediateDataSetID
 import org.apache.flink.runtime.leaderretrieval.{LeaderRetrievalListener, LeaderRetrievalService}
 import org.apache.flink.runtime.memory.MemoryManager
@@ -66,16 +61,15 @@ import org.apache.flink.runtime.messages.StackTraceSampleMessages.{ResponseStack
 import org.apache.flink.runtime.messages.TaskManagerMessages._
 import org.apache.flink.runtime.messages.TaskMessages._
 import org.apache.flink.runtime.messages.checkpoint.{AbstractCheckpointMessage, NotifyCheckpointComplete, TriggerCheckpoint}
-import org.apache.flink.runtime.metrics.{MetricRegistryConfiguration, MetricRegistry => FlinkMetricRegistry}
+import org.apache.flink.runtime.metrics.{MetricRegistry => FlinkMetricRegistry}
 import org.apache.flink.runtime.metrics.groups.TaskManagerMetricGroup
 import org.apache.flink.runtime.process.ProcessReaper
-import org.apache.flink.runtime.query.KvStateRegistry
-import org.apache.flink.runtime.query.netty.{DisabledKvStateRequestStats, KvStateServer}
 import org.apache.flink.runtime.security.SecurityContext.{FlinkSecuredRunner, SecurityConfiguration}
 import org.apache.flink.runtime.security.SecurityContext
+import org.apache.flink.runtime.taskexecutor.{TaskManagerConfiguration, TaskManagerServices, TaskManagerServicesConfiguration}
 import org.apache.flink.runtime.util._
 import org.apache.flink.runtime.{FlinkActor, LeaderSessionMessageFilter, LogMessages}
-import org.apache.flink.util.{MathUtils, NetUtils}
+import org.apache.flink.util.NetUtils
 
 import scala.collection.JavaConverters._
 import scala.concurrent._
@@ -142,7 +136,7 @@ class TaskManager(
   override val log = Logger(getClass)
 
   /** The timeout for all actor ask futures */
-  protected val askTimeout = new Timeout(config.timeout)
+  protected val askTimeout = new Timeout(config.getTimeout().getSize, config.getTimeout().getUnit())
 
   /** The TaskManager's physical execution resources */
   protected val resources = HardwareDescription.extractFromSystem(memoryManager.getMemorySize())
@@ -154,7 +148,7 @@ class TaskManager(
   protected val bcVarManager = new BroadcastVariableManager()
 
   /** Handler for distributed files cached by this TaskManager */
-  protected val fileCache = new FileCache(config.configuration)
+  protected val fileCache = new FileCache(config.getConfiguration())
 
   /** Registry of metrics periodically transmitted to the JobManager */
   private val metricRegistry = TaskManager.createMetricsRegistry()
@@ -190,8 +184,8 @@ class TaskManager(
 
   private val runtimeInfo = new TaskManagerRuntimeInfo(
        location.getHostname(),
-       new UnmodifiableConfiguration(config.configuration),
-       config.tmpDirPaths)
+       new UnmodifiableConfiguration(config.getConfiguration()),
+       config.getTmpDirPaths())
 
   private var scheduledTaskManagerRegistration: Option[Cancellable] = None
   private var currentRegistrationRun: UUID = UUID.randomUUID()
@@ -614,7 +608,9 @@ class TaskManager(
             )
 
             // the next timeout computes via exponential backoff with cap
-            val nextTimeout = (timeout * 2).min(config.maxRegistrationPause)
+            val nextTimeout = (timeout * 2).min(new FiniteDuration(
+              config.getMaxRegistrationPause().toMilliseconds,
+              TimeUnit.MILLISECONDS))
 
             // schedule (with our timeout s delay) a check triggers a new registration
             // attempt, if we are not registered by then
@@ -688,10 +684,14 @@ class TaskManager(
 
           if(jobManagerAkkaURL.isDefined) {
             // try the registration again after some time
-            val delay: FiniteDuration = config.refusedRegistrationPause
-            val deadline: Option[Deadline] = config.maxRegistrationDuration.map {
-              timeout => timeout + delay fromNow
-            }
+            val delay: FiniteDuration = new FiniteDuration(
+              config.getRefusedRegistrationPause().getSize(),
+              config.getRefusedRegistrationPause().getUnit())
+            val deadline: Option[Deadline] = Option(config.getMaxRegistrationDuration())
+              .map {
+                duration => new FiniteDuration(duration.getSize(), duration.getUnit()) +
+                  delay fromNow
+              }
 
             // start a new registration run
             currentRegistrationRun = UUID.randomUUID()
@@ -703,7 +703,9 @@ class TaskManager(
                 self ! decorateMessage(
                   TriggerTaskManagerRegistration(
                     jobManagerAkkaURL.get,
-                    config.initialRegistrationPause,
+                    new FiniteDuration(
+                      config.getInitialRegistrationPause().getSize(),
+                      config.getInitialRegistrationPause().getUnit()),
                     deadline,
                     1,
                     currentRegistrationRun)
@@ -842,7 +844,7 @@ class TaskManager(
       requestType: LogTypeRequest,
       jobManager: ActorRef)
     : Unit = {
-    val logFilePathOption = Option(config.configuration.getString(
+    val logFilePathOption = Option(config.getConfiguration().getString(
       ConfigConstants.TASK_MANAGER_LOG_PATH_KEY, System.getProperty("log.file")));
     logFilePathOption match {
       case None => throw new IOException("TaskManager log files are unavailable. " +
@@ -975,9 +977,10 @@ class TaskManager(
       log.info(s"Determined BLOB server address to be $address. Starting BLOB cache.")
 
       try {
-        val blobcache = new BlobCache(address, config.configuration)
+        val blobcache = new BlobCache(address, config.getConfiguration())
         blobService = Option(blobcache)
-        libraryCacheManager = Some(new BlobLibraryCacheManager(blobcache, config.cleanupInterval))
+        libraryCacheManager = Some(
+          new BlobLibraryCacheManager(blobcache, config.getCleanupInterval()))
       }
       catch {
         case e: Exception =>
@@ -1160,7 +1163,9 @@ class TaskManager(
         tdd.getJobID,
         tdd.getVertexID,
         tdd.getExecutionId,
-        config.timeout)
+        new FiniteDuration(
+          config.getTimeout().getSize(),
+          config.getTimeout().getUnit()))
 
       val task = new Task(
         tdd,
@@ -1427,7 +1432,8 @@ class TaskManager(
   def triggerTaskManagerRegistration(): Unit = {
     if(jobManagerAkkaURL.isDefined) {
       // begin attempts to reconnect
-      val deadline: Option[Deadline] = config.maxRegistrationDuration.map(_.fromNow)
+      val deadline: Option[Deadline] = Option(config.getMaxRegistrationDuration())
+        .map{ duration => new FiniteDuration(duration.getSize(), duration.getUnit()).fromNow }
 
       // start a new registration run
       currentRegistrationRun = UUID.randomUUID()
@@ -1437,7 +1443,9 @@ class TaskManager(
       self ! decorateMessage(
         TriggerTaskManagerRegistration(
           jobManagerAkkaURL.get,
-          config.initialRegistrationPause,
+          new FiniteDuration(
+            config.getInitialRegistrationPause().getSize(),
+            config.getInitialRegistrationPause().getUnit()),
           deadline,
           1,
           currentRegistrationRun)
@@ -1844,32 +1852,37 @@ object TaskManager {
       taskManagerClass: Class[_ <: TaskManager])
     : ActorRef = {
 
-    val (taskManagerConfig,
-      connectionInfo,
-      memoryManager,
-      ioManager,
-      network,
-      leaderRetrievalService,
-      metricsRegistry) = createTaskManagerComponents(
-      configuration,
-      resourceID,
-      taskManagerHostname,
-      localTaskManagerCommunication,
-      leaderRetrievalServiceOption)
+    val taskManagerAddress = InetAddress.getByName(taskManagerHostname)
+
+    val taskManagerServicesConfiguration = TaskManagerServicesConfiguration
+      .fromConfiguration(configuration, taskManagerAddress, false)
+
+    val taskManagerConfiguration = TaskManagerConfiguration.fromConfiguration(configuration)
+
+    val taskManagerServices = TaskManagerServices.fromConfiguration(
+      taskManagerServicesConfiguration,
+      resourceID)
+
+    val metricRegistry = taskManagerServices.getMetricRegistry()
+
+    val leaderRetrievalService = leaderRetrievalServiceOption match {
+      case Some(lrs) => lrs
+      case None => LeaderRetrievalUtils.createLeaderRetrievalService(configuration)
+    }
 
     // create the actor properties (which define the actor constructor parameters)
     val tmProps = getTaskManagerProps(
       taskManagerClass,
-      taskManagerConfig,
+      taskManagerConfiguration,
       resourceID,
-      connectionInfo,
-      memoryManager,
-      ioManager,
-      network,
+      taskManagerServices.getTaskManagerLocation(),
+      taskManagerServices.getMemoryManager(),
+      taskManagerServices.getIOManager(),
+      taskManagerServices.getNetworkEnvironment(),
       leaderRetrievalService,
-      metricsRegistry)
+      metricRegistry)
 
-    metricsRegistry.startQueryService(actorSystem)
+    metricRegistry.startQueryService(actorSystem)
 
     taskManagerActorName match {
       case Some(actorName) => actorSystem.actorOf(tmProps, actorName)
@@ -1896,211 +1909,11 @@ object TaskManager {
       memoryManager,
       ioManager,
       networkEnvironment,
-      taskManagerConfig.numberOfSlots,
+      taskManagerConfig.getNumberSlots(),
       leaderRetrievalService,
       metricsRegistry)
   }
 
-  def createTaskManagerComponents(
-    configuration: Configuration,
-    resourceID: ResourceID,
-    taskManagerHostname: String,
-    localTaskManagerCommunication: Boolean,
-    leaderRetrievalServiceOption: Option[LeaderRetrievalService]):
-      (TaskManagerConfiguration,
-      TaskManagerLocation,
-      MemoryManager,
-      IOManager,
-      NetworkEnvironment,
-      LeaderRetrievalService,
-      FlinkMetricRegistry) = {
-
-    val (taskManagerConfig : TaskManagerConfiguration,
-    netConfig: NetworkEnvironmentConfiguration,
-    taskManagerAddress: InetSocketAddress,
-    memType: MemoryType
-      ) = parseTaskManagerConfiguration(
-      configuration,
-      taskManagerHostname,
-      localTaskManagerCommunication)
-
-    // pre-start checks
-    checkTempDirs(taskManagerConfig.tmpDirPaths)
-
-    val networkBufferPool = new NetworkBufferPool(
-      netConfig.numNetworkBuffers,
-      netConfig.networkBufferSize,
-      netConfig.memoryType)
-
-    val connectionManager = Option(netConfig.nettyConfig) match {
-      case Some(nettyConfig) => new NettyConnectionManager(nettyConfig)
-      case None => new LocalConnectionManager()
-    }
-
-    val resultPartitionManager = new ResultPartitionManager()
-    val taskEventDispatcher = new TaskEventDispatcher()
-
-    val kvStateRegistry = new KvStateRegistry()
-
-    val kvStateServer = Option(netConfig.nettyConfig) match {
-      case Some(nettyConfig) =>
-
-        val numNetworkThreads = if (netConfig.queryServerNetworkThreads == 0) {
-          nettyConfig.getNumberOfSlots
-        } else {
-          netConfig.queryServerNetworkThreads
-        }
-
-        val numQueryThreads = if (netConfig.queryServerQueryThreads == 0) {
-          nettyConfig.getNumberOfSlots
-        } else {
-          netConfig.queryServerQueryThreads
-        }
-
-        new KvStateServer(
-          taskManagerAddress.getAddress(),
-          netConfig.queryServerPort,
-          numNetworkThreads,
-          numQueryThreads,
-          kvStateRegistry,
-          new DisabledKvStateRequestStats())
-
-      case None => null
-    }
-
-    // we start the network first, to make sure it can allocate its buffers first
-    val network = new NetworkEnvironment(
-      networkBufferPool,
-      connectionManager,
-      resultPartitionManager,
-      taskEventDispatcher,
-      kvStateRegistry,
-      kvStateServer,
-      netConfig.ioMode,
-      netConfig.partitionRequestInitialBackoff,
-      netConfig.partitinRequestMaxBackoff)
-
-    network.start()
-
-    val taskManagerLocation = new TaskManagerLocation(
-      resourceID,
-      taskManagerAddress.getAddress(),
-      network.getConnectionManager().getDataPort())
-
-    // computing the amount of memory to use depends on how much memory is available
-    // it strictly needs to happen AFTER the network stack has been initialized
-
-    // check if a value has been configured
-    val configuredMemory = configuration.getLong(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, -1L)
-    checkConfigParameter(configuredMemory == -1 || configuredMemory > 0, configuredMemory,
-                         ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY,
-                         "MemoryManager needs at least one MB of memory. " +
-                           "If you leave this config parameter empty, the system automatically " +
-                           "pick a fraction of the available memory.")
-
-
-    val preAllocateMemory = configuration.getBoolean(
-      ConfigConstants.TASK_MANAGER_MEMORY_PRE_ALLOCATE_KEY,
-      ConfigConstants.DEFAULT_TASK_MANAGER_MEMORY_PRE_ALLOCATE)
-
-    val memorySize = if (configuredMemory > 0) {
-      if (preAllocateMemory) {
-        LOG.info(s"Using $configuredMemory MB for managed memory.")
-      } else {
-        LOG.info(s"Limiting managed memory to $configuredMemory MB, " +
-                   s"memory will be allocated lazily.")
-      }
-      configuredMemory << 20 // megabytes to bytes
-    }
-    else {
-      val fraction = configuration.getFloat(
-        ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY,
-        ConfigConstants.DEFAULT_MEMORY_MANAGER_MEMORY_FRACTION)
-      checkConfigParameter(fraction > 0.0f && fraction < 1.0f, fraction,
-                           ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY,
-                           "MemoryManager fraction of the free memory must be between 0.0 and 1.0")
-
-      if (memType == MemoryType.HEAP) {
-        val relativeMemSize = (EnvironmentInformation.getSizeOfFreeHeapMemoryWithDefrag() *
-          fraction).toLong
-
-        if (preAllocateMemory) {
-          LOG.info(s"Using $fraction of the currently free heap space for managed " +
-                     s"heap memory (${relativeMemSize >> 20} MB).")
-        } else {
-          LOG.info(s"Limiting managed memory to $fraction of the currently free heap space " +
-                     s"(${relativeMemSize >> 20} MB), memory will be allocated lazily.")
-        }
-
-        relativeMemSize
-      }
-      else if (memType == MemoryType.OFF_HEAP) {
-
-        // The maximum heap memory has been adjusted according to the fraction
-        val maxMemory = EnvironmentInformation.getMaxJvmHeapMemory()
-        val directMemorySize = (maxMemory / (1.0 - fraction) * fraction).toLong
-
-        if (preAllocateMemory) {
-          LOG.info(s"Using $fraction of the maximum memory size for " +
-                     s"managed off-heap memory (${directMemorySize >> 20} MB).")
-        } else {
-          LOG.info(s"Limiting managed memory to $fraction of the maximum memory size " +
-                     s"(${directMemorySize >> 20} MB), memory will be allocated lazily.")
-        }
-
-        directMemorySize
-      }
-      else {
-        throw new RuntimeException("No supported memory type detected.")
-      }
-    }
-
-    // now start the memory manager
-    val memoryManager = try {
-      new MemoryManager(
-        memorySize,
-        taskManagerConfig.numberOfSlots,
-        netConfig.networkBufferSize,
-        memType,
-        preAllocateMemory)
-    }
-    catch {
-      case e: OutOfMemoryError =>
-        memType match {
-          case MemoryType.HEAP =>
-            throw new Exception(s"OutOfMemory error (${e.getMessage()})" +
-                      s" while allocating the TaskManager heap memory ($memorySize bytes).", e)
-
-          case MemoryType.OFF_HEAP =>
-            throw new Exception(s"OutOfMemory error (${e.getMessage()})" +
-                      s" while allocating the TaskManager off-heap memory ($memorySize bytes). " +
-                      s"Try increasing the maximum direct memory (-XX:MaxDirectMemorySize)", e)
-
-          case _ => throw e
-        }
-    }
-
-    // start the I/O manager last, it will create some temp directories.
-    val ioManager: IOManager = new IOManagerAsync(taskManagerConfig.tmpDirPaths)
-
-    val leaderRetrievalService = leaderRetrievalServiceOption match {
-      case Some(lrs) => lrs
-      case None => LeaderRetrievalUtils.createLeaderRetrievalService(configuration)
-    }
-
-    val metricsRegistry = new FlinkMetricRegistry(
-      MetricRegistryConfiguration.fromConfiguration(configuration))
-
-    (taskManagerConfig,
-      taskManagerLocation,
-      memoryManager,
-      ioManager,
-      network,
-      leaderRetrievalService,
-      metricsRegistry)
-  }
-
-
   // --------------------------------------------------------------------------
   //  Resolving the TaskManager actor
   // --------------------------------------------------------------------------
@@ -2140,239 +1953,6 @@ object TaskManager {
   // --------------------------------------------------------------------------
 
   /**
-   * Utility method to extract TaskManager config parameters from the configuration and to
-   * sanity check them.
-   *
-   * @param configuration The configuration.
-   * @param taskManagerHostname The host name under which the TaskManager communicates.
-   * @param localTaskManagerCommunication True, to skip initializing the network stack.
-   *                                      Use only in cases where only one task manager runs.
-   * @return A tuple (TaskManagerConfiguration, network configuration, inet socket address,
-    *         memory tyep).
-   */
-  @throws(classOf[IllegalArgumentException])
-  def parseTaskManagerConfiguration(
-      configuration: Configuration,
-      taskManagerHostname: String,
-      localTaskManagerCommunication: Boolean)
-    : (TaskManagerConfiguration,
-     NetworkEnvironmentConfiguration,
-     InetSocketAddress,
-     MemoryType) = {
-
-    // ------- read values from the config and check them ---------
-    //                      (a lot of them)
-
-    // ----> hosts / ports for communication and data exchange
-
-    val dataport = configuration.getInteger(ConfigConstants.TASK_MANAGER_DATA_PORT_KEY,
-      ConfigConstants.DEFAULT_TASK_MANAGER_DATA_PORT)
-
-    checkConfigParameter(dataport >= 0, dataport, ConfigConstants.TASK_MANAGER_DATA_PORT_KEY,
-      "Leave config parameter empty or use 0 to let the system choose a port automatically.")
-
-    val taskManagerAddress = InetAddress.getByName(taskManagerHostname)
-    val taskManagerInetSocketAddress = new InetSocketAddress(taskManagerAddress, dataport)
-
-    // ----> memory / network stack (shuffles/broadcasts), task slots, temp directories
-
-    // we need this because many configs have been written with a "-1" entry
-    val slots = configuration.getInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 1) match {
-      case -1 => 1
-      case x => x
-    }
-
-    checkConfigParameter(slots >= 1, slots, ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS,
-      "Number of task slots must be at least one.")
-
-    val numNetworkBuffers = configuration.getInteger(
-      ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY,
-      ConfigConstants.DEFAULT_TASK_MANAGER_NETWORK_NUM_BUFFERS)
-
-    checkConfigParameter(numNetworkBuffers > 0, numNetworkBuffers,
-      ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY)
-    
-    val pageSize: Int = configuration.getInteger(
-      ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY,
-      ConfigConstants.DEFAULT_TASK_MANAGER_MEMORY_SEGMENT_SIZE)
-
-    // check page size of for minimum size
-    checkConfigParameter(pageSize >= MemoryManager.MIN_PAGE_SIZE, pageSize,
-      ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY,
-      "Minimum memory segment size is " + MemoryManager.MIN_PAGE_SIZE)
-
-    // check page size for power of two
-    checkConfigParameter(MathUtils.isPowerOf2(pageSize), pageSize,
-      ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY,
-      "Memory segment size must be a power of 2.")
-    
-    // check whether we use heap or off-heap memory
-    val memType: MemoryType = 
-      if (configuration.getBoolean(ConfigConstants.TASK_MANAGER_MEMORY_OFF_HEAP_KEY, false)) {
-        MemoryType.OFF_HEAP
-      } else {
-        MemoryType.HEAP
-      }
-    
-    // initialize the memory segment factory accordingly
-    memType match {
-      case MemoryType.HEAP =>
-        if (!MemorySegmentFactory.initializeIfNotInitialized(HeapMemorySegment.FACTORY)) {
-          throw new Exception("Memory type is set to heap memory, but memory segment " +
-            "factory has been initialized for off-heap memory segments")
-        }
-
-      case MemoryType.OFF_HEAP =>
-        if (!MemorySegmentFactory.initializeIfNotInitialized(HybridMemorySegment.FACTORY)) {
-          throw new Exception("Memory type is set to off-heap memory, but memory segment " +
-            "factory has been initialized for heap memory segments")
-        }
-    }
-    
-    val tmpDirs = configuration.getString(
-      ConfigConstants.TASK_MANAGER_TMP_DIR_KEY,
-      ConfigConstants.DEFAULT_TASK_MANAGER_TMP_PATH)
-    .split(",|" + File.pathSeparator)
-
-    val nettyConfig = if (localTaskManagerCommunication) {
-      None
-    } else {
-      Some(
-        new NettyConfig(
-          taskManagerInetSocketAddress.getAddress(),
-          taskManagerInetSocketAddress.getPort(),
-          pageSize,
-          slots,
-          configuration)
-      )
-    }
-
-    // Default spill I/O mode for intermediate results
-    val syncOrAsync = configuration.getString(
-      ConfigConstants.TASK_MANAGER_NETWORK_DEFAULT_IO_MODE,
-      ConfigConstants.DEFAULT_TASK_MANAGER_NETWORK_DEFAULT_IO_MODE)
-
-    val ioMode : IOMode = if (syncOrAsync == "async") IOMode.ASYNC else IOMode.SYNC
-
-    val queryServerPort =  configuration.getInteger(
-      ConfigConstants.QUERYABLE_STATE_SERVER_PORT,
-      ConfigConstants.DEFAULT_QUERYABLE_STATE_SERVER_PORT)
-
-    val queryServerNetworkThreads =  configuration.getInteger(
-      ConfigConstants.QUERYABLE_STATE_SERVER_NETWORK_THREADS,
-      ConfigConstants.DEFAULT_QUERYABLE_STATE_SERVER_NETWORK_THREADS)
-
-    val queryServerQueryThreads =  configuration.getInteger(
-      ConfigConstants.QUERYABLE_STATE_SERVER_QUERY_THREADS,
-      ConfigConstants.DEFAULT_QUERYABLE_STATE_SERVER_QUERY_THREADS)
-
-    val networkConfig = NetworkEnvironmentConfiguration(
-      numNetworkBuffers,
-      pageSize,
-      memType,
-      ioMode,
-      queryServerPort,
-      queryServerNetworkThreads,
-      queryServerQueryThreads,
-      nettyConfig.getOrElse(null))
-
-    // ----> timeouts, library caching, profiling
-
-    val timeout = try {
-      AkkaUtils.getTimeout(configuration)
-    } catch {
-      case e: Exception => throw new IllegalArgumentException(
-        s"Invalid format for '${ConfigConstants.AKKA_ASK_TIMEOUT}'. " +
-          s"Use formats like '50 s' or '1 min' to specify the timeout.")
-    }
-    LOG.info("Messages between TaskManager and JobManager have a max timeout of " + timeout)
-
-    val cleanupInterval = configuration.getLong(
-      ConfigConstants.LIBRARY_CACHE_MANAGER_CLEANUP_INTERVAL,
-      ConfigConstants.DEFAULT_LIBRARY_CACHE_MANAGER_CLEANUP_INTERVAL) * 1000
-
-    val finiteRegistrationDuration = try {
-      val maxRegistrationDuration = Duration(configuration.getString(
-        ConfigConstants.TASK_MANAGER_MAX_REGISTRATION_DURATION,
-        ConfigConstants.DEFAULT_TASK_MANAGER_MAX_REGISTRATION_DURATION))
-
-      if (maxRegistrationDuration.isFinite()) {
-        Some(maxRegistrationDuration.asInstanceOf[FiniteDuration])
-      } else {
-        None
-      }
-    } catch {
-      case e: NumberFormatException => throw new IllegalArgumentException(
-        "Invalid format for parameter " + ConfigConstants.TASK_MANAGER_MAX_REGISTRATION_DURATION,
-        e)
-    }
-
-    val initialRegistrationPause = try {
-      val pause = Duration(configuration.getString(
-        ConfigConstants.TASK_MANAGER_INITIAL_REGISTRATION_PAUSE,
-        ConfigConstants.DEFAULT_TASK_MANAGER_INITIAL_REGISTRATION_PAUSE
-      ))
-
-      if (pause.isFinite()) {
-        pause.asInstanceOf[FiniteDuration]
-      } else {
-        throw new IllegalArgumentException(s"The initial registration pause must be finite: $pause")
-      }
-    } catch {
-      case e: NumberFormatException => throw new IllegalArgumentException(
-        "Invalid format for parameter " + ConfigConstants.TASK_MANAGER_INITIAL_REGISTRATION_PAUSE,
-        e)
-    }
-
-    val maxRegistrationPause = try {
-      val pause = Duration(configuration.getString(
-        ConfigConstants.TASK_MANAGER_MAX_REGISTARTION_PAUSE,
-        ConfigConstants.DEFAULT_TASK_MANAGER_MAX_REGISTRATION_PAUSE
-      ))
-
-      if (pause.isFinite()) {
-        pause.asInstanceOf[FiniteDuration]
-      } else {
-        throw new IllegalArgumentException(s"The maximum registration pause must be finite: $pause")
-      }
-    } catch {
-      case e: NumberFormatException => throw new IllegalArgumentException(
-        "Invalid format for parameter " + ConfigConstants.TASK_MANAGER_INITIAL_REGISTRATION_PAUSE,
-        e)
-    }
-
-    val refusedRegistrationPause = try {
-      val pause = Duration(configuration.getString(
-        ConfigConstants.TASK_MANAGER_REFUSED_REGISTRATION_PAUSE,
-        ConfigConstants.DEFAULT_TASK_MANAGER_REFUSED_REGISTRATION_PAUSE
-      ))
-
-      if (pause.isFinite()) {
-        pause.asInstanceOf[FiniteDuration]
-      } else {
-        throw new IllegalArgumentException(s"The refused registration pause must be finite: $pause")
-      }
-    } catch {
-      case e: NumberFormatException => throw new IllegalArgumentException(
-        "Invalid format for parameter " + ConfigConstants.TASK_MANAGER_INITIAL_REGISTRATION_PAUSE,
-        e)
-    }
-
-    val taskManagerConfig = TaskManagerConfiguration(
-      tmpDirs,
-      cleanupInterval,
-      timeout,
-      finiteRegistrationDuration,
-      slots,
-      configuration,
-      initialRegistrationPause,
-      maxRegistrationPause,
-      refusedRegistrationPause)
-
-    (taskManagerConfig, networkConfig, taskManagerInetSocketAddress, memType)
-  }
-
-  /**
    * Gets the hostname and port of the JobManager from the configuration. Also checks that
    * the hostname is not null and the port non-negative.
    *
@@ -2406,71 +1986,6 @@ object TaskManager {
   // --------------------------------------------------------------------------
 
   /**
-   * Validates a condition for a config parameter and displays a standard exception, if the
-   * the condition does not hold.
-   *
-   * @param condition The condition that must hold. If the condition is false, an
-   *                  exception is thrown.
-   * @param parameter The parameter value. Will be shown in the exception message.
-   * @param name The name of the config parameter. Will be shown in the exception message.
-   * @param errorMessage The optional custom error message to append to the exception message.
-   * @throws IllegalConfigurationException Thrown if the condition is violated.
-   */
-  @throws(classOf[IllegalConfigurationException])
-  private def checkConfigParameter(
-      condition: Boolean,
-      parameter: Any,
-      name: String,
-      errorMessage: String = "")
-    : Unit = {
-    if (!condition) {
-      throw new IllegalConfigurationException(
-        s"Invalid configuration value for '$name' : $parameter - $errorMessage")
-    }
-  }
-
-  /**
-   * Validates that all the directories denoted by the strings do actually exist, are proper
-   * directories (not files), and are writable.
-   *
-   * @param tmpDirs The array of directory paths to check.
-   * @throws Exception Thrown if any of the directories does not exist or is not writable
-   *                   or is a file, rather than a directory.
-   */
-  @throws(classOf[IOException])
-  private def checkTempDirs(tmpDirs: Array[String]): Unit = {
-    tmpDirs.zipWithIndex.foreach {
-      case (dir: String, _) =>
-        val file = new File(dir)
-
-        if (!file.exists) {
-          throw new IOException(
-            s"Temporary file directory ${file.getAbsolutePath} does not exist.")
-        }
-        if (!file.isDirectory) {
-          throw new IOException(
-            s"Temporary file directory ${file.getAbsolutePath} is not a directory.")
-        }
-        if (!file.canWrite) {
-          throw new IOException(
-            s"Temporary file directory ${file.getAbsolutePath} is not writable.")
-        }
-
-        if (LOG.isInfoEnabled) {
-          val totalSpaceGb = file.getTotalSpace >>  30
-          val usableSpaceGb = file.getUsableSpace >> 30
-          val usablePercentage = usableSpaceGb.asInstanceOf[Double] / totalSpaceGb * 100
-
-          val path = file.getAbsolutePath
-
-          LOG.info(f"Temporary file directory '$path': total $totalSpaceGb GB, " +
-            f"usable $usableSpaceGb GB ($usablePercentage%.2f%% usable)")
-        }
-      case (_, id) => throw new IllegalArgumentException(s"Temporary file directory #$id is null.")
-    }
-  }
-
-  /**
    * Creates the registry of default metrics, including stats about garbage collection, memory
    * usage, and system CPU load.
    *

http://git-wip-us.apache.org/repos/asf/flink/blob/bce292ae/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManagerConfiguration.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManagerConfiguration.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManagerConfiguration.scala
deleted file mode 100644
index aab3c5f..0000000
--- a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManagerConfiguration.scala
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.runtime.taskmanager
-
-import java.util.concurrent.TimeUnit
-
-import org.apache.flink.configuration.Configuration
-
-import scala.concurrent.duration.FiniteDuration
-
-case class TaskManagerConfiguration(
-    tmpDirPaths: Array[String],
-    cleanupInterval: Long,
-    timeout: FiniteDuration,
-    maxRegistrationDuration: Option[FiniteDuration],
-    numberOfSlots: Int,
-    configuration: Configuration,
-    initialRegistrationPause: FiniteDuration,
-    maxRegistrationPause: FiniteDuration,
-    refusedRegistrationPause: FiniteDuration) {
-
-  def this(
-      tmpDirPaths: Array[String],
-      cleanupInterval: Long,
-      timeout: FiniteDuration,
-      maxRegistrationDuration: Option[FiniteDuration],
-      numberOfSlots: Int,
-      configuration: Configuration) {
-    this (
-      tmpDirPaths,
-      cleanupInterval,
-      timeout,
-      maxRegistrationDuration,
-      numberOfSlots,
-      configuration,
-      FiniteDuration(500, TimeUnit.MILLISECONDS),
-      FiniteDuration(30, TimeUnit.SECONDS),
-      FiniteDuration(10, TimeUnit.SECONDS))
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/bce292ae/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerComponentsStartupShutdownTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerComponentsStartupShutdownTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerComponentsStartupShutdownTest.java
index 627a25a..500d1bd 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerComponentsStartupShutdownTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerComponentsStartupShutdownTest.java
@@ -26,6 +26,7 @@ import akka.actor.Kill;
 import akka.actor.Props;
 import akka.testkit.JavaTestKit;
 
+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.memory.MemoryType;
@@ -49,11 +50,11 @@ import org.apache.flink.runtime.messages.TaskManagerMessages;
 import org.apache.flink.runtime.metrics.MetricRegistry;
 import org.apache.flink.runtime.metrics.MetricRegistryConfiguration;
 import org.apache.flink.runtime.query.KvStateRegistry;
+import org.apache.flink.runtime.taskexecutor.TaskManagerConfiguration;
 import org.apache.flink.runtime.util.LeaderRetrievalUtils;
 
 import org.junit.Test;
 
-import scala.Option;
 import scala.concurrent.duration.FiniteDuration;
 
 import java.net.InetAddress;
@@ -69,7 +70,7 @@ public class TaskManagerComponentsStartupShutdownTest {
 	public void testComponentsStartupShutdown() {
 
 		final String[] TMP_DIR = new String[] { ConfigConstants.DEFAULT_TASK_MANAGER_TMP_PATH };
-		final FiniteDuration timeout = new FiniteDuration(100, TimeUnit.SECONDS);
+		final Time timeout = Time.seconds(100);
 		final int BUFFER_SIZE = 32 * 1024;
 
 		Configuration config = new Configuration();
@@ -93,14 +94,19 @@ public class TaskManagerComponentsStartupShutdownTest {
 				LeaderRetrievalUtils.createLeaderRetrievalService(config, jobManager),
 				StandaloneResourceManager.class);
 
+			final int numberOfSlots = 1;
+
 			// create the components for the TaskManager manually
 			final TaskManagerConfiguration tmConfig = new TaskManagerConfiguration(
-					TMP_DIR,
-					1000000,
-					timeout,
-					Option.<FiniteDuration>empty(),
-					1,
-					config);
+				numberOfSlots,
+				TMP_DIR,
+				timeout,
+				null,
+				Time.milliseconds(500),
+				Time.seconds(30),
+				Time.seconds(10),
+				1000000, // cleanup interval
+				config);
 
 			final NetworkEnvironmentConfiguration netConf = new NetworkEnvironmentConfiguration(
 					32, BUFFER_SIZE, MemoryType.HEAP, IOManager.IOMode.SYNC, 0, 0, 0,
@@ -125,8 +131,6 @@ public class TaskManagerComponentsStartupShutdownTest {
 
 			network.start();
 
-			final int numberOfSlots = 1;
-
 			LeaderRetrievalService leaderRetrievalService = new StandaloneLeaderRetrievalService(jobManager.path().toString());
 
 			MetricRegistryConfiguration metricRegistryConfiguration = MetricRegistryConfiguration.fromConfiguration(config);

http://git-wip-us.apache.org/repos/asf/flink/blob/bce292ae/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingTaskManager.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingTaskManager.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingTaskManager.scala
index 707401b..09dc5ed 100644
--- a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingTaskManager.scala
+++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingTaskManager.scala
@@ -24,7 +24,8 @@ import org.apache.flink.runtime.io.network.NetworkEnvironment
 import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService
 import org.apache.flink.runtime.memory.MemoryManager
 import org.apache.flink.runtime.metrics.MetricRegistry
-import org.apache.flink.runtime.taskmanager.{TaskManagerLocation, TaskManager, TaskManagerConfiguration}
+import org.apache.flink.runtime.taskexecutor.TaskManagerConfiguration
+import org.apache.flink.runtime.taskmanager.{TaskManager, TaskManagerLocation}
 
 import scala.language.postfixOps
 

http://git-wip-us.apache.org/repos/asf/flink/blob/bce292ae/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingUtils.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingUtils.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingUtils.scala
index 0abdd46..d311bc5 100644
--- a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingUtils.scala
+++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingUtils.scala
@@ -241,7 +241,6 @@ object TestingUtils {
     )
   }
 
-
   def createTaskManager(
       actorSystem: ActorSystem,
       jobManagerURL: String,

http://git-wip-us.apache.org/repos/asf/flink/blob/bce292ae/flink-yarn-tests/src/test/scala/org/apache/flink/yarn/TestingYarnTaskManager.scala
----------------------------------------------------------------------
diff --git a/flink-yarn-tests/src/test/scala/org/apache/flink/yarn/TestingYarnTaskManager.scala b/flink-yarn-tests/src/test/scala/org/apache/flink/yarn/TestingYarnTaskManager.scala
index 1010432..0f82faa 100644
--- a/flink-yarn-tests/src/test/scala/org/apache/flink/yarn/TestingYarnTaskManager.scala
+++ b/flink-yarn-tests/src/test/scala/org/apache/flink/yarn/TestingYarnTaskManager.scala
@@ -24,7 +24,8 @@ import org.apache.flink.runtime.io.network.NetworkEnvironment
 import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService
 import org.apache.flink.runtime.memory.MemoryManager
 import org.apache.flink.runtime.metrics.MetricRegistry
-import org.apache.flink.runtime.taskmanager.{TaskManagerConfiguration, TaskManagerLocation}
+import org.apache.flink.runtime.taskexecutor.TaskManagerConfiguration
+import org.apache.flink.runtime.taskmanager.TaskManagerLocation
 import org.apache.flink.runtime.testingUtils.TestingTaskManagerLike
 
 /** [[YarnTaskManager]] implementation which mixes in the [[TestingTaskManagerLike]] mixin.

http://git-wip-us.apache.org/repos/asf/flink/blob/bce292ae/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnTaskManager.scala
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnTaskManager.scala b/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnTaskManager.scala
index 2ab9b20..be31085 100644
--- a/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnTaskManager.scala
+++ b/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnTaskManager.scala
@@ -23,8 +23,9 @@ import org.apache.flink.runtime.io.disk.iomanager.IOManager
 import org.apache.flink.runtime.io.network.NetworkEnvironment
 import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService
 import org.apache.flink.runtime.memory.MemoryManager
-import org.apache.flink.runtime.taskmanager.{TaskManager, TaskManagerConfiguration, TaskManagerLocation}
+import org.apache.flink.runtime.taskmanager.{TaskManager, TaskManagerLocation}
 import org.apache.flink.runtime.metrics.MetricRegistry
+import org.apache.flink.runtime.taskexecutor.TaskManagerConfiguration
 
 /** An extension of the TaskManager that listens for additional YARN related
   * messages.


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

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


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

Branch: refs/heads/flip-6
Commit: 041dfd78a70a45a2b697029c8c1e914050ffee91
Parents: cef3191
Author: Kurt Young <yk...@gmail.com>
Authored: Tue Oct 4 23:00:22 2016 +0800
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Oct 14 15:14:43 2016 +0200

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


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

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

http://git-wip-us.apache.org/repos/asf/flink/blob/041dfd78/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/message/ClassloadingProps.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/message/ClassloadingProps.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/message/ClassloadingProps.java
new file mode 100644
index 0000000..2d670b4
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/message/ClassloadingProps.java
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.jobmaster.message;
+
+import org.apache.flink.runtime.blob.BlobKey;
+
+import java.io.Serializable;
+import java.net.URL;
+import java.util.List;
+
+/**
+ * The response of classloading props request to JobManager.
+ */
+public class ClassloadingProps implements Serializable {
+
+	private static final long serialVersionUID = -3282341310808511823L;
+
+	private final int blobManagerPort;
+
+	private final List<BlobKey> requiredJarFiles;
+
+	private final List<URL> requiredClasspaths;
+
+	/**
+	 * Constructor of ClassloadingProps.
+	 *
+	 * @param blobManagerPort    The port of the blobManager
+	 * @param requiredJarFiles   The blob keys of the required jar files
+	 * @param requiredClasspaths The urls of the required classpaths
+	 */
+	public ClassloadingProps(
+		final int blobManagerPort,
+		final List<BlobKey> requiredJarFiles,
+		final List<URL> requiredClasspaths)
+	{
+		this.blobManagerPort = blobManagerPort;
+		this.requiredJarFiles = requiredJarFiles;
+		this.requiredClasspaths = requiredClasspaths;
+	}
+
+	public int getBlobManagerPort() {
+		return blobManagerPort;
+	}
+
+	public List<BlobKey> getRequiredJarFiles() {
+		return requiredJarFiles;
+	}
+
+	public List<URL> getRequiredClasspaths() {
+		return requiredClasspaths;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/041dfd78/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/message/DisposeSavepointResponse.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/message/DisposeSavepointResponse.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/message/DisposeSavepointResponse.java
new file mode 100644
index 0000000..42bfc71
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/message/DisposeSavepointResponse.java
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.jobmaster.message;
+
+import java.io.Serializable;
+
+/**
+ * The response of the dispose savepoint request to JobManager.
+ */
+public abstract class DisposeSavepointResponse implements Serializable {
+
+	private static final long serialVersionUID = 6008792963949369567L;
+
+	public static class Success extends DisposeSavepointResponse implements Serializable {
+
+		private static final long serialVersionUID = 1572462960008711415L;
+	}
+
+	public static class Failure extends DisposeSavepointResponse implements Serializable {
+
+		private static final long serialVersionUID = -7505308325483022458L;
+
+		private final Throwable cause;
+
+		public Failure(final Throwable cause) {
+			this.cause = cause;
+		}
+
+		public Throwable getCause() {
+			return cause;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/041dfd78/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/message/TriggerSavepointResponse.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/message/TriggerSavepointResponse.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/message/TriggerSavepointResponse.java
new file mode 100644
index 0000000..0b0edc5
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/message/TriggerSavepointResponse.java
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.jobmaster.message;
+
+import org.apache.flink.api.common.JobID;
+
+import java.io.Serializable;
+
+/**
+ * The response of the trigger savepoint request to JobManager.
+ */
+public abstract class TriggerSavepointResponse implements Serializable {
+
+	private static final long serialVersionUID = 3139327824611807707L;
+
+	private final JobID jobID;
+
+	public JobID getJobID() {
+		return jobID;
+	}
+
+	public TriggerSavepointResponse(final JobID jobID) {
+		this.jobID = jobID;
+	}
+
+	public static class Success extends TriggerSavepointResponse implements Serializable {
+
+		private static final long serialVersionUID = -1100637460388881776L;
+
+		private final String savepointPath;
+
+		public Success(final JobID jobID, final String savepointPath) {
+			super(jobID);
+			this.savepointPath = savepointPath;
+		}
+
+		public String getSavepointPath() {
+			return savepointPath;
+		}
+	}
+
+	public static class Failure extends TriggerSavepointResponse implements Serializable {
+
+		private static final long serialVersionUID = -1668479003490615139L;
+
+		private final Throwable cause;
+
+		public Failure(final JobID jobID, final Throwable cause) {
+			super(jobID);
+			this.cause = cause;
+		}
+
+		public Throwable getCause() {
+			return cause;
+		}
+	}
+}
+


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

http://git-wip-us.apache.org/repos/asf/flink/blob/0615b62f/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 246c11d..9669da0 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
@@ -51,6 +51,6 @@ public class RpcCheckpointResponder implements CheckpointResponder {
 
 	@Override
 	public void declineCheckpoint(JobID jobID, ExecutionAttemptID executionAttemptID, CheckpointMetaData checkpoint) {
-		checkpointCoordinatorGateway.declineCheckpoint(jobID, executionAttemptID, checkpoint);
+		checkpointCoordinatorGateway.declineCheckpoint(jobID, executionAttemptID, checkpoint.getCheckpointId());
 	}
 }

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

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

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

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

http://git-wip-us.apache.org/repos/asf/flink/blob/0615b62f/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 
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/0615b62f/flink-runtime/src/test/java/org/apache/flink/runtime/operators/DataSinkTaskTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/DataSinkTaskTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/DataSinkTaskTest.java
index a41c25b..685440b 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/DataSinkTaskTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/DataSinkTaskTest.java
@@ -58,7 +58,7 @@ import static org.junit.Assert.assertFalse;
 
 @RunWith(PowerMockRunner.class)
 @PrepareForTest({Task.class, ResultPartitionWriter.class})
-@PowerMockIgnore({"javax.management.*", "com.sun.jndi.*"})
+@PowerMockIgnore({"javax.management.*", "com.sun.jndi.*", "org.apache.log4j.*"})
 public class DataSinkTaskTest extends TaskTestBase {
 	
 	private static final Logger LOG = LoggerFactory.getLogger(DataSinkTaskTest.class);


[32/50] [abbrv] flink git commit: [FLINK-4606] integrate features of old ResourceManager

Posted by se...@apache.org.
[FLINK-4606] integrate features of old ResourceManager

This closes #2540


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

Branch: refs/heads/flip-6
Commit: 1f198d8ca56ec6719d112cdc7180aeef6d18477a
Parents: 6e58ebf
Author: Maximilian Michels <mx...@apache.org>
Authored: Tue Sep 27 10:38:02 2016 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Oct 14 15:14:42 2016 +0200

----------------------------------------------------------------------
 .../InfoMessageListenerRpcGateway.java          |   1 -
 .../resourcemanager/ResourceManager.java        | 146 ++++++++++++-------
 .../resourcemanager/ResourceManagerGateway.java |   6 +-
 .../ResourceManagerServices.java                |  44 ++++++
 .../StandaloneResourceManager.java              |  19 ++-
 .../TaskExecutorRegistration.java               |  51 -------
 .../registration/TaskExecutorRegistration.java  |  51 +++++++
 .../slotmanager/SimpleSlotManager.java          |   6 -
 .../slotmanager/SlotManager.java                |  63 ++++++--
 .../slotmanager/SlotManagerTest.java            |  25 +++-
 .../slotmanager/SlotProtocolTest.java           |  42 +++---
 11 files changed, 295 insertions(+), 159 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/1f198d8c/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/InfoMessageListenerRpcGateway.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/InfoMessageListenerRpcGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/InfoMessageListenerRpcGateway.java
index c1eeefa..d1373ec 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/InfoMessageListenerRpcGateway.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/InfoMessageListenerRpcGateway.java
@@ -19,7 +19,6 @@
 package org.apache.flink.runtime.resourcemanager;
 
 import org.apache.flink.runtime.clusterframework.messages.InfoMessage;
-import org.apache.flink.runtime.jobmaster.JobMaster;
 import org.apache.flink.runtime.rpc.RpcGateway;
 
 /**

http://git-wip-us.apache.org/repos/asf/flink/blob/1f198d8c/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 83dc4db..190a4de 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,14 +20,18 @@ 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;
+import org.apache.flink.runtime.clusterframework.types.ResourceIDRetrievable;
+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.highavailability.HighAvailabilityServices;
+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;
@@ -48,11 +52,10 @@ import org.apache.flink.runtime.util.LeaderRetrievalUtils;
 import scala.concurrent.duration.FiniteDuration;
 
 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.TimeUnit;
 
 import static org.apache.flink.util.Preconditions.checkNotNull;
@@ -64,36 +67,43 @@ import static org.apache.flink.util.Preconditions.checkNotNull;
  * It offers the following methods as part of its rpc interface to interact with the him remotely:
  * <ul>
  *     <li>{@link #registerJobMaster(UUID, UUID, String, JobID)} registers a {@link JobMaster} at the resource manager</li>
- *     <li>{@link #requestSlot(SlotRequest)} requests a slot from the resource manager</li>
+ *     <li>{@link #requestSlot(UUID, UUID, SlotRequest)} requests a slot from the resource manager</li>
  * </ul>
  */
-public abstract class ResourceManager<ResourceManagerGateway, WorkerType extends TaskExecutorRegistration> extends RpcEndpoint implements LeaderContender {
+public abstract class ResourceManager<WorkerType extends ResourceIDRetrievable>
+		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;
 
 	private final Map<JobID, JobMasterGateway> jobMasterGateways;
 
-	private final Set<LeaderRetrievalListener> jobMasterLeaderRetrievalListeners;
+	private final Map<JobID, JobMasterLeaderListener> jobMasterLeaderRetrievalListeners;
 
 	private final Map<ResourceID, WorkerType> taskExecutorGateways;
 
 	private final HighAvailabilityServices highAvailabilityServices;
 
-	private LeaderElectionService leaderElectionService;
-
 	private final SlotManager slotManager;
 
+	private LeaderElectionService leaderElectionService;
+
 	private UUID leaderSessionID;
 
 	private Map<String, InfoMessageListenerRpcGateway> infoMessageListeners;
 
-	public ResourceManager(RpcService rpcService, HighAvailabilityServices highAvailabilityServices, SlotManager slotManager) {
+	private final Time timeout = Time.seconds(5);
+
+	public ResourceManager(
+			RpcService rpcService,
+			HighAvailabilityServices highAvailabilityServices,
+			SlotManager slotManager) {
 		super(rpcService);
 		this.highAvailabilityServices = checkNotNull(highAvailabilityServices);
 		this.jobMasterGateways = new HashMap<>();
 		this.slotManager = checkNotNull(slotManager);
-		this.jobMasterLeaderRetrievalListeners = new HashSet<>();
+		this.jobMasterLeaderRetrievalListeners = new HashMap<>();
 		this.taskExecutorGateways = new HashMap<>();
 		infoMessageListeners = new HashMap<>();
 	}
@@ -105,6 +115,7 @@ public abstract class ResourceManager<ResourceManagerGateway, WorkerType extends
 			super.start();
 			leaderElectionService = highAvailabilityServices.getResourceManagerLeaderElectionService();
 			leaderElectionService.start(this);
+			slotManager.setupResourceManagerServices(new DefaultResourceManagerServices());
 			// framework specific initialization
 			initialize();
 		} catch (Throwable e) {
@@ -117,7 +128,7 @@ public abstract class ResourceManager<ResourceManagerGateway, WorkerType extends
 	public void shutDown() {
 		try {
 			leaderElectionService.stop();
-			for(JobID jobID : jobMasterGateways.keySet()) {
+			for (JobID jobID : jobMasterGateways.keySet()) {
 				highAvailabilityServices.getJobMasterLeaderRetriever(jobID).stop();
 			}
 			super.shutDown();
@@ -189,15 +200,17 @@ public abstract class ResourceManager<ResourceManagerGateway, WorkerType extends
 					if (throwable != null) {
 						return new RegistrationResponse.Decline(throwable.getMessage());
 					} else {
-						JobMasterLeaderListener jobMasterLeaderListener = new JobMasterLeaderListener(jobID);
-						try {
-							LeaderRetrievalService jobMasterLeaderRetriever = highAvailabilityServices.getJobMasterLeaderRetriever(jobID);
-							jobMasterLeaderRetriever.start(jobMasterLeaderListener);
-						} catch (Exception e) {
-							log.warn("Failed to start JobMasterLeaderRetriever for JobID {}", jobID);
-							return new RegistrationResponse.Decline("Failed to retrieve JobMasterLeaderRetriever");
+						if (!jobMasterLeaderRetrievalListeners.containsKey(jobID)) {
+							JobMasterLeaderListener jobMasterLeaderListener = new JobMasterLeaderListener(jobID);
+							try {
+								LeaderRetrievalService jobMasterLeaderRetriever = highAvailabilityServices.getJobMasterLeaderRetriever(jobID);
+								jobMasterLeaderRetriever.start(jobMasterLeaderListener);
+							} catch (Exception e) {
+								log.warn("Failed to start JobMasterLeaderRetriever for JobID {}", jobID);
+								return new RegistrationResponse.Decline("Failed to retrieve JobMasterLeaderRetriever");
+							}
+							jobMasterLeaderRetrievalListeners.put(jobID, jobMasterLeaderListener);
 						}
-						jobMasterLeaderRetrievalListeners.add(jobMasterLeaderListener);
 						final JobMasterGateway existingGateway = jobMasterGateways.put(jobID, jobMasterGateway);
 						if (existingGateway != null) {
 							log.info("Replacing gateway for registered JobID {}.", jobID);
@@ -232,7 +245,6 @@ public abstract class ResourceManager<ResourceManagerGateway, WorkerType extends
 						resourceID, taskExecutorAddress, leaderSessionID, resourceManagerLeaderId);
 					throw new Exception("Invalid leader session id");
 				}
-
 				return getRpcService().connect(taskExecutorAddress, TaskExecutorGateway.class).get(5, TimeUnit.SECONDS);
 			}
 		}).handleAsync(new BiFunction<TaskExecutorGateway, Throwable, RegistrationResponse>() {
@@ -241,24 +253,14 @@ public abstract class ResourceManager<ResourceManagerGateway, WorkerType extends
 				if (throwable != null) {
 					return new RegistrationResponse.Decline(throwable.getMessage());
 				} else {
-					WorkerType startedWorker = taskExecutorGateways.get(resourceID);
-					if(startedWorker != null) {
-						String oldWorkerAddress = startedWorker.getTaskExecutorGateway().getAddress();
-						if (taskExecutorAddress.equals(oldWorkerAddress)) {
-							log.warn("Receive a duplicate registration from TaskExecutor {} at ({})", resourceID, taskExecutorAddress);
-						} else {
-							log.warn("Receive a duplicate registration from TaskExecutor {} at different address, previous ({}), new ({})",
-								resourceID, oldWorkerAddress, taskExecutorAddress);
-							// TODO :: suggest old taskExecutor to stop itself
-							slotManager.notifyTaskManagerFailure(resourceID);
-							startedWorker = workerStarted(resourceID, taskExecutorGateway);
-							taskExecutorGateways.put(resourceID, startedWorker);
-						}
-					} else {
-						startedWorker = workerStarted(resourceID, taskExecutorGateway);
-						taskExecutorGateways.put(resourceID, startedWorker);
+					WorkerType oldWorker = taskExecutorGateways.remove(resourceID);
+					if (oldWorker != null) {
+						// TODO :: suggest old taskExecutor to stop itself
+						slotManager.notifyTaskManagerFailure(resourceID);
 					}
-					return new TaskExecutorRegistrationSuccess(startedWorker.getInstanceID(), 5000);
+					WorkerType newWorker = workerStarted(resourceID);
+					taskExecutorGateways.put(resourceID, newWorker);
+					return new TaskExecutorRegistrationSuccess(new InstanceID(), 5000);
 				}
 			}
 		}, getMainThreadExecutor());
@@ -271,11 +273,20 @@ public abstract class ResourceManager<ResourceManagerGateway, WorkerType extends
 	 * @return Slot assignment
 	 */
 	@RpcMethod
-	public SlotRequestReply requestSlot(SlotRequest slotRequest) {
-		final JobID jobId = slotRequest.getJobId();
-		final JobMasterGateway jobMasterGateway = jobMasterGateways.get(jobId);
+	public SlotRequestReply requestSlot(
+			UUID jobMasterLeaderID,
+			UUID resourceManagerLeaderID,
+			SlotRequest slotRequest) {
+
+		JobID jobId = slotRequest.getJobId();
+		JobMasterGateway jobMasterGateway = jobMasterGateways.get(jobId);
+		JobMasterLeaderListener jobMasterLeaderListener = jobMasterLeaderRetrievalListeners.get(jobId);
+
+		UUID leaderID = jobMasterLeaderListener.getLeaderID();
 
-		if (jobMasterGateway != null) {
+		if (jobMasterGateway != null
+				&& jobMasterLeaderID.equals(leaderID)
+				&& resourceManagerLeaderID.equals(leaderSessionID)) {
 			return slotManager.requestSlot(slotRequest);
 		} else {
 			log.info("Ignoring slot request for unknown JobMaster with JobID {}", jobId);
@@ -379,7 +390,7 @@ public abstract class ResourceManager<ResourceManagerGateway, WorkerType extends
 	}
 
 	/**
-	 * Shutdowns cluster
+	 * Cleanup application and shut down cluster
 	 *
 	 * @param finalStatus
 	 * @param optionalDiagnostics
@@ -446,17 +457,11 @@ public abstract class ResourceManager<ResourceManagerGateway, WorkerType extends
 	protected abstract void initialize() throws Exception;
 
 	/**
-	 * Callback when a task executor register.
+	 * Notifies the resource master of a fatal error.
 	 *
-	 * @param resourceID The worker resource id
-	 * @param taskExecutorGateway the task executor gateway
-	 */
-	protected abstract WorkerType workerStarted(ResourceID resourceID, TaskExecutorGateway taskExecutorGateway);
-
-	/**
-	 * Callback when a resource manager faced a fatal error
-	 * @param message
-	 * @param 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);
 
@@ -472,6 +477,19 @@ public abstract class ResourceManager<ResourceManagerGateway, WorkerType extends
 	 */
 	protected abstract void shutDownApplication(ApplicationStatus finalStatus, String optionalDiagnostics);
 
+	/**
+	 * Allocates a resource using the resource profile.
+	 * @param resourceProfile The resource description
+	 */
+	@VisibleForTesting
+	public abstract void startNewWorker(ResourceProfile resourceProfile);
+
+	/**
+	 * Callback when a worker was started.
+	 * @param resourceID The worker resource id
+	 */
+	protected abstract WorkerType workerStarted(ResourceID resourceID);
+
 	// ------------------------------------------------------------------------
 	//  Info messaging
 	// ------------------------------------------------------------------------
@@ -489,6 +507,24 @@ public abstract class ResourceManager<ResourceManagerGateway, WorkerType extends
 		});
 	}
 
+	private class DefaultResourceManagerServices implements ResourceManagerServices {
+
+		@Override
+		public void allocateResource(ResourceProfile resourceProfile) {
+			ResourceManager.this.startNewWorker(resourceProfile);
+		}
+
+		@Override
+		public Executor getAsyncExecutor() {
+			return ResourceManager.this.getRpcService().getExecutor();
+		}
+
+		@Override
+		public Executor getExecutor() {
+			return ResourceManager.this.getMainThreadExecutor();
+		}
+	}
+
 	private static class JobMasterLeaderListener implements LeaderRetrievalListener {
 
 		private final JobID jobID;
@@ -498,6 +534,14 @@ public abstract class ResourceManager<ResourceManagerGateway, WorkerType extends
 			this.jobID = jobID;
 		}
 
+		public JobID getJobID() {
+			return jobID;
+		}
+
+		public UUID getLeaderID() {
+			return leaderID;
+		}
+
 		@Override
 		public void notifyLeaderAddress(final String leaderAddress, final UUID leaderSessionID) {
 			this.leaderID = leaderSessionID;

http://git-wip-us.apache.org/repos/asf/flink/blob/1f198d8c/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 7c44006..87303a1 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
@@ -59,7 +59,11 @@ public interface ResourceManagerGateway extends RpcGateway {
 	 * @param slotRequest Slot request
 	 * @return Future slot assignment
 	 */
-	Future<SlotRequestReply> requestSlot(SlotRequest slotRequest);
+	Future<SlotRequestReply> requestSlot(
+		UUID jobMasterLeaderID,
+		UUID resourceManagerLeaderID,
+		SlotRequest slotRequest,
+		@RpcTimeout Time timeout);
 
 	/**
 	 * Register a {@link org.apache.flink.runtime.taskexecutor.TaskExecutor} at the resource manager.

http://git-wip-us.apache.org/repos/asf/flink/blob/1f198d8c/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
new file mode 100644
index 0000000..30994dc
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerServices.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.runtime.resourcemanager;
+
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+
+import java.util.concurrent.Executor;
+
+/**
+ * Interface which provides access to services of the ResourceManager.
+ */
+public interface ResourceManagerServices {
+
+	/**
+	 * Allocates a resource according to the resource profile.
+	 */
+	void allocateResource(ResourceProfile resourceProfile);
+
+	/**
+	 * Gets the async excutor which executes outside of the main thread of the ResourceManager
+	 */
+	Executor getAsyncExecutor();
+
+	/**
+	 * Gets the executor which executes in the main thread of the ResourceManager
+	 */
+	Executor getExecutor();
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/1f198d8c/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 84db1ee..deca8d3 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
@@ -20,17 +20,18 @@ package org.apache.flink.runtime.resourcemanager;
 
 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.instance.InstanceID;
 import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager;
 import org.apache.flink.runtime.rpc.RpcService;
-import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway;
 
 /**
  * A standalone implementation of the resource manager. Used when the system is started in
  * standalone mode (via scripts), rather than via a resource framework like YARN or Mesos.
+ *
+ * This ResourceManager doesn't acquire new resources.
  */
-public class StandaloneResourceManager extends ResourceManager<ResourceManagerGateway, TaskExecutorRegistration> {
+public class StandaloneResourceManager extends ResourceManager<ResourceID> {
 
 	public StandaloneResourceManager(RpcService rpcService,
 		HighAvailabilityServices highAvailabilityServices,
@@ -51,14 +52,16 @@ public class StandaloneResourceManager extends ResourceManager<ResourceManagerGa
 	}
 
 	@Override
-	protected TaskExecutorRegistration workerStarted(ResourceID resourceID, TaskExecutorGateway taskExecutorGateway) {
-		InstanceID instanceID = new InstanceID();
-		TaskExecutorRegistration taskExecutorRegistration = new TaskExecutorRegistration(taskExecutorGateway, instanceID);
-		return taskExecutorRegistration;
+	protected void shutDownApplication(ApplicationStatus finalStatus, String optionalDiagnostics) {
 	}
 
 	@Override
-	protected void shutDownApplication(ApplicationStatus finalStatus, String optionalDiagnostics) {
+	public void startNewWorker(ResourceProfile resourceProfile) {
+	}
 
+	@Override
+	protected ResourceID workerStarted(ResourceID resourceID) {
+		return resourceID;
 	}
+
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/1f198d8c/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/TaskExecutorRegistration.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/TaskExecutorRegistration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/TaskExecutorRegistration.java
deleted file mode 100644
index f8dfdc7..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/TaskExecutorRegistration.java
+++ /dev/null
@@ -1,51 +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.instance.InstanceID;
-import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway;
-
-import java.io.Serializable;
-
-/**
- * This class is responsible for group the TaskExecutorGateway and the InstanceID of a registered task executor.
- */
-public class TaskExecutorRegistration implements Serializable {
-
-	private static final long serialVersionUID = -2062957799469434614L;
-
-	private TaskExecutorGateway taskExecutorGateway;
-
-	private InstanceID instanceID;
-
-	public TaskExecutorRegistration(TaskExecutorGateway taskExecutorGateway,
-									InstanceID instanceID) {
-		this.taskExecutorGateway = taskExecutorGateway;
-		this.instanceID = instanceID;
-	}
-
-	public InstanceID getInstanceID() {
-		return instanceID;
-	}
-
-	public TaskExecutorGateway getTaskExecutorGateway() {
-		return taskExecutorGateway;
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/1f198d8c/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/registration/TaskExecutorRegistration.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/registration/TaskExecutorRegistration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/registration/TaskExecutorRegistration.java
new file mode 100644
index 0000000..6b21f5c
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/registration/TaskExecutorRegistration.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.registration;
+
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway;
+
+import java.io.Serializable;
+
+/**
+ * This class is responsible for group the TaskExecutorGateway and the InstanceID of a registered task executor.
+ */
+public class TaskExecutorRegistration implements Serializable {
+
+	private static final long serialVersionUID = -2062957799469434614L;
+
+	private TaskExecutorGateway taskExecutorGateway;
+
+	private InstanceID instanceID;
+
+	public TaskExecutorRegistration(TaskExecutorGateway taskExecutorGateway,
+									InstanceID instanceID) {
+		this.taskExecutorGateway = taskExecutorGateway;
+		this.instanceID = instanceID;
+	}
+
+	public InstanceID getInstanceID() {
+		return instanceID;
+	}
+
+	public TaskExecutorGateway getTaskExecutorGateway() {
+		return taskExecutorGateway;
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/1f198d8c/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SimpleSlotManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SimpleSlotManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SimpleSlotManager.java
index ef5ce31..ae1de5a 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SimpleSlotManager.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SimpleSlotManager.java
@@ -18,7 +18,6 @@
 package org.apache.flink.runtime.resourcemanager.slotmanager;
 
 import org.apache.flink.runtime.clusterframework.types.AllocationID;
-import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
 import org.apache.flink.runtime.clusterframework.types.ResourceSlot;
 import org.apache.flink.runtime.clusterframework.types.SlotID;
 import org.apache.flink.runtime.resourcemanager.SlotRequest;
@@ -51,9 +50,4 @@ public class SimpleSlotManager extends SlotManager {
 		}
 	}
 
-	@Override
-	protected void allocateContainer(ResourceProfile resourceProfile) {
-		// TODO
-	}
-
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/1f198d8c/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManager.java
index a6d2196..a56b2f6 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManager.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManager.java
@@ -22,16 +22,18 @@ import org.apache.flink.annotation.VisibleForTesting;
 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.ResourceSlot;
 import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.concurrent.BiFunction;
 import org.apache.flink.runtime.concurrent.Future;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerServices;
 import org.apache.flink.runtime.resourcemanager.SlotRequest;
 import org.apache.flink.runtime.resourcemanager.SlotRequestRegistered;
 import org.apache.flink.runtime.resourcemanager.SlotRequestReply;
 import org.apache.flink.runtime.taskexecutor.SlotReport;
 import org.apache.flink.runtime.taskexecutor.SlotStatus;
 import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway;
+import org.apache.flink.util.Preconditions;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -82,6 +84,9 @@ public abstract class SlotManager {
 	/** The current leader id set by the ResourceManager */
 	private UUID leaderID;
 
+	/** The Resource allocation provider */
+	private ResourceManagerServices resourceManagerServices;
+
 	public SlotManager() {
 		this.registeredSlots = new HashMap<>(16);
 		this.pendingSlotRequests = new LinkedHashMap<>(16);
@@ -91,6 +96,16 @@ public abstract class SlotManager {
 		this.timeout = Time.seconds(10);
 	}
 
+	/**
+	 * Initializes the resource supplier which is needed to request new resources.
+	 */
+	public void setupResourceManagerServices(ResourceManagerServices resourceManagerServices) {
+		if (this.resourceManagerServices != null) {
+			throw new IllegalStateException("ResourceManagerServices may only be set once.");
+		}
+		this.resourceManagerServices = resourceManagerServices;
+	}
+
 
 	// ------------------------------------------------------------------------
 	//  slot managements
@@ -120,17 +135,32 @@ public abstract class SlotManager {
 
 			// record this allocation in bookkeeping
 			allocationMap.addAllocation(slot.getSlotId(), allocationId);
-
 			// remove selected slot from free pool
-			freeSlots.remove(slot.getSlotId());
+			final ResourceSlot removedSlot = freeSlots.remove(slot.getSlotId());
 
 			final Future<SlotRequestReply> slotRequestReplyFuture =
 				slot.getTaskExecutorGateway().requestSlot(allocationId, leaderID, timeout);
-			// TODO handle timeouts and response
+
+			slotRequestReplyFuture.handleAsync(new BiFunction<SlotRequestReply, Throwable, Object>() {
+				@Override
+				public Object apply(SlotRequestReply slotRequestReply, Throwable throwable) {
+					if (throwable != null) {
+						// we failed, put the slot and the request back again
+						if (allocationMap.isAllocated(slot.getSlotId())) {
+							// only re-add if the slot hasn't been removed in the meantime
+							freeSlots.put(slot.getSlotId(), removedSlot);
+						}
+						pendingSlotRequests.put(allocationId, request);
+					}
+					return null;
+				}
+			}, resourceManagerServices.getExecutor());
 		} else {
 			LOG.info("Cannot fulfil slot request, try to allocate a new container for it, " +
 				"AllocationID:{}, JobID:{}", allocationId, request.getJobId());
-			allocateContainer(request.getResourceProfile());
+			Preconditions.checkState(resourceManagerServices != null,
+				"Attempted to allocate resources but no ResourceManagerServices set.");
+			resourceManagerServices.allocateResource(request.getResourceProfile());
 			pendingSlotRequests.put(allocationId, request);
 		}
 
@@ -343,7 +373,7 @@ public abstract class SlotManager {
 
 		if (chosenRequest != null) {
 			final AllocationID allocationId = chosenRequest.getAllocationId();
-			pendingSlotRequests.remove(allocationId);
+			final SlotRequest removedSlotRequest = pendingSlotRequests.remove(allocationId);
 
 			LOG.info("Assigning SlotID({}) to AllocationID({}), JobID:{}", freeSlot.getSlotId(),
 				allocationId, chosenRequest.getJobId());
@@ -351,7 +381,19 @@ public abstract class SlotManager {
 
 			final Future<SlotRequestReply> slotRequestReplyFuture =
 				freeSlot.getTaskExecutorGateway().requestSlot(allocationId, leaderID, timeout);
-			// TODO handle timeouts and response
+
+			slotRequestReplyFuture.handleAsync(new BiFunction<SlotRequestReply, Throwable, Object>() {
+				@Override
+				public Object apply(SlotRequestReply slotRequestReply, Throwable throwable) {
+					if (throwable != null) {
+						// we failed, add the request back again
+						if (allocationMap.isAllocated(freeSlot.getSlotId())) {
+							pendingSlotRequests.put(allocationId, removedSlotRequest);
+						}
+					}
+					return null;
+				}
+			}, resourceManagerServices.getExecutor());
 		} else {
 			freeSlots.put(freeSlot.getSlotId(), freeSlot);
 		}
@@ -417,13 +459,6 @@ public abstract class SlotManager {
 	protected abstract SlotRequest chooseRequestToFulfill(final ResourceSlot offeredSlot,
 		final Map<AllocationID, SlotRequest> pendingRequests);
 
-	/**
-	 * The framework specific code for allocating a container for specified resource profile.
-	 *
-	 * @param resourceProfile The resource profile
-	 */
-	protected abstract void allocateContainer(final ResourceProfile resourceProfile);
-
 	// ------------------------------------------------------------------------
 	//  Helper classes
 	// ------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/flink/blob/1f198d8c/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerTest.java
index 9ee9690..0fed79e 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerTest.java
@@ -19,12 +19,16 @@
 package org.apache.flink.runtime.resourcemanager.slotmanager;
 
 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.ResourceSlot;
 import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.concurrent.impl.FlinkCompletableFuture;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerServices;
 import org.apache.flink.runtime.resourcemanager.SlotRequest;
+import org.apache.flink.runtime.resourcemanager.SlotRequestReply;
 import org.apache.flink.runtime.taskexecutor.SlotStatus;
 import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway;
 import org.junit.BeforeClass;
@@ -34,10 +38,13 @@ import org.mockito.Mockito;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.Executor;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
+import static org.mockito.Matchers.any;
 import static org.mockito.Mockito.mock;
 
 public class SlotManagerTest {
@@ -57,6 +64,8 @@ public class SlotManagerTest {
 	@BeforeClass
 	public static void setUp() {
 		taskExecutorGateway = Mockito.mock(TaskExecutorGateway.class);
+		Mockito.when(taskExecutorGateway.requestSlot(any(AllocationID.class), any(UUID.class), any(Time.class)))
+			.thenReturn(new FlinkCompletableFuture<SlotRequestReply>());
 	}
 
 	/**
@@ -498,12 +507,13 @@ public class SlotManagerTest {
 	//  testing classes
 	// ------------------------------------------------------------------------
 
-	private static class TestingSlotManager extends SlotManager {
+	private static class TestingSlotManager extends SlotManager implements ResourceManagerServices {
 
 		private final List<ResourceProfile> allocatedContainers;
 
 		TestingSlotManager() {
 			this.allocatedContainers = new LinkedList<>();
+			setupResourceManagerServices(this);
 		}
 
 		/**
@@ -543,12 +553,23 @@ public class SlotManagerTest {
 		}
 
 		@Override
-		protected void allocateContainer(ResourceProfile resourceProfile) {
+		public void allocateResource(ResourceProfile resourceProfile) {
 			allocatedContainers.add(resourceProfile);
 		}
 
+		@Override
+		public Executor getAsyncExecutor() {
+			return Mockito.mock(Executor.class);
+		}
+
+		@Override
+		public Executor getExecutor() {
+			return Mockito.mock(Executor.class);
+		}
+
 		List<ResourceProfile> getAllocatedContainers() {
 			return allocatedContainers;
 		}
+
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/1f198d8c/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 ff25897..e3018c9 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
@@ -24,18 +24,14 @@ 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.concurrent.Future;
-import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
-import org.apache.flink.runtime.highavailability.NonHaServices;
+import org.apache.flink.runtime.concurrent.impl.FlinkCompletableFuture;
+import org.apache.flink.runtime.concurrent.impl.FlinkFuture;
 import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices;
 import org.apache.flink.runtime.jobmaster.JobMasterGateway;
-import org.apache.flink.runtime.leaderelection.LeaderElectionService;
 import org.apache.flink.runtime.leaderelection.TestingLeaderElectionService;
 import org.apache.flink.runtime.leaderelection.TestingLeaderRetrievalService;
 import org.apache.flink.runtime.registration.RegistrationResponse;
-import org.apache.flink.runtime.resourcemanager.ResourceManager;
-import org.apache.flink.runtime.resourcemanager.SlotRequest;
-import org.apache.flink.runtime.resourcemanager.SlotRequestReply;
-import org.apache.flink.runtime.resourcemanager.StandaloneResourceManager;
+import org.apache.flink.runtime.resourcemanager.*;
 import org.apache.flink.runtime.rpc.TestingSerialRpcService;
 import org.apache.flink.runtime.taskexecutor.SlotReport;
 import org.apache.flink.runtime.taskexecutor.SlotStatus;
@@ -47,9 +43,12 @@ import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.mockito.Mockito;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
 
 import java.util.Collections;
 import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.TimeUnit;
 
 import static org.mockito.Matchers.any;
@@ -99,9 +98,9 @@ public class SlotProtocolTest extends TestLogger {
 		TestingLeaderElectionService rmLeaderElectionService =
 			configureHA(testingHaServices, jobID, rmAddress, rmLeaderID, jmAddress, jmLeaderID);
 
-		TestingSlotManager slotManager = Mockito.spy(new TestingSlotManager());
+		SlotManager slotManager = Mockito.spy(new SimpleSlotManager());
 		ResourceManager resourceManager =
-			new StandaloneResourceManager(testRpcService, testingHaServices, slotManager);
+			Mockito.spy(new StandaloneResourceManager(testRpcService, testingHaServices, slotManager));
 		resourceManager.start();
 		rmLeaderElectionService.isLeader(rmLeaderID);
 
@@ -118,7 +117,7 @@ public class SlotProtocolTest extends TestLogger {
 
 		SlotRequest slotRequest = new SlotRequest(jobID, allocationID, resourceProfile);
 		SlotRequestReply slotRequestReply =
-			resourceManager.requestSlot(slotRequest);
+			resourceManager.requestSlot(jmLeaderID, rmLeaderID, slotRequest);
 
 		// 1) SlotRequest is routed to the SlotManager
 		verify(slotManager).requestSlot(slotRequest);
@@ -129,13 +128,15 @@ public class SlotProtocolTest extends TestLogger {
 			allocationID);
 
 		// 3) SlotRequest leads to a container allocation
-		verify(slotManager, timeout(5000)).allocateContainer(resourceProfile);
+		verify(resourceManager, timeout(5000)).startNewWorker(resourceProfile);
 
 		Assert.assertFalse(slotManager.isAllocated(allocationID));
 
 		// slot becomes available
 		final String tmAddress = "/tm1";
 		TaskExecutorGateway taskExecutorGateway = mock(TaskExecutorGateway.class);
+		Mockito.when(taskExecutorGateway.requestSlot(any(AllocationID.class), any(UUID.class), any(Time.class)))
+			.thenReturn(new FlinkCompletableFuture<SlotRequestReply>());
 		testRpcService.registerGateway(tmAddress, taskExecutorGateway);
 
 		final ResourceID resourceID = ResourceID.generate();
@@ -176,11 +177,13 @@ public class SlotProtocolTest extends TestLogger {
 			configureHA(testingHaServices, jobID, rmAddress, rmLeaderID, jmAddress, jmLeaderID);
 
 		TaskExecutorGateway taskExecutorGateway = mock(TaskExecutorGateway.class);
+		Mockito.when(taskExecutorGateway.requestSlot(any(AllocationID.class), any(UUID.class), any(Time.class)))
+			.thenReturn(new FlinkCompletableFuture<SlotRequestReply>());
 		testRpcService.registerGateway(tmAddress, taskExecutorGateway);
 
-		TestingSlotManager slotManager = Mockito.spy(new TestingSlotManager());
+		SlotManager slotManager = Mockito.spy(new SimpleSlotManager());
 		ResourceManager resourceManager =
-			new StandaloneResourceManager(testRpcService, testingHaServices, slotManager);
+			Mockito.spy(new StandaloneResourceManager(testRpcService, testingHaServices, slotManager));
 		resourceManager.start();
 		rmLeaderElectionService.isLeader(rmLeaderID);
 
@@ -207,7 +210,7 @@ public class SlotProtocolTest extends TestLogger {
 
 		SlotRequest slotRequest = new SlotRequest(jobID, allocationID, resourceProfile);
 		SlotRequestReply slotRequestReply =
-			resourceManager.requestSlot(slotRequest);
+			resourceManager.requestSlot(jmLeaderID, rmLeaderID, slotRequest);
 
 		// 1) a SlotRequest is routed to the SlotManager
 		verify(slotManager).requestSlot(slotRequest);
@@ -241,15 +244,4 @@ public class SlotProtocolTest extends TestLogger {
 		return rmLeaderElectionService;
 	}
 
-	private static class TestingSlotManager extends SimpleSlotManager {
-
-		// change visibility of function to public for testing
-		@Override
-		public void allocateContainer(ResourceProfile resourceProfile) {
-			super.allocateContainer(resourceProfile);
-		}
-
-
-	}
-
 }


[36/50] [abbrv] flink git commit: [FLINK-4478] [flip-6] Add HeartbeatManager

Posted by se...@apache.org.
[FLINK-4478] [flip-6] Add HeartbeatManager

Add a heartbeat manager abstraction which can monitor heartbeat targets. Whenever
no heartbeat signal has been received for a heartbeat timeout interval, the
heartbeat manager will issue a heartbeat timeout notification.

Add resourceID to HeartbeatListener.reportPayload

Replace scala future by Flink's futures

Add unmonitoring test

This closes #2435.


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

Branch: refs/heads/flip-6
Commit: 3e4eb4f92012265b6fff27f0544fcd6d1629431f
Parents: 214113e
Author: Till Rohrmann <tr...@apache.org>
Authored: Thu Aug 25 14:05:07 2016 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Oct 14 15:14:42 2016 +0200

----------------------------------------------------------------------
 .../runtime/heartbeat/HeartbeatListener.java    |  62 ++++
 .../runtime/heartbeat/HeartbeatManager.java     |  67 ++++
 .../runtime/heartbeat/HeartbeatManagerImpl.java | 328 +++++++++++++++++++
 .../heartbeat/HeartbeatManagerSenderImpl.java   |  81 +++++
 .../runtime/heartbeat/HeartbeatTarget.java      |  50 +++
 .../runtime/heartbeat/HeartbeatManagerTest.java | 315 ++++++++++++++++++
 .../slotmanager/SlotProtocolTest.java           |   4 -
 7 files changed, 903 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/3e4eb4f9/flink-runtime/src/main/java/org/apache/flink/runtime/heartbeat/HeartbeatListener.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/heartbeat/HeartbeatListener.java b/flink-runtime/src/main/java/org/apache/flink/runtime/heartbeat/HeartbeatListener.java
new file mode 100644
index 0000000..8c08251
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/heartbeat/HeartbeatListener.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.heartbeat;
+
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.concurrent.Future;
+
+/**
+ * Interface for the interaction with the {@link HeartbeatManager}. The heartbeat listener is used
+ * for the following things:
+ * <p>
+ * <ul>
+ *     <il>Notifications about heartbeat timeouts</il>
+ *     <li>Payload reports of incoming heartbeats</li>
+ *     <li>Retrieval of payloads for outgoing heartbeats</li>
+ * </ul>
+ * @param <I> Type of the incoming payload
+ * @param <O> Type of the outgoing payload
+ */
+public interface HeartbeatListener<I, O> {
+
+	/**
+	 * Callback which is called if a heartbeat for the machine identified by the given resource
+	 * ID times out.
+	 *
+	 * @param resourceID Resource ID of the machine whose heartbeat has timed out
+	 */
+	void notifyHeartbeatTimeout(ResourceID resourceID);
+
+	/**
+	 * Callback which is called whenever a heartbeat with an associated payload is received. The
+	 * carried payload is given to this method.
+	 *
+	 * @param resourceID Resource ID identifying the sender of the payload
+	 * @param payload Payload of the received heartbeat
+	 */
+	void reportPayload(ResourceID resourceID, I payload);
+
+	/**
+	 * Retrieves the payload value for the next heartbeat message. Since the operation can happen
+	 * asynchronously, the result is returned wrapped in a future.
+	 *
+	 * @return Future containing the next payload for heartbeats
+	 */
+	Future<O> retrievePayload();
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/3e4eb4f9/flink-runtime/src/main/java/org/apache/flink/runtime/heartbeat/HeartbeatManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/heartbeat/HeartbeatManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/heartbeat/HeartbeatManager.java
new file mode 100644
index 0000000..12918ed
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/heartbeat/HeartbeatManager.java
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.heartbeat;
+
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+
+/**
+ * A heartbeat manager has to be able to do the following things:
+ *
+ * <ul>
+ *     <li>Monitor {@link HeartbeatTarget} and report heartbeat timeouts for this target</li>
+ *     <li>Stop monitoring a {@link HeartbeatTarget}</li>
+ * </ul>
+ *
+ *
+ * @param <I> Type of the incoming payload
+ * @param <O> Type of the outgoing payload
+ */
+public interface HeartbeatManager<I, O> {
+
+	/**
+	 * Start monitoring a {@link HeartbeatTarget}. Heartbeat timeouts for this target are reported
+	 * to the {@link HeartbeatListener} associated with this heartbeat manager.
+	 *
+	 * @param resourceID Resource ID identifying the heartbeat target
+	 * @param heartbeatTarget Interface to send heartbeat requests and responses to the heartbeat
+	 *                        target
+	 */
+	void monitorTarget(ResourceID resourceID, HeartbeatTarget<O> heartbeatTarget);
+
+	/**
+	 * Stops monitoring the heartbeat target with the associated resource ID.
+	 *
+	 * @param resourceID Resource ID of the heartbeat target which shall no longer be monitored
+	 */
+	void unmonitorTarget(ResourceID resourceID);
+
+	/**
+	 * Starts the heartbeat manager with the given {@link HeartbeatListener}. The heartbeat listener
+	 * is notified about heartbeat timeouts and heartbeat payloads are reported and retrieved to
+	 * and from it.
+	 *
+	 * @param heartbeatListener Heartbeat listener associated with the heartbeat manager
+	 */
+	void start(HeartbeatListener<I, O> heartbeatListener);
+
+	/**
+	 * Stops the heartbeat manager.
+	 */
+	void stop();
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/3e4eb4f9/flink-runtime/src/main/java/org/apache/flink/runtime/heartbeat/HeartbeatManagerImpl.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/heartbeat/HeartbeatManagerImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/heartbeat/HeartbeatManagerImpl.java
new file mode 100644
index 0000000..042f95b
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/heartbeat/HeartbeatManagerImpl.java
@@ -0,0 +1,328 @@
+/*
+ * 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.heartbeat;
+
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.concurrent.AcceptFunction;
+import org.apache.flink.runtime.concurrent.Future;
+import org.apache.flink.util.Preconditions;
+import org.slf4j.Logger;
+
+import javax.annotation.concurrent.ThreadSafe;
+import java.util.Collection;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+
+/**
+ * Heartbeat manager implementation. The heartbeat manager maintains a map of heartbeat monitors
+ * and resource IDs. Each monitor will be updated when a new heartbeat of the associated machine has
+ * been received. If the monitor detects that a heartbeat has timed out, it will notify the
+ * {@link HeartbeatListener} about it. A heartbeat times out iff no heartbeat signal has been
+ * received within a given timeout interval.
+ *
+ * @param <I> Type of the incoming heartbeat payload
+ * @param <O> Type of the outgoing heartbeat payload
+ */
+@ThreadSafe
+public class HeartbeatManagerImpl<I, O> implements HeartbeatManager<I, O>, HeartbeatTarget<I> {
+
+	/** Heartbeat timeout interval in milli seconds */
+	private final long heartbeatTimeoutIntervalMs;
+
+	/** Resource ID which is used to mark one own's heartbeat signals */
+	private final ResourceID ownResourceID;
+
+	/** Executor service used to run heartbeat timeout notifications */
+	private final ScheduledExecutorService scheduledExecutorService;
+
+	protected final Logger log;
+
+	/** Map containing the heartbeat monitors associated with the respective resource ID */
+	private final ConcurrentHashMap<ResourceID, HeartbeatManagerImpl.HeartbeatMonitor<O>> heartbeatTargets;
+
+	/** Execution context used to run future callbacks */
+	private final Executor executor;
+
+	/** Heartbeat listener with which the heartbeat manager has been associated */
+	private HeartbeatListener<I, O> heartbeatListener;
+
+	/** Running state of the heartbeat manager */
+	protected volatile boolean stopped;
+
+	public HeartbeatManagerImpl(
+		long heartbeatTimeoutIntervalMs,
+		ResourceID ownResourceID,
+		Executor executor,
+		ScheduledExecutorService scheduledExecutorService,
+		Logger log) {
+		Preconditions.checkArgument(heartbeatTimeoutIntervalMs > 0L, "The heartbeat timeout has to be larger than 0.");
+
+		this.heartbeatTimeoutIntervalMs = heartbeatTimeoutIntervalMs;
+		this.ownResourceID = Preconditions.checkNotNull(ownResourceID);
+		this.scheduledExecutorService = Preconditions.checkNotNull(scheduledExecutorService);
+		this.log = Preconditions.checkNotNull(log);
+		this.executor = Preconditions.checkNotNull(executor);
+		this.heartbeatTargets = new ConcurrentHashMap<>(16);
+
+		stopped = true;
+	}
+
+	//----------------------------------------------------------------------------------------------
+	// Getters
+	//----------------------------------------------------------------------------------------------
+
+	ResourceID getOwnResourceID() {
+		return ownResourceID;
+	}
+
+	Executor getExecutor() {
+		return executor;
+	}
+
+	HeartbeatListener<I, O> getHeartbeatListener() {
+		return heartbeatListener;
+	}
+
+	Collection<HeartbeatManagerImpl.HeartbeatMonitor<O>> getHeartbeatTargets() {
+		return heartbeatTargets.values();
+	}
+
+	//----------------------------------------------------------------------------------------------
+	// HeartbeatManager methods
+	//----------------------------------------------------------------------------------------------
+
+	@Override
+	public void monitorTarget(ResourceID resourceID, HeartbeatTarget<O> heartbeatTarget) {
+		if (!stopped) {
+			if (heartbeatTargets.containsKey(resourceID)) {
+				log.info("The target with resource ID {} is already been monitored.", resourceID);
+			} else {
+				HeartbeatManagerImpl.HeartbeatMonitor<O> heartbeatMonitor = new HeartbeatManagerImpl.HeartbeatMonitor<>(
+					resourceID,
+					heartbeatTarget,
+					scheduledExecutorService,
+					heartbeatListener,
+					heartbeatTimeoutIntervalMs);
+
+				heartbeatTargets.put(
+					resourceID,
+					heartbeatMonitor);
+
+				// check if we have stopped in the meantime (concurrent stop operation)
+				if (stopped) {
+					heartbeatMonitor.cancel();
+
+					heartbeatTargets.remove(resourceID);
+				}
+			}
+		}
+	}
+
+	@Override
+	public void unmonitorTarget(ResourceID resourceID) {
+		if (!stopped) {
+			HeartbeatManagerImpl.HeartbeatMonitor<O> heartbeatMonitor = heartbeatTargets.remove(resourceID);
+
+			if (heartbeatMonitor != null) {
+				heartbeatMonitor.cancel();
+			}
+		}
+	}
+
+	@Override
+	public void start(HeartbeatListener<I, O> heartbeatListener) {
+		Preconditions.checkState(stopped, "Cannot start an already started heartbeat manager.");
+
+		stopped = false;
+
+		this.heartbeatListener = Preconditions.checkNotNull(heartbeatListener);
+	}
+
+	@Override
+	public void stop() {
+		stopped = true;
+
+		for (HeartbeatManagerImpl.HeartbeatMonitor<O> heartbeatMonitor : heartbeatTargets.values()) {
+			heartbeatMonitor.cancel();
+		}
+
+		heartbeatTargets.clear();
+	}
+
+	//----------------------------------------------------------------------------------------------
+	// HeartbeatTarget methods
+	//----------------------------------------------------------------------------------------------
+
+	@Override
+	public void sendHeartbeat(ResourceID resourceID, I payload) {
+		if (!stopped) {
+			log.debug("Received heartbeat from {}.", resourceID);
+			reportHeartbeat(resourceID);
+
+			if (payload != null) {
+				heartbeatListener.reportPayload(resourceID, payload);
+			}
+		}
+	}
+
+	@Override
+	public void requestHeartbeat(ResourceID resourceID, I payload) {
+		if (!stopped) {
+			log.debug("Received heartbeat request from {}.", resourceID);
+
+			final HeartbeatTarget<O> heartbeatTarget = reportHeartbeat(resourceID);
+
+			if (heartbeatTarget != null) {
+				if (payload != null) {
+					heartbeatListener.reportPayload(resourceID, payload);
+				}
+
+				Future<O> futurePayload = heartbeatListener.retrievePayload();
+
+				if (futurePayload != null) {
+					futurePayload.thenAcceptAsync(new AcceptFunction<O>() {
+						@Override
+						public void accept(O retrievedPayload) {
+							heartbeatTarget.sendHeartbeat(getOwnResourceID(), retrievedPayload);
+						}
+					}, executor);
+				} else {
+					heartbeatTarget.sendHeartbeat(ownResourceID, null);
+				}
+			}
+		}
+	}
+
+	HeartbeatTarget<O> reportHeartbeat(ResourceID resourceID) {
+		if (heartbeatTargets.containsKey(resourceID)) {
+			HeartbeatManagerImpl.HeartbeatMonitor<O> heartbeatMonitor = heartbeatTargets.get(resourceID);
+			heartbeatMonitor.reportHeartbeat();
+
+			return heartbeatMonitor.getHeartbeatTarget();
+		} else {
+			return null;
+		}
+	}
+
+	//----------------------------------------------------------------------------------------------
+	// Utility classes
+	//----------------------------------------------------------------------------------------------
+
+	/**
+	 * Heartbeat monitor which manages the heartbeat state of the associated heartbeat target. The
+	 * monitor notifies the {@link HeartbeatListener} whenever it has not seen a heartbeat signal
+	 * in the specified heartbeat timeout interval. Each heartbeat signal resets this timer.
+	 *
+	 * @param <O> Type of the payload being sent to the associated heartbeat target
+	 */
+	static class HeartbeatMonitor<O> implements Runnable {
+
+		/** Resource ID of the monitored heartbeat target */
+		private final ResourceID resourceID;
+
+		/** Associated heartbeat target */
+		private final HeartbeatTarget<O> heartbeatTarget;
+
+		private final ScheduledExecutorService scheduledExecutorService;
+
+		/** Listener which is notified about heartbeat timeouts */
+		private final HeartbeatListener<?, ?> heartbeatListener;
+
+		/** Maximum heartbeat timeout interval */
+		private final long heartbeatTimeoutIntervalMs;
+
+		private volatile ScheduledFuture<?> futureTimeout;
+
+		private final AtomicReference<State> state = new AtomicReference<>(State.RUNNING);
+
+		HeartbeatMonitor(
+			ResourceID resourceID,
+			HeartbeatTarget<O> heartbeatTarget,
+			ScheduledExecutorService scheduledExecutorService,
+			HeartbeatListener<?, O> heartbeatListener,
+			long heartbeatTimeoutIntervalMs) {
+
+			this.resourceID = Preconditions.checkNotNull(resourceID);
+			this.heartbeatTarget = Preconditions.checkNotNull(heartbeatTarget);
+			this.scheduledExecutorService = Preconditions.checkNotNull(scheduledExecutorService);
+			this.heartbeatListener = Preconditions.checkNotNull(heartbeatListener);
+
+			Preconditions.checkArgument(heartbeatTimeoutIntervalMs >= 0L, "The heartbeat timeout interval has to be larger than 0.");
+			this.heartbeatTimeoutIntervalMs = heartbeatTimeoutIntervalMs;
+
+			resetHeartbeatTimeout(heartbeatTimeoutIntervalMs);
+		}
+
+		HeartbeatTarget<O> getHeartbeatTarget() {
+			return heartbeatTarget;
+		}
+
+		void reportHeartbeat() {
+			resetHeartbeatTimeout(heartbeatTimeoutIntervalMs);
+		}
+
+		void resetHeartbeatTimeout(long heartbeatTimeout) {
+			if (state.get() == State.RUNNING) {
+				cancelTimeout();
+
+				futureTimeout = scheduledExecutorService.schedule(this, heartbeatTimeout, TimeUnit.MILLISECONDS);
+
+				// Double check for concurrent accesses (e.g. a firing of the scheduled future)
+				if (state.get() != State.RUNNING) {
+					cancelTimeout();
+				}
+			}
+		}
+
+		void cancel() {
+			// we can only cancel if we are in state running
+			if (state.compareAndSet(State.RUNNING, State.CANCELED)) {
+				cancelTimeout();
+			}
+		}
+
+		private void cancelTimeout() {
+			if (futureTimeout != null) {
+				futureTimeout.cancel(true);
+			}
+		}
+
+		public boolean isCanceled() {
+			return state.get() == State.CANCELED;
+		}
+
+		@Override
+		public void run() {
+			// The heartbeat has timed out if we're in state running
+			if (state.compareAndSet(State.RUNNING, State.TIMEOUT)) {
+				heartbeatListener.notifyHeartbeatTimeout(resourceID);
+			}
+		}
+
+		private enum State {
+			RUNNING,
+			TIMEOUT,
+			CANCELED
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/3e4eb4f9/flink-runtime/src/main/java/org/apache/flink/runtime/heartbeat/HeartbeatManagerSenderImpl.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/heartbeat/HeartbeatManagerSenderImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/heartbeat/HeartbeatManagerSenderImpl.java
new file mode 100644
index 0000000..588ba7f
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/heartbeat/HeartbeatManagerSenderImpl.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.runtime.heartbeat;
+
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.concurrent.AcceptFunction;
+import org.apache.flink.runtime.concurrent.Future;
+import org.slf4j.Logger;
+
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * {@link HeartbeatManager} implementation which regularly requests a heartbeat response from
+ * its monitored {@link HeartbeatTarget}. The heartbeat period is configurable.
+ *
+ * @param <I> Type of the incoming heartbeat payload
+ * @param <O> Type of the outgoind heartbeat payload
+ */
+public class HeartbeatManagerSenderImpl<I, O> extends HeartbeatManagerImpl<I, O> implements Runnable {
+
+	private final ScheduledFuture<?> triggerFuture;
+
+	public HeartbeatManagerSenderImpl(
+		long heartbeatPeriod,
+		long heartbeatTimeout,
+		ResourceID ownResourceID,
+		ExecutorService executorService,
+		ScheduledExecutorService scheduledExecutorService,
+		Logger log) {
+		super(heartbeatTimeout, ownResourceID, executorService, scheduledExecutorService, log);
+
+		triggerFuture = scheduledExecutorService.scheduleAtFixedRate(this, 0L, heartbeatPeriod, TimeUnit.MILLISECONDS);
+	}
+
+	@Override
+	public void run() {
+		if (!stopped) {
+			log.debug("Trigger heartbeat request.");
+			for (HeartbeatMonitor<O> heartbeatMonitor : getHeartbeatTargets()) {
+				Future<O> futurePayload = getHeartbeatListener().retrievePayload();
+				final HeartbeatTarget<O> heartbeatTarget = heartbeatMonitor.getHeartbeatTarget();
+
+				if (futurePayload != null) {
+					futurePayload.thenAcceptAsync(new AcceptFunction<O>() {
+						@Override
+						public void accept(O payload) {
+							heartbeatTarget.requestHeartbeat(getOwnResourceID(), payload);
+						}
+					}, getExecutor());
+				} else {
+					heartbeatTarget.requestHeartbeat(getOwnResourceID(), null);
+				}
+			}
+		}
+	}
+
+	@Override
+	public void stop() {
+			triggerFuture.cancel(true);
+			super.stop();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/3e4eb4f9/flink-runtime/src/main/java/org/apache/flink/runtime/heartbeat/HeartbeatTarget.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/heartbeat/HeartbeatTarget.java b/flink-runtime/src/main/java/org/apache/flink/runtime/heartbeat/HeartbeatTarget.java
new file mode 100644
index 0000000..ef953de
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/heartbeat/HeartbeatTarget.java
@@ -0,0 +1,50 @@
+/*
+ * 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.heartbeat;
+
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+
+/**
+ * Interface for components which can be sent heartbeats and from which one can request a
+ * heartbeat response. Both the heartbeat response as well as the heartbeat request can carry a
+ * payload. This payload is reported to the heartbeat target and contains additional information.
+ * The payload can be empty which is indicated by a null value.
+ *
+ * @param <I> Type of the payload which is sent to the heartbeat target
+ */
+public interface HeartbeatTarget<I> {
+
+	/**
+	 * Sends a heartbeat response to the target. Each heartbeat response can carry a payload which
+	 * contains additional information for the heartbeat target.
+	 *
+	 * @param resourceID Resource ID identifying the machine for which a heartbeat shall be reported.
+	 * @param payload Payload of the heartbeat response. Null indicates an empty payload.
+	 */
+	void sendHeartbeat(ResourceID resourceID, I payload);
+
+	/**
+	 * Requests a heartbeat from the target. Each heartbeat request can carry a payload which
+	 * contains additional information for the heartbeat target.
+	 *
+	 * @param resourceID Resource ID identifying the machine issuing the heartbeat request.
+	 * @param payload Payload of the heartbeat response. Null indicates an empty payload.
+	 */
+	void requestHeartbeat(ResourceID resourceID, I payload);
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/3e4eb4f9/flink-runtime/src/test/java/org/apache/flink/runtime/heartbeat/HeartbeatManagerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/heartbeat/HeartbeatManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/heartbeat/HeartbeatManagerTest.java
new file mode 100644
index 0000000..1c62f17
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/heartbeat/HeartbeatManagerTest.java
@@ -0,0 +1,315 @@
+/*
+ * 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.heartbeat;
+
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+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.util.DirectExecutorService;
+import org.apache.flink.util.TestLogger;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyLong;
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.atLeast;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+
+public class HeartbeatManagerTest extends TestLogger {
+	private static final Logger LOG = LoggerFactory.getLogger(HeartbeatManagerTest.class);
+
+	/**
+	 * Tests that regular heartbeat signal triggers the right callback functions in the
+	 * {@link HeartbeatListener}.
+	 */
+	@Test
+	public void testRegularHeartbeat() {
+		long heartbeatTimeout = 1000L;
+		ResourceID ownResourceID = new ResourceID("foobar");
+		ResourceID targetResourceID = new ResourceID("barfoo");
+		HeartbeatListener<Object, Object> heartbeatListener = mock(HeartbeatListener.class);
+		ScheduledExecutorService scheduledExecutorService = mock(ScheduledExecutorService.class);
+
+		Object expectedObject = new Object();
+
+		when(heartbeatListener.retrievePayload()).thenReturn(FlinkCompletableFuture.completed(expectedObject));
+
+		HeartbeatManagerImpl<Object, Object> heartbeatManager = new HeartbeatManagerImpl<>(
+			heartbeatTimeout,
+			ownResourceID,
+			new DirectExecutorService(),
+			scheduledExecutorService,
+			LOG);
+
+		heartbeatManager.start(heartbeatListener);
+
+		HeartbeatTarget<Object> heartbeatTarget = mock(HeartbeatTarget.class);
+
+		heartbeatManager.monitorTarget(targetResourceID, heartbeatTarget);
+
+		heartbeatManager.requestHeartbeat(targetResourceID, expectedObject);
+
+		verify(heartbeatListener, times(1)).reportPayload(targetResourceID, expectedObject);
+		verify(heartbeatListener, times(1)).retrievePayload();
+		verify(heartbeatTarget, times(1)).sendHeartbeat(ownResourceID, expectedObject);
+
+		heartbeatManager.sendHeartbeat(targetResourceID, expectedObject);
+
+		verify(heartbeatListener, times(2)).reportPayload(targetResourceID, expectedObject);
+	}
+
+	/**
+	 * Tests that the heartbeat monitors are updated when receiving a new heartbeat signal.
+	 */
+	@Test
+	public void testHeartbeatMonitorUpdate() {
+		long heartbeatTimeout = 1000L;
+		ResourceID ownResourceID = new ResourceID("foobar");
+		ResourceID targetResourceID = new ResourceID("barfoo");
+		HeartbeatListener<Object, Object> heartbeatListener = mock(HeartbeatListener.class);
+		ScheduledExecutorService scheduledExecutorService = mock(ScheduledExecutorService.class);
+		ScheduledFuture<?> scheduledFuture = mock(ScheduledFuture.class);
+
+		doReturn(scheduledFuture).when(scheduledExecutorService).schedule(any(Runnable.class), anyLong(), any(TimeUnit.class));
+
+		Object expectedObject = new Object();
+
+		when(heartbeatListener.retrievePayload()).thenReturn(FlinkCompletableFuture.completed(expectedObject));
+
+		HeartbeatManagerImpl<Object, Object> heartbeatManager = new HeartbeatManagerImpl<>(
+			heartbeatTimeout,
+			ownResourceID,
+			new DirectExecutorService(),
+			scheduledExecutorService,
+			LOG);
+
+		heartbeatManager.start(heartbeatListener);
+
+		HeartbeatTarget<Object> heartbeatTarget = mock(HeartbeatTarget.class);
+
+		heartbeatManager.monitorTarget(targetResourceID, heartbeatTarget);
+
+		heartbeatManager.sendHeartbeat(targetResourceID, expectedObject);
+
+		verify(scheduledFuture, times(1)).cancel(true);
+		verify(scheduledExecutorService, times(2)).schedule(any(Runnable.class), eq(heartbeatTimeout), eq(TimeUnit.MILLISECONDS));
+	}
+
+	/**
+	 * Tests that a heartbeat timeout is signaled if the heartbeat is not reported in time.
+	 *
+	 * @throws Exception
+	 */
+	@Test
+	public void testHeartbeatTimeout() throws Exception {
+		long heartbeatTimeout = 100L;
+		int numHeartbeats = 10;
+		long heartbeatInterval = 20L;
+		Object payload = new Object();
+
+		ResourceID ownResourceID = new ResourceID("foobar");
+		ResourceID targetResourceID = new ResourceID("barfoo");
+		TestingHeartbeatListener heartbeatListener = new TestingHeartbeatListener(payload);
+		ScheduledExecutorService scheduledExecutorService = mock(ScheduledExecutorService.class);
+		ScheduledFuture<?> scheduledFuture = mock(ScheduledFuture.class);
+
+		doReturn(scheduledFuture).when(scheduledExecutorService).schedule(any(Runnable.class), anyLong(), any(TimeUnit.class));
+
+		Object expectedObject = new Object();
+
+		HeartbeatManagerImpl<Object, Object> heartbeatManager = new HeartbeatManagerImpl<>(
+			heartbeatTimeout,
+			ownResourceID,
+			new DirectExecutorService(),
+			new ScheduledThreadPoolExecutor(1),
+			LOG);
+
+		heartbeatManager.start(heartbeatListener);
+
+		HeartbeatTarget<Object> heartbeatTarget = mock(HeartbeatTarget.class);
+
+		Future<ResourceID> timeoutFuture = heartbeatListener.getTimeoutFuture();
+
+		heartbeatManager.monitorTarget(targetResourceID, heartbeatTarget);
+
+		for (int i = 0; i < numHeartbeats; i++) {
+			heartbeatManager.sendHeartbeat(targetResourceID, expectedObject);
+			Thread.sleep(heartbeatInterval);
+		}
+
+		assertFalse(timeoutFuture.isDone());
+
+		ResourceID timeoutResourceID = timeoutFuture.get(2 * heartbeatTimeout, TimeUnit.MILLISECONDS);
+
+		assertEquals(targetResourceID, timeoutResourceID);
+	}
+
+	/**
+	 * Tests the heartbeat interplay between the {@link HeartbeatManagerImpl} and the
+	 * {@link HeartbeatManagerSenderImpl}. The sender should regularly trigger heartbeat requests
+	 * which are fulfilled by the receiver. Upon stopping the receiver, the sender should notify
+	 * the heartbeat listener about the heartbeat timeout.
+	 *
+	 * @throws Exception
+	 */
+	@Test
+	public void testHeartbeatCluster() throws Exception {
+		long heartbeatTimeout = 100L;
+		long heartbeatPeriod = 20L;
+		Object object = new Object();
+		Object object2 = new Object();
+		ResourceID resourceID = new ResourceID("foobar");
+		ResourceID resourceID2 = new ResourceID("barfoo");
+		HeartbeatListener<Object, Object> heartbeatListener = mock(HeartbeatListener.class);
+
+		when(heartbeatListener.retrievePayload()).thenReturn(FlinkCompletableFuture.completed(object));
+
+		TestingHeartbeatListener heartbeatListener2 = new TestingHeartbeatListener(object2);
+
+		Future<ResourceID> futureTimeout = heartbeatListener2.getTimeoutFuture();
+
+		HeartbeatManagerImpl<Object, Object> heartbeatManager = new HeartbeatManagerImpl<>(
+			heartbeatTimeout,
+			resourceID,
+			new DirectExecutorService(),
+			new ScheduledThreadPoolExecutor(1),
+			LOG);
+
+		HeartbeatManagerSenderImpl<Object, Object> heartbeatManager2 = new HeartbeatManagerSenderImpl<>(
+			heartbeatPeriod,
+			heartbeatTimeout,
+			resourceID2,
+			new DirectExecutorService(),
+			new ScheduledThreadPoolExecutor(1),
+			LOG);;
+
+		heartbeatManager.start(heartbeatListener);
+		heartbeatManager2.start(heartbeatListener2);
+
+		heartbeatManager.monitorTarget(resourceID2, heartbeatManager2);
+		heartbeatManager2.monitorTarget(resourceID, heartbeatManager);
+
+		Thread.sleep(2 * heartbeatTimeout);
+
+		assertFalse(futureTimeout.isDone());
+
+		heartbeatManager.stop();
+
+		ResourceID timeoutResourceID = futureTimeout.get(2 * heartbeatTimeout, TimeUnit.MILLISECONDS);
+
+		assertEquals(resourceID, timeoutResourceID);
+
+		int numberHeartbeats = (int) (2 * heartbeatTimeout / heartbeatPeriod);
+
+		verify(heartbeatListener, atLeast(numberHeartbeats / 2)).reportPayload(resourceID2, object2);
+		assertTrue(heartbeatListener2.getNumberHeartbeatReports() >= numberHeartbeats / 2);
+	}
+
+	/**
+	 * Tests that after unmonitoring a target, there won't be a timeout triggered
+	 */
+	@Test
+	public void testTargetUnmonitoring() throws InterruptedException, ExecutionException {
+		// this might be too aggresive for Travis, let's see...
+		long heartbeatTimeout = 100L;
+		ResourceID resourceID = new ResourceID("foobar");
+		ResourceID targetID = new ResourceID("target");
+		Object object = new Object();
+
+		HeartbeatManager<Object, Object> heartbeatManager = new HeartbeatManagerImpl<>(
+			heartbeatTimeout,
+			resourceID,
+			new DirectExecutorService(),
+			new ScheduledThreadPoolExecutor(1),
+			LOG);
+
+		TestingHeartbeatListener heartbeatListener = new TestingHeartbeatListener(object);
+
+		heartbeatManager.start(heartbeatListener);
+
+		heartbeatManager.monitorTarget(targetID, mock(HeartbeatTarget.class));
+
+		heartbeatManager.unmonitorTarget(targetID);
+
+		Future<ResourceID> timeout = heartbeatListener.getTimeoutFuture();
+
+
+		try {
+			timeout.get(2 * heartbeatTimeout, TimeUnit.MILLISECONDS);
+			fail("Timeout should time out.");
+		} catch (TimeoutException e) {
+			// the timeout should not be completed since we unmonitored the target
+		}
+	}
+
+	static class TestingHeartbeatListener implements HeartbeatListener<Object, Object> {
+
+		private final CompletableFuture<ResourceID> future = new FlinkCompletableFuture<>();
+
+		private final Object payload;
+
+		private int numberHeartbeatReports;
+
+		TestingHeartbeatListener(Object payload) {
+			this.payload = payload;
+		}
+
+		public Future<ResourceID> getTimeoutFuture() {
+			return future;
+		}
+
+		public int getNumberHeartbeatReports() {
+			return numberHeartbeatReports;
+		}
+
+		@Override
+		public void notifyHeartbeatTimeout(ResourceID resourceID) {
+			future.complete(resourceID);
+		}
+
+		@Override
+		public void reportPayload(ResourceID resourceID, Object payload) {
+			numberHeartbeatReports++;
+		}
+
+		@Override
+		public Future<Object> retrievePayload() {
+			return FlinkCompletableFuture.completed(payload);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/3e4eb4f9/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 805ea71..a87fe42 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
@@ -25,7 +25,6 @@ import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
 import org.apache.flink.runtime.clusterframework.types.SlotID;
 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.highavailability.TestingHighAvailabilityServices;
 import org.apache.flink.runtime.jobmaster.JobMasterGateway;
 import org.apache.flink.runtime.leaderelection.TestingLeaderElectionService;
@@ -43,12 +42,9 @@ import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.mockito.Mockito;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
 
 import java.util.Collections;
 import java.util.UUID;
-import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.TimeUnit;
 
 import static org.mockito.Matchers.any;


[49/50] [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/0615b62f
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/0615b62f
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/0615b62f

Branch: refs/heads/flip-6
Commit: 0615b62fb2239f3353e450768f820e5032c0cb7e
Parents: 041dfd7
Author: Kurt Young <yk...@gmail.com>
Authored: Wed Oct 12 23:25:16 2016 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Oct 14 15:32:27 2016 +0200

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


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

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

http://git-wip-us.apache.org/repos/asf/flink/blob/0615b62f/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/0615b62f/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/0615b62f/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/0615b62f/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/0615b62f/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 5a01e4d..2634006 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
@@ -34,5 +34,5 @@ public interface CheckpointCoordinatorGateway extends RpcGateway {
 	void declineCheckpoint(
 			final JobID jobID,
 			final ExecutionAttemptID executionAttemptID,
-			final CheckpointMetaData checkpoint);
+			final long checkpointID);
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/0615b62f/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/0615b62f/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/0615b62f/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/0615b62f/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/ZookeeperHaServices.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/ZookeeperHaServices.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/ZookeeperHaServices.java
index d26b668..3a7736b 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/ZookeeperHaServices.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/ZookeeperHaServices.java
@@ -19,8 +19,15 @@
 package org.apache.flink.runtime.highavailability;
 
 import org.apache.curator.framework.CuratorFramework;
+
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
+import org.apache.flink.configuration.IllegalConfigurationException;
+import org.apache.flink.core.fs.FileSystem;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.runtime.blob.BlobStore;
+import org.apache.flink.runtime.blob.FileSystemBlobStore;
 import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory;
 import org.apache.flink.runtime.checkpoint.ZooKeeperCheckpointRecoveryFactory;
 import org.apache.flink.runtime.jobmanager.SubmittedJobGraphStore;
@@ -28,12 +35,56 @@ import org.apache.flink.runtime.leaderelection.LeaderElectionService;
 import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
 import org.apache.flink.runtime.util.ZooKeeperUtils;
 
+import java.io.IOException;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.StringUtils.isNullOrWhitespaceOnly;
+
 /**
- * An implementation of the {@link HighAvailabilityServices} with zookeeper.
+ * An implementation of the {@link HighAvailabilityServices} using Apache ZooKeeper.
+ * The services store data in ZooKeeper's nodes as illustrated by teh following tree structure:
+ * 
+ * <pre>
+ * /flink
+ *      +/cluster_id_1/resource_manager_lock
+ *      |            |
+ *      |            +/job-id-1/job_manager_lock
+ *      |            |         /checkpoints/latest
+ *      |            |                     /latest-1
+ *      |            |                     /latest-2
+ *      |            |
+ *      |            +/job-id-2/job_manager_lock
+ *      |      
+ *      +/cluster_id_2/resource_manager_lock
+ *                   |
+ *                   +/job-id-1/job_manager_lock
+ *                            |/checkpoints/latest
+ *                            |            /latest-1
+ *                            |/persisted_job_graph
+ * </pre>
+ * 
+ * <p>The root path "/flink" is configurable via the option {@link HighAvailabilityOptions#HA_ZOOKEEPER_ROOT}.
+ * This makes sure Flink stores its data under specific subtrees in ZooKeeper, for example to
+ * accommodate specific permission.
+ * 
+ * <p>The "cluster_id" part identifies the data stored for a specific Flink "cluster". 
+ * This "cluster" can be either a standalone or containerized Flink cluster, or it can be job
+ * on a framework like YARN or Mesos (in a "per-job-cluster" mode).
+ * 
+ * <p>In case of a "per-job-cluster" on YARN or Mesos, the cluster-id is generated and configured
+ * automatically by the client or dispatcher that submits the Job to YARN or Mesos.
+ * 
+ * <p>In the case of a standalone cluster, that cluster-id needs to be configured via
+ * {@link HighAvailabilityOptions#HA_CLUSTER_ID}. All nodes with the same cluster id will join the same
+ * cluster and participate in the execution of the same set of jobs.
  */
 public class ZookeeperHaServices implements HighAvailabilityServices {
 
-	private static final String DEFAULT_RESOURCE_MANAGER_PATH_SUFFIX = "/resource-manager";
+	private static final String RESOURCE_MANAGER_LEADER_PATH = "/resource_manager_lock";
+
+	private static final String JOB_MANAGER_LEADER_PATH = "/job_manager_lock";
+
+	// ------------------------------------------------------------------------
 
 	/** The ZooKeeper client to use */
 	private final CuratorFramework client;
@@ -41,29 +92,37 @@ public class ZookeeperHaServices implements HighAvailabilityServices {
 	/** The runtime configuration */
 	private final Configuration configuration;
 
+	public ZookeeperHaServices(Configuration configuration) {
+		this(ZooKeeperUtils.startCuratorFramework(configuration), configuration);
+	}
+
 	public ZookeeperHaServices(final CuratorFramework client, final Configuration configuration) {
-		this.client = client;
-		this.configuration = configuration;
+		this.client = checkNotNull(client);
+		this.configuration = checkNotNull(configuration);
 	}
 
+	// ------------------------------------------------------------------------
+	//  Services
+	// ------------------------------------------------------------------------
+
 	@Override
 	public LeaderRetrievalService getResourceManagerLeaderRetriever() throws Exception {
-		return ZooKeeperUtils.createLeaderRetrievalService(client, configuration, DEFAULT_RESOURCE_MANAGER_PATH_SUFFIX);
+		return ZooKeeperUtils.createLeaderRetrievalService(client, configuration, RESOURCE_MANAGER_LEADER_PATH);
 	}
 
 	@Override
 	public LeaderRetrievalService getJobManagerLeaderRetriever(JobID jobID) throws Exception {
-		return ZooKeeperUtils.createLeaderRetrievalService(client, configuration, getPathSuffixForJob(jobID));
+		return ZooKeeperUtils.createLeaderRetrievalService(client, configuration, getPathForJobManager(jobID));
 	}
 
 	@Override
 	public LeaderElectionService getResourceManagerLeaderElectionService() throws Exception {
-		return ZooKeeperUtils.createLeaderElectionService(client, configuration, DEFAULT_RESOURCE_MANAGER_PATH_SUFFIX);
+		return ZooKeeperUtils.createLeaderElectionService(client, configuration, RESOURCE_MANAGER_LEADER_PATH);
 	}
 
 	@Override
 	public LeaderElectionService getJobManagerLeaderElectionService(JobID jobID) throws Exception {
-		return ZooKeeperUtils.createLeaderElectionService(client, configuration, getPathSuffixForJob(jobID));
+		return ZooKeeperUtils.createLeaderElectionService(client, configuration, getPathForJobManager(jobID));
 	}
 
 	@Override
@@ -76,7 +135,43 @@ public class ZookeeperHaServices implements HighAvailabilityServices {
 		return ZooKeeperUtils.createSubmittedJobGraphs(client, configuration);
 	}
 
-	private static String getPathSuffixForJob(final JobID jobID) {
-		return String.format("/job-managers/%s", jobID);
+	@Override
+	public RunningJobsRegistry getRunningJobsRegistry() throws Exception {
+		throw new UnsupportedOperationException("not yet implemented");
+	}
+
+	@Override
+	public BlobStore createBlobStore() throws IOException {
+		final String storagePath = configuration.getValue(HighAvailabilityOptions.HA_STORAGE_PATH);
+		if (isNullOrWhitespaceOnly(storagePath)) {
+			throw new IllegalConfigurationException("Configuration is missing the mandatory parameter: " +
+					HighAvailabilityOptions.HA_STORAGE_PATH);
+		}
+
+		final Path path;
+		try {
+			path = new Path(storagePath);
+		} catch (Exception e) {
+			throw new IOException("Invalid path for highly available storage (" +
+					HighAvailabilityOptions.HA_STORAGE_PATH.key() + ')', e);
+		}
+
+		final FileSystem fileSystem;
+		try {
+			fileSystem = path.getFileSystem();
+		} catch (Exception e) {
+			throw new IOException("Could not create FileSystem for highly available storage (" +
+					HighAvailabilityOptions.HA_STORAGE_PATH.key() + ')', e);
+		}
+
+		return new FileSystemBlobStore(fileSystem, storagePath);
+	}
+
+	// ------------------------------------------------------------------------
+	//  Utilities
+	// ------------------------------------------------------------------------
+
+	private static String getPathForJobManager(final JobID jobID) {
+		return "/" + jobID + JOB_MANAGER_LEADER_PATH;
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/0615b62f/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/0615b62f/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/0615b62f/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/0615b62f/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()));
 	}
 }


[23/50] [abbrv] flink git commit: [FLINK-4535] rebase and refine

Posted by se...@apache.org.
[FLINK-4535] rebase and refine


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

Branch: refs/heads/flip-6
Commit: 485ef0035fe3f0d4335d880868ab9beb18731fdf
Parents: c9764c8
Author: Maximilian Michels <mx...@apache.org>
Authored: Wed Sep 21 20:20:25 2016 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Oct 14 15:14:41 2016 +0200

----------------------------------------------------------------------
 .../resourcemanager/JobMasterRegistration.java  |  64 ----
 .../resourcemanager/ResourceManager.java        | 322 ++++++++++++-------
 .../resourcemanager/ResourceManagerGateway.java |  36 +--
 .../TaskExecutorRegistration.java               |   2 +-
 .../slotmanager/SlotManager.java                |   1 -
 .../ResourceManagerJobMasterTest.java           | 174 ++++++++++
 .../ResourceManagerTaskExecutorTest.java        | 135 ++++++++
 .../resourcemanager/ResourceManagerTest.java    | 141 --------
 .../slotmanager/SlotProtocolTest.java           |  43 ++-
 9 files changed, 574 insertions(+), 344 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/485ef003/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/JobMasterRegistration.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/JobMasterRegistration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/JobMasterRegistration.java
deleted file mode 100644
index 981441f..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/JobMasterRegistration.java
+++ /dev/null
@@ -1,64 +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.api.common.JobID;
-import org.apache.flink.runtime.jobmaster.JobMasterGateway;
-import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener;
-
-import java.util.UUID;
-
-/**
- * This class is responsible for group the JobMasterGateway and the LeaderSessionID of a registered job master
- */
-public class JobMasterRegistration implements LeaderRetrievalListener {
-
-	private final JobMasterGateway gateway;
-	private final JobID jobID;
-	private final UUID leaderSessionID;
-	private LeaderRetrievalListener retriever;
-
-	public JobMasterRegistration(JobMasterGateway gateway, JobID jobID, UUID leaderSessionID) {
-		this.gateway = gateway;
-		this.jobID = jobID;
-		this.leaderSessionID = leaderSessionID;
-	}
-
-	public JobMasterGateway getGateway() {
-		return gateway;
-	}
-
-	public UUID getLeaderSessionID() {
-		return leaderSessionID;
-	}
-
-	public JobID getJobID() {
-		return jobID;
-	}
-
-	@Override
-	public void notifyLeaderAddress(String leaderAddress, UUID leaderSessionID) {
-		
-	}
-
-	@Override
-	public void handleError(Exception exception) {
-
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/485ef003/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 15692b6..88b8a11 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
@@ -18,29 +18,41 @@
 
 package org.apache.flink.runtime.resourcemanager;
 
-import akka.dispatch.Futures;
-import akka.dispatch.Mapper;
-
 import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.JobID;
 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.highavailability.HighAvailabilityServices;
 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.resourcemanager.slotmanager.SlotManager;
 import org.apache.flink.runtime.rpc.RpcMethod;
 import org.apache.flink.runtime.rpc.RpcEndpoint;
 import org.apache.flink.runtime.rpc.RpcService;
 import org.apache.flink.runtime.jobmaster.JobMaster;
 import org.apache.flink.runtime.jobmaster.JobMasterGateway;
-import org.apache.flink.runtime.rpc.exceptions.LeaderSessionIDException;
+import org.apache.flink.runtime.registration.RegistrationResponse;
+
 import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway;
 import org.apache.flink.runtime.taskexecutor.TaskExecutorRegistrationSuccess;
-import org.apache.flink.runtime.registration.RegistrationResponse;
-import scala.concurrent.Future;
+import org.apache.flink.runtime.util.LeaderConnectionInfo;
+import org.apache.flink.runtime.util.LeaderRetrievalUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.concurrent.duration.FiniteDuration;
 
 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.TimeUnit;
 
 import static org.apache.flink.util.Preconditions.checkNotNull;
 
@@ -50,25 +62,38 @@ import static org.apache.flink.util.Preconditions.checkNotNull;
  *
  * It offers the following methods as part of its rpc interface to interact with the him remotely:
  * <ul>
- *     <li>{@link #registerJobMaster(JobMasterRegistration)} registers a {@link JobMaster} at the resource manager</li>
+ *     <li>{@link #registerJobMaster(UUID, UUID, String, JobID)} registers a {@link JobMaster} at the resource manager</li>
  *     <li>{@link #requestSlot(SlotRequest)} requests a slot from the resource manager</li>
  * </ul>
  */
-public class ResourceManager extends RpcEndpoint<ResourceManagerGateway> {
-	private final Map<JobMasterGateway, InstanceID> jobMasterGateways;
+public class ResourceManager extends RpcEndpoint<ResourceManagerGateway> implements LeaderContender {
+
+	private final Logger LOG = LoggerFactory.getLogger(getClass());
 
-	/** ResourceID and TaskExecutorRegistration mapping relationship of registered taskExecutors */
-	private final Map<ResourceID, TaskExecutorRegistration>  startedTaskExecutorGateways;
+	private final Map<JobID, JobMasterGateway> jobMasterGateways;
+
+	private final Set<LeaderRetrievalListener> jobMasterLeaderRetrievalListeners;
+
+	private final Map<ResourceID, TaskExecutorRegistration> taskExecutorGateways;
 
 	private final HighAvailabilityServices highAvailabilityServices;
-	private LeaderElectionService leaderElectionService = null;
-	private UUID leaderSessionID = null;
 
-	public ResourceManager(RpcService rpcService, HighAvailabilityServices highAvailabilityServices) {
+	private LeaderElectionService leaderElectionService;
+
+	private final SlotManager slotManager;
+
+	private UUID leaderSessionID;
+
+	public ResourceManager(
+			RpcService rpcService,
+			HighAvailabilityServices highAvailabilityServices,
+			SlotManager slotManager) {
 		super(rpcService);
 		this.highAvailabilityServices = checkNotNull(highAvailabilityServices);
-		this.jobMasterGateways = new HashMap<>(16);
-		this.startedTaskExecutorGateways = new HashMap<>(16);
+		this.jobMasterGateways = new HashMap<>();
+		this.slotManager = slotManager;
+		this.jobMasterLeaderRetrievalListeners = new HashSet<>();
+		this.taskExecutorGateways = new HashMap<>();
 	}
 
 	@Override
@@ -77,7 +102,7 @@ public class ResourceManager extends RpcEndpoint<ResourceManagerGateway> {
 		try {
 			super.start();
 			leaderElectionService = highAvailabilityServices.getResourceManagerLeaderElectionService();
-			leaderElectionService.start(new ResourceManagerLeaderContender());
+			leaderElectionService.start(this);
 		} 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);
@@ -88,8 +113,11 @@ public class ResourceManager extends RpcEndpoint<ResourceManagerGateway> {
 	public void shutDown() {
 		try {
 			leaderElectionService.stop();
+			for(JobID jobID : jobMasterGateways.keySet()) {
+				highAvailabilityServices.getJobMasterLeaderRetriever(jobID).stop();
+			}
 			super.shutDown();
-		} catch(Throwable e) {
+		} 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);
 		}
@@ -102,48 +130,79 @@ public class ResourceManager extends RpcEndpoint<ResourceManagerGateway> {
 	 */
 	@VisibleForTesting
 	UUID getLeaderSessionID() {
-		return leaderSessionID;
+		return this.leaderSessionID;
 	}
 
 	/**
 	 * Register a {@link JobMaster} at the resource manager.
 	 *
-	 * @param jobMasterRegistration Job master registration information
+	 * @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(JobMasterRegistration jobMasterRegistration) {
-		Future<JobMasterGateway> jobMasterFuture = getRpcService().connect(jobMasterRegistration.getAddress(), JobMasterGateway.class);
+	public Future<RegistrationResponse> registerJobMaster(
+		final UUID resourceManagerLeaderId, final UUID jobMasterLeaderId,
+		final String jobMasterAddress, final JobID jobID) {
 
-		return jobMasterFuture.map(new Mapper<JobMasterGateway, RegistrationResponse>() {
-			@Override
-			public RegistrationResponse apply(final JobMasterGateway jobMasterGateway) {
-				InstanceID instanceID;
+		checkNotNull(jobMasterAddress);
+		checkNotNull(jobID);
 
-				if (jobMasterGateways.containsKey(jobMasterGateway)) {
-					instanceID = jobMasterGateways.get(jobMasterGateway);
-				} else {
-					instanceID = new InstanceID();
-					jobMasterGateways.put(jobMasterGateway, instanceID);
-				}
+		return getRpcService()
+			.execute(new Callable<JobMasterGateway>() {
+				@Override
+				public JobMasterGateway call() throws Exception {
 
-				return new TaskExecutorRegistrationSuccess(instanceID, 5000);
-			}
-		}, getMainThreadExecutionContext());
-	}
+					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");
+					}
 
-	/**
-	 * Requests a slot from the resource manager.
-	 *
-	 * @param slotRequest Slot request
-	 * @return Slot assignment
-	 */
-	@RpcMethod
-	public SlotAssignment requestSlot(SlotRequest slotRequest) {
-		System.out.println("SlotRequest: " + slotRequest);
-		return new SlotAssignment();
-	}
+					final LeaderConnectionInfo jobMasterLeaderInfo;
+					try {
+						jobMasterLeaderInfo = LeaderRetrievalUtils.retrieveLeaderConnectionInfo(
+							highAvailabilityServices.getJobMasterLeaderRetriever(jobID), new FiniteDuration(5, TimeUnit.SECONDS));
+					} catch (Exception e) {
+						LOG.warn("Failed to start JobMasterLeaderRetriever for JobID {}", jobID);
+						throw new Exception("Failed to retrieve JobMasterLeaderRetriever");
+					}
+
+					if (!jobMasterLeaderId.equals(jobMasterLeaderInfo.getLeaderSessionID())) {
+						LOG.info("Declining registration request from non-leading JobManager {}", jobMasterAddress);
+						throw new Exception("JobManager is not leading");
+					}
 
+					return getRpcService().connect(jobMasterAddress, JobMasterGateway.class).get(5, TimeUnit.SECONDS);
+				}
+			})
+			.handleAsync(new BiFunction<JobMasterGateway, Throwable, RegistrationResponse>() {
+				@Override
+				public RegistrationResponse apply(JobMasterGateway jobMasterGateway, Throwable throwable) {
+					
+					if (throwable != null) {
+						return new RegistrationResponse.Decline(throwable.getMessage());
+					} else {
+						JobMasterLeaderListener jobMasterLeaderListener = new JobMasterLeaderListener(jobID);
+						try {
+							LeaderRetrievalService jobMasterLeaderRetriever = highAvailabilityServices.getJobMasterLeaderRetriever(jobID);
+							jobMasterLeaderRetriever.start(jobMasterLeaderListener);
+						} catch (Exception e) {
+							LOG.warn("Failed to start JobMasterLeaderRetriever for JobID {}", jobID);
+							return new RegistrationResponse.Decline("Failed to retrieve JobMasterLeaderRetriever");
+						}
+						jobMasterLeaderRetrievalListeners.add(jobMasterLeaderListener);
+						final JobMasterGateway existingGateway = jobMasterGateways.put(jobID, jobMasterGateway);
+						if (existingGateway != null) {
+							log.info("Replacing gateway for registered JobID {}.", jobID);
+						}
+						return new JobMasterRegistrationSuccess(5000);
+					}
+				}
+			}, getMainThreadExecutor());
+	}
 
 	/**
 	 * Register a {@link org.apache.flink.runtime.taskexecutor.TaskExecutor} at the resource manager
@@ -160,90 +219,129 @@ public class ResourceManager extends RpcEndpoint<ResourceManagerGateway> {
 		final String taskExecutorAddress,
 		final ResourceID resourceID) {
 
-		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);
-			return Futures.failed(new LeaderSessionIDException(leaderSessionID, resourceManagerLeaderId));
+		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(5, TimeUnit.SECONDS);
+			}
+		}).handleAsync(new BiFunction<TaskExecutorGateway, Throwable, RegistrationResponse>() {
+			@Override
+			public RegistrationResponse apply(TaskExecutorGateway taskExecutorGateway, Throwable throwable) {
+				if (throwable != null) {
+					return new RegistrationResponse.Decline(throwable.getMessage());
+				} else {
+					InstanceID id = new InstanceID();
+					TaskExecutorRegistration oldTaskExecutor =
+						taskExecutorGateways.put(resourceID, new TaskExecutorRegistration(taskExecutorGateway, id));
+					if (oldTaskExecutor != null) {
+						log.warn("Receive a duplicate registration from TaskExecutor {} at ({})", resourceID, taskExecutorAddress);
+					}
+					return new TaskExecutorRegistrationSuccess(id, 5000);
+				}
+			}
+		}, getMainThreadExecutor());
+	}
+
+	/**
+	 * Requests a slot from the resource manager.
+	 *
+	 * @param slotRequest Slot request
+	 * @return Slot assignment
+	 */
+	@RpcMethod
+	public SlotRequestReply requestSlot(SlotRequest slotRequest) {
+		final JobID jobId = slotRequest.getJobId();
+		final JobMasterGateway jobMasterGateway = jobMasterGateways.get(jobId);
+
+		if (jobMasterGateway != null) {
+			return slotManager.requestSlot(slotRequest);
+		} else {
+			LOG.info("Ignoring slot request for unknown JobMaster with JobID {}", jobId);
+			return new SlotRequestRejected(slotRequest.getAllocationId());
 		}
+	}
 
-		Future<TaskExecutorGateway> taskExecutorGatewayFuture = getRpcService().connect(taskExecutorAddress, TaskExecutorGateway.class);
 
-		return taskExecutorGatewayFuture.map(new Mapper<TaskExecutorGateway, RegistrationResponse>() {
 
+
+	// ------------------------------------------------------------------------
+	//  Leader Contender
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Callback method when current resourceManager is granted leadership
+	 *
+	 * @param leaderSessionID unique leadershipID
+	 */
+	@Override
+	public void grantLeadership(final UUID leaderSessionID) {
+		runAsync(new Runnable() {
 			@Override
-			public RegistrationResponse apply(final TaskExecutorGateway taskExecutorGateway) {
-				InstanceID instanceID = null;
-				TaskExecutorRegistration taskExecutorRegistration = startedTaskExecutorGateways.get(resourceID);
-				if(taskExecutorRegistration != null) {
-					log.warn("Receive a duplicate registration from TaskExecutor {} at ({})", resourceID, taskExecutorAddress);
-					instanceID = taskExecutorRegistration.getInstanceID();
-				} else {
-					instanceID = new InstanceID();
-					startedTaskExecutorGateways.put(resourceID, new TaskExecutorRegistration(taskExecutorGateway, instanceID));
-				}
+			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);
+				// notify SlotManager
+				slotManager.setLeaderUUID(leaderSessionID);
+				ResourceManager.this.leaderSessionID = leaderSessionID;
+			}
+		});
+	}
 
-				return new TaskExecutorRegistrationSuccess(instanceID, 5000);
+	/**
+	 * Callback method when current resourceManager lose leadership.
+	 */
+	@Override
+	public void revokeLeadership() {
+		runAsync(new Runnable() {
+			@Override
+			public void run() {
+				log.info("ResourceManager {} was revoked leadership.", getAddress());
+				jobMasterGateways.clear();
+				taskExecutorGateways.clear();
+				slotManager.clearState();
+				leaderSessionID = null;
 			}
-		}, getMainThreadExecutionContext());
+		});
 	}
 
+	/**
+	 * Handles error occurring in the leader election service
+	 *
+	 * @param exception Exception being thrown in the leader election service
+	 */
+	@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();
+	}
 
-	private class ResourceManagerLeaderContender implements LeaderContender {
+	private static class JobMasterLeaderListener implements LeaderRetrievalListener {
 
-		/**
-		 * Callback method when current resourceManager is granted leadership
-		 *
-		 * @param leaderSessionID unique leadershipID
-		 */
-		@Override
-		public void grantLeadership(final UUID leaderSessionID) {
-			runAsync(new Runnable() {
-				@Override
-				public void run() {
-					log.info("ResourceManager {} was granted leadership with leader session ID {}", getAddress(), leaderSessionID);
-					ResourceManager.this.leaderSessionID = leaderSessionID;
-					// confirming the leader session ID might be blocking,
-					leaderElectionService.confirmLeaderSessionID(leaderSessionID);
-				}
-			});
-		}
+		private final JobID jobID;
+		private UUID leaderID;
 
-		/**
-		 * Callback method when current resourceManager lose leadership.
-		 */
-		@Override
-		public void revokeLeadership() {
-			runAsync(new Runnable() {
-				@Override
-				public void run() {
-					log.info("ResourceManager {} was revoked leadership.", getAddress());
-					jobMasterGateways.clear();
-					startedTaskExecutorGateways.clear();
-					leaderSessionID = null;
-				}
-			});
+		private JobMasterLeaderListener(JobID jobID) {
+			this.jobID = jobID;
 		}
 
 		@Override
-		public String getAddress() {
-			return ResourceManager.this.getAddress();
+		public void notifyLeaderAddress(final String leaderAddress, final UUID leaderSessionID) {
+			this.leaderID = leaderSessionID;
 		}
 
-		/**
-		 * Handles error occurring in the leader election service
-		 *
-		 * @param exception Exception being thrown in the leader election service
-		 */
 		@Override
 		public void handleError(final Exception exception) {
-			runAsync(new Runnable() {
-				@Override
-				public void run() {
-					log.error("ResourceManager received an error from the LeaderElectionService.", exception);
-					// terminate ResourceManager in case of an error
-					shutDown();
-				}
-			});
+			// TODO
 		}
 	}
 }
+

http://git-wip-us.apache.org/repos/asf/flink/blob/485ef003/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 30a096f..d8b8ebe 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
@@ -18,15 +18,16 @@
 
 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.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.concurrent.Future;
 import org.apache.flink.runtime.rpc.RpcGateway;
 import org.apache.flink.runtime.rpc.RpcTimeout;
 import org.apache.flink.runtime.jobmaster.JobMaster;
-
 import org.apache.flink.runtime.registration.RegistrationResponse;
-import scala.concurrent.Future;
-import scala.concurrent.duration.FiniteDuration;
 
+import org.apache.flink.runtime.registration.RegistrationResponse;
 import java.util.UUID;
 
 /**
@@ -37,21 +38,18 @@ public interface ResourceManagerGateway extends RpcGateway {
 	/**
 	 * Register a {@link JobMaster} at the resource manager.
 	 *
-	 * @param jobMasterRegistration Job master registration information
-	 * @param timeout Timeout for the future to complete
+	 * @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
+	 * @param timeout                 Timeout for the future to complete
 	 * @return Future registration response
 	 */
 	Future<RegistrationResponse> registerJobMaster(
-		JobMasterRegistration jobMasterRegistration,
-		@RpcTimeout FiniteDuration timeout);
+		UUID resourceManagerLeaderId,
+		String jobMasterAddress,
+		JobID jobID,
+		@RpcTimeout Time timeout);
 
-	/**
-	 * Register a {@link JobMaster} at the resource manager.
-	 *
-	 * @param jobMasterRegistration Job master registration information
-	 * @return Future registration response
-	 */
-	Future<RegistrationResponse> registerJobMaster(JobMasterRegistration jobMasterRegistration);
 
 	/**
 	 * Requests a slot from the resource manager.
@@ -59,15 +57,15 @@ public interface ResourceManagerGateway extends RpcGateway {
 	 * @param slotRequest Slot request
 	 * @return Future slot assignment
 	 */
-	Future<SlotAssignment> requestSlot(SlotRequest slotRequest);
+	Future<SlotRequestReply> requestSlot(SlotRequest slotRequest);
 
 	/**
 	 * Register a {@link org.apache.flink.runtime.taskexecutor.TaskExecutor} at the resource manager.
 	 *
 	 * @param resourceManagerLeaderId  The fencing token for the ResourceManager leader
-	 * @param taskExecutorAddress      The address of the TaskExecutor that registers
-	 * @param resourceID               The resource ID of the TaskExecutor that registers
-	 * @param timeout                  The timeout for the response.
+	 * @param taskExecutorAddress     The address of the TaskExecutor that registers
+	 * @param resourceID              The resource ID of the TaskExecutor that registers
+	 * @param timeout                 The timeout for the response.
 	 *
 	 * @return The future to the response by the ResourceManager.
 	 */
@@ -75,5 +73,5 @@ public interface ResourceManagerGateway extends RpcGateway {
 		UUID resourceManagerLeaderId,
 		String taskExecutorAddress,
 		ResourceID resourceID,
-		@RpcTimeout FiniteDuration timeout);
+		@RpcTimeout Time timeout);
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/485ef003/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/TaskExecutorRegistration.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/TaskExecutorRegistration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/TaskExecutorRegistration.java
index bd78a47..f8dfdc7 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/TaskExecutorRegistration.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/TaskExecutorRegistration.java
@@ -35,7 +35,7 @@ public class TaskExecutorRegistration implements Serializable {
 	private InstanceID instanceID;
 
 	public TaskExecutorRegistration(TaskExecutorGateway taskExecutorGateway,
-		InstanceID instanceID) {
+									InstanceID instanceID) {
 		this.taskExecutorGateway = taskExecutorGateway;
 		this.instanceID = instanceID;
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/485ef003/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManager.java
index 5d0013c..a6d2196 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManager.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManager.java
@@ -26,7 +26,6 @@ import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
 import org.apache.flink.runtime.clusterframework.types.ResourceSlot;
 import org.apache.flink.runtime.clusterframework.types.SlotID;
 import org.apache.flink.runtime.concurrent.Future;
-import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener;
 import org.apache.flink.runtime.resourcemanager.SlotRequest;
 import org.apache.flink.runtime.resourcemanager.SlotRequestRegistered;
 import org.apache.flink.runtime.resourcemanager.SlotRequestReply;

http://git-wip-us.apache.org/repos/asf/flink/blob/485ef003/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
new file mode 100644
index 0000000..332c093
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerJobMasterTest.java
@@ -0,0 +1,174 @@
+/*
+ * 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.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.resourcemanager.slotmanager.SimpleSlotManager;
+import org.apache.flink.runtime.rpc.TestingSerialRpcService;
+import org.apache.flink.runtime.registration.RegistrationResponse;
+import org.apache.flink.runtime.rpc.exceptions.LeaderSessionIDException;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.*;
+
+public class ResourceManagerJobMasterTest {
+
+	private TestingSerialRpcService rpcService;
+
+	@Before
+	public void setup() throws Exception {
+		rpcService = new TestingSerialRpcService();
+	}
+
+	@After
+	public void teardown() throws Exception {
+		rpcService.stopService();
+	}
+
+	/**
+	 * Test receive normal registration from job master and receive duplicate registration from job master
+	 */
+	@Test
+	public void testRegisterJobMaster() throws Exception {
+		String jobMasterAddress = "/jobMasterAddress1";
+		JobID jobID = mockJobMaster(jobMasterAddress);
+		TestingLeaderElectionService resourceManagerLeaderElectionService = new TestingLeaderElectionService();
+		UUID jmLeaderID = UUID.randomUUID();
+		TestingLeaderRetrievalService jobMasterLeaderRetrievalService = new TestingLeaderRetrievalService(jobMasterAddress, jmLeaderID);
+		final ResourceManager resourceManager = createAndStartResourceManager(resourceManagerLeaderElectionService, jobID, jobMasterLeaderRetrievalService);
+		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);
+	}
+
+	/**
+	 * Test receive registration with unmatched leadershipId from job master
+	 */
+	@Test
+	public void testRegisterJobMasterWithUnmatchedLeaderSessionId1() throws Exception {
+		String jobMasterAddress = "/jobMasterAddress1";
+		JobID jobID = mockJobMaster(jobMasterAddress);
+		TestingLeaderElectionService resourceManagerLeaderElectionService = new TestingLeaderElectionService();
+		UUID jmLeaderID = UUID.randomUUID();
+		TestingLeaderRetrievalService jobMasterLeaderRetrievalService = new TestingLeaderRetrievalService(jobMasterAddress, jmLeaderID);
+		final ResourceManager resourceManager = createAndStartResourceManager(resourceManagerLeaderElectionService, jobID, jobMasterLeaderRetrievalService);
+		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);
+	}
+
+	/**
+	 * Test receive registration with unmatched leadershipId from job master
+	 */
+	@Test
+	public void testRegisterJobMasterWithUnmatchedLeaderSessionId2() throws Exception {
+		String jobMasterAddress = "/jobMasterAddress1";
+		JobID jobID = mockJobMaster(jobMasterAddress);
+		TestingLeaderElectionService resourceManagerLeaderElectionService = new TestingLeaderElectionService();
+		TestingLeaderRetrievalService jobMasterLeaderRetrievalService = new TestingLeaderRetrievalService();
+		final ResourceManager resourceManager = createAndStartResourceManager(resourceManagerLeaderElectionService, jobID, jobMasterLeaderRetrievalService);
+		final UUID rmLeaderSessionId = grantResourceManagerLeadership(resourceManagerLeaderElectionService);
+		final UUID jmLeaderSessionId = 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(rmLeaderSessionId, differentLeaderSessionID, jobMasterAddress, jobID);
+		assertTrue(unMatchedLeaderFuture.get(5, TimeUnit.SECONDS) instanceof RegistrationResponse.Decline);
+	}
+
+	/**
+	 * Test receive registration with invalid address from job master
+	 */
+	@Test
+	public void testRegisterJobMasterFromInvalidAddress() throws Exception {
+		String jobMasterAddress = "/jobMasterAddress1";
+		JobID jobID = mockJobMaster(jobMasterAddress);
+		TestingLeaderElectionService resourceManagerLeaderElectionService = new TestingLeaderElectionService();
+		TestingLeaderRetrievalService jobMasterLeaderRetrievalService = new TestingLeaderRetrievalService();
+		final ResourceManager resourceManager = createAndStartResourceManager(resourceManagerLeaderElectionService, jobID, jobMasterLeaderRetrievalService);
+		final UUID rmLeaderSessionId = grantResourceManagerLeadership(resourceManagerLeaderElectionService);
+		final UUID jmLeaderSessionId = grantResourceManagerLeadership(resourceManagerLeaderElectionService);
+
+		// 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);
+		assertTrue(invalidAddressFuture.get(5, TimeUnit.SECONDS) instanceof RegistrationResponse.Decline);
+	}
+
+	/**
+	 * Check and verify return RegistrationResponse.Decline when failed to start a job master Leader retrieval listener
+	 */
+	@Test
+	public void testRegisterJobMasterWithFailureLeaderListener() throws Exception {
+		String jobMasterAddress = "/jobMasterAddress1";
+		JobID jobID = mockJobMaster(jobMasterAddress);
+		TestingLeaderElectionService resourceManagerLeaderElectionService = new TestingLeaderElectionService();
+		TestingLeaderRetrievalService jobMasterLeaderRetrievalService = new TestingLeaderRetrievalService();
+		final ResourceManager resourceManager = createAndStartResourceManager(resourceManagerLeaderElectionService, jobID, jobMasterLeaderRetrievalService);
+		final UUID rmLeaderSessionId = grantResourceManagerLeadership(resourceManagerLeaderElectionService);
+		final UUID jmLeaderSessionId = grantResourceManagerLeadership(resourceManagerLeaderElectionService);
+
+		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);
+		RegistrationResponse response = declineFuture.get(5, TimeUnit.SECONDS);
+		assertTrue(response instanceof RegistrationResponse.Decline);
+	}
+
+	private JobID mockJobMaster(String jobMasterAddress) {
+		JobID jobID = new JobID();
+		JobMasterGateway jobMasterGateway = mock(JobMasterGateway.class);
+		rpcService.registerGateway(jobMasterAddress, jobMasterGateway);
+		return jobID;
+	}
+
+	private ResourceManager createAndStartResourceManager(TestingLeaderElectionService resourceManagerLeaderElectionService, JobID jobID, TestingLeaderRetrievalService jobMasterLeaderRetrievalService) {
+		TestingHighAvailabilityServices highAvailabilityServices = new TestingHighAvailabilityServices();
+		highAvailabilityServices.setResourceManagerLeaderElectionService(resourceManagerLeaderElectionService);
+		highAvailabilityServices.setJobMasterLeaderRetriever(jobID, jobMasterLeaderRetrievalService);
+		ResourceManager resourceManager = new ResourceManager(rpcService, highAvailabilityServices, new SimpleSlotManager());
+		resourceManager.start();
+		return resourceManager;
+	}
+
+	private UUID grantResourceManagerLeadership(TestingLeaderElectionService resourceManagerLeaderElectionService) {
+		UUID leaderSessionId = UUID.randomUUID();
+		resourceManagerLeaderElectionService.isLeader(leaderSessionId);
+		return leaderSessionId;
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/485ef003/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
new file mode 100644
index 0000000..ed7c7d7
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTaskExecutorTest.java
@@ -0,0 +1,135 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager;
+
+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.resourcemanager.slotmanager.SimpleSlotManager;
+import org.apache.flink.runtime.rpc.TestingSerialRpcService;
+import org.apache.flink.runtime.registration.RegistrationResponse;
+import org.apache.flink.runtime.rpc.exceptions.LeaderSessionIDException;
+import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway;
+import org.apache.flink.runtime.taskexecutor.TaskExecutorRegistrationSuccess;
+import org.junit.After;
+import org.junit.Before;
+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;
+
+public class ResourceManagerTaskExecutorTest {
+
+	private TestingSerialRpcService rpcService;
+
+	@Before
+	public void setup() throws Exception {
+		rpcService = new TestingSerialRpcService();
+	}
+
+	@After
+	public void teardown() throws Exception {
+		rpcService.stopService();
+	}
+
+	/**
+	 * Test receive normal registration from task executor and receive duplicate registration from task executor
+	 */
+	@Test
+	public void testRegisterTaskExecutor() throws Exception {
+		String taskExecutorAddress = "/taskExecutor1";
+		ResourceID taskExecutorResourceID = mockTaskExecutor(taskExecutorAddress);
+		TestingLeaderElectionService rmLeaderElectionService = new TestingLeaderElectionService();
+		final ResourceManager resourceManager = createAndStartResourceManager(rmLeaderElectionService);
+		final UUID leaderSessionId = grantLeadership(rmLeaderElectionService);
+
+		// test response successful
+		Future<RegistrationResponse> successfulFuture = resourceManager.registerTaskExecutor(leaderSessionId, taskExecutorAddress, taskExecutorResourceID);
+		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);
+		RegistrationResponse duplicateResponse = duplicateFuture.get();
+		assertTrue(duplicateResponse instanceof TaskExecutorRegistrationSuccess);
+		assertNotEquals(((TaskExecutorRegistrationSuccess) response).getRegistrationId(), ((TaskExecutorRegistrationSuccess) duplicateResponse).getRegistrationId());
+	}
+
+	/**
+	 * Test receive registration with unmatched leadershipId from task executor
+	 */
+	@Test
+	public void testRegisterTaskExecutorWithUnmatchedLeaderSessionId() throws Exception {
+		String taskExecutorAddress = "/taskExecutor1";
+		ResourceID taskExecutorResourceID = mockTaskExecutor(taskExecutorAddress);
+		TestingLeaderElectionService rmLeaderElectionService = new TestingLeaderElectionService();
+		final ResourceManager resourceManager = createAndStartResourceManager(rmLeaderElectionService);
+		final UUID leaderSessionId = grantLeadership(rmLeaderElectionService);
+
+		// 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);
+		assertTrue(unMatchedLeaderFuture.get(5, TimeUnit.SECONDS) instanceof RegistrationResponse.Decline);
+	}
+
+	/**
+	 * Test receive registration with invalid address from task executor
+	 */
+	@Test
+	public void testRegisterTaskExecutorFromInvalidAddress() throws Exception {
+		String taskExecutorAddress = "/taskExecutor1";
+		ResourceID taskExecutorResourceID = mockTaskExecutor(taskExecutorAddress);
+		TestingLeaderElectionService leaderElectionService = new TestingLeaderElectionService();
+		final ResourceManager resourceManager = createAndStartResourceManager(leaderElectionService);
+		final UUID leaderSessionId = grantLeadership(leaderElectionService);
+
+		// test throw exception when receive a registration from taskExecutor which takes invalid address
+		String invalidAddress = "/taskExecutor2";
+		Future<RegistrationResponse> invalidAddressFuture = resourceManager.registerTaskExecutor(leaderSessionId, invalidAddress, taskExecutorResourceID);
+		assertTrue(invalidAddressFuture.get(5, TimeUnit.SECONDS) instanceof RegistrationResponse.Decline);
+	}
+
+	private ResourceID mockTaskExecutor(String taskExecutorAddress) {
+		TaskExecutorGateway taskExecutorGateway = mock(TaskExecutorGateway.class);
+		ResourceID taskExecutorResourceID = ResourceID.generate();
+		rpcService.registerGateway(taskExecutorAddress, taskExecutorGateway);
+		return taskExecutorResourceID;
+	}
+
+	private ResourceManager createAndStartResourceManager(TestingLeaderElectionService rmLeaderElectionService) {
+		TestingHighAvailabilityServices highAvailabilityServices = new TestingHighAvailabilityServices();
+		highAvailabilityServices.setResourceManagerLeaderElectionService(rmLeaderElectionService);
+		ResourceManager resourceManager = new ResourceManager(rpcService, highAvailabilityServices, new SimpleSlotManager());
+		resourceManager.start();
+		return resourceManager;
+	}
+
+	private UUID grantLeadership(TestingLeaderElectionService leaderElectionService) {
+		UUID leaderSessionId = UUID.randomUUID();
+		leaderElectionService.isLeader(leaderSessionId);
+		return leaderSessionId;
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/485ef003/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTest.java
deleted file mode 100644
index b75d9b8..0000000
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTest.java
+++ /dev/null
@@ -1,141 +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.clusterframework.types.ResourceID;
-import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices;
-import org.apache.flink.runtime.leaderelection.TestingLeaderElectionService;
-import org.apache.flink.runtime.rpc.TestingSerialRpcService;
-import org.apache.flink.runtime.registration.RegistrationResponse;
-import org.apache.flink.runtime.rpc.exceptions.LeaderSessionIDException;
-import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway;
-import org.apache.flink.runtime.taskexecutor.TaskExecutorRegistrationSuccess;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-import scala.concurrent.Await;
-import scala.concurrent.Future;
-import scala.concurrent.duration.FiniteDuration;
-
-import java.util.UUID;
-import java.util.concurrent.TimeUnit;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.mockito.Mockito.mock;
-
-public class ResourceManagerTest {
-
-	private TestingSerialRpcService rpcService;
-
-	@Before
-	public void setup() throws Exception {
-		rpcService = new TestingSerialRpcService();
-	}
-
-	@After
-	public void teardown() throws Exception {
-		rpcService.stopService();
-	}
-
-	/**
-	 * Test receive normal registration from task executor and receive duplicate registration from task executor
-	 *
-	 * @throws Exception
-	 */
-	@Test
-	public void testRegisterTaskExecutor() throws Exception {
-		String taskExecutorAddress = "/taskExecutor1";
-		ResourceID taskExecutorResourceID = mockTaskExecutor(taskExecutorAddress);
-		TestingLeaderElectionService leaderElectionService = new TestingLeaderElectionService();
-		final ResourceManager resourceManager = createAndStartResourceManager(leaderElectionService);
-		final UUID leaderSessionId = grantResourceManagerLeadership(leaderElectionService);
-
-		// test response successful
-		Future<RegistrationResponse> successfulFuture = resourceManager.registerTaskExecutor(leaderSessionId, taskExecutorAddress, taskExecutorResourceID);
-		RegistrationResponse response = Await.result(successfulFuture, new FiniteDuration(0, TimeUnit.SECONDS));
-		assertTrue(response instanceof TaskExecutorRegistrationSuccess);
-
-		// test response successful with previous instanceID when receive duplicate registration from taskExecutor
-		Future<RegistrationResponse> duplicateFuture = resourceManager.registerTaskExecutor(leaderSessionId, taskExecutorAddress, taskExecutorResourceID);
-		RegistrationResponse duplicateResponse = Await.result(duplicateFuture, new FiniteDuration(0, TimeUnit.SECONDS));
-		assertTrue(duplicateResponse instanceof TaskExecutorRegistrationSuccess);
-		assertEquals(((TaskExecutorRegistrationSuccess) response).getRegistrationId(), ((TaskExecutorRegistrationSuccess) duplicateResponse).getRegistrationId());
-	}
-
-	/**
-	 * Test receive registration with unmatched leadershipId from task executor
-	 *
-	 * @throws Exception
-	 */
-	@Test(expected = LeaderSessionIDException.class)
-	public void testRegisterTaskExecutorWithUnmatchedLeaderSessionId() throws Exception {
-		String taskExecutorAddress = "/taskExecutor1";
-		ResourceID taskExecutorResourceID = mockTaskExecutor(taskExecutorAddress);
-		TestingLeaderElectionService leaderElectionService = new TestingLeaderElectionService();
-		final ResourceManager resourceManager = createAndStartResourceManager(leaderElectionService);
-		final UUID leaderSessionId = grantResourceManagerLeadership(leaderElectionService);
-
-		// 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);
-		Await.result(unMatchedLeaderFuture, new FiniteDuration(200, TimeUnit.MILLISECONDS));
-	}
-
-	/**
-	 * Test receive registration with invalid address from task executor
-	 *
-	 * @throws Exception
-	 */
-	@Test(expected = Exception.class)
-	public void testRegisterTaskExecutorFromInvalidAddress() throws Exception {
-		String taskExecutorAddress = "/taskExecutor1";
-		ResourceID taskExecutorResourceID = mockTaskExecutor(taskExecutorAddress);
-		TestingLeaderElectionService leaderElectionService = new TestingLeaderElectionService();
-		final ResourceManager resourceManager = createAndStartResourceManager(leaderElectionService);
-		final UUID leaderSessionId = grantResourceManagerLeadership(leaderElectionService);
-
-		// test throw exception when receive a registration from taskExecutor which takes invalid address
-		String invalidAddress = "/taskExecutor2";
-		Future<RegistrationResponse> invalidAddressFuture = resourceManager.registerTaskExecutor(leaderSessionId, invalidAddress, taskExecutorResourceID);
-		Await.result(invalidAddressFuture, new FiniteDuration(200, TimeUnit.MILLISECONDS));
-	}
-
-	private ResourceID mockTaskExecutor(String taskExecutorAddress) {
-		TaskExecutorGateway taskExecutorGateway = mock(TaskExecutorGateway.class);
-		ResourceID taskExecutorResourceID = ResourceID.generate();
-		rpcService.registerGateway(taskExecutorAddress, taskExecutorGateway);
-		return taskExecutorResourceID;
-	}
-
-	private ResourceManager createAndStartResourceManager(TestingLeaderElectionService leaderElectionService) {
-		TestingHighAvailabilityServices highAvailabilityServices = new TestingHighAvailabilityServices();
-		highAvailabilityServices.setResourceManagerLeaderElectionService(leaderElectionService);
-		ResourceManager resourceManager = new ResourceManager(rpcService, highAvailabilityServices);
-		resourceManager.start();
-		return resourceManager;
-	}
-
-	private UUID grantResourceManagerLeadership(TestingLeaderElectionService leaderElectionService) {
-		UUID leaderSessionId = UUID.randomUUID();
-		leaderElectionService.isLeader(leaderSessionId);
-		return leaderSessionId;
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/485ef003/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 1f9e7e8..0232fab 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
@@ -24,10 +24,14 @@ 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.concurrent.Future;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
 import org.apache.flink.runtime.highavailability.NonHaServices;
+import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices;
 import org.apache.flink.runtime.jobmaster.JobMasterGateway;
-import org.apache.flink.runtime.resourcemanager.JobMasterRegistration;
-import org.apache.flink.runtime.resourcemanager.RegistrationResponse;
+import org.apache.flink.runtime.leaderelection.LeaderElectionService;
+import org.apache.flink.runtime.leaderelection.TestingLeaderElectionService;
+import org.apache.flink.runtime.leaderelection.TestingLeaderRetrievalService;
+import org.apache.flink.runtime.registration.RegistrationResponse;
 import org.apache.flink.runtime.resourcemanager.ResourceManager;
 import org.apache.flink.runtime.resourcemanager.SlotRequest;
 import org.apache.flink.runtime.resourcemanager.SlotRequestReply;
@@ -88,14 +92,20 @@ public class SlotProtocolTest extends TestLogger {
 
 		testRpcService.registerGateway(jmAddress, mock(JobMasterGateway.class));
 
+		final TestingHighAvailabilityServices testingHaServices = new TestingHighAvailabilityServices();
+		final UUID rmLeaderID = UUID.randomUUID();
+		final UUID jmLeaderID = UUID.randomUUID();
+		TestingLeaderElectionService rmLeaderElectionService =
+			configureHA(testingHaServices, jobID, rmAddress, rmLeaderID, jmAddress, jmLeaderID);
 
 		TestingSlotManager slotManager = Mockito.spy(new TestingSlotManager());
 		ResourceManager resourceManager =
-			new ResourceManager(testRpcService, new NonHaServices(rmAddress), slotManager);
+			new ResourceManager(testRpcService, testingHaServices, slotManager);
 		resourceManager.start();
+		rmLeaderElectionService.isLeader(rmLeaderID);
 
 		Future<RegistrationResponse> registrationFuture =
-			resourceManager.registerJobMaster(new JobMasterRegistration(jmAddress, jobID));
+			resourceManager.registerJobMaster(rmLeaderID, jmLeaderID, jmAddress, jobID);
 		try {
 			registrationFuture.get(5, TimeUnit.SECONDS);
 		} catch (Exception e) {
@@ -158,16 +168,23 @@ public class SlotProtocolTest extends TestLogger {
 
 		testRpcService.registerGateway(jmAddress, mock(JobMasterGateway.class));
 
+		final TestingHighAvailabilityServices testingHaServices = new TestingHighAvailabilityServices();
+		final UUID rmLeaderID = UUID.randomUUID();
+		final UUID jmLeaderID = UUID.randomUUID();
+		TestingLeaderElectionService rmLeaderElectionService =
+			configureHA(testingHaServices, jobID, rmAddress, rmLeaderID, jmAddress, jmLeaderID);
+
 		TaskExecutorGateway taskExecutorGateway = mock(TaskExecutorGateway.class);
 		testRpcService.registerGateway(tmAddress, taskExecutorGateway);
 
 		TestingSlotManager slotManager = Mockito.spy(new TestingSlotManager());
 		ResourceManager resourceManager =
-			new ResourceManager(testRpcService, new NonHaServices(rmAddress), slotManager);
+			new ResourceManager(testRpcService, testingHaServices, slotManager);
 		resourceManager.start();
+		rmLeaderElectionService.isLeader(rmLeaderID);
 
 		Future<RegistrationResponse> registrationFuture =
-			resourceManager.registerJobMaster(new JobMasterRegistration(jmAddress, jobID));
+			resourceManager.registerJobMaster(rmLeaderID, jmLeaderID, jmAddress, jobID);
 		try {
 			registrationFuture.get(5, TimeUnit.SECONDS);
 		} catch (Exception e) {
@@ -208,6 +225,20 @@ public class SlotProtocolTest extends TestLogger {
 		verify(taskExecutorGateway, timeout(5000)).requestSlot(eq(allocationID), any(UUID.class), any(Time.class));
 	}
 
+	private static TestingLeaderElectionService configureHA(
+			TestingHighAvailabilityServices testingHA, JobID jobID, String rmAddress, UUID rmID, String jmAddress, UUID jmID) {
+		final TestingLeaderElectionService rmLeaderElectionService = new TestingLeaderElectionService();
+		testingHA.setResourceManagerLeaderElectionService(rmLeaderElectionService);
+		final TestingLeaderRetrievalService rmLeaderRetrievalService = new TestingLeaderRetrievalService(rmAddress, rmID);
+		testingHA.setResourceManagerLeaderRetriever(rmLeaderRetrievalService);
+
+		final TestingLeaderElectionService jmLeaderElectionService = new TestingLeaderElectionService();
+		testingHA.setJobMasterLeaderElectionService(jmLeaderElectionService);
+		final TestingLeaderRetrievalService jmLeaderRetrievalService = new TestingLeaderRetrievalService(jmAddress, jmID);
+		testingHA.setJobMasterLeaderRetriever(jobID, jmLeaderRetrievalService);
+
+		return rmLeaderElectionService;
+	}
 
 	private static class TestingSlotManager extends SimpleSlotManager {
 


[02/50] [abbrv] flink git commit: [FLINK-4529] [flip-6] Move TaskExecutor, JobMaster and ResourceManager out of the rpc package

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/9c247d1f/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/SlotManagerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/SlotManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/SlotManagerTest.java
new file mode 100644
index 0000000..52d9d06
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/SlotManagerTest.java
@@ -0,0 +1,538 @@
+/*
+ * 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.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.ResourceSlot;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.taskexecutor.SlotStatus;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+
+public class SlotManagerTest {
+
+	private static final double DEFAULT_TESTING_CPU_CORES = 1.0;
+
+	private static final long DEFAULT_TESTING_MEMORY = 512;
+
+	private static final ResourceProfile DEFAULT_TESTING_PROFILE =
+		new ResourceProfile(DEFAULT_TESTING_CPU_CORES, DEFAULT_TESTING_MEMORY);
+
+	private static final ResourceProfile DEFAULT_TESTING_BIG_PROFILE =
+		new ResourceProfile(DEFAULT_TESTING_CPU_CORES * 2, DEFAULT_TESTING_MEMORY * 2);
+
+	private ResourceManagerGateway resourceManagerGateway;
+
+	@Before
+	public void setUp() {
+		resourceManagerGateway = mock(ResourceManagerGateway.class);
+	}
+
+	/**
+	 * Tests that there are no free slots when we request, need to allocate from cluster manager master
+	 */
+	@Test
+	public void testRequestSlotWithoutFreeSlot() {
+		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
+		slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE));
+
+		assertEquals(0, slotManager.getAllocatedSlotCount());
+		assertEquals(0, slotManager.getFreeSlotCount());
+		assertEquals(1, slotManager.getPendingRequestCount());
+		assertEquals(1, slotManager.getAllocatedContainers().size());
+		assertEquals(DEFAULT_TESTING_PROFILE, slotManager.getAllocatedContainers().get(0));
+	}
+
+	/**
+	 * Tests that there are some free slots when we request, and the request is fulfilled immediately
+	 */
+	@Test
+	public void testRequestSlotWithFreeSlot() {
+		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
+
+		directlyProvideFreeSlots(slotManager, DEFAULT_TESTING_PROFILE, 1);
+		assertEquals(1, slotManager.getFreeSlotCount());
+
+		slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE));
+		assertEquals(1, slotManager.getAllocatedSlotCount());
+		assertEquals(0, slotManager.getFreeSlotCount());
+		assertEquals(0, slotManager.getPendingRequestCount());
+		assertEquals(0, slotManager.getAllocatedContainers().size());
+	}
+
+	/**
+	 * Tests that there are some free slots when we request, but none of them are suitable
+	 */
+	@Test
+	public void testRequestSlotWithoutSuitableSlot() {
+		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
+
+		directlyProvideFreeSlots(slotManager, DEFAULT_TESTING_PROFILE, 2);
+		assertEquals(2, slotManager.getFreeSlotCount());
+
+		slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_BIG_PROFILE));
+		assertEquals(0, slotManager.getAllocatedSlotCount());
+		assertEquals(2, slotManager.getFreeSlotCount());
+		assertEquals(1, slotManager.getPendingRequestCount());
+		assertEquals(1, slotManager.getAllocatedContainers().size());
+		assertEquals(DEFAULT_TESTING_BIG_PROFILE, slotManager.getAllocatedContainers().get(0));
+	}
+
+	/**
+	 * Tests that we send duplicated slot request
+	 */
+	@Test
+	public void testDuplicatedSlotRequest() {
+		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
+		directlyProvideFreeSlots(slotManager, DEFAULT_TESTING_PROFILE, 1);
+
+		SlotRequest request1 = new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE);
+		SlotRequest request2 = new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_BIG_PROFILE);
+
+		slotManager.requestSlot(request1);
+		slotManager.requestSlot(request2);
+		slotManager.requestSlot(request2);
+		slotManager.requestSlot(request1);
+
+		assertEquals(1, slotManager.getAllocatedSlotCount());
+		assertEquals(0, slotManager.getFreeSlotCount());
+		assertEquals(1, slotManager.getPendingRequestCount());
+		assertEquals(1, slotManager.getAllocatedContainers().size());
+		assertEquals(DEFAULT_TESTING_BIG_PROFILE, slotManager.getAllocatedContainers().get(0));
+	}
+
+	/**
+	 * Tests that we send multiple slot requests
+	 */
+	@Test
+	public void testRequestMultipleSlots() {
+		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
+		directlyProvideFreeSlots(slotManager, DEFAULT_TESTING_PROFILE, 5);
+
+		// request 3 normal slots
+		for (int i = 0; i < 3; ++i) {
+			slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE));
+		}
+
+		// request 2 big slots
+		for (int i = 0; i < 2; ++i) {
+			slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_BIG_PROFILE));
+		}
+
+		// request 1 normal slot again
+		slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE));
+
+		assertEquals(4, slotManager.getAllocatedSlotCount());
+		assertEquals(1, slotManager.getFreeSlotCount());
+		assertEquals(2, slotManager.getPendingRequestCount());
+		assertEquals(2, slotManager.getAllocatedContainers().size());
+		assertEquals(DEFAULT_TESTING_BIG_PROFILE, slotManager.getAllocatedContainers().get(0));
+		assertEquals(DEFAULT_TESTING_BIG_PROFILE, slotManager.getAllocatedContainers().get(1));
+	}
+
+	/**
+	 * Tests that a new slot appeared in SlotReport, and we used it to fulfill a pending request
+	 */
+	@Test
+	public void testNewlyAppearedFreeSlotFulfillPendingRequest() {
+		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
+		slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE));
+		assertEquals(1, slotManager.getPendingRequestCount());
+
+		SlotID slotId = SlotID.generate();
+		SlotStatus slotStatus = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE);
+		slotManager.updateSlotStatus(slotStatus);
+
+		assertEquals(1, slotManager.getAllocatedSlotCount());
+		assertEquals(0, slotManager.getFreeSlotCount());
+		assertEquals(0, slotManager.getPendingRequestCount());
+		assertTrue(slotManager.isAllocated(slotId));
+	}
+
+	/**
+	 * Tests that a new slot appeared in SlotReport, but we have no pending request
+	 */
+	@Test
+	public void testNewlyAppearedFreeSlot() {
+		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
+		SlotID slotId = SlotID.generate();
+		SlotStatus slotStatus = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE);
+		slotManager.updateSlotStatus(slotStatus);
+
+		assertEquals(0, slotManager.getAllocatedSlotCount());
+		assertEquals(1, slotManager.getFreeSlotCount());
+	}
+
+	/**
+	 * Tests that a new slot appeared in SlotReport, but it't not suitable for all the pending requests
+	 */
+	@Test
+	public void testNewlyAppearedFreeSlotNotMatchPendingRequests() {
+		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
+		slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_BIG_PROFILE));
+		assertEquals(1, slotManager.getPendingRequestCount());
+
+		SlotID slotId = SlotID.generate();
+		SlotStatus slotStatus = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE);
+		slotManager.updateSlotStatus(slotStatus);
+
+		assertEquals(0, slotManager.getAllocatedSlotCount());
+		assertEquals(1, slotManager.getFreeSlotCount());
+		assertEquals(1, slotManager.getPendingRequestCount());
+		assertFalse(slotManager.isAllocated(slotId));
+	}
+
+	/**
+	 * Tests that a new slot appeared in SlotReport, and it's been reported using by some job
+	 */
+	@Test
+	public void testNewlyAppearedInUseSlot() {
+		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
+
+		SlotID slotId = SlotID.generate();
+		SlotStatus slotStatus = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE, new AllocationID(), new JobID());
+		slotManager.updateSlotStatus(slotStatus);
+
+		assertEquals(1, slotManager.getAllocatedSlotCount());
+		assertEquals(0, slotManager.getFreeSlotCount());
+		assertTrue(slotManager.isAllocated(slotId));
+	}
+
+	/**
+	 * Tests that we had a slot in-use, and it's confirmed by SlotReport
+	 */
+	@Test
+	public void testExistingInUseSlotUpdateStatus() {
+		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
+		SlotRequest request = new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE);
+		slotManager.requestSlot(request);
+
+		// make this slot in use
+		SlotID slotId = SlotID.generate();
+		SlotStatus slotStatus = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE);
+		slotManager.updateSlotStatus(slotStatus);
+
+		assertEquals(1, slotManager.getAllocatedSlotCount());
+		assertEquals(0, slotManager.getFreeSlotCount());
+		assertTrue(slotManager.isAllocated(slotId));
+
+		// slot status is confirmed
+		SlotStatus slotStatus2 = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE,
+			request.getAllocationId(), request.getJobId());
+		slotManager.updateSlotStatus(slotStatus2);
+
+		assertEquals(1, slotManager.getAllocatedSlotCount());
+		assertEquals(0, slotManager.getFreeSlotCount());
+		assertTrue(slotManager.isAllocated(slotId));
+	}
+
+	/**
+	 * Tests that we had a slot in-use, but it's empty according to the SlotReport
+	 */
+	@Test
+	public void testExistingInUseSlotAdjustedToEmpty() {
+		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
+		SlotRequest request1 = new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE);
+		slotManager.requestSlot(request1);
+
+		// make this slot in use
+		SlotID slotId = SlotID.generate();
+		SlotStatus slotStatus = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE);
+		slotManager.updateSlotStatus(slotStatus);
+
+		// another request pending
+		SlotRequest request2 = new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE);
+		slotManager.requestSlot(request2);
+
+		assertEquals(1, slotManager.getAllocatedSlotCount());
+		assertEquals(0, slotManager.getFreeSlotCount());
+		assertEquals(1, slotManager.getPendingRequestCount());
+		assertTrue(slotManager.isAllocated(slotId));
+		assertTrue(slotManager.isAllocated(request1.getAllocationId()));
+
+
+		// but slot is reported empty again, request2 will be fulfilled, request1 will be missing
+		slotManager.updateSlotStatus(slotStatus);
+
+		assertEquals(1, slotManager.getAllocatedSlotCount());
+		assertEquals(0, slotManager.getFreeSlotCount());
+		assertEquals(0, slotManager.getPendingRequestCount());
+		assertTrue(slotManager.isAllocated(slotId));
+		assertTrue(slotManager.isAllocated(request2.getAllocationId()));
+	}
+
+	/**
+	 * Tests that we had a slot in use, and it's also reported in use by TaskManager, but the allocation
+	 * information didn't match.
+	 */
+	@Test
+	public void testExistingInUseSlotWithDifferentAllocationInfo() {
+		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
+		SlotRequest request = new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE);
+		slotManager.requestSlot(request);
+
+		// make this slot in use
+		SlotID slotId = SlotID.generate();
+		SlotStatus slotStatus = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE);
+		slotManager.updateSlotStatus(slotStatus);
+
+		assertEquals(1, slotManager.getAllocatedSlotCount());
+		assertEquals(0, slotManager.getFreeSlotCount());
+		assertEquals(0, slotManager.getPendingRequestCount());
+		assertTrue(slotManager.isAllocated(slotId));
+		assertTrue(slotManager.isAllocated(request.getAllocationId()));
+
+		SlotStatus slotStatus2 = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE, new AllocationID(), new JobID());
+		// update slot status with different allocation info
+		slotManager.updateSlotStatus(slotStatus2);
+
+		// original request is missing and won't be allocated
+		assertEquals(1, slotManager.getAllocatedSlotCount());
+		assertEquals(0, slotManager.getFreeSlotCount());
+		assertEquals(0, slotManager.getPendingRequestCount());
+		assertTrue(slotManager.isAllocated(slotId));
+		assertFalse(slotManager.isAllocated(request.getAllocationId()));
+		assertTrue(slotManager.isAllocated(slotStatus2.getAllocationID()));
+	}
+
+	/**
+	 * Tests that we had a free slot, and it's confirmed by SlotReport
+	 */
+	@Test
+	public void testExistingEmptySlotUpdateStatus() {
+		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
+		ResourceSlot slot = new ResourceSlot(SlotID.generate(), DEFAULT_TESTING_PROFILE);
+		slotManager.addFreeSlot(slot);
+
+		SlotStatus slotStatus = new SlotStatus(slot.getSlotId(), DEFAULT_TESTING_PROFILE);
+		slotManager.updateSlotStatus(slotStatus);
+
+		assertEquals(0, slotManager.getAllocatedSlotCount());
+		assertEquals(1, slotManager.getFreeSlotCount());
+		assertEquals(0, slotManager.getPendingRequestCount());
+	}
+
+	/**
+	 * Tests that we had a free slot, and it's reported in-use by TaskManager
+	 */
+	@Test
+	public void testExistingEmptySlotAdjustedToInUse() {
+		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
+		ResourceSlot slot = new ResourceSlot(SlotID.generate(), DEFAULT_TESTING_PROFILE);
+		slotManager.addFreeSlot(slot);
+
+		SlotStatus slotStatus = new SlotStatus(slot.getSlotId(), DEFAULT_TESTING_PROFILE,
+			new AllocationID(), new JobID());
+		slotManager.updateSlotStatus(slotStatus);
+
+		assertEquals(1, slotManager.getAllocatedSlotCount());
+		assertEquals(0, slotManager.getFreeSlotCount());
+		assertEquals(0, slotManager.getPendingRequestCount());
+		assertTrue(slotManager.isAllocated(slot.getSlotId()));
+	}
+
+	/**
+	 * Tests that we did some allocation but failed / rejected by TaskManager, request will retry
+	 */
+	@Test
+	public void testSlotAllocationFailedAtTaskManager() {
+		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
+		ResourceSlot slot = new ResourceSlot(SlotID.generate(), DEFAULT_TESTING_PROFILE);
+		slotManager.addFreeSlot(slot);
+
+		SlotRequest request = new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE);
+		slotManager.requestSlot(request);
+
+		assertEquals(1, slotManager.getAllocatedSlotCount());
+		assertEquals(0, slotManager.getFreeSlotCount());
+		assertEquals(0, slotManager.getPendingRequestCount());
+		assertTrue(slotManager.isAllocated(slot.getSlotId()));
+
+		slotManager.handleSlotRequestFailedAtTaskManager(request, slot.getSlotId());
+
+		assertEquals(1, slotManager.getAllocatedSlotCount());
+		assertEquals(0, slotManager.getFreeSlotCount());
+		assertEquals(0, slotManager.getPendingRequestCount());
+	}
+
+
+	/**
+	 * Tests that we did some allocation but failed / rejected by TaskManager, and slot is occupied by another request
+	 */
+	@Test
+	public void testSlotAllocationFailedAtTaskManagerOccupiedByOther() {
+		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
+		ResourceSlot slot = new ResourceSlot(SlotID.generate(), DEFAULT_TESTING_PROFILE);
+		slotManager.addFreeSlot(slot);
+
+		SlotRequest request = new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE);
+		slotManager.requestSlot(request);
+
+		// slot is set empty by heartbeat
+		SlotStatus slotStatus = new SlotStatus(slot.getSlotId(), slot.getResourceProfile());
+		slotManager.updateSlotStatus(slotStatus);
+
+		// another request took this slot
+		SlotRequest request2 = new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE);
+		slotManager.requestSlot(request2);
+
+		assertEquals(1, slotManager.getAllocatedSlotCount());
+		assertEquals(0, slotManager.getFreeSlotCount());
+		assertEquals(0, slotManager.getPendingRequestCount());
+		assertFalse(slotManager.isAllocated(request.getAllocationId()));
+		assertTrue(slotManager.isAllocated(request2.getAllocationId()));
+
+		// original request should be pended
+		slotManager.handleSlotRequestFailedAtTaskManager(request, slot.getSlotId());
+
+		assertEquals(1, slotManager.getAllocatedSlotCount());
+		assertEquals(0, slotManager.getFreeSlotCount());
+		assertEquals(1, slotManager.getPendingRequestCount());
+		assertFalse(slotManager.isAllocated(request.getAllocationId()));
+		assertTrue(slotManager.isAllocated(request2.getAllocationId()));
+	}
+
+	@Test
+	public void testNotifyTaskManagerFailure() {
+		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
+
+		ResourceID resource1 = ResourceID.generate();
+		ResourceID resource2 = ResourceID.generate();
+
+		ResourceSlot slot11 = new ResourceSlot(new SlotID(resource1, 1), DEFAULT_TESTING_PROFILE);
+		ResourceSlot slot12 = new ResourceSlot(new SlotID(resource1, 2), DEFAULT_TESTING_PROFILE);
+		ResourceSlot slot21 = new ResourceSlot(new SlotID(resource2, 1), DEFAULT_TESTING_PROFILE);
+		ResourceSlot slot22 = new ResourceSlot(new SlotID(resource2, 2), DEFAULT_TESTING_PROFILE);
+
+		slotManager.addFreeSlot(slot11);
+		slotManager.addFreeSlot(slot21);
+
+		slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE));
+		slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE));
+
+		assertEquals(2, slotManager.getAllocatedSlotCount());
+		assertEquals(0, slotManager.getFreeSlotCount());
+		assertEquals(0, slotManager.getPendingRequestCount());
+
+		slotManager.addFreeSlot(slot12);
+		slotManager.addFreeSlot(slot22);
+
+		assertEquals(2, slotManager.getAllocatedSlotCount());
+		assertEquals(2, slotManager.getFreeSlotCount());
+		assertEquals(0, slotManager.getPendingRequestCount());
+
+		slotManager.notifyTaskManagerFailure(resource2);
+
+		assertEquals(1, slotManager.getAllocatedSlotCount());
+		assertEquals(1, slotManager.getFreeSlotCount());
+		assertEquals(0, slotManager.getPendingRequestCount());
+
+		// notify an not exist resource failure
+		slotManager.notifyTaskManagerFailure(ResourceID.generate());
+
+		assertEquals(1, slotManager.getAllocatedSlotCount());
+		assertEquals(1, slotManager.getFreeSlotCount());
+		assertEquals(0, slotManager.getPendingRequestCount());
+	}
+
+	// ------------------------------------------------------------------------
+	//  testing utilities
+	// ------------------------------------------------------------------------
+
+	private void directlyProvideFreeSlots(
+		final SlotManager slotManager,
+		final ResourceProfile resourceProfile,
+		final int freeSlotNum)
+	{
+		for (int i = 0; i < freeSlotNum; ++i) {
+			slotManager.addFreeSlot(new ResourceSlot(SlotID.generate(), new ResourceProfile(resourceProfile)));
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  testing classes
+	// ------------------------------------------------------------------------
+
+	private static class TestingSlotManager extends SlotManager {
+
+		private final List<ResourceProfile> allocatedContainers;
+
+		TestingSlotManager(ResourceManagerGateway resourceManagerGateway) {
+			super(resourceManagerGateway);
+			this.allocatedContainers = new LinkedList<>();
+		}
+
+		/**
+		 * Choose slot randomly if it matches requirement
+		 *
+		 * @param request   The slot request
+		 * @param freeSlots All slots which can be used
+		 * @return The chosen slot or null if cannot find a match
+		 */
+		@Override
+		protected ResourceSlot chooseSlotToUse(SlotRequest request, Map<SlotID, ResourceSlot> freeSlots) {
+			for (ResourceSlot slot : freeSlots.values()) {
+				if (slot.isMatchingRequirement(request.getResourceProfile())) {
+					return slot;
+				}
+			}
+			return null;
+		}
+
+		/**
+		 * Choose request randomly if offered slot can match its requirement
+		 *
+		 * @param offeredSlot     The free slot
+		 * @param pendingRequests All the pending slot requests
+		 * @return The chosen request's AllocationID or null if cannot find a match
+		 */
+		@Override
+		protected SlotRequest chooseRequestToFulfill(ResourceSlot offeredSlot,
+			Map<AllocationID, SlotRequest> pendingRequests)
+		{
+			for (Map.Entry<AllocationID, SlotRequest> pendingRequest : pendingRequests.entrySet()) {
+				if (offeredSlot.isMatchingRequirement(pendingRequest.getValue().getResourceProfile())) {
+					return pendingRequest.getValue();
+				}
+			}
+			return null;
+		}
+
+		@Override
+		protected void allocateContainer(ResourceProfile resourceProfile) {
+			allocatedContainers.add(resourceProfile);
+		}
+
+		List<ResourceProfile> getAllocatedContainers() {
+			return allocatedContainers;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9c247d1f/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java
index 2790cf8..f55069e 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java
@@ -21,28 +21,14 @@ package org.apache.flink.runtime.rpc.akka;
 import akka.actor.ActorSystem;
 import akka.util.Timeout;
 import org.apache.flink.core.testutils.OneShotLatch;
-import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.akka.AkkaUtils;
-import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
-import org.apache.flink.runtime.highavailability.NonHaServices;
-import org.apache.flink.runtime.jobgraph.JobGraph;
-import org.apache.flink.runtime.rpc.jobmaster.JobMaster;
-import org.apache.flink.runtime.rpc.resourcemanager.ResourceManagerGateway;
-import org.apache.flink.runtime.rpc.resourcemanager.ResourceManager;
 import org.apache.flink.util.TestLogger;
 
 import org.junit.AfterClass;
 import org.junit.Test;
 
-import org.mockito.Mockito;
-import scala.concurrent.duration.Deadline;
-import scala.concurrent.duration.FiniteDuration;
-
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.ForkJoinPool;
 import java.util.concurrent.TimeUnit;
 
-import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
 public class AkkaRpcServiceTest extends TestLogger {

http://git-wip-us.apache.org/repos/asf/flink/blob/9c247d1f/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/registration/RetryingRegistrationTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/registration/RetryingRegistrationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/registration/RetryingRegistrationTest.java
deleted file mode 100644
index 9508825..0000000
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/registration/RetryingRegistrationTest.java
+++ /dev/null
@@ -1,336 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.runtime.rpc.registration;
-
-import akka.dispatch.Futures;
-
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.runtime.rpc.RpcService;
-import org.apache.flink.runtime.rpc.TestingRpcService;
-import org.apache.flink.util.TestLogger;
-
-import org.junit.Test;
-
-import org.slf4j.LoggerFactory;
-
-import scala.concurrent.Await;
-import scala.concurrent.ExecutionContext$;
-import scala.concurrent.Future;
-import scala.concurrent.Promise;
-import scala.concurrent.duration.FiniteDuration;
-
-import java.util.UUID;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.TimeoutException;
-
-import static java.util.concurrent.TimeUnit.SECONDS;
-import static org.junit.Assert.*;
-import static org.mockito.Mockito.*;
-
-/**
- * Tests for the generic retrying registration class, validating the failure, retry, and back-off behavior.
- */
-public class RetryingRegistrationTest extends TestLogger {
-
-	@Test
-	public void testSimpleSuccessfulRegistration() throws Exception {
-		final String testId = "laissez les bon temps roulez";
-		final String testEndpointAddress = "<test-address>";
-		final UUID leaderId = UUID.randomUUID();
-
-		// an endpoint that immediately returns success
-		TestRegistrationGateway testGateway = new TestRegistrationGateway(new TestRegistrationSuccess(testId));
-		TestingRpcService rpc = new TestingRpcService();
-
-		try {
-			rpc.registerGateway(testEndpointAddress, testGateway);
-
-			TestRetryingRegistration registration = new TestRetryingRegistration(rpc, testEndpointAddress, leaderId);
-			registration.startRegistration();
-
-			Future<Tuple2<TestRegistrationGateway, TestRegistrationSuccess>> future = registration.getFuture();
-			assertNotNull(future);
-
-			// multiple accesses return the same future
-			assertEquals(future, registration.getFuture());
-
-			Tuple2<TestRegistrationGateway, TestRegistrationSuccess> success = 
-					Await.result(future, new FiniteDuration(10, SECONDS));
-
-			// validate correct invocation and result
-			assertEquals(testId, success.f1.getCorrelationId());
-			assertEquals(leaderId, testGateway.getInvocations().take().leaderId());
-		}
-		finally {
-			testGateway.stop();
-			rpc.stopService();
-		}
-	}
-	
-	@Test
-	public void testPropagateFailures() throws Exception {
-		final String testExceptionMessage = "testExceptionMessage";
-
-		// RPC service that fails with exception upon the connection
-		RpcService rpc = mock(RpcService.class);
-		when(rpc.connect(anyString(), any(Class.class))).thenThrow(new RuntimeException(testExceptionMessage));
-
-		TestRetryingRegistration registration = new TestRetryingRegistration(rpc, "testaddress", UUID.randomUUID());
-		registration.startRegistration();
-
-		Future<?> future = registration.getFuture();
-		assertTrue(future.failed().isCompleted());
-
-		assertEquals(testExceptionMessage, future.failed().value().get().get().getMessage());
-	}
-
-	@Test
-	public void testRetryConnectOnFailure() throws Exception {
-		final String testId = "laissez les bon temps roulez";
-		final UUID leaderId = UUID.randomUUID();
-
-		ExecutorService executor = Executors.newCachedThreadPool();
-		TestRegistrationGateway testGateway = new TestRegistrationGateway(new TestRegistrationSuccess(testId));
-
-		try {
-			// RPC service that fails upon the first connection, but succeeds on the second
-			RpcService rpc = mock(RpcService.class);
-			when(rpc.connect(anyString(), any(Class.class))).thenReturn(
-					Futures.failed(new Exception("test connect failure")),  // first connection attempt fails
-					Futures.successful(testGateway)                         // second connection attempt succeeds
-			);
-			when(rpc.getExecutionContext()).thenReturn(ExecutionContext$.MODULE$.fromExecutor(executor));
-
-			TestRetryingRegistration registration = new TestRetryingRegistration(rpc, "foobar address", leaderId);
-			registration.startRegistration();
-
-			Tuple2<TestRegistrationGateway, TestRegistrationSuccess> success =
-					Await.result(registration.getFuture(), new FiniteDuration(10, SECONDS));
-
-			// validate correct invocation and result
-			assertEquals(testId, success.f1.getCorrelationId());
-			assertEquals(leaderId, testGateway.getInvocations().take().leaderId());
-		}
-		finally {
-			testGateway.stop();
-			executor.shutdown();
-		}
-	}
-
-	@Test
-	public void testRetriesOnTimeouts() throws Exception {
-		final String testId = "rien ne va plus";
-		final String testEndpointAddress = "<test-address>";
-		final UUID leaderId = UUID.randomUUID();
-
-		// an endpoint that immediately returns futures with timeouts before returning a successful future
-		TestRegistrationGateway testGateway = new TestRegistrationGateway(
-				null, // timeout
-				null, // timeout
-				new TestRegistrationSuccess(testId) // success
-		);
-
-		TestingRpcService rpc = new TestingRpcService();
-
-		try {
-			rpc.registerGateway(testEndpointAddress, testGateway);
-	
-			TestRetryingRegistration registration = new TestRetryingRegistration(rpc, testEndpointAddress, leaderId);
-	
-			long started = System.nanoTime();
-			registration.startRegistration();
-	
-			Future<Tuple2<TestRegistrationGateway, TestRegistrationSuccess>> future = registration.getFuture();
-			Tuple2<TestRegistrationGateway, TestRegistrationSuccess> success =
-					Await.result(future, new FiniteDuration(10, SECONDS));
-	
-			long finished = System.nanoTime();
-			long elapsedMillis = (finished - started) / 1000000;
-	
-			// validate correct invocation and result
-			assertEquals(testId, success.f1.getCorrelationId());
-			assertEquals(leaderId, testGateway.getInvocations().take().leaderId());
-	
-			// validate that some retry-delay / back-off behavior happened
-			assertTrue("retries did not properly back off", elapsedMillis >= 3 * TestRetryingRegistration.INITIAL_TIMEOUT);
-		}
-		finally {
-			rpc.stopService();
-			testGateway.stop();
-		}
-	}
-
-	@Test
-	public void testDecline() throws Exception {
-		final String testId = "qui a coupe le fromage";
-		final String testEndpointAddress = "<test-address>";
-		final UUID leaderId = UUID.randomUUID();
-
-		TestingRpcService rpc = new TestingRpcService();
-
-		TestRegistrationGateway testGateway = new TestRegistrationGateway(
-				null, // timeout
-				new RegistrationResponse.Decline("no reason "),
-				null, // timeout
-				new TestRegistrationSuccess(testId) // success
-		);
-
-		try {
-			rpc.registerGateway(testEndpointAddress, testGateway);
-
-			TestRetryingRegistration registration = new TestRetryingRegistration(rpc, testEndpointAddress, leaderId);
-
-			long started = System.nanoTime();
-			registration.startRegistration();
-	
-			Future<Tuple2<TestRegistrationGateway, TestRegistrationSuccess>> future = registration.getFuture();
-			Tuple2<TestRegistrationGateway, TestRegistrationSuccess> success =
-					Await.result(future, new FiniteDuration(10, SECONDS));
-
-			long finished = System.nanoTime();
-			long elapsedMillis = (finished - started) / 1000000;
-
-			// validate correct invocation and result
-			assertEquals(testId, success.f1.getCorrelationId());
-			assertEquals(leaderId, testGateway.getInvocations().take().leaderId());
-
-			// validate that some retry-delay / back-off behavior happened
-			assertTrue("retries did not properly back off", elapsedMillis >= 
-					2 * TestRetryingRegistration.INITIAL_TIMEOUT + TestRetryingRegistration.DELAY_ON_DECLINE);
-		}
-		finally {
-			testGateway.stop();
-			rpc.stopService();
-		}
-	}
-	
-	@Test
-	@SuppressWarnings("unchecked")
-	public void testRetryOnError() throws Exception {
-		final String testId = "Petit a petit, l'oiseau fait son nid";
-		final String testEndpointAddress = "<test-address>";
-		final UUID leaderId = UUID.randomUUID();
-
-		TestingRpcService rpc = new TestingRpcService();
-
-		try {
-			// gateway that upon calls first responds with a failure, then with a success
-			TestRegistrationGateway testGateway = mock(TestRegistrationGateway.class);
-
-			when(testGateway.registrationCall(any(UUID.class), anyLong())).thenReturn(
-					Futures.<RegistrationResponse>failed(new Exception("test exception")),
-					Futures.<RegistrationResponse>successful(new TestRegistrationSuccess(testId)));
-			
-			rpc.registerGateway(testEndpointAddress, testGateway);
-
-			TestRetryingRegistration registration = new TestRetryingRegistration(rpc, testEndpointAddress, leaderId);
-
-			long started = System.nanoTime();
-			registration.startRegistration();
-
-			Future<Tuple2<TestRegistrationGateway, TestRegistrationSuccess>> future = registration.getFuture();
-			Tuple2<TestRegistrationGateway, TestRegistrationSuccess> success =
-					Await.result(future, new FiniteDuration(10, SECONDS));
-
-			long finished = System.nanoTime();
-			long elapsedMillis = (finished - started) / 1000000;
-			
-			assertEquals(testId, success.f1.getCorrelationId());
-
-			// validate that some retry-delay / back-off behavior happened
-			assertTrue("retries did not properly back off",
-					elapsedMillis >= TestRetryingRegistration.DELAY_ON_ERROR);
-		}
-		finally {
-			rpc.stopService();
-		}
-	}
-
-	@Test
-	public void testCancellation() throws Exception {
-		final String testEndpointAddress = "my-test-address";
-		final UUID leaderId = UUID.randomUUID();
-
-		TestingRpcService rpc = new TestingRpcService();
-
-		try {
-			Promise<RegistrationResponse> result = Futures.promise();
-
-			TestRegistrationGateway testGateway = mock(TestRegistrationGateway.class);
-			when(testGateway.registrationCall(any(UUID.class), anyLong())).thenReturn(result.future());
-
-			rpc.registerGateway(testEndpointAddress, testGateway);
-
-			TestRetryingRegistration registration = new TestRetryingRegistration(rpc, testEndpointAddress, leaderId);
-			registration.startRegistration();
-
-			// cancel and fail the current registration attempt
-			registration.cancel();
-			result.failure(new TimeoutException());
-
-			// there should not be a second registration attempt
-			verify(testGateway, atMost(1)).registrationCall(any(UUID.class), anyLong());
-		}
-		finally {
-			rpc.stopService();
-		}
-	}
-
-	// ------------------------------------------------------------------------
-	//  test registration
-	// ------------------------------------------------------------------------
-
-	private static class TestRegistrationSuccess extends RegistrationResponse.Success {
-		private static final long serialVersionUID = 5542698790917150604L;
-
-		private final String correlationId;
-
-		private TestRegistrationSuccess(String correlationId) {
-			this.correlationId = correlationId;
-		}
-
-		public String getCorrelationId() {
-			return correlationId;
-		}
-	}
-
-	private static class TestRetryingRegistration extends RetryingRegistration<TestRegistrationGateway, TestRegistrationSuccess> {
-
-		// we use shorter timeouts here to speed up the tests
-		static final long INITIAL_TIMEOUT = 20;
-		static final long MAX_TIMEOUT = 200;
-		static final long DELAY_ON_ERROR = 200;
-		static final long DELAY_ON_DECLINE = 200;
-
-		public TestRetryingRegistration(RpcService rpc, String targetAddress, UUID leaderId) {
-			super(LoggerFactory.getLogger(RetryingRegistrationTest.class),
-					rpc, "TestEndpoint",
-					TestRegistrationGateway.class,
-					targetAddress, leaderId,
-					INITIAL_TIMEOUT, MAX_TIMEOUT, DELAY_ON_ERROR, DELAY_ON_DECLINE);
-		}
-
-		@Override
-		protected Future<RegistrationResponse> invokeRegistration(
-				TestRegistrationGateway gateway, UUID leaderId, long timeoutMillis) {
-			return gateway.registrationCall(leaderId, timeoutMillis);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9c247d1f/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/registration/TestRegistrationGateway.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/registration/TestRegistrationGateway.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/registration/TestRegistrationGateway.java
deleted file mode 100644
index a049e48..0000000
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/registration/TestRegistrationGateway.java
+++ /dev/null
@@ -1,85 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.runtime.rpc.registration;
-
-import akka.dispatch.Futures;
-
-import org.apache.flink.runtime.rpc.TestingGatewayBase;
-import org.apache.flink.util.Preconditions;
-
-import scala.concurrent.Future;
-
-import java.util.UUID;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.LinkedBlockingQueue;
-
-public class TestRegistrationGateway extends TestingGatewayBase {
-
-	private final BlockingQueue<RegistrationCall> invocations;
-
-	private final RegistrationResponse[] responses;
-
-	private int pos;
-
-	public TestRegistrationGateway(RegistrationResponse... responses) {
-		Preconditions.checkArgument(responses != null && responses.length > 0);
-
-		this.invocations = new LinkedBlockingQueue<>();
-		this.responses = responses;
-		
-	}
-
-	// ------------------------------------------------------------------------
-
-	public Future<RegistrationResponse> registrationCall(UUID leaderId, long timeout) {
-		invocations.add(new RegistrationCall(leaderId, timeout));
-
-		RegistrationResponse response = responses[pos];
-		if (pos < responses.length - 1) {
-			pos++;
-		}
-
-		// return a completed future (for a proper value), or one that never completes and will time out (for null)
-		return response != null ? Futures.successful(response) : this.<RegistrationResponse>futureWithTimeout(timeout);
-	}
-
-	public BlockingQueue<RegistrationCall> getInvocations() {
-		return invocations;
-	}
-
-	// ------------------------------------------------------------------------
-
-	public static class RegistrationCall {
-		private final UUID leaderId;
-		private final long timeout;
-
-		public RegistrationCall(UUID leaderId, long timeout) {
-			this.leaderId = leaderId;
-			this.timeout = timeout;
-		}
-
-		public UUID leaderId() {
-			return leaderId;
-		}
-
-		public long timeout() {
-			return timeout;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9c247d1f/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManagerHATest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManagerHATest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManagerHATest.java
deleted file mode 100644
index dfffeda..0000000
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManagerHATest.java
+++ /dev/null
@@ -1,76 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.runtime.rpc.resourcemanager;
-
-import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices;
-import org.apache.flink.runtime.leaderelection.TestingLeaderElectionService;
-import org.apache.flink.runtime.rpc.MainThreadExecutor;
-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.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
- */
-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 execute runAsync call
-		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);
-
-		TestingLeaderElectionService leaderElectionService = new TestingLeaderElectionService();
-		TestingHighAvailabilityServices highAvailabilityServices = new TestingHighAvailabilityServices();
-		highAvailabilityServices.setResourceManagerLeaderElectionService(leaderElectionService);
-
-		final ResourceManager resourceManager = new ResourceManager(rpcService, highAvailabilityServices);
-		resourceManager.start();
-		// before grant leadership, resourceManager's leaderId is null
-		Assert.assertNull(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.assertNull(resourceManager.getLeaderSessionID());
-	}
-
-	private static abstract class TestingResourceManagerGatewayProxy implements MainThreadExecutor, StartStoppable, RpcGateway {
-		@Override
-		public void runAsync(Runnable runnable) {
-			runnable.run();
-		}
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9c247d1f/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorTest.java
deleted file mode 100644
index 25a670c..0000000
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorTest.java
+++ /dev/null
@@ -1,117 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.runtime.rpc.taskexecutor;
-
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.clusterframework.types.ResourceID;
-import org.apache.flink.runtime.highavailability.NonHaServices;
-import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices;
-import org.apache.flink.runtime.leaderelection.TestingLeaderRetrievalService;
-import org.apache.flink.runtime.rpc.TestingRpcService;
-import org.apache.flink.runtime.rpc.resourcemanager.ResourceManagerGateway;
-import org.apache.flink.util.TestLogger;
-
-import org.junit.Test;
-
-import scala.concurrent.duration.FiniteDuration;
-
-import java.util.UUID;
-
-import static org.junit.Assert.*;
-import static org.mockito.Mockito.*;
-
-public class TaskExecutorTest extends TestLogger {
-
-	@Test
-	public void testImmediatelyRegistersIfLeaderIsKnown() throws Exception {
-		final ResourceID resourceID = ResourceID.generate();
-		final String resourceManagerAddress = "/resource/manager/address/one";
-
-		final TestingRpcService rpc = new TestingRpcService();
-		try {
-			// register a mock resource manager gateway
-			ResourceManagerGateway rmGateway = mock(ResourceManagerGateway.class);
-			rpc.registerGateway(resourceManagerAddress, rmGateway);
-
-			NonHaServices haServices = new NonHaServices(resourceManagerAddress);
-			TaskExecutor taskManager = TaskExecutor.startTaskManagerComponentsAndActor(
-				new Configuration(), resourceID, rpc, "localhost", haServices, true);
-			String taskManagerAddress = taskManager.getAddress();
-			taskManager.start();
-
-			verify(rmGateway, timeout(5000)).registerTaskExecutor(
-					any(UUID.class), eq(taskManagerAddress), eq(resourceID), any(FiniteDuration.class));
-		}
-		finally {
-			rpc.stopService();
-		}
-	}
-
-	@Test
-	public void testTriggerRegistrationOnLeaderChange() throws Exception {
-		final ResourceID resourceID = ResourceID.generate();
-
-		final String address1 = "/resource/manager/address/one";
-		final String address2 = "/resource/manager/address/two";
-		final UUID leaderId1 = UUID.randomUUID();
-		final UUID leaderId2 = UUID.randomUUID();
-
-		final TestingRpcService rpc = new TestingRpcService();
-		try {
-			// register the mock resource manager gateways
-			ResourceManagerGateway rmGateway1 = mock(ResourceManagerGateway.class);
-			ResourceManagerGateway rmGateway2 = mock(ResourceManagerGateway.class);
-			rpc.registerGateway(address1, rmGateway1);
-			rpc.registerGateway(address2, rmGateway2);
-
-			TestingLeaderRetrievalService testLeaderService = new TestingLeaderRetrievalService();
-
-			TestingHighAvailabilityServices haServices = new TestingHighAvailabilityServices();
-			haServices.setResourceManagerLeaderRetriever(testLeaderService);
-
-			TaskExecutor taskManager = TaskExecutor.startTaskManagerComponentsAndActor(
-				new Configuration(), resourceID, rpc, "localhost", haServices, true);
-			String taskManagerAddress = taskManager.getAddress();
-			taskManager.start();
-
-			// no connection initially, since there is no leader
-			assertNull(taskManager.getResourceManagerConnection());
-
-			// define a leader and see that a registration happens
-			testLeaderService.notifyListener(address1, leaderId1);
-
-			verify(rmGateway1, timeout(5000)).registerTaskExecutor(
-					eq(leaderId1), eq(taskManagerAddress), eq(resourceID), any(FiniteDuration.class));
-			assertNotNull(taskManager.getResourceManagerConnection());
-
-			// cancel the leader 
-			testLeaderService.notifyListener(null, null);
-
-			// set a new leader, see that a registration happens 
-			testLeaderService.notifyListener(address2, leaderId2);
-
-			verify(rmGateway2, timeout(5000)).registerTaskExecutor(
-					eq(leaderId2), eq(taskManagerAddress), eq(resourceID), any(FiniteDuration.class));
-			assertNotNull(taskManager.getResourceManagerConnection());
-		}
-		finally {
-			rpc.stopService();
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9c247d1f/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
new file mode 100644
index 0000000..a8d5bd7
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java
@@ -0,0 +1,117 @@
+/*
+ * 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.configuration.Configuration;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.highavailability.NonHaServices;
+import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices;
+import org.apache.flink.runtime.leaderelection.TestingLeaderRetrievalService;
+import org.apache.flink.runtime.rpc.TestingRpcService;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Test;
+
+import scala.concurrent.duration.FiniteDuration;
+
+import java.util.UUID;
+
+import static org.junit.Assert.*;
+import static org.mockito.Mockito.*;
+
+public class TaskExecutorTest extends TestLogger {
+
+	@Test
+	public void testImmediatelyRegistersIfLeaderIsKnown() throws Exception {
+		final ResourceID resourceID = ResourceID.generate();
+		final String resourceManagerAddress = "/resource/manager/address/one";
+
+		final TestingRpcService rpc = new TestingRpcService();
+		try {
+			// register a mock resource manager gateway
+			ResourceManagerGateway rmGateway = mock(ResourceManagerGateway.class);
+			rpc.registerGateway(resourceManagerAddress, rmGateway);
+
+			NonHaServices haServices = new NonHaServices(resourceManagerAddress);
+			TaskExecutor taskManager = TaskExecutor.startTaskManagerComponentsAndActor(
+				new Configuration(), resourceID, rpc, "localhost", haServices, true);
+			String taskManagerAddress = taskManager.getAddress();
+			taskManager.start();
+
+			verify(rmGateway, timeout(5000)).registerTaskExecutor(
+					any(UUID.class), eq(taskManagerAddress), eq(resourceID), any(FiniteDuration.class));
+		}
+		finally {
+			rpc.stopService();
+		}
+	}
+
+	@Test
+	public void testTriggerRegistrationOnLeaderChange() throws Exception {
+		final ResourceID resourceID = ResourceID.generate();
+
+		final String address1 = "/resource/manager/address/one";
+		final String address2 = "/resource/manager/address/two";
+		final UUID leaderId1 = UUID.randomUUID();
+		final UUID leaderId2 = UUID.randomUUID();
+
+		final TestingRpcService rpc = new TestingRpcService();
+		try {
+			// register the mock resource manager gateways
+			ResourceManagerGateway rmGateway1 = mock(ResourceManagerGateway.class);
+			ResourceManagerGateway rmGateway2 = mock(ResourceManagerGateway.class);
+			rpc.registerGateway(address1, rmGateway1);
+			rpc.registerGateway(address2, rmGateway2);
+
+			TestingLeaderRetrievalService testLeaderService = new TestingLeaderRetrievalService();
+
+			TestingHighAvailabilityServices haServices = new TestingHighAvailabilityServices();
+			haServices.setResourceManagerLeaderRetriever(testLeaderService);
+
+			TaskExecutor taskManager = TaskExecutor.startTaskManagerComponentsAndActor(
+				new Configuration(), resourceID, rpc, "localhost", haServices, true);
+			String taskManagerAddress = taskManager.getAddress();
+			taskManager.start();
+
+			// no connection initially, since there is no leader
+			assertNull(taskManager.getResourceManagerConnection());
+
+			// define a leader and see that a registration happens
+			testLeaderService.notifyListener(address1, leaderId1);
+
+			verify(rmGateway1, timeout(5000)).registerTaskExecutor(
+					eq(leaderId1), eq(taskManagerAddress), eq(resourceID), any(FiniteDuration.class));
+			assertNotNull(taskManager.getResourceManagerConnection());
+
+			// cancel the leader 
+			testLeaderService.notifyListener(null, null);
+
+			// set a new leader, see that a registration happens 
+			testLeaderService.notifyListener(address2, leaderId2);
+
+			verify(rmGateway2, timeout(5000)).registerTaskExecutor(
+					eq(leaderId2), eq(taskManagerAddress), eq(resourceID), any(FiniteDuration.class));
+			assertNotNull(taskManager.getResourceManagerConnection());
+		}
+		finally {
+			rpc.stopService();
+		}
+	}
+}


[15/50] [abbrv] flink git commit: [hotfix] Remove unused imports from SlotRequestRegistered/Rejected and ResourceSlot

Posted by se...@apache.org.
[hotfix] Remove unused imports from SlotRequestRegistered/Rejected and ResourceSlot


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

Branch: refs/heads/flip-6
Commit: b2c5c0fd154b3233ef1fb5a65175a94f3111dcd3
Parents: 74570d4
Author: Till Rohrmann <tr...@apache.org>
Authored: Wed Sep 21 11:47:53 2016 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Oct 14 15:14:40 2016 +0200

----------------------------------------------------------------------
 .../flink/runtime/clusterframework/types/ResourceSlot.java       | 4 ----
 .../flink/runtime/resourcemanager/SlotRequestRegistered.java     | 4 ++--
 .../flink/runtime/resourcemanager/SlotRequestRejected.java       | 4 ++--
 3 files changed, 4 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/b2c5c0fd/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceSlot.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceSlot.java b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceSlot.java
index 5fb8aee..4a91a79 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceSlot.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceSlot.java
@@ -20,8 +20,6 @@ package org.apache.flink.runtime.clusterframework.types;
 
 import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway;
 
-import java.io.Serializable;
-
 import static org.apache.flink.util.Preconditions.checkNotNull;
 
 /**
@@ -30,8 +28,6 @@ import static org.apache.flink.util.Preconditions.checkNotNull;
  */
 public class ResourceSlot implements ResourceIDRetrievable {
 
-	private static final long serialVersionUID = -5853720153136840674L;
-
 	/** The unique identification of this slot */
 	private final SlotID slotId;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/b2c5c0fd/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotRequestRegistered.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotRequestRegistered.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotRequestRegistered.java
index 6b7f6dc..f719dce 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotRequestRegistered.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotRequestRegistered.java
@@ -20,13 +20,13 @@ package org.apache.flink.runtime.resourcemanager;
 
 import org.apache.flink.runtime.clusterframework.types.AllocationID;
 
-import java.io.Serializable;
-
 /**
  * Acknowledgment by the ResourceManager for a SlotRequest from the JobManager
  */
 public class SlotRequestRegistered extends SlotRequestReply {
 
+	private static final long serialVersionUID = 4760320859275256855L;
+
 	public SlotRequestRegistered(AllocationID allocationID) {
 		super(allocationID);
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/b2c5c0fd/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotRequestRejected.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotRequestRejected.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotRequestRejected.java
index cb3ec72..282a7d5 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotRequestRejected.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotRequestRejected.java
@@ -20,13 +20,13 @@ package org.apache.flink.runtime.resourcemanager;
 
 import org.apache.flink.runtime.clusterframework.types.AllocationID;
 
-import java.io.Serializable;
-
 /**
  * Rejection message by the ResourceManager for a SlotRequest from the JobManager
  */
 public class SlotRequestRejected extends SlotRequestReply {
 
+	private static final long serialVersionUID = 9049346740895325144L;
+
 	public SlotRequestRejected(AllocationID allocationID) {
 		super(allocationID);
 	}


[20/50] [abbrv] flink git commit: [FLINK-4505] [cluster mngt] Separate TaskManager service configuration from TaskManagerConfiguration; Implement TaskManagerRunner

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/bb781aef/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManagerRunner.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManagerRunner.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManagerRunner.java
deleted file mode 100644
index 4f756fb..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManagerRunner.java
+++ /dev/null
@@ -1,749 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.runtime.taskmanager;
-
-import org.apache.flink.api.common.JobID;
-import org.apache.flink.configuration.ConfigConstants;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.configuration.IllegalConfigurationException;
-import org.apache.flink.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.clusterframework.types.ResourceID;
-import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
-import org.apache.flink.runtime.io.disk.iomanager.IOManager;
-import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
-import org.apache.flink.runtime.io.network.ConnectionManager;
-import org.apache.flink.runtime.io.network.LocalConnectionManager;
-import org.apache.flink.runtime.io.network.NetworkEnvironment;
-import org.apache.flink.runtime.io.network.TaskEventDispatcher;
-import org.apache.flink.runtime.io.network.buffer.NetworkBufferPool;
-import org.apache.flink.runtime.io.network.netty.NettyConfig;
-import org.apache.flink.runtime.io.network.netty.NettyConnectionManager;
-import org.apache.flink.runtime.io.network.partition.ResultPartitionManager;
-import org.apache.flink.runtime.leaderelection.LeaderElectionService;
-import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
-import org.apache.flink.runtime.memory.MemoryManager;
-import org.apache.flink.runtime.query.KvStateRegistry;
-import org.apache.flink.runtime.query.netty.DisabledKvStateRequestStats;
-import org.apache.flink.runtime.query.netty.KvStateServer;
-import org.apache.flink.runtime.rpc.RpcService;
-import org.apache.flink.runtime.rpc.akka.AkkaRpcService;
-import org.apache.flink.runtime.taskexecutor.TaskExecutor;
-import org.apache.flink.runtime.taskexecutor.TaskExecutorConfiguration;
-import org.apache.flink.runtime.util.EnvironmentInformation;
-import org.apache.flink.runtime.util.LeaderRetrievalUtils;
-import org.apache.flink.util.MathUtils;
-import org.apache.flink.util.NetUtils;
-
-import akka.actor.ActorSystem;
-import akka.util.Timeout;
-import com.typesafe.config.Config;
-import org.apache.flink.util.Preconditions;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import scala.Option;
-import scala.Some;
-import scala.Tuple2;
-import scala.concurrent.duration.Duration;
-import scala.concurrent.duration.FiniteDuration;
-
-import java.io.File;
-import java.io.IOException;
-import java.net.InetAddress;
-import java.net.InetSocketAddress;
-import java.util.concurrent.TimeUnit;
-
-import static org.apache.flink.util.Preconditions.checkNotNull;
-
-/**
- * This class is the executable entry point for the task manager in yarn or standalone mode.
- * It constructs the related components (network, I/O manager, memory manager, RPC service, HA service)
- * and starts them.
- */
-public class TaskManagerRunner {
-
-	private static final Logger LOG = LoggerFactory.getLogger(TaskManagerRunner.class);
-
-	/**
-	 * Constructs related components of the TaskManager and starts them.
-	 *
-	 * @param configuration                 The configuration for the TaskManager.
-	 * @param resourceID                    The id of the resource which the task manager will run on.
-	 * @param rpcService                    Optionally, The rpc service which is used to start and connect to the TaskManager RpcEndpoint .
-	 *                                                 If none is given, then a RpcService is constructed from the configuration.
-	 * @param taskManagerHostname   Optionally, The hostname/address that describes the TaskManager's data location.
-	 *                                                 If none is given, it can be got from the configuration.
-	 * @param localTaskManagerCommunication      If true, the TaskManager will not initiate the TCP network stack.
-	 * @param haServices                    Optionally, a high availability service can be provided. If none is given,
-	 *                                                 then a HighAvailabilityServices is constructed from the configuration.
-	 */
-	public static void createAndStartComponents(
-		final Configuration configuration,
-		final ResourceID resourceID,
-		RpcService rpcService,
-		String taskManagerHostname,
-		boolean localTaskManagerCommunication,
-		HighAvailabilityServices haServices) throws Exception {
-
-		checkNotNull(configuration);
-		checkNotNull(resourceID);
-
-		if (taskManagerHostname == null || taskManagerHostname.isEmpty()) {
-			taskManagerHostname = selectNetworkInterface(configuration);
-		}
-
-		if (rpcService == null) {
-			// if no task manager port has been configured, use 0 (system will pick any free port)
-			final int actorSystemPort = configuration.getInteger(ConfigConstants.TASK_MANAGER_IPC_PORT_KEY, 0);
-			if (actorSystemPort < 0 || actorSystemPort > 65535) {
-				throw new IllegalConfigurationException("Invalid value for '" +
-					ConfigConstants.TASK_MANAGER_IPC_PORT_KEY +
-					"' (port for the TaskManager actor system) : " + actorSystemPort +
-					" - Leave config parameter empty or use 0 to let the system choose a port automatically.");
-			}
-			rpcService = createRpcService(configuration, taskManagerHostname, actorSystemPort);
-		}
-
-		if(haServices == null) {
-			// start high availability service to implement getResourceManagerLeaderRetriever method only
-			haServices = new HighAvailabilityServices() {
-				@Override
-				public LeaderRetrievalService getResourceManagerLeaderRetriever() throws Exception {
-					return LeaderRetrievalUtils.createLeaderRetrievalService(configuration);
-				}
-
-				@Override
-				public LeaderElectionService getResourceManagerLeaderElectionService() throws Exception {
-					return null;
-				}
-
-				@Override
-				public LeaderElectionService getJobMasterLeaderElectionService(JobID jobID) throws Exception {
-					return null;
-				}
-			};
-		}
-
-		createAndStartTaskManagerComponents(
-			configuration,
-			resourceID,
-			rpcService,
-			taskManagerHostname,
-			haServices,
-			localTaskManagerCommunication);
-	}
-
-	/**
-	 * <p/>
-	 * This method tries to select the network interface to use for the TaskManager
-	 * communication. The network interface is used both for the actor communication
-	 * (coordination) as well as for the data exchange between task managers. Unless
-	 * the hostname/interface is explicitly configured in the configuration, this
-	 * method will try out various interfaces and methods to connect to the JobManager
-	 * and select the one where the connection attempt is successful.
-	 * <p/>
-	 *
-	 * @param configuration    The configuration for the TaskManager.
-	 * @return  The host name under which the TaskManager communicates.
-	 */
-	private static String selectNetworkInterface(Configuration configuration) throws Exception {
-		String taskManagerHostname = configuration.getString(ConfigConstants.TASK_MANAGER_HOSTNAME_KEY, null);
-		if (taskManagerHostname != null) {
-			LOG.info("Using configured hostname/address for TaskManager: " + taskManagerHostname);
-		} else {
-			LeaderRetrievalService leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(configuration);
-			FiniteDuration lookupTimeout = AkkaUtils.getLookupTimeout(configuration);
-
-			InetAddress taskManagerAddress = LeaderRetrievalUtils.findConnectingAddress(leaderRetrievalService, lookupTimeout);
-			taskManagerHostname = taskManagerAddress.getHostName();
-			LOG.info("TaskManager will use hostname/address '{}' ({}) for communication.",
-				taskManagerHostname, taskManagerAddress.getHostAddress());
-		}
-
-		return taskManagerHostname;
-	}
-
-	/**
-	 * Utility method to create RPC service from configuration and hostname, port.
-	 *
-	 * @param configuration                 The configuration for the TaskManager.
-	 * @param taskManagerHostname   The hostname/address that describes the TaskManager's data location.
-	 * @param actorSystemPort           If true, the TaskManager will not initiate the TCP network stack.
-	 * @return   The rpc service which is used to start and connect to the TaskManager RpcEndpoint .
-	 * @throws java.io.IOException      Thrown, if the actor system can not bind to the address
-	 * @throws java.lang.Exception      Thrown is some other error occurs while creating akka actor system
-	 */
-	private static RpcService createRpcService(Configuration configuration, String taskManagerHostname, int actorSystemPort)
-		throws Exception{
-
-		// Bring up the TaskManager actor system first, bind it to the given address.
-
-		LOG.info("Starting TaskManager actor system at " +
-			NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort));
-
-		final ActorSystem taskManagerSystem;
-		try {
-			Tuple2<String, Object> address = new Tuple2<String, Object>(taskManagerHostname, actorSystemPort);
-			Config akkaConfig = AkkaUtils.getAkkaConfig(configuration, new Some<>(address));
-			LOG.debug("Using akka configuration\n " + akkaConfig);
-			taskManagerSystem = AkkaUtils.createActorSystem(akkaConfig);
-		} catch (Throwable t) {
-			if (t instanceof org.jboss.netty.channel.ChannelException) {
-				Throwable cause = t.getCause();
-				if (cause != null && t.getCause() instanceof java.net.BindException) {
-					String address = NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort);
-					throw new IOException("Unable to bind TaskManager actor system to address " +
-						address + " - " + cause.getMessage(), t);
-				}
-			}
-			throw new Exception("Could not create TaskManager actor system", t);
-		}
-
-		// start akka rpc service based on actor system
-		final Timeout timeout = new Timeout(AkkaUtils.getTimeout(configuration).toMillis(), TimeUnit.MILLISECONDS);
-		final AkkaRpcService akkaRpcService = new AkkaRpcService(taskManagerSystem, timeout);
-
-		return akkaRpcService;
-	}
-
-	/**
-	 * @param configuration                 The configuration for the TaskManager.
-	 * @param resourceID                    The id of the resource which the task manager will run on.
-	 * @param rpcService                    The rpc service which is used to start and connect to the TaskManager RpcEndpoint .
-	 * @param taskManagerHostname   The hostname/address that describes the TaskManager's data location.
-	 * @param haServices                    Optionally, a high availability service can be provided. If none is given,
-	 *                                                  then a HighAvailabilityServices is constructed from the configuration.
-	 * @param localTaskManagerCommunication     If true, the TaskManager will not initiate the TCP network stack.
-	 * @throws IllegalConfigurationException        Thrown, if the given config contains illegal values.
-	 * @throws IOException      Thrown, if any of the I/O components (such as buffer pools, I/O manager, ...)
-	 *                                              cannot be properly started.
-	 * @throws Exception      Thrown is some other error occurs while parsing the configuration or
-	 *                                              starting the TaskManager components.
-	 */
-	private static void createAndStartTaskManagerComponents(
-		Configuration configuration,
-		ResourceID resourceID,
-		RpcService rpcService,
-		String taskManagerHostname,
-		HighAvailabilityServices haServices,
-		boolean localTaskManagerCommunication) throws Exception {
-
-		final TaskExecutorConfiguration taskManagerConfig = parseTaskManagerConfiguration(
-			configuration, taskManagerHostname, localTaskManagerCommunication);
-
-		TaskManagerComponents taskManagerComponents = createTaskManagerComponents(
-			resourceID,
-			InetAddress.getByName(taskManagerHostname),
-			taskManagerConfig,
-			configuration);
-
-		final TaskExecutor taskExecutor = new TaskExecutor(
-			taskManagerConfig,
-			taskManagerComponents.getTaskManagerLocation(),
-			rpcService, taskManagerComponents.getMemoryManager(),
-			taskManagerComponents.getIOManager(),
-			taskManagerComponents.getNetworkEnvironment(),
-			haServices);
-
-		taskExecutor.start();
-	}
-
-	/**
-	 * Creates and returns the task manager components.
-	 *
-	 * @param resourceID resource ID of the task manager
-	 * @param taskManagerAddress address of the task manager
-	 * @param taskExecutorConfig task manager configuration
-	 * @param configuration of Flink
-	 * @return task manager components
-	 * @throws Exception
-	 */
-	private static TaskManagerComponents createTaskManagerComponents(
-		ResourceID resourceID,
-		InetAddress taskManagerAddress,
-		TaskExecutorConfiguration taskExecutorConfig,
-		Configuration configuration) throws Exception {
-
-		MemoryType memType = taskExecutorConfig.getNetworkConfig().memoryType();
-
-		// pre-start checks
-		checkTempDirs(taskExecutorConfig.getTmpDirPaths());
-
-		NetworkEnvironmentConfiguration networkEnvironmentConfiguration = taskExecutorConfig.getNetworkConfig();
-
-		NetworkBufferPool networkBufferPool = new NetworkBufferPool(
-			networkEnvironmentConfiguration.numNetworkBuffers(),
-			networkEnvironmentConfiguration.networkBufferSize(),
-			networkEnvironmentConfiguration.memoryType());
-
-		ConnectionManager connectionManager;
-
-		if (networkEnvironmentConfiguration.nettyConfig().isDefined()) {
-			connectionManager = new NettyConnectionManager(networkEnvironmentConfiguration.nettyConfig().get());
-		} else {
-			connectionManager = new LocalConnectionManager();
-		}
-
-		ResultPartitionManager resultPartitionManager = new ResultPartitionManager();
-		TaskEventDispatcher taskEventDispatcher = new TaskEventDispatcher();
-
-		KvStateRegistry kvStateRegistry = new KvStateRegistry();
-
-		KvStateServer kvStateServer;
-
-		if (networkEnvironmentConfiguration.nettyConfig().isDefined()) {
-			NettyConfig nettyConfig = networkEnvironmentConfiguration.nettyConfig().get();
-
-			int numNetworkThreads = networkEnvironmentConfiguration.queryServerNetworkThreads() == 0 ?
-				nettyConfig.getNumberOfSlots() : networkEnvironmentConfiguration.queryServerNetworkThreads();
-
-			int numQueryThreads = networkEnvironmentConfiguration.queryServerQueryThreads() == 0 ?
-				nettyConfig.getNumberOfSlots() : networkEnvironmentConfiguration.queryServerQueryThreads();
-
-			kvStateServer = new KvStateServer(
-				taskManagerAddress,
-				networkEnvironmentConfiguration.queryServerPort(),
-				numNetworkThreads,
-				numQueryThreads,
-				kvStateRegistry,
-				new DisabledKvStateRequestStats());
-		} else {
-			kvStateServer = null;
-		}
-
-		// we start the network first, to make sure it can allocate its buffers first
-		final NetworkEnvironment network = new NetworkEnvironment(
-			networkBufferPool,
-			connectionManager,
-			resultPartitionManager,
-			taskEventDispatcher,
-			kvStateRegistry,
-			kvStateServer,
-			networkEnvironmentConfiguration.ioMode(),
-			networkEnvironmentConfiguration.partitionRequestInitialBackoff(),
-			networkEnvironmentConfiguration.partitinRequestMaxBackoff());
-
-		network.start();
-
-		final TaskManagerLocation taskManagerLocation = new TaskManagerLocation(
-			resourceID,
-			taskManagerAddress,
-			network.getConnectionManager().getDataPort());
-
-		// computing the amount of memory to use depends on how much memory is available
-		// it strictly needs to happen AFTER the network stack has been initialized
-
-		// check if a value has been configured
-		long configuredMemory = configuration.getLong(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, -1L);
-		checkConfigParameter(configuredMemory == -1 || configuredMemory > 0, configuredMemory,
-			ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY,
-			"MemoryManager needs at least one MB of memory. " +
-				"If you leave this config parameter empty, the system automatically " +
-				"pick a fraction of the available memory.");
-
-		final long memorySize;
-		boolean preAllocateMemory = configuration.getBoolean(
-			ConfigConstants.TASK_MANAGER_MEMORY_PRE_ALLOCATE_KEY,
-			ConfigConstants.DEFAULT_TASK_MANAGER_MEMORY_PRE_ALLOCATE);
-		if (configuredMemory > 0) {
-			if (preAllocateMemory) {
-				LOG.info("Using {} MB for managed memory." , configuredMemory);
-			} else {
-				LOG.info("Limiting managed memory to {} MB, memory will be allocated lazily." , configuredMemory);
-			}
-			memorySize = configuredMemory << 20; // megabytes to bytes
-		} else {
-			float fraction = configuration.getFloat(
-				ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY,
-				ConfigConstants.DEFAULT_MEMORY_MANAGER_MEMORY_FRACTION);
-			checkConfigParameter(fraction > 0.0f && fraction < 1.0f, fraction,
-				ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY,
-				"MemoryManager fraction of the free memory must be between 0.0 and 1.0");
-
-			if (memType == MemoryType.HEAP) {
-				long relativeMemSize = (long) (EnvironmentInformation.getSizeOfFreeHeapMemoryWithDefrag() * fraction);
-				if (preAllocateMemory) {
-					LOG.info("Using {} of the currently free heap space for managed heap memory ({} MB)." ,
-						fraction , relativeMemSize >> 20);
-				} else {
-					LOG.info("Limiting managed memory to {} of the currently free heap space ({} MB), " +
-						"memory will be allocated lazily." , fraction , relativeMemSize >> 20);
-				}
-				memorySize = relativeMemSize;
-			} else if (memType == MemoryType.OFF_HEAP) {
-				// The maximum heap memory has been adjusted according to the fraction
-				long maxMemory = EnvironmentInformation.getMaxJvmHeapMemory();
-				long directMemorySize = (long) (maxMemory / (1.0 - fraction) * fraction);
-				if (preAllocateMemory) {
-					LOG.info("Using {} of the maximum memory size for managed off-heap memory ({} MB)." ,
-						fraction, directMemorySize >> 20);
-				} else {
-					LOG.info("Limiting managed memory to {} of the maximum memory size ({} MB)," +
-						" memory will be allocated lazily.", fraction, directMemorySize >> 20);
-				}
-				memorySize = directMemorySize;
-			} else {
-				throw new RuntimeException("No supported memory type detected.");
-			}
-		}
-
-		// now start the memory manager
-		final MemoryManager memoryManager;
-		try {
-			memoryManager = new MemoryManager(
-				memorySize,
-				taskExecutorConfig.getNumberOfSlots(),
-				taskExecutorConfig.getNetworkConfig().networkBufferSize(),
-				memType,
-				preAllocateMemory);
-		} catch (OutOfMemoryError e) {
-			if (memType == MemoryType.HEAP) {
-				throw new Exception("OutOfMemory error (" + e.getMessage() +
-					") while allocating the TaskManager heap memory (" + memorySize + " bytes).", e);
-			} else if (memType == MemoryType.OFF_HEAP) {
-				throw new Exception("OutOfMemory error (" + e.getMessage() +
-					") while allocating the TaskManager off-heap memory (" + memorySize +
-					" bytes).Try increasing the maximum direct memory (-XX:MaxDirectMemorySize)", e);
-			} else {
-				throw e;
-			}
-		}
-
-		// start the I/O manager, it will create some temp directories.
-		final IOManager ioManager = new IOManagerAsync(taskExecutorConfig.getTmpDirPaths());
-
-		return new TaskManagerComponents(taskManagerLocation, memoryManager, ioManager, network);
-	}
-
-	// --------------------------------------------------------------------------
-	//  Parsing and checking the TaskManager Configuration
-	// --------------------------------------------------------------------------
-
-	/**
-	 * Utility method to extract TaskManager config parameters from the configuration and to
-	 * sanity check them.
-	 *
-	 * @param configuration                         The configuration.
-	 * @param taskManagerHostname           The host name under which the TaskManager communicates.
-	 * @param localTaskManagerCommunication             True, to skip initializing the network stack.
-	 *                                      Use only in cases where only one task manager runs.
-	 * @return TaskExecutorConfiguration that wrappers InstanceConnectionInfo, NetworkEnvironmentConfiguration, etc.
-	 */
-	private static TaskExecutorConfiguration parseTaskManagerConfiguration(
-		Configuration configuration,
-		String taskManagerHostname,
-		boolean localTaskManagerCommunication) throws Exception {
-
-		// ------- read values from the config and check them ---------
-		//                      (a lot of them)
-
-		// ----> hosts / ports for communication and data exchange
-
-		int dataport = configuration.getInteger(ConfigConstants.TASK_MANAGER_DATA_PORT_KEY,
-			ConfigConstants.DEFAULT_TASK_MANAGER_DATA_PORT);
-
-		checkConfigParameter(dataport > 0, dataport, ConfigConstants.TASK_MANAGER_DATA_PORT_KEY,
-			"Leave config parameter empty or use 0 to let the system choose a port automatically.");
-
-		InetAddress taskManagerAddress = InetAddress.getByName(taskManagerHostname);
-		final InetSocketAddress taskManagerInetSocketAddress = new InetSocketAddress(taskManagerAddress, dataport);
-
-		// ----> memory / network stack (shuffles/broadcasts), task slots, temp directories
-
-		// we need this because many configs have been written with a "-1" entry
-		int slots = configuration.getInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 1);
-		if (slots == -1) {
-			slots = 1;
-		}
-
-		checkConfigParameter(slots >= 1, slots, ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS,
-			"Number of task slots must be at least one.");
-
-		final int numNetworkBuffers = configuration.getInteger(
-			ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY,
-			ConfigConstants.DEFAULT_TASK_MANAGER_NETWORK_NUM_BUFFERS);
-
-		checkConfigParameter(numNetworkBuffers > 0, numNetworkBuffers,
-			ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY, "");
-
-		final int pageSize = configuration.getInteger(
-			ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY,
-			ConfigConstants.DEFAULT_TASK_MANAGER_MEMORY_SEGMENT_SIZE);
-
-		// check page size of for minimum size
-		checkConfigParameter(pageSize >= MemoryManager.MIN_PAGE_SIZE, pageSize,
-			ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY,
-			"Minimum memory segment size is " + MemoryManager.MIN_PAGE_SIZE);
-
-		// check page size for power of two
-		checkConfigParameter(MathUtils.isPowerOf2(pageSize), pageSize,
-			ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY,
-			"Memory segment size must be a power of 2.");
-
-		// check whether we use heap or off-heap memory
-		final MemoryType memType;
-		if (configuration.getBoolean(ConfigConstants.TASK_MANAGER_MEMORY_OFF_HEAP_KEY, false)) {
-			memType = MemoryType.OFF_HEAP;
-		} else {
-			memType = MemoryType.HEAP;
-		}
-
-		// initialize the memory segment factory accordingly
-		if (memType == MemoryType.HEAP) {
-			if (!MemorySegmentFactory.initializeIfNotInitialized(HeapMemorySegment.FACTORY)) {
-				throw new Exception("Memory type is set to heap memory, but memory segment " +
-					"factory has been initialized for off-heap memory segments");
-			}
-		} else {
-			if (!MemorySegmentFactory.initializeIfNotInitialized(HybridMemorySegment.FACTORY)) {
-				throw new Exception("Memory type is set to off-heap memory, but memory segment " +
-					"factory has been initialized for heap memory segments");
-			}
-		}
-
-		final String[] tmpDirs = configuration.getString(
-			ConfigConstants.TASK_MANAGER_TMP_DIR_KEY,
-			ConfigConstants.DEFAULT_TASK_MANAGER_TMP_PATH).split(",|" + File.pathSeparator);
-
-		final NettyConfig nettyConfig;
-		if (!localTaskManagerCommunication) {
-			nettyConfig = new NettyConfig(taskManagerInetSocketAddress.getAddress(),
-				taskManagerInetSocketAddress.getPort(), pageSize, slots, configuration);
-		} else {
-			nettyConfig = null;
-		}
-
-		// Default spill I/O mode for intermediate results
-		final String syncOrAsync = configuration.getString(
-			ConfigConstants.TASK_MANAGER_NETWORK_DEFAULT_IO_MODE,
-			ConfigConstants.DEFAULT_TASK_MANAGER_NETWORK_DEFAULT_IO_MODE);
-
-		final IOManager.IOMode ioMode;
-		if (syncOrAsync.equals("async")) {
-			ioMode = IOManager.IOMode.ASYNC;
-		} else {
-			ioMode = IOManager.IOMode.SYNC;
-		}
-
-		final int queryServerPort =  configuration.getInteger(
-			ConfigConstants.QUERYABLE_STATE_SERVER_PORT,
-			ConfigConstants.DEFAULT_QUERYABLE_STATE_SERVER_PORT);
-
-		final int queryServerNetworkThreads =  configuration.getInteger(
-			ConfigConstants.QUERYABLE_STATE_SERVER_NETWORK_THREADS,
-			ConfigConstants.DEFAULT_QUERYABLE_STATE_SERVER_NETWORK_THREADS);
-
-		final int queryServerQueryThreads =  configuration.getInteger(
-			ConfigConstants.QUERYABLE_STATE_SERVER_QUERY_THREADS,
-			ConfigConstants.DEFAULT_QUERYABLE_STATE_SERVER_QUERY_THREADS);
-
-		final NetworkEnvironmentConfiguration networkConfig = new NetworkEnvironmentConfiguration(
-			numNetworkBuffers,
-			pageSize,
-			memType,
-			ioMode,
-			queryServerPort,
-			queryServerNetworkThreads,
-			queryServerQueryThreads,
-			Option.apply(nettyConfig),
-			500,
-			3000);
-
-		// ----> timeouts, library caching, profiling
-
-		final FiniteDuration timeout;
-		try {
-			timeout = AkkaUtils.getTimeout(configuration);
-		} catch (Exception e) {
-			throw new IllegalArgumentException(
-				"Invalid format for '" + ConfigConstants.AKKA_ASK_TIMEOUT +
-					"'.Use formats like '50 s' or '1 min' to specify the timeout.");
-		}
-		LOG.info("Messages between TaskManager and JobManager have a max timeout of " + timeout);
-
-		final long cleanupInterval = configuration.getLong(
-			ConfigConstants.LIBRARY_CACHE_MANAGER_CLEANUP_INTERVAL,
-			ConfigConstants.DEFAULT_LIBRARY_CACHE_MANAGER_CLEANUP_INTERVAL) * 1000;
-
-		final FiniteDuration finiteRegistrationDuration;
-		try {
-			Duration maxRegistrationDuration = Duration.create(configuration.getString(
-				ConfigConstants.TASK_MANAGER_MAX_REGISTRATION_DURATION,
-				ConfigConstants.DEFAULT_TASK_MANAGER_MAX_REGISTRATION_DURATION));
-			if (maxRegistrationDuration.isFinite()) {
-				finiteRegistrationDuration = new FiniteDuration(maxRegistrationDuration.toSeconds(), TimeUnit.SECONDS);
-			} else {
-				finiteRegistrationDuration = null;
-			}
-		} catch (NumberFormatException e) {
-			throw new IllegalArgumentException("Invalid format for parameter " +
-				ConfigConstants.TASK_MANAGER_MAX_REGISTRATION_DURATION, e);
-		}
-
-		final FiniteDuration initialRegistrationPause;
-		try {
-			Duration pause = Duration.create(configuration.getString(
-				ConfigConstants.TASK_MANAGER_INITIAL_REGISTRATION_PAUSE,
-				ConfigConstants.DEFAULT_TASK_MANAGER_INITIAL_REGISTRATION_PAUSE));
-			if (pause.isFinite()) {
-				initialRegistrationPause = new FiniteDuration(pause.toSeconds(), TimeUnit.SECONDS);
-			} else {
-				throw new IllegalArgumentException("The initial registration pause must be finite: " + pause);
-			}
-		} catch (NumberFormatException e) {
-			throw new IllegalArgumentException("Invalid format for parameter " +
-				ConfigConstants.TASK_MANAGER_INITIAL_REGISTRATION_PAUSE, e);
-		}
-
-		final FiniteDuration maxRegistrationPause;
-		try {
-			Duration pause = Duration.create(configuration.getString(
-				ConfigConstants.TASK_MANAGER_MAX_REGISTARTION_PAUSE,
-				ConfigConstants.DEFAULT_TASK_MANAGER_MAX_REGISTRATION_PAUSE));
-			if (pause.isFinite()) {
-				maxRegistrationPause = new FiniteDuration(pause.toSeconds(), TimeUnit.SECONDS);
-			} else {
-				throw new IllegalArgumentException("The maximum registration pause must be finite: " + pause);
-			}
-		} catch (NumberFormatException e) {
-			throw new IllegalArgumentException("Invalid format for parameter " +
-				ConfigConstants.TASK_MANAGER_INITIAL_REGISTRATION_PAUSE, e);
-		}
-
-		final FiniteDuration refusedRegistrationPause;
-		try {
-			Duration pause = Duration.create(configuration.getString(
-				ConfigConstants.TASK_MANAGER_REFUSED_REGISTRATION_PAUSE,
-				ConfigConstants.DEFAULT_TASK_MANAGER_REFUSED_REGISTRATION_PAUSE));
-			if (pause.isFinite()) {
-				refusedRegistrationPause = new FiniteDuration(pause.toSeconds(), TimeUnit.SECONDS);
-			} else {
-				throw new IllegalArgumentException("The refused registration pause must be finite: " + pause);
-			}
-		} catch (NumberFormatException e) {
-			throw new IllegalArgumentException("Invalid format for parameter " +
-				ConfigConstants.TASK_MANAGER_INITIAL_REGISTRATION_PAUSE, e);
-		}
-
-		return new TaskExecutorConfiguration(
-			tmpDirs,
-			cleanupInterval,
-			networkConfig,
-			timeout,
-			finiteRegistrationDuration,
-			slots,
-			configuration,
-			initialRegistrationPause,
-			maxRegistrationPause,
-			refusedRegistrationPause);
-	}
-
-	/**
-	 * Validates a condition for a config parameter and displays a standard exception, if the
-	 * the condition does not hold.
-	 *
-	 * @param condition             The condition that must hold. If the condition is false, an exception is thrown.
-	 * @param parameter         The parameter value. Will be shown in the exception message.
-	 * @param name              The name of the config parameter. Will be shown in the exception message.
-	 * @param errorMessage  The optional custom error message to append to the exception message.
-	 */
-	private static void checkConfigParameter(
-		boolean condition,
-		Object parameter,
-		String name,
-		String errorMessage) {
-		if (!condition) {
-			throw new IllegalConfigurationException("Invalid configuration value for " + name + " : " + parameter + " - " + errorMessage);
-		}
-	}
-
-	/**
-	 * Validates that all the directories denoted by the strings do actually exist, are proper
-	 * directories (not files), and are writable.
-	 *
-	 * @param tmpDirs       The array of directory paths to check.
-	 * @throws Exception    Thrown if any of the directories does not exist or is not writable
-	 *                   or is a file, rather than a directory.
-	 */
-	private static void checkTempDirs(String[] tmpDirs) throws IOException {
-		for (String dir : tmpDirs) {
-			if (dir != null && !dir.equals("")) {
-				File file = new File(dir);
-				if (!file.exists()) {
-					throw new IOException("Temporary file directory " + file.getAbsolutePath() + " does not exist.");
-				}
-				if (!file.isDirectory()) {
-					throw new IOException("Temporary file directory " + file.getAbsolutePath() + " is not a directory.");
-				}
-				if (!file.canWrite()) {
-					throw new IOException("Temporary file directory " + file.getAbsolutePath() + " is not writable.");
-				}
-
-				if (LOG.isInfoEnabled()) {
-					long totalSpaceGb = file.getTotalSpace() >> 30;
-					long usableSpaceGb = file.getUsableSpace() >> 30;
-					double usablePercentage = (double)usableSpaceGb / totalSpaceGb * 100;
-					String path = file.getAbsolutePath();
-					LOG.info(String.format("Temporary file directory '%s': total %d GB, " + "usable %d GB (%.2f%% usable)",
-						path, totalSpaceGb, usableSpaceGb, usablePercentage));
-				}
-			} else {
-				throw new IllegalArgumentException("Temporary file directory #$id is null.");
-			}
-		}
-	}
-
-	private static class TaskManagerComponents {
-		private final TaskManagerLocation taskManagerLocation;
-		private final MemoryManager memoryManager;
-		private final IOManager ioManager;
-		private final NetworkEnvironment networkEnvironment;
-
-		private TaskManagerComponents(
-			TaskManagerLocation taskManagerLocation,
-			MemoryManager memoryManager,
-			IOManager ioManager,
-			NetworkEnvironment networkEnvironment) {
-
-			this.taskManagerLocation = Preconditions.checkNotNull(taskManagerLocation);
-			this.memoryManager = Preconditions.checkNotNull(memoryManager);
-			this.ioManager = Preconditions.checkNotNull(ioManager);
-			this.networkEnvironment = Preconditions.checkNotNull(networkEnvironment);
-		}
-
-		public MemoryManager getMemoryManager() {
-			return memoryManager;
-		}
-
-		public IOManager getIOManager() {
-			return ioManager;
-		}
-
-		public NetworkEnvironment getNetworkEnvironment() {
-			return networkEnvironment;
-		}
-
-		public TaskManagerLocation getTaskManagerLocation() {
-			return taskManagerLocation;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/bb781aef/flink-runtime/src/main/java/org/apache/flink/runtime/util/LeaderRetrievalUtils.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/LeaderRetrievalUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/LeaderRetrievalUtils.java
index b6d9306..42655a2 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/util/LeaderRetrievalUtils.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/LeaderRetrievalUtils.java
@@ -22,6 +22,7 @@ import akka.actor.ActorRef;
 import akka.actor.ActorSystem;
 import akka.dispatch.Mapper;
 import akka.dispatch.OnComplete;
+import org.apache.flink.api.common.time.Time;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.IllegalConfigurationException;
@@ -170,6 +171,12 @@ public class LeaderRetrievalUtils {
 	}
 
 	public static InetAddress findConnectingAddress(
+		LeaderRetrievalService leaderRetrievalService,
+		Time timeout) throws LeaderRetrievalException {
+		return findConnectingAddress(leaderRetrievalService, new FiniteDuration(timeout.getSize(), timeout.getUnit()));
+	}
+
+	public static InetAddress findConnectingAddress(
 			LeaderRetrievalService leaderRetrievalService,
 			FiniteDuration timeout) throws LeaderRetrievalException {
 		ConnectionUtils.LeaderConnectingAddressListener listener = new ConnectionUtils.LeaderConnectingAddressListener();

http://git-wip-us.apache.org/repos/asf/flink/blob/bb781aef/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 bd3af33..84f5ac7 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
@@ -94,6 +94,10 @@ object AkkaUtils {
     createActorSystem(getDefaultAkkaConfig)
   }
 
+  def getAkkaConfig(configuration: Configuration, hostname: String, port: Int): Config = {
+    getAkkaConfig(configuration, if (hostname == null) Some((hostname, port)) else None)
+  }
+
   /**
    * Creates an akka config with the provided configuration values. If the listening address is
    * specified, then the actor system will listen on the respective address.

http://git-wip-us.apache.org/repos/asf/flink/blob/bb781aef/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/NetworkEnvironmentConfiguration.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/NetworkEnvironmentConfiguration.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/NetworkEnvironmentConfiguration.scala
index 893eaa8..97aae34 100644
--- a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/NetworkEnvironmentConfiguration.scala
+++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/NetworkEnvironmentConfiguration.scala
@@ -30,6 +30,6 @@ case class NetworkEnvironmentConfiguration(
   queryServerPort: Int,
   queryServerNetworkThreads: Int,
   queryServerQueryThreads: Int,
-  nettyConfig: Option[NettyConfig] = None,
+  nettyConfig: NettyConfig = null,
   partitionRequestInitialBackoff: Int = 500,
   partitinRequestMaxBackoff: Int = 3000)

http://git-wip-us.apache.org/repos/asf/flink/blob/bb781aef/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala
index af2b38f..79670a4 100644
--- a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala
+++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala
@@ -1932,7 +1932,7 @@ object TaskManager {
       netConfig.networkBufferSize,
       netConfig.memoryType)
 
-    val connectionManager = netConfig.nettyConfig match {
+    val connectionManager = Option(netConfig.nettyConfig) match {
       case Some(nettyConfig) => new NettyConnectionManager(nettyConfig)
       case None => new LocalConnectionManager()
     }
@@ -1942,7 +1942,7 @@ object TaskManager {
 
     val kvStateRegistry = new KvStateRegistry()
 
-    val kvStateServer = netConfig.nettyConfig match {
+    val kvStateServer = Option(netConfig.nettyConfig) match {
       case Some(nettyConfig) =>
 
         val numNetworkThreads = if (netConfig.queryServerNetworkThreads == 0) {
@@ -2274,7 +2274,7 @@ object TaskManager {
       queryServerPort,
       queryServerNetworkThreads,
       queryServerQueryThreads,
-      nettyConfig)
+      nettyConfig.getOrElse(null))
 
     // ----> timeouts, library caching, profiling
 

http://git-wip-us.apache.org/repos/asf/flink/blob/bb781aef/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/NetworkEnvironmentTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/NetworkEnvironmentTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/NetworkEnvironmentTest.java
index a9ad75d..cc50b66 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/NetworkEnvironmentTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/NetworkEnvironmentTest.java
@@ -24,7 +24,6 @@ import org.apache.flink.runtime.instance.ActorGateway;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter;
 import org.apache.flink.runtime.io.network.buffer.NetworkBufferPool;
-import org.apache.flink.runtime.io.network.netty.NettyConfig;
 import org.apache.flink.runtime.io.network.partition.ResultPartition;
 import org.apache.flink.runtime.io.network.partition.ResultPartitionConsumableNotifier;
 import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
@@ -38,7 +37,6 @@ import org.apache.flink.runtime.taskmanager.NetworkEnvironmentConfiguration;
 import org.apache.flink.runtime.taskmanager.Task;
 import org.apache.flink.runtime.testingUtils.TestingUtils;
 import org.junit.Test;
-import scala.Some;
 import scala.concurrent.duration.FiniteDuration;
 import scala.concurrent.impl.Promise;
 
@@ -75,7 +73,7 @@ public class NetworkEnvironmentTest {
 			0,
 			0,
 			0,
-			Some.<NettyConfig>empty(),
+			null,
 			0,
 			0);
 

http://git-wip-us.apache.org/repos/asf/flink/blob/bb781aef/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 5b8e6e6..2a004c5 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
@@ -30,7 +30,6 @@ import java.lang.annotation.Annotation;
 import java.lang.reflect.InvocationHandler;
 import java.lang.reflect.Method;
 import java.lang.reflect.Proxy;
-import java.net.InetAddress;
 import java.util.BitSet;
 import java.util.UUID;
 import java.util.concurrent.Callable;

http://git-wip-us.apache.org/repos/asf/flink/blob/bb781aef/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 26218dd..9c1f288 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
@@ -26,8 +26,9 @@ import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.io.network.NetworkEnvironment;
 import org.apache.flink.runtime.leaderelection.TestingLeaderRetrievalService;
 import org.apache.flink.runtime.memory.MemoryManager;
-import org.apache.flink.runtime.rpc.TestingRpcService;
+import org.apache.flink.runtime.metrics.MetricRegistry;
 import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
 import org.apache.flink.runtime.rpc.TestingSerialRpcService;
 import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
 import org.apache.flink.util.TestLogger;
@@ -51,8 +52,8 @@ public class TaskExecutorTest extends TestLogger {
 		try {
 			// register a mock resource manager gateway
 			ResourceManagerGateway rmGateway = mock(ResourceManagerGateway.class);
-			TaskExecutorConfiguration taskExecutorConfiguration = mock(TaskExecutorConfiguration.class);
-			PowerMockito.when(taskExecutorConfiguration.getNumberOfSlots()).thenReturn(1);
+			TaskManagerConfiguration taskManagerServicesConfiguration = mock(TaskManagerConfiguration.class);
+			PowerMockito.when(taskManagerServicesConfiguration.getNumberSlots()).thenReturn(1);
 			rpc.registerGateway(resourceManagerAddress, rmGateway);
 
 			TaskManagerLocation taskManagerLocation = mock(TaskManagerLocation.class);
@@ -61,12 +62,15 @@ public class TaskExecutorTest extends TestLogger {
 			NonHaServices haServices = new NonHaServices(resourceManagerAddress);
 
 			TaskExecutor taskManager = new TaskExecutor(
-				taskExecutorConfiguration,
+				taskManagerServicesConfiguration,
 				taskManagerLocation,
-				rpc, mock(MemoryManager.class),
+				rpc,
+				mock(MemoryManager.class),
 				mock(IOManager.class),
 				mock(NetworkEnvironment.class),
-				haServices);
+				haServices,
+				mock(MetricRegistry.class),
+				mock(FatalErrorHandler.class));
 
 			taskManager.start();
 			String taskManagerAddress = taskManager.getAddress();
@@ -101,19 +105,22 @@ public class TaskExecutorTest extends TestLogger {
 			TestingHighAvailabilityServices haServices = new TestingHighAvailabilityServices();
 			haServices.setResourceManagerLeaderRetriever(testLeaderService);
 
-			TaskExecutorConfiguration taskExecutorConfiguration = mock(TaskExecutorConfiguration.class);
-			PowerMockito.when(taskExecutorConfiguration.getNumberOfSlots()).thenReturn(1);
+			TaskManagerConfiguration taskManagerServicesConfiguration = mock(TaskManagerConfiguration.class);
+			PowerMockito.when(taskManagerServicesConfiguration.getNumberSlots()).thenReturn(1);
 
 			TaskManagerLocation taskManagerLocation = mock(TaskManagerLocation.class);
 			when(taskManagerLocation.getResourceID()).thenReturn(resourceID);
 
 			TaskExecutor taskManager = new TaskExecutor(
-				taskExecutorConfiguration,
+				taskManagerServicesConfiguration,
 				taskManagerLocation,
-				rpc, mock(MemoryManager.class),
+				rpc,
+				mock(MemoryManager.class),
 				mock(IOManager.class),
 				mock(NetworkEnvironment.class),
-				haServices);
+				haServices,
+				mock(MetricRegistry.class),
+				mock(FatalErrorHandler.class));
 
 			taskManager.start();
 			String taskManagerAddress = taskManager.getAddress();

http://git-wip-us.apache.org/repos/asf/flink/blob/bb781aef/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerComponentsStartupShutdownTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerComponentsStartupShutdownTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerComponentsStartupShutdownTest.java
index 1f93e9b..627a25a 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerComponentsStartupShutdownTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerComponentsStartupShutdownTest.java
@@ -39,7 +39,6 @@ import org.apache.flink.runtime.io.network.LocalConnectionManager;
 import org.apache.flink.runtime.io.network.NetworkEnvironment;
 import org.apache.flink.runtime.io.network.TaskEventDispatcher;
 import org.apache.flink.runtime.io.network.buffer.NetworkBufferPool;
-import org.apache.flink.runtime.io.network.netty.NettyConfig;
 import org.apache.flink.runtime.io.network.partition.ResultPartitionManager;
 import org.apache.flink.runtime.jobmanager.JobManager;
 import org.apache.flink.runtime.jobmanager.MemoryArchivist;
@@ -105,7 +104,7 @@ public class TaskManagerComponentsStartupShutdownTest {
 
 			final NetworkEnvironmentConfiguration netConf = new NetworkEnvironmentConfiguration(
 					32, BUFFER_SIZE, MemoryType.HEAP, IOManager.IOMode.SYNC, 0, 0, 0,
-					Option.<NettyConfig>empty(), 0, 0);
+					null, 0, 0);
 
 			ResourceID taskManagerId = ResourceID.generate();
 			

http://git-wip-us.apache.org/repos/asf/flink/blob/bb781aef/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerConfigurationTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerConfigurationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerConfigurationTest.java
index acfbbfd..c0d0455 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerConfigurationTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerConfigurationTest.java
@@ -29,7 +29,6 @@ import org.junit.Test;
 import scala.Tuple2;
 
 import java.io.File;
-import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.PrintWriter;
 import java.lang.reflect.Field;