You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by tillrohrmann <gi...@git.apache.org> on 2017/02/22 17:26:40 UTC

[GitHub] flink pull request #3394: [FLINK-5810] [flip6] Introduce a hardened slot man...

GitHub user tillrohrmann opened a pull request:

    https://github.com/apache/flink/pull/3394

    [FLINK-5810] [flip6] Introduce a hardened slot manager

    This PR is based on #3310.
    
    Harden the slot manager so that it better deals with lost and out of order messages
    from the TaskManager. The basic idea is that the TaskManager are considered the ground
    truth and the SlotManager tries to maintain a consistent view of what is reported to it
    by the TaskManagers. This has the assumption that the TaskManagers regularly report their
    slot status to the SlotManager piggy backed on the heartbeat signals to the ResourceManager (not yet implemented, though). That way it is possible to handle lost and out of order messages because the SlotManager will eventually converge on a consistent view of the actual slot allocation.
    
    Additionally, the hardened SlotManager registers for idle TaskManagers and pending slot
    requests a timeout. If the timeout expires, then the TaskManagers are released and the
    slot requests are failed, respectively. This prevents resource leaks and wasteful resource allocation.

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/tillrohrmann/flink newSlotManager

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/flink/pull/3394.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #3394
    
----
commit 336e479e9892acbdaf54b36d98dc810ea7192c39
Author: Till Rohrmann <tr...@apache.org>
Date:   2017-02-14T15:50:43Z

    [FLINK-5798] [rpc] Let the RpcService provide a ScheduledExecutorService
    
    This PR adds the getScheduledExecutorService method to the RpcService interface. So
    henceforth all RpcService implementations have to provide a ScheduledExecutorService
    implementation.
    
    Currently, we only support the AkkaRpcService. The AkkaRpcService returns a
    ScheduledExecutorService proxy which forwards the schedule calls to the ActorSystem's
    internal scheduler.

commit f5a7de2811ef21b55edcb74ad247664d251ac071
Author: Till Rohrmann <tr...@apache.org>
Date:   2017-02-22T16:49:33Z

    Introduce ScheduledExecutor interface to hide service methods from the ScheduledExecutorService

commit 857a8f7e6b363bc4a57d8950a74367e8e8bfe195
Author: Till Rohrmann <tr...@apache.org>
Date:   2017-02-09T10:59:45Z

    [FLINK-5810] [flip6] Introduce a hardened slot manager
    
    Harden the slot manager so that it better deals with lost and out of order messages
    from the TaskManager. The basic idea is that the TaskManager are considered the ground
    truth and the SlotManager tries to maintain a consistent view of what is reported to it
    by the TaskManagers. This has the assumption that the TaskManagers regularly report their
    slot status to the SlotManager piggy backed on the heartbeat signals to the ResourceManager.
    That way it is possible to handle lost and out of order messages because the SlotManager
    will eventually converge on a consistent view of the actual slot allocation.
    
    Additionally, the hardened SlotManager registers for idle TaskManagers and pending slot
    requests a timeout. If the timoeut expires, then the TaskManagers are released and the
    slot request is failed. This prevents resource leaks and wasteful resource allocation.

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3394: [FLINK-5810] [flip6] Introduce a hardened slot man...

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3394#discussion_r113497111
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManager.java ---
    @@ -21,519 +21,897 @@
     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.ResourceSlot;
    +import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
    +import org.apache.flink.runtime.clusterframework.types.TaskManagerSlot;
     import org.apache.flink.runtime.clusterframework.types.SlotID;
     import org.apache.flink.runtime.concurrent.BiFunction;
    +import org.apache.flink.runtime.concurrent.CompletableFuture;
     import org.apache.flink.runtime.concurrent.Future;
    -import org.apache.flink.runtime.resourcemanager.ResourceManagerServices;
    +import org.apache.flink.runtime.concurrent.ScheduledExecutor;
    +import org.apache.flink.runtime.concurrent.impl.FlinkCompletableFuture;
    +import org.apache.flink.runtime.instance.InstanceID;
    +import org.apache.flink.runtime.messages.Acknowledge;
     import org.apache.flink.runtime.resourcemanager.SlotRequest;
    -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.resourcemanager.exceptions.ResourceManagerException;
    +import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
     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.runtime.taskexecutor.exceptions.SlotAllocationException;
    +import org.apache.flink.runtime.taskexecutor.exceptions.SlotOccupiedException;
     import org.apache.flink.util.Preconditions;
     import org.slf4j.Logger;
     import org.slf4j.LoggerFactory;
     
     import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.Iterator;
     import java.util.LinkedHashMap;
     import java.util.Map;
    -
    -import static org.apache.flink.util.Preconditions.checkNotNull;
    +import java.util.Objects;
    +import java.util.UUID;
    +import java.util.concurrent.CancellationException;
    +import java.util.concurrent.Executor;
    +import java.util.concurrent.ScheduledFuture;
    +import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.TimeoutException;
     
     /**
    - * 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>.
    + * The slot manager is responsible for maintaining a view on all registered task manager slots,
    + * their allocation and all pending slot requests. Whenever a new slot is registered or and
    + * allocated slot is freed, then it tries to fulfill another pending slot request. Whenever there
    + * are not enough slots available the slot manager will notify the resource manager about it via
    + * {@link ResourceManagerActions#allocateResource(ResourceProfile)}.
    + *
    + * In order to free resources and avoid resource leaks, idling task managers (task managers whose
    + * slots are currently not used) and not fulfilled pending slot requests time out triggering their
    + * release and failure, respectively.
      */
    -public abstract class SlotManager {
    +public class SlotManager implements AutoCloseable {
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(SlotManager.class);
    +
    +	/** Scheduled executor for timeouts */
    +	private final ScheduledExecutor scheduledExecutor;
    +
    +	/** Timeout for slot requests to the task manager */
    +	private final Time taskManagerRequestTimeout;
    +
    +	/** Timeout after which an allocation is discarded */
    +	private final Time slotRequestTimeout;
    +
    +	/** Timeout after which an unused TaskManager is released */
    +	private final Time taskManagerTimeout;
    +
    +	/** Map for all registered slots */
    +	private final HashMap<SlotID, TaskManagerSlot> slots;
     
    -	protected final Logger LOG = LoggerFactory.getLogger(getClass());
    +	/** Index of all currently free slots */
    +	private final LinkedHashMap<SlotID, TaskManagerSlot> freeSlots;
     
    -	/** The Resource allocation provider */
    -	protected final ResourceManagerServices rmServices;
    +	/** All currently registered task managers */
    +	private final HashMap<InstanceID, TaskManagerRegistration> taskManagerRegistrations;
     
    -	/** All registered task managers with ResourceID and gateway. */
    -	private final Map<ResourceID, TaskExecutorRegistration> taskManagers;
    +	/** Map of fulfilled and active allocations for request deduplication purposes */
    +	private final HashMap<AllocationID, SlotID> fulfilledSlotRequests;
     
    -	/** All registered slots, including free and allocated slots */
    -	private final Map<ResourceID, Map<SlotID, ResourceSlot>> registeredSlots;
    +	/** Map of pending/unfulfilled slot allocation requests */
    +	private final HashMap<AllocationID, PendingSlotRequest> pendingSlotRequests;
     
    -	/** All pending slot requests, waiting available slots to fulfil */
    -	private final Map<AllocationID, SlotRequest> pendingSlotRequests;
    +	/** Leader id of the containing component */
    +	private UUID leaderId;
     
    -	/** All free slots that can be used to be allocated */
    -	private final Map<SlotID, ResourceSlot> freeSlots;
    +	/** Executor for future callbacks which have to be "synchronized" */
    +	private Executor mainThreadExecutor;
     
    -	/** All allocations, we can lookup allocations either by SlotID or AllocationID */
    -	private final AllocationMap allocationMap;
    +	/** Callbacks for resource (de-)allocations */
    +	private ResourceManagerActions resourceManagerActions;
     
    -	private final Time timeout;
    +	/** True iff the component has been started */
    +	private boolean started;
     
    -	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.taskManagers = new HashMap<>();
    -		this.timeout = Time.seconds(10);
    +	public SlotManager(
    +			ScheduledExecutor scheduledExecutor,
    +			Time taskManagerRequestTimeout,
    +			Time slotRequestTimeout,
    +			Time taskManagerTimeout) {
    +		this.scheduledExecutor = Preconditions.checkNotNull(scheduledExecutor);
    +		this.taskManagerRequestTimeout = Preconditions.checkNotNull(taskManagerRequestTimeout);
    +		this.slotRequestTimeout = Preconditions.checkNotNull(slotRequestTimeout);
    +		this.taskManagerTimeout = Preconditions.checkNotNull(taskManagerTimeout);
    +
    +		slots = new HashMap<>(16);
    +		freeSlots = new LinkedHashMap<>(16);
    +		taskManagerRegistrations = new HashMap<>(4);
    +		fulfilledSlotRequests = new HashMap<>(16);
    +		pendingSlotRequests = new HashMap<>(16);
    +
    +		leaderId = null;
    +		resourceManagerActions = null;
    +		started = false;
     	}
     
    -	// ------------------------------------------------------------------------
    -	//  slot managements
    -	// ------------------------------------------------------------------------
    +	// ---------------------------------------------------------------------------------------------
    +	// Component lifecycle methods
    +	// ---------------------------------------------------------------------------------------------
     
     	/**
    -	 * 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).
    +	 * Starts the slot manager with the given leader id and resource manager actions.
     	 *
    -	 * @param request The detailed request of the slot
    -	 * @return RMSlotRequestRegistered The confirmation message to be send to the caller
    +	 * @param newLeaderId to use for communication with the task managers
    +	 * @param newResourceManagerActions to use for resource (de-)allocations
    +	 */
    +	public void start(UUID newLeaderId, Executor newMainThreadExecutor, ResourceManagerActions newResourceManagerActions) {
    +		leaderId = Preconditions.checkNotNull(newLeaderId);
    +		mainThreadExecutor = Preconditions.checkNotNull(newMainThreadExecutor);
    +		resourceManagerActions = Preconditions.checkNotNull(newResourceManagerActions);
    +
    +		started = true;
    +	}
    +
    +	/**
    +	 * Suspends the component. This clears the internal state of the slot manager.
     	 */
    -	public RMSlotRequestRegistered requestSlot(final SlotRequest request) {
    -		final AllocationID allocationId = request.getAllocationId();
    -		if (isRequestDuplicated(request)) {
    -			LOG.warn("Duplicated slot request, AllocationID:{}", allocationId);
    -			return new RMSlotRequestRegistered(allocationId);
    +	public void suspend() {
    +		for (PendingSlotRequest pendingSlotRequest : pendingSlotRequests.values()) {
    +			cancelPendingSlotRequest(pendingSlotRequest);
     		}
     
    -		// 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());
    +		pendingSlotRequests.clear();
    +
    +		HashSet<InstanceID> registeredTaskManagers = new HashSet<>(taskManagerRegistrations.keySet());
    +
    +		for (InstanceID registeredTaskManager : registeredTaskManagers) {
    +			unregisterTaskManager(registeredTaskManager);
    +		}
    +
    +		leaderId = null;
    +		resourceManagerActions = null;
    +		started = false;
    +	}
    +
    +	/**
    +	 * Closes the slot manager.
    +	 *
    +	 * @throws Exception if the close operation fails
    +	 */
    +	@Override
    +	public void close() throws Exception {
    +		suspend();
    +	}
    +
    +	// ---------------------------------------------------------------------------------------------
    +	// Public API
    +	// ---------------------------------------------------------------------------------------------
    +
    +	/**
    +	 * Requests a slot with the respective resource profile.
    +	 *
    +	 * @param slotRequest specifying the requested slot specs
    +	 * @return true if the slot request was registered; false if the request is a duplicate
    +	 * @throws SlotManagerException if the slot request failed (e.g. not enough resources left)
    +	 */
    +	public boolean registerSlotRequest(SlotRequest slotRequest) throws SlotManagerException {
    +		checkInit();
     
    -			// record this allocation in bookkeeping
    -			allocationMap.addAllocation(slot.getSlotId(), allocationId);
    -			// remove selected slot from free pool
    -			freeSlots.remove(slot.getSlotId());
    +		if (checkDuplicateRequest(slotRequest.getAllocationId())) {
    +			LOG.debug("Ignoring a duplicate slot request with allocation id {}.", slotRequest.getAllocationId());
     
    -			sendSlotRequest(slot, request);
    +			return false;
     		} else {
    -			LOG.info("Cannot fulfil slot request, try to allocate a new container for it, " +
    -				"AllocationID:{}, JobID:{}", allocationId, request.getJobId());
    -			Preconditions.checkState(rmServices != null,
    -				"Attempted to allocate resources but no ResourceManagerServices set.");
    -			rmServices.allocateResource(request.getResourceProfile());
    -			pendingSlotRequests.put(allocationId, request);
    +			PendingSlotRequest pendingSlotRequest = new PendingSlotRequest(slotRequest);
    +
    +			pendingSlotRequests.put(slotRequest.getAllocationId(), pendingSlotRequest);
    +
    +			try {
    +				internalRequestSlot(pendingSlotRequest);
    +			} catch (ResourceManagerException e) {
    +				// requesting the slot failed --> remove pending slot request
    +				pendingSlotRequests.remove(slotRequest.getAllocationId());
    +
    +				throw new SlotManagerException("Could not fulfill slot request " + slotRequest.getAllocationId() + '.', e);
    +			}
    +
    +			return true;
     		}
    +	}
     
    -		return new RMSlotRequestRegistered(allocationId);
    +	/**
    +	 * Cancels and removes a pending slot request with the given allocation id. If there is no such
    +	 * pending request, then nothing is done.
    +	 *
    +	 * @param allocationId identifying the pending slot request
    +	 * @return True if a pending slot request was found; otherwise false
    +	 */
    +	public boolean unregisterSlotRequest(AllocationID allocationId) {
    +		checkInit();
    +
    +		PendingSlotRequest pendingSlotRequest = pendingSlotRequests.remove(allocationId);
    +
    +		if (null != pendingSlotRequest) {
    +			cancelPendingSlotRequest(pendingSlotRequest);
    +
    +			return true;
    +		} else {
    +			LOG.debug("No pending slot request with allocation id {} found.", allocationId);
    +
    +			return false;
    +		}
     	}
     
     	/**
    -	 * Notifies the SlotManager that a slot is available again after being allocated.
    -	 * @param slotID slot id of available slot
    +	 * Registers a new task manager at the slot manager. This will make the task managers slots
    +	 * known and, thus, available for allocation.
    +	 *
    +	 * @param taskExecutorConnection for the new task manager
    +	 * @param initialSlotReport for the new task manager
     	 */
    -	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");
    +	public void registerTaskManager(final TaskExecutorConnection taskExecutorConnection, SlotReport initialSlotReport) {
    +		checkInit();
    +
    +		// we identify task managers by their instance id
    +		if (!taskManagerRegistrations.containsKey(taskExecutorConnection.getInstanceID())) {
    +			TaskManagerRegistration taskManagerRegistration = new TaskManagerRegistration(taskExecutorConnection);
    +			taskManagerRegistrations.put(taskExecutorConnection.getInstanceID(), taskManagerRegistration);
     		}
    -		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.");
    +
    +		reportSlotStatus(taskExecutorConnection.getInstanceID(), initialSlotReport);
    +	}
    +
    +	/**
    +	 * Unregisters the task manager identified by the given instance id and its associated slots
    +	 * from the slot manager.
    +	 *
    +	 * @param instanceId identifying the task manager to unregister
    +	 * @return True if there existed a registered task manager with the given instance id
    +	 */
    +	public boolean unregisterTaskManager(InstanceID instanceId) {
    +		checkInit();
    +
    +		TaskManagerRegistration taskManagerRegistration = taskManagerRegistrations.remove(instanceId);
    +
    +		if (null != taskManagerRegistration) {
    +			removeSlots(taskManagerRegistration.getSlots());
    +
    +			taskManagerRegistration.cancelTimeout();
    +
    +			return true;
    +		} else {
    +			LOG.debug("There is no task manager registered with instance ID {}. Ignoring this message.", instanceId);
    +
    +			return false;
     		}
    -		handleFreeSlot(freeSlot);
     	}
     
     	/**
    -	 * 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.
    +	 * Reports the current slot allocations for a task manager identified by the given instance id.
     	 *
    -	 * @param originalRequest The original slot request
    -	 * @param slotId          The target SlotID
    +	 * @param instanceId identifying the task manager for which to report the slot status
    +	 * @param slotReport containing the status for all of its slots
    +	 * @return true if the slot status has been updated successfully, otherwise false
     	 */
    -	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());
    -
    -		if (allocationMap.isAllocated(slotId)) {
    -			final AllocationID expectedAllocationId = allocationMap.getAllocationID(slotId);
    -
    -			// 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);
    +	public boolean reportSlotStatus(InstanceID instanceId, SlotReport slotReport) {
    +		checkInit();
    +
    +		TaskManagerRegistration taskManagerRegistration = taskManagerRegistrations.get(instanceId);
    +
    +		if (null != taskManagerRegistration) {
    +			HashSet<SlotID> slotsToRemove = new HashSet<>(taskManagerRegistration.getSlots());
    +			boolean idle = true;
    +
    +			for (SlotStatus slotStatus : slotReport) {
    +				if (slotsToRemove.remove(slotStatus.getSlotID())) {
    +					// slot which was already registered
    +					updateSlot(slotStatus.getSlotID(), slotStatus.getAllocationID());
    +				} else {
    +					// new slot
    +					registerSlot(
    +						instanceId,
    +						slotStatus.getSlotID(),
    +						slotStatus.getAllocationID(),
    +						slotStatus.getResourceProfile(),
    +						taskManagerRegistration.getTaskManagerConnection());
    +				}
    +
    +				TaskManagerSlot slot = slots.get(slotStatus.getSlotID());
    +
    +				idle &= slot.isFree();
    +			}
    +
    +			// remove the slots for which we haven't received a slot status message
    +			removeSlots(slotsToRemove);
    +
    +			if (idle) {
    +				// no slot of this task manager is being used --> register timer to free this resource
    +				registerTaskManagerTimeout(taskManagerRegistration);
    +			}
    +
    +			return true;
    +		} else {
    +			LOG.debug("Received slot report for unknown task manager with instance id {}. Ignoring this report.", instanceId);
    +
    +			return false;
    +		}
    +	}
    +
    +	/**
    +	 * Free the given slot from the given allocation. If the slot is still allocated by the given
    +	 * allocation id, then the slot will be marked as free and will be subject to new slot requests.
    +	 *
    +	 * @param slotId identifying the slot to free
    +	 * @param allocationId with which the slot is presumably allocated
    +	 */
    +	public void freeSlot(SlotID slotId, AllocationID allocationId) {
    +		checkInit();
    +
    +		TaskManagerSlot slot = slots.get(slotId);
    +
    +		if (null != slot) {
    +			if (slot.isAllocated()) {
    +				if (Objects.equals(allocationId, slot.getAllocationId())) {
    +					// free the slot
    +					slot.setAllocationId(null);
    +					fulfilledSlotRequests.remove(allocationId);
    +
    +					if (slot.isFree()) {
    +						handleFreeSlot(slot);
    +					}
    +
    +					TaskManagerRegistration taskManagerRegistration = taskManagerRegistrations.get(slot.getInstanceId());
    +
    +					if (null != taskManagerRegistration && !anySlotUsed(taskManagerRegistration.getSlots())) {
    +						registerTaskManagerTimeout(taskManagerRegistration);
    +					}
    +				} else {
    +					LOG.debug("Received request to free slot {} with expected allocation id {}, " +
    +						"but actual allocation id {} differs. Ignoring the request.", slotId, allocationId, slot.getAllocationId());
    +				}
     			} else {
    -				LOG.error("Slot request failed for slot {} with allocation id {}:" +
    -						" Allocation id did not match the expected allocation id {}.",
    -					slotId, originalAllocationId, expectedAllocationId);
    +				LOG.debug("Slot {} has not been allocated.", allocationId);
     			}
     		} else {
    -			LOG.error("Slot request failed for slot {} with allocation id {}: " +
    -					"Slot was not previously registered.",
    -				slotId, originalAllocationId);
    +			LOG.debug("Trying to free a slot {} which has not been registered. Ignoring this message.", slotId);
     		}
     	}
     
    +	// ---------------------------------------------------------------------------------------------
    +	// Behaviour methods
    +	// ---------------------------------------------------------------------------------------------
    +
     	/**
    -	 * Registers a TaskExecutor
    -	 * @param resourceID TaskExecutor's ResourceID
    -	 * @param registration TaskExecutor's registration
    -	 * @param slotReport TaskExecutor's free and allocated slots
    +	 * Finds a matching slot request for a given resource profile. If there is no such request,
    +	 * the method returns null.
    +	 *
    +	 * Note: If you want to change the behaviour of the slot manager wrt slot allocation and
    +	 * request fulfillment, then you should override this method.
    +	 *
    +	 * @param slotResourceProfile defining the resources of an available slot
    +	 * @return A matching slot request which can be deployed in a slot with the given resource
    +	 * profile. Null if there is no such slot request pending.
     	 */
    -	public void registerTaskExecutor(
    -			ResourceID resourceID,
    -			TaskExecutorRegistration registration,
    -			SlotReport slotReport) {
    +	protected PendingSlotRequest findMatchingRequest(ResourceProfile slotResourceProfile) {
     
    -		if (taskManagers.get(resourceID) != null) {
    -			notifyTaskManagerFailure(resourceID);
    +		for (PendingSlotRequest pendingSlotRequest : pendingSlotRequests.values()) {
    +			if (!pendingSlotRequest.isAssigned() && slotResourceProfile.isMatching(pendingSlotRequest.getResourceProfile())) {
    +				return pendingSlotRequest;
    +			}
     		}
     
    -		this.taskManagers.put(resourceID, registration);
    +		return null;
    +	}
    +
    +	/**
    +	 * Finds a matching slot for a given resource profile. A matching slot has at least as many
    +	 * resources available as the given resource profile. If there is no such slot available, then
    +	 * the method returns null.
    +	 *
    +	 * Note: If you want to change the behaviour of the slot manager wrt slot allocation and
    +	 * request fulfillment, then you should override this method.
    +	 *
    +	 * @param requestResourceProfile specifying the resource requirements for the a slot request
    +	 * @return A matching slot which fulfills the given resource profile. Null if there is no such
    +	 * slot available.
    +	 */
    +	protected TaskManagerSlot findMatchingSlot(ResourceProfile requestResourceProfile) {
    +		Iterator<Map.Entry<SlotID, TaskManagerSlot>> iterator = freeSlots.entrySet().iterator();
    +
    +		while (iterator.hasNext()) {
    +			TaskManagerSlot taskManagerSlot = iterator.next().getValue();
     
    -		for (SlotStatus slotStatus : slotReport.getSlotsStatus()) {
    -			final SlotID slotId = slotStatus.getSlotID();
    +			// sanity check
    +			Preconditions.checkState(taskManagerSlot.isFree());
     
    -			final TaskExecutorRegistration taskExecutorRegistration = taskManagers.get(slotId.getResourceID());
    -			if (taskExecutorRegistration == null) {
    -				LOG.info("Received SlotStatus but ResourceID {} is unknown to the SlotManager",
    -					slotId.getResourceID());
    -				return;
    +			if (taskManagerSlot.getResourceProfile().isMatching(requestResourceProfile)) {
    +				iterator.remove();
    +				return taskManagerSlot;
     			}
    +		}
     
    -			final ResourceSlot slot = new ResourceSlot(slotId, slotStatus.getProfiler(), taskExecutorRegistration);
    +		return null;
    +	}
     
    -			registerNewSlot(slot);
    -			LOG.info("New slot appeared, SlotID:{}, AllocationID:{}", slotId, slotStatus.getAllocationID());
    +	// ---------------------------------------------------------------------------------------------
    +	// Internal slot operations
    +	// ---------------------------------------------------------------------------------------------
     
    -			if (slotStatus.getAllocationID() != null) {
    -				// slot in use, record this in bookkeeping
    -				allocationMap.addAllocation(slotId, slotStatus.getAllocationID());
    -			} else {
    +	/**
    +	 * Registers a slot for the given task manager at the slot manager. The task manager is
    +	 * identified by the given instance id and the slot is identified by the given slot id. The
    +	 * given resource profile defines the available resources for the slot. The task manager
    +	 * connection can be used to communicate with the task manager.
    +	 *
    +	 * @param instanceId identifying the task manager on which the slot lives
    +	 * @param slotId identifying the slot on the task manager
    +	 * @param allocationId which is currently deployed in the slot
    +	 * @param resourceProfile of the slot
    +	 * @param taskManagerConnection to communicate with the remote task manager
    +	 */
    +	private void registerSlot(
    +			InstanceID instanceId,
    +			SlotID slotId,
    +			AllocationID allocationId,
    +			ResourceProfile resourceProfile,
    +			TaskExecutorConnection taskManagerConnection) {
    +		TaskManagerRegistration taskManagerRegistration = taskManagerRegistrations.get(instanceId);
    +
    +		if (null != taskManagerRegistration) {
    +			TaskManagerSlot slot = new TaskManagerSlot(
    +				slotId,
    +				resourceProfile,
    +				taskManagerConnection,
    +				allocationId);
    +
    +			slots.put(slotId, slot);
    +
    +			taskManagerRegistration.addSlot(slotId);
    +
    +			if (slot.isFree()) {
     				handleFreeSlot(slot);
     			}
    +
    +			if (slot.isAllocated()) {
    +				fulfilledSlotRequests.put(slot.getAllocationId(), slotId);
    +			}
    +		} else {
    +			LOG.debug("Trying to register slot for unknown task manager with instance id {}.", instanceId);
     		}
     	}
     
     	/**
    -	 * Callback for TaskManager failures. In case that a TaskManager fails, we have to clean up all its slots.
    +	 * Updates a slot with the given allocation id.
     	 *
    -	 * @param resourceId The ResourceID of the TaskManager
    +	 * @param slotId to update
    +	 * @param allocationId specifying the current allocation of the slot
     	 */
    -	public void notifyTaskManagerFailure(final ResourceID resourceId) {
    -		LOG.info("Resource:{} been notified failure", resourceId);
    -		taskManagers.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);
    +	private void updateSlot(SlotID slotId, AllocationID allocationId) {
    +		TaskManagerSlot slot = slots.get(slotId);
    +
    +		if (null != slot) {
    +			// we assume the given allocation id to be the ground truth (coming from the TM)
    +			slot.setAllocationId(allocationId);
    +
    +			if (null != allocationId) {
    +				if (slot.hasPendingSlotRequest()){
    +					// we have a pending slot request --> check whether we have to reject it
    +					PendingSlotRequest pendingSlotRequest = slot.getAssignedSlotRequest();
    +
    +					if (Objects.equals(pendingSlotRequest.getAllocationId(), allocationId)) {
    +						// we can cancel the slot request because it has been fulfilled
    +						cancelPendingSlotRequest(pendingSlotRequest);
    +
    +						// remove the pending slot request, since it has been completed
    +						pendingSlotRequests.remove(pendingSlotRequest.getAllocationId());
    +					} else {
    +						// this will try to find a new slot for the request
    +						rejectPendingSlotRequest(
    +							pendingSlotRequest,
    +							new Exception("Task manager reported slot " + slotId + " being already allocated."));
    +					}
    +
    +					slot.setAssignedSlotRequest(null);
    +				}
    +
    +				fulfilledSlotRequests.put(allocationId, slotId);
    +
    +				TaskManagerRegistration taskManagerRegistration = taskManagerRegistrations.get(slot.getInstanceId());
    +
    +				if (null != taskManagerRegistration) {
    +					// disable any registered time out for the task manager
    +					taskManagerRegistration.cancelTimeout();
     				}
     			}
    +		} else {
    +			LOG.debug("Trying to update unknown slot with slot id {}.", slotId);
     		}
     	}
     
    -	// ------------------------------------------------------------------------
    -	//  internal behaviors
    -	// ------------------------------------------------------------------------
    -
     	/**
    -	 * 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.
    +	 * Tries to allocate a slot for the given slot request. If there is no slot available, the
    +	 * resource manager is informed to allocate more resources and a timeout for the request is
    +	 * registered.
     	 *
    -	 * @param freeSlot The free slot
    +	 * @param pendingSlotRequest to allocate a slot for
    +	 * @throws ResourceManagerException if the resource manager cannot allocate more resource
     	 */
    -	private void handleFreeSlot(final ResourceSlot freeSlot) {
    -		SlotRequest chosenRequest = chooseRequestToFulfill(freeSlot, pendingSlotRequests);
    +	private void internalRequestSlot(PendingSlotRequest pendingSlotRequest) throws ResourceManagerException {
    +		TaskManagerSlot taskManagerSlot = findMatchingSlot(pendingSlotRequest.getResourceProfile());
     
    -		if (chosenRequest != null) {
    -			final AllocationID allocationId = chosenRequest.getAllocationId();
    -			final SlotRequest slotRequest = pendingSlotRequests.remove(allocationId);
    +		if (taskManagerSlot != null) {
    +			allocateSlot(taskManagerSlot, pendingSlotRequest);
    +		} else {
    +			final UUID timeoutIdentifier = UUID.randomUUID();
    +			final AllocationID allocationId = pendingSlotRequest.getAllocationId();
    +
    +			// register timeout for slot request
    +			ScheduledFuture<?> timeoutFuture = scheduledExecutor.schedule(new Runnable() {
    +				@Override
    +				public void run() {
    +					mainThreadExecutor.execute(new Runnable() {
    +						@Override
    +						public void run() {
    +							timeoutSlotRequest(allocationId, timeoutIdentifier);
    +						}
    +					});
    +				}
    +			}, slotRequestTimeout.toMilliseconds(), TimeUnit.MILLISECONDS);
     
    -			LOG.info("Assigning SlotID({}) to AllocationID({}), JobID:{}", freeSlot.getSlotId(),
    -				allocationId, chosenRequest.getJobId());
    -			allocationMap.addAllocation(freeSlot.getSlotId(), allocationId);
    +			pendingSlotRequest.registerTimeout(timeoutFuture, timeoutIdentifier);
     
    -			sendSlotRequest(freeSlot, slotRequest);
    -		} else {
    -			freeSlots.put(freeSlot.getSlotId(), freeSlot);
    +			resourceManagerActions.allocateResource(pendingSlotRequest.getResourceProfile());
     		}
     	}
     
    -	private void sendSlotRequest(final ResourceSlot freeSlot, final SlotRequest slotRequest) {
    +	/**
    +	 * Allocates the given slot for the given slot request. This entails sending a registration
    +	 * message to the task manager and treating failures.
    +	 *
    +	 * @param taskManagerSlot to allocate for the given slot request
    +	 * @param pendingSlotRequest to allocate the given slot for
    +	 */
    +	private void allocateSlot(TaskManagerSlot taskManagerSlot, PendingSlotRequest pendingSlotRequest) {
    +		TaskExecutorConnection taskExecutorConnection = taskManagerSlot.getTaskManagerConnection();
    +		TaskExecutorGateway gateway = taskExecutorConnection.getTaskExecutorGateway();
    +
    +		final CompletableFuture<Acknowledge> completableFuture = new FlinkCompletableFuture<>();
    +		final AllocationID allocationId = pendingSlotRequest.getAllocationId();
    +		final SlotID slotId = taskManagerSlot.getSlotId();
    +
    +		taskManagerSlot.setAssignedSlotRequest(pendingSlotRequest);
    +		pendingSlotRequest.setRequestFuture(completableFuture);
    +
    +		// RPC call to the task manager
    +		Future<Acknowledge> requestFuture = gateway.requestSlot(
    +			slotId,
    +			pendingSlotRequest.getJobId(),
    +			allocationId,
    +			pendingSlotRequest.getTargetAddress(),
    +			leaderId,
    +			taskManagerRequestTimeout);
    +
    +		requestFuture.handle(new BiFunction<Acknowledge, Throwable, Void>() {
    --- End diff --
    
    The reason why we have this indirection is because we want to be able to cancel pending slot requests (e.g. if the SlotManager is suspended) from the `SlotManager` side. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3394: [FLINK-5810] [flip6] Introduce a hardened slot man...

Posted by StephanEwen <gi...@git.apache.org>.
Github user StephanEwen commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3394#discussion_r106503546
  
    --- Diff: flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotProtocolTest.java ---
    @@ -68,115 +52,79 @@
     
     public class SlotProtocolTest extends TestLogger {
     
    -	private static TestingSerialRpcService testRpcService;
    +	private static ScheduledExecutorService scheduledExecutorService;
     
     	@BeforeClass
     	public static void beforeClass() {
    -		testRpcService = new TestingSerialRpcService();
    +		scheduledExecutorService = new ScheduledThreadPoolExecutor(1);
     	}
     
     	@AfterClass
     	public static void afterClass() {
    -		testRpcService.stopService();
    -		testRpcService = null;
    -	}
    +		scheduledExecutorService.shutdown();
    +
    +		if (!scheduledExecutorService.isTerminated()) {
    +			List<Runnable> runnables = scheduledExecutorService.shutdownNow();
     
    -	@Before
    -	public void beforeTest(){
    -		testRpcService.clearGateways();
    +			for (Runnable runnable : runnables) {
    +				runnable.run();
    --- End diff --
    
    I don't quite understand the logic in general: If there are pending tasks: `shutdown()` does not affect them, and they will be returned in `shutdownNow()`. If there are no pending tasks, it does not hurt to do `shutdownNow()` immediately as well.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3394: [FLINK-5810] [flip6] Introduce a hardened slot man...

Posted by StephanEwen <gi...@git.apache.org>.
Github user StephanEwen commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3394#discussion_r106502004
  
    --- Diff: flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotProtocolTest.java ---
    @@ -68,115 +52,79 @@
     
     public class SlotProtocolTest extends TestLogger {
     
    -	private static TestingSerialRpcService testRpcService;
    +	private static ScheduledExecutorService scheduledExecutorService;
     
     	@BeforeClass
     	public static void beforeClass() {
    -		testRpcService = new TestingSerialRpcService();
    +		scheduledExecutorService = new ScheduledThreadPoolExecutor(1);
     	}
     
     	@AfterClass
     	public static void afterClass() {
    -		testRpcService.stopService();
    -		testRpcService = null;
    -	}
    +		scheduledExecutorService.shutdown();
    +
    +		if (!scheduledExecutorService.isTerminated()) {
    +			List<Runnable> runnables = scheduledExecutorService.shutdownNow();
     
    -	@Before
    -	public void beforeTest(){
    -		testRpcService.clearGateways();
    +			for (Runnable runnable : runnables) {
    +				runnable.run();
    --- End diff --
    
    Curious: Do we really want these to run after all tests are already complete?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3394: [FLINK-5810] [flip6] Introduce a hardened slot man...

Posted by KurtYoung <gi...@git.apache.org>.
Github user KurtYoung commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3394#discussion_r114070398
  
    --- Diff: flink-runtime/src/test/resources/log4j-test.properties ---
    @@ -16,7 +16,7 @@
     # limitations under the License.
     ################################################################################
     
    -log4j.rootLogger=OFF, console
    +log4j.rootLogger=DEBUG, console
    --- End diff --
    
    Is this change intended or accidental?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3394: [FLINK-5810] [flip6] Introduce a hardened slot man...

Posted by StephanEwen <gi...@git.apache.org>.
Github user StephanEwen commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3394#discussion_r106512992
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManager.java ---
    @@ -21,519 +21,897 @@
     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.ResourceSlot;
    +import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
    +import org.apache.flink.runtime.clusterframework.types.TaskManagerSlot;
     import org.apache.flink.runtime.clusterframework.types.SlotID;
     import org.apache.flink.runtime.concurrent.BiFunction;
    +import org.apache.flink.runtime.concurrent.CompletableFuture;
     import org.apache.flink.runtime.concurrent.Future;
    -import org.apache.flink.runtime.resourcemanager.ResourceManagerServices;
    +import org.apache.flink.runtime.concurrent.ScheduledExecutor;
    +import org.apache.flink.runtime.concurrent.impl.FlinkCompletableFuture;
    +import org.apache.flink.runtime.instance.InstanceID;
    +import org.apache.flink.runtime.messages.Acknowledge;
     import org.apache.flink.runtime.resourcemanager.SlotRequest;
    -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.resourcemanager.exceptions.ResourceManagerException;
    +import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
     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.runtime.taskexecutor.exceptions.SlotAllocationException;
    +import org.apache.flink.runtime.taskexecutor.exceptions.SlotOccupiedException;
     import org.apache.flink.util.Preconditions;
     import org.slf4j.Logger;
     import org.slf4j.LoggerFactory;
     
     import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.Iterator;
     import java.util.LinkedHashMap;
     import java.util.Map;
    -
    -import static org.apache.flink.util.Preconditions.checkNotNull;
    +import java.util.Objects;
    +import java.util.UUID;
    +import java.util.concurrent.CancellationException;
    +import java.util.concurrent.Executor;
    +import java.util.concurrent.ScheduledFuture;
    +import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.TimeoutException;
     
     /**
    - * 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>.
    + * The slot manager is responsible for maintaining a view on all registered task manager slots,
    + * their allocation and all pending slot requests. Whenever a new slot is registered or and
    + * allocated slot is freed, then it tries to fulfill another pending slot request. Whenever there
    + * are not enough slots available the slot manager will notify the resource manager about it via
    + * {@link ResourceManagerActions#allocateResource(ResourceProfile)}.
    + *
    + * In order to free resources and avoid resource leaks, idling task managers (task managers whose
    + * slots are currently not used) and not fulfilled pending slot requests time out triggering their
    + * release and failure, respectively.
      */
    -public abstract class SlotManager {
    +public class SlotManager implements AutoCloseable {
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(SlotManager.class);
    +
    +	/** Scheduled executor for timeouts */
    +	private final ScheduledExecutor scheduledExecutor;
    +
    +	/** Timeout for slot requests to the task manager */
    +	private final Time taskManagerRequestTimeout;
    +
    +	/** Timeout after which an allocation is discarded */
    +	private final Time slotRequestTimeout;
    +
    +	/** Timeout after which an unused TaskManager is released */
    +	private final Time taskManagerTimeout;
    +
    +	/** Map for all registered slots */
    +	private final HashMap<SlotID, TaskManagerSlot> slots;
     
    -	protected final Logger LOG = LoggerFactory.getLogger(getClass());
    +	/** Index of all currently free slots */
    +	private final LinkedHashMap<SlotID, TaskManagerSlot> freeSlots;
     
    -	/** The Resource allocation provider */
    -	protected final ResourceManagerServices rmServices;
    +	/** All currently registered task managers */
    +	private final HashMap<InstanceID, TaskManagerRegistration> taskManagerRegistrations;
     
    -	/** All registered task managers with ResourceID and gateway. */
    -	private final Map<ResourceID, TaskExecutorRegistration> taskManagers;
    +	/** Map of fulfilled and active allocations for request deduplication purposes */
    +	private final HashMap<AllocationID, SlotID> fulfilledSlotRequests;
     
    -	/** All registered slots, including free and allocated slots */
    -	private final Map<ResourceID, Map<SlotID, ResourceSlot>> registeredSlots;
    +	/** Map of pending/unfulfilled slot allocation requests */
    +	private final HashMap<AllocationID, PendingSlotRequest> pendingSlotRequests;
     
    -	/** All pending slot requests, waiting available slots to fulfil */
    -	private final Map<AllocationID, SlotRequest> pendingSlotRequests;
    +	/** Leader id of the containing component */
    +	private UUID leaderId;
     
    -	/** All free slots that can be used to be allocated */
    -	private final Map<SlotID, ResourceSlot> freeSlots;
    +	/** Executor for future callbacks which have to be "synchronized" */
    +	private Executor mainThreadExecutor;
     
    -	/** All allocations, we can lookup allocations either by SlotID or AllocationID */
    -	private final AllocationMap allocationMap;
    +	/** Callbacks for resource (de-)allocations */
    +	private ResourceManagerActions resourceManagerActions;
     
    -	private final Time timeout;
    +	/** True iff the component has been started */
    +	private boolean started;
     
    -	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.taskManagers = new HashMap<>();
    -		this.timeout = Time.seconds(10);
    +	public SlotManager(
    +			ScheduledExecutor scheduledExecutor,
    +			Time taskManagerRequestTimeout,
    +			Time slotRequestTimeout,
    +			Time taskManagerTimeout) {
    +		this.scheduledExecutor = Preconditions.checkNotNull(scheduledExecutor);
    +		this.taskManagerRequestTimeout = Preconditions.checkNotNull(taskManagerRequestTimeout);
    +		this.slotRequestTimeout = Preconditions.checkNotNull(slotRequestTimeout);
    +		this.taskManagerTimeout = Preconditions.checkNotNull(taskManagerTimeout);
    +
    +		slots = new HashMap<>(16);
    +		freeSlots = new LinkedHashMap<>(16);
    +		taskManagerRegistrations = new HashMap<>(4);
    +		fulfilledSlotRequests = new HashMap<>(16);
    +		pendingSlotRequests = new HashMap<>(16);
    +
    +		leaderId = null;
    +		resourceManagerActions = null;
    +		started = false;
     	}
     
    -	// ------------------------------------------------------------------------
    -	//  slot managements
    -	// ------------------------------------------------------------------------
    +	// ---------------------------------------------------------------------------------------------
    +	// Component lifecycle methods
    +	// ---------------------------------------------------------------------------------------------
     
     	/**
    -	 * 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).
    +	 * Starts the slot manager with the given leader id and resource manager actions.
     	 *
    -	 * @param request The detailed request of the slot
    -	 * @return RMSlotRequestRegistered The confirmation message to be send to the caller
    +	 * @param newLeaderId to use for communication with the task managers
    +	 * @param newResourceManagerActions to use for resource (de-)allocations
    +	 */
    +	public void start(UUID newLeaderId, Executor newMainThreadExecutor, ResourceManagerActions newResourceManagerActions) {
    +		leaderId = Preconditions.checkNotNull(newLeaderId);
    +		mainThreadExecutor = Preconditions.checkNotNull(newMainThreadExecutor);
    +		resourceManagerActions = Preconditions.checkNotNull(newResourceManagerActions);
    +
    +		started = true;
    +	}
    +
    +	/**
    +	 * Suspends the component. This clears the internal state of the slot manager.
     	 */
    -	public RMSlotRequestRegistered requestSlot(final SlotRequest request) {
    -		final AllocationID allocationId = request.getAllocationId();
    -		if (isRequestDuplicated(request)) {
    -			LOG.warn("Duplicated slot request, AllocationID:{}", allocationId);
    -			return new RMSlotRequestRegistered(allocationId);
    +	public void suspend() {
    +		for (PendingSlotRequest pendingSlotRequest : pendingSlotRequests.values()) {
    +			cancelPendingSlotRequest(pendingSlotRequest);
     		}
     
    -		// 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());
    +		pendingSlotRequests.clear();
    +
    +		HashSet<InstanceID> registeredTaskManagers = new HashSet<>(taskManagerRegistrations.keySet());
    +
    +		for (InstanceID registeredTaskManager : registeredTaskManagers) {
    +			unregisterTaskManager(registeredTaskManager);
    +		}
    +
    +		leaderId = null;
    +		resourceManagerActions = null;
    +		started = false;
    +	}
    +
    +	/**
    +	 * Closes the slot manager.
    +	 *
    +	 * @throws Exception if the close operation fails
    +	 */
    +	@Override
    +	public void close() throws Exception {
    +		suspend();
    +	}
    +
    +	// ---------------------------------------------------------------------------------------------
    +	// Public API
    +	// ---------------------------------------------------------------------------------------------
    +
    +	/**
    +	 * Requests a slot with the respective resource profile.
    +	 *
    +	 * @param slotRequest specifying the requested slot specs
    +	 * @return true if the slot request was registered; false if the request is a duplicate
    +	 * @throws SlotManagerException if the slot request failed (e.g. not enough resources left)
    +	 */
    +	public boolean registerSlotRequest(SlotRequest slotRequest) throws SlotManagerException {
    +		checkInit();
     
    -			// record this allocation in bookkeeping
    -			allocationMap.addAllocation(slot.getSlotId(), allocationId);
    -			// remove selected slot from free pool
    -			freeSlots.remove(slot.getSlotId());
    +		if (checkDuplicateRequest(slotRequest.getAllocationId())) {
    +			LOG.debug("Ignoring a duplicate slot request with allocation id {}.", slotRequest.getAllocationId());
     
    -			sendSlotRequest(slot, request);
    +			return false;
     		} else {
    -			LOG.info("Cannot fulfil slot request, try to allocate a new container for it, " +
    -				"AllocationID:{}, JobID:{}", allocationId, request.getJobId());
    -			Preconditions.checkState(rmServices != null,
    -				"Attempted to allocate resources but no ResourceManagerServices set.");
    -			rmServices.allocateResource(request.getResourceProfile());
    -			pendingSlotRequests.put(allocationId, request);
    +			PendingSlotRequest pendingSlotRequest = new PendingSlotRequest(slotRequest);
    +
    +			pendingSlotRequests.put(slotRequest.getAllocationId(), pendingSlotRequest);
    +
    +			try {
    +				internalRequestSlot(pendingSlotRequest);
    +			} catch (ResourceManagerException e) {
    +				// requesting the slot failed --> remove pending slot request
    +				pendingSlotRequests.remove(slotRequest.getAllocationId());
    +
    +				throw new SlotManagerException("Could not fulfill slot request " + slotRequest.getAllocationId() + '.', e);
    +			}
    +
    +			return true;
     		}
    +	}
     
    -		return new RMSlotRequestRegistered(allocationId);
    +	/**
    +	 * Cancels and removes a pending slot request with the given allocation id. If there is no such
    +	 * pending request, then nothing is done.
    +	 *
    +	 * @param allocationId identifying the pending slot request
    +	 * @return True if a pending slot request was found; otherwise false
    +	 */
    +	public boolean unregisterSlotRequest(AllocationID allocationId) {
    +		checkInit();
    +
    +		PendingSlotRequest pendingSlotRequest = pendingSlotRequests.remove(allocationId);
    +
    +		if (null != pendingSlotRequest) {
    +			cancelPendingSlotRequest(pendingSlotRequest);
    +
    +			return true;
    +		} else {
    +			LOG.debug("No pending slot request with allocation id {} found.", allocationId);
    +
    +			return false;
    +		}
     	}
     
     	/**
    -	 * Notifies the SlotManager that a slot is available again after being allocated.
    -	 * @param slotID slot id of available slot
    +	 * Registers a new task manager at the slot manager. This will make the task managers slots
    +	 * known and, thus, available for allocation.
    +	 *
    +	 * @param taskExecutorConnection for the new task manager
    +	 * @param initialSlotReport for the new task manager
     	 */
    -	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");
    +	public void registerTaskManager(final TaskExecutorConnection taskExecutorConnection, SlotReport initialSlotReport) {
    +		checkInit();
    +
    +		// we identify task managers by their instance id
    +		if (!taskManagerRegistrations.containsKey(taskExecutorConnection.getInstanceID())) {
    +			TaskManagerRegistration taskManagerRegistration = new TaskManagerRegistration(taskExecutorConnection);
    +			taskManagerRegistrations.put(taskExecutorConnection.getInstanceID(), taskManagerRegistration);
     		}
    -		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.");
    +
    +		reportSlotStatus(taskExecutorConnection.getInstanceID(), initialSlotReport);
    +	}
    +
    +	/**
    +	 * Unregisters the task manager identified by the given instance id and its associated slots
    +	 * from the slot manager.
    +	 *
    +	 * @param instanceId identifying the task manager to unregister
    +	 * @return True if there existed a registered task manager with the given instance id
    +	 */
    +	public boolean unregisterTaskManager(InstanceID instanceId) {
    +		checkInit();
    +
    +		TaskManagerRegistration taskManagerRegistration = taskManagerRegistrations.remove(instanceId);
    +
    +		if (null != taskManagerRegistration) {
    +			removeSlots(taskManagerRegistration.getSlots());
    +
    +			taskManagerRegistration.cancelTimeout();
    +
    +			return true;
    +		} else {
    +			LOG.debug("There is no task manager registered with instance ID {}. Ignoring this message.", instanceId);
    +
    +			return false;
     		}
    -		handleFreeSlot(freeSlot);
     	}
     
     	/**
    -	 * 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.
    +	 * Reports the current slot allocations for a task manager identified by the given instance id.
     	 *
    -	 * @param originalRequest The original slot request
    -	 * @param slotId          The target SlotID
    +	 * @param instanceId identifying the task manager for which to report the slot status
    +	 * @param slotReport containing the status for all of its slots
    +	 * @return true if the slot status has been updated successfully, otherwise false
     	 */
    -	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());
    -
    -		if (allocationMap.isAllocated(slotId)) {
    -			final AllocationID expectedAllocationId = allocationMap.getAllocationID(slotId);
    -
    -			// 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);
    +	public boolean reportSlotStatus(InstanceID instanceId, SlotReport slotReport) {
    +		checkInit();
    +
    +		TaskManagerRegistration taskManagerRegistration = taskManagerRegistrations.get(instanceId);
    +
    +		if (null != taskManagerRegistration) {
    +			HashSet<SlotID> slotsToRemove = new HashSet<>(taskManagerRegistration.getSlots());
    +			boolean idle = true;
    +
    +			for (SlotStatus slotStatus : slotReport) {
    +				if (slotsToRemove.remove(slotStatus.getSlotID())) {
    +					// slot which was already registered
    +					updateSlot(slotStatus.getSlotID(), slotStatus.getAllocationID());
    +				} else {
    +					// new slot
    +					registerSlot(
    +						instanceId,
    +						slotStatus.getSlotID(),
    +						slotStatus.getAllocationID(),
    +						slotStatus.getResourceProfile(),
    +						taskManagerRegistration.getTaskManagerConnection());
    +				}
    +
    +				TaskManagerSlot slot = slots.get(slotStatus.getSlotID());
    +
    +				idle &= slot.isFree();
    +			}
    +
    +			// remove the slots for which we haven't received a slot status message
    +			removeSlots(slotsToRemove);
    +
    +			if (idle) {
    --- End diff --
    
    I assume this breaks the TaskManager timeouts. Every time a heartbeat comes (with a slot report), the TaskManager is detected to be idle, and the timeout is scheduled, overriding the previous timeout (pushing the timeout further back).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3394: [FLINK-5810] [flip6] Introduce a hardened slot man...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/flink/pull/3394


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3394: [FLINK-5810] [flip6] Introduce a hardened slot man...

Posted by StephanEwen <gi...@git.apache.org>.
Github user StephanEwen commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3394#discussion_r106515494
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManager.java ---
    @@ -21,519 +21,897 @@
     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.ResourceSlot;
    +import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
    +import org.apache.flink.runtime.clusterframework.types.TaskManagerSlot;
     import org.apache.flink.runtime.clusterframework.types.SlotID;
     import org.apache.flink.runtime.concurrent.BiFunction;
    +import org.apache.flink.runtime.concurrent.CompletableFuture;
     import org.apache.flink.runtime.concurrent.Future;
    -import org.apache.flink.runtime.resourcemanager.ResourceManagerServices;
    +import org.apache.flink.runtime.concurrent.ScheduledExecutor;
    +import org.apache.flink.runtime.concurrent.impl.FlinkCompletableFuture;
    +import org.apache.flink.runtime.instance.InstanceID;
    +import org.apache.flink.runtime.messages.Acknowledge;
     import org.apache.flink.runtime.resourcemanager.SlotRequest;
    -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.resourcemanager.exceptions.ResourceManagerException;
    +import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
     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.runtime.taskexecutor.exceptions.SlotAllocationException;
    +import org.apache.flink.runtime.taskexecutor.exceptions.SlotOccupiedException;
     import org.apache.flink.util.Preconditions;
     import org.slf4j.Logger;
     import org.slf4j.LoggerFactory;
     
     import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.Iterator;
     import java.util.LinkedHashMap;
     import java.util.Map;
    -
    -import static org.apache.flink.util.Preconditions.checkNotNull;
    +import java.util.Objects;
    +import java.util.UUID;
    +import java.util.concurrent.CancellationException;
    +import java.util.concurrent.Executor;
    +import java.util.concurrent.ScheduledFuture;
    +import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.TimeoutException;
     
     /**
    - * 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>.
    + * The slot manager is responsible for maintaining a view on all registered task manager slots,
    + * their allocation and all pending slot requests. Whenever a new slot is registered or and
    + * allocated slot is freed, then it tries to fulfill another pending slot request. Whenever there
    + * are not enough slots available the slot manager will notify the resource manager about it via
    + * {@link ResourceManagerActions#allocateResource(ResourceProfile)}.
    + *
    + * In order to free resources and avoid resource leaks, idling task managers (task managers whose
    + * slots are currently not used) and not fulfilled pending slot requests time out triggering their
    + * release and failure, respectively.
      */
    -public abstract class SlotManager {
    +public class SlotManager implements AutoCloseable {
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(SlotManager.class);
    +
    +	/** Scheduled executor for timeouts */
    +	private final ScheduledExecutor scheduledExecutor;
    +
    +	/** Timeout for slot requests to the task manager */
    +	private final Time taskManagerRequestTimeout;
    +
    +	/** Timeout after which an allocation is discarded */
    +	private final Time slotRequestTimeout;
    +
    +	/** Timeout after which an unused TaskManager is released */
    +	private final Time taskManagerTimeout;
    +
    +	/** Map for all registered slots */
    +	private final HashMap<SlotID, TaskManagerSlot> slots;
     
    -	protected final Logger LOG = LoggerFactory.getLogger(getClass());
    +	/** Index of all currently free slots */
    +	private final LinkedHashMap<SlotID, TaskManagerSlot> freeSlots;
     
    -	/** The Resource allocation provider */
    -	protected final ResourceManagerServices rmServices;
    +	/** All currently registered task managers */
    +	private final HashMap<InstanceID, TaskManagerRegistration> taskManagerRegistrations;
     
    -	/** All registered task managers with ResourceID and gateway. */
    -	private final Map<ResourceID, TaskExecutorRegistration> taskManagers;
    +	/** Map of fulfilled and active allocations for request deduplication purposes */
    +	private final HashMap<AllocationID, SlotID> fulfilledSlotRequests;
     
    -	/** All registered slots, including free and allocated slots */
    -	private final Map<ResourceID, Map<SlotID, ResourceSlot>> registeredSlots;
    +	/** Map of pending/unfulfilled slot allocation requests */
    +	private final HashMap<AllocationID, PendingSlotRequest> pendingSlotRequests;
     
    -	/** All pending slot requests, waiting available slots to fulfil */
    -	private final Map<AllocationID, SlotRequest> pendingSlotRequests;
    +	/** Leader id of the containing component */
    +	private UUID leaderId;
     
    -	/** All free slots that can be used to be allocated */
    -	private final Map<SlotID, ResourceSlot> freeSlots;
    +	/** Executor for future callbacks which have to be "synchronized" */
    +	private Executor mainThreadExecutor;
     
    -	/** All allocations, we can lookup allocations either by SlotID or AllocationID */
    -	private final AllocationMap allocationMap;
    +	/** Callbacks for resource (de-)allocations */
    +	private ResourceManagerActions resourceManagerActions;
     
    -	private final Time timeout;
    +	/** True iff the component has been started */
    +	private boolean started;
     
    -	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.taskManagers = new HashMap<>();
    -		this.timeout = Time.seconds(10);
    +	public SlotManager(
    +			ScheduledExecutor scheduledExecutor,
    +			Time taskManagerRequestTimeout,
    +			Time slotRequestTimeout,
    +			Time taskManagerTimeout) {
    +		this.scheduledExecutor = Preconditions.checkNotNull(scheduledExecutor);
    +		this.taskManagerRequestTimeout = Preconditions.checkNotNull(taskManagerRequestTimeout);
    +		this.slotRequestTimeout = Preconditions.checkNotNull(slotRequestTimeout);
    +		this.taskManagerTimeout = Preconditions.checkNotNull(taskManagerTimeout);
    +
    +		slots = new HashMap<>(16);
    +		freeSlots = new LinkedHashMap<>(16);
    +		taskManagerRegistrations = new HashMap<>(4);
    +		fulfilledSlotRequests = new HashMap<>(16);
    +		pendingSlotRequests = new HashMap<>(16);
    +
    +		leaderId = null;
    +		resourceManagerActions = null;
    +		started = false;
     	}
     
    -	// ------------------------------------------------------------------------
    -	//  slot managements
    -	// ------------------------------------------------------------------------
    +	// ---------------------------------------------------------------------------------------------
    +	// Component lifecycle methods
    +	// ---------------------------------------------------------------------------------------------
     
     	/**
    -	 * 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).
    +	 * Starts the slot manager with the given leader id and resource manager actions.
     	 *
    -	 * @param request The detailed request of the slot
    -	 * @return RMSlotRequestRegistered The confirmation message to be send to the caller
    +	 * @param newLeaderId to use for communication with the task managers
    +	 * @param newResourceManagerActions to use for resource (de-)allocations
    +	 */
    +	public void start(UUID newLeaderId, Executor newMainThreadExecutor, ResourceManagerActions newResourceManagerActions) {
    +		leaderId = Preconditions.checkNotNull(newLeaderId);
    +		mainThreadExecutor = Preconditions.checkNotNull(newMainThreadExecutor);
    +		resourceManagerActions = Preconditions.checkNotNull(newResourceManagerActions);
    +
    +		started = true;
    +	}
    +
    +	/**
    +	 * Suspends the component. This clears the internal state of the slot manager.
     	 */
    -	public RMSlotRequestRegistered requestSlot(final SlotRequest request) {
    -		final AllocationID allocationId = request.getAllocationId();
    -		if (isRequestDuplicated(request)) {
    -			LOG.warn("Duplicated slot request, AllocationID:{}", allocationId);
    -			return new RMSlotRequestRegistered(allocationId);
    +	public void suspend() {
    +		for (PendingSlotRequest pendingSlotRequest : pendingSlotRequests.values()) {
    +			cancelPendingSlotRequest(pendingSlotRequest);
     		}
     
    -		// 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());
    +		pendingSlotRequests.clear();
    +
    +		HashSet<InstanceID> registeredTaskManagers = new HashSet<>(taskManagerRegistrations.keySet());
    +
    +		for (InstanceID registeredTaskManager : registeredTaskManagers) {
    +			unregisterTaskManager(registeredTaskManager);
    +		}
    +
    +		leaderId = null;
    +		resourceManagerActions = null;
    +		started = false;
    +	}
    +
    +	/**
    +	 * Closes the slot manager.
    +	 *
    +	 * @throws Exception if the close operation fails
    +	 */
    +	@Override
    +	public void close() throws Exception {
    +		suspend();
    +	}
    +
    +	// ---------------------------------------------------------------------------------------------
    +	// Public API
    +	// ---------------------------------------------------------------------------------------------
    +
    +	/**
    +	 * Requests a slot with the respective resource profile.
    +	 *
    +	 * @param slotRequest specifying the requested slot specs
    +	 * @return true if the slot request was registered; false if the request is a duplicate
    +	 * @throws SlotManagerException if the slot request failed (e.g. not enough resources left)
    +	 */
    +	public boolean registerSlotRequest(SlotRequest slotRequest) throws SlotManagerException {
    +		checkInit();
     
    -			// record this allocation in bookkeeping
    -			allocationMap.addAllocation(slot.getSlotId(), allocationId);
    -			// remove selected slot from free pool
    -			freeSlots.remove(slot.getSlotId());
    +		if (checkDuplicateRequest(slotRequest.getAllocationId())) {
    +			LOG.debug("Ignoring a duplicate slot request with allocation id {}.", slotRequest.getAllocationId());
     
    -			sendSlotRequest(slot, request);
    +			return false;
     		} else {
    -			LOG.info("Cannot fulfil slot request, try to allocate a new container for it, " +
    -				"AllocationID:{}, JobID:{}", allocationId, request.getJobId());
    -			Preconditions.checkState(rmServices != null,
    -				"Attempted to allocate resources but no ResourceManagerServices set.");
    -			rmServices.allocateResource(request.getResourceProfile());
    -			pendingSlotRequests.put(allocationId, request);
    +			PendingSlotRequest pendingSlotRequest = new PendingSlotRequest(slotRequest);
    +
    +			pendingSlotRequests.put(slotRequest.getAllocationId(), pendingSlotRequest);
    +
    +			try {
    +				internalRequestSlot(pendingSlotRequest);
    +			} catch (ResourceManagerException e) {
    +				// requesting the slot failed --> remove pending slot request
    +				pendingSlotRequests.remove(slotRequest.getAllocationId());
    +
    +				throw new SlotManagerException("Could not fulfill slot request " + slotRequest.getAllocationId() + '.', e);
    +			}
    +
    +			return true;
     		}
    +	}
     
    -		return new RMSlotRequestRegistered(allocationId);
    +	/**
    +	 * Cancels and removes a pending slot request with the given allocation id. If there is no such
    +	 * pending request, then nothing is done.
    +	 *
    +	 * @param allocationId identifying the pending slot request
    +	 * @return True if a pending slot request was found; otherwise false
    +	 */
    +	public boolean unregisterSlotRequest(AllocationID allocationId) {
    +		checkInit();
    +
    +		PendingSlotRequest pendingSlotRequest = pendingSlotRequests.remove(allocationId);
    +
    +		if (null != pendingSlotRequest) {
    +			cancelPendingSlotRequest(pendingSlotRequest);
    +
    +			return true;
    +		} else {
    +			LOG.debug("No pending slot request with allocation id {} found.", allocationId);
    +
    +			return false;
    +		}
     	}
     
     	/**
    -	 * Notifies the SlotManager that a slot is available again after being allocated.
    -	 * @param slotID slot id of available slot
    +	 * Registers a new task manager at the slot manager. This will make the task managers slots
    +	 * known and, thus, available for allocation.
    +	 *
    +	 * @param taskExecutorConnection for the new task manager
    +	 * @param initialSlotReport for the new task manager
     	 */
    -	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");
    +	public void registerTaskManager(final TaskExecutorConnection taskExecutorConnection, SlotReport initialSlotReport) {
    +		checkInit();
    +
    +		// we identify task managers by their instance id
    +		if (!taskManagerRegistrations.containsKey(taskExecutorConnection.getInstanceID())) {
    +			TaskManagerRegistration taskManagerRegistration = new TaskManagerRegistration(taskExecutorConnection);
    +			taskManagerRegistrations.put(taskExecutorConnection.getInstanceID(), taskManagerRegistration);
     		}
    -		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.");
    +
    +		reportSlotStatus(taskExecutorConnection.getInstanceID(), initialSlotReport);
    +	}
    +
    +	/**
    +	 * Unregisters the task manager identified by the given instance id and its associated slots
    +	 * from the slot manager.
    +	 *
    +	 * @param instanceId identifying the task manager to unregister
    +	 * @return True if there existed a registered task manager with the given instance id
    +	 */
    +	public boolean unregisterTaskManager(InstanceID instanceId) {
    +		checkInit();
    +
    +		TaskManagerRegistration taskManagerRegistration = taskManagerRegistrations.remove(instanceId);
    +
    +		if (null != taskManagerRegistration) {
    +			removeSlots(taskManagerRegistration.getSlots());
    +
    +			taskManagerRegistration.cancelTimeout();
    +
    +			return true;
    +		} else {
    +			LOG.debug("There is no task manager registered with instance ID {}. Ignoring this message.", instanceId);
    +
    +			return false;
     		}
    -		handleFreeSlot(freeSlot);
     	}
     
     	/**
    -	 * 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.
    +	 * Reports the current slot allocations for a task manager identified by the given instance id.
     	 *
    -	 * @param originalRequest The original slot request
    -	 * @param slotId          The target SlotID
    +	 * @param instanceId identifying the task manager for which to report the slot status
    +	 * @param slotReport containing the status for all of its slots
    +	 * @return true if the slot status has been updated successfully, otherwise false
     	 */
    -	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());
    -
    -		if (allocationMap.isAllocated(slotId)) {
    -			final AllocationID expectedAllocationId = allocationMap.getAllocationID(slotId);
    -
    -			// 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);
    +	public boolean reportSlotStatus(InstanceID instanceId, SlotReport slotReport) {
    +		checkInit();
    +
    +		TaskManagerRegistration taskManagerRegistration = taskManagerRegistrations.get(instanceId);
    +
    +		if (null != taskManagerRegistration) {
    +			HashSet<SlotID> slotsToRemove = new HashSet<>(taskManagerRegistration.getSlots());
    +			boolean idle = true;
    +
    +			for (SlotStatus slotStatus : slotReport) {
    +				if (slotsToRemove.remove(slotStatus.getSlotID())) {
    +					// slot which was already registered
    +					updateSlot(slotStatus.getSlotID(), slotStatus.getAllocationID());
    +				} else {
    +					// new slot
    +					registerSlot(
    +						instanceId,
    +						slotStatus.getSlotID(),
    +						slotStatus.getAllocationID(),
    +						slotStatus.getResourceProfile(),
    +						taskManagerRegistration.getTaskManagerConnection());
    +				}
    +
    +				TaskManagerSlot slot = slots.get(slotStatus.getSlotID());
    +
    +				idle &= slot.isFree();
    +			}
    +
    +			// remove the slots for which we haven't received a slot status message
    +			removeSlots(slotsToRemove);
    +
    +			if (idle) {
    +				// no slot of this task manager is being used --> register timer to free this resource
    +				registerTaskManagerTimeout(taskManagerRegistration);
    +			}
    +
    +			return true;
    +		} else {
    +			LOG.debug("Received slot report for unknown task manager with instance id {}. Ignoring this report.", instanceId);
    +
    +			return false;
    +		}
    +	}
    +
    +	/**
    +	 * Free the given slot from the given allocation. If the slot is still allocated by the given
    +	 * allocation id, then the slot will be marked as free and will be subject to new slot requests.
    +	 *
    +	 * @param slotId identifying the slot to free
    +	 * @param allocationId with which the slot is presumably allocated
    +	 */
    +	public void freeSlot(SlotID slotId, AllocationID allocationId) {
    +		checkInit();
    +
    +		TaskManagerSlot slot = slots.get(slotId);
    +
    +		if (null != slot) {
    +			if (slot.isAllocated()) {
    +				if (Objects.equals(allocationId, slot.getAllocationId())) {
    +					// free the slot
    +					slot.setAllocationId(null);
    +					fulfilledSlotRequests.remove(allocationId);
    +
    +					if (slot.isFree()) {
    +						handleFreeSlot(slot);
    +					}
    +
    +					TaskManagerRegistration taskManagerRegistration = taskManagerRegistrations.get(slot.getInstanceId());
    +
    +					if (null != taskManagerRegistration && !anySlotUsed(taskManagerRegistration.getSlots())) {
    +						registerTaskManagerTimeout(taskManagerRegistration);
    +					}
    +				} else {
    +					LOG.debug("Received request to free slot {} with expected allocation id {}, " +
    +						"but actual allocation id {} differs. Ignoring the request.", slotId, allocationId, slot.getAllocationId());
    +				}
     			} else {
    -				LOG.error("Slot request failed for slot {} with allocation id {}:" +
    -						" Allocation id did not match the expected allocation id {}.",
    -					slotId, originalAllocationId, expectedAllocationId);
    +				LOG.debug("Slot {} has not been allocated.", allocationId);
     			}
     		} else {
    -			LOG.error("Slot request failed for slot {} with allocation id {}: " +
    -					"Slot was not previously registered.",
    -				slotId, originalAllocationId);
    +			LOG.debug("Trying to free a slot {} which has not been registered. Ignoring this message.", slotId);
     		}
     	}
     
    +	// ---------------------------------------------------------------------------------------------
    +	// Behaviour methods
    +	// ---------------------------------------------------------------------------------------------
    +
     	/**
    -	 * Registers a TaskExecutor
    -	 * @param resourceID TaskExecutor's ResourceID
    -	 * @param registration TaskExecutor's registration
    -	 * @param slotReport TaskExecutor's free and allocated slots
    +	 * Finds a matching slot request for a given resource profile. If there is no such request,
    +	 * the method returns null.
    +	 *
    +	 * Note: If you want to change the behaviour of the slot manager wrt slot allocation and
    +	 * request fulfillment, then you should override this method.
    +	 *
    +	 * @param slotResourceProfile defining the resources of an available slot
    +	 * @return A matching slot request which can be deployed in a slot with the given resource
    +	 * profile. Null if there is no such slot request pending.
     	 */
    -	public void registerTaskExecutor(
    -			ResourceID resourceID,
    -			TaskExecutorRegistration registration,
    -			SlotReport slotReport) {
    +	protected PendingSlotRequest findMatchingRequest(ResourceProfile slotResourceProfile) {
     
    -		if (taskManagers.get(resourceID) != null) {
    -			notifyTaskManagerFailure(resourceID);
    +		for (PendingSlotRequest pendingSlotRequest : pendingSlotRequests.values()) {
    +			if (!pendingSlotRequest.isAssigned() && slotResourceProfile.isMatching(pendingSlotRequest.getResourceProfile())) {
    +				return pendingSlotRequest;
    +			}
     		}
     
    -		this.taskManagers.put(resourceID, registration);
    +		return null;
    +	}
    +
    +	/**
    +	 * Finds a matching slot for a given resource profile. A matching slot has at least as many
    +	 * resources available as the given resource profile. If there is no such slot available, then
    +	 * the method returns null.
    +	 *
    +	 * Note: If you want to change the behaviour of the slot manager wrt slot allocation and
    +	 * request fulfillment, then you should override this method.
    +	 *
    +	 * @param requestResourceProfile specifying the resource requirements for the a slot request
    +	 * @return A matching slot which fulfills the given resource profile. Null if there is no such
    +	 * slot available.
    +	 */
    +	protected TaskManagerSlot findMatchingSlot(ResourceProfile requestResourceProfile) {
    +		Iterator<Map.Entry<SlotID, TaskManagerSlot>> iterator = freeSlots.entrySet().iterator();
    +
    +		while (iterator.hasNext()) {
    +			TaskManagerSlot taskManagerSlot = iterator.next().getValue();
     
    -		for (SlotStatus slotStatus : slotReport.getSlotsStatus()) {
    -			final SlotID slotId = slotStatus.getSlotID();
    +			// sanity check
    +			Preconditions.checkState(taskManagerSlot.isFree());
     
    -			final TaskExecutorRegistration taskExecutorRegistration = taskManagers.get(slotId.getResourceID());
    -			if (taskExecutorRegistration == null) {
    -				LOG.info("Received SlotStatus but ResourceID {} is unknown to the SlotManager",
    -					slotId.getResourceID());
    -				return;
    +			if (taskManagerSlot.getResourceProfile().isMatching(requestResourceProfile)) {
    +				iterator.remove();
    +				return taskManagerSlot;
     			}
    +		}
     
    -			final ResourceSlot slot = new ResourceSlot(slotId, slotStatus.getProfiler(), taskExecutorRegistration);
    +		return null;
    +	}
     
    -			registerNewSlot(slot);
    -			LOG.info("New slot appeared, SlotID:{}, AllocationID:{}", slotId, slotStatus.getAllocationID());
    +	// ---------------------------------------------------------------------------------------------
    +	// Internal slot operations
    +	// ---------------------------------------------------------------------------------------------
     
    -			if (slotStatus.getAllocationID() != null) {
    -				// slot in use, record this in bookkeeping
    -				allocationMap.addAllocation(slotId, slotStatus.getAllocationID());
    -			} else {
    +	/**
    +	 * Registers a slot for the given task manager at the slot manager. The task manager is
    +	 * identified by the given instance id and the slot is identified by the given slot id. The
    +	 * given resource profile defines the available resources for the slot. The task manager
    +	 * connection can be used to communicate with the task manager.
    +	 *
    +	 * @param instanceId identifying the task manager on which the slot lives
    +	 * @param slotId identifying the slot on the task manager
    +	 * @param allocationId which is currently deployed in the slot
    +	 * @param resourceProfile of the slot
    +	 * @param taskManagerConnection to communicate with the remote task manager
    +	 */
    +	private void registerSlot(
    +			InstanceID instanceId,
    +			SlotID slotId,
    +			AllocationID allocationId,
    +			ResourceProfile resourceProfile,
    +			TaskExecutorConnection taskManagerConnection) {
    +		TaskManagerRegistration taskManagerRegistration = taskManagerRegistrations.get(instanceId);
    +
    +		if (null != taskManagerRegistration) {
    +			TaskManagerSlot slot = new TaskManagerSlot(
    +				slotId,
    +				resourceProfile,
    +				taskManagerConnection,
    +				allocationId);
    +
    +			slots.put(slotId, slot);
    +
    +			taskManagerRegistration.addSlot(slotId);
    +
    +			if (slot.isFree()) {
     				handleFreeSlot(slot);
     			}
    +
    +			if (slot.isAllocated()) {
    +				fulfilledSlotRequests.put(slot.getAllocationId(), slotId);
    +			}
    +		} else {
    +			LOG.debug("Trying to register slot for unknown task manager with instance id {}.", instanceId);
     		}
     	}
     
     	/**
    -	 * Callback for TaskManager failures. In case that a TaskManager fails, we have to clean up all its slots.
    +	 * Updates a slot with the given allocation id.
     	 *
    -	 * @param resourceId The ResourceID of the TaskManager
    +	 * @param slotId to update
    +	 * @param allocationId specifying the current allocation of the slot
     	 */
    -	public void notifyTaskManagerFailure(final ResourceID resourceId) {
    -		LOG.info("Resource:{} been notified failure", resourceId);
    -		taskManagers.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);
    +	private void updateSlot(SlotID slotId, AllocationID allocationId) {
    +		TaskManagerSlot slot = slots.get(slotId);
    +
    +		if (null != slot) {
    +			// we assume the given allocation id to be the ground truth (coming from the TM)
    +			slot.setAllocationId(allocationId);
    +
    +			if (null != allocationId) {
    +				if (slot.hasPendingSlotRequest()){
    +					// we have a pending slot request --> check whether we have to reject it
    +					PendingSlotRequest pendingSlotRequest = slot.getAssignedSlotRequest();
    +
    +					if (Objects.equals(pendingSlotRequest.getAllocationId(), allocationId)) {
    +						// we can cancel the slot request because it has been fulfilled
    +						cancelPendingSlotRequest(pendingSlotRequest);
    +
    +						// remove the pending slot request, since it has been completed
    +						pendingSlotRequests.remove(pendingSlotRequest.getAllocationId());
    +					} else {
    +						// this will try to find a new slot for the request
    +						rejectPendingSlotRequest(
    +							pendingSlotRequest,
    +							new Exception("Task manager reported slot " + slotId + " being already allocated."));
    +					}
    +
    +					slot.setAssignedSlotRequest(null);
    +				}
    +
    +				fulfilledSlotRequests.put(allocationId, slotId);
    +
    +				TaskManagerRegistration taskManagerRegistration = taskManagerRegistrations.get(slot.getInstanceId());
    +
    +				if (null != taskManagerRegistration) {
    +					// disable any registered time out for the task manager
    +					taskManagerRegistration.cancelTimeout();
     				}
     			}
    +		} else {
    +			LOG.debug("Trying to update unknown slot with slot id {}.", slotId);
     		}
     	}
     
    -	// ------------------------------------------------------------------------
    -	//  internal behaviors
    -	// ------------------------------------------------------------------------
    -
     	/**
    -	 * 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.
    +	 * Tries to allocate a slot for the given slot request. If there is no slot available, the
    +	 * resource manager is informed to allocate more resources and a timeout for the request is
    +	 * registered.
     	 *
    -	 * @param freeSlot The free slot
    +	 * @param pendingSlotRequest to allocate a slot for
    +	 * @throws ResourceManagerException if the resource manager cannot allocate more resource
     	 */
    -	private void handleFreeSlot(final ResourceSlot freeSlot) {
    -		SlotRequest chosenRequest = chooseRequestToFulfill(freeSlot, pendingSlotRequests);
    +	private void internalRequestSlot(PendingSlotRequest pendingSlotRequest) throws ResourceManagerException {
    +		TaskManagerSlot taskManagerSlot = findMatchingSlot(pendingSlotRequest.getResourceProfile());
     
    -		if (chosenRequest != null) {
    -			final AllocationID allocationId = chosenRequest.getAllocationId();
    -			final SlotRequest slotRequest = pendingSlotRequests.remove(allocationId);
    +		if (taskManagerSlot != null) {
    +			allocateSlot(taskManagerSlot, pendingSlotRequest);
    +		} else {
    +			final UUID timeoutIdentifier = UUID.randomUUID();
    +			final AllocationID allocationId = pendingSlotRequest.getAllocationId();
    +
    +			// register timeout for slot request
    +			ScheduledFuture<?> timeoutFuture = scheduledExecutor.schedule(new Runnable() {
    +				@Override
    +				public void run() {
    +					mainThreadExecutor.execute(new Runnable() {
    +						@Override
    +						public void run() {
    +							timeoutSlotRequest(allocationId, timeoutIdentifier);
    +						}
    +					});
    +				}
    +			}, slotRequestTimeout.toMilliseconds(), TimeUnit.MILLISECONDS);
     
    -			LOG.info("Assigning SlotID({}) to AllocationID({}), JobID:{}", freeSlot.getSlotId(),
    -				allocationId, chosenRequest.getJobId());
    -			allocationMap.addAllocation(freeSlot.getSlotId(), allocationId);
    +			pendingSlotRequest.registerTimeout(timeoutFuture, timeoutIdentifier);
     
    -			sendSlotRequest(freeSlot, slotRequest);
    -		} else {
    -			freeSlots.put(freeSlot.getSlotId(), freeSlot);
    +			resourceManagerActions.allocateResource(pendingSlotRequest.getResourceProfile());
     		}
     	}
     
    -	private void sendSlotRequest(final ResourceSlot freeSlot, final SlotRequest slotRequest) {
    +	/**
    +	 * Allocates the given slot for the given slot request. This entails sending a registration
    +	 * message to the task manager and treating failures.
    +	 *
    +	 * @param taskManagerSlot to allocate for the given slot request
    +	 * @param pendingSlotRequest to allocate the given slot for
    +	 */
    +	private void allocateSlot(TaskManagerSlot taskManagerSlot, PendingSlotRequest pendingSlotRequest) {
    +		TaskExecutorConnection taskExecutorConnection = taskManagerSlot.getTaskManagerConnection();
    +		TaskExecutorGateway gateway = taskExecutorConnection.getTaskExecutorGateway();
    +
    +		final CompletableFuture<Acknowledge> completableFuture = new FlinkCompletableFuture<>();
    +		final AllocationID allocationId = pendingSlotRequest.getAllocationId();
    +		final SlotID slotId = taskManagerSlot.getSlotId();
    +
    +		taskManagerSlot.setAssignedSlotRequest(pendingSlotRequest);
    +		pendingSlotRequest.setRequestFuture(completableFuture);
    +
    +		// RPC call to the task manager
    +		Future<Acknowledge> requestFuture = gateway.requestSlot(
    +			slotId,
    +			pendingSlotRequest.getJobId(),
    +			allocationId,
    +			pendingSlotRequest.getTargetAddress(),
    +			leaderId,
    +			taskManagerRequestTimeout);
    +
    +		requestFuture.handle(new BiFunction<Acknowledge, Throwable, Void>() {
    --- End diff --
    
    This `BiFunction` seems to be an *identity* function. Can we just skip the `completableFuture` and define the proper handler directly on the `requestFuture`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3394: [FLINK-5810] [flip6] Introduce a hardened slot man...

Posted by StephanEwen <gi...@git.apache.org>.
Github user StephanEwen commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3394#discussion_r114284390
  
    --- Diff: flink-runtime/src/test/resources/log4j-test.properties ---
    @@ -16,7 +16,7 @@
     # limitations under the License.
     ################################################################################
     
    -log4j.rootLogger=OFF, console
    +log4j.rootLogger=DEBUG, console
    --- End diff --
    
    That must have been accidental and should be reverted...


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3394: [FLINK-5810] [flip6] Introduce a hardened slot man...

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3394#discussion_r113482845
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManager.java ---
    @@ -21,519 +21,897 @@
     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.ResourceSlot;
    +import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
    +import org.apache.flink.runtime.clusterframework.types.TaskManagerSlot;
     import org.apache.flink.runtime.clusterframework.types.SlotID;
     import org.apache.flink.runtime.concurrent.BiFunction;
    +import org.apache.flink.runtime.concurrent.CompletableFuture;
     import org.apache.flink.runtime.concurrent.Future;
    -import org.apache.flink.runtime.resourcemanager.ResourceManagerServices;
    +import org.apache.flink.runtime.concurrent.ScheduledExecutor;
    +import org.apache.flink.runtime.concurrent.impl.FlinkCompletableFuture;
    +import org.apache.flink.runtime.instance.InstanceID;
    +import org.apache.flink.runtime.messages.Acknowledge;
     import org.apache.flink.runtime.resourcemanager.SlotRequest;
    -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.resourcemanager.exceptions.ResourceManagerException;
    +import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
     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.runtime.taskexecutor.exceptions.SlotAllocationException;
    +import org.apache.flink.runtime.taskexecutor.exceptions.SlotOccupiedException;
     import org.apache.flink.util.Preconditions;
     import org.slf4j.Logger;
     import org.slf4j.LoggerFactory;
     
     import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.Iterator;
     import java.util.LinkedHashMap;
     import java.util.Map;
    -
    -import static org.apache.flink.util.Preconditions.checkNotNull;
    +import java.util.Objects;
    +import java.util.UUID;
    +import java.util.concurrent.CancellationException;
    +import java.util.concurrent.Executor;
    +import java.util.concurrent.ScheduledFuture;
    +import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.TimeoutException;
     
     /**
    - * 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>.
    + * The slot manager is responsible for maintaining a view on all registered task manager slots,
    + * their allocation and all pending slot requests. Whenever a new slot is registered or and
    + * allocated slot is freed, then it tries to fulfill another pending slot request. Whenever there
    + * are not enough slots available the slot manager will notify the resource manager about it via
    + * {@link ResourceManagerActions#allocateResource(ResourceProfile)}.
    + *
    + * In order to free resources and avoid resource leaks, idling task managers (task managers whose
    + * slots are currently not used) and not fulfilled pending slot requests time out triggering their
    + * release and failure, respectively.
      */
    -public abstract class SlotManager {
    +public class SlotManager implements AutoCloseable {
    --- End diff --
    
    Some test cases do. Admittedly, this is not a strong argument, but on the other hand, it does not really hurt either.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3394: [FLINK-5810] [flip6] Introduce a hardened slot man...

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3394#discussion_r113482181
  
    --- Diff: flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotProtocolTest.java ---
    @@ -68,115 +52,79 @@
     
     public class SlotProtocolTest extends TestLogger {
     
    -	private static TestingSerialRpcService testRpcService;
    +	private static ScheduledExecutorService scheduledExecutorService;
     
     	@BeforeClass
     	public static void beforeClass() {
    -		testRpcService = new TestingSerialRpcService();
    +		scheduledExecutorService = new ScheduledThreadPoolExecutor(1);
     	}
     
     	@AfterClass
     	public static void afterClass() {
    -		testRpcService.stopService();
    -		testRpcService = null;
    -	}
    +		scheduledExecutorService.shutdown();
    +
    +		if (!scheduledExecutorService.isTerminated()) {
    +			List<Runnable> runnables = scheduledExecutorService.shutdownNow();
     
    -	@Before
    -	public void beforeTest(){
    -		testRpcService.clearGateways();
    +			for (Runnable runnable : runnables) {
    +				runnable.run();
    --- End diff --
    
    You're right. That way it does not make much sense. I think I wanted to call `scheduledExecutorService.awaitTermination` in the if condition in order to wait for the scheduled tasks to be executed.
    
    Will replace it with `Executors.gracefulShutdown`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3394: [FLINK-5810] [flip6] Introduce a hardened slot man...

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3394#discussion_r113484139
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManager.java ---
    @@ -21,519 +21,897 @@
     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.ResourceSlot;
    +import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
    +import org.apache.flink.runtime.clusterframework.types.TaskManagerSlot;
     import org.apache.flink.runtime.clusterframework.types.SlotID;
     import org.apache.flink.runtime.concurrent.BiFunction;
    +import org.apache.flink.runtime.concurrent.CompletableFuture;
     import org.apache.flink.runtime.concurrent.Future;
    -import org.apache.flink.runtime.resourcemanager.ResourceManagerServices;
    +import org.apache.flink.runtime.concurrent.ScheduledExecutor;
    +import org.apache.flink.runtime.concurrent.impl.FlinkCompletableFuture;
    +import org.apache.flink.runtime.instance.InstanceID;
    +import org.apache.flink.runtime.messages.Acknowledge;
     import org.apache.flink.runtime.resourcemanager.SlotRequest;
    -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.resourcemanager.exceptions.ResourceManagerException;
    +import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
     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.runtime.taskexecutor.exceptions.SlotAllocationException;
    +import org.apache.flink.runtime.taskexecutor.exceptions.SlotOccupiedException;
     import org.apache.flink.util.Preconditions;
     import org.slf4j.Logger;
     import org.slf4j.LoggerFactory;
     
     import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.Iterator;
     import java.util.LinkedHashMap;
     import java.util.Map;
    -
    -import static org.apache.flink.util.Preconditions.checkNotNull;
    +import java.util.Objects;
    +import java.util.UUID;
    +import java.util.concurrent.CancellationException;
    +import java.util.concurrent.Executor;
    +import java.util.concurrent.ScheduledFuture;
    +import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.TimeoutException;
     
     /**
    - * 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>.
    + * The slot manager is responsible for maintaining a view on all registered task manager slots,
    + * their allocation and all pending slot requests. Whenever a new slot is registered or and
    + * allocated slot is freed, then it tries to fulfill another pending slot request. Whenever there
    + * are not enough slots available the slot manager will notify the resource manager about it via
    + * {@link ResourceManagerActions#allocateResource(ResourceProfile)}.
    + *
    + * In order to free resources and avoid resource leaks, idling task managers (task managers whose
    + * slots are currently not used) and not fulfilled pending slot requests time out triggering their
    + * release and failure, respectively.
      */
    -public abstract class SlotManager {
    +public class SlotManager implements AutoCloseable {
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(SlotManager.class);
    +
    +	/** Scheduled executor for timeouts */
    +	private final ScheduledExecutor scheduledExecutor;
    +
    +	/** Timeout for slot requests to the task manager */
    +	private final Time taskManagerRequestTimeout;
    +
    +	/** Timeout after which an allocation is discarded */
    +	private final Time slotRequestTimeout;
    +
    +	/** Timeout after which an unused TaskManager is released */
    +	private final Time taskManagerTimeout;
    +
    +	/** Map for all registered slots */
    +	private final HashMap<SlotID, TaskManagerSlot> slots;
     
    -	protected final Logger LOG = LoggerFactory.getLogger(getClass());
    +	/** Index of all currently free slots */
    +	private final LinkedHashMap<SlotID, TaskManagerSlot> freeSlots;
     
    -	/** The Resource allocation provider */
    -	protected final ResourceManagerServices rmServices;
    +	/** All currently registered task managers */
    +	private final HashMap<InstanceID, TaskManagerRegistration> taskManagerRegistrations;
     
    -	/** All registered task managers with ResourceID and gateway. */
    -	private final Map<ResourceID, TaskExecutorRegistration> taskManagers;
    +	/** Map of fulfilled and active allocations for request deduplication purposes */
    +	private final HashMap<AllocationID, SlotID> fulfilledSlotRequests;
     
    -	/** All registered slots, including free and allocated slots */
    -	private final Map<ResourceID, Map<SlotID, ResourceSlot>> registeredSlots;
    +	/** Map of pending/unfulfilled slot allocation requests */
    +	private final HashMap<AllocationID, PendingSlotRequest> pendingSlotRequests;
     
    -	/** All pending slot requests, waiting available slots to fulfil */
    -	private final Map<AllocationID, SlotRequest> pendingSlotRequests;
    +	/** Leader id of the containing component */
    +	private UUID leaderId;
     
    -	/** All free slots that can be used to be allocated */
    -	private final Map<SlotID, ResourceSlot> freeSlots;
    +	/** Executor for future callbacks which have to be "synchronized" */
    +	private Executor mainThreadExecutor;
     
    -	/** All allocations, we can lookup allocations either by SlotID or AllocationID */
    -	private final AllocationMap allocationMap;
    +	/** Callbacks for resource (de-)allocations */
    +	private ResourceManagerActions resourceManagerActions;
     
    -	private final Time timeout;
    +	/** True iff the component has been started */
    +	private boolean started;
     
    -	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.taskManagers = new HashMap<>();
    -		this.timeout = Time.seconds(10);
    +	public SlotManager(
    +			ScheduledExecutor scheduledExecutor,
    +			Time taskManagerRequestTimeout,
    +			Time slotRequestTimeout,
    +			Time taskManagerTimeout) {
    +		this.scheduledExecutor = Preconditions.checkNotNull(scheduledExecutor);
    +		this.taskManagerRequestTimeout = Preconditions.checkNotNull(taskManagerRequestTimeout);
    +		this.slotRequestTimeout = Preconditions.checkNotNull(slotRequestTimeout);
    +		this.taskManagerTimeout = Preconditions.checkNotNull(taskManagerTimeout);
    +
    +		slots = new HashMap<>(16);
    +		freeSlots = new LinkedHashMap<>(16);
    +		taskManagerRegistrations = new HashMap<>(4);
    +		fulfilledSlotRequests = new HashMap<>(16);
    +		pendingSlotRequests = new HashMap<>(16);
    +
    +		leaderId = null;
    +		resourceManagerActions = null;
    +		started = false;
     	}
     
    -	// ------------------------------------------------------------------------
    -	//  slot managements
    -	// ------------------------------------------------------------------------
    +	// ---------------------------------------------------------------------------------------------
    +	// Component lifecycle methods
    +	// ---------------------------------------------------------------------------------------------
     
     	/**
    -	 * 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).
    +	 * Starts the slot manager with the given leader id and resource manager actions.
     	 *
    -	 * @param request The detailed request of the slot
    -	 * @return RMSlotRequestRegistered The confirmation message to be send to the caller
    +	 * @param newLeaderId to use for communication with the task managers
    +	 * @param newResourceManagerActions to use for resource (de-)allocations
    +	 */
    +	public void start(UUID newLeaderId, Executor newMainThreadExecutor, ResourceManagerActions newResourceManagerActions) {
    +		leaderId = Preconditions.checkNotNull(newLeaderId);
    +		mainThreadExecutor = Preconditions.checkNotNull(newMainThreadExecutor);
    +		resourceManagerActions = Preconditions.checkNotNull(newResourceManagerActions);
    +
    +		started = true;
    +	}
    +
    +	/**
    +	 * Suspends the component. This clears the internal state of the slot manager.
     	 */
    -	public RMSlotRequestRegistered requestSlot(final SlotRequest request) {
    -		final AllocationID allocationId = request.getAllocationId();
    -		if (isRequestDuplicated(request)) {
    -			LOG.warn("Duplicated slot request, AllocationID:{}", allocationId);
    -			return new RMSlotRequestRegistered(allocationId);
    +	public void suspend() {
    +		for (PendingSlotRequest pendingSlotRequest : pendingSlotRequests.values()) {
    +			cancelPendingSlotRequest(pendingSlotRequest);
     		}
     
    -		// 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());
    +		pendingSlotRequests.clear();
    +
    +		HashSet<InstanceID> registeredTaskManagers = new HashSet<>(taskManagerRegistrations.keySet());
    +
    +		for (InstanceID registeredTaskManager : registeredTaskManagers) {
    +			unregisterTaskManager(registeredTaskManager);
    +		}
    +
    +		leaderId = null;
    +		resourceManagerActions = null;
    +		started = false;
    +	}
    +
    +	/**
    +	 * Closes the slot manager.
    +	 *
    +	 * @throws Exception if the close operation fails
    +	 */
    +	@Override
    +	public void close() throws Exception {
    +		suspend();
    +	}
    +
    +	// ---------------------------------------------------------------------------------------------
    +	// Public API
    +	// ---------------------------------------------------------------------------------------------
    +
    +	/**
    +	 * Requests a slot with the respective resource profile.
    +	 *
    +	 * @param slotRequest specifying the requested slot specs
    +	 * @return true if the slot request was registered; false if the request is a duplicate
    +	 * @throws SlotManagerException if the slot request failed (e.g. not enough resources left)
    +	 */
    +	public boolean registerSlotRequest(SlotRequest slotRequest) throws SlotManagerException {
    +		checkInit();
     
    -			// record this allocation in bookkeeping
    -			allocationMap.addAllocation(slot.getSlotId(), allocationId);
    -			// remove selected slot from free pool
    -			freeSlots.remove(slot.getSlotId());
    +		if (checkDuplicateRequest(slotRequest.getAllocationId())) {
    +			LOG.debug("Ignoring a duplicate slot request with allocation id {}.", slotRequest.getAllocationId());
     
    -			sendSlotRequest(slot, request);
    +			return false;
     		} else {
    -			LOG.info("Cannot fulfil slot request, try to allocate a new container for it, " +
    -				"AllocationID:{}, JobID:{}", allocationId, request.getJobId());
    -			Preconditions.checkState(rmServices != null,
    -				"Attempted to allocate resources but no ResourceManagerServices set.");
    -			rmServices.allocateResource(request.getResourceProfile());
    -			pendingSlotRequests.put(allocationId, request);
    +			PendingSlotRequest pendingSlotRequest = new PendingSlotRequest(slotRequest);
    +
    +			pendingSlotRequests.put(slotRequest.getAllocationId(), pendingSlotRequest);
    +
    +			try {
    +				internalRequestSlot(pendingSlotRequest);
    +			} catch (ResourceManagerException e) {
    +				// requesting the slot failed --> remove pending slot request
    +				pendingSlotRequests.remove(slotRequest.getAllocationId());
    +
    +				throw new SlotManagerException("Could not fulfill slot request " + slotRequest.getAllocationId() + '.', e);
    +			}
    +
    +			return true;
     		}
    +	}
     
    -		return new RMSlotRequestRegistered(allocationId);
    +	/**
    +	 * Cancels and removes a pending slot request with the given allocation id. If there is no such
    +	 * pending request, then nothing is done.
    +	 *
    +	 * @param allocationId identifying the pending slot request
    +	 * @return True if a pending slot request was found; otherwise false
    +	 */
    +	public boolean unregisterSlotRequest(AllocationID allocationId) {
    +		checkInit();
    +
    +		PendingSlotRequest pendingSlotRequest = pendingSlotRequests.remove(allocationId);
    +
    +		if (null != pendingSlotRequest) {
    +			cancelPendingSlotRequest(pendingSlotRequest);
    +
    +			return true;
    +		} else {
    +			LOG.debug("No pending slot request with allocation id {} found.", allocationId);
    +
    +			return false;
    +		}
     	}
     
     	/**
    -	 * Notifies the SlotManager that a slot is available again after being allocated.
    -	 * @param slotID slot id of available slot
    +	 * Registers a new task manager at the slot manager. This will make the task managers slots
    +	 * known and, thus, available for allocation.
    +	 *
    +	 * @param taskExecutorConnection for the new task manager
    +	 * @param initialSlotReport for the new task manager
     	 */
    -	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");
    +	public void registerTaskManager(final TaskExecutorConnection taskExecutorConnection, SlotReport initialSlotReport) {
    +		checkInit();
    +
    +		// we identify task managers by their instance id
    +		if (!taskManagerRegistrations.containsKey(taskExecutorConnection.getInstanceID())) {
    +			TaskManagerRegistration taskManagerRegistration = new TaskManagerRegistration(taskExecutorConnection);
    +			taskManagerRegistrations.put(taskExecutorConnection.getInstanceID(), taskManagerRegistration);
     		}
    -		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.");
    +
    +		reportSlotStatus(taskExecutorConnection.getInstanceID(), initialSlotReport);
    +	}
    +
    +	/**
    +	 * Unregisters the task manager identified by the given instance id and its associated slots
    +	 * from the slot manager.
    +	 *
    +	 * @param instanceId identifying the task manager to unregister
    +	 * @return True if there existed a registered task manager with the given instance id
    +	 */
    +	public boolean unregisterTaskManager(InstanceID instanceId) {
    +		checkInit();
    +
    +		TaskManagerRegistration taskManagerRegistration = taskManagerRegistrations.remove(instanceId);
    +
    +		if (null != taskManagerRegistration) {
    +			removeSlots(taskManagerRegistration.getSlots());
    +
    +			taskManagerRegistration.cancelTimeout();
    +
    +			return true;
    +		} else {
    +			LOG.debug("There is no task manager registered with instance ID {}. Ignoring this message.", instanceId);
    +
    +			return false;
     		}
    -		handleFreeSlot(freeSlot);
     	}
     
     	/**
    -	 * 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.
    +	 * Reports the current slot allocations for a task manager identified by the given instance id.
     	 *
    -	 * @param originalRequest The original slot request
    -	 * @param slotId          The target SlotID
    +	 * @param instanceId identifying the task manager for which to report the slot status
    +	 * @param slotReport containing the status for all of its slots
    +	 * @return true if the slot status has been updated successfully, otherwise false
     	 */
    -	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());
    -
    -		if (allocationMap.isAllocated(slotId)) {
    -			final AllocationID expectedAllocationId = allocationMap.getAllocationID(slotId);
    -
    -			// 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);
    +	public boolean reportSlotStatus(InstanceID instanceId, SlotReport slotReport) {
    +		checkInit();
    +
    +		TaskManagerRegistration taskManagerRegistration = taskManagerRegistrations.get(instanceId);
    +
    +		if (null != taskManagerRegistration) {
    +			HashSet<SlotID> slotsToRemove = new HashSet<>(taskManagerRegistration.getSlots());
    --- End diff --
    
    Will also fix it.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3394: [FLINK-5810] [flip6] Introduce a hardened slot man...

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3394#discussion_r113502388
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManager.java ---
    @@ -21,519 +21,897 @@
     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.ResourceSlot;
    +import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
    +import org.apache.flink.runtime.clusterframework.types.TaskManagerSlot;
     import org.apache.flink.runtime.clusterframework.types.SlotID;
     import org.apache.flink.runtime.concurrent.BiFunction;
    +import org.apache.flink.runtime.concurrent.CompletableFuture;
     import org.apache.flink.runtime.concurrent.Future;
    -import org.apache.flink.runtime.resourcemanager.ResourceManagerServices;
    +import org.apache.flink.runtime.concurrent.ScheduledExecutor;
    +import org.apache.flink.runtime.concurrent.impl.FlinkCompletableFuture;
    +import org.apache.flink.runtime.instance.InstanceID;
    +import org.apache.flink.runtime.messages.Acknowledge;
     import org.apache.flink.runtime.resourcemanager.SlotRequest;
    -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.resourcemanager.exceptions.ResourceManagerException;
    +import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
     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.runtime.taskexecutor.exceptions.SlotAllocationException;
    +import org.apache.flink.runtime.taskexecutor.exceptions.SlotOccupiedException;
     import org.apache.flink.util.Preconditions;
     import org.slf4j.Logger;
     import org.slf4j.LoggerFactory;
     
     import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.Iterator;
     import java.util.LinkedHashMap;
     import java.util.Map;
    -
    -import static org.apache.flink.util.Preconditions.checkNotNull;
    +import java.util.Objects;
    +import java.util.UUID;
    +import java.util.concurrent.CancellationException;
    +import java.util.concurrent.Executor;
    +import java.util.concurrent.ScheduledFuture;
    +import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.TimeoutException;
     
     /**
    - * 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>.
    + * The slot manager is responsible for maintaining a view on all registered task manager slots,
    + * their allocation and all pending slot requests. Whenever a new slot is registered or and
    + * allocated slot is freed, then it tries to fulfill another pending slot request. Whenever there
    + * are not enough slots available the slot manager will notify the resource manager about it via
    + * {@link ResourceManagerActions#allocateResource(ResourceProfile)}.
    + *
    + * In order to free resources and avoid resource leaks, idling task managers (task managers whose
    + * slots are currently not used) and not fulfilled pending slot requests time out triggering their
    + * release and failure, respectively.
      */
    -public abstract class SlotManager {
    +public class SlotManager implements AutoCloseable {
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(SlotManager.class);
    +
    +	/** Scheduled executor for timeouts */
    +	private final ScheduledExecutor scheduledExecutor;
    +
    +	/** Timeout for slot requests to the task manager */
    +	private final Time taskManagerRequestTimeout;
    +
    +	/** Timeout after which an allocation is discarded */
    +	private final Time slotRequestTimeout;
    +
    +	/** Timeout after which an unused TaskManager is released */
    +	private final Time taskManagerTimeout;
    +
    +	/** Map for all registered slots */
    +	private final HashMap<SlotID, TaskManagerSlot> slots;
     
    -	protected final Logger LOG = LoggerFactory.getLogger(getClass());
    +	/** Index of all currently free slots */
    +	private final LinkedHashMap<SlotID, TaskManagerSlot> freeSlots;
     
    -	/** The Resource allocation provider */
    -	protected final ResourceManagerServices rmServices;
    +	/** All currently registered task managers */
    +	private final HashMap<InstanceID, TaskManagerRegistration> taskManagerRegistrations;
     
    -	/** All registered task managers with ResourceID and gateway. */
    -	private final Map<ResourceID, TaskExecutorRegistration> taskManagers;
    +	/** Map of fulfilled and active allocations for request deduplication purposes */
    +	private final HashMap<AllocationID, SlotID> fulfilledSlotRequests;
     
    -	/** All registered slots, including free and allocated slots */
    -	private final Map<ResourceID, Map<SlotID, ResourceSlot>> registeredSlots;
    +	/** Map of pending/unfulfilled slot allocation requests */
    +	private final HashMap<AllocationID, PendingSlotRequest> pendingSlotRequests;
     
    -	/** All pending slot requests, waiting available slots to fulfil */
    -	private final Map<AllocationID, SlotRequest> pendingSlotRequests;
    +	/** Leader id of the containing component */
    +	private UUID leaderId;
     
    -	/** All free slots that can be used to be allocated */
    -	private final Map<SlotID, ResourceSlot> freeSlots;
    +	/** Executor for future callbacks which have to be "synchronized" */
    +	private Executor mainThreadExecutor;
     
    -	/** All allocations, we can lookup allocations either by SlotID or AllocationID */
    -	private final AllocationMap allocationMap;
    +	/** Callbacks for resource (de-)allocations */
    +	private ResourceManagerActions resourceManagerActions;
     
    -	private final Time timeout;
    +	/** True iff the component has been started */
    +	private boolean started;
     
    -	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.taskManagers = new HashMap<>();
    -		this.timeout = Time.seconds(10);
    +	public SlotManager(
    +			ScheduledExecutor scheduledExecutor,
    +			Time taskManagerRequestTimeout,
    +			Time slotRequestTimeout,
    +			Time taskManagerTimeout) {
    +		this.scheduledExecutor = Preconditions.checkNotNull(scheduledExecutor);
    +		this.taskManagerRequestTimeout = Preconditions.checkNotNull(taskManagerRequestTimeout);
    +		this.slotRequestTimeout = Preconditions.checkNotNull(slotRequestTimeout);
    +		this.taskManagerTimeout = Preconditions.checkNotNull(taskManagerTimeout);
    +
    +		slots = new HashMap<>(16);
    +		freeSlots = new LinkedHashMap<>(16);
    +		taskManagerRegistrations = new HashMap<>(4);
    +		fulfilledSlotRequests = new HashMap<>(16);
    +		pendingSlotRequests = new HashMap<>(16);
    +
    +		leaderId = null;
    +		resourceManagerActions = null;
    +		started = false;
     	}
     
    -	// ------------------------------------------------------------------------
    -	//  slot managements
    -	// ------------------------------------------------------------------------
    +	// ---------------------------------------------------------------------------------------------
    +	// Component lifecycle methods
    +	// ---------------------------------------------------------------------------------------------
     
     	/**
    -	 * 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).
    +	 * Starts the slot manager with the given leader id and resource manager actions.
     	 *
    -	 * @param request The detailed request of the slot
    -	 * @return RMSlotRequestRegistered The confirmation message to be send to the caller
    +	 * @param newLeaderId to use for communication with the task managers
    +	 * @param newResourceManagerActions to use for resource (de-)allocations
    +	 */
    +	public void start(UUID newLeaderId, Executor newMainThreadExecutor, ResourceManagerActions newResourceManagerActions) {
    +		leaderId = Preconditions.checkNotNull(newLeaderId);
    +		mainThreadExecutor = Preconditions.checkNotNull(newMainThreadExecutor);
    +		resourceManagerActions = Preconditions.checkNotNull(newResourceManagerActions);
    +
    +		started = true;
    +	}
    +
    +	/**
    +	 * Suspends the component. This clears the internal state of the slot manager.
     	 */
    -	public RMSlotRequestRegistered requestSlot(final SlotRequest request) {
    -		final AllocationID allocationId = request.getAllocationId();
    -		if (isRequestDuplicated(request)) {
    -			LOG.warn("Duplicated slot request, AllocationID:{}", allocationId);
    -			return new RMSlotRequestRegistered(allocationId);
    +	public void suspend() {
    +		for (PendingSlotRequest pendingSlotRequest : pendingSlotRequests.values()) {
    +			cancelPendingSlotRequest(pendingSlotRequest);
     		}
     
    -		// 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());
    +		pendingSlotRequests.clear();
    +
    +		HashSet<InstanceID> registeredTaskManagers = new HashSet<>(taskManagerRegistrations.keySet());
    +
    +		for (InstanceID registeredTaskManager : registeredTaskManagers) {
    +			unregisterTaskManager(registeredTaskManager);
    +		}
    +
    +		leaderId = null;
    +		resourceManagerActions = null;
    +		started = false;
    +	}
    +
    +	/**
    +	 * Closes the slot manager.
    +	 *
    +	 * @throws Exception if the close operation fails
    +	 */
    +	@Override
    +	public void close() throws Exception {
    +		suspend();
    +	}
    +
    +	// ---------------------------------------------------------------------------------------------
    +	// Public API
    +	// ---------------------------------------------------------------------------------------------
    +
    +	/**
    +	 * Requests a slot with the respective resource profile.
    +	 *
    +	 * @param slotRequest specifying the requested slot specs
    +	 * @return true if the slot request was registered; false if the request is a duplicate
    +	 * @throws SlotManagerException if the slot request failed (e.g. not enough resources left)
    +	 */
    +	public boolean registerSlotRequest(SlotRequest slotRequest) throws SlotManagerException {
    +		checkInit();
     
    -			// record this allocation in bookkeeping
    -			allocationMap.addAllocation(slot.getSlotId(), allocationId);
    -			// remove selected slot from free pool
    -			freeSlots.remove(slot.getSlotId());
    +		if (checkDuplicateRequest(slotRequest.getAllocationId())) {
    +			LOG.debug("Ignoring a duplicate slot request with allocation id {}.", slotRequest.getAllocationId());
     
    -			sendSlotRequest(slot, request);
    +			return false;
     		} else {
    -			LOG.info("Cannot fulfil slot request, try to allocate a new container for it, " +
    -				"AllocationID:{}, JobID:{}", allocationId, request.getJobId());
    -			Preconditions.checkState(rmServices != null,
    -				"Attempted to allocate resources but no ResourceManagerServices set.");
    -			rmServices.allocateResource(request.getResourceProfile());
    -			pendingSlotRequests.put(allocationId, request);
    +			PendingSlotRequest pendingSlotRequest = new PendingSlotRequest(slotRequest);
    +
    +			pendingSlotRequests.put(slotRequest.getAllocationId(), pendingSlotRequest);
    +
    +			try {
    +				internalRequestSlot(pendingSlotRequest);
    +			} catch (ResourceManagerException e) {
    +				// requesting the slot failed --> remove pending slot request
    +				pendingSlotRequests.remove(slotRequest.getAllocationId());
    +
    +				throw new SlotManagerException("Could not fulfill slot request " + slotRequest.getAllocationId() + '.', e);
    +			}
    +
    +			return true;
     		}
    +	}
     
    -		return new RMSlotRequestRegistered(allocationId);
    +	/**
    +	 * Cancels and removes a pending slot request with the given allocation id. If there is no such
    +	 * pending request, then nothing is done.
    +	 *
    +	 * @param allocationId identifying the pending slot request
    +	 * @return True if a pending slot request was found; otherwise false
    +	 */
    +	public boolean unregisterSlotRequest(AllocationID allocationId) {
    +		checkInit();
    +
    +		PendingSlotRequest pendingSlotRequest = pendingSlotRequests.remove(allocationId);
    +
    +		if (null != pendingSlotRequest) {
    +			cancelPendingSlotRequest(pendingSlotRequest);
    +
    +			return true;
    +		} else {
    +			LOG.debug("No pending slot request with allocation id {} found.", allocationId);
    +
    +			return false;
    +		}
     	}
     
     	/**
    -	 * Notifies the SlotManager that a slot is available again after being allocated.
    -	 * @param slotID slot id of available slot
    +	 * Registers a new task manager at the slot manager. This will make the task managers slots
    +	 * known and, thus, available for allocation.
    +	 *
    +	 * @param taskExecutorConnection for the new task manager
    +	 * @param initialSlotReport for the new task manager
     	 */
    -	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");
    +	public void registerTaskManager(final TaskExecutorConnection taskExecutorConnection, SlotReport initialSlotReport) {
    +		checkInit();
    +
    +		// we identify task managers by their instance id
    +		if (!taskManagerRegistrations.containsKey(taskExecutorConnection.getInstanceID())) {
    +			TaskManagerRegistration taskManagerRegistration = new TaskManagerRegistration(taskExecutorConnection);
    +			taskManagerRegistrations.put(taskExecutorConnection.getInstanceID(), taskManagerRegistration);
     		}
    -		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.");
    +
    +		reportSlotStatus(taskExecutorConnection.getInstanceID(), initialSlotReport);
    +	}
    +
    +	/**
    +	 * Unregisters the task manager identified by the given instance id and its associated slots
    +	 * from the slot manager.
    +	 *
    +	 * @param instanceId identifying the task manager to unregister
    +	 * @return True if there existed a registered task manager with the given instance id
    +	 */
    +	public boolean unregisterTaskManager(InstanceID instanceId) {
    +		checkInit();
    +
    +		TaskManagerRegistration taskManagerRegistration = taskManagerRegistrations.remove(instanceId);
    +
    +		if (null != taskManagerRegistration) {
    +			removeSlots(taskManagerRegistration.getSlots());
    +
    +			taskManagerRegistration.cancelTimeout();
    +
    +			return true;
    +		} else {
    +			LOG.debug("There is no task manager registered with instance ID {}. Ignoring this message.", instanceId);
    +
    +			return false;
     		}
    -		handleFreeSlot(freeSlot);
     	}
     
     	/**
    -	 * 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.
    +	 * Reports the current slot allocations for a task manager identified by the given instance id.
     	 *
    -	 * @param originalRequest The original slot request
    -	 * @param slotId          The target SlotID
    +	 * @param instanceId identifying the task manager for which to report the slot status
    +	 * @param slotReport containing the status for all of its slots
    +	 * @return true if the slot status has been updated successfully, otherwise false
     	 */
    -	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());
    -
    -		if (allocationMap.isAllocated(slotId)) {
    -			final AllocationID expectedAllocationId = allocationMap.getAllocationID(slotId);
    -
    -			// 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);
    +	public boolean reportSlotStatus(InstanceID instanceId, SlotReport slotReport) {
    +		checkInit();
    +
    +		TaskManagerRegistration taskManagerRegistration = taskManagerRegistrations.get(instanceId);
    +
    +		if (null != taskManagerRegistration) {
    +			HashSet<SlotID> slotsToRemove = new HashSet<>(taskManagerRegistration.getSlots());
    +			boolean idle = true;
    +
    +			for (SlotStatus slotStatus : slotReport) {
    +				if (slotsToRemove.remove(slotStatus.getSlotID())) {
    +					// slot which was already registered
    +					updateSlot(slotStatus.getSlotID(), slotStatus.getAllocationID());
    +				} else {
    +					// new slot
    +					registerSlot(
    +						instanceId,
    +						slotStatus.getSlotID(),
    +						slotStatus.getAllocationID(),
    +						slotStatus.getResourceProfile(),
    +						taskManagerRegistration.getTaskManagerConnection());
    +				}
    +
    +				TaskManagerSlot slot = slots.get(slotStatus.getSlotID());
    +
    +				idle &= slot.isFree();
    +			}
    +
    +			// remove the slots for which we haven't received a slot status message
    +			removeSlots(slotsToRemove);
    +
    +			if (idle) {
    +				// no slot of this task manager is being used --> register timer to free this resource
    +				registerTaskManagerTimeout(taskManagerRegistration);
    +			}
    +
    +			return true;
    +		} else {
    +			LOG.debug("Received slot report for unknown task manager with instance id {}. Ignoring this report.", instanceId);
    +
    +			return false;
    +		}
    +	}
    +
    +	/**
    +	 * Free the given slot from the given allocation. If the slot is still allocated by the given
    +	 * allocation id, then the slot will be marked as free and will be subject to new slot requests.
    +	 *
    +	 * @param slotId identifying the slot to free
    +	 * @param allocationId with which the slot is presumably allocated
    +	 */
    +	public void freeSlot(SlotID slotId, AllocationID allocationId) {
    +		checkInit();
    +
    +		TaskManagerSlot slot = slots.get(slotId);
    +
    +		if (null != slot) {
    +			if (slot.isAllocated()) {
    +				if (Objects.equals(allocationId, slot.getAllocationId())) {
    +					// free the slot
    +					slot.setAllocationId(null);
    +					fulfilledSlotRequests.remove(allocationId);
    +
    +					if (slot.isFree()) {
    +						handleFreeSlot(slot);
    +					}
    +
    +					TaskManagerRegistration taskManagerRegistration = taskManagerRegistrations.get(slot.getInstanceId());
    +
    +					if (null != taskManagerRegistration && !anySlotUsed(taskManagerRegistration.getSlots())) {
    +						registerTaskManagerTimeout(taskManagerRegistration);
    +					}
    +				} else {
    +					LOG.debug("Received request to free slot {} with expected allocation id {}, " +
    +						"but actual allocation id {} differs. Ignoring the request.", slotId, allocationId, slot.getAllocationId());
    +				}
     			} else {
    -				LOG.error("Slot request failed for slot {} with allocation id {}:" +
    -						" Allocation id did not match the expected allocation id {}.",
    -					slotId, originalAllocationId, expectedAllocationId);
    +				LOG.debug("Slot {} has not been allocated.", allocationId);
     			}
     		} else {
    -			LOG.error("Slot request failed for slot {} with allocation id {}: " +
    -					"Slot was not previously registered.",
    -				slotId, originalAllocationId);
    +			LOG.debug("Trying to free a slot {} which has not been registered. Ignoring this message.", slotId);
     		}
     	}
     
    +	// ---------------------------------------------------------------------------------------------
    +	// Behaviour methods
    +	// ---------------------------------------------------------------------------------------------
    +
     	/**
    -	 * Registers a TaskExecutor
    -	 * @param resourceID TaskExecutor's ResourceID
    -	 * @param registration TaskExecutor's registration
    -	 * @param slotReport TaskExecutor's free and allocated slots
    +	 * Finds a matching slot request for a given resource profile. If there is no such request,
    +	 * the method returns null.
    +	 *
    +	 * Note: If you want to change the behaviour of the slot manager wrt slot allocation and
    +	 * request fulfillment, then you should override this method.
    +	 *
    +	 * @param slotResourceProfile defining the resources of an available slot
    +	 * @return A matching slot request which can be deployed in a slot with the given resource
    +	 * profile. Null if there is no such slot request pending.
     	 */
    -	public void registerTaskExecutor(
    -			ResourceID resourceID,
    -			TaskExecutorRegistration registration,
    -			SlotReport slotReport) {
    +	protected PendingSlotRequest findMatchingRequest(ResourceProfile slotResourceProfile) {
     
    -		if (taskManagers.get(resourceID) != null) {
    -			notifyTaskManagerFailure(resourceID);
    +		for (PendingSlotRequest pendingSlotRequest : pendingSlotRequests.values()) {
    +			if (!pendingSlotRequest.isAssigned() && slotResourceProfile.isMatching(pendingSlotRequest.getResourceProfile())) {
    +				return pendingSlotRequest;
    +			}
     		}
     
    -		this.taskManagers.put(resourceID, registration);
    +		return null;
    +	}
    +
    +	/**
    +	 * Finds a matching slot for a given resource profile. A matching slot has at least as many
    +	 * resources available as the given resource profile. If there is no such slot available, then
    +	 * the method returns null.
    +	 *
    +	 * Note: If you want to change the behaviour of the slot manager wrt slot allocation and
    +	 * request fulfillment, then you should override this method.
    +	 *
    +	 * @param requestResourceProfile specifying the resource requirements for the a slot request
    +	 * @return A matching slot which fulfills the given resource profile. Null if there is no such
    +	 * slot available.
    +	 */
    +	protected TaskManagerSlot findMatchingSlot(ResourceProfile requestResourceProfile) {
    +		Iterator<Map.Entry<SlotID, TaskManagerSlot>> iterator = freeSlots.entrySet().iterator();
    +
    +		while (iterator.hasNext()) {
    +			TaskManagerSlot taskManagerSlot = iterator.next().getValue();
     
    -		for (SlotStatus slotStatus : slotReport.getSlotsStatus()) {
    -			final SlotID slotId = slotStatus.getSlotID();
    +			// sanity check
    +			Preconditions.checkState(taskManagerSlot.isFree());
     
    -			final TaskExecutorRegistration taskExecutorRegistration = taskManagers.get(slotId.getResourceID());
    -			if (taskExecutorRegistration == null) {
    -				LOG.info("Received SlotStatus but ResourceID {} is unknown to the SlotManager",
    -					slotId.getResourceID());
    -				return;
    +			if (taskManagerSlot.getResourceProfile().isMatching(requestResourceProfile)) {
    +				iterator.remove();
    +				return taskManagerSlot;
     			}
    +		}
     
    -			final ResourceSlot slot = new ResourceSlot(slotId, slotStatus.getProfiler(), taskExecutorRegistration);
    +		return null;
    +	}
     
    -			registerNewSlot(slot);
    -			LOG.info("New slot appeared, SlotID:{}, AllocationID:{}", slotId, slotStatus.getAllocationID());
    +	// ---------------------------------------------------------------------------------------------
    +	// Internal slot operations
    +	// ---------------------------------------------------------------------------------------------
     
    -			if (slotStatus.getAllocationID() != null) {
    -				// slot in use, record this in bookkeeping
    -				allocationMap.addAllocation(slotId, slotStatus.getAllocationID());
    -			} else {
    +	/**
    +	 * Registers a slot for the given task manager at the slot manager. The task manager is
    +	 * identified by the given instance id and the slot is identified by the given slot id. The
    +	 * given resource profile defines the available resources for the slot. The task manager
    +	 * connection can be used to communicate with the task manager.
    +	 *
    +	 * @param instanceId identifying the task manager on which the slot lives
    +	 * @param slotId identifying the slot on the task manager
    +	 * @param allocationId which is currently deployed in the slot
    +	 * @param resourceProfile of the slot
    +	 * @param taskManagerConnection to communicate with the remote task manager
    +	 */
    +	private void registerSlot(
    +			InstanceID instanceId,
    +			SlotID slotId,
    +			AllocationID allocationId,
    +			ResourceProfile resourceProfile,
    +			TaskExecutorConnection taskManagerConnection) {
    +		TaskManagerRegistration taskManagerRegistration = taskManagerRegistrations.get(instanceId);
    +
    +		if (null != taskManagerRegistration) {
    +			TaskManagerSlot slot = new TaskManagerSlot(
    +				slotId,
    +				resourceProfile,
    +				taskManagerConnection,
    +				allocationId);
    +
    +			slots.put(slotId, slot);
    +
    +			taskManagerRegistration.addSlot(slotId);
    +
    +			if (slot.isFree()) {
     				handleFreeSlot(slot);
     			}
    +
    +			if (slot.isAllocated()) {
    +				fulfilledSlotRequests.put(slot.getAllocationId(), slotId);
    +			}
    +		} else {
    +			LOG.debug("Trying to register slot for unknown task manager with instance id {}.", instanceId);
     		}
     	}
     
     	/**
    -	 * Callback for TaskManager failures. In case that a TaskManager fails, we have to clean up all its slots.
    +	 * Updates a slot with the given allocation id.
     	 *
    -	 * @param resourceId The ResourceID of the TaskManager
    +	 * @param slotId to update
    +	 * @param allocationId specifying the current allocation of the slot
     	 */
    -	public void notifyTaskManagerFailure(final ResourceID resourceId) {
    -		LOG.info("Resource:{} been notified failure", resourceId);
    -		taskManagers.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);
    +	private void updateSlot(SlotID slotId, AllocationID allocationId) {
    +		TaskManagerSlot slot = slots.get(slotId);
    +
    +		if (null != slot) {
    +			// we assume the given allocation id to be the ground truth (coming from the TM)
    +			slot.setAllocationId(allocationId);
    +
    +			if (null != allocationId) {
    +				if (slot.hasPendingSlotRequest()){
    +					// we have a pending slot request --> check whether we have to reject it
    +					PendingSlotRequest pendingSlotRequest = slot.getAssignedSlotRequest();
    +
    +					if (Objects.equals(pendingSlotRequest.getAllocationId(), allocationId)) {
    +						// we can cancel the slot request because it has been fulfilled
    +						cancelPendingSlotRequest(pendingSlotRequest);
    +
    +						// remove the pending slot request, since it has been completed
    +						pendingSlotRequests.remove(pendingSlotRequest.getAllocationId());
    +					} else {
    +						// this will try to find a new slot for the request
    +						rejectPendingSlotRequest(
    +							pendingSlotRequest,
    +							new Exception("Task manager reported slot " + slotId + " being already allocated."));
    +					}
    +
    +					slot.setAssignedSlotRequest(null);
    +				}
    +
    +				fulfilledSlotRequests.put(allocationId, slotId);
    +
    +				TaskManagerRegistration taskManagerRegistration = taskManagerRegistrations.get(slot.getInstanceId());
    +
    +				if (null != taskManagerRegistration) {
    +					// disable any registered time out for the task manager
    +					taskManagerRegistration.cancelTimeout();
     				}
     			}
    +		} else {
    +			LOG.debug("Trying to update unknown slot with slot id {}.", slotId);
     		}
     	}
     
    -	// ------------------------------------------------------------------------
    -	//  internal behaviors
    -	// ------------------------------------------------------------------------
    -
     	/**
    -	 * 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.
    +	 * Tries to allocate a slot for the given slot request. If there is no slot available, the
    +	 * resource manager is informed to allocate more resources and a timeout for the request is
    +	 * registered.
     	 *
    -	 * @param freeSlot The free slot
    +	 * @param pendingSlotRequest to allocate a slot for
    +	 * @throws ResourceManagerException if the resource manager cannot allocate more resource
     	 */
    -	private void handleFreeSlot(final ResourceSlot freeSlot) {
    -		SlotRequest chosenRequest = chooseRequestToFulfill(freeSlot, pendingSlotRequests);
    +	private void internalRequestSlot(PendingSlotRequest pendingSlotRequest) throws ResourceManagerException {
    +		TaskManagerSlot taskManagerSlot = findMatchingSlot(pendingSlotRequest.getResourceProfile());
     
    -		if (chosenRequest != null) {
    -			final AllocationID allocationId = chosenRequest.getAllocationId();
    -			final SlotRequest slotRequest = pendingSlotRequests.remove(allocationId);
    +		if (taskManagerSlot != null) {
    +			allocateSlot(taskManagerSlot, pendingSlotRequest);
    +		} else {
    +			final UUID timeoutIdentifier = UUID.randomUUID();
    --- End diff --
    
    True, indeed. I think we don't need these identifier anymore if we have a single timeout task which periodically checks all task managers and pending slot requests.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3394: [FLINK-5810] [flip6] Introduce a hardened slot man...

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3394#discussion_r113495941
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/TaskManagerRegistration.java ---
    @@ -0,0 +1,89 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.SlotID;
    +import org.apache.flink.runtime.instance.InstanceID;
    +import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
    +import org.apache.flink.util.Preconditions;
    +
    +import java.util.HashSet;
    +import java.util.Set;
    +import java.util.UUID;
    +import java.util.concurrent.ScheduledFuture;
    +
    +public class TaskManagerRegistration {
    +
    +	private final TaskExecutorConnection taskManagerConnection;
    +
    +	private final HashSet<SlotID> slots;
    +
    +	private UUID timeoutIdentifier;
    +
    +	private ScheduledFuture<?> timeoutFuture;
    +
    +	public TaskManagerRegistration(TaskExecutorConnection taskManagerConnection) {
    +		this.taskManagerConnection = Preconditions.checkNotNull(taskManagerConnection);
    +
    +		slots = new HashSet<>(4);
    +
    +		timeoutIdentifier = null;
    +		timeoutFuture = null;
    +	}
    +
    +	public TaskExecutorConnection getTaskManagerConnection() {
    +		return taskManagerConnection;
    +	}
    +
    +	public InstanceID getInstanceId() {
    +		return taskManagerConnection.getInstanceID();
    +	}
    +
    +	public UUID getTimeoutIdentifier() {
    +		return timeoutIdentifier;
    +	}
    +
    +	public Set<SlotID> getSlots() {
    +		return slots;
    +	}
    +
    +	public boolean removeSlot(SlotID slotId) {
    +		return slots.remove(slotId);
    +	}
    +
    +	public void addSlot(SlotID slotId) {
    +		slots.add(slotId);
    +	}
    +
    +	public void cancelTimeout() {
    +		if (null != timeoutFuture) {
    +			timeoutFuture.cancel(true);
    --- End diff --
    
    True, will fix this.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3394: [FLINK-5810] [flip6] Introduce a hardened slot man...

Posted by StephanEwen <gi...@git.apache.org>.
Github user StephanEwen commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3394#discussion_r106514810
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManager.java ---
    @@ -21,519 +21,897 @@
     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.ResourceSlot;
    +import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
    +import org.apache.flink.runtime.clusterframework.types.TaskManagerSlot;
     import org.apache.flink.runtime.clusterframework.types.SlotID;
     import org.apache.flink.runtime.concurrent.BiFunction;
    +import org.apache.flink.runtime.concurrent.CompletableFuture;
     import org.apache.flink.runtime.concurrent.Future;
    -import org.apache.flink.runtime.resourcemanager.ResourceManagerServices;
    +import org.apache.flink.runtime.concurrent.ScheduledExecutor;
    +import org.apache.flink.runtime.concurrent.impl.FlinkCompletableFuture;
    +import org.apache.flink.runtime.instance.InstanceID;
    +import org.apache.flink.runtime.messages.Acknowledge;
     import org.apache.flink.runtime.resourcemanager.SlotRequest;
    -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.resourcemanager.exceptions.ResourceManagerException;
    +import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
     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.runtime.taskexecutor.exceptions.SlotAllocationException;
    +import org.apache.flink.runtime.taskexecutor.exceptions.SlotOccupiedException;
     import org.apache.flink.util.Preconditions;
     import org.slf4j.Logger;
     import org.slf4j.LoggerFactory;
     
     import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.Iterator;
     import java.util.LinkedHashMap;
     import java.util.Map;
    -
    -import static org.apache.flink.util.Preconditions.checkNotNull;
    +import java.util.Objects;
    +import java.util.UUID;
    +import java.util.concurrent.CancellationException;
    +import java.util.concurrent.Executor;
    +import java.util.concurrent.ScheduledFuture;
    +import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.TimeoutException;
     
     /**
    - * 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>.
    + * The slot manager is responsible for maintaining a view on all registered task manager slots,
    + * their allocation and all pending slot requests. Whenever a new slot is registered or and
    + * allocated slot is freed, then it tries to fulfill another pending slot request. Whenever there
    + * are not enough slots available the slot manager will notify the resource manager about it via
    + * {@link ResourceManagerActions#allocateResource(ResourceProfile)}.
    + *
    + * In order to free resources and avoid resource leaks, idling task managers (task managers whose
    + * slots are currently not used) and not fulfilled pending slot requests time out triggering their
    + * release and failure, respectively.
      */
    -public abstract class SlotManager {
    +public class SlotManager implements AutoCloseable {
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(SlotManager.class);
    +
    +	/** Scheduled executor for timeouts */
    +	private final ScheduledExecutor scheduledExecutor;
    +
    +	/** Timeout for slot requests to the task manager */
    +	private final Time taskManagerRequestTimeout;
    +
    +	/** Timeout after which an allocation is discarded */
    +	private final Time slotRequestTimeout;
    +
    +	/** Timeout after which an unused TaskManager is released */
    +	private final Time taskManagerTimeout;
    +
    +	/** Map for all registered slots */
    +	private final HashMap<SlotID, TaskManagerSlot> slots;
     
    -	protected final Logger LOG = LoggerFactory.getLogger(getClass());
    +	/** Index of all currently free slots */
    +	private final LinkedHashMap<SlotID, TaskManagerSlot> freeSlots;
     
    -	/** The Resource allocation provider */
    -	protected final ResourceManagerServices rmServices;
    +	/** All currently registered task managers */
    +	private final HashMap<InstanceID, TaskManagerRegistration> taskManagerRegistrations;
     
    -	/** All registered task managers with ResourceID and gateway. */
    -	private final Map<ResourceID, TaskExecutorRegistration> taskManagers;
    +	/** Map of fulfilled and active allocations for request deduplication purposes */
    +	private final HashMap<AllocationID, SlotID> fulfilledSlotRequests;
     
    -	/** All registered slots, including free and allocated slots */
    -	private final Map<ResourceID, Map<SlotID, ResourceSlot>> registeredSlots;
    +	/** Map of pending/unfulfilled slot allocation requests */
    +	private final HashMap<AllocationID, PendingSlotRequest> pendingSlotRequests;
     
    -	/** All pending slot requests, waiting available slots to fulfil */
    -	private final Map<AllocationID, SlotRequest> pendingSlotRequests;
    +	/** Leader id of the containing component */
    +	private UUID leaderId;
     
    -	/** All free slots that can be used to be allocated */
    -	private final Map<SlotID, ResourceSlot> freeSlots;
    +	/** Executor for future callbacks which have to be "synchronized" */
    +	private Executor mainThreadExecutor;
     
    -	/** All allocations, we can lookup allocations either by SlotID or AllocationID */
    -	private final AllocationMap allocationMap;
    +	/** Callbacks for resource (de-)allocations */
    +	private ResourceManagerActions resourceManagerActions;
     
    -	private final Time timeout;
    +	/** True iff the component has been started */
    +	private boolean started;
     
    -	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.taskManagers = new HashMap<>();
    -		this.timeout = Time.seconds(10);
    +	public SlotManager(
    +			ScheduledExecutor scheduledExecutor,
    +			Time taskManagerRequestTimeout,
    +			Time slotRequestTimeout,
    +			Time taskManagerTimeout) {
    +		this.scheduledExecutor = Preconditions.checkNotNull(scheduledExecutor);
    +		this.taskManagerRequestTimeout = Preconditions.checkNotNull(taskManagerRequestTimeout);
    +		this.slotRequestTimeout = Preconditions.checkNotNull(slotRequestTimeout);
    +		this.taskManagerTimeout = Preconditions.checkNotNull(taskManagerTimeout);
    +
    +		slots = new HashMap<>(16);
    +		freeSlots = new LinkedHashMap<>(16);
    +		taskManagerRegistrations = new HashMap<>(4);
    +		fulfilledSlotRequests = new HashMap<>(16);
    +		pendingSlotRequests = new HashMap<>(16);
    +
    +		leaderId = null;
    +		resourceManagerActions = null;
    +		started = false;
     	}
     
    -	// ------------------------------------------------------------------------
    -	//  slot managements
    -	// ------------------------------------------------------------------------
    +	// ---------------------------------------------------------------------------------------------
    +	// Component lifecycle methods
    +	// ---------------------------------------------------------------------------------------------
     
     	/**
    -	 * 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).
    +	 * Starts the slot manager with the given leader id and resource manager actions.
     	 *
    -	 * @param request The detailed request of the slot
    -	 * @return RMSlotRequestRegistered The confirmation message to be send to the caller
    +	 * @param newLeaderId to use for communication with the task managers
    +	 * @param newResourceManagerActions to use for resource (de-)allocations
    +	 */
    +	public void start(UUID newLeaderId, Executor newMainThreadExecutor, ResourceManagerActions newResourceManagerActions) {
    +		leaderId = Preconditions.checkNotNull(newLeaderId);
    +		mainThreadExecutor = Preconditions.checkNotNull(newMainThreadExecutor);
    +		resourceManagerActions = Preconditions.checkNotNull(newResourceManagerActions);
    +
    +		started = true;
    +	}
    +
    +	/**
    +	 * Suspends the component. This clears the internal state of the slot manager.
     	 */
    -	public RMSlotRequestRegistered requestSlot(final SlotRequest request) {
    -		final AllocationID allocationId = request.getAllocationId();
    -		if (isRequestDuplicated(request)) {
    -			LOG.warn("Duplicated slot request, AllocationID:{}", allocationId);
    -			return new RMSlotRequestRegistered(allocationId);
    +	public void suspend() {
    +		for (PendingSlotRequest pendingSlotRequest : pendingSlotRequests.values()) {
    +			cancelPendingSlotRequest(pendingSlotRequest);
     		}
     
    -		// 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());
    +		pendingSlotRequests.clear();
    +
    +		HashSet<InstanceID> registeredTaskManagers = new HashSet<>(taskManagerRegistrations.keySet());
    +
    +		for (InstanceID registeredTaskManager : registeredTaskManagers) {
    +			unregisterTaskManager(registeredTaskManager);
    +		}
    +
    +		leaderId = null;
    +		resourceManagerActions = null;
    +		started = false;
    +	}
    +
    +	/**
    +	 * Closes the slot manager.
    +	 *
    +	 * @throws Exception if the close operation fails
    +	 */
    +	@Override
    +	public void close() throws Exception {
    +		suspend();
    +	}
    +
    +	// ---------------------------------------------------------------------------------------------
    +	// Public API
    +	// ---------------------------------------------------------------------------------------------
    +
    +	/**
    +	 * Requests a slot with the respective resource profile.
    +	 *
    +	 * @param slotRequest specifying the requested slot specs
    +	 * @return true if the slot request was registered; false if the request is a duplicate
    +	 * @throws SlotManagerException if the slot request failed (e.g. not enough resources left)
    +	 */
    +	public boolean registerSlotRequest(SlotRequest slotRequest) throws SlotManagerException {
    +		checkInit();
     
    -			// record this allocation in bookkeeping
    -			allocationMap.addAllocation(slot.getSlotId(), allocationId);
    -			// remove selected slot from free pool
    -			freeSlots.remove(slot.getSlotId());
    +		if (checkDuplicateRequest(slotRequest.getAllocationId())) {
    +			LOG.debug("Ignoring a duplicate slot request with allocation id {}.", slotRequest.getAllocationId());
     
    -			sendSlotRequest(slot, request);
    +			return false;
     		} else {
    -			LOG.info("Cannot fulfil slot request, try to allocate a new container for it, " +
    -				"AllocationID:{}, JobID:{}", allocationId, request.getJobId());
    -			Preconditions.checkState(rmServices != null,
    -				"Attempted to allocate resources but no ResourceManagerServices set.");
    -			rmServices.allocateResource(request.getResourceProfile());
    -			pendingSlotRequests.put(allocationId, request);
    +			PendingSlotRequest pendingSlotRequest = new PendingSlotRequest(slotRequest);
    +
    +			pendingSlotRequests.put(slotRequest.getAllocationId(), pendingSlotRequest);
    +
    +			try {
    +				internalRequestSlot(pendingSlotRequest);
    +			} catch (ResourceManagerException e) {
    +				// requesting the slot failed --> remove pending slot request
    +				pendingSlotRequests.remove(slotRequest.getAllocationId());
    +
    +				throw new SlotManagerException("Could not fulfill slot request " + slotRequest.getAllocationId() + '.', e);
    +			}
    +
    +			return true;
     		}
    +	}
     
    -		return new RMSlotRequestRegistered(allocationId);
    +	/**
    +	 * Cancels and removes a pending slot request with the given allocation id. If there is no such
    +	 * pending request, then nothing is done.
    +	 *
    +	 * @param allocationId identifying the pending slot request
    +	 * @return True if a pending slot request was found; otherwise false
    +	 */
    +	public boolean unregisterSlotRequest(AllocationID allocationId) {
    +		checkInit();
    +
    +		PendingSlotRequest pendingSlotRequest = pendingSlotRequests.remove(allocationId);
    +
    +		if (null != pendingSlotRequest) {
    +			cancelPendingSlotRequest(pendingSlotRequest);
    +
    +			return true;
    +		} else {
    +			LOG.debug("No pending slot request with allocation id {} found.", allocationId);
    +
    +			return false;
    +		}
     	}
     
     	/**
    -	 * Notifies the SlotManager that a slot is available again after being allocated.
    -	 * @param slotID slot id of available slot
    +	 * Registers a new task manager at the slot manager. This will make the task managers slots
    +	 * known and, thus, available for allocation.
    +	 *
    +	 * @param taskExecutorConnection for the new task manager
    +	 * @param initialSlotReport for the new task manager
     	 */
    -	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");
    +	public void registerTaskManager(final TaskExecutorConnection taskExecutorConnection, SlotReport initialSlotReport) {
    +		checkInit();
    +
    +		// we identify task managers by their instance id
    +		if (!taskManagerRegistrations.containsKey(taskExecutorConnection.getInstanceID())) {
    +			TaskManagerRegistration taskManagerRegistration = new TaskManagerRegistration(taskExecutorConnection);
    +			taskManagerRegistrations.put(taskExecutorConnection.getInstanceID(), taskManagerRegistration);
     		}
    -		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.");
    +
    +		reportSlotStatus(taskExecutorConnection.getInstanceID(), initialSlotReport);
    +	}
    +
    +	/**
    +	 * Unregisters the task manager identified by the given instance id and its associated slots
    +	 * from the slot manager.
    +	 *
    +	 * @param instanceId identifying the task manager to unregister
    +	 * @return True if there existed a registered task manager with the given instance id
    +	 */
    +	public boolean unregisterTaskManager(InstanceID instanceId) {
    +		checkInit();
    +
    +		TaskManagerRegistration taskManagerRegistration = taskManagerRegistrations.remove(instanceId);
    +
    +		if (null != taskManagerRegistration) {
    +			removeSlots(taskManagerRegistration.getSlots());
    +
    +			taskManagerRegistration.cancelTimeout();
    +
    +			return true;
    +		} else {
    +			LOG.debug("There is no task manager registered with instance ID {}. Ignoring this message.", instanceId);
    +
    +			return false;
     		}
    -		handleFreeSlot(freeSlot);
     	}
     
     	/**
    -	 * 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.
    +	 * Reports the current slot allocations for a task manager identified by the given instance id.
     	 *
    -	 * @param originalRequest The original slot request
    -	 * @param slotId          The target SlotID
    +	 * @param instanceId identifying the task manager for which to report the slot status
    +	 * @param slotReport containing the status for all of its slots
    +	 * @return true if the slot status has been updated successfully, otherwise false
     	 */
    -	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());
    -
    -		if (allocationMap.isAllocated(slotId)) {
    -			final AllocationID expectedAllocationId = allocationMap.getAllocationID(slotId);
    -
    -			// 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);
    +	public boolean reportSlotStatus(InstanceID instanceId, SlotReport slotReport) {
    +		checkInit();
    +
    +		TaskManagerRegistration taskManagerRegistration = taskManagerRegistrations.get(instanceId);
    +
    +		if (null != taskManagerRegistration) {
    +			HashSet<SlotID> slotsToRemove = new HashSet<>(taskManagerRegistration.getSlots());
    +			boolean idle = true;
    +
    +			for (SlotStatus slotStatus : slotReport) {
    +				if (slotsToRemove.remove(slotStatus.getSlotID())) {
    +					// slot which was already registered
    +					updateSlot(slotStatus.getSlotID(), slotStatus.getAllocationID());
    +				} else {
    +					// new slot
    +					registerSlot(
    +						instanceId,
    +						slotStatus.getSlotID(),
    +						slotStatus.getAllocationID(),
    +						slotStatus.getResourceProfile(),
    +						taskManagerRegistration.getTaskManagerConnection());
    +				}
    +
    +				TaskManagerSlot slot = slots.get(slotStatus.getSlotID());
    +
    +				idle &= slot.isFree();
    +			}
    +
    +			// remove the slots for which we haven't received a slot status message
    +			removeSlots(slotsToRemove);
    +
    +			if (idle) {
    +				// no slot of this task manager is being used --> register timer to free this resource
    +				registerTaskManagerTimeout(taskManagerRegistration);
    +			}
    +
    +			return true;
    +		} else {
    +			LOG.debug("Received slot report for unknown task manager with instance id {}. Ignoring this report.", instanceId);
    +
    +			return false;
    +		}
    +	}
    +
    +	/**
    +	 * Free the given slot from the given allocation. If the slot is still allocated by the given
    +	 * allocation id, then the slot will be marked as free and will be subject to new slot requests.
    +	 *
    +	 * @param slotId identifying the slot to free
    +	 * @param allocationId with which the slot is presumably allocated
    +	 */
    +	public void freeSlot(SlotID slotId, AllocationID allocationId) {
    +		checkInit();
    +
    +		TaskManagerSlot slot = slots.get(slotId);
    +
    +		if (null != slot) {
    +			if (slot.isAllocated()) {
    +				if (Objects.equals(allocationId, slot.getAllocationId())) {
    +					// free the slot
    +					slot.setAllocationId(null);
    +					fulfilledSlotRequests.remove(allocationId);
    +
    +					if (slot.isFree()) {
    +						handleFreeSlot(slot);
    +					}
    +
    +					TaskManagerRegistration taskManagerRegistration = taskManagerRegistrations.get(slot.getInstanceId());
    +
    +					if (null != taskManagerRegistration && !anySlotUsed(taskManagerRegistration.getSlots())) {
    +						registerTaskManagerTimeout(taskManagerRegistration);
    +					}
    +				} else {
    +					LOG.debug("Received request to free slot {} with expected allocation id {}, " +
    +						"but actual allocation id {} differs. Ignoring the request.", slotId, allocationId, slot.getAllocationId());
    +				}
     			} else {
    -				LOG.error("Slot request failed for slot {} with allocation id {}:" +
    -						" Allocation id did not match the expected allocation id {}.",
    -					slotId, originalAllocationId, expectedAllocationId);
    +				LOG.debug("Slot {} has not been allocated.", allocationId);
     			}
     		} else {
    -			LOG.error("Slot request failed for slot {} with allocation id {}: " +
    -					"Slot was not previously registered.",
    -				slotId, originalAllocationId);
    +			LOG.debug("Trying to free a slot {} which has not been registered. Ignoring this message.", slotId);
     		}
     	}
     
    +	// ---------------------------------------------------------------------------------------------
    +	// Behaviour methods
    +	// ---------------------------------------------------------------------------------------------
    +
     	/**
    -	 * Registers a TaskExecutor
    -	 * @param resourceID TaskExecutor's ResourceID
    -	 * @param registration TaskExecutor's registration
    -	 * @param slotReport TaskExecutor's free and allocated slots
    +	 * Finds a matching slot request for a given resource profile. If there is no such request,
    +	 * the method returns null.
    +	 *
    +	 * Note: If you want to change the behaviour of the slot manager wrt slot allocation and
    +	 * request fulfillment, then you should override this method.
    +	 *
    +	 * @param slotResourceProfile defining the resources of an available slot
    +	 * @return A matching slot request which can be deployed in a slot with the given resource
    +	 * profile. Null if there is no such slot request pending.
     	 */
    -	public void registerTaskExecutor(
    -			ResourceID resourceID,
    -			TaskExecutorRegistration registration,
    -			SlotReport slotReport) {
    +	protected PendingSlotRequest findMatchingRequest(ResourceProfile slotResourceProfile) {
     
    -		if (taskManagers.get(resourceID) != null) {
    -			notifyTaskManagerFailure(resourceID);
    +		for (PendingSlotRequest pendingSlotRequest : pendingSlotRequests.values()) {
    +			if (!pendingSlotRequest.isAssigned() && slotResourceProfile.isMatching(pendingSlotRequest.getResourceProfile())) {
    +				return pendingSlotRequest;
    +			}
     		}
     
    -		this.taskManagers.put(resourceID, registration);
    +		return null;
    +	}
    +
    +	/**
    +	 * Finds a matching slot for a given resource profile. A matching slot has at least as many
    +	 * resources available as the given resource profile. If there is no such slot available, then
    +	 * the method returns null.
    +	 *
    +	 * Note: If you want to change the behaviour of the slot manager wrt slot allocation and
    +	 * request fulfillment, then you should override this method.
    +	 *
    +	 * @param requestResourceProfile specifying the resource requirements for the a slot request
    +	 * @return A matching slot which fulfills the given resource profile. Null if there is no such
    +	 * slot available.
    +	 */
    +	protected TaskManagerSlot findMatchingSlot(ResourceProfile requestResourceProfile) {
    +		Iterator<Map.Entry<SlotID, TaskManagerSlot>> iterator = freeSlots.entrySet().iterator();
    +
    +		while (iterator.hasNext()) {
    +			TaskManagerSlot taskManagerSlot = iterator.next().getValue();
     
    -		for (SlotStatus slotStatus : slotReport.getSlotsStatus()) {
    -			final SlotID slotId = slotStatus.getSlotID();
    +			// sanity check
    +			Preconditions.checkState(taskManagerSlot.isFree());
     
    -			final TaskExecutorRegistration taskExecutorRegistration = taskManagers.get(slotId.getResourceID());
    -			if (taskExecutorRegistration == null) {
    -				LOG.info("Received SlotStatus but ResourceID {} is unknown to the SlotManager",
    -					slotId.getResourceID());
    -				return;
    +			if (taskManagerSlot.getResourceProfile().isMatching(requestResourceProfile)) {
    +				iterator.remove();
    +				return taskManagerSlot;
     			}
    +		}
     
    -			final ResourceSlot slot = new ResourceSlot(slotId, slotStatus.getProfiler(), taskExecutorRegistration);
    +		return null;
    +	}
     
    -			registerNewSlot(slot);
    -			LOG.info("New slot appeared, SlotID:{}, AllocationID:{}", slotId, slotStatus.getAllocationID());
    +	// ---------------------------------------------------------------------------------------------
    +	// Internal slot operations
    +	// ---------------------------------------------------------------------------------------------
     
    -			if (slotStatus.getAllocationID() != null) {
    -				// slot in use, record this in bookkeeping
    -				allocationMap.addAllocation(slotId, slotStatus.getAllocationID());
    -			} else {
    +	/**
    +	 * Registers a slot for the given task manager at the slot manager. The task manager is
    +	 * identified by the given instance id and the slot is identified by the given slot id. The
    +	 * given resource profile defines the available resources for the slot. The task manager
    +	 * connection can be used to communicate with the task manager.
    +	 *
    +	 * @param instanceId identifying the task manager on which the slot lives
    +	 * @param slotId identifying the slot on the task manager
    +	 * @param allocationId which is currently deployed in the slot
    +	 * @param resourceProfile of the slot
    +	 * @param taskManagerConnection to communicate with the remote task manager
    +	 */
    +	private void registerSlot(
    +			InstanceID instanceId,
    +			SlotID slotId,
    +			AllocationID allocationId,
    +			ResourceProfile resourceProfile,
    +			TaskExecutorConnection taskManagerConnection) {
    +		TaskManagerRegistration taskManagerRegistration = taskManagerRegistrations.get(instanceId);
    +
    +		if (null != taskManagerRegistration) {
    +			TaskManagerSlot slot = new TaskManagerSlot(
    +				slotId,
    +				resourceProfile,
    +				taskManagerConnection,
    +				allocationId);
    +
    +			slots.put(slotId, slot);
    +
    +			taskManagerRegistration.addSlot(slotId);
    +
    +			if (slot.isFree()) {
     				handleFreeSlot(slot);
     			}
    +
    +			if (slot.isAllocated()) {
    +				fulfilledSlotRequests.put(slot.getAllocationId(), slotId);
    +			}
    +		} else {
    +			LOG.debug("Trying to register slot for unknown task manager with instance id {}.", instanceId);
     		}
     	}
     
     	/**
    -	 * Callback for TaskManager failures. In case that a TaskManager fails, we have to clean up all its slots.
    +	 * Updates a slot with the given allocation id.
     	 *
    -	 * @param resourceId The ResourceID of the TaskManager
    +	 * @param slotId to update
    +	 * @param allocationId specifying the current allocation of the slot
     	 */
    -	public void notifyTaskManagerFailure(final ResourceID resourceId) {
    -		LOG.info("Resource:{} been notified failure", resourceId);
    -		taskManagers.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);
    +	private void updateSlot(SlotID slotId, AllocationID allocationId) {
    +		TaskManagerSlot slot = slots.get(slotId);
    +
    +		if (null != slot) {
    +			// we assume the given allocation id to be the ground truth (coming from the TM)
    +			slot.setAllocationId(allocationId);
    +
    +			if (null != allocationId) {
    +				if (slot.hasPendingSlotRequest()){
    +					// we have a pending slot request --> check whether we have to reject it
    +					PendingSlotRequest pendingSlotRequest = slot.getAssignedSlotRequest();
    +
    +					if (Objects.equals(pendingSlotRequest.getAllocationId(), allocationId)) {
    +						// we can cancel the slot request because it has been fulfilled
    +						cancelPendingSlotRequest(pendingSlotRequest);
    +
    +						// remove the pending slot request, since it has been completed
    +						pendingSlotRequests.remove(pendingSlotRequest.getAllocationId());
    +					} else {
    +						// this will try to find a new slot for the request
    +						rejectPendingSlotRequest(
    +							pendingSlotRequest,
    +							new Exception("Task manager reported slot " + slotId + " being already allocated."));
    +					}
    +
    +					slot.setAssignedSlotRequest(null);
    +				}
    +
    +				fulfilledSlotRequests.put(allocationId, slotId);
    +
    +				TaskManagerRegistration taskManagerRegistration = taskManagerRegistrations.get(slot.getInstanceId());
    +
    +				if (null != taskManagerRegistration) {
    +					// disable any registered time out for the task manager
    +					taskManagerRegistration.cancelTimeout();
     				}
     			}
    +		} else {
    +			LOG.debug("Trying to update unknown slot with slot id {}.", slotId);
     		}
     	}
     
    -	// ------------------------------------------------------------------------
    -	//  internal behaviors
    -	// ------------------------------------------------------------------------
    -
     	/**
    -	 * 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.
    +	 * Tries to allocate a slot for the given slot request. If there is no slot available, the
    +	 * resource manager is informed to allocate more resources and a timeout for the request is
    +	 * registered.
     	 *
    -	 * @param freeSlot The free slot
    +	 * @param pendingSlotRequest to allocate a slot for
    +	 * @throws ResourceManagerException if the resource manager cannot allocate more resource
     	 */
    -	private void handleFreeSlot(final ResourceSlot freeSlot) {
    -		SlotRequest chosenRequest = chooseRequestToFulfill(freeSlot, pendingSlotRequests);
    +	private void internalRequestSlot(PendingSlotRequest pendingSlotRequest) throws ResourceManagerException {
    +		TaskManagerSlot taskManagerSlot = findMatchingSlot(pendingSlotRequest.getResourceProfile());
     
    -		if (chosenRequest != null) {
    -			final AllocationID allocationId = chosenRequest.getAllocationId();
    -			final SlotRequest slotRequest = pendingSlotRequests.remove(allocationId);
    +		if (taskManagerSlot != null) {
    +			allocateSlot(taskManagerSlot, pendingSlotRequest);
    +		} else {
    +			final UUID timeoutIdentifier = UUID.randomUUID();
    --- End diff --
    
    A simple (atomically) incrementing `long` is as efficient and cheaper compared to a cryptographically random uid.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3394: [FLINK-5810] [flip6] Introduce a hardened slot manager

Posted by StephanEwen <gi...@git.apache.org>.
Github user StephanEwen commented on the issue:

    https://github.com/apache/flink/pull/3394
  
    I have some suggested edits on top of this (not strictly tied to the changes here) in that commit: https://github.com/StephanEwen/incubator-flink/commit/910de21972992d0ed80f232f3a64bf107c6819f2


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3394: [FLINK-5810] [flip6] Introduce a hardened slot man...

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3394#discussion_r113479420
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/TaskManagerSlot.java ---
    @@ -18,15 +18,17 @@
     
     package org.apache.flink.runtime.clusterframework.types;
     
    -import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorRegistration;
    +import org.apache.flink.runtime.instance.InstanceID;
    +import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
    +import org.apache.flink.runtime.resourcemanager.slotmanager.PendingSlotRequest;
     
     import static org.apache.flink.util.Preconditions.checkNotNull;
     
     /**
      * A ResourceSlot represents a slot located in TaskManager from ResourceManager's view. It has a unique
    --- End diff --
    
    +1


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3394: [FLINK-5810] [flip6] Introduce a hardened slot man...

Posted by StephanEwen <gi...@git.apache.org>.
Github user StephanEwen commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3394#discussion_r106506811
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManager.java ---
    @@ -21,519 +21,897 @@
     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.ResourceSlot;
    +import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
    +import org.apache.flink.runtime.clusterframework.types.TaskManagerSlot;
     import org.apache.flink.runtime.clusterframework.types.SlotID;
     import org.apache.flink.runtime.concurrent.BiFunction;
    +import org.apache.flink.runtime.concurrent.CompletableFuture;
     import org.apache.flink.runtime.concurrent.Future;
    -import org.apache.flink.runtime.resourcemanager.ResourceManagerServices;
    +import org.apache.flink.runtime.concurrent.ScheduledExecutor;
    +import org.apache.flink.runtime.concurrent.impl.FlinkCompletableFuture;
    +import org.apache.flink.runtime.instance.InstanceID;
    +import org.apache.flink.runtime.messages.Acknowledge;
     import org.apache.flink.runtime.resourcemanager.SlotRequest;
    -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.resourcemanager.exceptions.ResourceManagerException;
    +import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
     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.runtime.taskexecutor.exceptions.SlotAllocationException;
    +import org.apache.flink.runtime.taskexecutor.exceptions.SlotOccupiedException;
     import org.apache.flink.util.Preconditions;
     import org.slf4j.Logger;
     import org.slf4j.LoggerFactory;
     
     import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.Iterator;
     import java.util.LinkedHashMap;
     import java.util.Map;
    -
    -import static org.apache.flink.util.Preconditions.checkNotNull;
    +import java.util.Objects;
    +import java.util.UUID;
    +import java.util.concurrent.CancellationException;
    +import java.util.concurrent.Executor;
    +import java.util.concurrent.ScheduledFuture;
    +import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.TimeoutException;
     
     /**
    - * 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>.
    + * The slot manager is responsible for maintaining a view on all registered task manager slots,
    + * their allocation and all pending slot requests. Whenever a new slot is registered or and
    + * allocated slot is freed, then it tries to fulfill another pending slot request. Whenever there
    + * are not enough slots available the slot manager will notify the resource manager about it via
    + * {@link ResourceManagerActions#allocateResource(ResourceProfile)}.
    + *
    + * In order to free resources and avoid resource leaks, idling task managers (task managers whose
    + * slots are currently not used) and not fulfilled pending slot requests time out triggering their
    + * release and failure, respectively.
      */
    -public abstract class SlotManager {
    +public class SlotManager implements AutoCloseable {
    --- End diff --
    
    Implementing `AutoCloseable` is a neat trick, but I find it almost a bit confusing. Would one ever use a SlotManager in a "try-with-resource" statement?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3394: [FLINK-5810] [flip6] Introduce a hardened slot manager

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on the issue:

    https://github.com/apache/flink/pull/3394
  
    Thanks for the thorough review @StephanEwen. I'll will pull in your commit with the improvements.
    
    Concerning the other points you've raised:
    
    - I think you're right. The timeouts should be configurable. I would like to do as a follow up because the PR is already big enough.
    
    - You're also right that the timeout logic is not properly working. I agree that the (2) approach is probably less brittle. Additionally we would get rid of the timeout identifier generation.
    
    - I think it is fair to assume that the number of slots don't change over the lifetime of a `TaskManager`. I will change the implementation accordingly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3394: [FLINK-5810] [flip6] Introduce a hardened slot man...

Posted by StephanEwen <gi...@git.apache.org>.
Github user StephanEwen commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3394#discussion_r106508907
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManager.java ---
    @@ -21,519 +21,897 @@
     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.ResourceSlot;
    +import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
    +import org.apache.flink.runtime.clusterframework.types.TaskManagerSlot;
     import org.apache.flink.runtime.clusterframework.types.SlotID;
     import org.apache.flink.runtime.concurrent.BiFunction;
    +import org.apache.flink.runtime.concurrent.CompletableFuture;
     import org.apache.flink.runtime.concurrent.Future;
    -import org.apache.flink.runtime.resourcemanager.ResourceManagerServices;
    +import org.apache.flink.runtime.concurrent.ScheduledExecutor;
    +import org.apache.flink.runtime.concurrent.impl.FlinkCompletableFuture;
    +import org.apache.flink.runtime.instance.InstanceID;
    +import org.apache.flink.runtime.messages.Acknowledge;
     import org.apache.flink.runtime.resourcemanager.SlotRequest;
    -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.resourcemanager.exceptions.ResourceManagerException;
    +import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
     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.runtime.taskexecutor.exceptions.SlotAllocationException;
    +import org.apache.flink.runtime.taskexecutor.exceptions.SlotOccupiedException;
     import org.apache.flink.util.Preconditions;
     import org.slf4j.Logger;
     import org.slf4j.LoggerFactory;
     
     import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.Iterator;
     import java.util.LinkedHashMap;
     import java.util.Map;
    -
    -import static org.apache.flink.util.Preconditions.checkNotNull;
    +import java.util.Objects;
    +import java.util.UUID;
    +import java.util.concurrent.CancellationException;
    +import java.util.concurrent.Executor;
    +import java.util.concurrent.ScheduledFuture;
    +import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.TimeoutException;
     
     /**
    - * 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>.
    + * The slot manager is responsible for maintaining a view on all registered task manager slots,
    + * their allocation and all pending slot requests. Whenever a new slot is registered or and
    + * allocated slot is freed, then it tries to fulfill another pending slot request. Whenever there
    + * are not enough slots available the slot manager will notify the resource manager about it via
    + * {@link ResourceManagerActions#allocateResource(ResourceProfile)}.
    + *
    + * In order to free resources and avoid resource leaks, idling task managers (task managers whose
    + * slots are currently not used) and not fulfilled pending slot requests time out triggering their
    + * release and failure, respectively.
      */
    -public abstract class SlotManager {
    +public class SlotManager implements AutoCloseable {
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(SlotManager.class);
    +
    +	/** Scheduled executor for timeouts */
    +	private final ScheduledExecutor scheduledExecutor;
    +
    +	/** Timeout for slot requests to the task manager */
    +	private final Time taskManagerRequestTimeout;
    +
    +	/** Timeout after which an allocation is discarded */
    +	private final Time slotRequestTimeout;
    +
    +	/** Timeout after which an unused TaskManager is released */
    +	private final Time taskManagerTimeout;
    +
    +	/** Map for all registered slots */
    +	private final HashMap<SlotID, TaskManagerSlot> slots;
     
    -	protected final Logger LOG = LoggerFactory.getLogger(getClass());
    +	/** Index of all currently free slots */
    +	private final LinkedHashMap<SlotID, TaskManagerSlot> freeSlots;
     
    -	/** The Resource allocation provider */
    -	protected final ResourceManagerServices rmServices;
    +	/** All currently registered task managers */
    +	private final HashMap<InstanceID, TaskManagerRegistration> taskManagerRegistrations;
     
    -	/** All registered task managers with ResourceID and gateway. */
    -	private final Map<ResourceID, TaskExecutorRegistration> taskManagers;
    +	/** Map of fulfilled and active allocations for request deduplication purposes */
    +	private final HashMap<AllocationID, SlotID> fulfilledSlotRequests;
     
    -	/** All registered slots, including free and allocated slots */
    -	private final Map<ResourceID, Map<SlotID, ResourceSlot>> registeredSlots;
    +	/** Map of pending/unfulfilled slot allocation requests */
    +	private final HashMap<AllocationID, PendingSlotRequest> pendingSlotRequests;
     
    -	/** All pending slot requests, waiting available slots to fulfil */
    -	private final Map<AllocationID, SlotRequest> pendingSlotRequests;
    +	/** Leader id of the containing component */
    +	private UUID leaderId;
     
    -	/** All free slots that can be used to be allocated */
    -	private final Map<SlotID, ResourceSlot> freeSlots;
    +	/** Executor for future callbacks which have to be "synchronized" */
    +	private Executor mainThreadExecutor;
     
    -	/** All allocations, we can lookup allocations either by SlotID or AllocationID */
    -	private final AllocationMap allocationMap;
    +	/** Callbacks for resource (de-)allocations */
    +	private ResourceManagerActions resourceManagerActions;
     
    -	private final Time timeout;
    +	/** True iff the component has been started */
    +	private boolean started;
     
    -	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.taskManagers = new HashMap<>();
    -		this.timeout = Time.seconds(10);
    +	public SlotManager(
    +			ScheduledExecutor scheduledExecutor,
    +			Time taskManagerRequestTimeout,
    +			Time slotRequestTimeout,
    +			Time taskManagerTimeout) {
    +		this.scheduledExecutor = Preconditions.checkNotNull(scheduledExecutor);
    +		this.taskManagerRequestTimeout = Preconditions.checkNotNull(taskManagerRequestTimeout);
    +		this.slotRequestTimeout = Preconditions.checkNotNull(slotRequestTimeout);
    +		this.taskManagerTimeout = Preconditions.checkNotNull(taskManagerTimeout);
    +
    +		slots = new HashMap<>(16);
    +		freeSlots = new LinkedHashMap<>(16);
    +		taskManagerRegistrations = new HashMap<>(4);
    +		fulfilledSlotRequests = new HashMap<>(16);
    +		pendingSlotRequests = new HashMap<>(16);
    +
    +		leaderId = null;
    +		resourceManagerActions = null;
    +		started = false;
     	}
     
    -	// ------------------------------------------------------------------------
    -	//  slot managements
    -	// ------------------------------------------------------------------------
    +	// ---------------------------------------------------------------------------------------------
    +	// Component lifecycle methods
    +	// ---------------------------------------------------------------------------------------------
     
     	/**
    -	 * 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).
    +	 * Starts the slot manager with the given leader id and resource manager actions.
     	 *
    -	 * @param request The detailed request of the slot
    -	 * @return RMSlotRequestRegistered The confirmation message to be send to the caller
    +	 * @param newLeaderId to use for communication with the task managers
    +	 * @param newResourceManagerActions to use for resource (de-)allocations
    +	 */
    +	public void start(UUID newLeaderId, Executor newMainThreadExecutor, ResourceManagerActions newResourceManagerActions) {
    +		leaderId = Preconditions.checkNotNull(newLeaderId);
    +		mainThreadExecutor = Preconditions.checkNotNull(newMainThreadExecutor);
    +		resourceManagerActions = Preconditions.checkNotNull(newResourceManagerActions);
    +
    +		started = true;
    +	}
    +
    +	/**
    +	 * Suspends the component. This clears the internal state of the slot manager.
     	 */
    -	public RMSlotRequestRegistered requestSlot(final SlotRequest request) {
    -		final AllocationID allocationId = request.getAllocationId();
    -		if (isRequestDuplicated(request)) {
    -			LOG.warn("Duplicated slot request, AllocationID:{}", allocationId);
    -			return new RMSlotRequestRegistered(allocationId);
    +	public void suspend() {
    +		for (PendingSlotRequest pendingSlotRequest : pendingSlotRequests.values()) {
    +			cancelPendingSlotRequest(pendingSlotRequest);
     		}
     
    -		// 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());
    +		pendingSlotRequests.clear();
    +
    +		HashSet<InstanceID> registeredTaskManagers = new HashSet<>(taskManagerRegistrations.keySet());
    +
    +		for (InstanceID registeredTaskManager : registeredTaskManagers) {
    +			unregisterTaskManager(registeredTaskManager);
    +		}
    +
    +		leaderId = null;
    +		resourceManagerActions = null;
    +		started = false;
    +	}
    +
    +	/**
    +	 * Closes the slot manager.
    +	 *
    +	 * @throws Exception if the close operation fails
    +	 */
    +	@Override
    +	public void close() throws Exception {
    +		suspend();
    +	}
    +
    +	// ---------------------------------------------------------------------------------------------
    +	// Public API
    +	// ---------------------------------------------------------------------------------------------
    +
    +	/**
    +	 * Requests a slot with the respective resource profile.
    +	 *
    +	 * @param slotRequest specifying the requested slot specs
    +	 * @return true if the slot request was registered; false if the request is a duplicate
    +	 * @throws SlotManagerException if the slot request failed (e.g. not enough resources left)
    +	 */
    +	public boolean registerSlotRequest(SlotRequest slotRequest) throws SlotManagerException {
    +		checkInit();
     
    -			// record this allocation in bookkeeping
    -			allocationMap.addAllocation(slot.getSlotId(), allocationId);
    -			// remove selected slot from free pool
    -			freeSlots.remove(slot.getSlotId());
    +		if (checkDuplicateRequest(slotRequest.getAllocationId())) {
    +			LOG.debug("Ignoring a duplicate slot request with allocation id {}.", slotRequest.getAllocationId());
     
    -			sendSlotRequest(slot, request);
    +			return false;
     		} else {
    -			LOG.info("Cannot fulfil slot request, try to allocate a new container for it, " +
    -				"AllocationID:{}, JobID:{}", allocationId, request.getJobId());
    -			Preconditions.checkState(rmServices != null,
    -				"Attempted to allocate resources but no ResourceManagerServices set.");
    -			rmServices.allocateResource(request.getResourceProfile());
    -			pendingSlotRequests.put(allocationId, request);
    +			PendingSlotRequest pendingSlotRequest = new PendingSlotRequest(slotRequest);
    +
    +			pendingSlotRequests.put(slotRequest.getAllocationId(), pendingSlotRequest);
    +
    +			try {
    +				internalRequestSlot(pendingSlotRequest);
    +			} catch (ResourceManagerException e) {
    +				// requesting the slot failed --> remove pending slot request
    +				pendingSlotRequests.remove(slotRequest.getAllocationId());
    +
    +				throw new SlotManagerException("Could not fulfill slot request " + slotRequest.getAllocationId() + '.', e);
    +			}
    +
    +			return true;
     		}
    +	}
     
    -		return new RMSlotRequestRegistered(allocationId);
    +	/**
    +	 * Cancels and removes a pending slot request with the given allocation id. If there is no such
    +	 * pending request, then nothing is done.
    +	 *
    +	 * @param allocationId identifying the pending slot request
    +	 * @return True if a pending slot request was found; otherwise false
    +	 */
    +	public boolean unregisterSlotRequest(AllocationID allocationId) {
    +		checkInit();
    +
    +		PendingSlotRequest pendingSlotRequest = pendingSlotRequests.remove(allocationId);
    +
    +		if (null != pendingSlotRequest) {
    +			cancelPendingSlotRequest(pendingSlotRequest);
    +
    +			return true;
    +		} else {
    +			LOG.debug("No pending slot request with allocation id {} found.", allocationId);
    +
    +			return false;
    +		}
     	}
     
     	/**
    -	 * Notifies the SlotManager that a slot is available again after being allocated.
    -	 * @param slotID slot id of available slot
    +	 * Registers a new task manager at the slot manager. This will make the task managers slots
    +	 * known and, thus, available for allocation.
    +	 *
    +	 * @param taskExecutorConnection for the new task manager
    +	 * @param initialSlotReport for the new task manager
     	 */
    -	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");
    +	public void registerTaskManager(final TaskExecutorConnection taskExecutorConnection, SlotReport initialSlotReport) {
    +		checkInit();
    +
    +		// we identify task managers by their instance id
    +		if (!taskManagerRegistrations.containsKey(taskExecutorConnection.getInstanceID())) {
    +			TaskManagerRegistration taskManagerRegistration = new TaskManagerRegistration(taskExecutorConnection);
    +			taskManagerRegistrations.put(taskExecutorConnection.getInstanceID(), taskManagerRegistration);
     		}
    -		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.");
    +
    +		reportSlotStatus(taskExecutorConnection.getInstanceID(), initialSlotReport);
    +	}
    +
    +	/**
    +	 * Unregisters the task manager identified by the given instance id and its associated slots
    +	 * from the slot manager.
    +	 *
    +	 * @param instanceId identifying the task manager to unregister
    +	 * @return True if there existed a registered task manager with the given instance id
    +	 */
    +	public boolean unregisterTaskManager(InstanceID instanceId) {
    +		checkInit();
    +
    +		TaskManagerRegistration taskManagerRegistration = taskManagerRegistrations.remove(instanceId);
    +
    +		if (null != taskManagerRegistration) {
    +			removeSlots(taskManagerRegistration.getSlots());
    +
    +			taskManagerRegistration.cancelTimeout();
    +
    +			return true;
    +		} else {
    +			LOG.debug("There is no task manager registered with instance ID {}. Ignoring this message.", instanceId);
    +
    +			return false;
     		}
    -		handleFreeSlot(freeSlot);
     	}
     
     	/**
    -	 * 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.
    +	 * Reports the current slot allocations for a task manager identified by the given instance id.
     	 *
    -	 * @param originalRequest The original slot request
    -	 * @param slotId          The target SlotID
    +	 * @param instanceId identifying the task manager for which to report the slot status
    +	 * @param slotReport containing the status for all of its slots
    +	 * @return true if the slot status has been updated successfully, otherwise false
     	 */
    -	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());
    -
    -		if (allocationMap.isAllocated(slotId)) {
    -			final AllocationID expectedAllocationId = allocationMap.getAllocationID(slotId);
    -
    -			// 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);
    +	public boolean reportSlotStatus(InstanceID instanceId, SlotReport slotReport) {
    +		checkInit();
    +
    +		TaskManagerRegistration taskManagerRegistration = taskManagerRegistrations.get(instanceId);
    +
    +		if (null != taskManagerRegistration) {
    +			HashSet<SlotID> slotsToRemove = new HashSet<>(taskManagerRegistration.getSlots());
    --- End diff --
    
    Same as above: If the copy of the collection is just for concurrent iteration, I would make it an `ArrayList`. Builds faster and iterates faster.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3394: [FLINK-5810] [flip6] Introduce a hardened slot man...

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3394#discussion_r113497948
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManager.java ---
    @@ -21,519 +21,897 @@
     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.ResourceSlot;
    +import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
    +import org.apache.flink.runtime.clusterframework.types.TaskManagerSlot;
     import org.apache.flink.runtime.clusterframework.types.SlotID;
     import org.apache.flink.runtime.concurrent.BiFunction;
    +import org.apache.flink.runtime.concurrent.CompletableFuture;
     import org.apache.flink.runtime.concurrent.Future;
    -import org.apache.flink.runtime.resourcemanager.ResourceManagerServices;
    +import org.apache.flink.runtime.concurrent.ScheduledExecutor;
    +import org.apache.flink.runtime.concurrent.impl.FlinkCompletableFuture;
    +import org.apache.flink.runtime.instance.InstanceID;
    +import org.apache.flink.runtime.messages.Acknowledge;
     import org.apache.flink.runtime.resourcemanager.SlotRequest;
    -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.resourcemanager.exceptions.ResourceManagerException;
    +import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
     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.runtime.taskexecutor.exceptions.SlotAllocationException;
    +import org.apache.flink.runtime.taskexecutor.exceptions.SlotOccupiedException;
     import org.apache.flink.util.Preconditions;
     import org.slf4j.Logger;
     import org.slf4j.LoggerFactory;
     
     import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.Iterator;
     import java.util.LinkedHashMap;
     import java.util.Map;
    -
    -import static org.apache.flink.util.Preconditions.checkNotNull;
    +import java.util.Objects;
    +import java.util.UUID;
    +import java.util.concurrent.CancellationException;
    +import java.util.concurrent.Executor;
    +import java.util.concurrent.ScheduledFuture;
    +import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.TimeoutException;
     
     /**
    - * 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>.
    + * The slot manager is responsible for maintaining a view on all registered task manager slots,
    + * their allocation and all pending slot requests. Whenever a new slot is registered or and
    + * allocated slot is freed, then it tries to fulfill another pending slot request. Whenever there
    + * are not enough slots available the slot manager will notify the resource manager about it via
    + * {@link ResourceManagerActions#allocateResource(ResourceProfile)}.
    + *
    + * In order to free resources and avoid resource leaks, idling task managers (task managers whose
    + * slots are currently not used) and not fulfilled pending slot requests time out triggering their
    + * release and failure, respectively.
      */
    -public abstract class SlotManager {
    +public class SlotManager implements AutoCloseable {
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(SlotManager.class);
    +
    +	/** Scheduled executor for timeouts */
    +	private final ScheduledExecutor scheduledExecutor;
    +
    +	/** Timeout for slot requests to the task manager */
    +	private final Time taskManagerRequestTimeout;
    +
    +	/** Timeout after which an allocation is discarded */
    +	private final Time slotRequestTimeout;
    +
    +	/** Timeout after which an unused TaskManager is released */
    +	private final Time taskManagerTimeout;
    +
    +	/** Map for all registered slots */
    +	private final HashMap<SlotID, TaskManagerSlot> slots;
     
    -	protected final Logger LOG = LoggerFactory.getLogger(getClass());
    +	/** Index of all currently free slots */
    +	private final LinkedHashMap<SlotID, TaskManagerSlot> freeSlots;
     
    -	/** The Resource allocation provider */
    -	protected final ResourceManagerServices rmServices;
    +	/** All currently registered task managers */
    +	private final HashMap<InstanceID, TaskManagerRegistration> taskManagerRegistrations;
     
    -	/** All registered task managers with ResourceID and gateway. */
    -	private final Map<ResourceID, TaskExecutorRegistration> taskManagers;
    +	/** Map of fulfilled and active allocations for request deduplication purposes */
    +	private final HashMap<AllocationID, SlotID> fulfilledSlotRequests;
     
    -	/** All registered slots, including free and allocated slots */
    -	private final Map<ResourceID, Map<SlotID, ResourceSlot>> registeredSlots;
    +	/** Map of pending/unfulfilled slot allocation requests */
    +	private final HashMap<AllocationID, PendingSlotRequest> pendingSlotRequests;
     
    -	/** All pending slot requests, waiting available slots to fulfil */
    -	private final Map<AllocationID, SlotRequest> pendingSlotRequests;
    +	/** Leader id of the containing component */
    +	private UUID leaderId;
     
    -	/** All free slots that can be used to be allocated */
    -	private final Map<SlotID, ResourceSlot> freeSlots;
    +	/** Executor for future callbacks which have to be "synchronized" */
    +	private Executor mainThreadExecutor;
     
    -	/** All allocations, we can lookup allocations either by SlotID or AllocationID */
    -	private final AllocationMap allocationMap;
    +	/** Callbacks for resource (de-)allocations */
    +	private ResourceManagerActions resourceManagerActions;
     
    -	private final Time timeout;
    +	/** True iff the component has been started */
    +	private boolean started;
     
    -	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.taskManagers = new HashMap<>();
    -		this.timeout = Time.seconds(10);
    +	public SlotManager(
    +			ScheduledExecutor scheduledExecutor,
    +			Time taskManagerRequestTimeout,
    +			Time slotRequestTimeout,
    +			Time taskManagerTimeout) {
    +		this.scheduledExecutor = Preconditions.checkNotNull(scheduledExecutor);
    +		this.taskManagerRequestTimeout = Preconditions.checkNotNull(taskManagerRequestTimeout);
    +		this.slotRequestTimeout = Preconditions.checkNotNull(slotRequestTimeout);
    +		this.taskManagerTimeout = Preconditions.checkNotNull(taskManagerTimeout);
    +
    +		slots = new HashMap<>(16);
    +		freeSlots = new LinkedHashMap<>(16);
    +		taskManagerRegistrations = new HashMap<>(4);
    +		fulfilledSlotRequests = new HashMap<>(16);
    +		pendingSlotRequests = new HashMap<>(16);
    +
    +		leaderId = null;
    +		resourceManagerActions = null;
    +		started = false;
     	}
     
    -	// ------------------------------------------------------------------------
    -	//  slot managements
    -	// ------------------------------------------------------------------------
    +	// ---------------------------------------------------------------------------------------------
    +	// Component lifecycle methods
    +	// ---------------------------------------------------------------------------------------------
     
     	/**
    -	 * 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).
    +	 * Starts the slot manager with the given leader id and resource manager actions.
     	 *
    -	 * @param request The detailed request of the slot
    -	 * @return RMSlotRequestRegistered The confirmation message to be send to the caller
    +	 * @param newLeaderId to use for communication with the task managers
    +	 * @param newResourceManagerActions to use for resource (de-)allocations
    +	 */
    +	public void start(UUID newLeaderId, Executor newMainThreadExecutor, ResourceManagerActions newResourceManagerActions) {
    +		leaderId = Preconditions.checkNotNull(newLeaderId);
    +		mainThreadExecutor = Preconditions.checkNotNull(newMainThreadExecutor);
    +		resourceManagerActions = Preconditions.checkNotNull(newResourceManagerActions);
    +
    +		started = true;
    +	}
    +
    +	/**
    +	 * Suspends the component. This clears the internal state of the slot manager.
     	 */
    -	public RMSlotRequestRegistered requestSlot(final SlotRequest request) {
    -		final AllocationID allocationId = request.getAllocationId();
    -		if (isRequestDuplicated(request)) {
    -			LOG.warn("Duplicated slot request, AllocationID:{}", allocationId);
    -			return new RMSlotRequestRegistered(allocationId);
    +	public void suspend() {
    +		for (PendingSlotRequest pendingSlotRequest : pendingSlotRequests.values()) {
    +			cancelPendingSlotRequest(pendingSlotRequest);
     		}
     
    -		// 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());
    +		pendingSlotRequests.clear();
    +
    +		HashSet<InstanceID> registeredTaskManagers = new HashSet<>(taskManagerRegistrations.keySet());
    +
    +		for (InstanceID registeredTaskManager : registeredTaskManagers) {
    +			unregisterTaskManager(registeredTaskManager);
    +		}
    +
    +		leaderId = null;
    +		resourceManagerActions = null;
    +		started = false;
    +	}
    +
    +	/**
    +	 * Closes the slot manager.
    +	 *
    +	 * @throws Exception if the close operation fails
    +	 */
    +	@Override
    +	public void close() throws Exception {
    +		suspend();
    +	}
    +
    +	// ---------------------------------------------------------------------------------------------
    +	// Public API
    +	// ---------------------------------------------------------------------------------------------
    +
    +	/**
    +	 * Requests a slot with the respective resource profile.
    +	 *
    +	 * @param slotRequest specifying the requested slot specs
    +	 * @return true if the slot request was registered; false if the request is a duplicate
    +	 * @throws SlotManagerException if the slot request failed (e.g. not enough resources left)
    +	 */
    +	public boolean registerSlotRequest(SlotRequest slotRequest) throws SlotManagerException {
    +		checkInit();
     
    -			// record this allocation in bookkeeping
    -			allocationMap.addAllocation(slot.getSlotId(), allocationId);
    -			// remove selected slot from free pool
    -			freeSlots.remove(slot.getSlotId());
    +		if (checkDuplicateRequest(slotRequest.getAllocationId())) {
    +			LOG.debug("Ignoring a duplicate slot request with allocation id {}.", slotRequest.getAllocationId());
     
    -			sendSlotRequest(slot, request);
    +			return false;
     		} else {
    -			LOG.info("Cannot fulfil slot request, try to allocate a new container for it, " +
    -				"AllocationID:{}, JobID:{}", allocationId, request.getJobId());
    -			Preconditions.checkState(rmServices != null,
    -				"Attempted to allocate resources but no ResourceManagerServices set.");
    -			rmServices.allocateResource(request.getResourceProfile());
    -			pendingSlotRequests.put(allocationId, request);
    +			PendingSlotRequest pendingSlotRequest = new PendingSlotRequest(slotRequest);
    +
    +			pendingSlotRequests.put(slotRequest.getAllocationId(), pendingSlotRequest);
    +
    +			try {
    +				internalRequestSlot(pendingSlotRequest);
    +			} catch (ResourceManagerException e) {
    +				// requesting the slot failed --> remove pending slot request
    +				pendingSlotRequests.remove(slotRequest.getAllocationId());
    +
    +				throw new SlotManagerException("Could not fulfill slot request " + slotRequest.getAllocationId() + '.', e);
    +			}
    +
    +			return true;
     		}
    +	}
     
    -		return new RMSlotRequestRegistered(allocationId);
    +	/**
    +	 * Cancels and removes a pending slot request with the given allocation id. If there is no such
    +	 * pending request, then nothing is done.
    +	 *
    +	 * @param allocationId identifying the pending slot request
    +	 * @return True if a pending slot request was found; otherwise false
    +	 */
    +	public boolean unregisterSlotRequest(AllocationID allocationId) {
    +		checkInit();
    +
    +		PendingSlotRequest pendingSlotRequest = pendingSlotRequests.remove(allocationId);
    +
    +		if (null != pendingSlotRequest) {
    +			cancelPendingSlotRequest(pendingSlotRequest);
    +
    +			return true;
    +		} else {
    +			LOG.debug("No pending slot request with allocation id {} found.", allocationId);
    +
    +			return false;
    +		}
     	}
     
     	/**
    -	 * Notifies the SlotManager that a slot is available again after being allocated.
    -	 * @param slotID slot id of available slot
    +	 * Registers a new task manager at the slot manager. This will make the task managers slots
    +	 * known and, thus, available for allocation.
    +	 *
    +	 * @param taskExecutorConnection for the new task manager
    +	 * @param initialSlotReport for the new task manager
     	 */
    -	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");
    +	public void registerTaskManager(final TaskExecutorConnection taskExecutorConnection, SlotReport initialSlotReport) {
    +		checkInit();
    +
    +		// we identify task managers by their instance id
    +		if (!taskManagerRegistrations.containsKey(taskExecutorConnection.getInstanceID())) {
    +			TaskManagerRegistration taskManagerRegistration = new TaskManagerRegistration(taskExecutorConnection);
    +			taskManagerRegistrations.put(taskExecutorConnection.getInstanceID(), taskManagerRegistration);
     		}
    -		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.");
    +
    +		reportSlotStatus(taskExecutorConnection.getInstanceID(), initialSlotReport);
    +	}
    +
    +	/**
    +	 * Unregisters the task manager identified by the given instance id and its associated slots
    +	 * from the slot manager.
    +	 *
    +	 * @param instanceId identifying the task manager to unregister
    +	 * @return True if there existed a registered task manager with the given instance id
    +	 */
    +	public boolean unregisterTaskManager(InstanceID instanceId) {
    +		checkInit();
    +
    +		TaskManagerRegistration taskManagerRegistration = taskManagerRegistrations.remove(instanceId);
    +
    +		if (null != taskManagerRegistration) {
    +			removeSlots(taskManagerRegistration.getSlots());
    +
    +			taskManagerRegistration.cancelTimeout();
    +
    +			return true;
    +		} else {
    +			LOG.debug("There is no task manager registered with instance ID {}. Ignoring this message.", instanceId);
    +
    +			return false;
     		}
    -		handleFreeSlot(freeSlot);
     	}
     
     	/**
    -	 * 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.
    +	 * Reports the current slot allocations for a task manager identified by the given instance id.
     	 *
    -	 * @param originalRequest The original slot request
    -	 * @param slotId          The target SlotID
    +	 * @param instanceId identifying the task manager for which to report the slot status
    +	 * @param slotReport containing the status for all of its slots
    +	 * @return true if the slot status has been updated successfully, otherwise false
     	 */
    -	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());
    -
    -		if (allocationMap.isAllocated(slotId)) {
    -			final AllocationID expectedAllocationId = allocationMap.getAllocationID(slotId);
    -
    -			// 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);
    +	public boolean reportSlotStatus(InstanceID instanceId, SlotReport slotReport) {
    +		checkInit();
    +
    +		TaskManagerRegistration taskManagerRegistration = taskManagerRegistrations.get(instanceId);
    +
    +		if (null != taskManagerRegistration) {
    +			HashSet<SlotID> slotsToRemove = new HashSet<>(taskManagerRegistration.getSlots());
    +			boolean idle = true;
    +
    +			for (SlotStatus slotStatus : slotReport) {
    +				if (slotsToRemove.remove(slotStatus.getSlotID())) {
    +					// slot which was already registered
    +					updateSlot(slotStatus.getSlotID(), slotStatus.getAllocationID());
    +				} else {
    +					// new slot
    +					registerSlot(
    +						instanceId,
    +						slotStatus.getSlotID(),
    +						slotStatus.getAllocationID(),
    +						slotStatus.getResourceProfile(),
    +						taskManagerRegistration.getTaskManagerConnection());
    +				}
    +
    +				TaskManagerSlot slot = slots.get(slotStatus.getSlotID());
    +
    +				idle &= slot.isFree();
    +			}
    +
    +			// remove the slots for which we haven't received a slot status message
    +			removeSlots(slotsToRemove);
    +
    +			if (idle) {
    +				// no slot of this task manager is being used --> register timer to free this resource
    +				registerTaskManagerTimeout(taskManagerRegistration);
    +			}
    +
    +			return true;
    +		} else {
    +			LOG.debug("Received slot report for unknown task manager with instance id {}. Ignoring this report.", instanceId);
    +
    +			return false;
    +		}
    +	}
    +
    +	/**
    +	 * Free the given slot from the given allocation. If the slot is still allocated by the given
    +	 * allocation id, then the slot will be marked as free and will be subject to new slot requests.
    +	 *
    +	 * @param slotId identifying the slot to free
    +	 * @param allocationId with which the slot is presumably allocated
    +	 */
    +	public void freeSlot(SlotID slotId, AllocationID allocationId) {
    +		checkInit();
    +
    +		TaskManagerSlot slot = slots.get(slotId);
    +
    +		if (null != slot) {
    +			if (slot.isAllocated()) {
    +				if (Objects.equals(allocationId, slot.getAllocationId())) {
    +					// free the slot
    +					slot.setAllocationId(null);
    +					fulfilledSlotRequests.remove(allocationId);
    +
    +					if (slot.isFree()) {
    +						handleFreeSlot(slot);
    +					}
    +
    +					TaskManagerRegistration taskManagerRegistration = taskManagerRegistrations.get(slot.getInstanceId());
    +
    +					if (null != taskManagerRegistration && !anySlotUsed(taskManagerRegistration.getSlots())) {
    +						registerTaskManagerTimeout(taskManagerRegistration);
    +					}
    +				} else {
    +					LOG.debug("Received request to free slot {} with expected allocation id {}, " +
    +						"but actual allocation id {} differs. Ignoring the request.", slotId, allocationId, slot.getAllocationId());
    +				}
     			} else {
    -				LOG.error("Slot request failed for slot {} with allocation id {}:" +
    -						" Allocation id did not match the expected allocation id {}.",
    -					slotId, originalAllocationId, expectedAllocationId);
    +				LOG.debug("Slot {} has not been allocated.", allocationId);
     			}
     		} else {
    -			LOG.error("Slot request failed for slot {} with allocation id {}: " +
    -					"Slot was not previously registered.",
    -				slotId, originalAllocationId);
    +			LOG.debug("Trying to free a slot {} which has not been registered. Ignoring this message.", slotId);
     		}
     	}
     
    +	// ---------------------------------------------------------------------------------------------
    +	// Behaviour methods
    +	// ---------------------------------------------------------------------------------------------
    +
     	/**
    -	 * Registers a TaskExecutor
    -	 * @param resourceID TaskExecutor's ResourceID
    -	 * @param registration TaskExecutor's registration
    -	 * @param slotReport TaskExecutor's free and allocated slots
    +	 * Finds a matching slot request for a given resource profile. If there is no such request,
    +	 * the method returns null.
    +	 *
    +	 * Note: If you want to change the behaviour of the slot manager wrt slot allocation and
    +	 * request fulfillment, then you should override this method.
    +	 *
    +	 * @param slotResourceProfile defining the resources of an available slot
    +	 * @return A matching slot request which can be deployed in a slot with the given resource
    +	 * profile. Null if there is no such slot request pending.
     	 */
    -	public void registerTaskExecutor(
    -			ResourceID resourceID,
    -			TaskExecutorRegistration registration,
    -			SlotReport slotReport) {
    +	protected PendingSlotRequest findMatchingRequest(ResourceProfile slotResourceProfile) {
     
    -		if (taskManagers.get(resourceID) != null) {
    -			notifyTaskManagerFailure(resourceID);
    +		for (PendingSlotRequest pendingSlotRequest : pendingSlotRequests.values()) {
    +			if (!pendingSlotRequest.isAssigned() && slotResourceProfile.isMatching(pendingSlotRequest.getResourceProfile())) {
    +				return pendingSlotRequest;
    +			}
     		}
     
    -		this.taskManagers.put(resourceID, registration);
    +		return null;
    +	}
    +
    +	/**
    +	 * Finds a matching slot for a given resource profile. A matching slot has at least as many
    +	 * resources available as the given resource profile. If there is no such slot available, then
    +	 * the method returns null.
    +	 *
    +	 * Note: If you want to change the behaviour of the slot manager wrt slot allocation and
    +	 * request fulfillment, then you should override this method.
    +	 *
    +	 * @param requestResourceProfile specifying the resource requirements for the a slot request
    +	 * @return A matching slot which fulfills the given resource profile. Null if there is no such
    +	 * slot available.
    +	 */
    +	protected TaskManagerSlot findMatchingSlot(ResourceProfile requestResourceProfile) {
    +		Iterator<Map.Entry<SlotID, TaskManagerSlot>> iterator = freeSlots.entrySet().iterator();
    +
    +		while (iterator.hasNext()) {
    +			TaskManagerSlot taskManagerSlot = iterator.next().getValue();
     
    -		for (SlotStatus slotStatus : slotReport.getSlotsStatus()) {
    -			final SlotID slotId = slotStatus.getSlotID();
    +			// sanity check
    +			Preconditions.checkState(taskManagerSlot.isFree());
     
    -			final TaskExecutorRegistration taskExecutorRegistration = taskManagers.get(slotId.getResourceID());
    -			if (taskExecutorRegistration == null) {
    -				LOG.info("Received SlotStatus but ResourceID {} is unknown to the SlotManager",
    -					slotId.getResourceID());
    -				return;
    +			if (taskManagerSlot.getResourceProfile().isMatching(requestResourceProfile)) {
    +				iterator.remove();
    +				return taskManagerSlot;
     			}
    +		}
     
    -			final ResourceSlot slot = new ResourceSlot(slotId, slotStatus.getProfiler(), taskExecutorRegistration);
    +		return null;
    +	}
     
    -			registerNewSlot(slot);
    -			LOG.info("New slot appeared, SlotID:{}, AllocationID:{}", slotId, slotStatus.getAllocationID());
    +	// ---------------------------------------------------------------------------------------------
    +	// Internal slot operations
    +	// ---------------------------------------------------------------------------------------------
     
    -			if (slotStatus.getAllocationID() != null) {
    -				// slot in use, record this in bookkeeping
    -				allocationMap.addAllocation(slotId, slotStatus.getAllocationID());
    -			} else {
    +	/**
    +	 * Registers a slot for the given task manager at the slot manager. The task manager is
    +	 * identified by the given instance id and the slot is identified by the given slot id. The
    +	 * given resource profile defines the available resources for the slot. The task manager
    +	 * connection can be used to communicate with the task manager.
    +	 *
    +	 * @param instanceId identifying the task manager on which the slot lives
    +	 * @param slotId identifying the slot on the task manager
    +	 * @param allocationId which is currently deployed in the slot
    +	 * @param resourceProfile of the slot
    +	 * @param taskManagerConnection to communicate with the remote task manager
    +	 */
    +	private void registerSlot(
    --- End diff --
    
    True indeed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3394: [FLINK-5810] [flip6] Introduce a hardened slot man...

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3394#discussion_r113502203
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManager.java ---
    @@ -21,519 +21,897 @@
     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.ResourceSlot;
    +import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
    +import org.apache.flink.runtime.clusterframework.types.TaskManagerSlot;
     import org.apache.flink.runtime.clusterframework.types.SlotID;
     import org.apache.flink.runtime.concurrent.BiFunction;
    +import org.apache.flink.runtime.concurrent.CompletableFuture;
     import org.apache.flink.runtime.concurrent.Future;
    -import org.apache.flink.runtime.resourcemanager.ResourceManagerServices;
    +import org.apache.flink.runtime.concurrent.ScheduledExecutor;
    +import org.apache.flink.runtime.concurrent.impl.FlinkCompletableFuture;
    +import org.apache.flink.runtime.instance.InstanceID;
    +import org.apache.flink.runtime.messages.Acknowledge;
     import org.apache.flink.runtime.resourcemanager.SlotRequest;
    -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.resourcemanager.exceptions.ResourceManagerException;
    +import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
     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.runtime.taskexecutor.exceptions.SlotAllocationException;
    +import org.apache.flink.runtime.taskexecutor.exceptions.SlotOccupiedException;
     import org.apache.flink.util.Preconditions;
     import org.slf4j.Logger;
     import org.slf4j.LoggerFactory;
     
     import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.Iterator;
     import java.util.LinkedHashMap;
     import java.util.Map;
    -
    -import static org.apache.flink.util.Preconditions.checkNotNull;
    +import java.util.Objects;
    +import java.util.UUID;
    +import java.util.concurrent.CancellationException;
    +import java.util.concurrent.Executor;
    +import java.util.concurrent.ScheduledFuture;
    +import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.TimeoutException;
     
     /**
    - * 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>.
    + * The slot manager is responsible for maintaining a view on all registered task manager slots,
    + * their allocation and all pending slot requests. Whenever a new slot is registered or and
    + * allocated slot is freed, then it tries to fulfill another pending slot request. Whenever there
    + * are not enough slots available the slot manager will notify the resource manager about it via
    + * {@link ResourceManagerActions#allocateResource(ResourceProfile)}.
    + *
    + * In order to free resources and avoid resource leaks, idling task managers (task managers whose
    + * slots are currently not used) and not fulfilled pending slot requests time out triggering their
    + * release and failure, respectively.
      */
    -public abstract class SlotManager {
    +public class SlotManager implements AutoCloseable {
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(SlotManager.class);
    +
    +	/** Scheduled executor for timeouts */
    +	private final ScheduledExecutor scheduledExecutor;
    +
    +	/** Timeout for slot requests to the task manager */
    +	private final Time taskManagerRequestTimeout;
    +
    +	/** Timeout after which an allocation is discarded */
    +	private final Time slotRequestTimeout;
    +
    +	/** Timeout after which an unused TaskManager is released */
    +	private final Time taskManagerTimeout;
    +
    +	/** Map for all registered slots */
    +	private final HashMap<SlotID, TaskManagerSlot> slots;
     
    -	protected final Logger LOG = LoggerFactory.getLogger(getClass());
    +	/** Index of all currently free slots */
    +	private final LinkedHashMap<SlotID, TaskManagerSlot> freeSlots;
     
    -	/** The Resource allocation provider */
    -	protected final ResourceManagerServices rmServices;
    +	/** All currently registered task managers */
    +	private final HashMap<InstanceID, TaskManagerRegistration> taskManagerRegistrations;
     
    -	/** All registered task managers with ResourceID and gateway. */
    -	private final Map<ResourceID, TaskExecutorRegistration> taskManagers;
    +	/** Map of fulfilled and active allocations for request deduplication purposes */
    +	private final HashMap<AllocationID, SlotID> fulfilledSlotRequests;
     
    -	/** All registered slots, including free and allocated slots */
    -	private final Map<ResourceID, Map<SlotID, ResourceSlot>> registeredSlots;
    +	/** Map of pending/unfulfilled slot allocation requests */
    +	private final HashMap<AllocationID, PendingSlotRequest> pendingSlotRequests;
     
    -	/** All pending slot requests, waiting available slots to fulfil */
    -	private final Map<AllocationID, SlotRequest> pendingSlotRequests;
    +	/** Leader id of the containing component */
    +	private UUID leaderId;
     
    -	/** All free slots that can be used to be allocated */
    -	private final Map<SlotID, ResourceSlot> freeSlots;
    +	/** Executor for future callbacks which have to be "synchronized" */
    +	private Executor mainThreadExecutor;
     
    -	/** All allocations, we can lookup allocations either by SlotID or AllocationID */
    -	private final AllocationMap allocationMap;
    +	/** Callbacks for resource (de-)allocations */
    +	private ResourceManagerActions resourceManagerActions;
     
    -	private final Time timeout;
    +	/** True iff the component has been started */
    +	private boolean started;
     
    -	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.taskManagers = new HashMap<>();
    -		this.timeout = Time.seconds(10);
    +	public SlotManager(
    +			ScheduledExecutor scheduledExecutor,
    +			Time taskManagerRequestTimeout,
    +			Time slotRequestTimeout,
    +			Time taskManagerTimeout) {
    +		this.scheduledExecutor = Preconditions.checkNotNull(scheduledExecutor);
    +		this.taskManagerRequestTimeout = Preconditions.checkNotNull(taskManagerRequestTimeout);
    +		this.slotRequestTimeout = Preconditions.checkNotNull(slotRequestTimeout);
    +		this.taskManagerTimeout = Preconditions.checkNotNull(taskManagerTimeout);
    +
    +		slots = new HashMap<>(16);
    +		freeSlots = new LinkedHashMap<>(16);
    +		taskManagerRegistrations = new HashMap<>(4);
    +		fulfilledSlotRequests = new HashMap<>(16);
    +		pendingSlotRequests = new HashMap<>(16);
    +
    +		leaderId = null;
    +		resourceManagerActions = null;
    +		started = false;
     	}
     
    -	// ------------------------------------------------------------------------
    -	//  slot managements
    -	// ------------------------------------------------------------------------
    +	// ---------------------------------------------------------------------------------------------
    +	// Component lifecycle methods
    +	// ---------------------------------------------------------------------------------------------
     
     	/**
    -	 * 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).
    +	 * Starts the slot manager with the given leader id and resource manager actions.
     	 *
    -	 * @param request The detailed request of the slot
    -	 * @return RMSlotRequestRegistered The confirmation message to be send to the caller
    +	 * @param newLeaderId to use for communication with the task managers
    +	 * @param newResourceManagerActions to use for resource (de-)allocations
    +	 */
    +	public void start(UUID newLeaderId, Executor newMainThreadExecutor, ResourceManagerActions newResourceManagerActions) {
    +		leaderId = Preconditions.checkNotNull(newLeaderId);
    +		mainThreadExecutor = Preconditions.checkNotNull(newMainThreadExecutor);
    +		resourceManagerActions = Preconditions.checkNotNull(newResourceManagerActions);
    +
    +		started = true;
    +	}
    +
    +	/**
    +	 * Suspends the component. This clears the internal state of the slot manager.
     	 */
    -	public RMSlotRequestRegistered requestSlot(final SlotRequest request) {
    -		final AllocationID allocationId = request.getAllocationId();
    -		if (isRequestDuplicated(request)) {
    -			LOG.warn("Duplicated slot request, AllocationID:{}", allocationId);
    -			return new RMSlotRequestRegistered(allocationId);
    +	public void suspend() {
    +		for (PendingSlotRequest pendingSlotRequest : pendingSlotRequests.values()) {
    +			cancelPendingSlotRequest(pendingSlotRequest);
     		}
     
    -		// 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());
    +		pendingSlotRequests.clear();
    +
    +		HashSet<InstanceID> registeredTaskManagers = new HashSet<>(taskManagerRegistrations.keySet());
    +
    +		for (InstanceID registeredTaskManager : registeredTaskManagers) {
    +			unregisterTaskManager(registeredTaskManager);
    +		}
    +
    +		leaderId = null;
    +		resourceManagerActions = null;
    +		started = false;
    +	}
    +
    +	/**
    +	 * Closes the slot manager.
    +	 *
    +	 * @throws Exception if the close operation fails
    +	 */
    +	@Override
    +	public void close() throws Exception {
    +		suspend();
    +	}
    +
    +	// ---------------------------------------------------------------------------------------------
    +	// Public API
    +	// ---------------------------------------------------------------------------------------------
    +
    +	/**
    +	 * Requests a slot with the respective resource profile.
    +	 *
    +	 * @param slotRequest specifying the requested slot specs
    +	 * @return true if the slot request was registered; false if the request is a duplicate
    +	 * @throws SlotManagerException if the slot request failed (e.g. not enough resources left)
    +	 */
    +	public boolean registerSlotRequest(SlotRequest slotRequest) throws SlotManagerException {
    +		checkInit();
     
    -			// record this allocation in bookkeeping
    -			allocationMap.addAllocation(slot.getSlotId(), allocationId);
    -			// remove selected slot from free pool
    -			freeSlots.remove(slot.getSlotId());
    +		if (checkDuplicateRequest(slotRequest.getAllocationId())) {
    +			LOG.debug("Ignoring a duplicate slot request with allocation id {}.", slotRequest.getAllocationId());
     
    -			sendSlotRequest(slot, request);
    +			return false;
     		} else {
    -			LOG.info("Cannot fulfil slot request, try to allocate a new container for it, " +
    -				"AllocationID:{}, JobID:{}", allocationId, request.getJobId());
    -			Preconditions.checkState(rmServices != null,
    -				"Attempted to allocate resources but no ResourceManagerServices set.");
    -			rmServices.allocateResource(request.getResourceProfile());
    -			pendingSlotRequests.put(allocationId, request);
    +			PendingSlotRequest pendingSlotRequest = new PendingSlotRequest(slotRequest);
    +
    +			pendingSlotRequests.put(slotRequest.getAllocationId(), pendingSlotRequest);
    +
    +			try {
    +				internalRequestSlot(pendingSlotRequest);
    +			} catch (ResourceManagerException e) {
    +				// requesting the slot failed --> remove pending slot request
    +				pendingSlotRequests.remove(slotRequest.getAllocationId());
    +
    +				throw new SlotManagerException("Could not fulfill slot request " + slotRequest.getAllocationId() + '.', e);
    +			}
    +
    +			return true;
     		}
    +	}
     
    -		return new RMSlotRequestRegistered(allocationId);
    +	/**
    +	 * Cancels and removes a pending slot request with the given allocation id. If there is no such
    +	 * pending request, then nothing is done.
    +	 *
    +	 * @param allocationId identifying the pending slot request
    +	 * @return True if a pending slot request was found; otherwise false
    +	 */
    +	public boolean unregisterSlotRequest(AllocationID allocationId) {
    +		checkInit();
    +
    +		PendingSlotRequest pendingSlotRequest = pendingSlotRequests.remove(allocationId);
    +
    +		if (null != pendingSlotRequest) {
    +			cancelPendingSlotRequest(pendingSlotRequest);
    +
    +			return true;
    +		} else {
    +			LOG.debug("No pending slot request with allocation id {} found.", allocationId);
    +
    +			return false;
    +		}
     	}
     
     	/**
    -	 * Notifies the SlotManager that a slot is available again after being allocated.
    -	 * @param slotID slot id of available slot
    +	 * Registers a new task manager at the slot manager. This will make the task managers slots
    +	 * known and, thus, available for allocation.
    +	 *
    +	 * @param taskExecutorConnection for the new task manager
    +	 * @param initialSlotReport for the new task manager
     	 */
    -	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");
    +	public void registerTaskManager(final TaskExecutorConnection taskExecutorConnection, SlotReport initialSlotReport) {
    +		checkInit();
    +
    +		// we identify task managers by their instance id
    +		if (!taskManagerRegistrations.containsKey(taskExecutorConnection.getInstanceID())) {
    +			TaskManagerRegistration taskManagerRegistration = new TaskManagerRegistration(taskExecutorConnection);
    +			taskManagerRegistrations.put(taskExecutorConnection.getInstanceID(), taskManagerRegistration);
     		}
    -		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.");
    +
    +		reportSlotStatus(taskExecutorConnection.getInstanceID(), initialSlotReport);
    +	}
    +
    +	/**
    +	 * Unregisters the task manager identified by the given instance id and its associated slots
    +	 * from the slot manager.
    +	 *
    +	 * @param instanceId identifying the task manager to unregister
    +	 * @return True if there existed a registered task manager with the given instance id
    +	 */
    +	public boolean unregisterTaskManager(InstanceID instanceId) {
    +		checkInit();
    +
    +		TaskManagerRegistration taskManagerRegistration = taskManagerRegistrations.remove(instanceId);
    +
    +		if (null != taskManagerRegistration) {
    +			removeSlots(taskManagerRegistration.getSlots());
    +
    +			taskManagerRegistration.cancelTimeout();
    +
    +			return true;
    +		} else {
    +			LOG.debug("There is no task manager registered with instance ID {}. Ignoring this message.", instanceId);
    +
    +			return false;
     		}
    -		handleFreeSlot(freeSlot);
     	}
     
     	/**
    -	 * 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.
    +	 * Reports the current slot allocations for a task manager identified by the given instance id.
     	 *
    -	 * @param originalRequest The original slot request
    -	 * @param slotId          The target SlotID
    +	 * @param instanceId identifying the task manager for which to report the slot status
    +	 * @param slotReport containing the status for all of its slots
    +	 * @return true if the slot status has been updated successfully, otherwise false
     	 */
    -	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());
    -
    -		if (allocationMap.isAllocated(slotId)) {
    -			final AllocationID expectedAllocationId = allocationMap.getAllocationID(slotId);
    -
    -			// 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);
    +	public boolean reportSlotStatus(InstanceID instanceId, SlotReport slotReport) {
    +		checkInit();
    +
    +		TaskManagerRegistration taskManagerRegistration = taskManagerRegistrations.get(instanceId);
    +
    +		if (null != taskManagerRegistration) {
    +			HashSet<SlotID> slotsToRemove = new HashSet<>(taskManagerRegistration.getSlots());
    +			boolean idle = true;
    +
    +			for (SlotStatus slotStatus : slotReport) {
    +				if (slotsToRemove.remove(slotStatus.getSlotID())) {
    +					// slot which was already registered
    +					updateSlot(slotStatus.getSlotID(), slotStatus.getAllocationID());
    +				} else {
    +					// new slot
    +					registerSlot(
    +						instanceId,
    +						slotStatus.getSlotID(),
    +						slotStatus.getAllocationID(),
    +						slotStatus.getResourceProfile(),
    +						taskManagerRegistration.getTaskManagerConnection());
    +				}
    +
    +				TaskManagerSlot slot = slots.get(slotStatus.getSlotID());
    +
    +				idle &= slot.isFree();
    +			}
    +
    +			// remove the slots for which we haven't received a slot status message
    +			removeSlots(slotsToRemove);
    +
    +			if (idle) {
    --- End diff --
    
    Yes you're right. The timeout logic is faulty. I guess it's better to follow your proposal wrt to the timeout logic (having a periodic task checking all task manager and pending slot requests). That way we wouldn't have to create these timeout identifiers, too.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3394: [FLINK-5810] [flip6] Introduce a hardened slot man...

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3394#discussion_r113483755
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManager.java ---
    @@ -21,519 +21,897 @@
     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.ResourceSlot;
    +import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
    +import org.apache.flink.runtime.clusterframework.types.TaskManagerSlot;
     import org.apache.flink.runtime.clusterframework.types.SlotID;
     import org.apache.flink.runtime.concurrent.BiFunction;
    +import org.apache.flink.runtime.concurrent.CompletableFuture;
     import org.apache.flink.runtime.concurrent.Future;
    -import org.apache.flink.runtime.resourcemanager.ResourceManagerServices;
    +import org.apache.flink.runtime.concurrent.ScheduledExecutor;
    +import org.apache.flink.runtime.concurrent.impl.FlinkCompletableFuture;
    +import org.apache.flink.runtime.instance.InstanceID;
    +import org.apache.flink.runtime.messages.Acknowledge;
     import org.apache.flink.runtime.resourcemanager.SlotRequest;
    -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.resourcemanager.exceptions.ResourceManagerException;
    +import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
     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.runtime.taskexecutor.exceptions.SlotAllocationException;
    +import org.apache.flink.runtime.taskexecutor.exceptions.SlotOccupiedException;
     import org.apache.flink.util.Preconditions;
     import org.slf4j.Logger;
     import org.slf4j.LoggerFactory;
     
     import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.Iterator;
     import java.util.LinkedHashMap;
     import java.util.Map;
    -
    -import static org.apache.flink.util.Preconditions.checkNotNull;
    +import java.util.Objects;
    +import java.util.UUID;
    +import java.util.concurrent.CancellationException;
    +import java.util.concurrent.Executor;
    +import java.util.concurrent.ScheduledFuture;
    +import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.TimeoutException;
     
     /**
    - * 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>.
    + * The slot manager is responsible for maintaining a view on all registered task manager slots,
    + * their allocation and all pending slot requests. Whenever a new slot is registered or and
    + * allocated slot is freed, then it tries to fulfill another pending slot request. Whenever there
    + * are not enough slots available the slot manager will notify the resource manager about it via
    + * {@link ResourceManagerActions#allocateResource(ResourceProfile)}.
    + *
    + * In order to free resources and avoid resource leaks, idling task managers (task managers whose
    + * slots are currently not used) and not fulfilled pending slot requests time out triggering their
    + * release and failure, respectively.
      */
    -public abstract class SlotManager {
    +public class SlotManager implements AutoCloseable {
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(SlotManager.class);
    +
    +	/** Scheduled executor for timeouts */
    +	private final ScheduledExecutor scheduledExecutor;
    +
    +	/** Timeout for slot requests to the task manager */
    +	private final Time taskManagerRequestTimeout;
    +
    +	/** Timeout after which an allocation is discarded */
    +	private final Time slotRequestTimeout;
    +
    +	/** Timeout after which an unused TaskManager is released */
    +	private final Time taskManagerTimeout;
    +
    +	/** Map for all registered slots */
    +	private final HashMap<SlotID, TaskManagerSlot> slots;
     
    -	protected final Logger LOG = LoggerFactory.getLogger(getClass());
    +	/** Index of all currently free slots */
    +	private final LinkedHashMap<SlotID, TaskManagerSlot> freeSlots;
     
    -	/** The Resource allocation provider */
    -	protected final ResourceManagerServices rmServices;
    +	/** All currently registered task managers */
    +	private final HashMap<InstanceID, TaskManagerRegistration> taskManagerRegistrations;
     
    -	/** All registered task managers with ResourceID and gateway. */
    -	private final Map<ResourceID, TaskExecutorRegistration> taskManagers;
    +	/** Map of fulfilled and active allocations for request deduplication purposes */
    +	private final HashMap<AllocationID, SlotID> fulfilledSlotRequests;
     
    -	/** All registered slots, including free and allocated slots */
    -	private final Map<ResourceID, Map<SlotID, ResourceSlot>> registeredSlots;
    +	/** Map of pending/unfulfilled slot allocation requests */
    +	private final HashMap<AllocationID, PendingSlotRequest> pendingSlotRequests;
     
    -	/** All pending slot requests, waiting available slots to fulfil */
    -	private final Map<AllocationID, SlotRequest> pendingSlotRequests;
    +	/** Leader id of the containing component */
    +	private UUID leaderId;
     
    -	/** All free slots that can be used to be allocated */
    -	private final Map<SlotID, ResourceSlot> freeSlots;
    +	/** Executor for future callbacks which have to be "synchronized" */
    +	private Executor mainThreadExecutor;
     
    -	/** All allocations, we can lookup allocations either by SlotID or AllocationID */
    -	private final AllocationMap allocationMap;
    +	/** Callbacks for resource (de-)allocations */
    +	private ResourceManagerActions resourceManagerActions;
     
    -	private final Time timeout;
    +	/** True iff the component has been started */
    +	private boolean started;
     
    -	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.taskManagers = new HashMap<>();
    -		this.timeout = Time.seconds(10);
    +	public SlotManager(
    +			ScheduledExecutor scheduledExecutor,
    +			Time taskManagerRequestTimeout,
    +			Time slotRequestTimeout,
    +			Time taskManagerTimeout) {
    +		this.scheduledExecutor = Preconditions.checkNotNull(scheduledExecutor);
    +		this.taskManagerRequestTimeout = Preconditions.checkNotNull(taskManagerRequestTimeout);
    +		this.slotRequestTimeout = Preconditions.checkNotNull(slotRequestTimeout);
    +		this.taskManagerTimeout = Preconditions.checkNotNull(taskManagerTimeout);
    +
    +		slots = new HashMap<>(16);
    +		freeSlots = new LinkedHashMap<>(16);
    +		taskManagerRegistrations = new HashMap<>(4);
    +		fulfilledSlotRequests = new HashMap<>(16);
    +		pendingSlotRequests = new HashMap<>(16);
    +
    +		leaderId = null;
    +		resourceManagerActions = null;
    +		started = false;
     	}
     
    -	// ------------------------------------------------------------------------
    -	//  slot managements
    -	// ------------------------------------------------------------------------
    +	// ---------------------------------------------------------------------------------------------
    +	// Component lifecycle methods
    +	// ---------------------------------------------------------------------------------------------
     
     	/**
    -	 * 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).
    +	 * Starts the slot manager with the given leader id and resource manager actions.
     	 *
    -	 * @param request The detailed request of the slot
    -	 * @return RMSlotRequestRegistered The confirmation message to be send to the caller
    +	 * @param newLeaderId to use for communication with the task managers
    +	 * @param newResourceManagerActions to use for resource (de-)allocations
    +	 */
    +	public void start(UUID newLeaderId, Executor newMainThreadExecutor, ResourceManagerActions newResourceManagerActions) {
    +		leaderId = Preconditions.checkNotNull(newLeaderId);
    +		mainThreadExecutor = Preconditions.checkNotNull(newMainThreadExecutor);
    +		resourceManagerActions = Preconditions.checkNotNull(newResourceManagerActions);
    +
    +		started = true;
    +	}
    +
    +	/**
    +	 * Suspends the component. This clears the internal state of the slot manager.
     	 */
    -	public RMSlotRequestRegistered requestSlot(final SlotRequest request) {
    -		final AllocationID allocationId = request.getAllocationId();
    -		if (isRequestDuplicated(request)) {
    -			LOG.warn("Duplicated slot request, AllocationID:{}", allocationId);
    -			return new RMSlotRequestRegistered(allocationId);
    +	public void suspend() {
    +		for (PendingSlotRequest pendingSlotRequest : pendingSlotRequests.values()) {
    +			cancelPendingSlotRequest(pendingSlotRequest);
     		}
     
    -		// 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());
    +		pendingSlotRequests.clear();
    +
    +		HashSet<InstanceID> registeredTaskManagers = new HashSet<>(taskManagerRegistrations.keySet());
    --- End diff --
    
    True. Good point.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3394: [FLINK-5810] [flip6] Introduce a hardened slot man...

Posted by StephanEwen <gi...@git.apache.org>.
Github user StephanEwen commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3394#discussion_r106458045
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/TaskManagerSlot.java ---
    @@ -18,15 +18,17 @@
     
     package org.apache.flink.runtime.clusterframework.types;
     
    -import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorRegistration;
    +import org.apache.flink.runtime.instance.InstanceID;
    +import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
    +import org.apache.flink.runtime.resourcemanager.slotmanager.PendingSlotRequest;
     
     import static org.apache.flink.util.Preconditions.checkNotNull;
     
     /**
      * A ResourceSlot represents a slot located in TaskManager from ResourceManager's view. It has a unique
    --- End diff --
    
    `ResourceSlot` -->`TaskManagerSlot`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3394: [FLINK-5810] [flip6] Introduce a hardened slot man...

Posted by StephanEwen <gi...@git.apache.org>.
Github user StephanEwen commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3394#discussion_r106507099
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManager.java ---
    @@ -21,519 +21,897 @@
     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.ResourceSlot;
    +import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
    +import org.apache.flink.runtime.clusterframework.types.TaskManagerSlot;
     import org.apache.flink.runtime.clusterframework.types.SlotID;
     import org.apache.flink.runtime.concurrent.BiFunction;
    +import org.apache.flink.runtime.concurrent.CompletableFuture;
     import org.apache.flink.runtime.concurrent.Future;
    -import org.apache.flink.runtime.resourcemanager.ResourceManagerServices;
    +import org.apache.flink.runtime.concurrent.ScheduledExecutor;
    +import org.apache.flink.runtime.concurrent.impl.FlinkCompletableFuture;
    +import org.apache.flink.runtime.instance.InstanceID;
    +import org.apache.flink.runtime.messages.Acknowledge;
     import org.apache.flink.runtime.resourcemanager.SlotRequest;
    -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.resourcemanager.exceptions.ResourceManagerException;
    +import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
     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.runtime.taskexecutor.exceptions.SlotAllocationException;
    +import org.apache.flink.runtime.taskexecutor.exceptions.SlotOccupiedException;
     import org.apache.flink.util.Preconditions;
     import org.slf4j.Logger;
     import org.slf4j.LoggerFactory;
     
     import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.Iterator;
     import java.util.LinkedHashMap;
     import java.util.Map;
    -
    -import static org.apache.flink.util.Preconditions.checkNotNull;
    +import java.util.Objects;
    +import java.util.UUID;
    +import java.util.concurrent.CancellationException;
    +import java.util.concurrent.Executor;
    +import java.util.concurrent.ScheduledFuture;
    +import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.TimeoutException;
     
     /**
    - * 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>.
    + * The slot manager is responsible for maintaining a view on all registered task manager slots,
    + * their allocation and all pending slot requests. Whenever a new slot is registered or and
    + * allocated slot is freed, then it tries to fulfill another pending slot request. Whenever there
    + * are not enough slots available the slot manager will notify the resource manager about it via
    + * {@link ResourceManagerActions#allocateResource(ResourceProfile)}.
    + *
    + * In order to free resources and avoid resource leaks, idling task managers (task managers whose
    + * slots are currently not used) and not fulfilled pending slot requests time out triggering their
    + * release and failure, respectively.
      */
    -public abstract class SlotManager {
    +public class SlotManager implements AutoCloseable {
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(SlotManager.class);
    +
    +	/** Scheduled executor for timeouts */
    +	private final ScheduledExecutor scheduledExecutor;
    +
    +	/** Timeout for slot requests to the task manager */
    +	private final Time taskManagerRequestTimeout;
    +
    +	/** Timeout after which an allocation is discarded */
    +	private final Time slotRequestTimeout;
    +
    +	/** Timeout after which an unused TaskManager is released */
    +	private final Time taskManagerTimeout;
    +
    +	/** Map for all registered slots */
    +	private final HashMap<SlotID, TaskManagerSlot> slots;
     
    -	protected final Logger LOG = LoggerFactory.getLogger(getClass());
    +	/** Index of all currently free slots */
    +	private final LinkedHashMap<SlotID, TaskManagerSlot> freeSlots;
     
    -	/** The Resource allocation provider */
    -	protected final ResourceManagerServices rmServices;
    +	/** All currently registered task managers */
    +	private final HashMap<InstanceID, TaskManagerRegistration> taskManagerRegistrations;
     
    -	/** All registered task managers with ResourceID and gateway. */
    -	private final Map<ResourceID, TaskExecutorRegistration> taskManagers;
    +	/** Map of fulfilled and active allocations for request deduplication purposes */
    +	private final HashMap<AllocationID, SlotID> fulfilledSlotRequests;
     
    -	/** All registered slots, including free and allocated slots */
    -	private final Map<ResourceID, Map<SlotID, ResourceSlot>> registeredSlots;
    +	/** Map of pending/unfulfilled slot allocation requests */
    +	private final HashMap<AllocationID, PendingSlotRequest> pendingSlotRequests;
     
    -	/** All pending slot requests, waiting available slots to fulfil */
    -	private final Map<AllocationID, SlotRequest> pendingSlotRequests;
    +	/** Leader id of the containing component */
    +	private UUID leaderId;
     
    -	/** All free slots that can be used to be allocated */
    -	private final Map<SlotID, ResourceSlot> freeSlots;
    +	/** Executor for future callbacks which have to be "synchronized" */
    +	private Executor mainThreadExecutor;
     
    -	/** All allocations, we can lookup allocations either by SlotID or AllocationID */
    -	private final AllocationMap allocationMap;
    +	/** Callbacks for resource (de-)allocations */
    +	private ResourceManagerActions resourceManagerActions;
     
    -	private final Time timeout;
    +	/** True iff the component has been started */
    +	private boolean started;
     
    -	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.taskManagers = new HashMap<>();
    -		this.timeout = Time.seconds(10);
    +	public SlotManager(
    +			ScheduledExecutor scheduledExecutor,
    +			Time taskManagerRequestTimeout,
    +			Time slotRequestTimeout,
    +			Time taskManagerTimeout) {
    +		this.scheduledExecutor = Preconditions.checkNotNull(scheduledExecutor);
    +		this.taskManagerRequestTimeout = Preconditions.checkNotNull(taskManagerRequestTimeout);
    +		this.slotRequestTimeout = Preconditions.checkNotNull(slotRequestTimeout);
    +		this.taskManagerTimeout = Preconditions.checkNotNull(taskManagerTimeout);
    +
    +		slots = new HashMap<>(16);
    +		freeSlots = new LinkedHashMap<>(16);
    +		taskManagerRegistrations = new HashMap<>(4);
    +		fulfilledSlotRequests = new HashMap<>(16);
    +		pendingSlotRequests = new HashMap<>(16);
    +
    +		leaderId = null;
    +		resourceManagerActions = null;
    +		started = false;
     	}
     
    -	// ------------------------------------------------------------------------
    -	//  slot managements
    -	// ------------------------------------------------------------------------
    +	// ---------------------------------------------------------------------------------------------
    +	// Component lifecycle methods
    +	// ---------------------------------------------------------------------------------------------
     
     	/**
    -	 * 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).
    +	 * Starts the slot manager with the given leader id and resource manager actions.
     	 *
    -	 * @param request The detailed request of the slot
    -	 * @return RMSlotRequestRegistered The confirmation message to be send to the caller
    +	 * @param newLeaderId to use for communication with the task managers
    +	 * @param newResourceManagerActions to use for resource (de-)allocations
    +	 */
    +	public void start(UUID newLeaderId, Executor newMainThreadExecutor, ResourceManagerActions newResourceManagerActions) {
    +		leaderId = Preconditions.checkNotNull(newLeaderId);
    +		mainThreadExecutor = Preconditions.checkNotNull(newMainThreadExecutor);
    +		resourceManagerActions = Preconditions.checkNotNull(newResourceManagerActions);
    +
    +		started = true;
    +	}
    +
    +	/**
    +	 * Suspends the component. This clears the internal state of the slot manager.
     	 */
    -	public RMSlotRequestRegistered requestSlot(final SlotRequest request) {
    -		final AllocationID allocationId = request.getAllocationId();
    -		if (isRequestDuplicated(request)) {
    -			LOG.warn("Duplicated slot request, AllocationID:{}", allocationId);
    -			return new RMSlotRequestRegistered(allocationId);
    +	public void suspend() {
    +		for (PendingSlotRequest pendingSlotRequest : pendingSlotRequests.values()) {
    +			cancelPendingSlotRequest(pendingSlotRequest);
     		}
     
    -		// 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());
    +		pendingSlotRequests.clear();
    +
    +		HashSet<InstanceID> registeredTaskManagers = new HashSet<>(taskManagerRegistrations.keySet());
    --- End diff --
    
    If the copy of the collection is just for concurrent iteration, I would make it an `ArrayList`. Builds faster and iterates faster.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3394: [FLINK-5810] [flip6] Introduce a hardened slot man...

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3394#discussion_r114303364
  
    --- Diff: flink-runtime/src/test/resources/log4j-test.properties ---
    @@ -16,7 +16,7 @@
     # limitations under the License.
     ################################################################################
     
    -log4j.rootLogger=OFF, console
    +log4j.rootLogger=DEBUG, console
    --- End diff --
    
    Thanks for pointing it out @KurtYoung. Will fix it.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3394: [FLINK-5810] [flip6] Introduce a hardened slot man...

Posted by StephanEwen <gi...@git.apache.org>.
Github user StephanEwen commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3394#discussion_r106511170
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/TaskManagerRegistration.java ---
    @@ -0,0 +1,89 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.SlotID;
    +import org.apache.flink.runtime.instance.InstanceID;
    +import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
    +import org.apache.flink.util.Preconditions;
    +
    +import java.util.HashSet;
    +import java.util.Set;
    +import java.util.UUID;
    +import java.util.concurrent.ScheduledFuture;
    +
    +public class TaskManagerRegistration {
    +
    +	private final TaskExecutorConnection taskManagerConnection;
    +
    +	private final HashSet<SlotID> slots;
    +
    +	private UUID timeoutIdentifier;
    +
    +	private ScheduledFuture<?> timeoutFuture;
    +
    +	public TaskManagerRegistration(TaskExecutorConnection taskManagerConnection) {
    +		this.taskManagerConnection = Preconditions.checkNotNull(taskManagerConnection);
    +
    +		slots = new HashSet<>(4);
    +
    +		timeoutIdentifier = null;
    +		timeoutFuture = null;
    +	}
    +
    +	public TaskExecutorConnection getTaskManagerConnection() {
    +		return taskManagerConnection;
    +	}
    +
    +	public InstanceID getInstanceId() {
    +		return taskManagerConnection.getInstanceID();
    +	}
    +
    +	public UUID getTimeoutIdentifier() {
    +		return timeoutIdentifier;
    +	}
    +
    +	public Set<SlotID> getSlots() {
    +		return slots;
    +	}
    +
    +	public boolean removeSlot(SlotID slotId) {
    +		return slots.remove(slotId);
    +	}
    +
    +	public void addSlot(SlotID slotId) {
    +		slots.add(slotId);
    +	}
    +
    +	public void cancelTimeout() {
    +		if (null != timeoutFuture) {
    +			timeoutFuture.cancel(true);
    --- End diff --
    
    I think I would not "interrupt on cancel" here. The work of the future is simple (schedule the call via the main thread executor). An not sure what would actually happen on an interrupt while scheduling the call, but it seems not required.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3394: [FLINK-5810] [flip6] Introduce a hardened slot manager

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on the issue:

    https://github.com/apache/flink/pull/3394
  
    Incorporated your feedback @StephanEwen and rebased onto the latest master.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3394: [FLINK-5810] [flip6] Introduce a hardened slot man...

Posted by StephanEwen <gi...@git.apache.org>.
Github user StephanEwen commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3394#discussion_r106513577
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManager.java ---
    @@ -21,519 +21,897 @@
     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.ResourceSlot;
    +import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
    +import org.apache.flink.runtime.clusterframework.types.TaskManagerSlot;
     import org.apache.flink.runtime.clusterframework.types.SlotID;
     import org.apache.flink.runtime.concurrent.BiFunction;
    +import org.apache.flink.runtime.concurrent.CompletableFuture;
     import org.apache.flink.runtime.concurrent.Future;
    -import org.apache.flink.runtime.resourcemanager.ResourceManagerServices;
    +import org.apache.flink.runtime.concurrent.ScheduledExecutor;
    +import org.apache.flink.runtime.concurrent.impl.FlinkCompletableFuture;
    +import org.apache.flink.runtime.instance.InstanceID;
    +import org.apache.flink.runtime.messages.Acknowledge;
     import org.apache.flink.runtime.resourcemanager.SlotRequest;
    -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.resourcemanager.exceptions.ResourceManagerException;
    +import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
     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.runtime.taskexecutor.exceptions.SlotAllocationException;
    +import org.apache.flink.runtime.taskexecutor.exceptions.SlotOccupiedException;
     import org.apache.flink.util.Preconditions;
     import org.slf4j.Logger;
     import org.slf4j.LoggerFactory;
     
     import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.Iterator;
     import java.util.LinkedHashMap;
     import java.util.Map;
    -
    -import static org.apache.flink.util.Preconditions.checkNotNull;
    +import java.util.Objects;
    +import java.util.UUID;
    +import java.util.concurrent.CancellationException;
    +import java.util.concurrent.Executor;
    +import java.util.concurrent.ScheduledFuture;
    +import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.TimeoutException;
     
     /**
    - * 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>.
    + * The slot manager is responsible for maintaining a view on all registered task manager slots,
    + * their allocation and all pending slot requests. Whenever a new slot is registered or and
    + * allocated slot is freed, then it tries to fulfill another pending slot request. Whenever there
    + * are not enough slots available the slot manager will notify the resource manager about it via
    + * {@link ResourceManagerActions#allocateResource(ResourceProfile)}.
    + *
    + * In order to free resources and avoid resource leaks, idling task managers (task managers whose
    + * slots are currently not used) and not fulfilled pending slot requests time out triggering their
    + * release and failure, respectively.
      */
    -public abstract class SlotManager {
    +public class SlotManager implements AutoCloseable {
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(SlotManager.class);
    +
    +	/** Scheduled executor for timeouts */
    +	private final ScheduledExecutor scheduledExecutor;
    +
    +	/** Timeout for slot requests to the task manager */
    +	private final Time taskManagerRequestTimeout;
    +
    +	/** Timeout after which an allocation is discarded */
    +	private final Time slotRequestTimeout;
    +
    +	/** Timeout after which an unused TaskManager is released */
    +	private final Time taskManagerTimeout;
    +
    +	/** Map for all registered slots */
    +	private final HashMap<SlotID, TaskManagerSlot> slots;
     
    -	protected final Logger LOG = LoggerFactory.getLogger(getClass());
    +	/** Index of all currently free slots */
    +	private final LinkedHashMap<SlotID, TaskManagerSlot> freeSlots;
     
    -	/** The Resource allocation provider */
    -	protected final ResourceManagerServices rmServices;
    +	/** All currently registered task managers */
    +	private final HashMap<InstanceID, TaskManagerRegistration> taskManagerRegistrations;
     
    -	/** All registered task managers with ResourceID and gateway. */
    -	private final Map<ResourceID, TaskExecutorRegistration> taskManagers;
    +	/** Map of fulfilled and active allocations for request deduplication purposes */
    +	private final HashMap<AllocationID, SlotID> fulfilledSlotRequests;
     
    -	/** All registered slots, including free and allocated slots */
    -	private final Map<ResourceID, Map<SlotID, ResourceSlot>> registeredSlots;
    +	/** Map of pending/unfulfilled slot allocation requests */
    +	private final HashMap<AllocationID, PendingSlotRequest> pendingSlotRequests;
     
    -	/** All pending slot requests, waiting available slots to fulfil */
    -	private final Map<AllocationID, SlotRequest> pendingSlotRequests;
    +	/** Leader id of the containing component */
    +	private UUID leaderId;
     
    -	/** All free slots that can be used to be allocated */
    -	private final Map<SlotID, ResourceSlot> freeSlots;
    +	/** Executor for future callbacks which have to be "synchronized" */
    +	private Executor mainThreadExecutor;
     
    -	/** All allocations, we can lookup allocations either by SlotID or AllocationID */
    -	private final AllocationMap allocationMap;
    +	/** Callbacks for resource (de-)allocations */
    +	private ResourceManagerActions resourceManagerActions;
     
    -	private final Time timeout;
    +	/** True iff the component has been started */
    +	private boolean started;
     
    -	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.taskManagers = new HashMap<>();
    -		this.timeout = Time.seconds(10);
    +	public SlotManager(
    +			ScheduledExecutor scheduledExecutor,
    +			Time taskManagerRequestTimeout,
    +			Time slotRequestTimeout,
    +			Time taskManagerTimeout) {
    +		this.scheduledExecutor = Preconditions.checkNotNull(scheduledExecutor);
    +		this.taskManagerRequestTimeout = Preconditions.checkNotNull(taskManagerRequestTimeout);
    +		this.slotRequestTimeout = Preconditions.checkNotNull(slotRequestTimeout);
    +		this.taskManagerTimeout = Preconditions.checkNotNull(taskManagerTimeout);
    +
    +		slots = new HashMap<>(16);
    +		freeSlots = new LinkedHashMap<>(16);
    +		taskManagerRegistrations = new HashMap<>(4);
    +		fulfilledSlotRequests = new HashMap<>(16);
    +		pendingSlotRequests = new HashMap<>(16);
    +
    +		leaderId = null;
    +		resourceManagerActions = null;
    +		started = false;
     	}
     
    -	// ------------------------------------------------------------------------
    -	//  slot managements
    -	// ------------------------------------------------------------------------
    +	// ---------------------------------------------------------------------------------------------
    +	// Component lifecycle methods
    +	// ---------------------------------------------------------------------------------------------
     
     	/**
    -	 * 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).
    +	 * Starts the slot manager with the given leader id and resource manager actions.
     	 *
    -	 * @param request The detailed request of the slot
    -	 * @return RMSlotRequestRegistered The confirmation message to be send to the caller
    +	 * @param newLeaderId to use for communication with the task managers
    +	 * @param newResourceManagerActions to use for resource (de-)allocations
    +	 */
    +	public void start(UUID newLeaderId, Executor newMainThreadExecutor, ResourceManagerActions newResourceManagerActions) {
    +		leaderId = Preconditions.checkNotNull(newLeaderId);
    +		mainThreadExecutor = Preconditions.checkNotNull(newMainThreadExecutor);
    +		resourceManagerActions = Preconditions.checkNotNull(newResourceManagerActions);
    +
    +		started = true;
    +	}
    +
    +	/**
    +	 * Suspends the component. This clears the internal state of the slot manager.
     	 */
    -	public RMSlotRequestRegistered requestSlot(final SlotRequest request) {
    -		final AllocationID allocationId = request.getAllocationId();
    -		if (isRequestDuplicated(request)) {
    -			LOG.warn("Duplicated slot request, AllocationID:{}", allocationId);
    -			return new RMSlotRequestRegistered(allocationId);
    +	public void suspend() {
    +		for (PendingSlotRequest pendingSlotRequest : pendingSlotRequests.values()) {
    +			cancelPendingSlotRequest(pendingSlotRequest);
     		}
     
    -		// 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());
    +		pendingSlotRequests.clear();
    +
    +		HashSet<InstanceID> registeredTaskManagers = new HashSet<>(taskManagerRegistrations.keySet());
    +
    +		for (InstanceID registeredTaskManager : registeredTaskManagers) {
    +			unregisterTaskManager(registeredTaskManager);
    +		}
    +
    +		leaderId = null;
    +		resourceManagerActions = null;
    +		started = false;
    +	}
    +
    +	/**
    +	 * Closes the slot manager.
    +	 *
    +	 * @throws Exception if the close operation fails
    +	 */
    +	@Override
    +	public void close() throws Exception {
    +		suspend();
    +	}
    +
    +	// ---------------------------------------------------------------------------------------------
    +	// Public API
    +	// ---------------------------------------------------------------------------------------------
    +
    +	/**
    +	 * Requests a slot with the respective resource profile.
    +	 *
    +	 * @param slotRequest specifying the requested slot specs
    +	 * @return true if the slot request was registered; false if the request is a duplicate
    +	 * @throws SlotManagerException if the slot request failed (e.g. not enough resources left)
    +	 */
    +	public boolean registerSlotRequest(SlotRequest slotRequest) throws SlotManagerException {
    +		checkInit();
     
    -			// record this allocation in bookkeeping
    -			allocationMap.addAllocation(slot.getSlotId(), allocationId);
    -			// remove selected slot from free pool
    -			freeSlots.remove(slot.getSlotId());
    +		if (checkDuplicateRequest(slotRequest.getAllocationId())) {
    +			LOG.debug("Ignoring a duplicate slot request with allocation id {}.", slotRequest.getAllocationId());
     
    -			sendSlotRequest(slot, request);
    +			return false;
     		} else {
    -			LOG.info("Cannot fulfil slot request, try to allocate a new container for it, " +
    -				"AllocationID:{}, JobID:{}", allocationId, request.getJobId());
    -			Preconditions.checkState(rmServices != null,
    -				"Attempted to allocate resources but no ResourceManagerServices set.");
    -			rmServices.allocateResource(request.getResourceProfile());
    -			pendingSlotRequests.put(allocationId, request);
    +			PendingSlotRequest pendingSlotRequest = new PendingSlotRequest(slotRequest);
    +
    +			pendingSlotRequests.put(slotRequest.getAllocationId(), pendingSlotRequest);
    +
    +			try {
    +				internalRequestSlot(pendingSlotRequest);
    +			} catch (ResourceManagerException e) {
    +				// requesting the slot failed --> remove pending slot request
    +				pendingSlotRequests.remove(slotRequest.getAllocationId());
    +
    +				throw new SlotManagerException("Could not fulfill slot request " + slotRequest.getAllocationId() + '.', e);
    +			}
    +
    +			return true;
     		}
    +	}
     
    -		return new RMSlotRequestRegistered(allocationId);
    +	/**
    +	 * Cancels and removes a pending slot request with the given allocation id. If there is no such
    +	 * pending request, then nothing is done.
    +	 *
    +	 * @param allocationId identifying the pending slot request
    +	 * @return True if a pending slot request was found; otherwise false
    +	 */
    +	public boolean unregisterSlotRequest(AllocationID allocationId) {
    +		checkInit();
    +
    +		PendingSlotRequest pendingSlotRequest = pendingSlotRequests.remove(allocationId);
    +
    +		if (null != pendingSlotRequest) {
    +			cancelPendingSlotRequest(pendingSlotRequest);
    +
    +			return true;
    +		} else {
    +			LOG.debug("No pending slot request with allocation id {} found.", allocationId);
    +
    +			return false;
    +		}
     	}
     
     	/**
    -	 * Notifies the SlotManager that a slot is available again after being allocated.
    -	 * @param slotID slot id of available slot
    +	 * Registers a new task manager at the slot manager. This will make the task managers slots
    +	 * known and, thus, available for allocation.
    +	 *
    +	 * @param taskExecutorConnection for the new task manager
    +	 * @param initialSlotReport for the new task manager
     	 */
    -	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");
    +	public void registerTaskManager(final TaskExecutorConnection taskExecutorConnection, SlotReport initialSlotReport) {
    +		checkInit();
    +
    +		// we identify task managers by their instance id
    +		if (!taskManagerRegistrations.containsKey(taskExecutorConnection.getInstanceID())) {
    +			TaskManagerRegistration taskManagerRegistration = new TaskManagerRegistration(taskExecutorConnection);
    +			taskManagerRegistrations.put(taskExecutorConnection.getInstanceID(), taskManagerRegistration);
     		}
    -		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.");
    +
    +		reportSlotStatus(taskExecutorConnection.getInstanceID(), initialSlotReport);
    +	}
    +
    +	/**
    +	 * Unregisters the task manager identified by the given instance id and its associated slots
    +	 * from the slot manager.
    +	 *
    +	 * @param instanceId identifying the task manager to unregister
    +	 * @return True if there existed a registered task manager with the given instance id
    +	 */
    +	public boolean unregisterTaskManager(InstanceID instanceId) {
    +		checkInit();
    +
    +		TaskManagerRegistration taskManagerRegistration = taskManagerRegistrations.remove(instanceId);
    +
    +		if (null != taskManagerRegistration) {
    +			removeSlots(taskManagerRegistration.getSlots());
    +
    +			taskManagerRegistration.cancelTimeout();
    +
    +			return true;
    +		} else {
    +			LOG.debug("There is no task manager registered with instance ID {}. Ignoring this message.", instanceId);
    +
    +			return false;
     		}
    -		handleFreeSlot(freeSlot);
     	}
     
     	/**
    -	 * 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.
    +	 * Reports the current slot allocations for a task manager identified by the given instance id.
     	 *
    -	 * @param originalRequest The original slot request
    -	 * @param slotId          The target SlotID
    +	 * @param instanceId identifying the task manager for which to report the slot status
    +	 * @param slotReport containing the status for all of its slots
    +	 * @return true if the slot status has been updated successfully, otherwise false
     	 */
    -	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());
    -
    -		if (allocationMap.isAllocated(slotId)) {
    -			final AllocationID expectedAllocationId = allocationMap.getAllocationID(slotId);
    -
    -			// 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);
    +	public boolean reportSlotStatus(InstanceID instanceId, SlotReport slotReport) {
    +		checkInit();
    +
    +		TaskManagerRegistration taskManagerRegistration = taskManagerRegistrations.get(instanceId);
    +
    +		if (null != taskManagerRegistration) {
    +			HashSet<SlotID> slotsToRemove = new HashSet<>(taskManagerRegistration.getSlots());
    +			boolean idle = true;
    +
    +			for (SlotStatus slotStatus : slotReport) {
    +				if (slotsToRemove.remove(slotStatus.getSlotID())) {
    +					// slot which was already registered
    +					updateSlot(slotStatus.getSlotID(), slotStatus.getAllocationID());
    +				} else {
    +					// new slot
    +					registerSlot(
    +						instanceId,
    +						slotStatus.getSlotID(),
    +						slotStatus.getAllocationID(),
    +						slotStatus.getResourceProfile(),
    +						taskManagerRegistration.getTaskManagerConnection());
    +				}
    +
    +				TaskManagerSlot slot = slots.get(slotStatus.getSlotID());
    +
    +				idle &= slot.isFree();
    +			}
    +
    +			// remove the slots for which we haven't received a slot status message
    +			removeSlots(slotsToRemove);
    +
    +			if (idle) {
    +				// no slot of this task manager is being used --> register timer to free this resource
    +				registerTaskManagerTimeout(taskManagerRegistration);
    +			}
    +
    +			return true;
    +		} else {
    +			LOG.debug("Received slot report for unknown task manager with instance id {}. Ignoring this report.", instanceId);
    +
    +			return false;
    +		}
    +	}
    +
    +	/**
    +	 * Free the given slot from the given allocation. If the slot is still allocated by the given
    +	 * allocation id, then the slot will be marked as free and will be subject to new slot requests.
    +	 *
    +	 * @param slotId identifying the slot to free
    +	 * @param allocationId with which the slot is presumably allocated
    +	 */
    +	public void freeSlot(SlotID slotId, AllocationID allocationId) {
    +		checkInit();
    +
    +		TaskManagerSlot slot = slots.get(slotId);
    +
    +		if (null != slot) {
    +			if (slot.isAllocated()) {
    +				if (Objects.equals(allocationId, slot.getAllocationId())) {
    +					// free the slot
    +					slot.setAllocationId(null);
    +					fulfilledSlotRequests.remove(allocationId);
    +
    +					if (slot.isFree()) {
    +						handleFreeSlot(slot);
    +					}
    +
    +					TaskManagerRegistration taskManagerRegistration = taskManagerRegistrations.get(slot.getInstanceId());
    +
    +					if (null != taskManagerRegistration && !anySlotUsed(taskManagerRegistration.getSlots())) {
    +						registerTaskManagerTimeout(taskManagerRegistration);
    +					}
    +				} else {
    +					LOG.debug("Received request to free slot {} with expected allocation id {}, " +
    +						"but actual allocation id {} differs. Ignoring the request.", slotId, allocationId, slot.getAllocationId());
    +				}
     			} else {
    -				LOG.error("Slot request failed for slot {} with allocation id {}:" +
    -						" Allocation id did not match the expected allocation id {}.",
    -					slotId, originalAllocationId, expectedAllocationId);
    +				LOG.debug("Slot {} has not been allocated.", allocationId);
     			}
     		} else {
    -			LOG.error("Slot request failed for slot {} with allocation id {}: " +
    -					"Slot was not previously registered.",
    -				slotId, originalAllocationId);
    +			LOG.debug("Trying to free a slot {} which has not been registered. Ignoring this message.", slotId);
     		}
     	}
     
    +	// ---------------------------------------------------------------------------------------------
    +	// Behaviour methods
    +	// ---------------------------------------------------------------------------------------------
    +
     	/**
    -	 * Registers a TaskExecutor
    -	 * @param resourceID TaskExecutor's ResourceID
    -	 * @param registration TaskExecutor's registration
    -	 * @param slotReport TaskExecutor's free and allocated slots
    +	 * Finds a matching slot request for a given resource profile. If there is no such request,
    +	 * the method returns null.
    +	 *
    +	 * Note: If you want to change the behaviour of the slot manager wrt slot allocation and
    +	 * request fulfillment, then you should override this method.
    +	 *
    +	 * @param slotResourceProfile defining the resources of an available slot
    +	 * @return A matching slot request which can be deployed in a slot with the given resource
    +	 * profile. Null if there is no such slot request pending.
     	 */
    -	public void registerTaskExecutor(
    -			ResourceID resourceID,
    -			TaskExecutorRegistration registration,
    -			SlotReport slotReport) {
    +	protected PendingSlotRequest findMatchingRequest(ResourceProfile slotResourceProfile) {
     
    -		if (taskManagers.get(resourceID) != null) {
    -			notifyTaskManagerFailure(resourceID);
    +		for (PendingSlotRequest pendingSlotRequest : pendingSlotRequests.values()) {
    +			if (!pendingSlotRequest.isAssigned() && slotResourceProfile.isMatching(pendingSlotRequest.getResourceProfile())) {
    +				return pendingSlotRequest;
    +			}
     		}
     
    -		this.taskManagers.put(resourceID, registration);
    +		return null;
    +	}
    +
    +	/**
    +	 * Finds a matching slot for a given resource profile. A matching slot has at least as many
    +	 * resources available as the given resource profile. If there is no such slot available, then
    +	 * the method returns null.
    +	 *
    +	 * Note: If you want to change the behaviour of the slot manager wrt slot allocation and
    +	 * request fulfillment, then you should override this method.
    +	 *
    +	 * @param requestResourceProfile specifying the resource requirements for the a slot request
    +	 * @return A matching slot which fulfills the given resource profile. Null if there is no such
    +	 * slot available.
    +	 */
    +	protected TaskManagerSlot findMatchingSlot(ResourceProfile requestResourceProfile) {
    +		Iterator<Map.Entry<SlotID, TaskManagerSlot>> iterator = freeSlots.entrySet().iterator();
    +
    +		while (iterator.hasNext()) {
    +			TaskManagerSlot taskManagerSlot = iterator.next().getValue();
     
    -		for (SlotStatus slotStatus : slotReport.getSlotsStatus()) {
    -			final SlotID slotId = slotStatus.getSlotID();
    +			// sanity check
    +			Preconditions.checkState(taskManagerSlot.isFree());
     
    -			final TaskExecutorRegistration taskExecutorRegistration = taskManagers.get(slotId.getResourceID());
    -			if (taskExecutorRegistration == null) {
    -				LOG.info("Received SlotStatus but ResourceID {} is unknown to the SlotManager",
    -					slotId.getResourceID());
    -				return;
    +			if (taskManagerSlot.getResourceProfile().isMatching(requestResourceProfile)) {
    +				iterator.remove();
    +				return taskManagerSlot;
     			}
    +		}
     
    -			final ResourceSlot slot = new ResourceSlot(slotId, slotStatus.getProfiler(), taskExecutorRegistration);
    +		return null;
    +	}
     
    -			registerNewSlot(slot);
    -			LOG.info("New slot appeared, SlotID:{}, AllocationID:{}", slotId, slotStatus.getAllocationID());
    +	// ---------------------------------------------------------------------------------------------
    +	// Internal slot operations
    +	// ---------------------------------------------------------------------------------------------
     
    -			if (slotStatus.getAllocationID() != null) {
    -				// slot in use, record this in bookkeeping
    -				allocationMap.addAllocation(slotId, slotStatus.getAllocationID());
    -			} else {
    +	/**
    +	 * Registers a slot for the given task manager at the slot manager. The task manager is
    +	 * identified by the given instance id and the slot is identified by the given slot id. The
    +	 * given resource profile defines the available resources for the slot. The task manager
    +	 * connection can be used to communicate with the task manager.
    +	 *
    +	 * @param instanceId identifying the task manager on which the slot lives
    +	 * @param slotId identifying the slot on the task manager
    +	 * @param allocationId which is currently deployed in the slot
    +	 * @param resourceProfile of the slot
    +	 * @param taskManagerConnection to communicate with the remote task manager
    +	 */
    +	private void registerSlot(
    --- End diff --
    
    I would pass the `TaskManagerRegistration` into this method. Looking it up again is redundant work and leads to the "what if it is not found" case (where the question comes up whether to log or to throw an exception), which can be avoided.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---