You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2020/10/07 13:06:33 UTC

[GitHub] [flink] zentol opened a new pull request #13553: [FLINK-19306][coordination] Add DeclarativeSlotManager

zentol opened a new pull request #13553:
URL: https://github.com/apache/flink/pull/13553


   Based on #13544.
   
   Adds the `DeclarativeSlotManager` (DSM), an alternative `SlotManager` implementation that supports FLIP-138.
   
   The core difference to the existing implementation is that the DSM does not receive individual slot requests from the JobMaster, but a `ResourceRequirements` instead that outlines the absolute requirements for the job.
   The DSM keeps track of these, along with all registered and pending slots, matches requirements to slots and initiates the allocation of the slot.
   
   Most of this logic and book-keeping is handled by recently added components (`SlotTracker`, `ResourceTracker`, `TaskExecutorManager`).
   
   What is left in the DSM itself is
   a) a bunch of glue between these components and the `SlotManager` API
   b) the logic for matching missing to available/pending resources and initiating the corresponding slot allocations.
   
   The entry point for b) is `checkResourceRequirements()`, which is called on any significant change to the state of requirements or resources; it is called when requirements change, slots are added/removed, allocations are complete/failed.
   This method essentially does the following:
   - retrieve the currently missing resources from the `ResourceTracker`
   - try assigning these resources to free slots, provided by the `SlotTracker`
     - if a match was found, start the allocation of the slot
   - if at the end of this process there are still missing resource, match the remaining missing resources against the pending slots, provided by the `TaskExecutorManager`
     - if a match was found for a requirement, do nothing. We do not maintain a mapping of requirements to pending resources as it would double the book-keeping. We just consider this requirement as fulfilled within the scope of this check.
     - if no match was found for a requirement, try allocating a new worker (implicitly increasing the pool of pending slots)
   - if at the end of this process missing resource are still present (== no more workers can be allocated) inform the JobMaster about there not being enough resources to fulfill the requirement
   
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13553: [FLINK-19306][coordination] Add DeclarativeSlotManager

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13553:
URL: https://github.com/apache/flink/pull/13553#issuecomment-704936985


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "15d4c353c4c0a646b2bdf34bbedbd81359cb5b5f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7264",
       "triggerID" : "15d4c353c4c0a646b2bdf34bbedbd81359cb5b5f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15d4c353c4c0a646b2bdf34bbedbd81359cb5b5f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7283",
       "triggerID" : "705384887",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "15d4c353c4c0a646b2bdf34bbedbd81359cb5b5f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7264",
       "triggerID" : "705384887",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "484cbbdfcc7e371ca51de4d88bab086a299795a6",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7514",
       "triggerID" : "484cbbdfcc7e371ca51de4d88bab086a299795a6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7a4c350c9514dc81429de2d9887837045b5894e0",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7527",
       "triggerID" : "7a4c350c9514dc81429de2d9887837045b5894e0",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 484cbbdfcc7e371ca51de4d88bab086a299795a6 Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7514) 
   * 7a4c350c9514dc81429de2d9887837045b5894e0 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7527) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13553: [FLINK-19306][coordination] Add DeclarativeSlotManager

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13553:
URL: https://github.com/apache/flink/pull/13553#issuecomment-704936985


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "15d4c353c4c0a646b2bdf34bbedbd81359cb5b5f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7264",
       "triggerID" : "15d4c353c4c0a646b2bdf34bbedbd81359cb5b5f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15d4c353c4c0a646b2bdf34bbedbd81359cb5b5f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7283",
       "triggerID" : "705384887",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "15d4c353c4c0a646b2bdf34bbedbd81359cb5b5f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7264",
       "triggerID" : "705384887",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "484cbbdfcc7e371ca51de4d88bab086a299795a6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7514",
       "triggerID" : "484cbbdfcc7e371ca51de4d88bab086a299795a6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7a4c350c9514dc81429de2d9887837045b5894e0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7527",
       "triggerID" : "7a4c350c9514dc81429de2d9887837045b5894e0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4b9288681085da65fc2c3f2742c898d13bf4af46",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7580",
       "triggerID" : "4b9288681085da65fc2c3f2742c898d13bf4af46",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f89b7098fb826ded9b17b0f2eb48c747451623d9",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "f89b7098fb826ded9b17b0f2eb48c747451623d9",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 4b9288681085da65fc2c3f2742c898d13bf4af46 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7580) 
   * f89b7098fb826ded9b17b0f2eb48c747451623d9 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13553: [FLINK-19306][coordination] Add DeclarativeSlotManager

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13553:
URL: https://github.com/apache/flink/pull/13553#issuecomment-704936985


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "15d4c353c4c0a646b2bdf34bbedbd81359cb5b5f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7264",
       "triggerID" : "15d4c353c4c0a646b2bdf34bbedbd81359cb5b5f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15d4c353c4c0a646b2bdf34bbedbd81359cb5b5f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7283",
       "triggerID" : "705384887",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "15d4c353c4c0a646b2bdf34bbedbd81359cb5b5f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7264",
       "triggerID" : "705384887",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "484cbbdfcc7e371ca51de4d88bab086a299795a6",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "484cbbdfcc7e371ca51de4d88bab086a299795a6",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 15d4c353c4c0a646b2bdf34bbedbd81359cb5b5f Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7283) Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7264) 
   * 484cbbdfcc7e371ca51de4d88bab086a299795a6 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] zentol commented on a change in pull request #13553: [FLINK-19306][coordination] Add DeclarativeSlotManager

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #13553:
URL: https://github.com/apache/flink/pull/13553#discussion_r502340740



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManager.java
##########
@@ -0,0 +1,609 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.metrics.MetricNames;
+import org.apache.flink.runtime.metrics.groups.SlotManagerMetricGroup;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerId;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.slots.ResourceCounter;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+import org.apache.flink.runtime.slots.ResourceRequirements;
+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.SlotOccupiedException;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ScheduledFuture;
+import java.util.function.BiFunction;
+import java.util.stream.Collectors;
+
+/**
+ * Implementation of {@link SlotManager} supporting declarative slot management.
+ */
+public class DeclarativeSlotManager implements SlotManager {
+	private static final Logger LOG = LoggerFactory.getLogger(DeclarativeSlotManager.class);
+
+	private final SlotTracker slotTracker;
+	private final ResourceTracker resourceTracker;
+	private final BiFunction<Executor, ResourceActions, TaskExecutorManager> taskExecutorManagerFactory;
+	private TaskExecutorManager taskExecutorManager;
+
+	/** Timeout for slot requests to the task manager. */
+	private final Time taskManagerRequestTimeout;
+	private ScheduledFuture<?> slotRequestTimeoutCheck;
+
+	private final SlotMatchingStrategy slotMatchingStrategy;
+
+	private final SlotManagerMetricGroup slotManagerMetricGroup;
+
+	private final Map<JobID, String> jobMasterTargetAddresses = new HashMap<>();
+	private final HashMap<SlotID, CompletableFuture<Acknowledge>> pendingSlotAllocationFutures;
+
+	/** ResourceManager's id. */
+	private ResourceManagerId resourceManagerId;
+
+	/** Executor for future callbacks which have to be "synchronized". */
+	private Executor mainThreadExecutor;
+
+	/** Callbacks for resource (de-)allocations. */
+	private ResourceActions resourceActions;
+
+	/** True iff the component has been started. */
+	private boolean started;
+
+	public DeclarativeSlotManager(
+			ScheduledExecutor scheduledExecutor,
+			SlotManagerConfiguration slotManagerConfiguration,
+			SlotManagerMetricGroup slotManagerMetricGroup,
+			ResourceTracker resourceTracker,
+			SlotTracker slotTracker) {
+
+		Preconditions.checkNotNull(slotManagerConfiguration);
+		this.taskManagerRequestTimeout = slotManagerConfiguration.getTaskManagerRequestTimeout();
+		this.slotManagerMetricGroup = Preconditions.checkNotNull(slotManagerMetricGroup);
+		this.resourceTracker = Preconditions.checkNotNull(resourceTracker);
+
+		pendingSlotAllocationFutures = new HashMap<>(16);
+
+		this.slotTracker = Preconditions.checkNotNull(slotTracker);
+		slotTracker.registerSlotStatusUpdateListener(createSlotStatusUpdateListener());
+
+		slotMatchingStrategy = slotManagerConfiguration.getSlotMatchingStrategy();
+
+		taskExecutorManagerFactory = (executor, resourceActions) -> new TaskExecutorManager(
+			slotManagerConfiguration.getDefaultWorkerResourceSpec(),
+			slotManagerConfiguration.getNumSlotsPerWorker(),
+			slotManagerConfiguration.getMaxSlotNum(),
+			slotManagerConfiguration.isWaitResultConsumedBeforeRelease(),
+			slotManagerConfiguration.getRedundantTaskManagerNum(),
+			slotManagerConfiguration.getTaskManagerTimeout(),
+			scheduledExecutor,
+			executor,
+			resourceActions);
+
+		resourceManagerId = null;
+		resourceActions = null;
+		mainThreadExecutor = null;
+		slotRequestTimeoutCheck = null;
+		taskExecutorManager = null;
+
+		started = false;
+	}
+
+	private SlotStatusUpdateListener createSlotStatusUpdateListener() {
+		return (taskManagerSlot, previous, current, jobId) -> {
+			if (previous == SlotState.PENDING) {
+				cancelAllocationFuture(taskManagerSlot.getSlotId());
+			}
+
+			if (current == SlotState.PENDING) {
+				resourceTracker.notifyAcquiredResource(jobId, taskManagerSlot.getResourceProfile());
+			}
+			if (current == SlotState.FREE) {
+				resourceTracker.notifyLostResource(jobId, taskManagerSlot.getResourceProfile());
+			}
+
+			if (current == SlotState.ALLOCATED) {
+				taskExecutorManager.occupySlot(taskManagerSlot.getInstanceId());
+			}
+			if (previous == SlotState.ALLOCATED && current == SlotState.FREE) {
+				taskExecutorManager.freeSlot(taskManagerSlot.getInstanceId());
+			}
+		};
+	}
+
+	private void cancelAllocationFuture(SlotID slotId) {
+		final CompletableFuture<Acknowledge> acknowledgeCompletableFuture = pendingSlotAllocationFutures.remove(slotId);
+		// the future may be null if we are just re-playing the state transitions due to a slot report
+		if (acknowledgeCompletableFuture != null) {
+			acknowledgeCompletableFuture.cancel(false);
+		}
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Component lifecycle methods
+	// ---------------------------------------------------------------------------------------------
+
+	/**
+	 * Starts the slot manager with the given leader id and resource manager actions.
+	 *
+	 * @param newResourceManagerId to use for communication with the task managers
+	 * @param newMainThreadExecutor to use to run code in the ResourceManager's main thread
+	 * @param newResourceActions to use for resource (de-)allocations
+	 */
+	@Override
+	public void start(ResourceManagerId newResourceManagerId, Executor newMainThreadExecutor, ResourceActions newResourceActions) {
+		LOG.info("Starting the slot manager.");
+
+		this.resourceManagerId = Preconditions.checkNotNull(newResourceManagerId);
+		mainThreadExecutor = Preconditions.checkNotNull(newMainThreadExecutor);
+		resourceActions = Preconditions.checkNotNull(newResourceActions);
+		taskExecutorManager = taskExecutorManagerFactory.apply(newMainThreadExecutor, newResourceActions);
+
+		started = true;
+
+		registerSlotManagerMetrics();
+	}
+
+	private void registerSlotManagerMetrics() {
+		slotManagerMetricGroup.gauge(
+			MetricNames.TASK_SLOTS_AVAILABLE,
+			() -> (long) getNumberFreeSlots());
+		slotManagerMetricGroup.gauge(
+			MetricNames.TASK_SLOTS_TOTAL,
+			() -> (long) getNumberRegisteredSlots());
+	}
+
+	/**
+	 * Suspends the component. This clears the internal state of the slot manager.
+	 */
+	@Override
+	public void suspend() {
+		LOG.info("Suspending the slot manager.");
+
+		if (slotRequestTimeoutCheck != null) {
+			slotRequestTimeoutCheck.cancel(false);
+			slotRequestTimeoutCheck = null;
+		}
+
+		resourceTracker.clear();
+		taskExecutorManager.close();
+
+		for (InstanceID registeredTaskManager : taskExecutorManager.getTaskExecutors()) {
+			unregisterTaskManager(registeredTaskManager, new SlotManagerException("The slot manager is being suspended."));
+		}
+
+		taskExecutorManager = null;
+		resourceManagerId = null;
+		resourceActions = null;
+		started = false;
+	}
+
+	/**
+	 * Closes the slot manager.
+	 *
+	 * @throws Exception if the close operation fails
+	 */
+	@Override
+	public void close() throws Exception {
+		LOG.info("Closing the slot manager.");
+
+		suspend();
+		slotManagerMetricGroup.close();
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Public API
+	// ---------------------------------------------------------------------------------------------
+
+	@Override
+	public void processResourceRequirements(ResourceRequirements resourceRequirements) {
+		checkInit();
+		LOG.debug("Received resource requirements from job {}: {}", resourceRequirements.getJobId(), resourceRequirements.getResourceRequirements());
+
+		if (resourceRequirements.getResourceRequirements().isEmpty()) {
+			jobMasterTargetAddresses.remove(resourceRequirements.getJobId());
+		} else {
+			jobMasterTargetAddresses.put(resourceRequirements.getJobId(), resourceRequirements.getTargetAddress());
+		}
+		resourceTracker.notifyResourceRequirements(resourceRequirements.getJobId(), resourceRequirements.getResourceRequirements());
+		checkResourceRequirements();
+	}
+
+	/**
+	 * 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
+	 * @return True if the task manager has not been registered before and is registered successfully; otherwise false
+	 */
+	@Override
+	public boolean registerTaskManager(final TaskExecutorConnection taskExecutorConnection, SlotReport initialSlotReport) {
+		checkInit();
+		LOG.debug("Registering task executor {} under {} at the slot manager.", taskExecutorConnection.getResourceID(), taskExecutorConnection.getInstanceID());
+
+		// we identify task managers by their instance id
+		if (taskExecutorManager.isTaskManagerRegistered(taskExecutorConnection.getInstanceID())) {
+			LOG.debug("Task executor {} was already registered.", taskExecutorConnection.getResourceID());
+			reportSlotStatus(taskExecutorConnection.getInstanceID(), initialSlotReport);
+			return false;
+		} else {
+			if (!taskExecutorManager.registerTaskManager(taskExecutorConnection, initialSlotReport)) {
+				LOG.debug("Task executor {} could not be registered.", taskExecutorConnection.getResourceID());
+				return false;
+			}
+
+			// register the new slots
+			for (SlotStatus slotStatus : initialSlotReport) {
+				slotTracker.addSlot(
+					slotStatus.getSlotID(),
+					slotStatus.getResourceProfile(),
+					taskExecutorConnection,
+					slotStatus.getJobID());
+			}
+
+			checkResourceRequirements();
+			return true;
+		}
+	}
+
+	@Override
+	public boolean unregisterTaskManager(InstanceID instanceId, Exception cause) {
+		checkInit();
+
+		LOG.debug("Unregistering task executor {} from the slot manager.", instanceId);
+
+		if (taskExecutorManager.isTaskManagerRegistered(instanceId)) {
+			slotTracker.removeSlots(taskExecutorManager.getSlotsOf(instanceId));
+			taskExecutorManager.unregisterTaskExecutor(instanceId);
+			checkResourceRequirements();
+
+			return true;
+		} else {
+			LOG.debug("There is no task executor registered with instance ID {}. Ignoring this message.", instanceId);
+
+			return false;
+		}
+	}
+
+	/**
+	 * Reports the current slot allocations for a task manager identified by the given instance id.
+	 *
+	 * @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
+	 */
+	@Override
+	public boolean reportSlotStatus(InstanceID instanceId, SlotReport slotReport) {
+		checkInit();
+
+		LOG.debug("Received slot report from instance {}: {}.", instanceId, slotReport);
+
+		if (taskExecutorManager.isTaskManagerRegistered(instanceId)) {
+			slotTracker.notifySlotStatus(slotReport);
+			checkResourceRequirements();
+			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
+	 */
+	@Override
+	public void freeSlot(SlotID slotId, AllocationID allocationId) {
+		checkInit();
+		LOG.debug("Freeing slot {}.", slotId);
+
+		slotTracker.notifyFree(slotId);
+		checkResourceRequirements();
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Requirement matching
+	// ---------------------------------------------------------------------------------------------
+
+	private void checkResourceRequirements() {
+		final Map<JobID, Collection<ResourceRequirement>> missingResources = resourceTracker.getMissingResources();
+		if (missingResources.isEmpty()) {
+			return;
+		}
+
+		final Map<JobID, ResourceCounter> outstandingRequirements = new LinkedHashMap<>();
+		for (Map.Entry<JobID, Collection<ResourceRequirement>> resourceRequirements : missingResources.entrySet()) {
+			JobID jobId = resourceRequirements.getKey();
+
+			for (ResourceRequirement resourceRequirement : resourceRequirements.getValue()) {
+				int numMissingSlots = internalRequestSlots(jobId, jobMasterTargetAddresses.get(jobId), resourceRequirement);
+				if (numMissingSlots > 0) {
+					outstandingRequirements
+						.computeIfAbsent(jobId, ignored -> new ResourceCounter())
+						.incrementCount(resourceRequirement.getResourceProfile(), numMissingSlots);
+				}
+			}
+		}
+
+		final ResourceCounter pendingSlots = new ResourceCounter(taskExecutorManager.getPendingTaskManagerSlots().stream().collect(
+			Collectors.groupingBy(
+				PendingTaskManagerSlot::getResourceProfile,
+				Collectors.summingInt(x -> 1))));
+
+		for (Map.Entry<JobID, ResourceCounter> unfulfilledRequirement : outstandingRequirements.entrySet()) {
+			tryFulfillRequirementsWithPendingOrNewSlots(
+				unfulfilledRequirement.getKey(),
+				unfulfilledRequirement.getValue().getResourceProfilesWithCount(),
+				pendingSlots);
+		}
+	}
+
+	/**
+	 * Tries to allocate slots for the given requirement. If there are not enough slots available, the
+	 * resource manager is informed to allocate more resources.
+	 *
+	 * @param jobId job to allocate slots for
+	 * @param targetAddress address of the jobmaster
+	 * @param resourceRequirement required slots
+	 * @return the number of missing slots
+	 */
+	private int internalRequestSlots(JobID jobId, String targetAddress, ResourceRequirement resourceRequirement) {
+		final ResourceProfile requiredResource = resourceRequirement.getResourceProfile();
+		Collection<TaskManagerSlotInformation> freeSlots = slotTracker.getFreeSlots();
+
+		int numUnfulfilled = 0;
+		for (int x = 0; x < resourceRequirement.getNumberOfRequiredSlots(); x++) {
+
+			final Optional<TaskManagerSlotInformation> reservedSlot = slotMatchingStrategy.findMatchingSlot(requiredResource, freeSlots, this::getNumberRegisteredSlotsOf);
+			if (reservedSlot.isPresent()) {
+				// we do not need to modify freeSlots because it is indirectly modified by the allocation
+				allocateSlot(reservedSlot.get(), jobId, targetAddress, requiredResource);
+			} else {
+				numUnfulfilled++;
+			}
+		}
+		return numUnfulfilled;
+	}
+
+	/**
+	 * Allocates the given slot. This entails sending a registration message to the task manager and treating failures.
+	 *
+	 * @param taskManagerSlot slot to allocate
+	 * @param jobId job for which the slot should be allocated for
+	 * @param targetAddress address of the job master
+	 * @param resourceProfile resource profile for the requirement for which the slot is used
+	 */
+	private void allocateSlot(TaskManagerSlotInformation taskManagerSlot, JobID jobId, String targetAddress, ResourceProfile resourceProfile) {
+		TaskExecutorConnection taskExecutorConnection = taskManagerSlot.getTaskManagerConnection();
+		TaskExecutorGateway gateway = taskExecutorConnection.getTaskExecutorGateway();
+
+		final CompletableFuture<Acknowledge> completableFuture = new CompletableFuture<>();
+		final SlotID slotId = taskManagerSlot.getSlotId();
+		final InstanceID instanceId = taskManagerSlot.getInstanceId();
+
+		LOG.debug("Starting allocation of slot {} for job {} with resource profile {}.", taskManagerSlot.getSlotId(), jobId, resourceProfile);
+
+		slotTracker.notifyAllocationStart(slotId, jobId);
+
+		if (!taskExecutorManager.isTaskManagerRegistered(instanceId)) {
+			throw new IllegalStateException("Could not find a registered task manager for instance id " +
+				instanceId + '.');
+		}
+
+		taskExecutorManager.markUsed(instanceId);
+
+		// RPC call to the task manager
+		CompletableFuture<Acknowledge> requestFuture = gateway.requestSlot(
+			slotId,
+			jobId,
+			new AllocationID(),
+			resourceProfile,
+			targetAddress,
+			resourceManagerId,
+			taskManagerRequestTimeout);
+
+		requestFuture.whenComplete(
+			(Acknowledge acknowledge, Throwable throwable) -> {
+				if (acknowledge != null) {
+					completableFuture.complete(acknowledge);
+				} else {
+					completableFuture.completeExceptionally(throwable);
+				}
+			});
+
+		completableFuture.whenCompleteAsync(
+			(Acknowledge acknowledge, Throwable throwable) -> {
+				try {
+					if (acknowledge != null) {
+						LOG.trace("Completed allocation of slot {} for job {}.", slotId, jobId);
+						slotTracker.notifyAllocationComplete(slotId, jobId);
+					} else {
+						if (throwable instanceof SlotOccupiedException) {
+							SlotOccupiedException exception = (SlotOccupiedException) throwable;
+							LOG.debug("Tried allocating slot {} for job {}, but it was already allocated for job {}.", slotId, jobId, exception.getJobId());
+							// report as a slot status to force the state transition
+							// this could be a problem if we ever assume that the task executor always reports about all slots
+							slotTracker.notifySlotStatus(Collections.singleton(new SlotStatus(slotId, taskManagerSlot.getResourceProfile(), exception.getJobId(), exception.getAllocationId())));
+						} else {
+							if (throwable instanceof CancellationException) {
+								LOG.debug("Cancelled allocation of slot {} for job {}.", slotId, jobId, throwable);

Review comment:
       Whenever a pending slot switches to another state (see `createSlotStatusUpdateListener`)




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] zentol commented on a change in pull request #13553: [FLINK-19306][coordination] Add DeclarativeSlotManager

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #13553:
URL: https://github.com/apache/flink/pull/13553#discussion_r502361600



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManager.java
##########
@@ -0,0 +1,609 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.metrics.MetricNames;
+import org.apache.flink.runtime.metrics.groups.SlotManagerMetricGroup;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerId;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.slots.ResourceCounter;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+import org.apache.flink.runtime.slots.ResourceRequirements;
+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.SlotOccupiedException;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ScheduledFuture;
+import java.util.function.BiFunction;
+import java.util.stream.Collectors;
+
+/**
+ * Implementation of {@link SlotManager} supporting declarative slot management.
+ */
+public class DeclarativeSlotManager implements SlotManager {
+	private static final Logger LOG = LoggerFactory.getLogger(DeclarativeSlotManager.class);
+
+	private final SlotTracker slotTracker;
+	private final ResourceTracker resourceTracker;
+	private final BiFunction<Executor, ResourceActions, TaskExecutorManager> taskExecutorManagerFactory;
+	private TaskExecutorManager taskExecutorManager;
+
+	/** Timeout for slot requests to the task manager. */
+	private final Time taskManagerRequestTimeout;
+	private ScheduledFuture<?> slotRequestTimeoutCheck;
+
+	private final SlotMatchingStrategy slotMatchingStrategy;
+
+	private final SlotManagerMetricGroup slotManagerMetricGroup;
+
+	private final Map<JobID, String> jobMasterTargetAddresses = new HashMap<>();
+	private final HashMap<SlotID, CompletableFuture<Acknowledge>> pendingSlotAllocationFutures;
+
+	/** ResourceManager's id. */
+	private ResourceManagerId resourceManagerId;
+
+	/** Executor for future callbacks which have to be "synchronized". */
+	private Executor mainThreadExecutor;
+
+	/** Callbacks for resource (de-)allocations. */
+	private ResourceActions resourceActions;
+
+	/** True iff the component has been started. */
+	private boolean started;
+
+	public DeclarativeSlotManager(
+			ScheduledExecutor scheduledExecutor,
+			SlotManagerConfiguration slotManagerConfiguration,
+			SlotManagerMetricGroup slotManagerMetricGroup,
+			ResourceTracker resourceTracker,
+			SlotTracker slotTracker) {
+
+		Preconditions.checkNotNull(slotManagerConfiguration);
+		this.taskManagerRequestTimeout = slotManagerConfiguration.getTaskManagerRequestTimeout();
+		this.slotManagerMetricGroup = Preconditions.checkNotNull(slotManagerMetricGroup);
+		this.resourceTracker = Preconditions.checkNotNull(resourceTracker);
+
+		pendingSlotAllocationFutures = new HashMap<>(16);
+
+		this.slotTracker = Preconditions.checkNotNull(slotTracker);
+		slotTracker.registerSlotStatusUpdateListener(createSlotStatusUpdateListener());
+
+		slotMatchingStrategy = slotManagerConfiguration.getSlotMatchingStrategy();
+
+		taskExecutorManagerFactory = (executor, resourceActions) -> new TaskExecutorManager(
+			slotManagerConfiguration.getDefaultWorkerResourceSpec(),
+			slotManagerConfiguration.getNumSlotsPerWorker(),
+			slotManagerConfiguration.getMaxSlotNum(),
+			slotManagerConfiguration.isWaitResultConsumedBeforeRelease(),
+			slotManagerConfiguration.getRedundantTaskManagerNum(),
+			slotManagerConfiguration.getTaskManagerTimeout(),
+			scheduledExecutor,
+			executor,
+			resourceActions);
+
+		resourceManagerId = null;
+		resourceActions = null;
+		mainThreadExecutor = null;
+		slotRequestTimeoutCheck = null;
+		taskExecutorManager = null;
+
+		started = false;
+	}
+
+	private SlotStatusUpdateListener createSlotStatusUpdateListener() {
+		return (taskManagerSlot, previous, current, jobId) -> {
+			if (previous == SlotState.PENDING) {
+				cancelAllocationFuture(taskManagerSlot.getSlotId());
+			}
+
+			if (current == SlotState.PENDING) {
+				resourceTracker.notifyAcquiredResource(jobId, taskManagerSlot.getResourceProfile());
+			}
+			if (current == SlotState.FREE) {
+				resourceTracker.notifyLostResource(jobId, taskManagerSlot.getResourceProfile());
+			}
+
+			if (current == SlotState.ALLOCATED) {
+				taskExecutorManager.occupySlot(taskManagerSlot.getInstanceId());
+			}
+			if (previous == SlotState.ALLOCATED && current == SlotState.FREE) {
+				taskExecutorManager.freeSlot(taskManagerSlot.getInstanceId());
+			}
+		};
+	}
+
+	private void cancelAllocationFuture(SlotID slotId) {
+		final CompletableFuture<Acknowledge> acknowledgeCompletableFuture = pendingSlotAllocationFutures.remove(slotId);
+		// the future may be null if we are just re-playing the state transitions due to a slot report
+		if (acknowledgeCompletableFuture != null) {
+			acknowledgeCompletableFuture.cancel(false);
+		}
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Component lifecycle methods
+	// ---------------------------------------------------------------------------------------------
+
+	/**
+	 * Starts the slot manager with the given leader id and resource manager actions.
+	 *
+	 * @param newResourceManagerId to use for communication with the task managers
+	 * @param newMainThreadExecutor to use to run code in the ResourceManager's main thread
+	 * @param newResourceActions to use for resource (de-)allocations
+	 */
+	@Override
+	public void start(ResourceManagerId newResourceManagerId, Executor newMainThreadExecutor, ResourceActions newResourceActions) {
+		LOG.info("Starting the slot manager.");
+
+		this.resourceManagerId = Preconditions.checkNotNull(newResourceManagerId);
+		mainThreadExecutor = Preconditions.checkNotNull(newMainThreadExecutor);
+		resourceActions = Preconditions.checkNotNull(newResourceActions);
+		taskExecutorManager = taskExecutorManagerFactory.apply(newMainThreadExecutor, newResourceActions);
+
+		started = true;
+
+		registerSlotManagerMetrics();
+	}
+
+	private void registerSlotManagerMetrics() {
+		slotManagerMetricGroup.gauge(
+			MetricNames.TASK_SLOTS_AVAILABLE,
+			() -> (long) getNumberFreeSlots());
+		slotManagerMetricGroup.gauge(
+			MetricNames.TASK_SLOTS_TOTAL,
+			() -> (long) getNumberRegisteredSlots());
+	}

Review comment:
       not a problem in the sense that things will still work. But we are attempting to register metrics multiple which will cause warnings.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] zentol commented on a change in pull request #13553: [FLINK-19306][coordination] Add DeclarativeSlotManager

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #13553:
URL: https://github.com/apache/flink/pull/13553#discussion_r503338458



##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManagerTest.java
##########
@@ -0,0 +1,1092 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple6;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.concurrent.ManuallyTriggeredScheduledExecutor;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerId;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+import org.apache.flink.runtime.slots.ResourceRequirements;
+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.TestingTaskExecutorGateway;
+import org.apache.flink.runtime.taskexecutor.TestingTaskExecutorGatewayBuilder;
+import org.apache.flink.runtime.taskexecutor.exceptions.SlotAllocationException;
+import org.apache.flink.runtime.taskexecutor.exceptions.SlotOccupiedException;
+import org.apache.flink.runtime.testingUtils.TestingUtils;
+import org.apache.flink.util.FlinkException;
+import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.function.FunctionUtils;
+
+import akka.pattern.AskTimeoutException;
+import org.junit.Ignore;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.hamcrest.CoreMatchers.hasItem;
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.empty;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.hasSize;
+import static org.hamcrest.Matchers.is;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertSame;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Tests for the {@link DeclarativeSlotManager}.
+ */
+public class DeclarativeSlotManagerTest extends TestLogger {
+
+	private static final FlinkException TEST_EXCEPTION = new FlinkException("Test exception");
+
+	private static final WorkerResourceSpec WORKER_RESOURCE_SPEC = new WorkerResourceSpec.Builder()
+		.setCpuCores(100.0)
+		.setTaskHeapMemoryMB(10000)
+		.setTaskOffHeapMemoryMB(10000)
+		.setNetworkMemoryMB(10000)
+		.setManagedMemoryMB(10000)
+		.build();
+
+	/**
+	 * Tests that we can register task manager and their slots at the slot manager.
+	 */
+	@Test
+	public void testTaskManagerRegistration() throws Exception {
+		final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder().createTestingTaskExecutorGateway();
+		final ResourceID resourceId = ResourceID.generate();
+		final TaskExecutorConnection taskManagerConnection = new TaskExecutorConnection(resourceId, taskExecutorGateway);
+
+		final SlotID slotId1 = new SlotID(resourceId, 0);
+		final SlotID slotId2 = new SlotID(resourceId, 1);
+		final SlotReport slotReport = new SlotReport(Arrays.asList(createFreeSlotStatus(slotId1), createFreeSlotStatus(slotId2)));
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport);
+
+			assertThat("The number registered slots does not equal the expected number.", slotManager.getNumberRegisteredSlots(), is(2));
+
+			assertNotNull(slotTracker.getSlot(slotId1));
+			assertNotNull(slotTracker.getSlot(slotId2));
+		}
+	}
+
+	/**
+	 * Tests that un-registration of task managers will free and remove all registered slots.
+	 */
+	@Test
+	public void testTaskManagerUnregistration() throws Exception {
+		final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder()
+			.setRequestSlotFunction(tuple6 -> new CompletableFuture<>())
+			.createTestingTaskExecutorGateway();
+		final TaskExecutorConnection taskManagerConnection = createTaskExecutorConnection(taskExecutorGateway);
+		final ResourceID resourceId = taskManagerConnection.getResourceID();
+
+		final SlotID slotId1 = new SlotID(resourceId, 0);
+		final SlotID slotId2 = new SlotID(resourceId, 1);
+		final SlotReport slotReport = new SlotReport(Arrays.asList(createAllocatedSlotStatus(slotId1), createFreeSlotStatus(slotId2)));
+
+		final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot();
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport);
+
+			assertEquals("The number registered slots does not equal the expected number.", 2, slotManager.getNumberRegisteredSlots());
+
+			DeclarativeTaskManagerSlot slot1 = slotTracker.getSlot(slotId1);
+			DeclarativeTaskManagerSlot slot2 = slotTracker.getSlot(slotId2);
+
+			assertSame(SlotState.ALLOCATED, slot1.getState());
+			assertSame(SlotState.FREE, slot2.getState());
+
+			slotManager.processResourceRequirements(resourceRequirements);
+
+			assertSame(SlotState.PENDING, slot2.getState());
+
+			slotManager.unregisterTaskManager(taskManagerConnection.getInstanceID(), TEST_EXCEPTION);
+
+			assertEquals(0, slotManager.getNumberRegisteredSlots());
+		}
+	}
+
+	/**
+	 * Tests that a slot request with no free slots will trigger the resource allocation.
+	 */
+	@Test
+	public void testSlotRequestWithoutFreeSlots() throws Exception {
+		final ResourceManagerId resourceManagerId = ResourceManagerId.generate();
+
+		final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot();
+
+		CompletableFuture<WorkerResourceSpec> allocateResourceFuture = new CompletableFuture<>();
+		ResourceActions resourceManagerActions = new TestingResourceActionsBuilder()
+			.setAllocateResourceConsumer(allocateResourceFuture::complete)
+			.build();
+
+		try (SlotManager slotManager = createSlotManager(resourceManagerId, resourceManagerActions)) {
+
+			slotManager.processResourceRequirements(resourceRequirements);
+
+			allocateResourceFuture.get();
+		}
+	}
+
+	/**
+	 * Tests that resources continue to be considered missing if we cannot allocate more resources.
+	 */
+	@Test
+	public void testResourceDeclarationWithResourceAllocationFailure() throws Exception {
+		final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot();
+
+		ResourceActions resourceManagerActions = new TestingResourceActionsBuilder()
+			.setAllocateResourceFunction(value -> false)
+			.build();
+
+		final ResourceTracker resourceTracker = new DefaultResourceTracker();
+
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setResourceTracker(resourceTracker)
+			.buildAndStartWithDirectExec(ResourceManagerId.generate(), resourceManagerActions)) {
+
+			slotManager.processResourceRequirements(resourceRequirements);
+
+			final JobID jobId = resourceRequirements.getJobId();
+			assertThat(getTotalResourceCount(resourceTracker.getMissingResources().get(jobId)), is(1));
+		}
+	}
+
+	/**
+	 * Tests that a slot request which can be fulfilled will trigger a slot allocation.
+	 */
+	@Test
+	public void testSlotRequestWithFreeSlot() throws Exception {
+		final ResourceManagerId resourceManagerId = ResourceManagerId.generate();
+		final ResourceID resourceID = ResourceID.generate();
+		final JobID jobId = new JobID();
+		final SlotID slotId = new SlotID(resourceID, 0);
+		final String targetAddress = "localhost";
+		final ResourceProfile resourceProfile = ResourceProfile.fromResources(42.0, 1337);
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec(resourceManagerId, new TestingResourceActionsBuilder().build())) {
+
+			final CompletableFuture<Tuple6<SlotID, JobID, AllocationID, ResourceProfile, String, ResourceManagerId>> requestFuture = new CompletableFuture<>();
+			// accept an incoming slot request
+			final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder()
+				.setRequestSlotFunction(tuple6 -> {
+					requestFuture.complete(Tuple6.of(tuple6.f0, tuple6.f1, tuple6.f2, tuple6.f3, tuple6.f4, tuple6.f5));
+					return CompletableFuture.completedFuture(Acknowledge.get());
+				})
+				.createTestingTaskExecutorGateway();
+
+			final TaskExecutorConnection taskExecutorConnection = new TaskExecutorConnection(resourceID, taskExecutorGateway);
+
+			final SlotStatus slotStatus = new SlotStatus(slotId, resourceProfile);
+			final SlotReport slotReport = new SlotReport(slotStatus);
+
+			slotManager.registerTaskManager(
+				taskExecutorConnection,
+				slotReport);
+
+			ResourceRequirements requirements = ResourceRequirements.create(
+				jobId,
+				targetAddress,
+				Collections.singleton(ResourceRequirement.create(resourceProfile, 1)));
+			slotManager.processResourceRequirements(requirements);
+
+			assertThat(requestFuture.get(), is(equalTo(Tuple6.of(slotId, jobId, requestFuture.get().f2, resourceProfile, targetAddress, resourceManagerId))));
+
+			DeclarativeTaskManagerSlot slot = slotTracker.getSlot(slotId);
+
+			assertEquals("The slot has not been allocated to the expected allocation id.", jobId, slot.getJobId());
+		}
+	}
+
+	/**
+	 * Checks that reducing resource requirements while a slot allocation is in-progress results in the slot being freed
+	 * one the allocation completes.
+	 */
+	@Test
+	@Ignore
+	public void testResourceRequirementReductionDuringAllocation() throws Exception {
+		final TaskExecutorConnection taskManagerConnection = createTaskExecutorConnection();
+		final ResourceID resourceID = taskManagerConnection.getResourceID();
+		final SlotID slotId = new SlotID(resourceID, 0);
+
+		final SlotReport slotReport = new SlotReport(createFreeSlotStatus(slotId));
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport);
+
+			DeclarativeTaskManagerSlot slot = slotTracker.getSlot(slotId);
+
+			final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot();
+			slotManager.processResourceRequirements(resourceRequirements);
+
+			assertSame(SlotState.PENDING, slot.getState());
+
+			slotManager.processResourceRequirements(ResourceRequirements.create(resourceRequirements.getJobId(), "foobar", Collections.emptyList()));
+
+			slot = slotTracker.getSlot(slotId);
+
+			assertSame(SlotState.FREE, slot.getState());
+		}
+	}
+
+	/**
+	 * Tests that pending slot requests are tried to be fulfilled upon new slot registrations.
+	 */
+	@Test
+	public void testFulfillingPendingSlotRequest() throws Exception {
+		final ResourceManagerId resourceManagerId = ResourceManagerId.generate();
+		final ResourceID resourceID = ResourceID.generate();
+		final JobID jobId = new JobID();
+		final SlotID slotId = new SlotID(resourceID, 0);
+		final String targetAddress = "localhost";
+		final ResourceProfile resourceProfile = ResourceProfile.fromResources(42.0, 1337);
+
+		final AtomicInteger numberAllocateResourceCalls = new AtomicInteger(0);
+		ResourceActions resourceManagerActions = new TestingResourceActionsBuilder()
+			.setAllocateResourceConsumer(ignored -> numberAllocateResourceCalls.incrementAndGet())
+			.build();
+
+		final CompletableFuture<Tuple6<SlotID, JobID, AllocationID, ResourceProfile, String, ResourceManagerId>> requestFuture = new CompletableFuture<>();
+		// accept an incoming slot request
+		final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder()
+			.setRequestSlotFunction(tuple6 -> {
+				requestFuture.complete(Tuple6.of(tuple6.f0, tuple6.f1, tuple6.f2, tuple6.f3, tuple6.f4, tuple6.f5));
+				return CompletableFuture.completedFuture(Acknowledge.get());
+			})
+			.createTestingTaskExecutorGateway();
+
+		final TaskExecutorConnection taskExecutorConnection = new TaskExecutorConnection(resourceID, taskExecutorGateway);
+
+		final SlotStatus slotStatus = new SlotStatus(slotId, resourceProfile);
+		final SlotReport slotReport = new SlotReport(slotStatus);
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec(resourceManagerId, resourceManagerActions)) {
+
+			final ResourceRequirements resourceRequirements = ResourceRequirements.create(
+				jobId,
+				targetAddress,
+				Collections.singleton(ResourceRequirement.create(resourceProfile, 1)));
+			slotManager.processResourceRequirements(resourceRequirements);
+
+			assertThat(numberAllocateResourceCalls.get(), is(1));
+
+			slotManager.registerTaskManager(
+				taskExecutorConnection,
+				slotReport);
+
+			assertThat(requestFuture.get(), is(equalTo(Tuple6.of(slotId, jobId, requestFuture.get().f2, resourceProfile, targetAddress, resourceManagerId))));
+
+			DeclarativeTaskManagerSlot slot = slotTracker.getSlot(slotId);
+
+			assertEquals("The slot has not been allocated to the expected job id.", jobId, slot.getJobId());
+		}
+	}
+
+	/**
+	 * Tests that freeing a slot will correctly reset the slot and mark it as a free slot.
+	 */
+	@Test
+	public void testFreeSlot() throws Exception {
+		final TaskExecutorConnection taskExecutorConnection = createTaskExecutorConnection();
+		final ResourceID resourceID = taskExecutorConnection.getResourceID();
+		final SlotID slotId = new SlotID(resourceID, 0);
+
+		final SlotReport slotReport = new SlotReport(createAllocatedSlotStatus(slotId));
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			slotManager.registerTaskManager(
+				taskExecutorConnection,
+				slotReport);
+
+			DeclarativeTaskManagerSlot slot = slotTracker.getSlot(slotId);
+
+			assertSame(SlotState.ALLOCATED, slot.getState());
+
+			slotManager.freeSlot(slotId, new AllocationID());
+
+			assertSame(SlotState.FREE, slot.getState());
+
+			assertEquals(1, slotManager.getNumberFreeSlots());
+		}
+	}
+
+	/**
+	 * Tests that duplicate resource requirement declaration do not result in additional slots being allocated after a
+	 * pending slot request has been fulfilled but not yet freed.
+	 */
+	@Test
+	public void testDuplicateResourceRequirementDeclarationAfterSuccessfulAllocation() throws Exception {
+		final ResourceManagerId resourceManagerId = ResourceManagerId.generate();
+		final AtomicInteger allocateResourceCalls = new AtomicInteger(0);
+		final ResourceActions resourceManagerActions = new TestingResourceActionsBuilder()
+			.setAllocateResourceConsumer(ignored -> allocateResourceCalls.incrementAndGet())
+			.build();
+		ResourceRequirements requirements = createResourceRequirementsForSingleSlot();
+
+		final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder().createTestingTaskExecutorGateway();
+
+		final ResourceID resourceID = ResourceID.generate();
+
+		final TaskExecutorConnection taskManagerConnection = new TaskExecutorConnection(resourceID, taskExecutorGateway);
+
+		final SlotID slotId = new SlotID(resourceID, 0);
+		final SlotReport slotReport = new SlotReport(createFreeSlotStatus(slotId));
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec(resourceManagerId, resourceManagerActions)) {
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport);
+
+			slotManager.processResourceRequirements(requirements);
+
+			DeclarativeTaskManagerSlot slot = slotTracker.getSlot(slotId);
+
+			assertThat(slot.getState(), is(SlotState.ALLOCATED));
+
+			slotManager.processResourceRequirements(requirements);
+		}
+
+		// check that we have only called the resource allocation only for the first slot request,
+		// since the second request is a duplicate
+		assertThat(allocateResourceCalls.get(), is(0));
+	}
+
+	/**
+	 * Tests that a slot allocated for one job can be allocated for another job after being freed.
+	 */
+	@Test
+	public void testSlotCanBeAllocatedForDifferentJobAfterFree() throws Exception {
+		final AllocationID allocationId = new AllocationID();
+		final ResourceRequirements resourceRequirements1 = createResourceRequirementsForSingleSlot();
+		final ResourceRequirements resourceRequirements2 = createResourceRequirementsForSingleSlot();
+
+		final TaskExecutorConnection taskManagerConnection = createTaskExecutorConnection();
+		final ResourceID resourceID = taskManagerConnection.getResourceID();
+
+		final SlotID slotId = new SlotID(resourceID, 0);
+		final SlotReport slotReport = new SlotReport(createFreeSlotStatus(slotId));
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport);
+
+			slotManager.processResourceRequirements(resourceRequirements1);
+
+			DeclarativeTaskManagerSlot slot = slotTracker.getSlot(slotId);
+
+			assertEquals("The slot has not been allocated to the expected job id.", resourceRequirements1.getJobId(), slot.getJobId());
+
+			// clear resource requirements, so that the slot isn't immediately re-assigned
+			slotManager.processResourceRequirements(ResourceRequirements.create(resourceRequirements1.getJobId(), resourceRequirements1.getTargetAddress(), Collections.emptyList()));
+			slotManager.freeSlot(slotId, allocationId);
+
+			// check that the slot has been freed
+			assertSame(SlotState.FREE, slot.getState());
+
+			slotManager.processResourceRequirements(resourceRequirements2);
+
+			assertEquals("The slot has not been allocated to the expected job id.", resourceRequirements2.getJobId(), slot.getJobId());
+		}
+	}

Review comment:
       that's a good idea, will add it




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] tillrohrmann commented on a change in pull request #13553: [FLINK-19306][coordination] Add DeclarativeSlotManager

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #13553:
URL: https://github.com/apache/flink/pull/13553#discussion_r502525136



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java
##########
@@ -861,6 +884,8 @@ protected void closeJobManagerConnection(JobID jobId, Exception cause) {
 
 			jmResourceIdRegistrations.remove(jobManagerResourceId);
 
+			slotManager.processResourceRequirements(ResourceRequirements.empty(jobId, jobMasterGateway.getAddress()));

Review comment:
       What happens in the non-HA case, where the job simply failed? Wouldn't we have to wait for 5 minutes until the resources will be freed for other jobs?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] zentol commented on a change in pull request #13553: [FLINK-19306][coordination] Add DeclarativeSlotManager

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #13553:
URL: https://github.com/apache/flink/pull/13553#discussion_r502343518



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManager.java
##########
@@ -0,0 +1,609 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.metrics.MetricNames;
+import org.apache.flink.runtime.metrics.groups.SlotManagerMetricGroup;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerId;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.slots.ResourceCounter;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+import org.apache.flink.runtime.slots.ResourceRequirements;
+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.SlotOccupiedException;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ScheduledFuture;
+import java.util.function.BiFunction;
+import java.util.stream.Collectors;
+
+/**
+ * Implementation of {@link SlotManager} supporting declarative slot management.
+ */
+public class DeclarativeSlotManager implements SlotManager {
+	private static final Logger LOG = LoggerFactory.getLogger(DeclarativeSlotManager.class);
+
+	private final SlotTracker slotTracker;
+	private final ResourceTracker resourceTracker;
+	private final BiFunction<Executor, ResourceActions, TaskExecutorManager> taskExecutorManagerFactory;
+	private TaskExecutorManager taskExecutorManager;
+
+	/** Timeout for slot requests to the task manager. */
+	private final Time taskManagerRequestTimeout;
+	private ScheduledFuture<?> slotRequestTimeoutCheck;
+
+	private final SlotMatchingStrategy slotMatchingStrategy;
+
+	private final SlotManagerMetricGroup slotManagerMetricGroup;
+
+	private final Map<JobID, String> jobMasterTargetAddresses = new HashMap<>();
+	private final HashMap<SlotID, CompletableFuture<Acknowledge>> pendingSlotAllocationFutures;
+
+	/** ResourceManager's id. */
+	private ResourceManagerId resourceManagerId;
+
+	/** Executor for future callbacks which have to be "synchronized". */
+	private Executor mainThreadExecutor;
+
+	/** Callbacks for resource (de-)allocations. */
+	private ResourceActions resourceActions;
+
+	/** True iff the component has been started. */
+	private boolean started;
+
+	public DeclarativeSlotManager(
+			ScheduledExecutor scheduledExecutor,
+			SlotManagerConfiguration slotManagerConfiguration,
+			SlotManagerMetricGroup slotManagerMetricGroup,
+			ResourceTracker resourceTracker,
+			SlotTracker slotTracker) {
+
+		Preconditions.checkNotNull(slotManagerConfiguration);
+		this.taskManagerRequestTimeout = slotManagerConfiguration.getTaskManagerRequestTimeout();
+		this.slotManagerMetricGroup = Preconditions.checkNotNull(slotManagerMetricGroup);
+		this.resourceTracker = Preconditions.checkNotNull(resourceTracker);
+
+		pendingSlotAllocationFutures = new HashMap<>(16);
+
+		this.slotTracker = Preconditions.checkNotNull(slotTracker);
+		slotTracker.registerSlotStatusUpdateListener(createSlotStatusUpdateListener());
+
+		slotMatchingStrategy = slotManagerConfiguration.getSlotMatchingStrategy();
+
+		taskExecutorManagerFactory = (executor, resourceActions) -> new TaskExecutorManager(
+			slotManagerConfiguration.getDefaultWorkerResourceSpec(),
+			slotManagerConfiguration.getNumSlotsPerWorker(),
+			slotManagerConfiguration.getMaxSlotNum(),
+			slotManagerConfiguration.isWaitResultConsumedBeforeRelease(),
+			slotManagerConfiguration.getRedundantTaskManagerNum(),
+			slotManagerConfiguration.getTaskManagerTimeout(),
+			scheduledExecutor,
+			executor,
+			resourceActions);
+
+		resourceManagerId = null;
+		resourceActions = null;
+		mainThreadExecutor = null;
+		slotRequestTimeoutCheck = null;
+		taskExecutorManager = null;
+
+		started = false;
+	}
+
+	private SlotStatusUpdateListener createSlotStatusUpdateListener() {
+		return (taskManagerSlot, previous, current, jobId) -> {
+			if (previous == SlotState.PENDING) {
+				cancelAllocationFuture(taskManagerSlot.getSlotId());
+			}
+
+			if (current == SlotState.PENDING) {
+				resourceTracker.notifyAcquiredResource(jobId, taskManagerSlot.getResourceProfile());
+			}
+			if (current == SlotState.FREE) {
+				resourceTracker.notifyLostResource(jobId, taskManagerSlot.getResourceProfile());
+			}
+
+			if (current == SlotState.ALLOCATED) {
+				taskExecutorManager.occupySlot(taskManagerSlot.getInstanceId());
+			}
+			if (previous == SlotState.ALLOCATED && current == SlotState.FREE) {
+				taskExecutorManager.freeSlot(taskManagerSlot.getInstanceId());
+			}
+		};
+	}
+
+	private void cancelAllocationFuture(SlotID slotId) {
+		final CompletableFuture<Acknowledge> acknowledgeCompletableFuture = pendingSlotAllocationFutures.remove(slotId);
+		// the future may be null if we are just re-playing the state transitions due to a slot report
+		if (acknowledgeCompletableFuture != null) {
+			acknowledgeCompletableFuture.cancel(false);
+		}
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Component lifecycle methods
+	// ---------------------------------------------------------------------------------------------
+
+	/**
+	 * Starts the slot manager with the given leader id and resource manager actions.
+	 *
+	 * @param newResourceManagerId to use for communication with the task managers
+	 * @param newMainThreadExecutor to use to run code in the ResourceManager's main thread
+	 * @param newResourceActions to use for resource (de-)allocations
+	 */
+	@Override
+	public void start(ResourceManagerId newResourceManagerId, Executor newMainThreadExecutor, ResourceActions newResourceActions) {
+		LOG.info("Starting the slot manager.");
+
+		this.resourceManagerId = Preconditions.checkNotNull(newResourceManagerId);
+		mainThreadExecutor = Preconditions.checkNotNull(newMainThreadExecutor);
+		resourceActions = Preconditions.checkNotNull(newResourceActions);
+		taskExecutorManager = taskExecutorManagerFactory.apply(newMainThreadExecutor, newResourceActions);
+
+		started = true;
+
+		registerSlotManagerMetrics();
+	}
+
+	private void registerSlotManagerMetrics() {
+		slotManagerMetricGroup.gauge(
+			MetricNames.TASK_SLOTS_AVAILABLE,
+			() -> (long) getNumberFreeSlots());
+		slotManagerMetricGroup.gauge(
+			MetricNames.TASK_SLOTS_TOTAL,
+			() -> (long) getNumberRegisteredSlots());
+	}
+
+	/**
+	 * Suspends the component. This clears the internal state of the slot manager.
+	 */
+	@Override
+	public void suspend() {
+		LOG.info("Suspending the slot manager.");
+
+		if (slotRequestTimeoutCheck != null) {
+			slotRequestTimeoutCheck.cancel(false);
+			slotRequestTimeoutCheck = null;
+		}
+
+		resourceTracker.clear();
+		taskExecutorManager.close();
+
+		for (InstanceID registeredTaskManager : taskExecutorManager.getTaskExecutors()) {
+			unregisterTaskManager(registeredTaskManager, new SlotManagerException("The slot manager is being suspended."));
+		}
+
+		taskExecutorManager = null;
+		resourceManagerId = null;
+		resourceActions = null;
+		started = false;
+	}
+
+	/**
+	 * Closes the slot manager.
+	 *
+	 * @throws Exception if the close operation fails
+	 */
+	@Override
+	public void close() throws Exception {
+		LOG.info("Closing the slot manager.");
+
+		suspend();
+		slotManagerMetricGroup.close();
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Public API
+	// ---------------------------------------------------------------------------------------------
+
+	@Override
+	public void processResourceRequirements(ResourceRequirements resourceRequirements) {
+		checkInit();
+		LOG.debug("Received resource requirements from job {}: {}", resourceRequirements.getJobId(), resourceRequirements.getResourceRequirements());
+
+		if (resourceRequirements.getResourceRequirements().isEmpty()) {
+			jobMasterTargetAddresses.remove(resourceRequirements.getJobId());
+		} else {
+			jobMasterTargetAddresses.put(resourceRequirements.getJobId(), resourceRequirements.getTargetAddress());
+		}
+		resourceTracker.notifyResourceRequirements(resourceRequirements.getJobId(), resourceRequirements.getResourceRequirements());
+		checkResourceRequirements();
+	}
+
+	/**
+	 * 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
+	 * @return True if the task manager has not been registered before and is registered successfully; otherwise false
+	 */
+	@Override
+	public boolean registerTaskManager(final TaskExecutorConnection taskExecutorConnection, SlotReport initialSlotReport) {
+		checkInit();
+		LOG.debug("Registering task executor {} under {} at the slot manager.", taskExecutorConnection.getResourceID(), taskExecutorConnection.getInstanceID());
+
+		// we identify task managers by their instance id
+		if (taskExecutorManager.isTaskManagerRegistered(taskExecutorConnection.getInstanceID())) {
+			LOG.debug("Task executor {} was already registered.", taskExecutorConnection.getResourceID());
+			reportSlotStatus(taskExecutorConnection.getInstanceID(), initialSlotReport);
+			return false;
+		} else {
+			if (!taskExecutorManager.registerTaskManager(taskExecutorConnection, initialSlotReport)) {
+				LOG.debug("Task executor {} could not be registered.", taskExecutorConnection.getResourceID());
+				return false;
+			}
+
+			// register the new slots
+			for (SlotStatus slotStatus : initialSlotReport) {
+				slotTracker.addSlot(
+					slotStatus.getSlotID(),
+					slotStatus.getResourceProfile(),
+					taskExecutorConnection,
+					slotStatus.getJobID());
+			}
+
+			checkResourceRequirements();
+			return true;
+		}
+	}
+
+	@Override
+	public boolean unregisterTaskManager(InstanceID instanceId, Exception cause) {
+		checkInit();
+
+		LOG.debug("Unregistering task executor {} from the slot manager.", instanceId);
+
+		if (taskExecutorManager.isTaskManagerRegistered(instanceId)) {
+			slotTracker.removeSlots(taskExecutorManager.getSlotsOf(instanceId));
+			taskExecutorManager.unregisterTaskExecutor(instanceId);
+			checkResourceRequirements();
+
+			return true;
+		} else {
+			LOG.debug("There is no task executor registered with instance ID {}. Ignoring this message.", instanceId);
+
+			return false;
+		}
+	}
+
+	/**
+	 * Reports the current slot allocations for a task manager identified by the given instance id.
+	 *
+	 * @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
+	 */
+	@Override
+	public boolean reportSlotStatus(InstanceID instanceId, SlotReport slotReport) {
+		checkInit();
+
+		LOG.debug("Received slot report from instance {}: {}.", instanceId, slotReport);
+
+		if (taskExecutorManager.isTaskManagerRegistered(instanceId)) {
+			slotTracker.notifySlotStatus(slotReport);
+			checkResourceRequirements();
+			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
+	 */
+	@Override
+	public void freeSlot(SlotID slotId, AllocationID allocationId) {
+		checkInit();
+		LOG.debug("Freeing slot {}.", slotId);
+
+		slotTracker.notifyFree(slotId);
+		checkResourceRequirements();
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Requirement matching
+	// ---------------------------------------------------------------------------------------------
+
+	private void checkResourceRequirements() {
+		final Map<JobID, Collection<ResourceRequirement>> missingResources = resourceTracker.getMissingResources();
+		if (missingResources.isEmpty()) {
+			return;
+		}
+
+		final Map<JobID, ResourceCounter> outstandingRequirements = new LinkedHashMap<>();
+		for (Map.Entry<JobID, Collection<ResourceRequirement>> resourceRequirements : missingResources.entrySet()) {
+			JobID jobId = resourceRequirements.getKey();
+
+			for (ResourceRequirement resourceRequirement : resourceRequirements.getValue()) {
+				int numMissingSlots = internalRequestSlots(jobId, jobMasterTargetAddresses.get(jobId), resourceRequirement);
+				if (numMissingSlots > 0) {
+					outstandingRequirements
+						.computeIfAbsent(jobId, ignored -> new ResourceCounter())
+						.incrementCount(resourceRequirement.getResourceProfile(), numMissingSlots);
+				}
+			}
+		}
+
+		final ResourceCounter pendingSlots = new ResourceCounter(taskExecutorManager.getPendingTaskManagerSlots().stream().collect(
+			Collectors.groupingBy(
+				PendingTaskManagerSlot::getResourceProfile,
+				Collectors.summingInt(x -> 1))));
+
+		for (Map.Entry<JobID, ResourceCounter> unfulfilledRequirement : outstandingRequirements.entrySet()) {
+			tryFulfillRequirementsWithPendingOrNewSlots(
+				unfulfilledRequirement.getKey(),
+				unfulfilledRequirement.getValue().getResourceProfilesWithCount(),
+				pendingSlots);
+		}
+	}
+
+	/**
+	 * Tries to allocate slots for the given requirement. If there are not enough slots available, the
+	 * resource manager is informed to allocate more resources.
+	 *
+	 * @param jobId job to allocate slots for
+	 * @param targetAddress address of the jobmaster
+	 * @param resourceRequirement required slots
+	 * @return the number of missing slots
+	 */
+	private int internalRequestSlots(JobID jobId, String targetAddress, ResourceRequirement resourceRequirement) {
+		final ResourceProfile requiredResource = resourceRequirement.getResourceProfile();
+		Collection<TaskManagerSlotInformation> freeSlots = slotTracker.getFreeSlots();
+
+		int numUnfulfilled = 0;
+		for (int x = 0; x < resourceRequirement.getNumberOfRequiredSlots(); x++) {
+
+			final Optional<TaskManagerSlotInformation> reservedSlot = slotMatchingStrategy.findMatchingSlot(requiredResource, freeSlots, this::getNumberRegisteredSlotsOf);
+			if (reservedSlot.isPresent()) {
+				// we do not need to modify freeSlots because it is indirectly modified by the allocation
+				allocateSlot(reservedSlot.get(), jobId, targetAddress, requiredResource);
+			} else {
+				numUnfulfilled++;
+			}
+		}
+		return numUnfulfilled;
+	}
+
+	/**
+	 * Allocates the given slot. This entails sending a registration message to the task manager and treating failures.
+	 *
+	 * @param taskManagerSlot slot to allocate
+	 * @param jobId job for which the slot should be allocated for
+	 * @param targetAddress address of the job master
+	 * @param resourceProfile resource profile for the requirement for which the slot is used
+	 */
+	private void allocateSlot(TaskManagerSlotInformation taskManagerSlot, JobID jobId, String targetAddress, ResourceProfile resourceProfile) {
+		TaskExecutorConnection taskExecutorConnection = taskManagerSlot.getTaskManagerConnection();
+		TaskExecutorGateway gateway = taskExecutorConnection.getTaskExecutorGateway();
+
+		final CompletableFuture<Acknowledge> completableFuture = new CompletableFuture<>();
+		final SlotID slotId = taskManagerSlot.getSlotId();
+		final InstanceID instanceId = taskManagerSlot.getInstanceId();
+
+		LOG.debug("Starting allocation of slot {} for job {} with resource profile {}.", taskManagerSlot.getSlotId(), jobId, resourceProfile);
+
+		slotTracker.notifyAllocationStart(slotId, jobId);
+
+		if (!taskExecutorManager.isTaskManagerRegistered(instanceId)) {
+			throw new IllegalStateException("Could not find a registered task manager for instance id " +
+				instanceId + '.');
+		}
+
+		taskExecutorManager.markUsed(instanceId);
+
+		// RPC call to the task manager
+		CompletableFuture<Acknowledge> requestFuture = gateway.requestSlot(
+			slotId,
+			jobId,
+			new AllocationID(),
+			resourceProfile,
+			targetAddress,
+			resourceManagerId,
+			taskManagerRequestTimeout);
+
+		requestFuture.whenComplete(
+			(Acknowledge acknowledge, Throwable throwable) -> {
+				if (acknowledge != null) {
+					completableFuture.complete(acknowledge);
+				} else {
+					completableFuture.completeExceptionally(throwable);
+				}
+			});
+
+		completableFuture.whenCompleteAsync(
+			(Acknowledge acknowledge, Throwable throwable) -> {
+				try {
+					if (acknowledge != null) {
+						LOG.trace("Completed allocation of slot {} for job {}.", slotId, jobId);
+						slotTracker.notifyAllocationComplete(slotId, jobId);
+					} else {
+						if (throwable instanceof SlotOccupiedException) {
+							SlotOccupiedException exception = (SlotOccupiedException) throwable;
+							LOG.debug("Tried allocating slot {} for job {}, but it was already allocated for job {}.", slotId, jobId, exception.getJobId());
+							// report as a slot status to force the state transition
+							// this could be a problem if we ever assume that the task executor always reports about all slots
+							slotTracker.notifySlotStatus(Collections.singleton(new SlotStatus(slotId, taskManagerSlot.getResourceProfile(), exception.getJobId(), exception.getAllocationId())));
+						} else {
+							if (throwable instanceof CancellationException) {
+								LOG.debug("Cancelled allocation of slot {} for job {}.", slotId, jobId, throwable);
+							} else {
+								LOG.warn("Slot allocation for slot {} for job {} failed.", slotId, jobId, throwable);
+								slotTracker.notifyFree(slotId);
+							}
+						}
+					}
+					checkResourceRequirements();
+				} catch (Exception e) {
+					LOG.error("Error while completing the slot allocation.", e);
+				}

Review comment:
       This was just taken as is from the existing `SlotManagerImpl`. I can change it though.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] tillrohrmann commented on a change in pull request #13553: [FLINK-19306][coordination] Add DeclarativeSlotManager

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #13553:
URL: https://github.com/apache/flink/pull/13553#discussion_r501720427



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java
##########
@@ -861,6 +884,8 @@ protected void closeJobManagerConnection(JobID jobId, Exception cause) {
 
 			jmResourceIdRegistrations.remove(jobManagerResourceId);
 
+			slotManager.processResourceRequirements(ResourceRequirements.empty(jobId, jobMasterGateway.getAddress()));

Review comment:
       Are you sure that we want to decrease the requirements of a job if the `JobManager` connection is being closed? I was wondering whether this shouldn't happen if we remove the job in `ResourceManager.removeJob`?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManager.java
##########
@@ -0,0 +1,609 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.metrics.MetricNames;
+import org.apache.flink.runtime.metrics.groups.SlotManagerMetricGroup;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerId;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.slots.ResourceCounter;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+import org.apache.flink.runtime.slots.ResourceRequirements;
+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.SlotOccupiedException;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ScheduledFuture;
+import java.util.function.BiFunction;
+import java.util.stream.Collectors;
+
+/**
+ * Implementation of {@link SlotManager} supporting declarative slot management.
+ */
+public class DeclarativeSlotManager implements SlotManager {
+	private static final Logger LOG = LoggerFactory.getLogger(DeclarativeSlotManager.class);
+
+	private final SlotTracker slotTracker;
+	private final ResourceTracker resourceTracker;
+	private final BiFunction<Executor, ResourceActions, TaskExecutorManager> taskExecutorManagerFactory;
+	private TaskExecutorManager taskExecutorManager;
+
+	/** Timeout for slot requests to the task manager. */
+	private final Time taskManagerRequestTimeout;
+	private ScheduledFuture<?> slotRequestTimeoutCheck;
+
+	private final SlotMatchingStrategy slotMatchingStrategy;
+
+	private final SlotManagerMetricGroup slotManagerMetricGroup;
+
+	private final Map<JobID, String> jobMasterTargetAddresses = new HashMap<>();
+	private final HashMap<SlotID, CompletableFuture<Acknowledge>> pendingSlotAllocationFutures;
+
+	/** ResourceManager's id. */
+	private ResourceManagerId resourceManagerId;

Review comment:
       Since these fields can be `null`, we might wanna add `@Nullable`.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManager.java
##########
@@ -0,0 +1,609 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.metrics.MetricNames;
+import org.apache.flink.runtime.metrics.groups.SlotManagerMetricGroup;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerId;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.slots.ResourceCounter;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+import org.apache.flink.runtime.slots.ResourceRequirements;
+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.SlotOccupiedException;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ScheduledFuture;
+import java.util.function.BiFunction;
+import java.util.stream.Collectors;
+
+/**
+ * Implementation of {@link SlotManager} supporting declarative slot management.
+ */
+public class DeclarativeSlotManager implements SlotManager {
+	private static final Logger LOG = LoggerFactory.getLogger(DeclarativeSlotManager.class);
+
+	private final SlotTracker slotTracker;
+	private final ResourceTracker resourceTracker;
+	private final BiFunction<Executor, ResourceActions, TaskExecutorManager> taskExecutorManagerFactory;
+	private TaskExecutorManager taskExecutorManager;
+
+	/** Timeout for slot requests to the task manager. */
+	private final Time taskManagerRequestTimeout;
+	private ScheduledFuture<?> slotRequestTimeoutCheck;
+
+	private final SlotMatchingStrategy slotMatchingStrategy;
+
+	private final SlotManagerMetricGroup slotManagerMetricGroup;
+
+	private final Map<JobID, String> jobMasterTargetAddresses = new HashMap<>();
+	private final HashMap<SlotID, CompletableFuture<Acknowledge>> pendingSlotAllocationFutures;
+
+	/** ResourceManager's id. */
+	private ResourceManagerId resourceManagerId;
+
+	/** Executor for future callbacks which have to be "synchronized". */
+	private Executor mainThreadExecutor;
+
+	/** Callbacks for resource (de-)allocations. */
+	private ResourceActions resourceActions;
+
+	/** True iff the component has been started. */
+	private boolean started;
+
+	public DeclarativeSlotManager(
+			ScheduledExecutor scheduledExecutor,
+			SlotManagerConfiguration slotManagerConfiguration,
+			SlotManagerMetricGroup slotManagerMetricGroup,
+			ResourceTracker resourceTracker,
+			SlotTracker slotTracker) {
+
+		Preconditions.checkNotNull(slotManagerConfiguration);
+		this.taskManagerRequestTimeout = slotManagerConfiguration.getTaskManagerRequestTimeout();
+		this.slotManagerMetricGroup = Preconditions.checkNotNull(slotManagerMetricGroup);
+		this.resourceTracker = Preconditions.checkNotNull(resourceTracker);
+
+		pendingSlotAllocationFutures = new HashMap<>(16);
+
+		this.slotTracker = Preconditions.checkNotNull(slotTracker);
+		slotTracker.registerSlotStatusUpdateListener(createSlotStatusUpdateListener());
+
+		slotMatchingStrategy = slotManagerConfiguration.getSlotMatchingStrategy();
+
+		taskExecutorManagerFactory = (executor, resourceActions) -> new TaskExecutorManager(
+			slotManagerConfiguration.getDefaultWorkerResourceSpec(),
+			slotManagerConfiguration.getNumSlotsPerWorker(),
+			slotManagerConfiguration.getMaxSlotNum(),
+			slotManagerConfiguration.isWaitResultConsumedBeforeRelease(),
+			slotManagerConfiguration.getRedundantTaskManagerNum(),
+			slotManagerConfiguration.getTaskManagerTimeout(),
+			scheduledExecutor,
+			executor,
+			resourceActions);
+
+		resourceManagerId = null;
+		resourceActions = null;
+		mainThreadExecutor = null;
+		slotRequestTimeoutCheck = null;
+		taskExecutorManager = null;
+
+		started = false;
+	}
+
+	private SlotStatusUpdateListener createSlotStatusUpdateListener() {
+		return (taskManagerSlot, previous, current, jobId) -> {
+			if (previous == SlotState.PENDING) {
+				cancelAllocationFuture(taskManagerSlot.getSlotId());
+			}
+
+			if (current == SlotState.PENDING) {
+				resourceTracker.notifyAcquiredResource(jobId, taskManagerSlot.getResourceProfile());
+			}
+			if (current == SlotState.FREE) {
+				resourceTracker.notifyLostResource(jobId, taskManagerSlot.getResourceProfile());
+			}
+
+			if (current == SlotState.ALLOCATED) {
+				taskExecutorManager.occupySlot(taskManagerSlot.getInstanceId());
+			}
+			if (previous == SlotState.ALLOCATED && current == SlotState.FREE) {
+				taskExecutorManager.freeSlot(taskManagerSlot.getInstanceId());
+			}
+		};
+	}
+
+	private void cancelAllocationFuture(SlotID slotId) {
+		final CompletableFuture<Acknowledge> acknowledgeCompletableFuture = pendingSlotAllocationFutures.remove(slotId);
+		// the future may be null if we are just re-playing the state transitions due to a slot report
+		if (acknowledgeCompletableFuture != null) {
+			acknowledgeCompletableFuture.cancel(false);
+		}
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Component lifecycle methods
+	// ---------------------------------------------------------------------------------------------
+
+	/**
+	 * Starts the slot manager with the given leader id and resource manager actions.
+	 *
+	 * @param newResourceManagerId to use for communication with the task managers
+	 * @param newMainThreadExecutor to use to run code in the ResourceManager's main thread
+	 * @param newResourceActions to use for resource (de-)allocations
+	 */
+	@Override
+	public void start(ResourceManagerId newResourceManagerId, Executor newMainThreadExecutor, ResourceActions newResourceActions) {
+		LOG.info("Starting the slot manager.");
+
+		this.resourceManagerId = Preconditions.checkNotNull(newResourceManagerId);
+		mainThreadExecutor = Preconditions.checkNotNull(newMainThreadExecutor);
+		resourceActions = Preconditions.checkNotNull(newResourceActions);
+		taskExecutorManager = taskExecutorManagerFactory.apply(newMainThreadExecutor, newResourceActions);
+
+		started = true;
+
+		registerSlotManagerMetrics();
+	}
+
+	private void registerSlotManagerMetrics() {
+		slotManagerMetricGroup.gauge(
+			MetricNames.TASK_SLOTS_AVAILABLE,
+			() -> (long) getNumberFreeSlots());
+		slotManagerMetricGroup.gauge(
+			MetricNames.TASK_SLOTS_TOTAL,
+			() -> (long) getNumberRegisteredSlots());
+	}
+
+	/**
+	 * Suspends the component. This clears the internal state of the slot manager.
+	 */
+	@Override
+	public void suspend() {
+		LOG.info("Suspending the slot manager.");
+
+		if (slotRequestTimeoutCheck != null) {
+			slotRequestTimeoutCheck.cancel(false);
+			slotRequestTimeoutCheck = null;
+		}
+
+		resourceTracker.clear();
+		taskExecutorManager.close();
+
+		for (InstanceID registeredTaskManager : taskExecutorManager.getTaskExecutors()) {
+			unregisterTaskManager(registeredTaskManager, new SlotManagerException("The slot manager is being suspended."));
+		}
+
+		taskExecutorManager = null;
+		resourceManagerId = null;
+		resourceActions = null;
+		started = false;
+	}
+
+	/**
+	 * Closes the slot manager.
+	 *
+	 * @throws Exception if the close operation fails
+	 */
+	@Override
+	public void close() throws Exception {
+		LOG.info("Closing the slot manager.");
+
+		suspend();
+		slotManagerMetricGroup.close();
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Public API
+	// ---------------------------------------------------------------------------------------------
+
+	@Override
+	public void processResourceRequirements(ResourceRequirements resourceRequirements) {
+		checkInit();
+		LOG.debug("Received resource requirements from job {}: {}", resourceRequirements.getJobId(), resourceRequirements.getResourceRequirements());
+
+		if (resourceRequirements.getResourceRequirements().isEmpty()) {
+			jobMasterTargetAddresses.remove(resourceRequirements.getJobId());
+		} else {
+			jobMasterTargetAddresses.put(resourceRequirements.getJobId(), resourceRequirements.getTargetAddress());
+		}
+		resourceTracker.notifyResourceRequirements(resourceRequirements.getJobId(), resourceRequirements.getResourceRequirements());
+		checkResourceRequirements();
+	}
+
+	/**
+	 * 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
+	 * @return True if the task manager has not been registered before and is registered successfully; otherwise false
+	 */
+	@Override
+	public boolean registerTaskManager(final TaskExecutorConnection taskExecutorConnection, SlotReport initialSlotReport) {
+		checkInit();
+		LOG.debug("Registering task executor {} under {} at the slot manager.", taskExecutorConnection.getResourceID(), taskExecutorConnection.getInstanceID());
+
+		// we identify task managers by their instance id
+		if (taskExecutorManager.isTaskManagerRegistered(taskExecutorConnection.getInstanceID())) {
+			LOG.debug("Task executor {} was already registered.", taskExecutorConnection.getResourceID());
+			reportSlotStatus(taskExecutorConnection.getInstanceID(), initialSlotReport);
+			return false;
+		} else {
+			if (!taskExecutorManager.registerTaskManager(taskExecutorConnection, initialSlotReport)) {
+				LOG.debug("Task executor {} could not be registered.", taskExecutorConnection.getResourceID());
+				return false;
+			}
+
+			// register the new slots
+			for (SlotStatus slotStatus : initialSlotReport) {
+				slotTracker.addSlot(
+					slotStatus.getSlotID(),
+					slotStatus.getResourceProfile(),
+					taskExecutorConnection,
+					slotStatus.getJobID());
+			}
+
+			checkResourceRequirements();
+			return true;
+		}
+	}
+
+	@Override
+	public boolean unregisterTaskManager(InstanceID instanceId, Exception cause) {
+		checkInit();
+
+		LOG.debug("Unregistering task executor {} from the slot manager.", instanceId);
+
+		if (taskExecutorManager.isTaskManagerRegistered(instanceId)) {
+			slotTracker.removeSlots(taskExecutorManager.getSlotsOf(instanceId));
+			taskExecutorManager.unregisterTaskExecutor(instanceId);
+			checkResourceRequirements();
+
+			return true;
+		} else {
+			LOG.debug("There is no task executor registered with instance ID {}. Ignoring this message.", instanceId);
+
+			return false;
+		}
+	}
+
+	/**
+	 * Reports the current slot allocations for a task manager identified by the given instance id.
+	 *
+	 * @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
+	 */
+	@Override
+	public boolean reportSlotStatus(InstanceID instanceId, SlotReport slotReport) {
+		checkInit();
+
+		LOG.debug("Received slot report from instance {}: {}.", instanceId, slotReport);
+
+		if (taskExecutorManager.isTaskManagerRegistered(instanceId)) {
+			slotTracker.notifySlotStatus(slotReport);
+			checkResourceRequirements();
+			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
+	 */
+	@Override
+	public void freeSlot(SlotID slotId, AllocationID allocationId) {
+		checkInit();
+		LOG.debug("Freeing slot {}.", slotId);
+
+		slotTracker.notifyFree(slotId);
+		checkResourceRequirements();
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Requirement matching
+	// ---------------------------------------------------------------------------------------------
+
+	private void checkResourceRequirements() {
+		final Map<JobID, Collection<ResourceRequirement>> missingResources = resourceTracker.getMissingResources();
+		if (missingResources.isEmpty()) {
+			return;
+		}
+
+		final Map<JobID, ResourceCounter> outstandingRequirements = new LinkedHashMap<>();
+		for (Map.Entry<JobID, Collection<ResourceRequirement>> resourceRequirements : missingResources.entrySet()) {
+			JobID jobId = resourceRequirements.getKey();
+
+			for (ResourceRequirement resourceRequirement : resourceRequirements.getValue()) {
+				int numMissingSlots = internalRequestSlots(jobId, jobMasterTargetAddresses.get(jobId), resourceRequirement);
+				if (numMissingSlots > 0) {
+					outstandingRequirements
+						.computeIfAbsent(jobId, ignored -> new ResourceCounter())
+						.incrementCount(resourceRequirement.getResourceProfile(), numMissingSlots);
+				}
+			}
+		}
+
+		final ResourceCounter pendingSlots = new ResourceCounter(taskExecutorManager.getPendingTaskManagerSlots().stream().collect(
+			Collectors.groupingBy(
+				PendingTaskManagerSlot::getResourceProfile,
+				Collectors.summingInt(x -> 1))));
+
+		for (Map.Entry<JobID, ResourceCounter> unfulfilledRequirement : outstandingRequirements.entrySet()) {
+			tryFulfillRequirementsWithPendingOrNewSlots(
+				unfulfilledRequirement.getKey(),
+				unfulfilledRequirement.getValue().getResourceProfilesWithCount(),
+				pendingSlots);
+		}
+	}
+
+	/**
+	 * Tries to allocate slots for the given requirement. If there are not enough slots available, the
+	 * resource manager is informed to allocate more resources.
+	 *
+	 * @param jobId job to allocate slots for
+	 * @param targetAddress address of the jobmaster
+	 * @param resourceRequirement required slots
+	 * @return the number of missing slots
+	 */
+	private int internalRequestSlots(JobID jobId, String targetAddress, ResourceRequirement resourceRequirement) {
+		final ResourceProfile requiredResource = resourceRequirement.getResourceProfile();
+		Collection<TaskManagerSlotInformation> freeSlots = slotTracker.getFreeSlots();
+
+		int numUnfulfilled = 0;
+		for (int x = 0; x < resourceRequirement.getNumberOfRequiredSlots(); x++) {
+
+			final Optional<TaskManagerSlotInformation> reservedSlot = slotMatchingStrategy.findMatchingSlot(requiredResource, freeSlots, this::getNumberRegisteredSlotsOf);
+			if (reservedSlot.isPresent()) {
+				// we do not need to modify freeSlots because it is indirectly modified by the allocation
+				allocateSlot(reservedSlot.get(), jobId, targetAddress, requiredResource);
+			} else {
+				numUnfulfilled++;
+			}
+		}
+		return numUnfulfilled;
+	}
+
+	/**
+	 * Allocates the given slot. This entails sending a registration message to the task manager and treating failures.
+	 *
+	 * @param taskManagerSlot slot to allocate
+	 * @param jobId job for which the slot should be allocated for
+	 * @param targetAddress address of the job master
+	 * @param resourceProfile resource profile for the requirement for which the slot is used
+	 */
+	private void allocateSlot(TaskManagerSlotInformation taskManagerSlot, JobID jobId, String targetAddress, ResourceProfile resourceProfile) {
+		TaskExecutorConnection taskExecutorConnection = taskManagerSlot.getTaskManagerConnection();
+		TaskExecutorGateway gateway = taskExecutorConnection.getTaskExecutorGateway();
+
+		final CompletableFuture<Acknowledge> completableFuture = new CompletableFuture<>();
+		final SlotID slotId = taskManagerSlot.getSlotId();
+		final InstanceID instanceId = taskManagerSlot.getInstanceId();
+
+		LOG.debug("Starting allocation of slot {} for job {} with resource profile {}.", taskManagerSlot.getSlotId(), jobId, resourceProfile);
+
+		slotTracker.notifyAllocationStart(slotId, jobId);
+
+		if (!taskExecutorManager.isTaskManagerRegistered(instanceId)) {
+			throw new IllegalStateException("Could not find a registered task manager for instance id " +
+				instanceId + '.');
+		}
+
+		taskExecutorManager.markUsed(instanceId);
+
+		// RPC call to the task manager
+		CompletableFuture<Acknowledge> requestFuture = gateway.requestSlot(
+			slotId,
+			jobId,
+			new AllocationID(),
+			resourceProfile,
+			targetAddress,
+			resourceManagerId,
+			taskManagerRequestTimeout);
+
+		requestFuture.whenComplete(
+			(Acknowledge acknowledge, Throwable throwable) -> {
+				if (acknowledge != null) {
+					completableFuture.complete(acknowledge);
+				} else {
+					completableFuture.completeExceptionally(throwable);
+				}
+			});
+
+		completableFuture.whenCompleteAsync(
+			(Acknowledge acknowledge, Throwable throwable) -> {
+				try {
+					if (acknowledge != null) {
+						LOG.trace("Completed allocation of slot {} for job {}.", slotId, jobId);
+						slotTracker.notifyAllocationComplete(slotId, jobId);
+					} else {
+						if (throwable instanceof SlotOccupiedException) {
+							SlotOccupiedException exception = (SlotOccupiedException) throwable;
+							LOG.debug("Tried allocating slot {} for job {}, but it was already allocated for job {}.", slotId, jobId, exception.getJobId());
+							// report as a slot status to force the state transition
+							// this could be a problem if we ever assume that the task executor always reports about all slots
+							slotTracker.notifySlotStatus(Collections.singleton(new SlotStatus(slotId, taskManagerSlot.getResourceProfile(), exception.getJobId(), exception.getAllocationId())));
+						} else {
+							if (throwable instanceof CancellationException) {
+								LOG.debug("Cancelled allocation of slot {} for job {}.", slotId, jobId, throwable);
+							} else {
+								LOG.warn("Slot allocation for slot {} for job {} failed.", slotId, jobId, throwable);
+								slotTracker.notifyFree(slotId);
+							}
+						}
+					}
+					checkResourceRequirements();

Review comment:
       Should we also call `checkResourceRequirements` in case of a successful allocation? Shouldn't we only call this method if the allocation failed for some reason and we need to find another slot?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManager.java
##########
@@ -0,0 +1,609 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.metrics.MetricNames;
+import org.apache.flink.runtime.metrics.groups.SlotManagerMetricGroup;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerId;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.slots.ResourceCounter;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+import org.apache.flink.runtime.slots.ResourceRequirements;
+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.SlotOccupiedException;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ScheduledFuture;
+import java.util.function.BiFunction;
+import java.util.stream.Collectors;
+
+/**
+ * Implementation of {@link SlotManager} supporting declarative slot management.
+ */
+public class DeclarativeSlotManager implements SlotManager {
+	private static final Logger LOG = LoggerFactory.getLogger(DeclarativeSlotManager.class);
+
+	private final SlotTracker slotTracker;
+	private final ResourceTracker resourceTracker;
+	private final BiFunction<Executor, ResourceActions, TaskExecutorManager> taskExecutorManagerFactory;
+	private TaskExecutorManager taskExecutorManager;
+
+	/** Timeout for slot requests to the task manager. */
+	private final Time taskManagerRequestTimeout;
+	private ScheduledFuture<?> slotRequestTimeoutCheck;
+
+	private final SlotMatchingStrategy slotMatchingStrategy;
+
+	private final SlotManagerMetricGroup slotManagerMetricGroup;
+
+	private final Map<JobID, String> jobMasterTargetAddresses = new HashMap<>();
+	private final HashMap<SlotID, CompletableFuture<Acknowledge>> pendingSlotAllocationFutures;
+
+	/** ResourceManager's id. */
+	private ResourceManagerId resourceManagerId;
+
+	/** Executor for future callbacks which have to be "synchronized". */
+	private Executor mainThreadExecutor;
+
+	/** Callbacks for resource (de-)allocations. */
+	private ResourceActions resourceActions;
+
+	/** True iff the component has been started. */
+	private boolean started;
+
+	public DeclarativeSlotManager(
+			ScheduledExecutor scheduledExecutor,
+			SlotManagerConfiguration slotManagerConfiguration,
+			SlotManagerMetricGroup slotManagerMetricGroup,
+			ResourceTracker resourceTracker,
+			SlotTracker slotTracker) {
+
+		Preconditions.checkNotNull(slotManagerConfiguration);
+		this.taskManagerRequestTimeout = slotManagerConfiguration.getTaskManagerRequestTimeout();
+		this.slotManagerMetricGroup = Preconditions.checkNotNull(slotManagerMetricGroup);
+		this.resourceTracker = Preconditions.checkNotNull(resourceTracker);
+
+		pendingSlotAllocationFutures = new HashMap<>(16);
+
+		this.slotTracker = Preconditions.checkNotNull(slotTracker);
+		slotTracker.registerSlotStatusUpdateListener(createSlotStatusUpdateListener());
+
+		slotMatchingStrategy = slotManagerConfiguration.getSlotMatchingStrategy();
+
+		taskExecutorManagerFactory = (executor, resourceActions) -> new TaskExecutorManager(
+			slotManagerConfiguration.getDefaultWorkerResourceSpec(),
+			slotManagerConfiguration.getNumSlotsPerWorker(),
+			slotManagerConfiguration.getMaxSlotNum(),
+			slotManagerConfiguration.isWaitResultConsumedBeforeRelease(),
+			slotManagerConfiguration.getRedundantTaskManagerNum(),
+			slotManagerConfiguration.getTaskManagerTimeout(),
+			scheduledExecutor,
+			executor,
+			resourceActions);
+
+		resourceManagerId = null;
+		resourceActions = null;
+		mainThreadExecutor = null;
+		slotRequestTimeoutCheck = null;
+		taskExecutorManager = null;
+
+		started = false;
+	}
+
+	private SlotStatusUpdateListener createSlotStatusUpdateListener() {
+		return (taskManagerSlot, previous, current, jobId) -> {
+			if (previous == SlotState.PENDING) {
+				cancelAllocationFuture(taskManagerSlot.getSlotId());
+			}
+
+			if (current == SlotState.PENDING) {
+				resourceTracker.notifyAcquiredResource(jobId, taskManagerSlot.getResourceProfile());
+			}
+			if (current == SlotState.FREE) {
+				resourceTracker.notifyLostResource(jobId, taskManagerSlot.getResourceProfile());
+			}
+
+			if (current == SlotState.ALLOCATED) {
+				taskExecutorManager.occupySlot(taskManagerSlot.getInstanceId());
+			}
+			if (previous == SlotState.ALLOCATED && current == SlotState.FREE) {
+				taskExecutorManager.freeSlot(taskManagerSlot.getInstanceId());
+			}
+		};
+	}
+
+	private void cancelAllocationFuture(SlotID slotId) {
+		final CompletableFuture<Acknowledge> acknowledgeCompletableFuture = pendingSlotAllocationFutures.remove(slotId);
+		// the future may be null if we are just re-playing the state transitions due to a slot report
+		if (acknowledgeCompletableFuture != null) {
+			acknowledgeCompletableFuture.cancel(false);
+		}
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Component lifecycle methods
+	// ---------------------------------------------------------------------------------------------
+
+	/**
+	 * Starts the slot manager with the given leader id and resource manager actions.
+	 *
+	 * @param newResourceManagerId to use for communication with the task managers
+	 * @param newMainThreadExecutor to use to run code in the ResourceManager's main thread
+	 * @param newResourceActions to use for resource (de-)allocations
+	 */
+	@Override
+	public void start(ResourceManagerId newResourceManagerId, Executor newMainThreadExecutor, ResourceActions newResourceActions) {
+		LOG.info("Starting the slot manager.");
+
+		this.resourceManagerId = Preconditions.checkNotNull(newResourceManagerId);
+		mainThreadExecutor = Preconditions.checkNotNull(newMainThreadExecutor);
+		resourceActions = Preconditions.checkNotNull(newResourceActions);
+		taskExecutorManager = taskExecutorManagerFactory.apply(newMainThreadExecutor, newResourceActions);
+
+		started = true;
+
+		registerSlotManagerMetrics();
+	}
+
+	private void registerSlotManagerMetrics() {
+		slotManagerMetricGroup.gauge(
+			MetricNames.TASK_SLOTS_AVAILABLE,
+			() -> (long) getNumberFreeSlots());
+		slotManagerMetricGroup.gauge(
+			MetricNames.TASK_SLOTS_TOTAL,
+			() -> (long) getNumberRegisteredSlots());
+	}
+
+	/**
+	 * Suspends the component. This clears the internal state of the slot manager.
+	 */
+	@Override
+	public void suspend() {
+		LOG.info("Suspending the slot manager.");
+
+		if (slotRequestTimeoutCheck != null) {
+			slotRequestTimeoutCheck.cancel(false);
+			slotRequestTimeoutCheck = null;
+		}
+
+		resourceTracker.clear();
+		taskExecutorManager.close();
+
+		for (InstanceID registeredTaskManager : taskExecutorManager.getTaskExecutors()) {
+			unregisterTaskManager(registeredTaskManager, new SlotManagerException("The slot manager is being suspended."));
+		}
+
+		taskExecutorManager = null;
+		resourceManagerId = null;
+		resourceActions = null;
+		started = false;
+	}
+
+	/**
+	 * Closes the slot manager.
+	 *
+	 * @throws Exception if the close operation fails
+	 */
+	@Override
+	public void close() throws Exception {
+		LOG.info("Closing the slot manager.");
+
+		suspend();
+		slotManagerMetricGroup.close();
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Public API
+	// ---------------------------------------------------------------------------------------------
+
+	@Override
+	public void processResourceRequirements(ResourceRequirements resourceRequirements) {
+		checkInit();
+		LOG.debug("Received resource requirements from job {}: {}", resourceRequirements.getJobId(), resourceRequirements.getResourceRequirements());
+
+		if (resourceRequirements.getResourceRequirements().isEmpty()) {
+			jobMasterTargetAddresses.remove(resourceRequirements.getJobId());
+		} else {
+			jobMasterTargetAddresses.put(resourceRequirements.getJobId(), resourceRequirements.getTargetAddress());
+		}
+		resourceTracker.notifyResourceRequirements(resourceRequirements.getJobId(), resourceRequirements.getResourceRequirements());
+		checkResourceRequirements();
+	}
+
+	/**
+	 * 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
+	 * @return True if the task manager has not been registered before and is registered successfully; otherwise false
+	 */
+	@Override
+	public boolean registerTaskManager(final TaskExecutorConnection taskExecutorConnection, SlotReport initialSlotReport) {
+		checkInit();
+		LOG.debug("Registering task executor {} under {} at the slot manager.", taskExecutorConnection.getResourceID(), taskExecutorConnection.getInstanceID());
+
+		// we identify task managers by their instance id
+		if (taskExecutorManager.isTaskManagerRegistered(taskExecutorConnection.getInstanceID())) {
+			LOG.debug("Task executor {} was already registered.", taskExecutorConnection.getResourceID());
+			reportSlotStatus(taskExecutorConnection.getInstanceID(), initialSlotReport);
+			return false;
+		} else {
+			if (!taskExecutorManager.registerTaskManager(taskExecutorConnection, initialSlotReport)) {
+				LOG.debug("Task executor {} could not be registered.", taskExecutorConnection.getResourceID());
+				return false;
+			}
+
+			// register the new slots
+			for (SlotStatus slotStatus : initialSlotReport) {
+				slotTracker.addSlot(
+					slotStatus.getSlotID(),
+					slotStatus.getResourceProfile(),
+					taskExecutorConnection,
+					slotStatus.getJobID());
+			}
+
+			checkResourceRequirements();
+			return true;
+		}
+	}
+
+	@Override
+	public boolean unregisterTaskManager(InstanceID instanceId, Exception cause) {
+		checkInit();
+
+		LOG.debug("Unregistering task executor {} from the slot manager.", instanceId);
+
+		if (taskExecutorManager.isTaskManagerRegistered(instanceId)) {
+			slotTracker.removeSlots(taskExecutorManager.getSlotsOf(instanceId));
+			taskExecutorManager.unregisterTaskExecutor(instanceId);
+			checkResourceRequirements();
+
+			return true;
+		} else {
+			LOG.debug("There is no task executor registered with instance ID {}. Ignoring this message.", instanceId);
+
+			return false;
+		}
+	}
+
+	/**
+	 * Reports the current slot allocations for a task manager identified by the given instance id.
+	 *
+	 * @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
+	 */
+	@Override
+	public boolean reportSlotStatus(InstanceID instanceId, SlotReport slotReport) {
+		checkInit();
+
+		LOG.debug("Received slot report from instance {}: {}.", instanceId, slotReport);
+
+		if (taskExecutorManager.isTaskManagerRegistered(instanceId)) {
+			slotTracker.notifySlotStatus(slotReport);
+			checkResourceRequirements();
+			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
+	 */
+	@Override
+	public void freeSlot(SlotID slotId, AllocationID allocationId) {
+		checkInit();
+		LOG.debug("Freeing slot {}.", slotId);
+
+		slotTracker.notifyFree(slotId);
+		checkResourceRequirements();
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Requirement matching
+	// ---------------------------------------------------------------------------------------------
+
+	private void checkResourceRequirements() {
+		final Map<JobID, Collection<ResourceRequirement>> missingResources = resourceTracker.getMissingResources();
+		if (missingResources.isEmpty()) {
+			return;
+		}
+
+		final Map<JobID, ResourceCounter> outstandingRequirements = new LinkedHashMap<>();
+		for (Map.Entry<JobID, Collection<ResourceRequirement>> resourceRequirements : missingResources.entrySet()) {
+			JobID jobId = resourceRequirements.getKey();
+
+			for (ResourceRequirement resourceRequirement : resourceRequirements.getValue()) {
+				int numMissingSlots = internalRequestSlots(jobId, jobMasterTargetAddresses.get(jobId), resourceRequirement);
+				if (numMissingSlots > 0) {
+					outstandingRequirements
+						.computeIfAbsent(jobId, ignored -> new ResourceCounter())
+						.incrementCount(resourceRequirement.getResourceProfile(), numMissingSlots);
+				}
+			}
+		}
+
+		final ResourceCounter pendingSlots = new ResourceCounter(taskExecutorManager.getPendingTaskManagerSlots().stream().collect(
+			Collectors.groupingBy(
+				PendingTaskManagerSlot::getResourceProfile,
+				Collectors.summingInt(x -> 1))));
+
+		for (Map.Entry<JobID, ResourceCounter> unfulfilledRequirement : outstandingRequirements.entrySet()) {
+			tryFulfillRequirementsWithPendingOrNewSlots(
+				unfulfilledRequirement.getKey(),
+				unfulfilledRequirement.getValue().getResourceProfilesWithCount(),
+				pendingSlots);
+		}
+	}
+
+	/**
+	 * Tries to allocate slots for the given requirement. If there are not enough slots available, the
+	 * resource manager is informed to allocate more resources.
+	 *
+	 * @param jobId job to allocate slots for
+	 * @param targetAddress address of the jobmaster
+	 * @param resourceRequirement required slots
+	 * @return the number of missing slots
+	 */
+	private int internalRequestSlots(JobID jobId, String targetAddress, ResourceRequirement resourceRequirement) {
+		final ResourceProfile requiredResource = resourceRequirement.getResourceProfile();
+		Collection<TaskManagerSlotInformation> freeSlots = slotTracker.getFreeSlots();
+
+		int numUnfulfilled = 0;
+		for (int x = 0; x < resourceRequirement.getNumberOfRequiredSlots(); x++) {
+
+			final Optional<TaskManagerSlotInformation> reservedSlot = slotMatchingStrategy.findMatchingSlot(requiredResource, freeSlots, this::getNumberRegisteredSlotsOf);
+			if (reservedSlot.isPresent()) {
+				// we do not need to modify freeSlots because it is indirectly modified by the allocation
+				allocateSlot(reservedSlot.get(), jobId, targetAddress, requiredResource);
+			} else {
+				numUnfulfilled++;
+			}
+		}
+		return numUnfulfilled;
+	}
+
+	/**
+	 * Allocates the given slot. This entails sending a registration message to the task manager and treating failures.
+	 *
+	 * @param taskManagerSlot slot to allocate
+	 * @param jobId job for which the slot should be allocated for
+	 * @param targetAddress address of the job master
+	 * @param resourceProfile resource profile for the requirement for which the slot is used
+	 */
+	private void allocateSlot(TaskManagerSlotInformation taskManagerSlot, JobID jobId, String targetAddress, ResourceProfile resourceProfile) {
+		TaskExecutorConnection taskExecutorConnection = taskManagerSlot.getTaskManagerConnection();
+		TaskExecutorGateway gateway = taskExecutorConnection.getTaskExecutorGateway();
+
+		final CompletableFuture<Acknowledge> completableFuture = new CompletableFuture<>();
+		final SlotID slotId = taskManagerSlot.getSlotId();
+		final InstanceID instanceId = taskManagerSlot.getInstanceId();
+
+		LOG.debug("Starting allocation of slot {} for job {} with resource profile {}.", taskManagerSlot.getSlotId(), jobId, resourceProfile);
+
+		slotTracker.notifyAllocationStart(slotId, jobId);
+
+		if (!taskExecutorManager.isTaskManagerRegistered(instanceId)) {
+			throw new IllegalStateException("Could not find a registered task manager for instance id " +
+				instanceId + '.');
+		}

Review comment:
       Maybe move this to the top of the method.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManager.java
##########
@@ -0,0 +1,609 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.metrics.MetricNames;
+import org.apache.flink.runtime.metrics.groups.SlotManagerMetricGroup;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerId;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.slots.ResourceCounter;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+import org.apache.flink.runtime.slots.ResourceRequirements;
+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.SlotOccupiedException;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ScheduledFuture;
+import java.util.function.BiFunction;
+import java.util.stream.Collectors;
+
+/**
+ * Implementation of {@link SlotManager} supporting declarative slot management.
+ */
+public class DeclarativeSlotManager implements SlotManager {
+	private static final Logger LOG = LoggerFactory.getLogger(DeclarativeSlotManager.class);
+
+	private final SlotTracker slotTracker;
+	private final ResourceTracker resourceTracker;
+	private final BiFunction<Executor, ResourceActions, TaskExecutorManager> taskExecutorManagerFactory;
+	private TaskExecutorManager taskExecutorManager;
+
+	/** Timeout for slot requests to the task manager. */
+	private final Time taskManagerRequestTimeout;
+	private ScheduledFuture<?> slotRequestTimeoutCheck;
+
+	private final SlotMatchingStrategy slotMatchingStrategy;
+
+	private final SlotManagerMetricGroup slotManagerMetricGroup;
+
+	private final Map<JobID, String> jobMasterTargetAddresses = new HashMap<>();
+	private final HashMap<SlotID, CompletableFuture<Acknowledge>> pendingSlotAllocationFutures;
+
+	/** ResourceManager's id. */
+	private ResourceManagerId resourceManagerId;
+
+	/** Executor for future callbacks which have to be "synchronized". */
+	private Executor mainThreadExecutor;
+
+	/** Callbacks for resource (de-)allocations. */
+	private ResourceActions resourceActions;
+
+	/** True iff the component has been started. */
+	private boolean started;
+
+	public DeclarativeSlotManager(
+			ScheduledExecutor scheduledExecutor,
+			SlotManagerConfiguration slotManagerConfiguration,
+			SlotManagerMetricGroup slotManagerMetricGroup,
+			ResourceTracker resourceTracker,
+			SlotTracker slotTracker) {
+
+		Preconditions.checkNotNull(slotManagerConfiguration);
+		this.taskManagerRequestTimeout = slotManagerConfiguration.getTaskManagerRequestTimeout();
+		this.slotManagerMetricGroup = Preconditions.checkNotNull(slotManagerMetricGroup);
+		this.resourceTracker = Preconditions.checkNotNull(resourceTracker);
+
+		pendingSlotAllocationFutures = new HashMap<>(16);
+
+		this.slotTracker = Preconditions.checkNotNull(slotTracker);
+		slotTracker.registerSlotStatusUpdateListener(createSlotStatusUpdateListener());
+
+		slotMatchingStrategy = slotManagerConfiguration.getSlotMatchingStrategy();
+
+		taskExecutorManagerFactory = (executor, resourceActions) -> new TaskExecutorManager(
+			slotManagerConfiguration.getDefaultWorkerResourceSpec(),
+			slotManagerConfiguration.getNumSlotsPerWorker(),
+			slotManagerConfiguration.getMaxSlotNum(),
+			slotManagerConfiguration.isWaitResultConsumedBeforeRelease(),
+			slotManagerConfiguration.getRedundantTaskManagerNum(),
+			slotManagerConfiguration.getTaskManagerTimeout(),
+			scheduledExecutor,
+			executor,
+			resourceActions);
+
+		resourceManagerId = null;
+		resourceActions = null;
+		mainThreadExecutor = null;
+		slotRequestTimeoutCheck = null;
+		taskExecutorManager = null;
+
+		started = false;
+	}
+
+	private SlotStatusUpdateListener createSlotStatusUpdateListener() {
+		return (taskManagerSlot, previous, current, jobId) -> {
+			if (previous == SlotState.PENDING) {
+				cancelAllocationFuture(taskManagerSlot.getSlotId());
+			}
+
+			if (current == SlotState.PENDING) {
+				resourceTracker.notifyAcquiredResource(jobId, taskManagerSlot.getResourceProfile());
+			}
+			if (current == SlotState.FREE) {
+				resourceTracker.notifyLostResource(jobId, taskManagerSlot.getResourceProfile());
+			}
+
+			if (current == SlotState.ALLOCATED) {
+				taskExecutorManager.occupySlot(taskManagerSlot.getInstanceId());
+			}
+			if (previous == SlotState.ALLOCATED && current == SlotState.FREE) {
+				taskExecutorManager.freeSlot(taskManagerSlot.getInstanceId());
+			}
+		};
+	}
+
+	private void cancelAllocationFuture(SlotID slotId) {
+		final CompletableFuture<Acknowledge> acknowledgeCompletableFuture = pendingSlotAllocationFutures.remove(slotId);
+		// the future may be null if we are just re-playing the state transitions due to a slot report
+		if (acknowledgeCompletableFuture != null) {
+			acknowledgeCompletableFuture.cancel(false);
+		}
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Component lifecycle methods
+	// ---------------------------------------------------------------------------------------------
+
+	/**
+	 * Starts the slot manager with the given leader id and resource manager actions.
+	 *
+	 * @param newResourceManagerId to use for communication with the task managers
+	 * @param newMainThreadExecutor to use to run code in the ResourceManager's main thread
+	 * @param newResourceActions to use for resource (de-)allocations
+	 */
+	@Override
+	public void start(ResourceManagerId newResourceManagerId, Executor newMainThreadExecutor, ResourceActions newResourceActions) {
+		LOG.info("Starting the slot manager.");
+
+		this.resourceManagerId = Preconditions.checkNotNull(newResourceManagerId);
+		mainThreadExecutor = Preconditions.checkNotNull(newMainThreadExecutor);
+		resourceActions = Preconditions.checkNotNull(newResourceActions);
+		taskExecutorManager = taskExecutorManagerFactory.apply(newMainThreadExecutor, newResourceActions);
+
+		started = true;
+
+		registerSlotManagerMetrics();
+	}
+
+	private void registerSlotManagerMetrics() {
+		slotManagerMetricGroup.gauge(
+			MetricNames.TASK_SLOTS_AVAILABLE,
+			() -> (long) getNumberFreeSlots());
+		slotManagerMetricGroup.gauge(
+			MetricNames.TASK_SLOTS_TOTAL,
+			() -> (long) getNumberRegisteredSlots());
+	}
+
+	/**
+	 * Suspends the component. This clears the internal state of the slot manager.
+	 */
+	@Override
+	public void suspend() {
+		LOG.info("Suspending the slot manager.");
+
+		if (slotRequestTimeoutCheck != null) {
+			slotRequestTimeoutCheck.cancel(false);
+			slotRequestTimeoutCheck = null;
+		}
+
+		resourceTracker.clear();
+		taskExecutorManager.close();
+
+		for (InstanceID registeredTaskManager : taskExecutorManager.getTaskExecutors()) {
+			unregisterTaskManager(registeredTaskManager, new SlotManagerException("The slot manager is being suspended."));
+		}
+
+		taskExecutorManager = null;
+		resourceManagerId = null;
+		resourceActions = null;
+		started = false;
+	}
+
+	/**
+	 * Closes the slot manager.
+	 *
+	 * @throws Exception if the close operation fails
+	 */
+	@Override
+	public void close() throws Exception {
+		LOG.info("Closing the slot manager.");
+
+		suspend();
+		slotManagerMetricGroup.close();
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Public API
+	// ---------------------------------------------------------------------------------------------
+
+	@Override
+	public void processResourceRequirements(ResourceRequirements resourceRequirements) {
+		checkInit();
+		LOG.debug("Received resource requirements from job {}: {}", resourceRequirements.getJobId(), resourceRequirements.getResourceRequirements());
+
+		if (resourceRequirements.getResourceRequirements().isEmpty()) {
+			jobMasterTargetAddresses.remove(resourceRequirements.getJobId());
+		} else {
+			jobMasterTargetAddresses.put(resourceRequirements.getJobId(), resourceRequirements.getTargetAddress());
+		}
+		resourceTracker.notifyResourceRequirements(resourceRequirements.getJobId(), resourceRequirements.getResourceRequirements());
+		checkResourceRequirements();
+	}
+
+	/**
+	 * 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
+	 * @return True if the task manager has not been registered before and is registered successfully; otherwise false
+	 */
+	@Override
+	public boolean registerTaskManager(final TaskExecutorConnection taskExecutorConnection, SlotReport initialSlotReport) {
+		checkInit();
+		LOG.debug("Registering task executor {} under {} at the slot manager.", taskExecutorConnection.getResourceID(), taskExecutorConnection.getInstanceID());
+
+		// we identify task managers by their instance id
+		if (taskExecutorManager.isTaskManagerRegistered(taskExecutorConnection.getInstanceID())) {
+			LOG.debug("Task executor {} was already registered.", taskExecutorConnection.getResourceID());
+			reportSlotStatus(taskExecutorConnection.getInstanceID(), initialSlotReport);
+			return false;
+		} else {
+			if (!taskExecutorManager.registerTaskManager(taskExecutorConnection, initialSlotReport)) {
+				LOG.debug("Task executor {} could not be registered.", taskExecutorConnection.getResourceID());
+				return false;
+			}
+
+			// register the new slots
+			for (SlotStatus slotStatus : initialSlotReport) {
+				slotTracker.addSlot(
+					slotStatus.getSlotID(),
+					slotStatus.getResourceProfile(),
+					taskExecutorConnection,
+					slotStatus.getJobID());
+			}
+
+			checkResourceRequirements();
+			return true;
+		}
+	}
+
+	@Override
+	public boolean unregisterTaskManager(InstanceID instanceId, Exception cause) {
+		checkInit();
+
+		LOG.debug("Unregistering task executor {} from the slot manager.", instanceId);
+
+		if (taskExecutorManager.isTaskManagerRegistered(instanceId)) {
+			slotTracker.removeSlots(taskExecutorManager.getSlotsOf(instanceId));
+			taskExecutorManager.unregisterTaskExecutor(instanceId);
+			checkResourceRequirements();
+
+			return true;
+		} else {
+			LOG.debug("There is no task executor registered with instance ID {}. Ignoring this message.", instanceId);
+
+			return false;
+		}
+	}
+
+	/**
+	 * Reports the current slot allocations for a task manager identified by the given instance id.
+	 *
+	 * @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
+	 */
+	@Override
+	public boolean reportSlotStatus(InstanceID instanceId, SlotReport slotReport) {
+		checkInit();
+
+		LOG.debug("Received slot report from instance {}: {}.", instanceId, slotReport);
+
+		if (taskExecutorManager.isTaskManagerRegistered(instanceId)) {
+			slotTracker.notifySlotStatus(slotReport);
+			checkResourceRequirements();
+			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
+	 */
+	@Override
+	public void freeSlot(SlotID slotId, AllocationID allocationId) {
+		checkInit();
+		LOG.debug("Freeing slot {}.", slotId);
+
+		slotTracker.notifyFree(slotId);
+		checkResourceRequirements();
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Requirement matching
+	// ---------------------------------------------------------------------------------------------
+
+	private void checkResourceRequirements() {
+		final Map<JobID, Collection<ResourceRequirement>> missingResources = resourceTracker.getMissingResources();
+		if (missingResources.isEmpty()) {
+			return;
+		}
+
+		final Map<JobID, ResourceCounter> outstandingRequirements = new LinkedHashMap<>();
+		for (Map.Entry<JobID, Collection<ResourceRequirement>> resourceRequirements : missingResources.entrySet()) {
+			JobID jobId = resourceRequirements.getKey();
+
+			for (ResourceRequirement resourceRequirement : resourceRequirements.getValue()) {
+				int numMissingSlots = internalRequestSlots(jobId, jobMasterTargetAddresses.get(jobId), resourceRequirement);
+				if (numMissingSlots > 0) {
+					outstandingRequirements
+						.computeIfAbsent(jobId, ignored -> new ResourceCounter())
+						.incrementCount(resourceRequirement.getResourceProfile(), numMissingSlots);
+				}
+			}
+		}
+
+		final ResourceCounter pendingSlots = new ResourceCounter(taskExecutorManager.getPendingTaskManagerSlots().stream().collect(
+			Collectors.groupingBy(
+				PendingTaskManagerSlot::getResourceProfile,
+				Collectors.summingInt(x -> 1))));
+
+		for (Map.Entry<JobID, ResourceCounter> unfulfilledRequirement : outstandingRequirements.entrySet()) {
+			tryFulfillRequirementsWithPendingOrNewSlots(
+				unfulfilledRequirement.getKey(),
+				unfulfilledRequirement.getValue().getResourceProfilesWithCount(),
+				pendingSlots);
+		}
+	}
+
+	/**
+	 * Tries to allocate slots for the given requirement. If there are not enough slots available, the
+	 * resource manager is informed to allocate more resources.
+	 *
+	 * @param jobId job to allocate slots for
+	 * @param targetAddress address of the jobmaster
+	 * @param resourceRequirement required slots
+	 * @return the number of missing slots
+	 */
+	private int internalRequestSlots(JobID jobId, String targetAddress, ResourceRequirement resourceRequirement) {
+		final ResourceProfile requiredResource = resourceRequirement.getResourceProfile();
+		Collection<TaskManagerSlotInformation> freeSlots = slotTracker.getFreeSlots();
+
+		int numUnfulfilled = 0;
+		for (int x = 0; x < resourceRequirement.getNumberOfRequiredSlots(); x++) {
+
+			final Optional<TaskManagerSlotInformation> reservedSlot = slotMatchingStrategy.findMatchingSlot(requiredResource, freeSlots, this::getNumberRegisteredSlotsOf);
+			if (reservedSlot.isPresent()) {
+				// we do not need to modify freeSlots because it is indirectly modified by the allocation
+				allocateSlot(reservedSlot.get(), jobId, targetAddress, requiredResource);
+			} else {
+				numUnfulfilled++;
+			}
+		}
+		return numUnfulfilled;
+	}
+
+	/**
+	 * Allocates the given slot. This entails sending a registration message to the task manager and treating failures.
+	 *
+	 * @param taskManagerSlot slot to allocate
+	 * @param jobId job for which the slot should be allocated for
+	 * @param targetAddress address of the job master
+	 * @param resourceProfile resource profile for the requirement for which the slot is used
+	 */
+	private void allocateSlot(TaskManagerSlotInformation taskManagerSlot, JobID jobId, String targetAddress, ResourceProfile resourceProfile) {
+		TaskExecutorConnection taskExecutorConnection = taskManagerSlot.getTaskManagerConnection();
+		TaskExecutorGateway gateway = taskExecutorConnection.getTaskExecutorGateway();
+
+		final CompletableFuture<Acknowledge> completableFuture = new CompletableFuture<>();
+		final SlotID slotId = taskManagerSlot.getSlotId();
+		final InstanceID instanceId = taskManagerSlot.getInstanceId();
+
+		LOG.debug("Starting allocation of slot {} for job {} with resource profile {}.", taskManagerSlot.getSlotId(), jobId, resourceProfile);
+
+		slotTracker.notifyAllocationStart(slotId, jobId);
+
+		if (!taskExecutorManager.isTaskManagerRegistered(instanceId)) {
+			throw new IllegalStateException("Could not find a registered task manager for instance id " +
+				instanceId + '.');
+		}
+
+		taskExecutorManager.markUsed(instanceId);
+
+		// RPC call to the task manager
+		CompletableFuture<Acknowledge> requestFuture = gateway.requestSlot(
+			slotId,
+			jobId,
+			new AllocationID(),
+			resourceProfile,
+			targetAddress,
+			resourceManagerId,
+			taskManagerRequestTimeout);
+
+		requestFuture.whenComplete(
+			(Acknowledge acknowledge, Throwable throwable) -> {
+				if (acknowledge != null) {
+					completableFuture.complete(acknowledge);
+				} else {
+					completableFuture.completeExceptionally(throwable);
+				}
+			});
+
+		completableFuture.whenCompleteAsync(
+			(Acknowledge acknowledge, Throwable throwable) -> {
+				try {
+					if (acknowledge != null) {
+						LOG.trace("Completed allocation of slot {} for job {}.", slotId, jobId);
+						slotTracker.notifyAllocationComplete(slotId, jobId);
+					} else {
+						if (throwable instanceof SlotOccupiedException) {
+							SlotOccupiedException exception = (SlotOccupiedException) throwable;
+							LOG.debug("Tried allocating slot {} for job {}, but it was already allocated for job {}.", slotId, jobId, exception.getJobId());
+							// report as a slot status to force the state transition
+							// this could be a problem if we ever assume that the task executor always reports about all slots
+							slotTracker.notifySlotStatus(Collections.singleton(new SlotStatus(slotId, taskManagerSlot.getResourceProfile(), exception.getJobId(), exception.getAllocationId())));
+						} else {
+							if (throwable instanceof CancellationException) {
+								LOG.debug("Cancelled allocation of slot {} for job {}.", slotId, jobId, throwable);
+							} else {
+								LOG.warn("Slot allocation for slot {} for job {} failed.", slotId, jobId, throwable);
+								slotTracker.notifyFree(slotId);
+							}
+						}
+					}
+					checkResourceRequirements();
+				} catch (Exception e) {
+					LOG.error("Error while completing the slot allocation.", e);
+				}
+			},
+			mainThreadExecutor);
+	}
+
+	private void tryFulfillRequirementsWithPendingOrNewSlots(JobID jobId, Map<ResourceProfile, Integer> missingResources, ResourceCounter pendingSlots) {
+		for (Map.Entry<ResourceProfile, Integer> missingResource : missingResources.entrySet()) {
+			ResourceProfile profile = missingResource.getKey();
+			for (int i = 0; i < missingResource.getValue(); i++) {
+				if (!tryFulfillWithPendingSlots(profile, pendingSlots)) {
+					Optional<ResourceRequirement> newlyFulfillableRequirements = taskExecutorManager.allocateWorker(profile);
+					if (newlyFulfillableRequirements.isPresent()) {
+						ResourceRequirement newSlots = newlyFulfillableRequirements.get();
+						// reserve one of the new slots
+						if (newSlots.getNumberOfRequiredSlots() > 1) {
+							pendingSlots.incrementCount(newSlots.getResourceProfile(), newSlots.getNumberOfRequiredSlots() - 1);
+						}
+					} else {
+						LOG.warn("Could not fulfill resource requirements of job {}.", jobId);
+						resourceActions.notifyNotEnoughResourcesAvailable(jobId, resourceTracker.getAcquiredResources(jobId));
+						return;
+					}
+				}
+			}
+		}
+	}
+
+	private boolean tryFulfillWithPendingSlots(ResourceProfile resourceProfile, ResourceCounter pendingSlots) {
+		Set<ResourceProfile> pendingSlotProfiles = pendingSlots.getResourceProfiles();
+
+		// short-cut, pretty much only applicable to fine-grained resource management
+		if (pendingSlotProfiles.contains(resourceProfile)) {
+			pendingSlots.decrementCount(resourceProfile, 1);
+			return true;
+		}
+
+		for (ResourceProfile pendingSlotProfile : pendingSlotProfiles) {
+			if (pendingSlotProfile.isMatching(resourceProfile)) {
+				pendingSlots.decrementCount(pendingSlotProfile, 1);
+				return true;
+			}
+		}
+
+		return false;
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Legacy APIs
+	// ---------------------------------------------------------------------------------------------
+
+	@Override
+	public int getNumberRegisteredSlots() {
+		return taskExecutorManager.getNumberRegisteredSlots();
+	}
+
+	@Override
+	public int getNumberRegisteredSlotsOf(InstanceID instanceId) {
+		return taskExecutorManager.getNumberRegisteredSlotsOf(instanceId);
+	}
+
+	@Override
+	public int getNumberFreeSlots() {
+		return taskExecutorManager.getNumberFreeSlots();
+	}
+
+	@Override
+	public int getNumberFreeSlotsOf(InstanceID instanceId) {
+		return taskExecutorManager.getNumberFreeSlotsOf(instanceId);
+	}
+
+	@Override
+	public Map<WorkerResourceSpec, Integer> getRequiredResources() {
+		return taskExecutorManager.getRequiredWorkers();
+	}
+
+	@Override
+	public ResourceProfile getRegisteredResource() {
+		return taskExecutorManager.getTotalRegisteredResources();
+	}
+
+	@Override
+	public ResourceProfile getRegisteredResourceOf(InstanceID instanceID) {
+		return taskExecutorManager.getTotalRegisteredResourcesOf(instanceID);
+	}
+
+	@Override
+	public ResourceProfile getFreeResource() {
+		return taskExecutorManager.getTotalFreeResources();
+	}
+
+	@Override
+	public ResourceProfile getFreeResourceOf(InstanceID instanceID) {
+		return taskExecutorManager.getTotalFreeResourcesOf(instanceID);
+	}
+
+	@Override
+	public void setFailUnfulfillableRequest(boolean failUnfulfillableRequest) {
+		// we always send notifications if we cannot fulfill requests, and it is the responsibility of the JobManager
+		// to handle it (e.g., by reducing requirements and failing outright)
+	}
+
+	@Override
+	public int getNumberPendingSlotRequests() {
+		// only exists for testing purposes
+		throw new UnsupportedOperationException();
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Internal utility methods
+	// ---------------------------------------------------------------------------------------------
+
+	private void checkInit() {
+		Preconditions.checkState(started, "The slot manager has not been started.");
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Testing methods
+	// ---------------------------------------------------------------------------------------------
+
+	@Override
+	@VisibleForTesting
+	public void unregisterTaskManagersAndReleaseResources() {

Review comment:
       Why is this method not implemented?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java
##########
@@ -1190,6 +1215,11 @@ public void notifyAllocationFailure(JobID jobId, AllocationID allocationId, Exce
 				jobManagerRegistration.getJobManagerGateway().notifyAllocationFailure(allocationId, cause);
 			}
 		}
+
+		@Override
+		public void notifyNotEnoughResourcesAvailable(JobID jobId, Collection<ResourceRequirement> acquiredResources) {
+			validateRunsInMainThread();
+		}

Review comment:
       What is missing here?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManager.java
##########
@@ -0,0 +1,609 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.metrics.MetricNames;
+import org.apache.flink.runtime.metrics.groups.SlotManagerMetricGroup;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerId;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.slots.ResourceCounter;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+import org.apache.flink.runtime.slots.ResourceRequirements;
+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.SlotOccupiedException;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ScheduledFuture;
+import java.util.function.BiFunction;
+import java.util.stream.Collectors;
+
+/**
+ * Implementation of {@link SlotManager} supporting declarative slot management.
+ */
+public class DeclarativeSlotManager implements SlotManager {
+	private static final Logger LOG = LoggerFactory.getLogger(DeclarativeSlotManager.class);
+
+	private final SlotTracker slotTracker;
+	private final ResourceTracker resourceTracker;
+	private final BiFunction<Executor, ResourceActions, TaskExecutorManager> taskExecutorManagerFactory;
+	private TaskExecutorManager taskExecutorManager;
+
+	/** Timeout for slot requests to the task manager. */
+	private final Time taskManagerRequestTimeout;
+	private ScheduledFuture<?> slotRequestTimeoutCheck;
+
+	private final SlotMatchingStrategy slotMatchingStrategy;
+
+	private final SlotManagerMetricGroup slotManagerMetricGroup;
+
+	private final Map<JobID, String> jobMasterTargetAddresses = new HashMap<>();
+	private final HashMap<SlotID, CompletableFuture<Acknowledge>> pendingSlotAllocationFutures;
+
+	/** ResourceManager's id. */
+	private ResourceManagerId resourceManagerId;
+
+	/** Executor for future callbacks which have to be "synchronized". */
+	private Executor mainThreadExecutor;
+
+	/** Callbacks for resource (de-)allocations. */
+	private ResourceActions resourceActions;
+
+	/** True iff the component has been started. */
+	private boolean started;
+
+	public DeclarativeSlotManager(
+			ScheduledExecutor scheduledExecutor,
+			SlotManagerConfiguration slotManagerConfiguration,
+			SlotManagerMetricGroup slotManagerMetricGroup,
+			ResourceTracker resourceTracker,
+			SlotTracker slotTracker) {
+
+		Preconditions.checkNotNull(slotManagerConfiguration);
+		this.taskManagerRequestTimeout = slotManagerConfiguration.getTaskManagerRequestTimeout();
+		this.slotManagerMetricGroup = Preconditions.checkNotNull(slotManagerMetricGroup);
+		this.resourceTracker = Preconditions.checkNotNull(resourceTracker);
+
+		pendingSlotAllocationFutures = new HashMap<>(16);
+
+		this.slotTracker = Preconditions.checkNotNull(slotTracker);
+		slotTracker.registerSlotStatusUpdateListener(createSlotStatusUpdateListener());
+
+		slotMatchingStrategy = slotManagerConfiguration.getSlotMatchingStrategy();
+
+		taskExecutorManagerFactory = (executor, resourceActions) -> new TaskExecutorManager(
+			slotManagerConfiguration.getDefaultWorkerResourceSpec(),
+			slotManagerConfiguration.getNumSlotsPerWorker(),
+			slotManagerConfiguration.getMaxSlotNum(),
+			slotManagerConfiguration.isWaitResultConsumedBeforeRelease(),
+			slotManagerConfiguration.getRedundantTaskManagerNum(),
+			slotManagerConfiguration.getTaskManagerTimeout(),
+			scheduledExecutor,
+			executor,
+			resourceActions);
+
+		resourceManagerId = null;
+		resourceActions = null;
+		mainThreadExecutor = null;
+		slotRequestTimeoutCheck = null;
+		taskExecutorManager = null;
+
+		started = false;
+	}
+
+	private SlotStatusUpdateListener createSlotStatusUpdateListener() {
+		return (taskManagerSlot, previous, current, jobId) -> {
+			if (previous == SlotState.PENDING) {
+				cancelAllocationFuture(taskManagerSlot.getSlotId());
+			}
+
+			if (current == SlotState.PENDING) {
+				resourceTracker.notifyAcquiredResource(jobId, taskManagerSlot.getResourceProfile());
+			}
+			if (current == SlotState.FREE) {
+				resourceTracker.notifyLostResource(jobId, taskManagerSlot.getResourceProfile());
+			}
+
+			if (current == SlotState.ALLOCATED) {
+				taskExecutorManager.occupySlot(taskManagerSlot.getInstanceId());
+			}
+			if (previous == SlotState.ALLOCATED && current == SlotState.FREE) {
+				taskExecutorManager.freeSlot(taskManagerSlot.getInstanceId());
+			}
+		};
+	}
+
+	private void cancelAllocationFuture(SlotID slotId) {
+		final CompletableFuture<Acknowledge> acknowledgeCompletableFuture = pendingSlotAllocationFutures.remove(slotId);
+		// the future may be null if we are just re-playing the state transitions due to a slot report
+		if (acknowledgeCompletableFuture != null) {
+			acknowledgeCompletableFuture.cancel(false);
+		}
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Component lifecycle methods
+	// ---------------------------------------------------------------------------------------------
+
+	/**
+	 * Starts the slot manager with the given leader id and resource manager actions.
+	 *
+	 * @param newResourceManagerId to use for communication with the task managers
+	 * @param newMainThreadExecutor to use to run code in the ResourceManager's main thread
+	 * @param newResourceActions to use for resource (de-)allocations
+	 */
+	@Override
+	public void start(ResourceManagerId newResourceManagerId, Executor newMainThreadExecutor, ResourceActions newResourceActions) {
+		LOG.info("Starting the slot manager.");
+
+		this.resourceManagerId = Preconditions.checkNotNull(newResourceManagerId);
+		mainThreadExecutor = Preconditions.checkNotNull(newMainThreadExecutor);
+		resourceActions = Preconditions.checkNotNull(newResourceActions);
+		taskExecutorManager = taskExecutorManagerFactory.apply(newMainThreadExecutor, newResourceActions);
+
+		started = true;
+
+		registerSlotManagerMetrics();
+	}
+
+	private void registerSlotManagerMetrics() {
+		slotManagerMetricGroup.gauge(
+			MetricNames.TASK_SLOTS_AVAILABLE,
+			() -> (long) getNumberFreeSlots());
+		slotManagerMetricGroup.gauge(
+			MetricNames.TASK_SLOTS_TOTAL,
+			() -> (long) getNumberRegisteredSlots());
+	}

Review comment:
       Is it a problem that we might register the gauges multiple times in case that the `DSM` goes start -> suspend -> start -> suspend -> ...?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManager.java
##########
@@ -0,0 +1,609 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.metrics.MetricNames;
+import org.apache.flink.runtime.metrics.groups.SlotManagerMetricGroup;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerId;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.slots.ResourceCounter;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+import org.apache.flink.runtime.slots.ResourceRequirements;
+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.SlotOccupiedException;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ScheduledFuture;
+import java.util.function.BiFunction;
+import java.util.stream.Collectors;
+
+/**
+ * Implementation of {@link SlotManager} supporting declarative slot management.
+ */
+public class DeclarativeSlotManager implements SlotManager {
+	private static final Logger LOG = LoggerFactory.getLogger(DeclarativeSlotManager.class);
+
+	private final SlotTracker slotTracker;
+	private final ResourceTracker resourceTracker;
+	private final BiFunction<Executor, ResourceActions, TaskExecutorManager> taskExecutorManagerFactory;
+	private TaskExecutorManager taskExecutorManager;
+
+	/** Timeout for slot requests to the task manager. */
+	private final Time taskManagerRequestTimeout;
+	private ScheduledFuture<?> slotRequestTimeoutCheck;
+
+	private final SlotMatchingStrategy slotMatchingStrategy;
+
+	private final SlotManagerMetricGroup slotManagerMetricGroup;
+
+	private final Map<JobID, String> jobMasterTargetAddresses = new HashMap<>();
+	private final HashMap<SlotID, CompletableFuture<Acknowledge>> pendingSlotAllocationFutures;
+
+	/** ResourceManager's id. */
+	private ResourceManagerId resourceManagerId;
+
+	/** Executor for future callbacks which have to be "synchronized". */
+	private Executor mainThreadExecutor;
+
+	/** Callbacks for resource (de-)allocations. */
+	private ResourceActions resourceActions;
+
+	/** True iff the component has been started. */
+	private boolean started;
+
+	public DeclarativeSlotManager(
+			ScheduledExecutor scheduledExecutor,
+			SlotManagerConfiguration slotManagerConfiguration,
+			SlotManagerMetricGroup slotManagerMetricGroup,
+			ResourceTracker resourceTracker,
+			SlotTracker slotTracker) {
+
+		Preconditions.checkNotNull(slotManagerConfiguration);
+		this.taskManagerRequestTimeout = slotManagerConfiguration.getTaskManagerRequestTimeout();
+		this.slotManagerMetricGroup = Preconditions.checkNotNull(slotManagerMetricGroup);
+		this.resourceTracker = Preconditions.checkNotNull(resourceTracker);
+
+		pendingSlotAllocationFutures = new HashMap<>(16);
+
+		this.slotTracker = Preconditions.checkNotNull(slotTracker);
+		slotTracker.registerSlotStatusUpdateListener(createSlotStatusUpdateListener());
+
+		slotMatchingStrategy = slotManagerConfiguration.getSlotMatchingStrategy();
+
+		taskExecutorManagerFactory = (executor, resourceActions) -> new TaskExecutorManager(
+			slotManagerConfiguration.getDefaultWorkerResourceSpec(),
+			slotManagerConfiguration.getNumSlotsPerWorker(),
+			slotManagerConfiguration.getMaxSlotNum(),
+			slotManagerConfiguration.isWaitResultConsumedBeforeRelease(),
+			slotManagerConfiguration.getRedundantTaskManagerNum(),
+			slotManagerConfiguration.getTaskManagerTimeout(),
+			scheduledExecutor,
+			executor,
+			resourceActions);
+
+		resourceManagerId = null;
+		resourceActions = null;
+		mainThreadExecutor = null;
+		slotRequestTimeoutCheck = null;
+		taskExecutorManager = null;
+
+		started = false;
+	}
+
+	private SlotStatusUpdateListener createSlotStatusUpdateListener() {
+		return (taskManagerSlot, previous, current, jobId) -> {
+			if (previous == SlotState.PENDING) {
+				cancelAllocationFuture(taskManagerSlot.getSlotId());
+			}
+
+			if (current == SlotState.PENDING) {
+				resourceTracker.notifyAcquiredResource(jobId, taskManagerSlot.getResourceProfile());
+			}
+			if (current == SlotState.FREE) {
+				resourceTracker.notifyLostResource(jobId, taskManagerSlot.getResourceProfile());
+			}
+
+			if (current == SlotState.ALLOCATED) {
+				taskExecutorManager.occupySlot(taskManagerSlot.getInstanceId());
+			}
+			if (previous == SlotState.ALLOCATED && current == SlotState.FREE) {
+				taskExecutorManager.freeSlot(taskManagerSlot.getInstanceId());
+			}
+		};
+	}
+
+	private void cancelAllocationFuture(SlotID slotId) {
+		final CompletableFuture<Acknowledge> acknowledgeCompletableFuture = pendingSlotAllocationFutures.remove(slotId);
+		// the future may be null if we are just re-playing the state transitions due to a slot report
+		if (acknowledgeCompletableFuture != null) {
+			acknowledgeCompletableFuture.cancel(false);
+		}
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Component lifecycle methods
+	// ---------------------------------------------------------------------------------------------
+
+	/**
+	 * Starts the slot manager with the given leader id and resource manager actions.
+	 *
+	 * @param newResourceManagerId to use for communication with the task managers
+	 * @param newMainThreadExecutor to use to run code in the ResourceManager's main thread
+	 * @param newResourceActions to use for resource (de-)allocations
+	 */
+	@Override
+	public void start(ResourceManagerId newResourceManagerId, Executor newMainThreadExecutor, ResourceActions newResourceActions) {
+		LOG.info("Starting the slot manager.");
+
+		this.resourceManagerId = Preconditions.checkNotNull(newResourceManagerId);
+		mainThreadExecutor = Preconditions.checkNotNull(newMainThreadExecutor);
+		resourceActions = Preconditions.checkNotNull(newResourceActions);
+		taskExecutorManager = taskExecutorManagerFactory.apply(newMainThreadExecutor, newResourceActions);
+
+		started = true;
+
+		registerSlotManagerMetrics();
+	}
+
+	private void registerSlotManagerMetrics() {
+		slotManagerMetricGroup.gauge(
+			MetricNames.TASK_SLOTS_AVAILABLE,
+			() -> (long) getNumberFreeSlots());
+		slotManagerMetricGroup.gauge(
+			MetricNames.TASK_SLOTS_TOTAL,
+			() -> (long) getNumberRegisteredSlots());
+	}
+
+	/**
+	 * Suspends the component. This clears the internal state of the slot manager.
+	 */
+	@Override
+	public void suspend() {
+		LOG.info("Suspending the slot manager.");
+
+		if (slotRequestTimeoutCheck != null) {
+			slotRequestTimeoutCheck.cancel(false);
+			slotRequestTimeoutCheck = null;
+		}
+
+		resourceTracker.clear();
+		taskExecutorManager.close();
+
+		for (InstanceID registeredTaskManager : taskExecutorManager.getTaskExecutors()) {
+			unregisterTaskManager(registeredTaskManager, new SlotManagerException("The slot manager is being suspended."));
+		}
+
+		taskExecutorManager = null;
+		resourceManagerId = null;
+		resourceActions = null;
+		started = false;
+	}
+
+	/**
+	 * Closes the slot manager.
+	 *
+	 * @throws Exception if the close operation fails
+	 */
+	@Override
+	public void close() throws Exception {
+		LOG.info("Closing the slot manager.");
+
+		suspend();
+		slotManagerMetricGroup.close();
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Public API
+	// ---------------------------------------------------------------------------------------------
+
+	@Override
+	public void processResourceRequirements(ResourceRequirements resourceRequirements) {
+		checkInit();
+		LOG.debug("Received resource requirements from job {}: {}", resourceRequirements.getJobId(), resourceRequirements.getResourceRequirements());
+
+		if (resourceRequirements.getResourceRequirements().isEmpty()) {
+			jobMasterTargetAddresses.remove(resourceRequirements.getJobId());
+		} else {
+			jobMasterTargetAddresses.put(resourceRequirements.getJobId(), resourceRequirements.getTargetAddress());
+		}
+		resourceTracker.notifyResourceRequirements(resourceRequirements.getJobId(), resourceRequirements.getResourceRequirements());
+		checkResourceRequirements();
+	}
+
+	/**
+	 * 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
+	 * @return True if the task manager has not been registered before and is registered successfully; otherwise false
+	 */
+	@Override
+	public boolean registerTaskManager(final TaskExecutorConnection taskExecutorConnection, SlotReport initialSlotReport) {
+		checkInit();
+		LOG.debug("Registering task executor {} under {} at the slot manager.", taskExecutorConnection.getResourceID(), taskExecutorConnection.getInstanceID());
+
+		// we identify task managers by their instance id
+		if (taskExecutorManager.isTaskManagerRegistered(taskExecutorConnection.getInstanceID())) {
+			LOG.debug("Task executor {} was already registered.", taskExecutorConnection.getResourceID());
+			reportSlotStatus(taskExecutorConnection.getInstanceID(), initialSlotReport);
+			return false;
+		} else {
+			if (!taskExecutorManager.registerTaskManager(taskExecutorConnection, initialSlotReport)) {
+				LOG.debug("Task executor {} could not be registered.", taskExecutorConnection.getResourceID());
+				return false;
+			}
+
+			// register the new slots
+			for (SlotStatus slotStatus : initialSlotReport) {
+				slotTracker.addSlot(
+					slotStatus.getSlotID(),
+					slotStatus.getResourceProfile(),
+					taskExecutorConnection,
+					slotStatus.getJobID());
+			}
+
+			checkResourceRequirements();
+			return true;
+		}
+	}
+
+	@Override
+	public boolean unregisterTaskManager(InstanceID instanceId, Exception cause) {
+		checkInit();
+
+		LOG.debug("Unregistering task executor {} from the slot manager.", instanceId);
+
+		if (taskExecutorManager.isTaskManagerRegistered(instanceId)) {
+			slotTracker.removeSlots(taskExecutorManager.getSlotsOf(instanceId));
+			taskExecutorManager.unregisterTaskExecutor(instanceId);
+			checkResourceRequirements();
+
+			return true;
+		} else {
+			LOG.debug("There is no task executor registered with instance ID {}. Ignoring this message.", instanceId);
+
+			return false;
+		}
+	}
+
+	/**
+	 * Reports the current slot allocations for a task manager identified by the given instance id.
+	 *
+	 * @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
+	 */
+	@Override
+	public boolean reportSlotStatus(InstanceID instanceId, SlotReport slotReport) {
+		checkInit();
+
+		LOG.debug("Received slot report from instance {}: {}.", instanceId, slotReport);
+
+		if (taskExecutorManager.isTaskManagerRegistered(instanceId)) {
+			slotTracker.notifySlotStatus(slotReport);
+			checkResourceRequirements();
+			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
+	 */
+	@Override
+	public void freeSlot(SlotID slotId, AllocationID allocationId) {
+		checkInit();
+		LOG.debug("Freeing slot {}.", slotId);
+
+		slotTracker.notifyFree(slotId);
+		checkResourceRequirements();
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Requirement matching
+	// ---------------------------------------------------------------------------------------------
+
+	private void checkResourceRequirements() {
+		final Map<JobID, Collection<ResourceRequirement>> missingResources = resourceTracker.getMissingResources();
+		if (missingResources.isEmpty()) {
+			return;
+		}
+
+		final Map<JobID, ResourceCounter> outstandingRequirements = new LinkedHashMap<>();
+		for (Map.Entry<JobID, Collection<ResourceRequirement>> resourceRequirements : missingResources.entrySet()) {
+			JobID jobId = resourceRequirements.getKey();
+
+			for (ResourceRequirement resourceRequirement : resourceRequirements.getValue()) {
+				int numMissingSlots = internalRequestSlots(jobId, jobMasterTargetAddresses.get(jobId), resourceRequirement);
+				if (numMissingSlots > 0) {
+					outstandingRequirements
+						.computeIfAbsent(jobId, ignored -> new ResourceCounter())
+						.incrementCount(resourceRequirement.getResourceProfile(), numMissingSlots);
+				}
+			}

Review comment:
       I would suggest to factor this out into a separate method something like `requestsSlotsForJob` or so.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManager.java
##########
@@ -0,0 +1,609 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.metrics.MetricNames;
+import org.apache.flink.runtime.metrics.groups.SlotManagerMetricGroup;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerId;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.slots.ResourceCounter;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+import org.apache.flink.runtime.slots.ResourceRequirements;
+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.SlotOccupiedException;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ScheduledFuture;
+import java.util.function.BiFunction;
+import java.util.stream.Collectors;
+
+/**
+ * Implementation of {@link SlotManager} supporting declarative slot management.
+ */
+public class DeclarativeSlotManager implements SlotManager {
+	private static final Logger LOG = LoggerFactory.getLogger(DeclarativeSlotManager.class);
+
+	private final SlotTracker slotTracker;
+	private final ResourceTracker resourceTracker;
+	private final BiFunction<Executor, ResourceActions, TaskExecutorManager> taskExecutorManagerFactory;
+	private TaskExecutorManager taskExecutorManager;
+
+	/** Timeout for slot requests to the task manager. */
+	private final Time taskManagerRequestTimeout;
+	private ScheduledFuture<?> slotRequestTimeoutCheck;
+
+	private final SlotMatchingStrategy slotMatchingStrategy;
+
+	private final SlotManagerMetricGroup slotManagerMetricGroup;
+
+	private final Map<JobID, String> jobMasterTargetAddresses = new HashMap<>();
+	private final HashMap<SlotID, CompletableFuture<Acknowledge>> pendingSlotAllocationFutures;
+
+	/** ResourceManager's id. */
+	private ResourceManagerId resourceManagerId;
+
+	/** Executor for future callbacks which have to be "synchronized". */
+	private Executor mainThreadExecutor;
+
+	/** Callbacks for resource (de-)allocations. */
+	private ResourceActions resourceActions;
+
+	/** True iff the component has been started. */
+	private boolean started;
+
+	public DeclarativeSlotManager(
+			ScheduledExecutor scheduledExecutor,
+			SlotManagerConfiguration slotManagerConfiguration,
+			SlotManagerMetricGroup slotManagerMetricGroup,
+			ResourceTracker resourceTracker,
+			SlotTracker slotTracker) {
+
+		Preconditions.checkNotNull(slotManagerConfiguration);
+		this.taskManagerRequestTimeout = slotManagerConfiguration.getTaskManagerRequestTimeout();
+		this.slotManagerMetricGroup = Preconditions.checkNotNull(slotManagerMetricGroup);
+		this.resourceTracker = Preconditions.checkNotNull(resourceTracker);
+
+		pendingSlotAllocationFutures = new HashMap<>(16);
+
+		this.slotTracker = Preconditions.checkNotNull(slotTracker);
+		slotTracker.registerSlotStatusUpdateListener(createSlotStatusUpdateListener());
+
+		slotMatchingStrategy = slotManagerConfiguration.getSlotMatchingStrategy();
+
+		taskExecutorManagerFactory = (executor, resourceActions) -> new TaskExecutorManager(
+			slotManagerConfiguration.getDefaultWorkerResourceSpec(),
+			slotManagerConfiguration.getNumSlotsPerWorker(),
+			slotManagerConfiguration.getMaxSlotNum(),
+			slotManagerConfiguration.isWaitResultConsumedBeforeRelease(),
+			slotManagerConfiguration.getRedundantTaskManagerNum(),
+			slotManagerConfiguration.getTaskManagerTimeout(),
+			scheduledExecutor,
+			executor,
+			resourceActions);
+
+		resourceManagerId = null;
+		resourceActions = null;
+		mainThreadExecutor = null;
+		slotRequestTimeoutCheck = null;
+		taskExecutorManager = null;
+
+		started = false;
+	}
+
+	private SlotStatusUpdateListener createSlotStatusUpdateListener() {
+		return (taskManagerSlot, previous, current, jobId) -> {
+			if (previous == SlotState.PENDING) {
+				cancelAllocationFuture(taskManagerSlot.getSlotId());
+			}
+
+			if (current == SlotState.PENDING) {
+				resourceTracker.notifyAcquiredResource(jobId, taskManagerSlot.getResourceProfile());
+			}
+			if (current == SlotState.FREE) {
+				resourceTracker.notifyLostResource(jobId, taskManagerSlot.getResourceProfile());
+			}
+
+			if (current == SlotState.ALLOCATED) {
+				taskExecutorManager.occupySlot(taskManagerSlot.getInstanceId());
+			}
+			if (previous == SlotState.ALLOCATED && current == SlotState.FREE) {
+				taskExecutorManager.freeSlot(taskManagerSlot.getInstanceId());
+			}
+		};
+	}
+
+	private void cancelAllocationFuture(SlotID slotId) {
+		final CompletableFuture<Acknowledge> acknowledgeCompletableFuture = pendingSlotAllocationFutures.remove(slotId);

Review comment:
       I think `pendingSlotAllocationFutures` is never populated with any futures. What's the idea behind this structure?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManager.java
##########
@@ -0,0 +1,609 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.metrics.MetricNames;
+import org.apache.flink.runtime.metrics.groups.SlotManagerMetricGroup;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerId;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.slots.ResourceCounter;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+import org.apache.flink.runtime.slots.ResourceRequirements;
+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.SlotOccupiedException;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ScheduledFuture;
+import java.util.function.BiFunction;
+import java.util.stream.Collectors;
+
+/**
+ * Implementation of {@link SlotManager} supporting declarative slot management.
+ */
+public class DeclarativeSlotManager implements SlotManager {
+	private static final Logger LOG = LoggerFactory.getLogger(DeclarativeSlotManager.class);
+
+	private final SlotTracker slotTracker;
+	private final ResourceTracker resourceTracker;
+	private final BiFunction<Executor, ResourceActions, TaskExecutorManager> taskExecutorManagerFactory;
+	private TaskExecutorManager taskExecutorManager;
+
+	/** Timeout for slot requests to the task manager. */
+	private final Time taskManagerRequestTimeout;
+	private ScheduledFuture<?> slotRequestTimeoutCheck;
+
+	private final SlotMatchingStrategy slotMatchingStrategy;
+
+	private final SlotManagerMetricGroup slotManagerMetricGroup;
+
+	private final Map<JobID, String> jobMasterTargetAddresses = new HashMap<>();
+	private final HashMap<SlotID, CompletableFuture<Acknowledge>> pendingSlotAllocationFutures;
+
+	/** ResourceManager's id. */
+	private ResourceManagerId resourceManagerId;
+
+	/** Executor for future callbacks which have to be "synchronized". */
+	private Executor mainThreadExecutor;
+
+	/** Callbacks for resource (de-)allocations. */
+	private ResourceActions resourceActions;
+
+	/** True iff the component has been started. */
+	private boolean started;
+
+	public DeclarativeSlotManager(
+			ScheduledExecutor scheduledExecutor,
+			SlotManagerConfiguration slotManagerConfiguration,
+			SlotManagerMetricGroup slotManagerMetricGroup,
+			ResourceTracker resourceTracker,
+			SlotTracker slotTracker) {
+
+		Preconditions.checkNotNull(slotManagerConfiguration);
+		this.taskManagerRequestTimeout = slotManagerConfiguration.getTaskManagerRequestTimeout();
+		this.slotManagerMetricGroup = Preconditions.checkNotNull(slotManagerMetricGroup);
+		this.resourceTracker = Preconditions.checkNotNull(resourceTracker);
+
+		pendingSlotAllocationFutures = new HashMap<>(16);
+
+		this.slotTracker = Preconditions.checkNotNull(slotTracker);
+		slotTracker.registerSlotStatusUpdateListener(createSlotStatusUpdateListener());
+
+		slotMatchingStrategy = slotManagerConfiguration.getSlotMatchingStrategy();
+
+		taskExecutorManagerFactory = (executor, resourceActions) -> new TaskExecutorManager(
+			slotManagerConfiguration.getDefaultWorkerResourceSpec(),
+			slotManagerConfiguration.getNumSlotsPerWorker(),
+			slotManagerConfiguration.getMaxSlotNum(),
+			slotManagerConfiguration.isWaitResultConsumedBeforeRelease(),
+			slotManagerConfiguration.getRedundantTaskManagerNum(),
+			slotManagerConfiguration.getTaskManagerTimeout(),
+			scheduledExecutor,
+			executor,
+			resourceActions);
+
+		resourceManagerId = null;
+		resourceActions = null;
+		mainThreadExecutor = null;
+		slotRequestTimeoutCheck = null;
+		taskExecutorManager = null;
+
+		started = false;
+	}
+
+	private SlotStatusUpdateListener createSlotStatusUpdateListener() {
+		return (taskManagerSlot, previous, current, jobId) -> {
+			if (previous == SlotState.PENDING) {
+				cancelAllocationFuture(taskManagerSlot.getSlotId());
+			}
+
+			if (current == SlotState.PENDING) {
+				resourceTracker.notifyAcquiredResource(jobId, taskManagerSlot.getResourceProfile());
+			}
+			if (current == SlotState.FREE) {
+				resourceTracker.notifyLostResource(jobId, taskManagerSlot.getResourceProfile());
+			}
+
+			if (current == SlotState.ALLOCATED) {
+				taskExecutorManager.occupySlot(taskManagerSlot.getInstanceId());
+			}
+			if (previous == SlotState.ALLOCATED && current == SlotState.FREE) {
+				taskExecutorManager.freeSlot(taskManagerSlot.getInstanceId());
+			}
+		};
+	}
+
+	private void cancelAllocationFuture(SlotID slotId) {
+		final CompletableFuture<Acknowledge> acknowledgeCompletableFuture = pendingSlotAllocationFutures.remove(slotId);
+		// the future may be null if we are just re-playing the state transitions due to a slot report
+		if (acknowledgeCompletableFuture != null) {
+			acknowledgeCompletableFuture.cancel(false);
+		}
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Component lifecycle methods
+	// ---------------------------------------------------------------------------------------------
+
+	/**
+	 * Starts the slot manager with the given leader id and resource manager actions.
+	 *
+	 * @param newResourceManagerId to use for communication with the task managers
+	 * @param newMainThreadExecutor to use to run code in the ResourceManager's main thread
+	 * @param newResourceActions to use for resource (de-)allocations
+	 */
+	@Override
+	public void start(ResourceManagerId newResourceManagerId, Executor newMainThreadExecutor, ResourceActions newResourceActions) {
+		LOG.info("Starting the slot manager.");
+
+		this.resourceManagerId = Preconditions.checkNotNull(newResourceManagerId);
+		mainThreadExecutor = Preconditions.checkNotNull(newMainThreadExecutor);
+		resourceActions = Preconditions.checkNotNull(newResourceActions);
+		taskExecutorManager = taskExecutorManagerFactory.apply(newMainThreadExecutor, newResourceActions);
+
+		started = true;
+
+		registerSlotManagerMetrics();
+	}
+
+	private void registerSlotManagerMetrics() {
+		slotManagerMetricGroup.gauge(
+			MetricNames.TASK_SLOTS_AVAILABLE,
+			() -> (long) getNumberFreeSlots());
+		slotManagerMetricGroup.gauge(
+			MetricNames.TASK_SLOTS_TOTAL,
+			() -> (long) getNumberRegisteredSlots());
+	}
+
+	/**
+	 * Suspends the component. This clears the internal state of the slot manager.
+	 */
+	@Override
+	public void suspend() {
+		LOG.info("Suspending the slot manager.");
+
+		if (slotRequestTimeoutCheck != null) {
+			slotRequestTimeoutCheck.cancel(false);
+			slotRequestTimeoutCheck = null;
+		}
+
+		resourceTracker.clear();
+		taskExecutorManager.close();
+
+		for (InstanceID registeredTaskManager : taskExecutorManager.getTaskExecutors()) {
+			unregisterTaskManager(registeredTaskManager, new SlotManagerException("The slot manager is being suspended."));
+		}
+
+		taskExecutorManager = null;
+		resourceManagerId = null;
+		resourceActions = null;
+		started = false;
+	}
+
+	/**
+	 * Closes the slot manager.
+	 *
+	 * @throws Exception if the close operation fails
+	 */
+	@Override
+	public void close() throws Exception {
+		LOG.info("Closing the slot manager.");
+
+		suspend();
+		slotManagerMetricGroup.close();
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Public API
+	// ---------------------------------------------------------------------------------------------
+
+	@Override
+	public void processResourceRequirements(ResourceRequirements resourceRequirements) {
+		checkInit();
+		LOG.debug("Received resource requirements from job {}: {}", resourceRequirements.getJobId(), resourceRequirements.getResourceRequirements());
+
+		if (resourceRequirements.getResourceRequirements().isEmpty()) {
+			jobMasterTargetAddresses.remove(resourceRequirements.getJobId());
+		} else {
+			jobMasterTargetAddresses.put(resourceRequirements.getJobId(), resourceRequirements.getTargetAddress());
+		}
+		resourceTracker.notifyResourceRequirements(resourceRequirements.getJobId(), resourceRequirements.getResourceRequirements());
+		checkResourceRequirements();
+	}
+
+	/**
+	 * 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
+	 * @return True if the task manager has not been registered before and is registered successfully; otherwise false
+	 */
+	@Override
+	public boolean registerTaskManager(final TaskExecutorConnection taskExecutorConnection, SlotReport initialSlotReport) {
+		checkInit();
+		LOG.debug("Registering task executor {} under {} at the slot manager.", taskExecutorConnection.getResourceID(), taskExecutorConnection.getInstanceID());
+
+		// we identify task managers by their instance id
+		if (taskExecutorManager.isTaskManagerRegistered(taskExecutorConnection.getInstanceID())) {
+			LOG.debug("Task executor {} was already registered.", taskExecutorConnection.getResourceID());
+			reportSlotStatus(taskExecutorConnection.getInstanceID(), initialSlotReport);
+			return false;
+		} else {
+			if (!taskExecutorManager.registerTaskManager(taskExecutorConnection, initialSlotReport)) {
+				LOG.debug("Task executor {} could not be registered.", taskExecutorConnection.getResourceID());
+				return false;
+			}
+
+			// register the new slots
+			for (SlotStatus slotStatus : initialSlotReport) {
+				slotTracker.addSlot(
+					slotStatus.getSlotID(),
+					slotStatus.getResourceProfile(),
+					taskExecutorConnection,
+					slotStatus.getJobID());
+			}
+
+			checkResourceRequirements();
+			return true;
+		}
+	}
+
+	@Override
+	public boolean unregisterTaskManager(InstanceID instanceId, Exception cause) {
+		checkInit();
+
+		LOG.debug("Unregistering task executor {} from the slot manager.", instanceId);
+
+		if (taskExecutorManager.isTaskManagerRegistered(instanceId)) {
+			slotTracker.removeSlots(taskExecutorManager.getSlotsOf(instanceId));
+			taskExecutorManager.unregisterTaskExecutor(instanceId);
+			checkResourceRequirements();
+
+			return true;
+		} else {
+			LOG.debug("There is no task executor registered with instance ID {}. Ignoring this message.", instanceId);
+
+			return false;
+		}
+	}
+
+	/**
+	 * Reports the current slot allocations for a task manager identified by the given instance id.
+	 *
+	 * @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
+	 */
+	@Override
+	public boolean reportSlotStatus(InstanceID instanceId, SlotReport slotReport) {
+		checkInit();
+
+		LOG.debug("Received slot report from instance {}: {}.", instanceId, slotReport);
+
+		if (taskExecutorManager.isTaskManagerRegistered(instanceId)) {
+			slotTracker.notifySlotStatus(slotReport);
+			checkResourceRequirements();
+			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
+	 */
+	@Override
+	public void freeSlot(SlotID slotId, AllocationID allocationId) {
+		checkInit();
+		LOG.debug("Freeing slot {}.", slotId);
+
+		slotTracker.notifyFree(slotId);
+		checkResourceRequirements();
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Requirement matching
+	// ---------------------------------------------------------------------------------------------
+
+	private void checkResourceRequirements() {
+		final Map<JobID, Collection<ResourceRequirement>> missingResources = resourceTracker.getMissingResources();
+		if (missingResources.isEmpty()) {
+			return;
+		}
+
+		final Map<JobID, ResourceCounter> outstandingRequirements = new LinkedHashMap<>();
+		for (Map.Entry<JobID, Collection<ResourceRequirement>> resourceRequirements : missingResources.entrySet()) {
+			JobID jobId = resourceRequirements.getKey();
+
+			for (ResourceRequirement resourceRequirement : resourceRequirements.getValue()) {
+				int numMissingSlots = internalRequestSlots(jobId, jobMasterTargetAddresses.get(jobId), resourceRequirement);
+				if (numMissingSlots > 0) {
+					outstandingRequirements
+						.computeIfAbsent(jobId, ignored -> new ResourceCounter())
+						.incrementCount(resourceRequirement.getResourceProfile(), numMissingSlots);
+				}
+			}
+		}
+
+		final ResourceCounter pendingSlots = new ResourceCounter(taskExecutorManager.getPendingTaskManagerSlots().stream().collect(
+			Collectors.groupingBy(
+				PendingTaskManagerSlot::getResourceProfile,
+				Collectors.summingInt(x -> 1))));
+
+		for (Map.Entry<JobID, ResourceCounter> unfulfilledRequirement : outstandingRequirements.entrySet()) {
+			tryFulfillRequirementsWithPendingOrNewSlots(
+				unfulfilledRequirement.getKey(),
+				unfulfilledRequirement.getValue().getResourceProfilesWithCount(),
+				pendingSlots);
+		}
+	}
+
+	/**
+	 * Tries to allocate slots for the given requirement. If there are not enough slots available, the
+	 * resource manager is informed to allocate more resources.
+	 *
+	 * @param jobId job to allocate slots for
+	 * @param targetAddress address of the jobmaster
+	 * @param resourceRequirement required slots
+	 * @return the number of missing slots
+	 */
+	private int internalRequestSlots(JobID jobId, String targetAddress, ResourceRequirement resourceRequirement) {
+		final ResourceProfile requiredResource = resourceRequirement.getResourceProfile();
+		Collection<TaskManagerSlotInformation> freeSlots = slotTracker.getFreeSlots();
+
+		int numUnfulfilled = 0;
+		for (int x = 0; x < resourceRequirement.getNumberOfRequiredSlots(); x++) {
+
+			final Optional<TaskManagerSlotInformation> reservedSlot = slotMatchingStrategy.findMatchingSlot(requiredResource, freeSlots, this::getNumberRegisteredSlotsOf);
+			if (reservedSlot.isPresent()) {
+				// we do not need to modify freeSlots because it is indirectly modified by the allocation
+				allocateSlot(reservedSlot.get(), jobId, targetAddress, requiredResource);
+			} else {
+				numUnfulfilled++;
+			}
+		}
+		return numUnfulfilled;
+	}
+
+	/**
+	 * Allocates the given slot. This entails sending a registration message to the task manager and treating failures.
+	 *
+	 * @param taskManagerSlot slot to allocate
+	 * @param jobId job for which the slot should be allocated for
+	 * @param targetAddress address of the job master
+	 * @param resourceProfile resource profile for the requirement for which the slot is used
+	 */
+	private void allocateSlot(TaskManagerSlotInformation taskManagerSlot, JobID jobId, String targetAddress, ResourceProfile resourceProfile) {
+		TaskExecutorConnection taskExecutorConnection = taskManagerSlot.getTaskManagerConnection();
+		TaskExecutorGateway gateway = taskExecutorConnection.getTaskExecutorGateway();
+
+		final CompletableFuture<Acknowledge> completableFuture = new CompletableFuture<>();
+		final SlotID slotId = taskManagerSlot.getSlotId();
+		final InstanceID instanceId = taskManagerSlot.getInstanceId();
+
+		LOG.debug("Starting allocation of slot {} for job {} with resource profile {}.", taskManagerSlot.getSlotId(), jobId, resourceProfile);
+
+		slotTracker.notifyAllocationStart(slotId, jobId);
+
+		if (!taskExecutorManager.isTaskManagerRegistered(instanceId)) {
+			throw new IllegalStateException("Could not find a registered task manager for instance id " +
+				instanceId + '.');
+		}
+
+		taskExecutorManager.markUsed(instanceId);
+
+		// RPC call to the task manager
+		CompletableFuture<Acknowledge> requestFuture = gateway.requestSlot(
+			slotId,
+			jobId,
+			new AllocationID(),
+			resourceProfile,
+			targetAddress,
+			resourceManagerId,
+			taskManagerRequestTimeout);
+
+		requestFuture.whenComplete(
+			(Acknowledge acknowledge, Throwable throwable) -> {
+				if (acknowledge != null) {
+					completableFuture.complete(acknowledge);
+				} else {
+					completableFuture.completeExceptionally(throwable);
+				}
+			});
+
+		completableFuture.whenCompleteAsync(
+			(Acknowledge acknowledge, Throwable throwable) -> {
+				try {
+					if (acknowledge != null) {
+						LOG.trace("Completed allocation of slot {} for job {}.", slotId, jobId);
+						slotTracker.notifyAllocationComplete(slotId, jobId);
+					} else {
+						if (throwable instanceof SlotOccupiedException) {
+							SlotOccupiedException exception = (SlotOccupiedException) throwable;
+							LOG.debug("Tried allocating slot {} for job {}, but it was already allocated for job {}.", slotId, jobId, exception.getJobId());
+							// report as a slot status to force the state transition
+							// this could be a problem if we ever assume that the task executor always reports about all slots
+							slotTracker.notifySlotStatus(Collections.singleton(new SlotStatus(slotId, taskManagerSlot.getResourceProfile(), exception.getJobId(), exception.getAllocationId())));
+						} else {
+							if (throwable instanceof CancellationException) {
+								LOG.debug("Cancelled allocation of slot {} for job {}.", slotId, jobId, throwable);
+							} else {
+								LOG.warn("Slot allocation for slot {} for job {} failed.", slotId, jobId, throwable);
+								slotTracker.notifyFree(slotId);
+							}
+						}
+					}
+					checkResourceRequirements();
+				} catch (Exception e) {
+					LOG.error("Error while completing the slot allocation.", e);
+				}
+			},
+			mainThreadExecutor);
+	}
+
+	private void tryFulfillRequirementsWithPendingOrNewSlots(JobID jobId, Map<ResourceProfile, Integer> missingResources, ResourceCounter pendingSlots) {

Review comment:
       Aren't we calling this method always with pending slots and no new slots?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java
##########
@@ -861,6 +884,8 @@ protected void closeJobManagerConnection(JobID jobId, Exception cause) {
 
 			jmResourceIdRegistrations.remove(jobManagerResourceId);
 
+			slotManager.processResourceRequirements(ResourceRequirements.empty(jobId, jobMasterGateway.getAddress()));

Review comment:
       On the other hand a job will only be removed after `resourcemanager.job.timeout` which defaults to 5 minutes. Then I guess it is fine that we don't try to actively allocated new resources for a job whose job manager has died/timed out.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManager.java
##########
@@ -0,0 +1,609 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.metrics.MetricNames;
+import org.apache.flink.runtime.metrics.groups.SlotManagerMetricGroup;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerId;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.slots.ResourceCounter;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+import org.apache.flink.runtime.slots.ResourceRequirements;
+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.SlotOccupiedException;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ScheduledFuture;
+import java.util.function.BiFunction;
+import java.util.stream.Collectors;
+
+/**
+ * Implementation of {@link SlotManager} supporting declarative slot management.
+ */
+public class DeclarativeSlotManager implements SlotManager {
+	private static final Logger LOG = LoggerFactory.getLogger(DeclarativeSlotManager.class);
+
+	private final SlotTracker slotTracker;
+	private final ResourceTracker resourceTracker;
+	private final BiFunction<Executor, ResourceActions, TaskExecutorManager> taskExecutorManagerFactory;
+	private TaskExecutorManager taskExecutorManager;
+
+	/** Timeout for slot requests to the task manager. */
+	private final Time taskManagerRequestTimeout;
+	private ScheduledFuture<?> slotRequestTimeoutCheck;
+
+	private final SlotMatchingStrategy slotMatchingStrategy;
+
+	private final SlotManagerMetricGroup slotManagerMetricGroup;
+
+	private final Map<JobID, String> jobMasterTargetAddresses = new HashMap<>();
+	private final HashMap<SlotID, CompletableFuture<Acknowledge>> pendingSlotAllocationFutures;
+
+	/** ResourceManager's id. */
+	private ResourceManagerId resourceManagerId;
+
+	/** Executor for future callbacks which have to be "synchronized". */
+	private Executor mainThreadExecutor;
+
+	/** Callbacks for resource (de-)allocations. */
+	private ResourceActions resourceActions;
+
+	/** True iff the component has been started. */
+	private boolean started;
+
+	public DeclarativeSlotManager(
+			ScheduledExecutor scheduledExecutor,
+			SlotManagerConfiguration slotManagerConfiguration,
+			SlotManagerMetricGroup slotManagerMetricGroup,
+			ResourceTracker resourceTracker,
+			SlotTracker slotTracker) {
+
+		Preconditions.checkNotNull(slotManagerConfiguration);
+		this.taskManagerRequestTimeout = slotManagerConfiguration.getTaskManagerRequestTimeout();
+		this.slotManagerMetricGroup = Preconditions.checkNotNull(slotManagerMetricGroup);
+		this.resourceTracker = Preconditions.checkNotNull(resourceTracker);
+
+		pendingSlotAllocationFutures = new HashMap<>(16);
+
+		this.slotTracker = Preconditions.checkNotNull(slotTracker);
+		slotTracker.registerSlotStatusUpdateListener(createSlotStatusUpdateListener());
+
+		slotMatchingStrategy = slotManagerConfiguration.getSlotMatchingStrategy();
+
+		taskExecutorManagerFactory = (executor, resourceActions) -> new TaskExecutorManager(
+			slotManagerConfiguration.getDefaultWorkerResourceSpec(),
+			slotManagerConfiguration.getNumSlotsPerWorker(),
+			slotManagerConfiguration.getMaxSlotNum(),
+			slotManagerConfiguration.isWaitResultConsumedBeforeRelease(),
+			slotManagerConfiguration.getRedundantTaskManagerNum(),
+			slotManagerConfiguration.getTaskManagerTimeout(),
+			scheduledExecutor,
+			executor,
+			resourceActions);
+
+		resourceManagerId = null;
+		resourceActions = null;
+		mainThreadExecutor = null;
+		slotRequestTimeoutCheck = null;
+		taskExecutorManager = null;
+
+		started = false;
+	}
+
+	private SlotStatusUpdateListener createSlotStatusUpdateListener() {
+		return (taskManagerSlot, previous, current, jobId) -> {
+			if (previous == SlotState.PENDING) {
+				cancelAllocationFuture(taskManagerSlot.getSlotId());
+			}
+
+			if (current == SlotState.PENDING) {
+				resourceTracker.notifyAcquiredResource(jobId, taskManagerSlot.getResourceProfile());
+			}
+			if (current == SlotState.FREE) {
+				resourceTracker.notifyLostResource(jobId, taskManagerSlot.getResourceProfile());
+			}
+
+			if (current == SlotState.ALLOCATED) {
+				taskExecutorManager.occupySlot(taskManagerSlot.getInstanceId());
+			}
+			if (previous == SlotState.ALLOCATED && current == SlotState.FREE) {
+				taskExecutorManager.freeSlot(taskManagerSlot.getInstanceId());
+			}
+		};
+	}
+
+	private void cancelAllocationFuture(SlotID slotId) {
+		final CompletableFuture<Acknowledge> acknowledgeCompletableFuture = pendingSlotAllocationFutures.remove(slotId);
+		// the future may be null if we are just re-playing the state transitions due to a slot report
+		if (acknowledgeCompletableFuture != null) {
+			acknowledgeCompletableFuture.cancel(false);
+		}
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Component lifecycle methods
+	// ---------------------------------------------------------------------------------------------
+
+	/**
+	 * Starts the slot manager with the given leader id and resource manager actions.
+	 *
+	 * @param newResourceManagerId to use for communication with the task managers
+	 * @param newMainThreadExecutor to use to run code in the ResourceManager's main thread
+	 * @param newResourceActions to use for resource (de-)allocations
+	 */
+	@Override
+	public void start(ResourceManagerId newResourceManagerId, Executor newMainThreadExecutor, ResourceActions newResourceActions) {
+		LOG.info("Starting the slot manager.");
+
+		this.resourceManagerId = Preconditions.checkNotNull(newResourceManagerId);
+		mainThreadExecutor = Preconditions.checkNotNull(newMainThreadExecutor);
+		resourceActions = Preconditions.checkNotNull(newResourceActions);
+		taskExecutorManager = taskExecutorManagerFactory.apply(newMainThreadExecutor, newResourceActions);
+
+		started = true;
+
+		registerSlotManagerMetrics();
+	}
+
+	private void registerSlotManagerMetrics() {
+		slotManagerMetricGroup.gauge(
+			MetricNames.TASK_SLOTS_AVAILABLE,
+			() -> (long) getNumberFreeSlots());
+		slotManagerMetricGroup.gauge(
+			MetricNames.TASK_SLOTS_TOTAL,
+			() -> (long) getNumberRegisteredSlots());
+	}
+
+	/**
+	 * Suspends the component. This clears the internal state of the slot manager.
+	 */
+	@Override
+	public void suspend() {
+		LOG.info("Suspending the slot manager.");
+
+		if (slotRequestTimeoutCheck != null) {
+			slotRequestTimeoutCheck.cancel(false);
+			slotRequestTimeoutCheck = null;
+		}
+
+		resourceTracker.clear();
+		taskExecutorManager.close();
+
+		for (InstanceID registeredTaskManager : taskExecutorManager.getTaskExecutors()) {
+			unregisterTaskManager(registeredTaskManager, new SlotManagerException("The slot manager is being suspended."));
+		}
+
+		taskExecutorManager = null;
+		resourceManagerId = null;
+		resourceActions = null;
+		started = false;
+	}
+
+	/**
+	 * Closes the slot manager.
+	 *
+	 * @throws Exception if the close operation fails
+	 */
+	@Override
+	public void close() throws Exception {
+		LOG.info("Closing the slot manager.");
+
+		suspend();
+		slotManagerMetricGroup.close();
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Public API
+	// ---------------------------------------------------------------------------------------------
+
+	@Override
+	public void processResourceRequirements(ResourceRequirements resourceRequirements) {
+		checkInit();
+		LOG.debug("Received resource requirements from job {}: {}", resourceRequirements.getJobId(), resourceRequirements.getResourceRequirements());
+
+		if (resourceRequirements.getResourceRequirements().isEmpty()) {
+			jobMasterTargetAddresses.remove(resourceRequirements.getJobId());
+		} else {
+			jobMasterTargetAddresses.put(resourceRequirements.getJobId(), resourceRequirements.getTargetAddress());
+		}
+		resourceTracker.notifyResourceRequirements(resourceRequirements.getJobId(), resourceRequirements.getResourceRequirements());
+		checkResourceRequirements();
+	}
+
+	/**
+	 * 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
+	 * @return True if the task manager has not been registered before and is registered successfully; otherwise false
+	 */
+	@Override
+	public boolean registerTaskManager(final TaskExecutorConnection taskExecutorConnection, SlotReport initialSlotReport) {
+		checkInit();
+		LOG.debug("Registering task executor {} under {} at the slot manager.", taskExecutorConnection.getResourceID(), taskExecutorConnection.getInstanceID());
+
+		// we identify task managers by their instance id
+		if (taskExecutorManager.isTaskManagerRegistered(taskExecutorConnection.getInstanceID())) {
+			LOG.debug("Task executor {} was already registered.", taskExecutorConnection.getResourceID());
+			reportSlotStatus(taskExecutorConnection.getInstanceID(), initialSlotReport);
+			return false;
+		} else {
+			if (!taskExecutorManager.registerTaskManager(taskExecutorConnection, initialSlotReport)) {
+				LOG.debug("Task executor {} could not be registered.", taskExecutorConnection.getResourceID());
+				return false;
+			}
+
+			// register the new slots
+			for (SlotStatus slotStatus : initialSlotReport) {
+				slotTracker.addSlot(
+					slotStatus.getSlotID(),
+					slotStatus.getResourceProfile(),
+					taskExecutorConnection,
+					slotStatus.getJobID());
+			}
+
+			checkResourceRequirements();
+			return true;
+		}
+	}
+
+	@Override
+	public boolean unregisterTaskManager(InstanceID instanceId, Exception cause) {
+		checkInit();
+
+		LOG.debug("Unregistering task executor {} from the slot manager.", instanceId);
+
+		if (taskExecutorManager.isTaskManagerRegistered(instanceId)) {
+			slotTracker.removeSlots(taskExecutorManager.getSlotsOf(instanceId));
+			taskExecutorManager.unregisterTaskExecutor(instanceId);
+			checkResourceRequirements();
+
+			return true;
+		} else {
+			LOG.debug("There is no task executor registered with instance ID {}. Ignoring this message.", instanceId);
+
+			return false;
+		}
+	}
+
+	/**
+	 * Reports the current slot allocations for a task manager identified by the given instance id.
+	 *
+	 * @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
+	 */
+	@Override
+	public boolean reportSlotStatus(InstanceID instanceId, SlotReport slotReport) {
+		checkInit();
+
+		LOG.debug("Received slot report from instance {}: {}.", instanceId, slotReport);
+
+		if (taskExecutorManager.isTaskManagerRegistered(instanceId)) {
+			slotTracker.notifySlotStatus(slotReport);
+			checkResourceRequirements();
+			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
+	 */
+	@Override
+	public void freeSlot(SlotID slotId, AllocationID allocationId) {
+		checkInit();
+		LOG.debug("Freeing slot {}.", slotId);
+
+		slotTracker.notifyFree(slotId);
+		checkResourceRequirements();
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Requirement matching
+	// ---------------------------------------------------------------------------------------------
+
+	private void checkResourceRequirements() {
+		final Map<JobID, Collection<ResourceRequirement>> missingResources = resourceTracker.getMissingResources();
+		if (missingResources.isEmpty()) {
+			return;
+		}
+
+		final Map<JobID, ResourceCounter> outstandingRequirements = new LinkedHashMap<>();
+		for (Map.Entry<JobID, Collection<ResourceRequirement>> resourceRequirements : missingResources.entrySet()) {
+			JobID jobId = resourceRequirements.getKey();
+
+			for (ResourceRequirement resourceRequirement : resourceRequirements.getValue()) {
+				int numMissingSlots = internalRequestSlots(jobId, jobMasterTargetAddresses.get(jobId), resourceRequirement);
+				if (numMissingSlots > 0) {
+					outstandingRequirements
+						.computeIfAbsent(jobId, ignored -> new ResourceCounter())
+						.incrementCount(resourceRequirement.getResourceProfile(), numMissingSlots);
+				}
+			}
+		}
+
+		final ResourceCounter pendingSlots = new ResourceCounter(taskExecutorManager.getPendingTaskManagerSlots().stream().collect(
+			Collectors.groupingBy(
+				PendingTaskManagerSlot::getResourceProfile,
+				Collectors.summingInt(x -> 1))));
+
+		for (Map.Entry<JobID, ResourceCounter> unfulfilledRequirement : outstandingRequirements.entrySet()) {
+			tryFulfillRequirementsWithPendingOrNewSlots(
+				unfulfilledRequirement.getKey(),
+				unfulfilledRequirement.getValue().getResourceProfilesWithCount(),
+				pendingSlots);
+		}
+	}
+
+	/**
+	 * Tries to allocate slots for the given requirement. If there are not enough slots available, the
+	 * resource manager is informed to allocate more resources.
+	 *
+	 * @param jobId job to allocate slots for
+	 * @param targetAddress address of the jobmaster
+	 * @param resourceRequirement required slots
+	 * @return the number of missing slots
+	 */
+	private int internalRequestSlots(JobID jobId, String targetAddress, ResourceRequirement resourceRequirement) {
+		final ResourceProfile requiredResource = resourceRequirement.getResourceProfile();
+		Collection<TaskManagerSlotInformation> freeSlots = slotTracker.getFreeSlots();
+
+		int numUnfulfilled = 0;
+		for (int x = 0; x < resourceRequirement.getNumberOfRequiredSlots(); x++) {
+
+			final Optional<TaskManagerSlotInformation> reservedSlot = slotMatchingStrategy.findMatchingSlot(requiredResource, freeSlots, this::getNumberRegisteredSlotsOf);
+			if (reservedSlot.isPresent()) {
+				// we do not need to modify freeSlots because it is indirectly modified by the allocation
+				allocateSlot(reservedSlot.get(), jobId, targetAddress, requiredResource);
+			} else {
+				numUnfulfilled++;
+			}
+		}
+		return numUnfulfilled;
+	}
+
+	/**
+	 * Allocates the given slot. This entails sending a registration message to the task manager and treating failures.
+	 *
+	 * @param taskManagerSlot slot to allocate
+	 * @param jobId job for which the slot should be allocated for
+	 * @param targetAddress address of the job master
+	 * @param resourceProfile resource profile for the requirement for which the slot is used
+	 */
+	private void allocateSlot(TaskManagerSlotInformation taskManagerSlot, JobID jobId, String targetAddress, ResourceProfile resourceProfile) {
+		TaskExecutorConnection taskExecutorConnection = taskManagerSlot.getTaskManagerConnection();
+		TaskExecutorGateway gateway = taskExecutorConnection.getTaskExecutorGateway();
+
+		final CompletableFuture<Acknowledge> completableFuture = new CompletableFuture<>();
+		final SlotID slotId = taskManagerSlot.getSlotId();
+		final InstanceID instanceId = taskManagerSlot.getInstanceId();
+
+		LOG.debug("Starting allocation of slot {} for job {} with resource profile {}.", taskManagerSlot.getSlotId(), jobId, resourceProfile);
+
+		slotTracker.notifyAllocationStart(slotId, jobId);
+
+		if (!taskExecutorManager.isTaskManagerRegistered(instanceId)) {
+			throw new IllegalStateException("Could not find a registered task manager for instance id " +
+				instanceId + '.');
+		}
+
+		taskExecutorManager.markUsed(instanceId);
+
+		// RPC call to the task manager
+		CompletableFuture<Acknowledge> requestFuture = gateway.requestSlot(
+			slotId,
+			jobId,
+			new AllocationID(),
+			resourceProfile,
+			targetAddress,
+			resourceManagerId,
+			taskManagerRequestTimeout);
+
+		requestFuture.whenComplete(
+			(Acknowledge acknowledge, Throwable throwable) -> {
+				if (acknowledge != null) {
+					completableFuture.complete(acknowledge);
+				} else {
+					completableFuture.completeExceptionally(throwable);
+				}
+			});
+
+		completableFuture.whenCompleteAsync(
+			(Acknowledge acknowledge, Throwable throwable) -> {
+				try {
+					if (acknowledge != null) {
+						LOG.trace("Completed allocation of slot {} for job {}.", slotId, jobId);
+						slotTracker.notifyAllocationComplete(slotId, jobId);
+					} else {
+						if (throwable instanceof SlotOccupiedException) {
+							SlotOccupiedException exception = (SlotOccupiedException) throwable;
+							LOG.debug("Tried allocating slot {} for job {}, but it was already allocated for job {}.", slotId, jobId, exception.getJobId());
+							// report as a slot status to force the state transition
+							// this could be a problem if we ever assume that the task executor always reports about all slots
+							slotTracker.notifySlotStatus(Collections.singleton(new SlotStatus(slotId, taskManagerSlot.getResourceProfile(), exception.getJobId(), exception.getAllocationId())));
+						} else {
+							if (throwable instanceof CancellationException) {
+								LOG.debug("Cancelled allocation of slot {} for job {}.", slotId, jobId, throwable);
+							} else {
+								LOG.warn("Slot allocation for slot {} for job {} failed.", slotId, jobId, throwable);
+								slotTracker.notifyFree(slotId);
+							}
+						}
+					}
+					checkResourceRequirements();
+				} catch (Exception e) {
+					LOG.error("Error while completing the slot allocation.", e);
+				}

Review comment:
       Where does the exception come from? If this exception is the result of an illegal state, then we should instead use handleAsync and wrap it with `FutureUtils.assertNoException` to fail hard in case of an illegal state.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManager.java
##########
@@ -0,0 +1,609 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.metrics.MetricNames;
+import org.apache.flink.runtime.metrics.groups.SlotManagerMetricGroup;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerId;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.slots.ResourceCounter;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+import org.apache.flink.runtime.slots.ResourceRequirements;
+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.SlotOccupiedException;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ScheduledFuture;
+import java.util.function.BiFunction;
+import java.util.stream.Collectors;
+
+/**
+ * Implementation of {@link SlotManager} supporting declarative slot management.
+ */
+public class DeclarativeSlotManager implements SlotManager {
+	private static final Logger LOG = LoggerFactory.getLogger(DeclarativeSlotManager.class);
+
+	private final SlotTracker slotTracker;
+	private final ResourceTracker resourceTracker;
+	private final BiFunction<Executor, ResourceActions, TaskExecutorManager> taskExecutorManagerFactory;
+	private TaskExecutorManager taskExecutorManager;
+
+	/** Timeout for slot requests to the task manager. */
+	private final Time taskManagerRequestTimeout;
+	private ScheduledFuture<?> slotRequestTimeoutCheck;

Review comment:
       For what do we need the `slotRequestTimeoutCheck`?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManager.java
##########
@@ -0,0 +1,609 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.metrics.MetricNames;
+import org.apache.flink.runtime.metrics.groups.SlotManagerMetricGroup;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerId;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.slots.ResourceCounter;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+import org.apache.flink.runtime.slots.ResourceRequirements;
+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.SlotOccupiedException;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ScheduledFuture;
+import java.util.function.BiFunction;
+import java.util.stream.Collectors;
+
+/**
+ * Implementation of {@link SlotManager} supporting declarative slot management.
+ */
+public class DeclarativeSlotManager implements SlotManager {
+	private static final Logger LOG = LoggerFactory.getLogger(DeclarativeSlotManager.class);
+
+	private final SlotTracker slotTracker;
+	private final ResourceTracker resourceTracker;
+	private final BiFunction<Executor, ResourceActions, TaskExecutorManager> taskExecutorManagerFactory;
+	private TaskExecutorManager taskExecutorManager;
+
+	/** Timeout for slot requests to the task manager. */
+	private final Time taskManagerRequestTimeout;
+	private ScheduledFuture<?> slotRequestTimeoutCheck;
+
+	private final SlotMatchingStrategy slotMatchingStrategy;
+
+	private final SlotManagerMetricGroup slotManagerMetricGroup;
+
+	private final Map<JobID, String> jobMasterTargetAddresses = new HashMap<>();
+	private final HashMap<SlotID, CompletableFuture<Acknowledge>> pendingSlotAllocationFutures;
+
+	/** ResourceManager's id. */
+	private ResourceManagerId resourceManagerId;
+
+	/** Executor for future callbacks which have to be "synchronized". */
+	private Executor mainThreadExecutor;
+
+	/** Callbacks for resource (de-)allocations. */
+	private ResourceActions resourceActions;
+
+	/** True iff the component has been started. */
+	private boolean started;
+
+	public DeclarativeSlotManager(
+			ScheduledExecutor scheduledExecutor,
+			SlotManagerConfiguration slotManagerConfiguration,
+			SlotManagerMetricGroup slotManagerMetricGroup,
+			ResourceTracker resourceTracker,
+			SlotTracker slotTracker) {
+
+		Preconditions.checkNotNull(slotManagerConfiguration);
+		this.taskManagerRequestTimeout = slotManagerConfiguration.getTaskManagerRequestTimeout();
+		this.slotManagerMetricGroup = Preconditions.checkNotNull(slotManagerMetricGroup);
+		this.resourceTracker = Preconditions.checkNotNull(resourceTracker);
+
+		pendingSlotAllocationFutures = new HashMap<>(16);
+
+		this.slotTracker = Preconditions.checkNotNull(slotTracker);
+		slotTracker.registerSlotStatusUpdateListener(createSlotStatusUpdateListener());
+
+		slotMatchingStrategy = slotManagerConfiguration.getSlotMatchingStrategy();
+
+		taskExecutorManagerFactory = (executor, resourceActions) -> new TaskExecutorManager(
+			slotManagerConfiguration.getDefaultWorkerResourceSpec(),
+			slotManagerConfiguration.getNumSlotsPerWorker(),
+			slotManagerConfiguration.getMaxSlotNum(),
+			slotManagerConfiguration.isWaitResultConsumedBeforeRelease(),
+			slotManagerConfiguration.getRedundantTaskManagerNum(),
+			slotManagerConfiguration.getTaskManagerTimeout(),
+			scheduledExecutor,
+			executor,
+			resourceActions);
+
+		resourceManagerId = null;
+		resourceActions = null;
+		mainThreadExecutor = null;
+		slotRequestTimeoutCheck = null;
+		taskExecutorManager = null;
+
+		started = false;
+	}
+
+	private SlotStatusUpdateListener createSlotStatusUpdateListener() {
+		return (taskManagerSlot, previous, current, jobId) -> {
+			if (previous == SlotState.PENDING) {
+				cancelAllocationFuture(taskManagerSlot.getSlotId());
+			}
+
+			if (current == SlotState.PENDING) {
+				resourceTracker.notifyAcquiredResource(jobId, taskManagerSlot.getResourceProfile());
+			}
+			if (current == SlotState.FREE) {
+				resourceTracker.notifyLostResource(jobId, taskManagerSlot.getResourceProfile());
+			}
+
+			if (current == SlotState.ALLOCATED) {
+				taskExecutorManager.occupySlot(taskManagerSlot.getInstanceId());
+			}
+			if (previous == SlotState.ALLOCATED && current == SlotState.FREE) {
+				taskExecutorManager.freeSlot(taskManagerSlot.getInstanceId());
+			}
+		};
+	}
+
+	private void cancelAllocationFuture(SlotID slotId) {
+		final CompletableFuture<Acknowledge> acknowledgeCompletableFuture = pendingSlotAllocationFutures.remove(slotId);
+		// the future may be null if we are just re-playing the state transitions due to a slot report
+		if (acknowledgeCompletableFuture != null) {
+			acknowledgeCompletableFuture.cancel(false);
+		}
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Component lifecycle methods
+	// ---------------------------------------------------------------------------------------------
+
+	/**
+	 * Starts the slot manager with the given leader id and resource manager actions.
+	 *
+	 * @param newResourceManagerId to use for communication with the task managers
+	 * @param newMainThreadExecutor to use to run code in the ResourceManager's main thread
+	 * @param newResourceActions to use for resource (de-)allocations
+	 */
+	@Override
+	public void start(ResourceManagerId newResourceManagerId, Executor newMainThreadExecutor, ResourceActions newResourceActions) {
+		LOG.info("Starting the slot manager.");
+
+		this.resourceManagerId = Preconditions.checkNotNull(newResourceManagerId);
+		mainThreadExecutor = Preconditions.checkNotNull(newMainThreadExecutor);
+		resourceActions = Preconditions.checkNotNull(newResourceActions);
+		taskExecutorManager = taskExecutorManagerFactory.apply(newMainThreadExecutor, newResourceActions);
+
+		started = true;
+
+		registerSlotManagerMetrics();
+	}
+
+	private void registerSlotManagerMetrics() {
+		slotManagerMetricGroup.gauge(
+			MetricNames.TASK_SLOTS_AVAILABLE,
+			() -> (long) getNumberFreeSlots());
+		slotManagerMetricGroup.gauge(
+			MetricNames.TASK_SLOTS_TOTAL,
+			() -> (long) getNumberRegisteredSlots());
+	}
+
+	/**
+	 * Suspends the component. This clears the internal state of the slot manager.
+	 */
+	@Override
+	public void suspend() {
+		LOG.info("Suspending the slot manager.");
+
+		if (slotRequestTimeoutCheck != null) {
+			slotRequestTimeoutCheck.cancel(false);
+			slotRequestTimeoutCheck = null;
+		}
+
+		resourceTracker.clear();
+		taskExecutorManager.close();
+
+		for (InstanceID registeredTaskManager : taskExecutorManager.getTaskExecutors()) {
+			unregisterTaskManager(registeredTaskManager, new SlotManagerException("The slot manager is being suspended."));
+		}
+
+		taskExecutorManager = null;
+		resourceManagerId = null;
+		resourceActions = null;
+		started = false;
+	}
+
+	/**
+	 * Closes the slot manager.
+	 *
+	 * @throws Exception if the close operation fails
+	 */
+	@Override
+	public void close() throws Exception {
+		LOG.info("Closing the slot manager.");
+
+		suspend();
+		slotManagerMetricGroup.close();
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Public API
+	// ---------------------------------------------------------------------------------------------
+
+	@Override
+	public void processResourceRequirements(ResourceRequirements resourceRequirements) {
+		checkInit();
+		LOG.debug("Received resource requirements from job {}: {}", resourceRequirements.getJobId(), resourceRequirements.getResourceRequirements());
+
+		if (resourceRequirements.getResourceRequirements().isEmpty()) {
+			jobMasterTargetAddresses.remove(resourceRequirements.getJobId());
+		} else {
+			jobMasterTargetAddresses.put(resourceRequirements.getJobId(), resourceRequirements.getTargetAddress());
+		}
+		resourceTracker.notifyResourceRequirements(resourceRequirements.getJobId(), resourceRequirements.getResourceRequirements());
+		checkResourceRequirements();
+	}
+
+	/**
+	 * 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
+	 * @return True if the task manager has not been registered before and is registered successfully; otherwise false
+	 */
+	@Override
+	public boolean registerTaskManager(final TaskExecutorConnection taskExecutorConnection, SlotReport initialSlotReport) {
+		checkInit();
+		LOG.debug("Registering task executor {} under {} at the slot manager.", taskExecutorConnection.getResourceID(), taskExecutorConnection.getInstanceID());
+
+		// we identify task managers by their instance id
+		if (taskExecutorManager.isTaskManagerRegistered(taskExecutorConnection.getInstanceID())) {
+			LOG.debug("Task executor {} was already registered.", taskExecutorConnection.getResourceID());
+			reportSlotStatus(taskExecutorConnection.getInstanceID(), initialSlotReport);
+			return false;
+		} else {
+			if (!taskExecutorManager.registerTaskManager(taskExecutorConnection, initialSlotReport)) {
+				LOG.debug("Task executor {} could not be registered.", taskExecutorConnection.getResourceID());
+				return false;
+			}
+
+			// register the new slots
+			for (SlotStatus slotStatus : initialSlotReport) {
+				slotTracker.addSlot(
+					slotStatus.getSlotID(),
+					slotStatus.getResourceProfile(),
+					taskExecutorConnection,
+					slotStatus.getJobID());
+			}
+
+			checkResourceRequirements();
+			return true;
+		}
+	}
+
+	@Override
+	public boolean unregisterTaskManager(InstanceID instanceId, Exception cause) {
+		checkInit();
+
+		LOG.debug("Unregistering task executor {} from the slot manager.", instanceId);
+
+		if (taskExecutorManager.isTaskManagerRegistered(instanceId)) {
+			slotTracker.removeSlots(taskExecutorManager.getSlotsOf(instanceId));
+			taskExecutorManager.unregisterTaskExecutor(instanceId);
+			checkResourceRequirements();
+
+			return true;
+		} else {
+			LOG.debug("There is no task executor registered with instance ID {}. Ignoring this message.", instanceId);
+
+			return false;
+		}
+	}
+
+	/**
+	 * Reports the current slot allocations for a task manager identified by the given instance id.
+	 *
+	 * @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
+	 */
+	@Override
+	public boolean reportSlotStatus(InstanceID instanceId, SlotReport slotReport) {
+		checkInit();
+
+		LOG.debug("Received slot report from instance {}: {}.", instanceId, slotReport);
+
+		if (taskExecutorManager.isTaskManagerRegistered(instanceId)) {
+			slotTracker.notifySlotStatus(slotReport);
+			checkResourceRequirements();
+			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
+	 */
+	@Override
+	public void freeSlot(SlotID slotId, AllocationID allocationId) {
+		checkInit();
+		LOG.debug("Freeing slot {}.", slotId);
+
+		slotTracker.notifyFree(slotId);
+		checkResourceRequirements();
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Requirement matching
+	// ---------------------------------------------------------------------------------------------
+
+	private void checkResourceRequirements() {
+		final Map<JobID, Collection<ResourceRequirement>> missingResources = resourceTracker.getMissingResources();
+		if (missingResources.isEmpty()) {
+			return;
+		}
+
+		final Map<JobID, ResourceCounter> outstandingRequirements = new LinkedHashMap<>();
+		for (Map.Entry<JobID, Collection<ResourceRequirement>> resourceRequirements : missingResources.entrySet()) {
+			JobID jobId = resourceRequirements.getKey();
+
+			for (ResourceRequirement resourceRequirement : resourceRequirements.getValue()) {
+				int numMissingSlots = internalRequestSlots(jobId, jobMasterTargetAddresses.get(jobId), resourceRequirement);
+				if (numMissingSlots > 0) {
+					outstandingRequirements
+						.computeIfAbsent(jobId, ignored -> new ResourceCounter())
+						.incrementCount(resourceRequirement.getResourceProfile(), numMissingSlots);
+				}
+			}
+		}
+
+		final ResourceCounter pendingSlots = new ResourceCounter(taskExecutorManager.getPendingTaskManagerSlots().stream().collect(
+			Collectors.groupingBy(
+				PendingTaskManagerSlot::getResourceProfile,
+				Collectors.summingInt(x -> 1))));
+
+		for (Map.Entry<JobID, ResourceCounter> unfulfilledRequirement : outstandingRequirements.entrySet()) {
+			tryFulfillRequirementsWithPendingOrNewSlots(
+				unfulfilledRequirement.getKey(),
+				unfulfilledRequirement.getValue().getResourceProfilesWithCount(),
+				pendingSlots);
+		}
+	}
+
+	/**
+	 * Tries to allocate slots for the given requirement. If there are not enough slots available, the
+	 * resource manager is informed to allocate more resources.
+	 *
+	 * @param jobId job to allocate slots for
+	 * @param targetAddress address of the jobmaster
+	 * @param resourceRequirement required slots
+	 * @return the number of missing slots
+	 */
+	private int internalRequestSlots(JobID jobId, String targetAddress, ResourceRequirement resourceRequirement) {
+		final ResourceProfile requiredResource = resourceRequirement.getResourceProfile();
+		Collection<TaskManagerSlotInformation> freeSlots = slotTracker.getFreeSlots();
+
+		int numUnfulfilled = 0;
+		for (int x = 0; x < resourceRequirement.getNumberOfRequiredSlots(); x++) {
+
+			final Optional<TaskManagerSlotInformation> reservedSlot = slotMatchingStrategy.findMatchingSlot(requiredResource, freeSlots, this::getNumberRegisteredSlotsOf);
+			if (reservedSlot.isPresent()) {
+				// we do not need to modify freeSlots because it is indirectly modified by the allocation
+				allocateSlot(reservedSlot.get(), jobId, targetAddress, requiredResource);
+			} else {
+				numUnfulfilled++;
+			}
+		}
+		return numUnfulfilled;
+	}
+
+	/**
+	 * Allocates the given slot. This entails sending a registration message to the task manager and treating failures.
+	 *
+	 * @param taskManagerSlot slot to allocate
+	 * @param jobId job for which the slot should be allocated for
+	 * @param targetAddress address of the job master
+	 * @param resourceProfile resource profile for the requirement for which the slot is used
+	 */
+	private void allocateSlot(TaskManagerSlotInformation taskManagerSlot, JobID jobId, String targetAddress, ResourceProfile resourceProfile) {
+		TaskExecutorConnection taskExecutorConnection = taskManagerSlot.getTaskManagerConnection();
+		TaskExecutorGateway gateway = taskExecutorConnection.getTaskExecutorGateway();
+
+		final CompletableFuture<Acknowledge> completableFuture = new CompletableFuture<>();
+		final SlotID slotId = taskManagerSlot.getSlotId();
+		final InstanceID instanceId = taskManagerSlot.getInstanceId();
+
+		LOG.debug("Starting allocation of slot {} for job {} with resource profile {}.", taskManagerSlot.getSlotId(), jobId, resourceProfile);
+
+		slotTracker.notifyAllocationStart(slotId, jobId);
+
+		if (!taskExecutorManager.isTaskManagerRegistered(instanceId)) {
+			throw new IllegalStateException("Could not find a registered task manager for instance id " +
+				instanceId + '.');
+		}
+
+		taskExecutorManager.markUsed(instanceId);
+
+		// RPC call to the task manager
+		CompletableFuture<Acknowledge> requestFuture = gateway.requestSlot(
+			slotId,
+			jobId,
+			new AllocationID(),
+			resourceProfile,
+			targetAddress,
+			resourceManagerId,
+			taskManagerRequestTimeout);
+
+		requestFuture.whenComplete(
+			(Acknowledge acknowledge, Throwable throwable) -> {
+				if (acknowledge != null) {
+					completableFuture.complete(acknowledge);
+				} else {
+					completableFuture.completeExceptionally(throwable);
+				}
+			});
+
+		completableFuture.whenCompleteAsync(
+			(Acknowledge acknowledge, Throwable throwable) -> {
+				try {
+					if (acknowledge != null) {
+						LOG.trace("Completed allocation of slot {} for job {}.", slotId, jobId);
+						slotTracker.notifyAllocationComplete(slotId, jobId);
+					} else {
+						if (throwable instanceof SlotOccupiedException) {
+							SlotOccupiedException exception = (SlotOccupiedException) throwable;
+							LOG.debug("Tried allocating slot {} for job {}, but it was already allocated for job {}.", slotId, jobId, exception.getJobId());
+							// report as a slot status to force the state transition
+							// this could be a problem if we ever assume that the task executor always reports about all slots
+							slotTracker.notifySlotStatus(Collections.singleton(new SlotStatus(slotId, taskManagerSlot.getResourceProfile(), exception.getJobId(), exception.getAllocationId())));
+						} else {
+							if (throwable instanceof CancellationException) {
+								LOG.debug("Cancelled allocation of slot {} for job {}.", slotId, jobId, throwable);

Review comment:
       Where are we cancelling `completableFuture`?

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManagerTest.java
##########
@@ -0,0 +1,1092 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple6;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.concurrent.ManuallyTriggeredScheduledExecutor;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerId;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+import org.apache.flink.runtime.slots.ResourceRequirements;
+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.TestingTaskExecutorGateway;
+import org.apache.flink.runtime.taskexecutor.TestingTaskExecutorGatewayBuilder;
+import org.apache.flink.runtime.taskexecutor.exceptions.SlotAllocationException;
+import org.apache.flink.runtime.taskexecutor.exceptions.SlotOccupiedException;
+import org.apache.flink.runtime.testingUtils.TestingUtils;
+import org.apache.flink.util.FlinkException;
+import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.function.FunctionUtils;
+
+import akka.pattern.AskTimeoutException;
+import org.junit.Ignore;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.hamcrest.CoreMatchers.hasItem;
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.empty;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.hasSize;
+import static org.hamcrest.Matchers.is;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertSame;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Tests for the {@link DeclarativeSlotManager}.
+ */
+public class DeclarativeSlotManagerTest extends TestLogger {
+
+	private static final FlinkException TEST_EXCEPTION = new FlinkException("Test exception");
+
+	private static final WorkerResourceSpec WORKER_RESOURCE_SPEC = new WorkerResourceSpec.Builder()
+		.setCpuCores(100.0)
+		.setTaskHeapMemoryMB(10000)
+		.setTaskOffHeapMemoryMB(10000)
+		.setNetworkMemoryMB(10000)
+		.setManagedMemoryMB(10000)
+		.build();
+
+	/**
+	 * Tests that we can register task manager and their slots at the slot manager.
+	 */
+	@Test
+	public void testTaskManagerRegistration() throws Exception {
+		final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder().createTestingTaskExecutorGateway();
+		final ResourceID resourceId = ResourceID.generate();
+		final TaskExecutorConnection taskManagerConnection = new TaskExecutorConnection(resourceId, taskExecutorGateway);
+
+		final SlotID slotId1 = new SlotID(resourceId, 0);
+		final SlotID slotId2 = new SlotID(resourceId, 1);
+		final SlotReport slotReport = new SlotReport(Arrays.asList(createFreeSlotStatus(slotId1), createFreeSlotStatus(slotId2)));
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport);
+
+			assertThat("The number registered slots does not equal the expected number.", slotManager.getNumberRegisteredSlots(), is(2));
+
+			assertNotNull(slotTracker.getSlot(slotId1));
+			assertNotNull(slotTracker.getSlot(slotId2));
+		}
+	}
+
+	/**
+	 * Tests that un-registration of task managers will free and remove all registered slots.
+	 */
+	@Test
+	public void testTaskManagerUnregistration() throws Exception {
+		final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder()
+			.setRequestSlotFunction(tuple6 -> new CompletableFuture<>())
+			.createTestingTaskExecutorGateway();
+		final TaskExecutorConnection taskManagerConnection = createTaskExecutorConnection(taskExecutorGateway);
+		final ResourceID resourceId = taskManagerConnection.getResourceID();
+
+		final SlotID slotId1 = new SlotID(resourceId, 0);
+		final SlotID slotId2 = new SlotID(resourceId, 1);
+		final SlotReport slotReport = new SlotReport(Arrays.asList(createAllocatedSlotStatus(slotId1), createFreeSlotStatus(slotId2)));
+
+		final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot();
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport);
+
+			assertEquals("The number registered slots does not equal the expected number.", 2, slotManager.getNumberRegisteredSlots());
+
+			DeclarativeTaskManagerSlot slot1 = slotTracker.getSlot(slotId1);
+			DeclarativeTaskManagerSlot slot2 = slotTracker.getSlot(slotId2);
+
+			assertSame(SlotState.ALLOCATED, slot1.getState());
+			assertSame(SlotState.FREE, slot2.getState());

Review comment:
       Do we really have to look at the internal state of the slots in order to ensure that after a TM gets unregistered that there are no more slots? One could simply use `slotManager.getNumberRegisteredSlots()` for this.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManager.java
##########
@@ -0,0 +1,609 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.metrics.MetricNames;
+import org.apache.flink.runtime.metrics.groups.SlotManagerMetricGroup;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerId;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.slots.ResourceCounter;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+import org.apache.flink.runtime.slots.ResourceRequirements;
+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.SlotOccupiedException;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ScheduledFuture;
+import java.util.function.BiFunction;
+import java.util.stream.Collectors;
+
+/**
+ * Implementation of {@link SlotManager} supporting declarative slot management.
+ */
+public class DeclarativeSlotManager implements SlotManager {
+	private static final Logger LOG = LoggerFactory.getLogger(DeclarativeSlotManager.class);
+
+	private final SlotTracker slotTracker;
+	private final ResourceTracker resourceTracker;
+	private final BiFunction<Executor, ResourceActions, TaskExecutorManager> taskExecutorManagerFactory;
+	private TaskExecutorManager taskExecutorManager;
+
+	/** Timeout for slot requests to the task manager. */
+	private final Time taskManagerRequestTimeout;
+	private ScheduledFuture<?> slotRequestTimeoutCheck;
+
+	private final SlotMatchingStrategy slotMatchingStrategy;
+
+	private final SlotManagerMetricGroup slotManagerMetricGroup;
+
+	private final Map<JobID, String> jobMasterTargetAddresses = new HashMap<>();
+	private final HashMap<SlotID, CompletableFuture<Acknowledge>> pendingSlotAllocationFutures;
+
+	/** ResourceManager's id. */
+	private ResourceManagerId resourceManagerId;
+
+	/** Executor for future callbacks which have to be "synchronized". */
+	private Executor mainThreadExecutor;
+
+	/** Callbacks for resource (de-)allocations. */
+	private ResourceActions resourceActions;
+
+	/** True iff the component has been started. */
+	private boolean started;
+
+	public DeclarativeSlotManager(
+			ScheduledExecutor scheduledExecutor,
+			SlotManagerConfiguration slotManagerConfiguration,
+			SlotManagerMetricGroup slotManagerMetricGroup,
+			ResourceTracker resourceTracker,
+			SlotTracker slotTracker) {
+
+		Preconditions.checkNotNull(slotManagerConfiguration);
+		this.taskManagerRequestTimeout = slotManagerConfiguration.getTaskManagerRequestTimeout();
+		this.slotManagerMetricGroup = Preconditions.checkNotNull(slotManagerMetricGroup);
+		this.resourceTracker = Preconditions.checkNotNull(resourceTracker);
+
+		pendingSlotAllocationFutures = new HashMap<>(16);
+
+		this.slotTracker = Preconditions.checkNotNull(slotTracker);
+		slotTracker.registerSlotStatusUpdateListener(createSlotStatusUpdateListener());
+
+		slotMatchingStrategy = slotManagerConfiguration.getSlotMatchingStrategy();
+
+		taskExecutorManagerFactory = (executor, resourceActions) -> new TaskExecutorManager(
+			slotManagerConfiguration.getDefaultWorkerResourceSpec(),
+			slotManagerConfiguration.getNumSlotsPerWorker(),
+			slotManagerConfiguration.getMaxSlotNum(),
+			slotManagerConfiguration.isWaitResultConsumedBeforeRelease(),
+			slotManagerConfiguration.getRedundantTaskManagerNum(),
+			slotManagerConfiguration.getTaskManagerTimeout(),
+			scheduledExecutor,
+			executor,
+			resourceActions);
+
+		resourceManagerId = null;
+		resourceActions = null;
+		mainThreadExecutor = null;
+		slotRequestTimeoutCheck = null;
+		taskExecutorManager = null;
+
+		started = false;
+	}
+
+	private SlotStatusUpdateListener createSlotStatusUpdateListener() {
+		return (taskManagerSlot, previous, current, jobId) -> {
+			if (previous == SlotState.PENDING) {
+				cancelAllocationFuture(taskManagerSlot.getSlotId());
+			}
+
+			if (current == SlotState.PENDING) {
+				resourceTracker.notifyAcquiredResource(jobId, taskManagerSlot.getResourceProfile());
+			}
+			if (current == SlotState.FREE) {
+				resourceTracker.notifyLostResource(jobId, taskManagerSlot.getResourceProfile());
+			}
+
+			if (current == SlotState.ALLOCATED) {
+				taskExecutorManager.occupySlot(taskManagerSlot.getInstanceId());
+			}
+			if (previous == SlotState.ALLOCATED && current == SlotState.FREE) {
+				taskExecutorManager.freeSlot(taskManagerSlot.getInstanceId());
+			}
+		};
+	}
+
+	private void cancelAllocationFuture(SlotID slotId) {
+		final CompletableFuture<Acknowledge> acknowledgeCompletableFuture = pendingSlotAllocationFutures.remove(slotId);
+		// the future may be null if we are just re-playing the state transitions due to a slot report
+		if (acknowledgeCompletableFuture != null) {
+			acknowledgeCompletableFuture.cancel(false);
+		}
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Component lifecycle methods
+	// ---------------------------------------------------------------------------------------------
+
+	/**
+	 * Starts the slot manager with the given leader id and resource manager actions.
+	 *
+	 * @param newResourceManagerId to use for communication with the task managers
+	 * @param newMainThreadExecutor to use to run code in the ResourceManager's main thread
+	 * @param newResourceActions to use for resource (de-)allocations
+	 */
+	@Override
+	public void start(ResourceManagerId newResourceManagerId, Executor newMainThreadExecutor, ResourceActions newResourceActions) {
+		LOG.info("Starting the slot manager.");
+
+		this.resourceManagerId = Preconditions.checkNotNull(newResourceManagerId);
+		mainThreadExecutor = Preconditions.checkNotNull(newMainThreadExecutor);
+		resourceActions = Preconditions.checkNotNull(newResourceActions);
+		taskExecutorManager = taskExecutorManagerFactory.apply(newMainThreadExecutor, newResourceActions);
+
+		started = true;
+
+		registerSlotManagerMetrics();
+	}
+
+	private void registerSlotManagerMetrics() {
+		slotManagerMetricGroup.gauge(
+			MetricNames.TASK_SLOTS_AVAILABLE,
+			() -> (long) getNumberFreeSlots());
+		slotManagerMetricGroup.gauge(
+			MetricNames.TASK_SLOTS_TOTAL,
+			() -> (long) getNumberRegisteredSlots());
+	}
+
+	/**
+	 * Suspends the component. This clears the internal state of the slot manager.
+	 */
+	@Override
+	public void suspend() {
+		LOG.info("Suspending the slot manager.");
+
+		if (slotRequestTimeoutCheck != null) {
+			slotRequestTimeoutCheck.cancel(false);
+			slotRequestTimeoutCheck = null;
+		}
+
+		resourceTracker.clear();
+		taskExecutorManager.close();
+
+		for (InstanceID registeredTaskManager : taskExecutorManager.getTaskExecutors()) {
+			unregisterTaskManager(registeredTaskManager, new SlotManagerException("The slot manager is being suspended."));
+		}
+
+		taskExecutorManager = null;
+		resourceManagerId = null;
+		resourceActions = null;
+		started = false;
+	}
+
+	/**
+	 * Closes the slot manager.
+	 *
+	 * @throws Exception if the close operation fails
+	 */
+	@Override
+	public void close() throws Exception {
+		LOG.info("Closing the slot manager.");
+
+		suspend();
+		slotManagerMetricGroup.close();
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Public API
+	// ---------------------------------------------------------------------------------------------
+
+	@Override
+	public void processResourceRequirements(ResourceRequirements resourceRequirements) {
+		checkInit();
+		LOG.debug("Received resource requirements from job {}: {}", resourceRequirements.getJobId(), resourceRequirements.getResourceRequirements());
+
+		if (resourceRequirements.getResourceRequirements().isEmpty()) {
+			jobMasterTargetAddresses.remove(resourceRequirements.getJobId());
+		} else {
+			jobMasterTargetAddresses.put(resourceRequirements.getJobId(), resourceRequirements.getTargetAddress());
+		}
+		resourceTracker.notifyResourceRequirements(resourceRequirements.getJobId(), resourceRequirements.getResourceRequirements());
+		checkResourceRequirements();
+	}
+
+	/**
+	 * 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
+	 * @return True if the task manager has not been registered before and is registered successfully; otherwise false
+	 */
+	@Override
+	public boolean registerTaskManager(final TaskExecutorConnection taskExecutorConnection, SlotReport initialSlotReport) {
+		checkInit();
+		LOG.debug("Registering task executor {} under {} at the slot manager.", taskExecutorConnection.getResourceID(), taskExecutorConnection.getInstanceID());
+
+		// we identify task managers by their instance id
+		if (taskExecutorManager.isTaskManagerRegistered(taskExecutorConnection.getInstanceID())) {
+			LOG.debug("Task executor {} was already registered.", taskExecutorConnection.getResourceID());
+			reportSlotStatus(taskExecutorConnection.getInstanceID(), initialSlotReport);
+			return false;
+		} else {
+			if (!taskExecutorManager.registerTaskManager(taskExecutorConnection, initialSlotReport)) {
+				LOG.debug("Task executor {} could not be registered.", taskExecutorConnection.getResourceID());
+				return false;
+			}
+
+			// register the new slots
+			for (SlotStatus slotStatus : initialSlotReport) {
+				slotTracker.addSlot(
+					slotStatus.getSlotID(),
+					slotStatus.getResourceProfile(),
+					taskExecutorConnection,
+					slotStatus.getJobID());
+			}
+
+			checkResourceRequirements();
+			return true;
+		}
+	}
+
+	@Override
+	public boolean unregisterTaskManager(InstanceID instanceId, Exception cause) {
+		checkInit();
+
+		LOG.debug("Unregistering task executor {} from the slot manager.", instanceId);
+
+		if (taskExecutorManager.isTaskManagerRegistered(instanceId)) {
+			slotTracker.removeSlots(taskExecutorManager.getSlotsOf(instanceId));
+			taskExecutorManager.unregisterTaskExecutor(instanceId);
+			checkResourceRequirements();
+
+			return true;
+		} else {
+			LOG.debug("There is no task executor registered with instance ID {}. Ignoring this message.", instanceId);
+
+			return false;
+		}
+	}
+
+	/**
+	 * Reports the current slot allocations for a task manager identified by the given instance id.
+	 *
+	 * @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
+	 */
+	@Override
+	public boolean reportSlotStatus(InstanceID instanceId, SlotReport slotReport) {
+		checkInit();
+
+		LOG.debug("Received slot report from instance {}: {}.", instanceId, slotReport);
+
+		if (taskExecutorManager.isTaskManagerRegistered(instanceId)) {
+			slotTracker.notifySlotStatus(slotReport);
+			checkResourceRequirements();
+			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
+	 */
+	@Override
+	public void freeSlot(SlotID slotId, AllocationID allocationId) {
+		checkInit();
+		LOG.debug("Freeing slot {}.", slotId);
+
+		slotTracker.notifyFree(slotId);
+		checkResourceRequirements();
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Requirement matching
+	// ---------------------------------------------------------------------------------------------
+
+	private void checkResourceRequirements() {
+		final Map<JobID, Collection<ResourceRequirement>> missingResources = resourceTracker.getMissingResources();
+		if (missingResources.isEmpty()) {
+			return;
+		}
+
+		final Map<JobID, ResourceCounter> outstandingRequirements = new LinkedHashMap<>();
+		for (Map.Entry<JobID, Collection<ResourceRequirement>> resourceRequirements : missingResources.entrySet()) {
+			JobID jobId = resourceRequirements.getKey();
+
+			for (ResourceRequirement resourceRequirement : resourceRequirements.getValue()) {
+				int numMissingSlots = internalRequestSlots(jobId, jobMasterTargetAddresses.get(jobId), resourceRequirement);
+				if (numMissingSlots > 0) {
+					outstandingRequirements
+						.computeIfAbsent(jobId, ignored -> new ResourceCounter())
+						.incrementCount(resourceRequirement.getResourceProfile(), numMissingSlots);
+				}
+			}
+		}
+
+		final ResourceCounter pendingSlots = new ResourceCounter(taskExecutorManager.getPendingTaskManagerSlots().stream().collect(
+			Collectors.groupingBy(
+				PendingTaskManagerSlot::getResourceProfile,
+				Collectors.summingInt(x -> 1))));
+
+		for (Map.Entry<JobID, ResourceCounter> unfulfilledRequirement : outstandingRequirements.entrySet()) {
+			tryFulfillRequirementsWithPendingOrNewSlots(
+				unfulfilledRequirement.getKey(),
+				unfulfilledRequirement.getValue().getResourceProfilesWithCount(),
+				pendingSlots);
+		}
+	}
+
+	/**
+	 * Tries to allocate slots for the given requirement. If there are not enough slots available, the
+	 * resource manager is informed to allocate more resources.
+	 *
+	 * @param jobId job to allocate slots for
+	 * @param targetAddress address of the jobmaster
+	 * @param resourceRequirement required slots
+	 * @return the number of missing slots
+	 */
+	private int internalRequestSlots(JobID jobId, String targetAddress, ResourceRequirement resourceRequirement) {
+		final ResourceProfile requiredResource = resourceRequirement.getResourceProfile();
+		Collection<TaskManagerSlotInformation> freeSlots = slotTracker.getFreeSlots();
+
+		int numUnfulfilled = 0;
+		for (int x = 0; x < resourceRequirement.getNumberOfRequiredSlots(); x++) {
+
+			final Optional<TaskManagerSlotInformation> reservedSlot = slotMatchingStrategy.findMatchingSlot(requiredResource, freeSlots, this::getNumberRegisteredSlotsOf);
+			if (reservedSlot.isPresent()) {
+				// we do not need to modify freeSlots because it is indirectly modified by the allocation
+				allocateSlot(reservedSlot.get(), jobId, targetAddress, requiredResource);
+			} else {
+				numUnfulfilled++;
+			}
+		}
+		return numUnfulfilled;
+	}
+
+	/**
+	 * Allocates the given slot. This entails sending a registration message to the task manager and treating failures.
+	 *
+	 * @param taskManagerSlot slot to allocate
+	 * @param jobId job for which the slot should be allocated for
+	 * @param targetAddress address of the job master
+	 * @param resourceProfile resource profile for the requirement for which the slot is used
+	 */
+	private void allocateSlot(TaskManagerSlotInformation taskManagerSlot, JobID jobId, String targetAddress, ResourceProfile resourceProfile) {
+		TaskExecutorConnection taskExecutorConnection = taskManagerSlot.getTaskManagerConnection();
+		TaskExecutorGateway gateway = taskExecutorConnection.getTaskExecutorGateway();
+
+		final CompletableFuture<Acknowledge> completableFuture = new CompletableFuture<>();
+		final SlotID slotId = taskManagerSlot.getSlotId();
+		final InstanceID instanceId = taskManagerSlot.getInstanceId();
+
+		LOG.debug("Starting allocation of slot {} for job {} with resource profile {}.", taskManagerSlot.getSlotId(), jobId, resourceProfile);
+
+		slotTracker.notifyAllocationStart(slotId, jobId);
+
+		if (!taskExecutorManager.isTaskManagerRegistered(instanceId)) {
+			throw new IllegalStateException("Could not find a registered task manager for instance id " +
+				instanceId + '.');
+		}
+
+		taskExecutorManager.markUsed(instanceId);
+
+		// RPC call to the task manager
+		CompletableFuture<Acknowledge> requestFuture = gateway.requestSlot(
+			slotId,
+			jobId,
+			new AllocationID(),
+			resourceProfile,
+			targetAddress,
+			resourceManagerId,
+			taskManagerRequestTimeout);
+
+		requestFuture.whenComplete(
+			(Acknowledge acknowledge, Throwable throwable) -> {
+				if (acknowledge != null) {
+					completableFuture.complete(acknowledge);
+				} else {
+					completableFuture.completeExceptionally(throwable);
+				}
+			});
+
+		completableFuture.whenCompleteAsync(
+			(Acknowledge acknowledge, Throwable throwable) -> {
+				try {
+					if (acknowledge != null) {
+						LOG.trace("Completed allocation of slot {} for job {}.", slotId, jobId);
+						slotTracker.notifyAllocationComplete(slotId, jobId);
+					} else {
+						if (throwable instanceof SlotOccupiedException) {
+							SlotOccupiedException exception = (SlotOccupiedException) throwable;
+							LOG.debug("Tried allocating slot {} for job {}, but it was already allocated for job {}.", slotId, jobId, exception.getJobId());
+							// report as a slot status to force the state transition
+							// this could be a problem if we ever assume that the task executor always reports about all slots
+							slotTracker.notifySlotStatus(Collections.singleton(new SlotStatus(slotId, taskManagerSlot.getResourceProfile(), exception.getJobId(), exception.getAllocationId())));
+						} else {
+							if (throwable instanceof CancellationException) {
+								LOG.debug("Cancelled allocation of slot {} for job {}.", slotId, jobId, throwable);
+							} else {
+								LOG.warn("Slot allocation for slot {} for job {} failed.", slotId, jobId, throwable);
+								slotTracker.notifyFree(slotId);
+							}
+						}
+					}
+					checkResourceRequirements();
+				} catch (Exception e) {
+					LOG.error("Error while completing the slot allocation.", e);
+				}
+			},
+			mainThreadExecutor);
+	}
+
+	private void tryFulfillRequirementsWithPendingOrNewSlots(JobID jobId, Map<ResourceProfile, Integer> missingResources, ResourceCounter pendingSlots) {
+		for (Map.Entry<ResourceProfile, Integer> missingResource : missingResources.entrySet()) {
+			ResourceProfile profile = missingResource.getKey();
+			for (int i = 0; i < missingResource.getValue(); i++) {
+				if (!tryFulfillWithPendingSlots(profile, pendingSlots)) {
+					Optional<ResourceRequirement> newlyFulfillableRequirements = taskExecutorManager.allocateWorker(profile);
+					if (newlyFulfillableRequirements.isPresent()) {
+						ResourceRequirement newSlots = newlyFulfillableRequirements.get();
+						// reserve one of the new slots
+						if (newSlots.getNumberOfRequiredSlots() > 1) {
+							pendingSlots.incrementCount(newSlots.getResourceProfile(), newSlots.getNumberOfRequiredSlots() - 1);
+						}
+					} else {
+						LOG.warn("Could not fulfill resource requirements of job {}.", jobId);
+						resourceActions.notifyNotEnoughResourcesAvailable(jobId, resourceTracker.getAcquiredResources(jobId));
+						return;
+					}

Review comment:
       One could factor this part out into a separate method which might make it a bit less complex.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManagerTest.java
##########
@@ -0,0 +1,1092 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple6;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.concurrent.ManuallyTriggeredScheduledExecutor;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerId;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+import org.apache.flink.runtime.slots.ResourceRequirements;
+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.TestingTaskExecutorGateway;
+import org.apache.flink.runtime.taskexecutor.TestingTaskExecutorGatewayBuilder;
+import org.apache.flink.runtime.taskexecutor.exceptions.SlotAllocationException;
+import org.apache.flink.runtime.taskexecutor.exceptions.SlotOccupiedException;
+import org.apache.flink.runtime.testingUtils.TestingUtils;
+import org.apache.flink.util.FlinkException;
+import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.function.FunctionUtils;
+
+import akka.pattern.AskTimeoutException;
+import org.junit.Ignore;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.hamcrest.CoreMatchers.hasItem;
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.empty;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.hasSize;
+import static org.hamcrest.Matchers.is;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertSame;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Tests for the {@link DeclarativeSlotManager}.
+ */
+public class DeclarativeSlotManagerTest extends TestLogger {
+
+	private static final FlinkException TEST_EXCEPTION = new FlinkException("Test exception");
+
+	private static final WorkerResourceSpec WORKER_RESOURCE_SPEC = new WorkerResourceSpec.Builder()
+		.setCpuCores(100.0)
+		.setTaskHeapMemoryMB(10000)
+		.setTaskOffHeapMemoryMB(10000)
+		.setNetworkMemoryMB(10000)
+		.setManagedMemoryMB(10000)
+		.build();
+
+	/**
+	 * Tests that we can register task manager and their slots at the slot manager.
+	 */
+	@Test
+	public void testTaskManagerRegistration() throws Exception {
+		final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder().createTestingTaskExecutorGateway();
+		final ResourceID resourceId = ResourceID.generate();
+		final TaskExecutorConnection taskManagerConnection = new TaskExecutorConnection(resourceId, taskExecutorGateway);
+
+		final SlotID slotId1 = new SlotID(resourceId, 0);
+		final SlotID slotId2 = new SlotID(resourceId, 1);
+		final SlotReport slotReport = new SlotReport(Arrays.asList(createFreeSlotStatus(slotId1), createFreeSlotStatus(slotId2)));
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport);
+
+			assertThat("The number registered slots does not equal the expected number.", slotManager.getNumberRegisteredSlots(), is(2));
+
+			assertNotNull(slotTracker.getSlot(slotId1));
+			assertNotNull(slotTracker.getSlot(slotId2));
+		}
+	}
+
+	/**
+	 * Tests that un-registration of task managers will free and remove all registered slots.
+	 */
+	@Test
+	public void testTaskManagerUnregistration() throws Exception {
+		final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder()
+			.setRequestSlotFunction(tuple6 -> new CompletableFuture<>())
+			.createTestingTaskExecutorGateway();
+		final TaskExecutorConnection taskManagerConnection = createTaskExecutorConnection(taskExecutorGateway);
+		final ResourceID resourceId = taskManagerConnection.getResourceID();
+
+		final SlotID slotId1 = new SlotID(resourceId, 0);
+		final SlotID slotId2 = new SlotID(resourceId, 1);
+		final SlotReport slotReport = new SlotReport(Arrays.asList(createAllocatedSlotStatus(slotId1), createFreeSlotStatus(slotId2)));
+
+		final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot();
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport);
+
+			assertEquals("The number registered slots does not equal the expected number.", 2, slotManager.getNumberRegisteredSlots());
+
+			DeclarativeTaskManagerSlot slot1 = slotTracker.getSlot(slotId1);
+			DeclarativeTaskManagerSlot slot2 = slotTracker.getSlot(slotId2);
+
+			assertSame(SlotState.ALLOCATED, slot1.getState());
+			assertSame(SlotState.FREE, slot2.getState());
+
+			slotManager.processResourceRequirements(resourceRequirements);
+
+			assertSame(SlotState.PENDING, slot2.getState());
+
+			slotManager.unregisterTaskManager(taskManagerConnection.getInstanceID(), TEST_EXCEPTION);
+
+			assertEquals(0, slotManager.getNumberRegisteredSlots());
+		}
+	}
+
+	/**
+	 * Tests that a slot request with no free slots will trigger the resource allocation.
+	 */
+	@Test
+	public void testSlotRequestWithoutFreeSlots() throws Exception {
+		final ResourceManagerId resourceManagerId = ResourceManagerId.generate();
+
+		final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot();
+
+		CompletableFuture<WorkerResourceSpec> allocateResourceFuture = new CompletableFuture<>();
+		ResourceActions resourceManagerActions = new TestingResourceActionsBuilder()
+			.setAllocateResourceConsumer(allocateResourceFuture::complete)
+			.build();
+
+		try (SlotManager slotManager = createSlotManager(resourceManagerId, resourceManagerActions)) {
+
+			slotManager.processResourceRequirements(resourceRequirements);
+
+			allocateResourceFuture.get();

Review comment:
       Maybe check that the right sized resource is requested.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] tillrohrmann commented on a change in pull request #13553: [FLINK-19306][coordination] Add DeclarativeSlotManager

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #13553:
URL: https://github.com/apache/flink/pull/13553#discussion_r502528012



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManager.java
##########
@@ -0,0 +1,609 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.metrics.MetricNames;
+import org.apache.flink.runtime.metrics.groups.SlotManagerMetricGroup;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerId;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.slots.ResourceCounter;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+import org.apache.flink.runtime.slots.ResourceRequirements;
+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.SlotOccupiedException;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ScheduledFuture;
+import java.util.function.BiFunction;
+import java.util.stream.Collectors;
+
+/**
+ * Implementation of {@link SlotManager} supporting declarative slot management.
+ */
+public class DeclarativeSlotManager implements SlotManager {
+	private static final Logger LOG = LoggerFactory.getLogger(DeclarativeSlotManager.class);
+
+	private final SlotTracker slotTracker;
+	private final ResourceTracker resourceTracker;
+	private final BiFunction<Executor, ResourceActions, TaskExecutorManager> taskExecutorManagerFactory;
+	private TaskExecutorManager taskExecutorManager;
+
+	/** Timeout for slot requests to the task manager. */
+	private final Time taskManagerRequestTimeout;
+	private ScheduledFuture<?> slotRequestTimeoutCheck;
+
+	private final SlotMatchingStrategy slotMatchingStrategy;
+
+	private final SlotManagerMetricGroup slotManagerMetricGroup;
+
+	private final Map<JobID, String> jobMasterTargetAddresses = new HashMap<>();
+	private final HashMap<SlotID, CompletableFuture<Acknowledge>> pendingSlotAllocationFutures;
+
+	/** ResourceManager's id. */
+	private ResourceManagerId resourceManagerId;
+
+	/** Executor for future callbacks which have to be "synchronized". */
+	private Executor mainThreadExecutor;
+
+	/** Callbacks for resource (de-)allocations. */
+	private ResourceActions resourceActions;
+
+	/** True iff the component has been started. */
+	private boolean started;
+
+	public DeclarativeSlotManager(
+			ScheduledExecutor scheduledExecutor,
+			SlotManagerConfiguration slotManagerConfiguration,
+			SlotManagerMetricGroup slotManagerMetricGroup,
+			ResourceTracker resourceTracker,
+			SlotTracker slotTracker) {
+
+		Preconditions.checkNotNull(slotManagerConfiguration);
+		this.taskManagerRequestTimeout = slotManagerConfiguration.getTaskManagerRequestTimeout();
+		this.slotManagerMetricGroup = Preconditions.checkNotNull(slotManagerMetricGroup);
+		this.resourceTracker = Preconditions.checkNotNull(resourceTracker);
+
+		pendingSlotAllocationFutures = new HashMap<>(16);
+
+		this.slotTracker = Preconditions.checkNotNull(slotTracker);
+		slotTracker.registerSlotStatusUpdateListener(createSlotStatusUpdateListener());
+
+		slotMatchingStrategy = slotManagerConfiguration.getSlotMatchingStrategy();
+
+		taskExecutorManagerFactory = (executor, resourceActions) -> new TaskExecutorManager(
+			slotManagerConfiguration.getDefaultWorkerResourceSpec(),
+			slotManagerConfiguration.getNumSlotsPerWorker(),
+			slotManagerConfiguration.getMaxSlotNum(),
+			slotManagerConfiguration.isWaitResultConsumedBeforeRelease(),
+			slotManagerConfiguration.getRedundantTaskManagerNum(),
+			slotManagerConfiguration.getTaskManagerTimeout(),
+			scheduledExecutor,
+			executor,
+			resourceActions);
+
+		resourceManagerId = null;
+		resourceActions = null;
+		mainThreadExecutor = null;
+		slotRequestTimeoutCheck = null;
+		taskExecutorManager = null;
+
+		started = false;
+	}
+
+	private SlotStatusUpdateListener createSlotStatusUpdateListener() {
+		return (taskManagerSlot, previous, current, jobId) -> {
+			if (previous == SlotState.PENDING) {
+				cancelAllocationFuture(taskManagerSlot.getSlotId());
+			}
+
+			if (current == SlotState.PENDING) {
+				resourceTracker.notifyAcquiredResource(jobId, taskManagerSlot.getResourceProfile());
+			}
+			if (current == SlotState.FREE) {
+				resourceTracker.notifyLostResource(jobId, taskManagerSlot.getResourceProfile());
+			}
+
+			if (current == SlotState.ALLOCATED) {
+				taskExecutorManager.occupySlot(taskManagerSlot.getInstanceId());
+			}
+			if (previous == SlotState.ALLOCATED && current == SlotState.FREE) {
+				taskExecutorManager.freeSlot(taskManagerSlot.getInstanceId());
+			}
+		};
+	}
+
+	private void cancelAllocationFuture(SlotID slotId) {
+		final CompletableFuture<Acknowledge> acknowledgeCompletableFuture = pendingSlotAllocationFutures.remove(slotId);
+		// the future may be null if we are just re-playing the state transitions due to a slot report
+		if (acknowledgeCompletableFuture != null) {
+			acknowledgeCompletableFuture.cancel(false);
+		}
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Component lifecycle methods
+	// ---------------------------------------------------------------------------------------------
+
+	/**
+	 * Starts the slot manager with the given leader id and resource manager actions.
+	 *
+	 * @param newResourceManagerId to use for communication with the task managers
+	 * @param newMainThreadExecutor to use to run code in the ResourceManager's main thread
+	 * @param newResourceActions to use for resource (de-)allocations
+	 */
+	@Override
+	public void start(ResourceManagerId newResourceManagerId, Executor newMainThreadExecutor, ResourceActions newResourceActions) {
+		LOG.info("Starting the slot manager.");
+
+		this.resourceManagerId = Preconditions.checkNotNull(newResourceManagerId);
+		mainThreadExecutor = Preconditions.checkNotNull(newMainThreadExecutor);
+		resourceActions = Preconditions.checkNotNull(newResourceActions);
+		taskExecutorManager = taskExecutorManagerFactory.apply(newMainThreadExecutor, newResourceActions);
+
+		started = true;
+
+		registerSlotManagerMetrics();
+	}
+
+	private void registerSlotManagerMetrics() {
+		slotManagerMetricGroup.gauge(
+			MetricNames.TASK_SLOTS_AVAILABLE,
+			() -> (long) getNumberFreeSlots());
+		slotManagerMetricGroup.gauge(
+			MetricNames.TASK_SLOTS_TOTAL,
+			() -> (long) getNumberRegisteredSlots());
+	}
+
+	/**
+	 * Suspends the component. This clears the internal state of the slot manager.
+	 */
+	@Override
+	public void suspend() {
+		LOG.info("Suspending the slot manager.");
+
+		if (slotRequestTimeoutCheck != null) {
+			slotRequestTimeoutCheck.cancel(false);
+			slotRequestTimeoutCheck = null;
+		}
+
+		resourceTracker.clear();
+		taskExecutorManager.close();
+
+		for (InstanceID registeredTaskManager : taskExecutorManager.getTaskExecutors()) {
+			unregisterTaskManager(registeredTaskManager, new SlotManagerException("The slot manager is being suspended."));
+		}
+
+		taskExecutorManager = null;
+		resourceManagerId = null;
+		resourceActions = null;
+		started = false;
+	}
+
+	/**
+	 * Closes the slot manager.
+	 *
+	 * @throws Exception if the close operation fails
+	 */
+	@Override
+	public void close() throws Exception {
+		LOG.info("Closing the slot manager.");
+
+		suspend();
+		slotManagerMetricGroup.close();
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Public API
+	// ---------------------------------------------------------------------------------------------
+
+	@Override
+	public void processResourceRequirements(ResourceRequirements resourceRequirements) {
+		checkInit();
+		LOG.debug("Received resource requirements from job {}: {}", resourceRequirements.getJobId(), resourceRequirements.getResourceRequirements());
+
+		if (resourceRequirements.getResourceRequirements().isEmpty()) {
+			jobMasterTargetAddresses.remove(resourceRequirements.getJobId());
+		} else {
+			jobMasterTargetAddresses.put(resourceRequirements.getJobId(), resourceRequirements.getTargetAddress());
+		}
+		resourceTracker.notifyResourceRequirements(resourceRequirements.getJobId(), resourceRequirements.getResourceRequirements());
+		checkResourceRequirements();
+	}
+
+	/**
+	 * 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
+	 * @return True if the task manager has not been registered before and is registered successfully; otherwise false
+	 */
+	@Override
+	public boolean registerTaskManager(final TaskExecutorConnection taskExecutorConnection, SlotReport initialSlotReport) {
+		checkInit();
+		LOG.debug("Registering task executor {} under {} at the slot manager.", taskExecutorConnection.getResourceID(), taskExecutorConnection.getInstanceID());
+
+		// we identify task managers by their instance id
+		if (taskExecutorManager.isTaskManagerRegistered(taskExecutorConnection.getInstanceID())) {
+			LOG.debug("Task executor {} was already registered.", taskExecutorConnection.getResourceID());
+			reportSlotStatus(taskExecutorConnection.getInstanceID(), initialSlotReport);
+			return false;
+		} else {
+			if (!taskExecutorManager.registerTaskManager(taskExecutorConnection, initialSlotReport)) {
+				LOG.debug("Task executor {} could not be registered.", taskExecutorConnection.getResourceID());
+				return false;
+			}
+
+			// register the new slots
+			for (SlotStatus slotStatus : initialSlotReport) {
+				slotTracker.addSlot(
+					slotStatus.getSlotID(),
+					slotStatus.getResourceProfile(),
+					taskExecutorConnection,
+					slotStatus.getJobID());
+			}
+
+			checkResourceRequirements();
+			return true;
+		}
+	}
+
+	@Override
+	public boolean unregisterTaskManager(InstanceID instanceId, Exception cause) {
+		checkInit();
+
+		LOG.debug("Unregistering task executor {} from the slot manager.", instanceId);
+
+		if (taskExecutorManager.isTaskManagerRegistered(instanceId)) {
+			slotTracker.removeSlots(taskExecutorManager.getSlotsOf(instanceId));
+			taskExecutorManager.unregisterTaskExecutor(instanceId);
+			checkResourceRequirements();
+
+			return true;
+		} else {
+			LOG.debug("There is no task executor registered with instance ID {}. Ignoring this message.", instanceId);
+
+			return false;
+		}
+	}
+
+	/**
+	 * Reports the current slot allocations for a task manager identified by the given instance id.
+	 *
+	 * @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
+	 */
+	@Override
+	public boolean reportSlotStatus(InstanceID instanceId, SlotReport slotReport) {
+		checkInit();
+
+		LOG.debug("Received slot report from instance {}: {}.", instanceId, slotReport);
+
+		if (taskExecutorManager.isTaskManagerRegistered(instanceId)) {
+			slotTracker.notifySlotStatus(slotReport);
+			checkResourceRequirements();
+			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
+	 */
+	@Override
+	public void freeSlot(SlotID slotId, AllocationID allocationId) {
+		checkInit();
+		LOG.debug("Freeing slot {}.", slotId);
+
+		slotTracker.notifyFree(slotId);
+		checkResourceRequirements();
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Requirement matching
+	// ---------------------------------------------------------------------------------------------
+
+	private void checkResourceRequirements() {
+		final Map<JobID, Collection<ResourceRequirement>> missingResources = resourceTracker.getMissingResources();
+		if (missingResources.isEmpty()) {
+			return;
+		}
+
+		final Map<JobID, ResourceCounter> outstandingRequirements = new LinkedHashMap<>();
+		for (Map.Entry<JobID, Collection<ResourceRequirement>> resourceRequirements : missingResources.entrySet()) {
+			JobID jobId = resourceRequirements.getKey();
+
+			for (ResourceRequirement resourceRequirement : resourceRequirements.getValue()) {
+				int numMissingSlots = internalRequestSlots(jobId, jobMasterTargetAddresses.get(jobId), resourceRequirement);
+				if (numMissingSlots > 0) {
+					outstandingRequirements
+						.computeIfAbsent(jobId, ignored -> new ResourceCounter())
+						.incrementCount(resourceRequirement.getResourceProfile(), numMissingSlots);
+				}
+			}
+		}
+
+		final ResourceCounter pendingSlots = new ResourceCounter(taskExecutorManager.getPendingTaskManagerSlots().stream().collect(
+			Collectors.groupingBy(
+				PendingTaskManagerSlot::getResourceProfile,
+				Collectors.summingInt(x -> 1))));
+
+		for (Map.Entry<JobID, ResourceCounter> unfulfilledRequirement : outstandingRequirements.entrySet()) {
+			tryFulfillRequirementsWithPendingOrNewSlots(
+				unfulfilledRequirement.getKey(),
+				unfulfilledRequirement.getValue().getResourceProfilesWithCount(),
+				pendingSlots);
+		}
+	}
+
+	/**
+	 * Tries to allocate slots for the given requirement. If there are not enough slots available, the
+	 * resource manager is informed to allocate more resources.
+	 *
+	 * @param jobId job to allocate slots for
+	 * @param targetAddress address of the jobmaster
+	 * @param resourceRequirement required slots
+	 * @return the number of missing slots
+	 */
+	private int internalRequestSlots(JobID jobId, String targetAddress, ResourceRequirement resourceRequirement) {
+		final ResourceProfile requiredResource = resourceRequirement.getResourceProfile();
+		Collection<TaskManagerSlotInformation> freeSlots = slotTracker.getFreeSlots();
+
+		int numUnfulfilled = 0;
+		for (int x = 0; x < resourceRequirement.getNumberOfRequiredSlots(); x++) {
+
+			final Optional<TaskManagerSlotInformation> reservedSlot = slotMatchingStrategy.findMatchingSlot(requiredResource, freeSlots, this::getNumberRegisteredSlotsOf);
+			if (reservedSlot.isPresent()) {
+				// we do not need to modify freeSlots because it is indirectly modified by the allocation
+				allocateSlot(reservedSlot.get(), jobId, targetAddress, requiredResource);
+			} else {
+				numUnfulfilled++;
+			}
+		}
+		return numUnfulfilled;
+	}
+
+	/**
+	 * Allocates the given slot. This entails sending a registration message to the task manager and treating failures.
+	 *
+	 * @param taskManagerSlot slot to allocate
+	 * @param jobId job for which the slot should be allocated for
+	 * @param targetAddress address of the job master
+	 * @param resourceProfile resource profile for the requirement for which the slot is used
+	 */
+	private void allocateSlot(TaskManagerSlotInformation taskManagerSlot, JobID jobId, String targetAddress, ResourceProfile resourceProfile) {
+		TaskExecutorConnection taskExecutorConnection = taskManagerSlot.getTaskManagerConnection();
+		TaskExecutorGateway gateway = taskExecutorConnection.getTaskExecutorGateway();
+
+		final CompletableFuture<Acknowledge> completableFuture = new CompletableFuture<>();
+		final SlotID slotId = taskManagerSlot.getSlotId();
+		final InstanceID instanceId = taskManagerSlot.getInstanceId();
+
+		LOG.debug("Starting allocation of slot {} for job {} with resource profile {}.", taskManagerSlot.getSlotId(), jobId, resourceProfile);
+
+		slotTracker.notifyAllocationStart(slotId, jobId);
+
+		if (!taskExecutorManager.isTaskManagerRegistered(instanceId)) {
+			throw new IllegalStateException("Could not find a registered task manager for instance id " +
+				instanceId + '.');
+		}
+
+		taskExecutorManager.markUsed(instanceId);
+
+		// RPC call to the task manager
+		CompletableFuture<Acknowledge> requestFuture = gateway.requestSlot(
+			slotId,
+			jobId,
+			new AllocationID(),
+			resourceProfile,
+			targetAddress,
+			resourceManagerId,
+			taskManagerRequestTimeout);
+
+		requestFuture.whenComplete(
+			(Acknowledge acknowledge, Throwable throwable) -> {
+				if (acknowledge != null) {
+					completableFuture.complete(acknowledge);
+				} else {
+					completableFuture.completeExceptionally(throwable);
+				}
+			});
+
+		completableFuture.whenCompleteAsync(
+			(Acknowledge acknowledge, Throwable throwable) -> {
+				try {
+					if (acknowledge != null) {
+						LOG.trace("Completed allocation of slot {} for job {}.", slotId, jobId);
+						slotTracker.notifyAllocationComplete(slotId, jobId);
+					} else {
+						if (throwable instanceof SlotOccupiedException) {
+							SlotOccupiedException exception = (SlotOccupiedException) throwable;
+							LOG.debug("Tried allocating slot {} for job {}, but it was already allocated for job {}.", slotId, jobId, exception.getJobId());
+							// report as a slot status to force the state transition
+							// this could be a problem if we ever assume that the task executor always reports about all slots
+							slotTracker.notifySlotStatus(Collections.singleton(new SlotStatus(slotId, taskManagerSlot.getResourceProfile(), exception.getJobId(), exception.getAllocationId())));
+						} else {
+							if (throwable instanceof CancellationException) {
+								LOG.debug("Cancelled allocation of slot {} for job {}.", slotId, jobId, throwable);
+							} else {
+								LOG.warn("Slot allocation for slot {} for job {} failed.", slotId, jobId, throwable);
+								slotTracker.notifyFree(slotId);
+							}
+						}
+					}
+					checkResourceRequirements();

Review comment:
       Then let's maybe not do it.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] tillrohrmann commented on a change in pull request #13553: [FLINK-19306][coordination] Add DeclarativeSlotManager

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #13553:
URL: https://github.com/apache/flink/pull/13553#discussion_r503766650



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java
##########
@@ -861,6 +884,8 @@ protected void closeJobManagerConnection(JobID jobId, Exception cause) {
 
 			jmResourceIdRegistrations.remove(jobManagerResourceId);
 
+			slotManager.processResourceRequirements(ResourceRequirements.empty(jobId, jobMasterGateway.getAddress()));

Review comment:
       I think I would be in favour of leaving it like it is because it gives a bit snappier behaviour wrt other jobs when a JM crashes.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] zentol commented on a change in pull request #13553: [FLINK-19306][coordination] Add DeclarativeSlotManager

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #13553:
URL: https://github.com/apache/flink/pull/13553#discussion_r502328815



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManager.java
##########
@@ -0,0 +1,609 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.metrics.MetricNames;
+import org.apache.flink.runtime.metrics.groups.SlotManagerMetricGroup;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerId;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.slots.ResourceCounter;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+import org.apache.flink.runtime.slots.ResourceRequirements;
+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.SlotOccupiedException;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ScheduledFuture;
+import java.util.function.BiFunction;
+import java.util.stream.Collectors;
+
+/**
+ * Implementation of {@link SlotManager} supporting declarative slot management.
+ */
+public class DeclarativeSlotManager implements SlotManager {
+	private static final Logger LOG = LoggerFactory.getLogger(DeclarativeSlotManager.class);
+
+	private final SlotTracker slotTracker;
+	private final ResourceTracker resourceTracker;
+	private final BiFunction<Executor, ResourceActions, TaskExecutorManager> taskExecutorManagerFactory;
+	private TaskExecutorManager taskExecutorManager;
+
+	/** Timeout for slot requests to the task manager. */
+	private final Time taskManagerRequestTimeout;
+	private ScheduledFuture<?> slotRequestTimeoutCheck;
+
+	private final SlotMatchingStrategy slotMatchingStrategy;
+
+	private final SlotManagerMetricGroup slotManagerMetricGroup;
+
+	private final Map<JobID, String> jobMasterTargetAddresses = new HashMap<>();
+	private final HashMap<SlotID, CompletableFuture<Acknowledge>> pendingSlotAllocationFutures;
+
+	/** ResourceManager's id. */
+	private ResourceManagerId resourceManagerId;
+
+	/** Executor for future callbacks which have to be "synchronized". */
+	private Executor mainThreadExecutor;
+
+	/** Callbacks for resource (de-)allocations. */
+	private ResourceActions resourceActions;
+
+	/** True iff the component has been started. */
+	private boolean started;
+
+	public DeclarativeSlotManager(
+			ScheduledExecutor scheduledExecutor,
+			SlotManagerConfiguration slotManagerConfiguration,
+			SlotManagerMetricGroup slotManagerMetricGroup,
+			ResourceTracker resourceTracker,
+			SlotTracker slotTracker) {
+
+		Preconditions.checkNotNull(slotManagerConfiguration);
+		this.taskManagerRequestTimeout = slotManagerConfiguration.getTaskManagerRequestTimeout();
+		this.slotManagerMetricGroup = Preconditions.checkNotNull(slotManagerMetricGroup);
+		this.resourceTracker = Preconditions.checkNotNull(resourceTracker);
+
+		pendingSlotAllocationFutures = new HashMap<>(16);
+
+		this.slotTracker = Preconditions.checkNotNull(slotTracker);
+		slotTracker.registerSlotStatusUpdateListener(createSlotStatusUpdateListener());
+
+		slotMatchingStrategy = slotManagerConfiguration.getSlotMatchingStrategy();
+
+		taskExecutorManagerFactory = (executor, resourceActions) -> new TaskExecutorManager(
+			slotManagerConfiguration.getDefaultWorkerResourceSpec(),
+			slotManagerConfiguration.getNumSlotsPerWorker(),
+			slotManagerConfiguration.getMaxSlotNum(),
+			slotManagerConfiguration.isWaitResultConsumedBeforeRelease(),
+			slotManagerConfiguration.getRedundantTaskManagerNum(),
+			slotManagerConfiguration.getTaskManagerTimeout(),
+			scheduledExecutor,
+			executor,
+			resourceActions);
+
+		resourceManagerId = null;
+		resourceActions = null;
+		mainThreadExecutor = null;
+		slotRequestTimeoutCheck = null;
+		taskExecutorManager = null;
+
+		started = false;
+	}
+
+	private SlotStatusUpdateListener createSlotStatusUpdateListener() {
+		return (taskManagerSlot, previous, current, jobId) -> {
+			if (previous == SlotState.PENDING) {
+				cancelAllocationFuture(taskManagerSlot.getSlotId());
+			}
+
+			if (current == SlotState.PENDING) {
+				resourceTracker.notifyAcquiredResource(jobId, taskManagerSlot.getResourceProfile());
+			}
+			if (current == SlotState.FREE) {
+				resourceTracker.notifyLostResource(jobId, taskManagerSlot.getResourceProfile());
+			}
+
+			if (current == SlotState.ALLOCATED) {
+				taskExecutorManager.occupySlot(taskManagerSlot.getInstanceId());
+			}
+			if (previous == SlotState.ALLOCATED && current == SlotState.FREE) {
+				taskExecutorManager.freeSlot(taskManagerSlot.getInstanceId());
+			}
+		};
+	}
+
+	private void cancelAllocationFuture(SlotID slotId) {
+		final CompletableFuture<Acknowledge> acknowledgeCompletableFuture = pendingSlotAllocationFutures.remove(slotId);

Review comment:
       It is _supposed_ to be populated in `allocateSlot`. It is mostly there since the current SM also does such book-keeping (just in the TaskManagerSlot), but it could very well be unnecessary at this point.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13553: [FLINK-19306][coordination] Add DeclarativeSlotManager

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13553:
URL: https://github.com/apache/flink/pull/13553#issuecomment-704936985






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] tillrohrmann commented on a change in pull request #13553: [FLINK-19306][coordination] Add DeclarativeSlotManager

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #13553:
URL: https://github.com/apache/flink/pull/13553#discussion_r502536675



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManager.java
##########
@@ -0,0 +1,609 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.metrics.MetricNames;
+import org.apache.flink.runtime.metrics.groups.SlotManagerMetricGroup;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerId;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.slots.ResourceCounter;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+import org.apache.flink.runtime.slots.ResourceRequirements;
+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.SlotOccupiedException;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ScheduledFuture;
+import java.util.function.BiFunction;
+import java.util.stream.Collectors;
+
+/**
+ * Implementation of {@link SlotManager} supporting declarative slot management.
+ */
+public class DeclarativeSlotManager implements SlotManager {
+	private static final Logger LOG = LoggerFactory.getLogger(DeclarativeSlotManager.class);
+
+	private final SlotTracker slotTracker;
+	private final ResourceTracker resourceTracker;
+	private final BiFunction<Executor, ResourceActions, TaskExecutorManager> taskExecutorManagerFactory;
+	private TaskExecutorManager taskExecutorManager;
+
+	/** Timeout for slot requests to the task manager. */
+	private final Time taskManagerRequestTimeout;
+	private ScheduledFuture<?> slotRequestTimeoutCheck;
+
+	private final SlotMatchingStrategy slotMatchingStrategy;
+
+	private final SlotManagerMetricGroup slotManagerMetricGroup;
+
+	private final Map<JobID, String> jobMasterTargetAddresses = new HashMap<>();
+	private final HashMap<SlotID, CompletableFuture<Acknowledge>> pendingSlotAllocationFutures;
+
+	/** ResourceManager's id. */
+	private ResourceManagerId resourceManagerId;
+
+	/** Executor for future callbacks which have to be "synchronized". */
+	private Executor mainThreadExecutor;
+
+	/** Callbacks for resource (de-)allocations. */
+	private ResourceActions resourceActions;
+
+	/** True iff the component has been started. */
+	private boolean started;
+
+	public DeclarativeSlotManager(
+			ScheduledExecutor scheduledExecutor,
+			SlotManagerConfiguration slotManagerConfiguration,
+			SlotManagerMetricGroup slotManagerMetricGroup,
+			ResourceTracker resourceTracker,
+			SlotTracker slotTracker) {
+
+		Preconditions.checkNotNull(slotManagerConfiguration);
+		this.taskManagerRequestTimeout = slotManagerConfiguration.getTaskManagerRequestTimeout();
+		this.slotManagerMetricGroup = Preconditions.checkNotNull(slotManagerMetricGroup);
+		this.resourceTracker = Preconditions.checkNotNull(resourceTracker);
+
+		pendingSlotAllocationFutures = new HashMap<>(16);
+
+		this.slotTracker = Preconditions.checkNotNull(slotTracker);
+		slotTracker.registerSlotStatusUpdateListener(createSlotStatusUpdateListener());
+
+		slotMatchingStrategy = slotManagerConfiguration.getSlotMatchingStrategy();
+
+		taskExecutorManagerFactory = (executor, resourceActions) -> new TaskExecutorManager(
+			slotManagerConfiguration.getDefaultWorkerResourceSpec(),
+			slotManagerConfiguration.getNumSlotsPerWorker(),
+			slotManagerConfiguration.getMaxSlotNum(),
+			slotManagerConfiguration.isWaitResultConsumedBeforeRelease(),
+			slotManagerConfiguration.getRedundantTaskManagerNum(),
+			slotManagerConfiguration.getTaskManagerTimeout(),
+			scheduledExecutor,
+			executor,
+			resourceActions);
+
+		resourceManagerId = null;
+		resourceActions = null;
+		mainThreadExecutor = null;
+		slotRequestTimeoutCheck = null;
+		taskExecutorManager = null;
+
+		started = false;
+	}
+
+	private SlotStatusUpdateListener createSlotStatusUpdateListener() {
+		return (taskManagerSlot, previous, current, jobId) -> {
+			if (previous == SlotState.PENDING) {
+				cancelAllocationFuture(taskManagerSlot.getSlotId());
+			}
+
+			if (current == SlotState.PENDING) {
+				resourceTracker.notifyAcquiredResource(jobId, taskManagerSlot.getResourceProfile());
+			}
+			if (current == SlotState.FREE) {
+				resourceTracker.notifyLostResource(jobId, taskManagerSlot.getResourceProfile());
+			}
+
+			if (current == SlotState.ALLOCATED) {
+				taskExecutorManager.occupySlot(taskManagerSlot.getInstanceId());
+			}
+			if (previous == SlotState.ALLOCATED && current == SlotState.FREE) {
+				taskExecutorManager.freeSlot(taskManagerSlot.getInstanceId());
+			}
+		};
+	}
+
+	private void cancelAllocationFuture(SlotID slotId) {
+		final CompletableFuture<Acknowledge> acknowledgeCompletableFuture = pendingSlotAllocationFutures.remove(slotId);

Review comment:
       Ah ok, I think I remember. The idea was to disable the callback on the response from the `TaskExecutor` when calling `requestSlot`.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13553: [FLINK-19306][coordination] Add DeclarativeSlotManager

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13553:
URL: https://github.com/apache/flink/pull/13553#issuecomment-704936985


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "15d4c353c4c0a646b2bdf34bbedbd81359cb5b5f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7264",
       "triggerID" : "15d4c353c4c0a646b2bdf34bbedbd81359cb5b5f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15d4c353c4c0a646b2bdf34bbedbd81359cb5b5f",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7283",
       "triggerID" : "705384887",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 15d4c353c4c0a646b2bdf34bbedbd81359cb5b5f Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7264) Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7283) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] zentol commented on a change in pull request #13553: [FLINK-19306][coordination] Add DeclarativeSlotManager

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #13553:
URL: https://github.com/apache/flink/pull/13553#discussion_r503150537



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java
##########
@@ -861,6 +884,8 @@ protected void closeJobManagerConnection(JobID jobId, Exception cause) {
 
 			jmResourceIdRegistrations.remove(jobManagerResourceId);
 
+			slotManager.processResourceRequirements(ResourceRequirements.empty(jobId, jobMasterGateway.getAddress()));

Review comment:
       Current PR state(`closeJobManagerConnection`):
   
   If the _job_ fails, then resources are currently cleaned up immediately, because the JM closes the RM connection, leading to `closeJobManagerConnection`. The JM is also supposed to reduce the requirements to 0.
   
   If the JM _crashes_, then we have to wait for the `heartbeat.timeout`.
   
   Cleanup in removeJob:
   
   Resources are always cleaned up after `resourcemanager.job.timeout`.
   
   
   Conceptually, it seems _correct_ to wait for the job timeout; it's just that for the non-HA case this should be set to 0?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] zentol merged pull request #13553: [FLINK-19306][coordination] Add DeclarativeSlotManager

Posted by GitBox <gi...@apache.org>.
zentol merged pull request #13553:
URL: https://github.com/apache/flink/pull/13553


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13553: [FLINK-19306][coordination] Add DeclarativeSlotManager

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13553:
URL: https://github.com/apache/flink/pull/13553#issuecomment-704936985


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "15d4c353c4c0a646b2bdf34bbedbd81359cb5b5f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7264",
       "triggerID" : "15d4c353c4c0a646b2bdf34bbedbd81359cb5b5f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15d4c353c4c0a646b2bdf34bbedbd81359cb5b5f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7283",
       "triggerID" : "705384887",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "15d4c353c4c0a646b2bdf34bbedbd81359cb5b5f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7264",
       "triggerID" : "705384887",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "484cbbdfcc7e371ca51de4d88bab086a299795a6",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7514",
       "triggerID" : "484cbbdfcc7e371ca51de4d88bab086a299795a6",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 15d4c353c4c0a646b2bdf34bbedbd81359cb5b5f Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7283) Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7264) 
   * 484cbbdfcc7e371ca51de4d88bab086a299795a6 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7514) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] zentol commented on a change in pull request #13553: [FLINK-19306][coordination] Add DeclarativeSlotManager

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #13553:
URL: https://github.com/apache/flink/pull/13553#discussion_r502364278



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java
##########
@@ -861,6 +884,8 @@ protected void closeJobManagerConnection(JobID jobId, Exception cause) {
 
 			jmResourceIdRegistrations.remove(jobManagerResourceId);
 
+			slotManager.processResourceRequirements(ResourceRequirements.empty(jobId, jobMasterGateway.getAddress()));

Review comment:
       hmm...i think the current behavior is fine for the case where a JM just crashes, but when we have HA _I think_ we are also clearing the requirements when the leader lost leadership.
   Soo...`removeJob` may be a better place to put it.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13553: [FLINK-19306][coordination] Add DeclarativeSlotManager

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13553:
URL: https://github.com/apache/flink/pull/13553#issuecomment-704936985


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "15d4c353c4c0a646b2bdf34bbedbd81359cb5b5f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7264",
       "triggerID" : "15d4c353c4c0a646b2bdf34bbedbd81359cb5b5f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15d4c353c4c0a646b2bdf34bbedbd81359cb5b5f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7283",
       "triggerID" : "705384887",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "15d4c353c4c0a646b2bdf34bbedbd81359cb5b5f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7264",
       "triggerID" : "705384887",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "484cbbdfcc7e371ca51de4d88bab086a299795a6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7514",
       "triggerID" : "484cbbdfcc7e371ca51de4d88bab086a299795a6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7a4c350c9514dc81429de2d9887837045b5894e0",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7527",
       "triggerID" : "7a4c350c9514dc81429de2d9887837045b5894e0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4b9288681085da65fc2c3f2742c898d13bf4af46",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "4b9288681085da65fc2c3f2742c898d13bf4af46",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7a4c350c9514dc81429de2d9887837045b5894e0 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7527) 
   * 4b9288681085da65fc2c3f2742c898d13bf4af46 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] zentol commented on a change in pull request #13553: [FLINK-19306][coordination] Add DeclarativeSlotManager

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #13553:
URL: https://github.com/apache/flink/pull/13553#discussion_r503331081



##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManagerTest.java
##########
@@ -0,0 +1,1092 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple6;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.concurrent.ManuallyTriggeredScheduledExecutor;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerId;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+import org.apache.flink.runtime.slots.ResourceRequirements;
+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.TestingTaskExecutorGateway;
+import org.apache.flink.runtime.taskexecutor.TestingTaskExecutorGatewayBuilder;
+import org.apache.flink.runtime.taskexecutor.exceptions.SlotAllocationException;
+import org.apache.flink.runtime.taskexecutor.exceptions.SlotOccupiedException;
+import org.apache.flink.runtime.testingUtils.TestingUtils;
+import org.apache.flink.util.FlinkException;
+import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.function.FunctionUtils;
+
+import akka.pattern.AskTimeoutException;
+import org.junit.Ignore;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.hamcrest.CoreMatchers.hasItem;
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.empty;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.hasSize;
+import static org.hamcrest.Matchers.is;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertSame;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Tests for the {@link DeclarativeSlotManager}.
+ */
+public class DeclarativeSlotManagerTest extends TestLogger {
+
+	private static final FlinkException TEST_EXCEPTION = new FlinkException("Test exception");
+
+	private static final WorkerResourceSpec WORKER_RESOURCE_SPEC = new WorkerResourceSpec.Builder()
+		.setCpuCores(100.0)
+		.setTaskHeapMemoryMB(10000)
+		.setTaskOffHeapMemoryMB(10000)
+		.setNetworkMemoryMB(10000)
+		.setManagedMemoryMB(10000)
+		.build();
+
+	/**
+	 * Tests that we can register task manager and their slots at the slot manager.
+	 */
+	@Test
+	public void testTaskManagerRegistration() throws Exception {
+		final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder().createTestingTaskExecutorGateway();
+		final ResourceID resourceId = ResourceID.generate();
+		final TaskExecutorConnection taskManagerConnection = new TaskExecutorConnection(resourceId, taskExecutorGateway);
+
+		final SlotID slotId1 = new SlotID(resourceId, 0);
+		final SlotID slotId2 = new SlotID(resourceId, 1);
+		final SlotReport slotReport = new SlotReport(Arrays.asList(createFreeSlotStatus(slotId1), createFreeSlotStatus(slotId2)));
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport);
+
+			assertThat("The number registered slots does not equal the expected number.", slotManager.getNumberRegisteredSlots(), is(2));
+
+			assertNotNull(slotTracker.getSlot(slotId1));
+			assertNotNull(slotTracker.getSlot(slotId2));
+		}
+	}
+
+	/**
+	 * Tests that un-registration of task managers will free and remove all registered slots.
+	 */
+	@Test
+	public void testTaskManagerUnregistration() throws Exception {
+		final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder()
+			.setRequestSlotFunction(tuple6 -> new CompletableFuture<>())
+			.createTestingTaskExecutorGateway();
+		final TaskExecutorConnection taskManagerConnection = createTaskExecutorConnection(taskExecutorGateway);
+		final ResourceID resourceId = taskManagerConnection.getResourceID();
+
+		final SlotID slotId1 = new SlotID(resourceId, 0);
+		final SlotID slotId2 = new SlotID(resourceId, 1);
+		final SlotReport slotReport = new SlotReport(Arrays.asList(createAllocatedSlotStatus(slotId1), createFreeSlotStatus(slotId2)));
+
+		final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot();
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport);
+
+			assertEquals("The number registered slots does not equal the expected number.", 2, slotManager.getNumberRegisteredSlots());
+
+			DeclarativeTaskManagerSlot slot1 = slotTracker.getSlot(slotId1);
+			DeclarativeTaskManagerSlot slot2 = slotTracker.getSlot(slotId2);
+
+			assertSame(SlotState.ALLOCATED, slot1.getState());
+			assertSame(SlotState.FREE, slot2.getState());
+
+			slotManager.processResourceRequirements(resourceRequirements);
+
+			assertSame(SlotState.PENDING, slot2.getState());
+
+			slotManager.unregisterTaskManager(taskManagerConnection.getInstanceID(), TEST_EXCEPTION);
+
+			assertEquals(0, slotManager.getNumberRegisteredSlots());
+		}
+	}
+
+	/**
+	 * Tests that a slot request with no free slots will trigger the resource allocation.
+	 */
+	@Test
+	public void testSlotRequestWithoutFreeSlots() throws Exception {
+		final ResourceManagerId resourceManagerId = ResourceManagerId.generate();
+
+		final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot();
+
+		CompletableFuture<WorkerResourceSpec> allocateResourceFuture = new CompletableFuture<>();
+		ResourceActions resourceManagerActions = new TestingResourceActionsBuilder()
+			.setAllocateResourceConsumer(allocateResourceFuture::complete)
+			.build();
+
+		try (SlotManager slotManager = createSlotManager(resourceManagerId, resourceManagerActions)) {
+
+			slotManager.processResourceRequirements(resourceRequirements);
+
+			allocateResourceFuture.get();
+		}
+	}
+
+	/**
+	 * Tests that resources continue to be considered missing if we cannot allocate more resources.
+	 */
+	@Test
+	public void testResourceDeclarationWithResourceAllocationFailure() throws Exception {
+		final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot();
+
+		ResourceActions resourceManagerActions = new TestingResourceActionsBuilder()
+			.setAllocateResourceFunction(value -> false)
+			.build();
+
+		final ResourceTracker resourceTracker = new DefaultResourceTracker();
+
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setResourceTracker(resourceTracker)
+			.buildAndStartWithDirectExec(ResourceManagerId.generate(), resourceManagerActions)) {
+
+			slotManager.processResourceRequirements(resourceRequirements);
+
+			final JobID jobId = resourceRequirements.getJobId();
+			assertThat(getTotalResourceCount(resourceTracker.getMissingResources().get(jobId)), is(1));
+		}
+	}
+
+	/**
+	 * Tests that a slot request which can be fulfilled will trigger a slot allocation.
+	 */
+	@Test
+	public void testSlotRequestWithFreeSlot() throws Exception {
+		final ResourceManagerId resourceManagerId = ResourceManagerId.generate();
+		final ResourceID resourceID = ResourceID.generate();
+		final JobID jobId = new JobID();
+		final SlotID slotId = new SlotID(resourceID, 0);
+		final String targetAddress = "localhost";
+		final ResourceProfile resourceProfile = ResourceProfile.fromResources(42.0, 1337);
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec(resourceManagerId, new TestingResourceActionsBuilder().build())) {
+
+			final CompletableFuture<Tuple6<SlotID, JobID, AllocationID, ResourceProfile, String, ResourceManagerId>> requestFuture = new CompletableFuture<>();
+			// accept an incoming slot request
+			final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder()
+				.setRequestSlotFunction(tuple6 -> {
+					requestFuture.complete(Tuple6.of(tuple6.f0, tuple6.f1, tuple6.f2, tuple6.f3, tuple6.f4, tuple6.f5));
+					return CompletableFuture.completedFuture(Acknowledge.get());
+				})
+				.createTestingTaskExecutorGateway();
+
+			final TaskExecutorConnection taskExecutorConnection = new TaskExecutorConnection(resourceID, taskExecutorGateway);
+
+			final SlotStatus slotStatus = new SlotStatus(slotId, resourceProfile);
+			final SlotReport slotReport = new SlotReport(slotStatus);
+
+			slotManager.registerTaskManager(
+				taskExecutorConnection,
+				slotReport);
+
+			ResourceRequirements requirements = ResourceRequirements.create(
+				jobId,
+				targetAddress,
+				Collections.singleton(ResourceRequirement.create(resourceProfile, 1)));
+			slotManager.processResourceRequirements(requirements);
+
+			assertThat(requestFuture.get(), is(equalTo(Tuple6.of(slotId, jobId, requestFuture.get().f2, resourceProfile, targetAddress, resourceManagerId))));
+
+			DeclarativeTaskManagerSlot slot = slotTracker.getSlot(slotId);
+
+			assertEquals("The slot has not been allocated to the expected allocation id.", jobId, slot.getJobId());
+		}
+	}
+
+	/**
+	 * Checks that reducing resource requirements while a slot allocation is in-progress results in the slot being freed
+	 * one the allocation completes.
+	 */
+	@Test
+	@Ignore
+	public void testResourceRequirementReductionDuringAllocation() throws Exception {
+		final TaskExecutorConnection taskManagerConnection = createTaskExecutorConnection();
+		final ResourceID resourceID = taskManagerConnection.getResourceID();
+		final SlotID slotId = new SlotID(resourceID, 0);
+
+		final SlotReport slotReport = new SlotReport(createFreeSlotStatus(slotId));
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport);
+
+			DeclarativeTaskManagerSlot slot = slotTracker.getSlot(slotId);
+
+			final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot();
+			slotManager.processResourceRequirements(resourceRequirements);
+
+			assertSame(SlotState.PENDING, slot.getState());
+
+			slotManager.processResourceRequirements(ResourceRequirements.create(resourceRequirements.getJobId(), "foobar", Collections.emptyList()));
+
+			slot = slotTracker.getSlot(slotId);
+
+			assertSame(SlotState.FREE, slot.getState());
+		}
+	}
+
+	/**
+	 * Tests that pending slot requests are tried to be fulfilled upon new slot registrations.
+	 */
+	@Test
+	public void testFulfillingPendingSlotRequest() throws Exception {
+		final ResourceManagerId resourceManagerId = ResourceManagerId.generate();
+		final ResourceID resourceID = ResourceID.generate();
+		final JobID jobId = new JobID();
+		final SlotID slotId = new SlotID(resourceID, 0);
+		final String targetAddress = "localhost";
+		final ResourceProfile resourceProfile = ResourceProfile.fromResources(42.0, 1337);
+
+		final AtomicInteger numberAllocateResourceCalls = new AtomicInteger(0);
+		ResourceActions resourceManagerActions = new TestingResourceActionsBuilder()
+			.setAllocateResourceConsumer(ignored -> numberAllocateResourceCalls.incrementAndGet())
+			.build();
+
+		final CompletableFuture<Tuple6<SlotID, JobID, AllocationID, ResourceProfile, String, ResourceManagerId>> requestFuture = new CompletableFuture<>();
+		// accept an incoming slot request
+		final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder()
+			.setRequestSlotFunction(tuple6 -> {
+				requestFuture.complete(Tuple6.of(tuple6.f0, tuple6.f1, tuple6.f2, tuple6.f3, tuple6.f4, tuple6.f5));
+				return CompletableFuture.completedFuture(Acknowledge.get());
+			})
+			.createTestingTaskExecutorGateway();
+
+		final TaskExecutorConnection taskExecutorConnection = new TaskExecutorConnection(resourceID, taskExecutorGateway);
+
+		final SlotStatus slotStatus = new SlotStatus(slotId, resourceProfile);
+		final SlotReport slotReport = new SlotReport(slotStatus);
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec(resourceManagerId, resourceManagerActions)) {
+
+			final ResourceRequirements resourceRequirements = ResourceRequirements.create(
+				jobId,
+				targetAddress,
+				Collections.singleton(ResourceRequirement.create(resourceProfile, 1)));
+			slotManager.processResourceRequirements(resourceRequirements);
+
+			assertThat(numberAllocateResourceCalls.get(), is(1));
+
+			slotManager.registerTaskManager(
+				taskExecutorConnection,
+				slotReport);
+
+			assertThat(requestFuture.get(), is(equalTo(Tuple6.of(slotId, jobId, requestFuture.get().f2, resourceProfile, targetAddress, resourceManagerId))));
+
+			DeclarativeTaskManagerSlot slot = slotTracker.getSlot(slotId);
+
+			assertEquals("The slot has not been allocated to the expected job id.", jobId, slot.getJobId());
+		}
+	}
+
+	/**
+	 * Tests that freeing a slot will correctly reset the slot and mark it as a free slot.
+	 */
+	@Test
+	public void testFreeSlot() throws Exception {
+		final TaskExecutorConnection taskExecutorConnection = createTaskExecutorConnection();
+		final ResourceID resourceID = taskExecutorConnection.getResourceID();
+		final SlotID slotId = new SlotID(resourceID, 0);
+
+		final SlotReport slotReport = new SlotReport(createAllocatedSlotStatus(slotId));
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			slotManager.registerTaskManager(
+				taskExecutorConnection,
+				slotReport);
+
+			DeclarativeTaskManagerSlot slot = slotTracker.getSlot(slotId);
+
+			assertSame(SlotState.ALLOCATED, slot.getState());
+
+			slotManager.freeSlot(slotId, new AllocationID());
+
+			assertSame(SlotState.FREE, slot.getState());
+
+			assertEquals(1, slotManager.getNumberFreeSlots());
+		}
+	}
+
+	/**
+	 * Tests that duplicate resource requirement declaration do not result in additional slots being allocated after a
+	 * pending slot request has been fulfilled but not yet freed.
+	 */
+	@Test
+	public void testDuplicateResourceRequirementDeclarationAfterSuccessfulAllocation() throws Exception {
+		final ResourceManagerId resourceManagerId = ResourceManagerId.generate();
+		final AtomicInteger allocateResourceCalls = new AtomicInteger(0);
+		final ResourceActions resourceManagerActions = new TestingResourceActionsBuilder()
+			.setAllocateResourceConsumer(ignored -> allocateResourceCalls.incrementAndGet())
+			.build();
+		ResourceRequirements requirements = createResourceRequirementsForSingleSlot();
+
+		final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder().createTestingTaskExecutorGateway();
+
+		final ResourceID resourceID = ResourceID.generate();
+
+		final TaskExecutorConnection taskManagerConnection = new TaskExecutorConnection(resourceID, taskExecutorGateway);
+
+		final SlotID slotId = new SlotID(resourceID, 0);
+		final SlotReport slotReport = new SlotReport(createFreeSlotStatus(slotId));
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec(resourceManagerId, resourceManagerActions)) {
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport);
+
+			slotManager.processResourceRequirements(requirements);
+
+			DeclarativeTaskManagerSlot slot = slotTracker.getSlot(slotId);
+
+			assertThat(slot.getState(), is(SlotState.ALLOCATED));
+
+			slotManager.processResourceRequirements(requirements);
+		}
+
+		// check that we have only called the resource allocation only for the first slot request,
+		// since the second request is a duplicate
+		assertThat(allocateResourceCalls.get(), is(0));
+	}
+
+	/**
+	 * Tests that a slot allocated for one job can be allocated for another job after being freed.
+	 */
+	@Test
+	public void testSlotCanBeAllocatedForDifferentJobAfterFree() throws Exception {
+		final AllocationID allocationId = new AllocationID();
+		final ResourceRequirements resourceRequirements1 = createResourceRequirementsForSingleSlot();
+		final ResourceRequirements resourceRequirements2 = createResourceRequirementsForSingleSlot();
+
+		final TaskExecutorConnection taskManagerConnection = createTaskExecutorConnection();
+		final ResourceID resourceID = taskManagerConnection.getResourceID();
+
+		final SlotID slotId = new SlotID(resourceID, 0);
+		final SlotReport slotReport = new SlotReport(createFreeSlotStatus(slotId));
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport);
+
+			slotManager.processResourceRequirements(resourceRequirements1);
+
+			DeclarativeTaskManagerSlot slot = slotTracker.getSlot(slotId);
+
+			assertEquals("The slot has not been allocated to the expected job id.", resourceRequirements1.getJobId(), slot.getJobId());
+
+			// clear resource requirements, so that the slot isn't immediately re-assigned
+			slotManager.processResourceRequirements(ResourceRequirements.create(resourceRequirements1.getJobId(), resourceRequirements1.getTargetAddress(), Collections.emptyList()));
+			slotManager.freeSlot(slotId, allocationId);
+
+			// check that the slot has been freed
+			assertSame(SlotState.FREE, slot.getState());
+
+			slotManager.processResourceRequirements(resourceRequirements2);
+
+			assertEquals("The slot has not been allocated to the expected job id.", resourceRequirements2.getJobId(), slot.getJobId());
+		}
+	}
+
+	/**
+	 * Tests that the slot manager ignores slot reports of unknown origin (not registered
+	 * task managers).
+	 */
+	@Test
+	public void testReceivingUnknownSlotReport() throws Exception {
+		final ResourceManagerId resourceManagerId = ResourceManagerId.generate();
+		final ResourceActions resourceManagerActions = new TestingResourceActionsBuilder().build();
+
+		final InstanceID unknownInstanceID = new InstanceID();
+		final SlotID unknownSlotId = new SlotID(ResourceID.generate(), 0);
+		final SlotReport unknownSlotReport = new SlotReport(createFreeSlotStatus(unknownSlotId));
+
+		try (SlotManager slotManager = createSlotManager(resourceManagerId, resourceManagerActions)) {
+			// check that we don't have any slots registered
+			assertThat(slotManager.getNumberRegisteredSlots(), is(0));
+
+			// this should not update anything since the instance id is not known to the slot manager
+			assertFalse(slotManager.reportSlotStatus(unknownInstanceID, unknownSlotReport));
+
+			assertThat(slotManager.getNumberRegisteredSlots(), is(0));
+		}
+	}
+
+	/**
+	 * Tests that slots are updated with respect to the latest incoming slot report. This means that
+	 * slots for which a report was received are updated accordingly.
+	 */
+	@Test
+	public void testUpdateSlotReport() throws Exception {
+		final TaskExecutorConnection taskManagerConnection = createTaskExecutorConnection();
+		final ResourceID resourceId = taskManagerConnection.getResourceID();
+
+		final SlotID slotId1 = new SlotID(resourceId, 0);
+		final SlotID slotId2 = new SlotID(resourceId, 1);
+
+		final SlotStatus slotStatus1 = createFreeSlotStatus(slotId1);
+		final SlotStatus slotStatus2 = createFreeSlotStatus(slotId2);
+
+		final SlotStatus newSlotStatus2 = createAllocatedSlotStatus(slotId2);
+		final JobID jobId = newSlotStatus2.getJobID();
+
+		final SlotReport slotReport1 = new SlotReport(Arrays.asList(slotStatus1, slotStatus2));
+		final SlotReport slotReport2 = new SlotReport(Arrays.asList(newSlotStatus2, slotStatus1));
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			// check that we don't have any slots registered
+			assertEquals(0, slotManager.getNumberRegisteredSlots());
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport1);
+
+			DeclarativeTaskManagerSlot slot1 = slotTracker.getSlot(slotId1);
+			DeclarativeTaskManagerSlot slot2 = slotTracker.getSlot(slotId2);
+
+			assertEquals(2, slotManager.getNumberRegisteredSlots());
+
+			assertSame(SlotState.FREE, slot1.getState());
+			assertSame(SlotState.FREE, slot2.getState());
+
+			assertTrue(slotManager.reportSlotStatus(taskManagerConnection.getInstanceID(), slotReport2));
+
+			assertEquals(2, slotManager.getNumberRegisteredSlots());
+
+			assertNotNull(slotTracker.getSlot(slotId1));
+			assertNotNull(slotTracker.getSlot(slotId2));
+
+			// slotId2 should have been allocated for jobiD
+			assertEquals(jobId, slotTracker.getSlot(slotId2).getJobId());

Review comment:
       correct




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13553: [FLINK-19306][coordination] Add DeclarativeSlotManager

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13553:
URL: https://github.com/apache/flink/pull/13553#issuecomment-704936985


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "15d4c353c4c0a646b2bdf34bbedbd81359cb5b5f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7264",
       "triggerID" : "15d4c353c4c0a646b2bdf34bbedbd81359cb5b5f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15d4c353c4c0a646b2bdf34bbedbd81359cb5b5f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7283",
       "triggerID" : "705384887",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "15d4c353c4c0a646b2bdf34bbedbd81359cb5b5f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7264",
       "triggerID" : "705384887",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "484cbbdfcc7e371ca51de4d88bab086a299795a6",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7514",
       "triggerID" : "484cbbdfcc7e371ca51de4d88bab086a299795a6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7a4c350c9514dc81429de2d9887837045b5894e0",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7a4c350c9514dc81429de2d9887837045b5894e0",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 15d4c353c4c0a646b2bdf34bbedbd81359cb5b5f Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7283) Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7264) 
   * 484cbbdfcc7e371ca51de4d88bab086a299795a6 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7514) 
   * 7a4c350c9514dc81429de2d9887837045b5894e0 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13553: [FLINK-19306][coordination] Add DeclarativeSlotManager

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13553:
URL: https://github.com/apache/flink/pull/13553#issuecomment-704936985


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "15d4c353c4c0a646b2bdf34bbedbd81359cb5b5f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7264",
       "triggerID" : "15d4c353c4c0a646b2bdf34bbedbd81359cb5b5f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15d4c353c4c0a646b2bdf34bbedbd81359cb5b5f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7283",
       "triggerID" : "705384887",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "15d4c353c4c0a646b2bdf34bbedbd81359cb5b5f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7264",
       "triggerID" : "705384887",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "484cbbdfcc7e371ca51de4d88bab086a299795a6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7514",
       "triggerID" : "484cbbdfcc7e371ca51de4d88bab086a299795a6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7a4c350c9514dc81429de2d9887837045b5894e0",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7527",
       "triggerID" : "7a4c350c9514dc81429de2d9887837045b5894e0",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7a4c350c9514dc81429de2d9887837045b5894e0 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7527) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13553: [FLINK-19306][coordination] Add DeclarativeSlotManager

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13553:
URL: https://github.com/apache/flink/pull/13553#issuecomment-704936985


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "15d4c353c4c0a646b2bdf34bbedbd81359cb5b5f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7264",
       "triggerID" : "15d4c353c4c0a646b2bdf34bbedbd81359cb5b5f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15d4c353c4c0a646b2bdf34bbedbd81359cb5b5f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7283",
       "triggerID" : "705384887",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "15d4c353c4c0a646b2bdf34bbedbd81359cb5b5f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7264",
       "triggerID" : "705384887",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "484cbbdfcc7e371ca51de4d88bab086a299795a6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7514",
       "triggerID" : "484cbbdfcc7e371ca51de4d88bab086a299795a6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7a4c350c9514dc81429de2d9887837045b5894e0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7527",
       "triggerID" : "7a4c350c9514dc81429de2d9887837045b5894e0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4b9288681085da65fc2c3f2742c898d13bf4af46",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7580",
       "triggerID" : "4b9288681085da65fc2c3f2742c898d13bf4af46",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f89b7098fb826ded9b17b0f2eb48c747451623d9",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7667",
       "triggerID" : "f89b7098fb826ded9b17b0f2eb48c747451623d9",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 4b9288681085da65fc2c3f2742c898d13bf4af46 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7580) 
   * f89b7098fb826ded9b17b0f2eb48c747451623d9 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7667) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] tillrohrmann commented on a change in pull request #13553: [FLINK-19306][coordination] Add DeclarativeSlotManager

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #13553:
URL: https://github.com/apache/flink/pull/13553#discussion_r501720427



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java
##########
@@ -861,6 +884,8 @@ protected void closeJobManagerConnection(JobID jobId, Exception cause) {
 
 			jmResourceIdRegistrations.remove(jobManagerResourceId);
 
+			slotManager.processResourceRequirements(ResourceRequirements.empty(jobId, jobMasterGateway.getAddress()));

Review comment:
       Are you sure that we want to decrease the requirements of a job if the `JobManager` connection is being closed? I was wondering whether this shouldn't happen if we remove the job in `ResourceManager.removeJob`?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManager.java
##########
@@ -0,0 +1,609 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.metrics.MetricNames;
+import org.apache.flink.runtime.metrics.groups.SlotManagerMetricGroup;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerId;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.slots.ResourceCounter;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+import org.apache.flink.runtime.slots.ResourceRequirements;
+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.SlotOccupiedException;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ScheduledFuture;
+import java.util.function.BiFunction;
+import java.util.stream.Collectors;
+
+/**
+ * Implementation of {@link SlotManager} supporting declarative slot management.
+ */
+public class DeclarativeSlotManager implements SlotManager {
+	private static final Logger LOG = LoggerFactory.getLogger(DeclarativeSlotManager.class);
+
+	private final SlotTracker slotTracker;
+	private final ResourceTracker resourceTracker;
+	private final BiFunction<Executor, ResourceActions, TaskExecutorManager> taskExecutorManagerFactory;
+	private TaskExecutorManager taskExecutorManager;
+
+	/** Timeout for slot requests to the task manager. */
+	private final Time taskManagerRequestTimeout;
+	private ScheduledFuture<?> slotRequestTimeoutCheck;
+
+	private final SlotMatchingStrategy slotMatchingStrategy;
+
+	private final SlotManagerMetricGroup slotManagerMetricGroup;
+
+	private final Map<JobID, String> jobMasterTargetAddresses = new HashMap<>();
+	private final HashMap<SlotID, CompletableFuture<Acknowledge>> pendingSlotAllocationFutures;
+
+	/** ResourceManager's id. */
+	private ResourceManagerId resourceManagerId;

Review comment:
       Since these fields can be `null`, we might wanna add `@Nullable`.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManager.java
##########
@@ -0,0 +1,609 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.metrics.MetricNames;
+import org.apache.flink.runtime.metrics.groups.SlotManagerMetricGroup;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerId;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.slots.ResourceCounter;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+import org.apache.flink.runtime.slots.ResourceRequirements;
+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.SlotOccupiedException;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ScheduledFuture;
+import java.util.function.BiFunction;
+import java.util.stream.Collectors;
+
+/**
+ * Implementation of {@link SlotManager} supporting declarative slot management.
+ */
+public class DeclarativeSlotManager implements SlotManager {
+	private static final Logger LOG = LoggerFactory.getLogger(DeclarativeSlotManager.class);
+
+	private final SlotTracker slotTracker;
+	private final ResourceTracker resourceTracker;
+	private final BiFunction<Executor, ResourceActions, TaskExecutorManager> taskExecutorManagerFactory;
+	private TaskExecutorManager taskExecutorManager;
+
+	/** Timeout for slot requests to the task manager. */
+	private final Time taskManagerRequestTimeout;
+	private ScheduledFuture<?> slotRequestTimeoutCheck;
+
+	private final SlotMatchingStrategy slotMatchingStrategy;
+
+	private final SlotManagerMetricGroup slotManagerMetricGroup;
+
+	private final Map<JobID, String> jobMasterTargetAddresses = new HashMap<>();
+	private final HashMap<SlotID, CompletableFuture<Acknowledge>> pendingSlotAllocationFutures;
+
+	/** ResourceManager's id. */
+	private ResourceManagerId resourceManagerId;
+
+	/** Executor for future callbacks which have to be "synchronized". */
+	private Executor mainThreadExecutor;
+
+	/** Callbacks for resource (de-)allocations. */
+	private ResourceActions resourceActions;
+
+	/** True iff the component has been started. */
+	private boolean started;
+
+	public DeclarativeSlotManager(
+			ScheduledExecutor scheduledExecutor,
+			SlotManagerConfiguration slotManagerConfiguration,
+			SlotManagerMetricGroup slotManagerMetricGroup,
+			ResourceTracker resourceTracker,
+			SlotTracker slotTracker) {
+
+		Preconditions.checkNotNull(slotManagerConfiguration);
+		this.taskManagerRequestTimeout = slotManagerConfiguration.getTaskManagerRequestTimeout();
+		this.slotManagerMetricGroup = Preconditions.checkNotNull(slotManagerMetricGroup);
+		this.resourceTracker = Preconditions.checkNotNull(resourceTracker);
+
+		pendingSlotAllocationFutures = new HashMap<>(16);
+
+		this.slotTracker = Preconditions.checkNotNull(slotTracker);
+		slotTracker.registerSlotStatusUpdateListener(createSlotStatusUpdateListener());
+
+		slotMatchingStrategy = slotManagerConfiguration.getSlotMatchingStrategy();
+
+		taskExecutorManagerFactory = (executor, resourceActions) -> new TaskExecutorManager(
+			slotManagerConfiguration.getDefaultWorkerResourceSpec(),
+			slotManagerConfiguration.getNumSlotsPerWorker(),
+			slotManagerConfiguration.getMaxSlotNum(),
+			slotManagerConfiguration.isWaitResultConsumedBeforeRelease(),
+			slotManagerConfiguration.getRedundantTaskManagerNum(),
+			slotManagerConfiguration.getTaskManagerTimeout(),
+			scheduledExecutor,
+			executor,
+			resourceActions);
+
+		resourceManagerId = null;
+		resourceActions = null;
+		mainThreadExecutor = null;
+		slotRequestTimeoutCheck = null;
+		taskExecutorManager = null;
+
+		started = false;
+	}
+
+	private SlotStatusUpdateListener createSlotStatusUpdateListener() {
+		return (taskManagerSlot, previous, current, jobId) -> {
+			if (previous == SlotState.PENDING) {
+				cancelAllocationFuture(taskManagerSlot.getSlotId());
+			}
+
+			if (current == SlotState.PENDING) {
+				resourceTracker.notifyAcquiredResource(jobId, taskManagerSlot.getResourceProfile());
+			}
+			if (current == SlotState.FREE) {
+				resourceTracker.notifyLostResource(jobId, taskManagerSlot.getResourceProfile());
+			}
+
+			if (current == SlotState.ALLOCATED) {
+				taskExecutorManager.occupySlot(taskManagerSlot.getInstanceId());
+			}
+			if (previous == SlotState.ALLOCATED && current == SlotState.FREE) {
+				taskExecutorManager.freeSlot(taskManagerSlot.getInstanceId());
+			}
+		};
+	}
+
+	private void cancelAllocationFuture(SlotID slotId) {
+		final CompletableFuture<Acknowledge> acknowledgeCompletableFuture = pendingSlotAllocationFutures.remove(slotId);
+		// the future may be null if we are just re-playing the state transitions due to a slot report
+		if (acknowledgeCompletableFuture != null) {
+			acknowledgeCompletableFuture.cancel(false);
+		}
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Component lifecycle methods
+	// ---------------------------------------------------------------------------------------------
+
+	/**
+	 * Starts the slot manager with the given leader id and resource manager actions.
+	 *
+	 * @param newResourceManagerId to use for communication with the task managers
+	 * @param newMainThreadExecutor to use to run code in the ResourceManager's main thread
+	 * @param newResourceActions to use for resource (de-)allocations
+	 */
+	@Override
+	public void start(ResourceManagerId newResourceManagerId, Executor newMainThreadExecutor, ResourceActions newResourceActions) {
+		LOG.info("Starting the slot manager.");
+
+		this.resourceManagerId = Preconditions.checkNotNull(newResourceManagerId);
+		mainThreadExecutor = Preconditions.checkNotNull(newMainThreadExecutor);
+		resourceActions = Preconditions.checkNotNull(newResourceActions);
+		taskExecutorManager = taskExecutorManagerFactory.apply(newMainThreadExecutor, newResourceActions);
+
+		started = true;
+
+		registerSlotManagerMetrics();
+	}
+
+	private void registerSlotManagerMetrics() {
+		slotManagerMetricGroup.gauge(
+			MetricNames.TASK_SLOTS_AVAILABLE,
+			() -> (long) getNumberFreeSlots());
+		slotManagerMetricGroup.gauge(
+			MetricNames.TASK_SLOTS_TOTAL,
+			() -> (long) getNumberRegisteredSlots());
+	}
+
+	/**
+	 * Suspends the component. This clears the internal state of the slot manager.
+	 */
+	@Override
+	public void suspend() {
+		LOG.info("Suspending the slot manager.");
+
+		if (slotRequestTimeoutCheck != null) {
+			slotRequestTimeoutCheck.cancel(false);
+			slotRequestTimeoutCheck = null;
+		}
+
+		resourceTracker.clear();
+		taskExecutorManager.close();
+
+		for (InstanceID registeredTaskManager : taskExecutorManager.getTaskExecutors()) {
+			unregisterTaskManager(registeredTaskManager, new SlotManagerException("The slot manager is being suspended."));
+		}
+
+		taskExecutorManager = null;
+		resourceManagerId = null;
+		resourceActions = null;
+		started = false;
+	}
+
+	/**
+	 * Closes the slot manager.
+	 *
+	 * @throws Exception if the close operation fails
+	 */
+	@Override
+	public void close() throws Exception {
+		LOG.info("Closing the slot manager.");
+
+		suspend();
+		slotManagerMetricGroup.close();
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Public API
+	// ---------------------------------------------------------------------------------------------
+
+	@Override
+	public void processResourceRequirements(ResourceRequirements resourceRequirements) {
+		checkInit();
+		LOG.debug("Received resource requirements from job {}: {}", resourceRequirements.getJobId(), resourceRequirements.getResourceRequirements());
+
+		if (resourceRequirements.getResourceRequirements().isEmpty()) {
+			jobMasterTargetAddresses.remove(resourceRequirements.getJobId());
+		} else {
+			jobMasterTargetAddresses.put(resourceRequirements.getJobId(), resourceRequirements.getTargetAddress());
+		}
+		resourceTracker.notifyResourceRequirements(resourceRequirements.getJobId(), resourceRequirements.getResourceRequirements());
+		checkResourceRequirements();
+	}
+
+	/**
+	 * 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
+	 * @return True if the task manager has not been registered before and is registered successfully; otherwise false
+	 */
+	@Override
+	public boolean registerTaskManager(final TaskExecutorConnection taskExecutorConnection, SlotReport initialSlotReport) {
+		checkInit();
+		LOG.debug("Registering task executor {} under {} at the slot manager.", taskExecutorConnection.getResourceID(), taskExecutorConnection.getInstanceID());
+
+		// we identify task managers by their instance id
+		if (taskExecutorManager.isTaskManagerRegistered(taskExecutorConnection.getInstanceID())) {
+			LOG.debug("Task executor {} was already registered.", taskExecutorConnection.getResourceID());
+			reportSlotStatus(taskExecutorConnection.getInstanceID(), initialSlotReport);
+			return false;
+		} else {
+			if (!taskExecutorManager.registerTaskManager(taskExecutorConnection, initialSlotReport)) {
+				LOG.debug("Task executor {} could not be registered.", taskExecutorConnection.getResourceID());
+				return false;
+			}
+
+			// register the new slots
+			for (SlotStatus slotStatus : initialSlotReport) {
+				slotTracker.addSlot(
+					slotStatus.getSlotID(),
+					slotStatus.getResourceProfile(),
+					taskExecutorConnection,
+					slotStatus.getJobID());
+			}
+
+			checkResourceRequirements();
+			return true;
+		}
+	}
+
+	@Override
+	public boolean unregisterTaskManager(InstanceID instanceId, Exception cause) {
+		checkInit();
+
+		LOG.debug("Unregistering task executor {} from the slot manager.", instanceId);
+
+		if (taskExecutorManager.isTaskManagerRegistered(instanceId)) {
+			slotTracker.removeSlots(taskExecutorManager.getSlotsOf(instanceId));
+			taskExecutorManager.unregisterTaskExecutor(instanceId);
+			checkResourceRequirements();
+
+			return true;
+		} else {
+			LOG.debug("There is no task executor registered with instance ID {}. Ignoring this message.", instanceId);
+
+			return false;
+		}
+	}
+
+	/**
+	 * Reports the current slot allocations for a task manager identified by the given instance id.
+	 *
+	 * @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
+	 */
+	@Override
+	public boolean reportSlotStatus(InstanceID instanceId, SlotReport slotReport) {
+		checkInit();
+
+		LOG.debug("Received slot report from instance {}: {}.", instanceId, slotReport);
+
+		if (taskExecutorManager.isTaskManagerRegistered(instanceId)) {
+			slotTracker.notifySlotStatus(slotReport);
+			checkResourceRequirements();
+			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
+	 */
+	@Override
+	public void freeSlot(SlotID slotId, AllocationID allocationId) {
+		checkInit();
+		LOG.debug("Freeing slot {}.", slotId);
+
+		slotTracker.notifyFree(slotId);
+		checkResourceRequirements();
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Requirement matching
+	// ---------------------------------------------------------------------------------------------
+
+	private void checkResourceRequirements() {
+		final Map<JobID, Collection<ResourceRequirement>> missingResources = resourceTracker.getMissingResources();
+		if (missingResources.isEmpty()) {
+			return;
+		}
+
+		final Map<JobID, ResourceCounter> outstandingRequirements = new LinkedHashMap<>();
+		for (Map.Entry<JobID, Collection<ResourceRequirement>> resourceRequirements : missingResources.entrySet()) {
+			JobID jobId = resourceRequirements.getKey();
+
+			for (ResourceRequirement resourceRequirement : resourceRequirements.getValue()) {
+				int numMissingSlots = internalRequestSlots(jobId, jobMasterTargetAddresses.get(jobId), resourceRequirement);
+				if (numMissingSlots > 0) {
+					outstandingRequirements
+						.computeIfAbsent(jobId, ignored -> new ResourceCounter())
+						.incrementCount(resourceRequirement.getResourceProfile(), numMissingSlots);
+				}
+			}
+		}
+
+		final ResourceCounter pendingSlots = new ResourceCounter(taskExecutorManager.getPendingTaskManagerSlots().stream().collect(
+			Collectors.groupingBy(
+				PendingTaskManagerSlot::getResourceProfile,
+				Collectors.summingInt(x -> 1))));
+
+		for (Map.Entry<JobID, ResourceCounter> unfulfilledRequirement : outstandingRequirements.entrySet()) {
+			tryFulfillRequirementsWithPendingOrNewSlots(
+				unfulfilledRequirement.getKey(),
+				unfulfilledRequirement.getValue().getResourceProfilesWithCount(),
+				pendingSlots);
+		}
+	}
+
+	/**
+	 * Tries to allocate slots for the given requirement. If there are not enough slots available, the
+	 * resource manager is informed to allocate more resources.
+	 *
+	 * @param jobId job to allocate slots for
+	 * @param targetAddress address of the jobmaster
+	 * @param resourceRequirement required slots
+	 * @return the number of missing slots
+	 */
+	private int internalRequestSlots(JobID jobId, String targetAddress, ResourceRequirement resourceRequirement) {
+		final ResourceProfile requiredResource = resourceRequirement.getResourceProfile();
+		Collection<TaskManagerSlotInformation> freeSlots = slotTracker.getFreeSlots();
+
+		int numUnfulfilled = 0;
+		for (int x = 0; x < resourceRequirement.getNumberOfRequiredSlots(); x++) {
+
+			final Optional<TaskManagerSlotInformation> reservedSlot = slotMatchingStrategy.findMatchingSlot(requiredResource, freeSlots, this::getNumberRegisteredSlotsOf);
+			if (reservedSlot.isPresent()) {
+				// we do not need to modify freeSlots because it is indirectly modified by the allocation
+				allocateSlot(reservedSlot.get(), jobId, targetAddress, requiredResource);
+			} else {
+				numUnfulfilled++;
+			}
+		}
+		return numUnfulfilled;
+	}
+
+	/**
+	 * Allocates the given slot. This entails sending a registration message to the task manager and treating failures.
+	 *
+	 * @param taskManagerSlot slot to allocate
+	 * @param jobId job for which the slot should be allocated for
+	 * @param targetAddress address of the job master
+	 * @param resourceProfile resource profile for the requirement for which the slot is used
+	 */
+	private void allocateSlot(TaskManagerSlotInformation taskManagerSlot, JobID jobId, String targetAddress, ResourceProfile resourceProfile) {
+		TaskExecutorConnection taskExecutorConnection = taskManagerSlot.getTaskManagerConnection();
+		TaskExecutorGateway gateway = taskExecutorConnection.getTaskExecutorGateway();
+
+		final CompletableFuture<Acknowledge> completableFuture = new CompletableFuture<>();
+		final SlotID slotId = taskManagerSlot.getSlotId();
+		final InstanceID instanceId = taskManagerSlot.getInstanceId();
+
+		LOG.debug("Starting allocation of slot {} for job {} with resource profile {}.", taskManagerSlot.getSlotId(), jobId, resourceProfile);
+
+		slotTracker.notifyAllocationStart(slotId, jobId);
+
+		if (!taskExecutorManager.isTaskManagerRegistered(instanceId)) {
+			throw new IllegalStateException("Could not find a registered task manager for instance id " +
+				instanceId + '.');
+		}
+
+		taskExecutorManager.markUsed(instanceId);
+
+		// RPC call to the task manager
+		CompletableFuture<Acknowledge> requestFuture = gateway.requestSlot(
+			slotId,
+			jobId,
+			new AllocationID(),
+			resourceProfile,
+			targetAddress,
+			resourceManagerId,
+			taskManagerRequestTimeout);
+
+		requestFuture.whenComplete(
+			(Acknowledge acknowledge, Throwable throwable) -> {
+				if (acknowledge != null) {
+					completableFuture.complete(acknowledge);
+				} else {
+					completableFuture.completeExceptionally(throwable);
+				}
+			});
+
+		completableFuture.whenCompleteAsync(
+			(Acknowledge acknowledge, Throwable throwable) -> {
+				try {
+					if (acknowledge != null) {
+						LOG.trace("Completed allocation of slot {} for job {}.", slotId, jobId);
+						slotTracker.notifyAllocationComplete(slotId, jobId);
+					} else {
+						if (throwable instanceof SlotOccupiedException) {
+							SlotOccupiedException exception = (SlotOccupiedException) throwable;
+							LOG.debug("Tried allocating slot {} for job {}, but it was already allocated for job {}.", slotId, jobId, exception.getJobId());
+							// report as a slot status to force the state transition
+							// this could be a problem if we ever assume that the task executor always reports about all slots
+							slotTracker.notifySlotStatus(Collections.singleton(new SlotStatus(slotId, taskManagerSlot.getResourceProfile(), exception.getJobId(), exception.getAllocationId())));
+						} else {
+							if (throwable instanceof CancellationException) {
+								LOG.debug("Cancelled allocation of slot {} for job {}.", slotId, jobId, throwable);
+							} else {
+								LOG.warn("Slot allocation for slot {} for job {} failed.", slotId, jobId, throwable);
+								slotTracker.notifyFree(slotId);
+							}
+						}
+					}
+					checkResourceRequirements();

Review comment:
       Should we also call `checkResourceRequirements` in case of a successful allocation? Shouldn't we only call this method if the allocation failed for some reason and we need to find another slot?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManager.java
##########
@@ -0,0 +1,609 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.metrics.MetricNames;
+import org.apache.flink.runtime.metrics.groups.SlotManagerMetricGroup;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerId;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.slots.ResourceCounter;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+import org.apache.flink.runtime.slots.ResourceRequirements;
+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.SlotOccupiedException;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ScheduledFuture;
+import java.util.function.BiFunction;
+import java.util.stream.Collectors;
+
+/**
+ * Implementation of {@link SlotManager} supporting declarative slot management.
+ */
+public class DeclarativeSlotManager implements SlotManager {
+	private static final Logger LOG = LoggerFactory.getLogger(DeclarativeSlotManager.class);
+
+	private final SlotTracker slotTracker;
+	private final ResourceTracker resourceTracker;
+	private final BiFunction<Executor, ResourceActions, TaskExecutorManager> taskExecutorManagerFactory;
+	private TaskExecutorManager taskExecutorManager;
+
+	/** Timeout for slot requests to the task manager. */
+	private final Time taskManagerRequestTimeout;
+	private ScheduledFuture<?> slotRequestTimeoutCheck;
+
+	private final SlotMatchingStrategy slotMatchingStrategy;
+
+	private final SlotManagerMetricGroup slotManagerMetricGroup;
+
+	private final Map<JobID, String> jobMasterTargetAddresses = new HashMap<>();
+	private final HashMap<SlotID, CompletableFuture<Acknowledge>> pendingSlotAllocationFutures;
+
+	/** ResourceManager's id. */
+	private ResourceManagerId resourceManagerId;
+
+	/** Executor for future callbacks which have to be "synchronized". */
+	private Executor mainThreadExecutor;
+
+	/** Callbacks for resource (de-)allocations. */
+	private ResourceActions resourceActions;
+
+	/** True iff the component has been started. */
+	private boolean started;
+
+	public DeclarativeSlotManager(
+			ScheduledExecutor scheduledExecutor,
+			SlotManagerConfiguration slotManagerConfiguration,
+			SlotManagerMetricGroup slotManagerMetricGroup,
+			ResourceTracker resourceTracker,
+			SlotTracker slotTracker) {
+
+		Preconditions.checkNotNull(slotManagerConfiguration);
+		this.taskManagerRequestTimeout = slotManagerConfiguration.getTaskManagerRequestTimeout();
+		this.slotManagerMetricGroup = Preconditions.checkNotNull(slotManagerMetricGroup);
+		this.resourceTracker = Preconditions.checkNotNull(resourceTracker);
+
+		pendingSlotAllocationFutures = new HashMap<>(16);
+
+		this.slotTracker = Preconditions.checkNotNull(slotTracker);
+		slotTracker.registerSlotStatusUpdateListener(createSlotStatusUpdateListener());
+
+		slotMatchingStrategy = slotManagerConfiguration.getSlotMatchingStrategy();
+
+		taskExecutorManagerFactory = (executor, resourceActions) -> new TaskExecutorManager(
+			slotManagerConfiguration.getDefaultWorkerResourceSpec(),
+			slotManagerConfiguration.getNumSlotsPerWorker(),
+			slotManagerConfiguration.getMaxSlotNum(),
+			slotManagerConfiguration.isWaitResultConsumedBeforeRelease(),
+			slotManagerConfiguration.getRedundantTaskManagerNum(),
+			slotManagerConfiguration.getTaskManagerTimeout(),
+			scheduledExecutor,
+			executor,
+			resourceActions);
+
+		resourceManagerId = null;
+		resourceActions = null;
+		mainThreadExecutor = null;
+		slotRequestTimeoutCheck = null;
+		taskExecutorManager = null;
+
+		started = false;
+	}
+
+	private SlotStatusUpdateListener createSlotStatusUpdateListener() {
+		return (taskManagerSlot, previous, current, jobId) -> {
+			if (previous == SlotState.PENDING) {
+				cancelAllocationFuture(taskManagerSlot.getSlotId());
+			}
+
+			if (current == SlotState.PENDING) {
+				resourceTracker.notifyAcquiredResource(jobId, taskManagerSlot.getResourceProfile());
+			}
+			if (current == SlotState.FREE) {
+				resourceTracker.notifyLostResource(jobId, taskManagerSlot.getResourceProfile());
+			}
+
+			if (current == SlotState.ALLOCATED) {
+				taskExecutorManager.occupySlot(taskManagerSlot.getInstanceId());
+			}
+			if (previous == SlotState.ALLOCATED && current == SlotState.FREE) {
+				taskExecutorManager.freeSlot(taskManagerSlot.getInstanceId());
+			}
+		};
+	}
+
+	private void cancelAllocationFuture(SlotID slotId) {
+		final CompletableFuture<Acknowledge> acknowledgeCompletableFuture = pendingSlotAllocationFutures.remove(slotId);
+		// the future may be null if we are just re-playing the state transitions due to a slot report
+		if (acknowledgeCompletableFuture != null) {
+			acknowledgeCompletableFuture.cancel(false);
+		}
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Component lifecycle methods
+	// ---------------------------------------------------------------------------------------------
+
+	/**
+	 * Starts the slot manager with the given leader id and resource manager actions.
+	 *
+	 * @param newResourceManagerId to use for communication with the task managers
+	 * @param newMainThreadExecutor to use to run code in the ResourceManager's main thread
+	 * @param newResourceActions to use for resource (de-)allocations
+	 */
+	@Override
+	public void start(ResourceManagerId newResourceManagerId, Executor newMainThreadExecutor, ResourceActions newResourceActions) {
+		LOG.info("Starting the slot manager.");
+
+		this.resourceManagerId = Preconditions.checkNotNull(newResourceManagerId);
+		mainThreadExecutor = Preconditions.checkNotNull(newMainThreadExecutor);
+		resourceActions = Preconditions.checkNotNull(newResourceActions);
+		taskExecutorManager = taskExecutorManagerFactory.apply(newMainThreadExecutor, newResourceActions);
+
+		started = true;
+
+		registerSlotManagerMetrics();
+	}
+
+	private void registerSlotManagerMetrics() {
+		slotManagerMetricGroup.gauge(
+			MetricNames.TASK_SLOTS_AVAILABLE,
+			() -> (long) getNumberFreeSlots());
+		slotManagerMetricGroup.gauge(
+			MetricNames.TASK_SLOTS_TOTAL,
+			() -> (long) getNumberRegisteredSlots());
+	}
+
+	/**
+	 * Suspends the component. This clears the internal state of the slot manager.
+	 */
+	@Override
+	public void suspend() {
+		LOG.info("Suspending the slot manager.");
+
+		if (slotRequestTimeoutCheck != null) {
+			slotRequestTimeoutCheck.cancel(false);
+			slotRequestTimeoutCheck = null;
+		}
+
+		resourceTracker.clear();
+		taskExecutorManager.close();
+
+		for (InstanceID registeredTaskManager : taskExecutorManager.getTaskExecutors()) {
+			unregisterTaskManager(registeredTaskManager, new SlotManagerException("The slot manager is being suspended."));
+		}
+
+		taskExecutorManager = null;
+		resourceManagerId = null;
+		resourceActions = null;
+		started = false;
+	}
+
+	/**
+	 * Closes the slot manager.
+	 *
+	 * @throws Exception if the close operation fails
+	 */
+	@Override
+	public void close() throws Exception {
+		LOG.info("Closing the slot manager.");
+
+		suspend();
+		slotManagerMetricGroup.close();
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Public API
+	// ---------------------------------------------------------------------------------------------
+
+	@Override
+	public void processResourceRequirements(ResourceRequirements resourceRequirements) {
+		checkInit();
+		LOG.debug("Received resource requirements from job {}: {}", resourceRequirements.getJobId(), resourceRequirements.getResourceRequirements());
+
+		if (resourceRequirements.getResourceRequirements().isEmpty()) {
+			jobMasterTargetAddresses.remove(resourceRequirements.getJobId());
+		} else {
+			jobMasterTargetAddresses.put(resourceRequirements.getJobId(), resourceRequirements.getTargetAddress());
+		}
+		resourceTracker.notifyResourceRequirements(resourceRequirements.getJobId(), resourceRequirements.getResourceRequirements());
+		checkResourceRequirements();
+	}
+
+	/**
+	 * 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
+	 * @return True if the task manager has not been registered before and is registered successfully; otherwise false
+	 */
+	@Override
+	public boolean registerTaskManager(final TaskExecutorConnection taskExecutorConnection, SlotReport initialSlotReport) {
+		checkInit();
+		LOG.debug("Registering task executor {} under {} at the slot manager.", taskExecutorConnection.getResourceID(), taskExecutorConnection.getInstanceID());
+
+		// we identify task managers by their instance id
+		if (taskExecutorManager.isTaskManagerRegistered(taskExecutorConnection.getInstanceID())) {
+			LOG.debug("Task executor {} was already registered.", taskExecutorConnection.getResourceID());
+			reportSlotStatus(taskExecutorConnection.getInstanceID(), initialSlotReport);
+			return false;
+		} else {
+			if (!taskExecutorManager.registerTaskManager(taskExecutorConnection, initialSlotReport)) {
+				LOG.debug("Task executor {} could not be registered.", taskExecutorConnection.getResourceID());
+				return false;
+			}
+
+			// register the new slots
+			for (SlotStatus slotStatus : initialSlotReport) {
+				slotTracker.addSlot(
+					slotStatus.getSlotID(),
+					slotStatus.getResourceProfile(),
+					taskExecutorConnection,
+					slotStatus.getJobID());
+			}
+
+			checkResourceRequirements();
+			return true;
+		}
+	}
+
+	@Override
+	public boolean unregisterTaskManager(InstanceID instanceId, Exception cause) {
+		checkInit();
+
+		LOG.debug("Unregistering task executor {} from the slot manager.", instanceId);
+
+		if (taskExecutorManager.isTaskManagerRegistered(instanceId)) {
+			slotTracker.removeSlots(taskExecutorManager.getSlotsOf(instanceId));
+			taskExecutorManager.unregisterTaskExecutor(instanceId);
+			checkResourceRequirements();
+
+			return true;
+		} else {
+			LOG.debug("There is no task executor registered with instance ID {}. Ignoring this message.", instanceId);
+
+			return false;
+		}
+	}
+
+	/**
+	 * Reports the current slot allocations for a task manager identified by the given instance id.
+	 *
+	 * @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
+	 */
+	@Override
+	public boolean reportSlotStatus(InstanceID instanceId, SlotReport slotReport) {
+		checkInit();
+
+		LOG.debug("Received slot report from instance {}: {}.", instanceId, slotReport);
+
+		if (taskExecutorManager.isTaskManagerRegistered(instanceId)) {
+			slotTracker.notifySlotStatus(slotReport);
+			checkResourceRequirements();
+			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
+	 */
+	@Override
+	public void freeSlot(SlotID slotId, AllocationID allocationId) {
+		checkInit();
+		LOG.debug("Freeing slot {}.", slotId);
+
+		slotTracker.notifyFree(slotId);
+		checkResourceRequirements();
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Requirement matching
+	// ---------------------------------------------------------------------------------------------
+
+	private void checkResourceRequirements() {
+		final Map<JobID, Collection<ResourceRequirement>> missingResources = resourceTracker.getMissingResources();
+		if (missingResources.isEmpty()) {
+			return;
+		}
+
+		final Map<JobID, ResourceCounter> outstandingRequirements = new LinkedHashMap<>();
+		for (Map.Entry<JobID, Collection<ResourceRequirement>> resourceRequirements : missingResources.entrySet()) {
+			JobID jobId = resourceRequirements.getKey();
+
+			for (ResourceRequirement resourceRequirement : resourceRequirements.getValue()) {
+				int numMissingSlots = internalRequestSlots(jobId, jobMasterTargetAddresses.get(jobId), resourceRequirement);
+				if (numMissingSlots > 0) {
+					outstandingRequirements
+						.computeIfAbsent(jobId, ignored -> new ResourceCounter())
+						.incrementCount(resourceRequirement.getResourceProfile(), numMissingSlots);
+				}
+			}
+		}
+
+		final ResourceCounter pendingSlots = new ResourceCounter(taskExecutorManager.getPendingTaskManagerSlots().stream().collect(
+			Collectors.groupingBy(
+				PendingTaskManagerSlot::getResourceProfile,
+				Collectors.summingInt(x -> 1))));
+
+		for (Map.Entry<JobID, ResourceCounter> unfulfilledRequirement : outstandingRequirements.entrySet()) {
+			tryFulfillRequirementsWithPendingOrNewSlots(
+				unfulfilledRequirement.getKey(),
+				unfulfilledRequirement.getValue().getResourceProfilesWithCount(),
+				pendingSlots);
+		}
+	}
+
+	/**
+	 * Tries to allocate slots for the given requirement. If there are not enough slots available, the
+	 * resource manager is informed to allocate more resources.
+	 *
+	 * @param jobId job to allocate slots for
+	 * @param targetAddress address of the jobmaster
+	 * @param resourceRequirement required slots
+	 * @return the number of missing slots
+	 */
+	private int internalRequestSlots(JobID jobId, String targetAddress, ResourceRequirement resourceRequirement) {
+		final ResourceProfile requiredResource = resourceRequirement.getResourceProfile();
+		Collection<TaskManagerSlotInformation> freeSlots = slotTracker.getFreeSlots();
+
+		int numUnfulfilled = 0;
+		for (int x = 0; x < resourceRequirement.getNumberOfRequiredSlots(); x++) {
+
+			final Optional<TaskManagerSlotInformation> reservedSlot = slotMatchingStrategy.findMatchingSlot(requiredResource, freeSlots, this::getNumberRegisteredSlotsOf);
+			if (reservedSlot.isPresent()) {
+				// we do not need to modify freeSlots because it is indirectly modified by the allocation
+				allocateSlot(reservedSlot.get(), jobId, targetAddress, requiredResource);
+			} else {
+				numUnfulfilled++;
+			}
+		}
+		return numUnfulfilled;
+	}
+
+	/**
+	 * Allocates the given slot. This entails sending a registration message to the task manager and treating failures.
+	 *
+	 * @param taskManagerSlot slot to allocate
+	 * @param jobId job for which the slot should be allocated for
+	 * @param targetAddress address of the job master
+	 * @param resourceProfile resource profile for the requirement for which the slot is used
+	 */
+	private void allocateSlot(TaskManagerSlotInformation taskManagerSlot, JobID jobId, String targetAddress, ResourceProfile resourceProfile) {
+		TaskExecutorConnection taskExecutorConnection = taskManagerSlot.getTaskManagerConnection();
+		TaskExecutorGateway gateway = taskExecutorConnection.getTaskExecutorGateway();
+
+		final CompletableFuture<Acknowledge> completableFuture = new CompletableFuture<>();
+		final SlotID slotId = taskManagerSlot.getSlotId();
+		final InstanceID instanceId = taskManagerSlot.getInstanceId();
+
+		LOG.debug("Starting allocation of slot {} for job {} with resource profile {}.", taskManagerSlot.getSlotId(), jobId, resourceProfile);
+
+		slotTracker.notifyAllocationStart(slotId, jobId);
+
+		if (!taskExecutorManager.isTaskManagerRegistered(instanceId)) {
+			throw new IllegalStateException("Could not find a registered task manager for instance id " +
+				instanceId + '.');
+		}

Review comment:
       Maybe move this to the top of the method.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManager.java
##########
@@ -0,0 +1,609 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.metrics.MetricNames;
+import org.apache.flink.runtime.metrics.groups.SlotManagerMetricGroup;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerId;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.slots.ResourceCounter;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+import org.apache.flink.runtime.slots.ResourceRequirements;
+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.SlotOccupiedException;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ScheduledFuture;
+import java.util.function.BiFunction;
+import java.util.stream.Collectors;
+
+/**
+ * Implementation of {@link SlotManager} supporting declarative slot management.
+ */
+public class DeclarativeSlotManager implements SlotManager {
+	private static final Logger LOG = LoggerFactory.getLogger(DeclarativeSlotManager.class);
+
+	private final SlotTracker slotTracker;
+	private final ResourceTracker resourceTracker;
+	private final BiFunction<Executor, ResourceActions, TaskExecutorManager> taskExecutorManagerFactory;
+	private TaskExecutorManager taskExecutorManager;
+
+	/** Timeout for slot requests to the task manager. */
+	private final Time taskManagerRequestTimeout;
+	private ScheduledFuture<?> slotRequestTimeoutCheck;
+
+	private final SlotMatchingStrategy slotMatchingStrategy;
+
+	private final SlotManagerMetricGroup slotManagerMetricGroup;
+
+	private final Map<JobID, String> jobMasterTargetAddresses = new HashMap<>();
+	private final HashMap<SlotID, CompletableFuture<Acknowledge>> pendingSlotAllocationFutures;
+
+	/** ResourceManager's id. */
+	private ResourceManagerId resourceManagerId;
+
+	/** Executor for future callbacks which have to be "synchronized". */
+	private Executor mainThreadExecutor;
+
+	/** Callbacks for resource (de-)allocations. */
+	private ResourceActions resourceActions;
+
+	/** True iff the component has been started. */
+	private boolean started;
+
+	public DeclarativeSlotManager(
+			ScheduledExecutor scheduledExecutor,
+			SlotManagerConfiguration slotManagerConfiguration,
+			SlotManagerMetricGroup slotManagerMetricGroup,
+			ResourceTracker resourceTracker,
+			SlotTracker slotTracker) {
+
+		Preconditions.checkNotNull(slotManagerConfiguration);
+		this.taskManagerRequestTimeout = slotManagerConfiguration.getTaskManagerRequestTimeout();
+		this.slotManagerMetricGroup = Preconditions.checkNotNull(slotManagerMetricGroup);
+		this.resourceTracker = Preconditions.checkNotNull(resourceTracker);
+
+		pendingSlotAllocationFutures = new HashMap<>(16);
+
+		this.slotTracker = Preconditions.checkNotNull(slotTracker);
+		slotTracker.registerSlotStatusUpdateListener(createSlotStatusUpdateListener());
+
+		slotMatchingStrategy = slotManagerConfiguration.getSlotMatchingStrategy();
+
+		taskExecutorManagerFactory = (executor, resourceActions) -> new TaskExecutorManager(
+			slotManagerConfiguration.getDefaultWorkerResourceSpec(),
+			slotManagerConfiguration.getNumSlotsPerWorker(),
+			slotManagerConfiguration.getMaxSlotNum(),
+			slotManagerConfiguration.isWaitResultConsumedBeforeRelease(),
+			slotManagerConfiguration.getRedundantTaskManagerNum(),
+			slotManagerConfiguration.getTaskManagerTimeout(),
+			scheduledExecutor,
+			executor,
+			resourceActions);
+
+		resourceManagerId = null;
+		resourceActions = null;
+		mainThreadExecutor = null;
+		slotRequestTimeoutCheck = null;
+		taskExecutorManager = null;
+
+		started = false;
+	}
+
+	private SlotStatusUpdateListener createSlotStatusUpdateListener() {
+		return (taskManagerSlot, previous, current, jobId) -> {
+			if (previous == SlotState.PENDING) {
+				cancelAllocationFuture(taskManagerSlot.getSlotId());
+			}
+
+			if (current == SlotState.PENDING) {
+				resourceTracker.notifyAcquiredResource(jobId, taskManagerSlot.getResourceProfile());
+			}
+			if (current == SlotState.FREE) {
+				resourceTracker.notifyLostResource(jobId, taskManagerSlot.getResourceProfile());
+			}
+
+			if (current == SlotState.ALLOCATED) {
+				taskExecutorManager.occupySlot(taskManagerSlot.getInstanceId());
+			}
+			if (previous == SlotState.ALLOCATED && current == SlotState.FREE) {
+				taskExecutorManager.freeSlot(taskManagerSlot.getInstanceId());
+			}
+		};
+	}
+
+	private void cancelAllocationFuture(SlotID slotId) {
+		final CompletableFuture<Acknowledge> acknowledgeCompletableFuture = pendingSlotAllocationFutures.remove(slotId);
+		// the future may be null if we are just re-playing the state transitions due to a slot report
+		if (acknowledgeCompletableFuture != null) {
+			acknowledgeCompletableFuture.cancel(false);
+		}
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Component lifecycle methods
+	// ---------------------------------------------------------------------------------------------
+
+	/**
+	 * Starts the slot manager with the given leader id and resource manager actions.
+	 *
+	 * @param newResourceManagerId to use for communication with the task managers
+	 * @param newMainThreadExecutor to use to run code in the ResourceManager's main thread
+	 * @param newResourceActions to use for resource (de-)allocations
+	 */
+	@Override
+	public void start(ResourceManagerId newResourceManagerId, Executor newMainThreadExecutor, ResourceActions newResourceActions) {
+		LOG.info("Starting the slot manager.");
+
+		this.resourceManagerId = Preconditions.checkNotNull(newResourceManagerId);
+		mainThreadExecutor = Preconditions.checkNotNull(newMainThreadExecutor);
+		resourceActions = Preconditions.checkNotNull(newResourceActions);
+		taskExecutorManager = taskExecutorManagerFactory.apply(newMainThreadExecutor, newResourceActions);
+
+		started = true;
+
+		registerSlotManagerMetrics();
+	}
+
+	private void registerSlotManagerMetrics() {
+		slotManagerMetricGroup.gauge(
+			MetricNames.TASK_SLOTS_AVAILABLE,
+			() -> (long) getNumberFreeSlots());
+		slotManagerMetricGroup.gauge(
+			MetricNames.TASK_SLOTS_TOTAL,
+			() -> (long) getNumberRegisteredSlots());
+	}
+
+	/**
+	 * Suspends the component. This clears the internal state of the slot manager.
+	 */
+	@Override
+	public void suspend() {
+		LOG.info("Suspending the slot manager.");
+
+		if (slotRequestTimeoutCheck != null) {
+			slotRequestTimeoutCheck.cancel(false);
+			slotRequestTimeoutCheck = null;
+		}
+
+		resourceTracker.clear();
+		taskExecutorManager.close();
+
+		for (InstanceID registeredTaskManager : taskExecutorManager.getTaskExecutors()) {
+			unregisterTaskManager(registeredTaskManager, new SlotManagerException("The slot manager is being suspended."));
+		}
+
+		taskExecutorManager = null;
+		resourceManagerId = null;
+		resourceActions = null;
+		started = false;
+	}
+
+	/**
+	 * Closes the slot manager.
+	 *
+	 * @throws Exception if the close operation fails
+	 */
+	@Override
+	public void close() throws Exception {
+		LOG.info("Closing the slot manager.");
+
+		suspend();
+		slotManagerMetricGroup.close();
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Public API
+	// ---------------------------------------------------------------------------------------------
+
+	@Override
+	public void processResourceRequirements(ResourceRequirements resourceRequirements) {
+		checkInit();
+		LOG.debug("Received resource requirements from job {}: {}", resourceRequirements.getJobId(), resourceRequirements.getResourceRequirements());
+
+		if (resourceRequirements.getResourceRequirements().isEmpty()) {
+			jobMasterTargetAddresses.remove(resourceRequirements.getJobId());
+		} else {
+			jobMasterTargetAddresses.put(resourceRequirements.getJobId(), resourceRequirements.getTargetAddress());
+		}
+		resourceTracker.notifyResourceRequirements(resourceRequirements.getJobId(), resourceRequirements.getResourceRequirements());
+		checkResourceRequirements();
+	}
+
+	/**
+	 * 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
+	 * @return True if the task manager has not been registered before and is registered successfully; otherwise false
+	 */
+	@Override
+	public boolean registerTaskManager(final TaskExecutorConnection taskExecutorConnection, SlotReport initialSlotReport) {
+		checkInit();
+		LOG.debug("Registering task executor {} under {} at the slot manager.", taskExecutorConnection.getResourceID(), taskExecutorConnection.getInstanceID());
+
+		// we identify task managers by their instance id
+		if (taskExecutorManager.isTaskManagerRegistered(taskExecutorConnection.getInstanceID())) {
+			LOG.debug("Task executor {} was already registered.", taskExecutorConnection.getResourceID());
+			reportSlotStatus(taskExecutorConnection.getInstanceID(), initialSlotReport);
+			return false;
+		} else {
+			if (!taskExecutorManager.registerTaskManager(taskExecutorConnection, initialSlotReport)) {
+				LOG.debug("Task executor {} could not be registered.", taskExecutorConnection.getResourceID());
+				return false;
+			}
+
+			// register the new slots
+			for (SlotStatus slotStatus : initialSlotReport) {
+				slotTracker.addSlot(
+					slotStatus.getSlotID(),
+					slotStatus.getResourceProfile(),
+					taskExecutorConnection,
+					slotStatus.getJobID());
+			}
+
+			checkResourceRequirements();
+			return true;
+		}
+	}
+
+	@Override
+	public boolean unregisterTaskManager(InstanceID instanceId, Exception cause) {
+		checkInit();
+
+		LOG.debug("Unregistering task executor {} from the slot manager.", instanceId);
+
+		if (taskExecutorManager.isTaskManagerRegistered(instanceId)) {
+			slotTracker.removeSlots(taskExecutorManager.getSlotsOf(instanceId));
+			taskExecutorManager.unregisterTaskExecutor(instanceId);
+			checkResourceRequirements();
+
+			return true;
+		} else {
+			LOG.debug("There is no task executor registered with instance ID {}. Ignoring this message.", instanceId);
+
+			return false;
+		}
+	}
+
+	/**
+	 * Reports the current slot allocations for a task manager identified by the given instance id.
+	 *
+	 * @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
+	 */
+	@Override
+	public boolean reportSlotStatus(InstanceID instanceId, SlotReport slotReport) {
+		checkInit();
+
+		LOG.debug("Received slot report from instance {}: {}.", instanceId, slotReport);
+
+		if (taskExecutorManager.isTaskManagerRegistered(instanceId)) {
+			slotTracker.notifySlotStatus(slotReport);
+			checkResourceRequirements();
+			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
+	 */
+	@Override
+	public void freeSlot(SlotID slotId, AllocationID allocationId) {
+		checkInit();
+		LOG.debug("Freeing slot {}.", slotId);
+
+		slotTracker.notifyFree(slotId);
+		checkResourceRequirements();
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Requirement matching
+	// ---------------------------------------------------------------------------------------------
+
+	private void checkResourceRequirements() {
+		final Map<JobID, Collection<ResourceRequirement>> missingResources = resourceTracker.getMissingResources();
+		if (missingResources.isEmpty()) {
+			return;
+		}
+
+		final Map<JobID, ResourceCounter> outstandingRequirements = new LinkedHashMap<>();
+		for (Map.Entry<JobID, Collection<ResourceRequirement>> resourceRequirements : missingResources.entrySet()) {
+			JobID jobId = resourceRequirements.getKey();
+
+			for (ResourceRequirement resourceRequirement : resourceRequirements.getValue()) {
+				int numMissingSlots = internalRequestSlots(jobId, jobMasterTargetAddresses.get(jobId), resourceRequirement);
+				if (numMissingSlots > 0) {
+					outstandingRequirements
+						.computeIfAbsent(jobId, ignored -> new ResourceCounter())
+						.incrementCount(resourceRequirement.getResourceProfile(), numMissingSlots);
+				}
+			}
+		}
+
+		final ResourceCounter pendingSlots = new ResourceCounter(taskExecutorManager.getPendingTaskManagerSlots().stream().collect(
+			Collectors.groupingBy(
+				PendingTaskManagerSlot::getResourceProfile,
+				Collectors.summingInt(x -> 1))));
+
+		for (Map.Entry<JobID, ResourceCounter> unfulfilledRequirement : outstandingRequirements.entrySet()) {
+			tryFulfillRequirementsWithPendingOrNewSlots(
+				unfulfilledRequirement.getKey(),
+				unfulfilledRequirement.getValue().getResourceProfilesWithCount(),
+				pendingSlots);
+		}
+	}
+
+	/**
+	 * Tries to allocate slots for the given requirement. If there are not enough slots available, the
+	 * resource manager is informed to allocate more resources.
+	 *
+	 * @param jobId job to allocate slots for
+	 * @param targetAddress address of the jobmaster
+	 * @param resourceRequirement required slots
+	 * @return the number of missing slots
+	 */
+	private int internalRequestSlots(JobID jobId, String targetAddress, ResourceRequirement resourceRequirement) {
+		final ResourceProfile requiredResource = resourceRequirement.getResourceProfile();
+		Collection<TaskManagerSlotInformation> freeSlots = slotTracker.getFreeSlots();
+
+		int numUnfulfilled = 0;
+		for (int x = 0; x < resourceRequirement.getNumberOfRequiredSlots(); x++) {
+
+			final Optional<TaskManagerSlotInformation> reservedSlot = slotMatchingStrategy.findMatchingSlot(requiredResource, freeSlots, this::getNumberRegisteredSlotsOf);
+			if (reservedSlot.isPresent()) {
+				// we do not need to modify freeSlots because it is indirectly modified by the allocation
+				allocateSlot(reservedSlot.get(), jobId, targetAddress, requiredResource);
+			} else {
+				numUnfulfilled++;
+			}
+		}
+		return numUnfulfilled;
+	}
+
+	/**
+	 * Allocates the given slot. This entails sending a registration message to the task manager and treating failures.
+	 *
+	 * @param taskManagerSlot slot to allocate
+	 * @param jobId job for which the slot should be allocated for
+	 * @param targetAddress address of the job master
+	 * @param resourceProfile resource profile for the requirement for which the slot is used
+	 */
+	private void allocateSlot(TaskManagerSlotInformation taskManagerSlot, JobID jobId, String targetAddress, ResourceProfile resourceProfile) {
+		TaskExecutorConnection taskExecutorConnection = taskManagerSlot.getTaskManagerConnection();
+		TaskExecutorGateway gateway = taskExecutorConnection.getTaskExecutorGateway();
+
+		final CompletableFuture<Acknowledge> completableFuture = new CompletableFuture<>();
+		final SlotID slotId = taskManagerSlot.getSlotId();
+		final InstanceID instanceId = taskManagerSlot.getInstanceId();
+
+		LOG.debug("Starting allocation of slot {} for job {} with resource profile {}.", taskManagerSlot.getSlotId(), jobId, resourceProfile);
+
+		slotTracker.notifyAllocationStart(slotId, jobId);
+
+		if (!taskExecutorManager.isTaskManagerRegistered(instanceId)) {
+			throw new IllegalStateException("Could not find a registered task manager for instance id " +
+				instanceId + '.');
+		}
+
+		taskExecutorManager.markUsed(instanceId);
+
+		// RPC call to the task manager
+		CompletableFuture<Acknowledge> requestFuture = gateway.requestSlot(
+			slotId,
+			jobId,
+			new AllocationID(),
+			resourceProfile,
+			targetAddress,
+			resourceManagerId,
+			taskManagerRequestTimeout);
+
+		requestFuture.whenComplete(
+			(Acknowledge acknowledge, Throwable throwable) -> {
+				if (acknowledge != null) {
+					completableFuture.complete(acknowledge);
+				} else {
+					completableFuture.completeExceptionally(throwable);
+				}
+			});
+
+		completableFuture.whenCompleteAsync(
+			(Acknowledge acknowledge, Throwable throwable) -> {
+				try {
+					if (acknowledge != null) {
+						LOG.trace("Completed allocation of slot {} for job {}.", slotId, jobId);
+						slotTracker.notifyAllocationComplete(slotId, jobId);
+					} else {
+						if (throwable instanceof SlotOccupiedException) {
+							SlotOccupiedException exception = (SlotOccupiedException) throwable;
+							LOG.debug("Tried allocating slot {} for job {}, but it was already allocated for job {}.", slotId, jobId, exception.getJobId());
+							// report as a slot status to force the state transition
+							// this could be a problem if we ever assume that the task executor always reports about all slots
+							slotTracker.notifySlotStatus(Collections.singleton(new SlotStatus(slotId, taskManagerSlot.getResourceProfile(), exception.getJobId(), exception.getAllocationId())));
+						} else {
+							if (throwable instanceof CancellationException) {
+								LOG.debug("Cancelled allocation of slot {} for job {}.", slotId, jobId, throwable);
+							} else {
+								LOG.warn("Slot allocation for slot {} for job {} failed.", slotId, jobId, throwable);
+								slotTracker.notifyFree(slotId);
+							}
+						}
+					}
+					checkResourceRequirements();
+				} catch (Exception e) {
+					LOG.error("Error while completing the slot allocation.", e);
+				}
+			},
+			mainThreadExecutor);
+	}
+
+	private void tryFulfillRequirementsWithPendingOrNewSlots(JobID jobId, Map<ResourceProfile, Integer> missingResources, ResourceCounter pendingSlots) {
+		for (Map.Entry<ResourceProfile, Integer> missingResource : missingResources.entrySet()) {
+			ResourceProfile profile = missingResource.getKey();
+			for (int i = 0; i < missingResource.getValue(); i++) {
+				if (!tryFulfillWithPendingSlots(profile, pendingSlots)) {
+					Optional<ResourceRequirement> newlyFulfillableRequirements = taskExecutorManager.allocateWorker(profile);
+					if (newlyFulfillableRequirements.isPresent()) {
+						ResourceRequirement newSlots = newlyFulfillableRequirements.get();
+						// reserve one of the new slots
+						if (newSlots.getNumberOfRequiredSlots() > 1) {
+							pendingSlots.incrementCount(newSlots.getResourceProfile(), newSlots.getNumberOfRequiredSlots() - 1);
+						}
+					} else {
+						LOG.warn("Could not fulfill resource requirements of job {}.", jobId);
+						resourceActions.notifyNotEnoughResourcesAvailable(jobId, resourceTracker.getAcquiredResources(jobId));
+						return;
+					}
+				}
+			}
+		}
+	}
+
+	private boolean tryFulfillWithPendingSlots(ResourceProfile resourceProfile, ResourceCounter pendingSlots) {
+		Set<ResourceProfile> pendingSlotProfiles = pendingSlots.getResourceProfiles();
+
+		// short-cut, pretty much only applicable to fine-grained resource management
+		if (pendingSlotProfiles.contains(resourceProfile)) {
+			pendingSlots.decrementCount(resourceProfile, 1);
+			return true;
+		}
+
+		for (ResourceProfile pendingSlotProfile : pendingSlotProfiles) {
+			if (pendingSlotProfile.isMatching(resourceProfile)) {
+				pendingSlots.decrementCount(pendingSlotProfile, 1);
+				return true;
+			}
+		}
+
+		return false;
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Legacy APIs
+	// ---------------------------------------------------------------------------------------------
+
+	@Override
+	public int getNumberRegisteredSlots() {
+		return taskExecutorManager.getNumberRegisteredSlots();
+	}
+
+	@Override
+	public int getNumberRegisteredSlotsOf(InstanceID instanceId) {
+		return taskExecutorManager.getNumberRegisteredSlotsOf(instanceId);
+	}
+
+	@Override
+	public int getNumberFreeSlots() {
+		return taskExecutorManager.getNumberFreeSlots();
+	}
+
+	@Override
+	public int getNumberFreeSlotsOf(InstanceID instanceId) {
+		return taskExecutorManager.getNumberFreeSlotsOf(instanceId);
+	}
+
+	@Override
+	public Map<WorkerResourceSpec, Integer> getRequiredResources() {
+		return taskExecutorManager.getRequiredWorkers();
+	}
+
+	@Override
+	public ResourceProfile getRegisteredResource() {
+		return taskExecutorManager.getTotalRegisteredResources();
+	}
+
+	@Override
+	public ResourceProfile getRegisteredResourceOf(InstanceID instanceID) {
+		return taskExecutorManager.getTotalRegisteredResourcesOf(instanceID);
+	}
+
+	@Override
+	public ResourceProfile getFreeResource() {
+		return taskExecutorManager.getTotalFreeResources();
+	}
+
+	@Override
+	public ResourceProfile getFreeResourceOf(InstanceID instanceID) {
+		return taskExecutorManager.getTotalFreeResourcesOf(instanceID);
+	}
+
+	@Override
+	public void setFailUnfulfillableRequest(boolean failUnfulfillableRequest) {
+		// we always send notifications if we cannot fulfill requests, and it is the responsibility of the JobManager
+		// to handle it (e.g., by reducing requirements and failing outright)
+	}
+
+	@Override
+	public int getNumberPendingSlotRequests() {
+		// only exists for testing purposes
+		throw new UnsupportedOperationException();
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Internal utility methods
+	// ---------------------------------------------------------------------------------------------
+
+	private void checkInit() {
+		Preconditions.checkState(started, "The slot manager has not been started.");
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Testing methods
+	// ---------------------------------------------------------------------------------------------
+
+	@Override
+	@VisibleForTesting
+	public void unregisterTaskManagersAndReleaseResources() {

Review comment:
       Why is this method not implemented?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java
##########
@@ -1190,6 +1215,11 @@ public void notifyAllocationFailure(JobID jobId, AllocationID allocationId, Exce
 				jobManagerRegistration.getJobManagerGateway().notifyAllocationFailure(allocationId, cause);
 			}
 		}
+
+		@Override
+		public void notifyNotEnoughResourcesAvailable(JobID jobId, Collection<ResourceRequirement> acquiredResources) {
+			validateRunsInMainThread();
+		}

Review comment:
       What is missing here?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManager.java
##########
@@ -0,0 +1,609 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.metrics.MetricNames;
+import org.apache.flink.runtime.metrics.groups.SlotManagerMetricGroup;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerId;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.slots.ResourceCounter;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+import org.apache.flink.runtime.slots.ResourceRequirements;
+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.SlotOccupiedException;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ScheduledFuture;
+import java.util.function.BiFunction;
+import java.util.stream.Collectors;
+
+/**
+ * Implementation of {@link SlotManager} supporting declarative slot management.
+ */
+public class DeclarativeSlotManager implements SlotManager {
+	private static final Logger LOG = LoggerFactory.getLogger(DeclarativeSlotManager.class);
+
+	private final SlotTracker slotTracker;
+	private final ResourceTracker resourceTracker;
+	private final BiFunction<Executor, ResourceActions, TaskExecutorManager> taskExecutorManagerFactory;
+	private TaskExecutorManager taskExecutorManager;
+
+	/** Timeout for slot requests to the task manager. */
+	private final Time taskManagerRequestTimeout;
+	private ScheduledFuture<?> slotRequestTimeoutCheck;
+
+	private final SlotMatchingStrategy slotMatchingStrategy;
+
+	private final SlotManagerMetricGroup slotManagerMetricGroup;
+
+	private final Map<JobID, String> jobMasterTargetAddresses = new HashMap<>();
+	private final HashMap<SlotID, CompletableFuture<Acknowledge>> pendingSlotAllocationFutures;
+
+	/** ResourceManager's id. */
+	private ResourceManagerId resourceManagerId;
+
+	/** Executor for future callbacks which have to be "synchronized". */
+	private Executor mainThreadExecutor;
+
+	/** Callbacks for resource (de-)allocations. */
+	private ResourceActions resourceActions;
+
+	/** True iff the component has been started. */
+	private boolean started;
+
+	public DeclarativeSlotManager(
+			ScheduledExecutor scheduledExecutor,
+			SlotManagerConfiguration slotManagerConfiguration,
+			SlotManagerMetricGroup slotManagerMetricGroup,
+			ResourceTracker resourceTracker,
+			SlotTracker slotTracker) {
+
+		Preconditions.checkNotNull(slotManagerConfiguration);
+		this.taskManagerRequestTimeout = slotManagerConfiguration.getTaskManagerRequestTimeout();
+		this.slotManagerMetricGroup = Preconditions.checkNotNull(slotManagerMetricGroup);
+		this.resourceTracker = Preconditions.checkNotNull(resourceTracker);
+
+		pendingSlotAllocationFutures = new HashMap<>(16);
+
+		this.slotTracker = Preconditions.checkNotNull(slotTracker);
+		slotTracker.registerSlotStatusUpdateListener(createSlotStatusUpdateListener());
+
+		slotMatchingStrategy = slotManagerConfiguration.getSlotMatchingStrategy();
+
+		taskExecutorManagerFactory = (executor, resourceActions) -> new TaskExecutorManager(
+			slotManagerConfiguration.getDefaultWorkerResourceSpec(),
+			slotManagerConfiguration.getNumSlotsPerWorker(),
+			slotManagerConfiguration.getMaxSlotNum(),
+			slotManagerConfiguration.isWaitResultConsumedBeforeRelease(),
+			slotManagerConfiguration.getRedundantTaskManagerNum(),
+			slotManagerConfiguration.getTaskManagerTimeout(),
+			scheduledExecutor,
+			executor,
+			resourceActions);
+
+		resourceManagerId = null;
+		resourceActions = null;
+		mainThreadExecutor = null;
+		slotRequestTimeoutCheck = null;
+		taskExecutorManager = null;
+
+		started = false;
+	}
+
+	private SlotStatusUpdateListener createSlotStatusUpdateListener() {
+		return (taskManagerSlot, previous, current, jobId) -> {
+			if (previous == SlotState.PENDING) {
+				cancelAllocationFuture(taskManagerSlot.getSlotId());
+			}
+
+			if (current == SlotState.PENDING) {
+				resourceTracker.notifyAcquiredResource(jobId, taskManagerSlot.getResourceProfile());
+			}
+			if (current == SlotState.FREE) {
+				resourceTracker.notifyLostResource(jobId, taskManagerSlot.getResourceProfile());
+			}
+
+			if (current == SlotState.ALLOCATED) {
+				taskExecutorManager.occupySlot(taskManagerSlot.getInstanceId());
+			}
+			if (previous == SlotState.ALLOCATED && current == SlotState.FREE) {
+				taskExecutorManager.freeSlot(taskManagerSlot.getInstanceId());
+			}
+		};
+	}
+
+	private void cancelAllocationFuture(SlotID slotId) {
+		final CompletableFuture<Acknowledge> acknowledgeCompletableFuture = pendingSlotAllocationFutures.remove(slotId);
+		// the future may be null if we are just re-playing the state transitions due to a slot report
+		if (acknowledgeCompletableFuture != null) {
+			acknowledgeCompletableFuture.cancel(false);
+		}
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Component lifecycle methods
+	// ---------------------------------------------------------------------------------------------
+
+	/**
+	 * Starts the slot manager with the given leader id and resource manager actions.
+	 *
+	 * @param newResourceManagerId to use for communication with the task managers
+	 * @param newMainThreadExecutor to use to run code in the ResourceManager's main thread
+	 * @param newResourceActions to use for resource (de-)allocations
+	 */
+	@Override
+	public void start(ResourceManagerId newResourceManagerId, Executor newMainThreadExecutor, ResourceActions newResourceActions) {
+		LOG.info("Starting the slot manager.");
+
+		this.resourceManagerId = Preconditions.checkNotNull(newResourceManagerId);
+		mainThreadExecutor = Preconditions.checkNotNull(newMainThreadExecutor);
+		resourceActions = Preconditions.checkNotNull(newResourceActions);
+		taskExecutorManager = taskExecutorManagerFactory.apply(newMainThreadExecutor, newResourceActions);
+
+		started = true;
+
+		registerSlotManagerMetrics();
+	}
+
+	private void registerSlotManagerMetrics() {
+		slotManagerMetricGroup.gauge(
+			MetricNames.TASK_SLOTS_AVAILABLE,
+			() -> (long) getNumberFreeSlots());
+		slotManagerMetricGroup.gauge(
+			MetricNames.TASK_SLOTS_TOTAL,
+			() -> (long) getNumberRegisteredSlots());
+	}

Review comment:
       Is it a problem that we might register the gauges multiple times in case that the `DSM` goes start -> suspend -> start -> suspend -> ...?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManager.java
##########
@@ -0,0 +1,609 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.metrics.MetricNames;
+import org.apache.flink.runtime.metrics.groups.SlotManagerMetricGroup;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerId;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.slots.ResourceCounter;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+import org.apache.flink.runtime.slots.ResourceRequirements;
+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.SlotOccupiedException;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ScheduledFuture;
+import java.util.function.BiFunction;
+import java.util.stream.Collectors;
+
+/**
+ * Implementation of {@link SlotManager} supporting declarative slot management.
+ */
+public class DeclarativeSlotManager implements SlotManager {
+	private static final Logger LOG = LoggerFactory.getLogger(DeclarativeSlotManager.class);
+
+	private final SlotTracker slotTracker;
+	private final ResourceTracker resourceTracker;
+	private final BiFunction<Executor, ResourceActions, TaskExecutorManager> taskExecutorManagerFactory;
+	private TaskExecutorManager taskExecutorManager;
+
+	/** Timeout for slot requests to the task manager. */
+	private final Time taskManagerRequestTimeout;
+	private ScheduledFuture<?> slotRequestTimeoutCheck;
+
+	private final SlotMatchingStrategy slotMatchingStrategy;
+
+	private final SlotManagerMetricGroup slotManagerMetricGroup;
+
+	private final Map<JobID, String> jobMasterTargetAddresses = new HashMap<>();
+	private final HashMap<SlotID, CompletableFuture<Acknowledge>> pendingSlotAllocationFutures;
+
+	/** ResourceManager's id. */
+	private ResourceManagerId resourceManagerId;
+
+	/** Executor for future callbacks which have to be "synchronized". */
+	private Executor mainThreadExecutor;
+
+	/** Callbacks for resource (de-)allocations. */
+	private ResourceActions resourceActions;
+
+	/** True iff the component has been started. */
+	private boolean started;
+
+	public DeclarativeSlotManager(
+			ScheduledExecutor scheduledExecutor,
+			SlotManagerConfiguration slotManagerConfiguration,
+			SlotManagerMetricGroup slotManagerMetricGroup,
+			ResourceTracker resourceTracker,
+			SlotTracker slotTracker) {
+
+		Preconditions.checkNotNull(slotManagerConfiguration);
+		this.taskManagerRequestTimeout = slotManagerConfiguration.getTaskManagerRequestTimeout();
+		this.slotManagerMetricGroup = Preconditions.checkNotNull(slotManagerMetricGroup);
+		this.resourceTracker = Preconditions.checkNotNull(resourceTracker);
+
+		pendingSlotAllocationFutures = new HashMap<>(16);
+
+		this.slotTracker = Preconditions.checkNotNull(slotTracker);
+		slotTracker.registerSlotStatusUpdateListener(createSlotStatusUpdateListener());
+
+		slotMatchingStrategy = slotManagerConfiguration.getSlotMatchingStrategy();
+
+		taskExecutorManagerFactory = (executor, resourceActions) -> new TaskExecutorManager(
+			slotManagerConfiguration.getDefaultWorkerResourceSpec(),
+			slotManagerConfiguration.getNumSlotsPerWorker(),
+			slotManagerConfiguration.getMaxSlotNum(),
+			slotManagerConfiguration.isWaitResultConsumedBeforeRelease(),
+			slotManagerConfiguration.getRedundantTaskManagerNum(),
+			slotManagerConfiguration.getTaskManagerTimeout(),
+			scheduledExecutor,
+			executor,
+			resourceActions);
+
+		resourceManagerId = null;
+		resourceActions = null;
+		mainThreadExecutor = null;
+		slotRequestTimeoutCheck = null;
+		taskExecutorManager = null;
+
+		started = false;
+	}
+
+	private SlotStatusUpdateListener createSlotStatusUpdateListener() {
+		return (taskManagerSlot, previous, current, jobId) -> {
+			if (previous == SlotState.PENDING) {
+				cancelAllocationFuture(taskManagerSlot.getSlotId());
+			}
+
+			if (current == SlotState.PENDING) {
+				resourceTracker.notifyAcquiredResource(jobId, taskManagerSlot.getResourceProfile());
+			}
+			if (current == SlotState.FREE) {
+				resourceTracker.notifyLostResource(jobId, taskManagerSlot.getResourceProfile());
+			}
+
+			if (current == SlotState.ALLOCATED) {
+				taskExecutorManager.occupySlot(taskManagerSlot.getInstanceId());
+			}
+			if (previous == SlotState.ALLOCATED && current == SlotState.FREE) {
+				taskExecutorManager.freeSlot(taskManagerSlot.getInstanceId());
+			}
+		};
+	}
+
+	private void cancelAllocationFuture(SlotID slotId) {
+		final CompletableFuture<Acknowledge> acknowledgeCompletableFuture = pendingSlotAllocationFutures.remove(slotId);
+		// the future may be null if we are just re-playing the state transitions due to a slot report
+		if (acknowledgeCompletableFuture != null) {
+			acknowledgeCompletableFuture.cancel(false);
+		}
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Component lifecycle methods
+	// ---------------------------------------------------------------------------------------------
+
+	/**
+	 * Starts the slot manager with the given leader id and resource manager actions.
+	 *
+	 * @param newResourceManagerId to use for communication with the task managers
+	 * @param newMainThreadExecutor to use to run code in the ResourceManager's main thread
+	 * @param newResourceActions to use for resource (de-)allocations
+	 */
+	@Override
+	public void start(ResourceManagerId newResourceManagerId, Executor newMainThreadExecutor, ResourceActions newResourceActions) {
+		LOG.info("Starting the slot manager.");
+
+		this.resourceManagerId = Preconditions.checkNotNull(newResourceManagerId);
+		mainThreadExecutor = Preconditions.checkNotNull(newMainThreadExecutor);
+		resourceActions = Preconditions.checkNotNull(newResourceActions);
+		taskExecutorManager = taskExecutorManagerFactory.apply(newMainThreadExecutor, newResourceActions);
+
+		started = true;
+
+		registerSlotManagerMetrics();
+	}
+
+	private void registerSlotManagerMetrics() {
+		slotManagerMetricGroup.gauge(
+			MetricNames.TASK_SLOTS_AVAILABLE,
+			() -> (long) getNumberFreeSlots());
+		slotManagerMetricGroup.gauge(
+			MetricNames.TASK_SLOTS_TOTAL,
+			() -> (long) getNumberRegisteredSlots());
+	}
+
+	/**
+	 * Suspends the component. This clears the internal state of the slot manager.
+	 */
+	@Override
+	public void suspend() {
+		LOG.info("Suspending the slot manager.");
+
+		if (slotRequestTimeoutCheck != null) {
+			slotRequestTimeoutCheck.cancel(false);
+			slotRequestTimeoutCheck = null;
+		}
+
+		resourceTracker.clear();
+		taskExecutorManager.close();
+
+		for (InstanceID registeredTaskManager : taskExecutorManager.getTaskExecutors()) {
+			unregisterTaskManager(registeredTaskManager, new SlotManagerException("The slot manager is being suspended."));
+		}
+
+		taskExecutorManager = null;
+		resourceManagerId = null;
+		resourceActions = null;
+		started = false;
+	}
+
+	/**
+	 * Closes the slot manager.
+	 *
+	 * @throws Exception if the close operation fails
+	 */
+	@Override
+	public void close() throws Exception {
+		LOG.info("Closing the slot manager.");
+
+		suspend();
+		slotManagerMetricGroup.close();
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Public API
+	// ---------------------------------------------------------------------------------------------
+
+	@Override
+	public void processResourceRequirements(ResourceRequirements resourceRequirements) {
+		checkInit();
+		LOG.debug("Received resource requirements from job {}: {}", resourceRequirements.getJobId(), resourceRequirements.getResourceRequirements());
+
+		if (resourceRequirements.getResourceRequirements().isEmpty()) {
+			jobMasterTargetAddresses.remove(resourceRequirements.getJobId());
+		} else {
+			jobMasterTargetAddresses.put(resourceRequirements.getJobId(), resourceRequirements.getTargetAddress());
+		}
+		resourceTracker.notifyResourceRequirements(resourceRequirements.getJobId(), resourceRequirements.getResourceRequirements());
+		checkResourceRequirements();
+	}
+
+	/**
+	 * 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
+	 * @return True if the task manager has not been registered before and is registered successfully; otherwise false
+	 */
+	@Override
+	public boolean registerTaskManager(final TaskExecutorConnection taskExecutorConnection, SlotReport initialSlotReport) {
+		checkInit();
+		LOG.debug("Registering task executor {} under {} at the slot manager.", taskExecutorConnection.getResourceID(), taskExecutorConnection.getInstanceID());
+
+		// we identify task managers by their instance id
+		if (taskExecutorManager.isTaskManagerRegistered(taskExecutorConnection.getInstanceID())) {
+			LOG.debug("Task executor {} was already registered.", taskExecutorConnection.getResourceID());
+			reportSlotStatus(taskExecutorConnection.getInstanceID(), initialSlotReport);
+			return false;
+		} else {
+			if (!taskExecutorManager.registerTaskManager(taskExecutorConnection, initialSlotReport)) {
+				LOG.debug("Task executor {} could not be registered.", taskExecutorConnection.getResourceID());
+				return false;
+			}
+
+			// register the new slots
+			for (SlotStatus slotStatus : initialSlotReport) {
+				slotTracker.addSlot(
+					slotStatus.getSlotID(),
+					slotStatus.getResourceProfile(),
+					taskExecutorConnection,
+					slotStatus.getJobID());
+			}
+
+			checkResourceRequirements();
+			return true;
+		}
+	}
+
+	@Override
+	public boolean unregisterTaskManager(InstanceID instanceId, Exception cause) {
+		checkInit();
+
+		LOG.debug("Unregistering task executor {} from the slot manager.", instanceId);
+
+		if (taskExecutorManager.isTaskManagerRegistered(instanceId)) {
+			slotTracker.removeSlots(taskExecutorManager.getSlotsOf(instanceId));
+			taskExecutorManager.unregisterTaskExecutor(instanceId);
+			checkResourceRequirements();
+
+			return true;
+		} else {
+			LOG.debug("There is no task executor registered with instance ID {}. Ignoring this message.", instanceId);
+
+			return false;
+		}
+	}
+
+	/**
+	 * Reports the current slot allocations for a task manager identified by the given instance id.
+	 *
+	 * @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
+	 */
+	@Override
+	public boolean reportSlotStatus(InstanceID instanceId, SlotReport slotReport) {
+		checkInit();
+
+		LOG.debug("Received slot report from instance {}: {}.", instanceId, slotReport);
+
+		if (taskExecutorManager.isTaskManagerRegistered(instanceId)) {
+			slotTracker.notifySlotStatus(slotReport);
+			checkResourceRequirements();
+			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
+	 */
+	@Override
+	public void freeSlot(SlotID slotId, AllocationID allocationId) {
+		checkInit();
+		LOG.debug("Freeing slot {}.", slotId);
+
+		slotTracker.notifyFree(slotId);
+		checkResourceRequirements();
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Requirement matching
+	// ---------------------------------------------------------------------------------------------
+
+	private void checkResourceRequirements() {
+		final Map<JobID, Collection<ResourceRequirement>> missingResources = resourceTracker.getMissingResources();
+		if (missingResources.isEmpty()) {
+			return;
+		}
+
+		final Map<JobID, ResourceCounter> outstandingRequirements = new LinkedHashMap<>();
+		for (Map.Entry<JobID, Collection<ResourceRequirement>> resourceRequirements : missingResources.entrySet()) {
+			JobID jobId = resourceRequirements.getKey();
+
+			for (ResourceRequirement resourceRequirement : resourceRequirements.getValue()) {
+				int numMissingSlots = internalRequestSlots(jobId, jobMasterTargetAddresses.get(jobId), resourceRequirement);
+				if (numMissingSlots > 0) {
+					outstandingRequirements
+						.computeIfAbsent(jobId, ignored -> new ResourceCounter())
+						.incrementCount(resourceRequirement.getResourceProfile(), numMissingSlots);
+				}
+			}

Review comment:
       I would suggest to factor this out into a separate method something like `requestsSlotsForJob` or so.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManager.java
##########
@@ -0,0 +1,609 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.metrics.MetricNames;
+import org.apache.flink.runtime.metrics.groups.SlotManagerMetricGroup;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerId;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.slots.ResourceCounter;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+import org.apache.flink.runtime.slots.ResourceRequirements;
+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.SlotOccupiedException;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ScheduledFuture;
+import java.util.function.BiFunction;
+import java.util.stream.Collectors;
+
+/**
+ * Implementation of {@link SlotManager} supporting declarative slot management.
+ */
+public class DeclarativeSlotManager implements SlotManager {
+	private static final Logger LOG = LoggerFactory.getLogger(DeclarativeSlotManager.class);
+
+	private final SlotTracker slotTracker;
+	private final ResourceTracker resourceTracker;
+	private final BiFunction<Executor, ResourceActions, TaskExecutorManager> taskExecutorManagerFactory;
+	private TaskExecutorManager taskExecutorManager;
+
+	/** Timeout for slot requests to the task manager. */
+	private final Time taskManagerRequestTimeout;
+	private ScheduledFuture<?> slotRequestTimeoutCheck;
+
+	private final SlotMatchingStrategy slotMatchingStrategy;
+
+	private final SlotManagerMetricGroup slotManagerMetricGroup;
+
+	private final Map<JobID, String> jobMasterTargetAddresses = new HashMap<>();
+	private final HashMap<SlotID, CompletableFuture<Acknowledge>> pendingSlotAllocationFutures;
+
+	/** ResourceManager's id. */
+	private ResourceManagerId resourceManagerId;
+
+	/** Executor for future callbacks which have to be "synchronized". */
+	private Executor mainThreadExecutor;
+
+	/** Callbacks for resource (de-)allocations. */
+	private ResourceActions resourceActions;
+
+	/** True iff the component has been started. */
+	private boolean started;
+
+	public DeclarativeSlotManager(
+			ScheduledExecutor scheduledExecutor,
+			SlotManagerConfiguration slotManagerConfiguration,
+			SlotManagerMetricGroup slotManagerMetricGroup,
+			ResourceTracker resourceTracker,
+			SlotTracker slotTracker) {
+
+		Preconditions.checkNotNull(slotManagerConfiguration);
+		this.taskManagerRequestTimeout = slotManagerConfiguration.getTaskManagerRequestTimeout();
+		this.slotManagerMetricGroup = Preconditions.checkNotNull(slotManagerMetricGroup);
+		this.resourceTracker = Preconditions.checkNotNull(resourceTracker);
+
+		pendingSlotAllocationFutures = new HashMap<>(16);
+
+		this.slotTracker = Preconditions.checkNotNull(slotTracker);
+		slotTracker.registerSlotStatusUpdateListener(createSlotStatusUpdateListener());
+
+		slotMatchingStrategy = slotManagerConfiguration.getSlotMatchingStrategy();
+
+		taskExecutorManagerFactory = (executor, resourceActions) -> new TaskExecutorManager(
+			slotManagerConfiguration.getDefaultWorkerResourceSpec(),
+			slotManagerConfiguration.getNumSlotsPerWorker(),
+			slotManagerConfiguration.getMaxSlotNum(),
+			slotManagerConfiguration.isWaitResultConsumedBeforeRelease(),
+			slotManagerConfiguration.getRedundantTaskManagerNum(),
+			slotManagerConfiguration.getTaskManagerTimeout(),
+			scheduledExecutor,
+			executor,
+			resourceActions);
+
+		resourceManagerId = null;
+		resourceActions = null;
+		mainThreadExecutor = null;
+		slotRequestTimeoutCheck = null;
+		taskExecutorManager = null;
+
+		started = false;
+	}
+
+	private SlotStatusUpdateListener createSlotStatusUpdateListener() {
+		return (taskManagerSlot, previous, current, jobId) -> {
+			if (previous == SlotState.PENDING) {
+				cancelAllocationFuture(taskManagerSlot.getSlotId());
+			}
+
+			if (current == SlotState.PENDING) {
+				resourceTracker.notifyAcquiredResource(jobId, taskManagerSlot.getResourceProfile());
+			}
+			if (current == SlotState.FREE) {
+				resourceTracker.notifyLostResource(jobId, taskManagerSlot.getResourceProfile());
+			}
+
+			if (current == SlotState.ALLOCATED) {
+				taskExecutorManager.occupySlot(taskManagerSlot.getInstanceId());
+			}
+			if (previous == SlotState.ALLOCATED && current == SlotState.FREE) {
+				taskExecutorManager.freeSlot(taskManagerSlot.getInstanceId());
+			}
+		};
+	}
+
+	private void cancelAllocationFuture(SlotID slotId) {
+		final CompletableFuture<Acknowledge> acknowledgeCompletableFuture = pendingSlotAllocationFutures.remove(slotId);

Review comment:
       I think `pendingSlotAllocationFutures` is never populated with any futures. What's the idea behind this structure?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManager.java
##########
@@ -0,0 +1,609 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.metrics.MetricNames;
+import org.apache.flink.runtime.metrics.groups.SlotManagerMetricGroup;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerId;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.slots.ResourceCounter;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+import org.apache.flink.runtime.slots.ResourceRequirements;
+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.SlotOccupiedException;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ScheduledFuture;
+import java.util.function.BiFunction;
+import java.util.stream.Collectors;
+
+/**
+ * Implementation of {@link SlotManager} supporting declarative slot management.
+ */
+public class DeclarativeSlotManager implements SlotManager {
+	private static final Logger LOG = LoggerFactory.getLogger(DeclarativeSlotManager.class);
+
+	private final SlotTracker slotTracker;
+	private final ResourceTracker resourceTracker;
+	private final BiFunction<Executor, ResourceActions, TaskExecutorManager> taskExecutorManagerFactory;
+	private TaskExecutorManager taskExecutorManager;
+
+	/** Timeout for slot requests to the task manager. */
+	private final Time taskManagerRequestTimeout;
+	private ScheduledFuture<?> slotRequestTimeoutCheck;
+
+	private final SlotMatchingStrategy slotMatchingStrategy;
+
+	private final SlotManagerMetricGroup slotManagerMetricGroup;
+
+	private final Map<JobID, String> jobMasterTargetAddresses = new HashMap<>();
+	private final HashMap<SlotID, CompletableFuture<Acknowledge>> pendingSlotAllocationFutures;
+
+	/** ResourceManager's id. */
+	private ResourceManagerId resourceManagerId;
+
+	/** Executor for future callbacks which have to be "synchronized". */
+	private Executor mainThreadExecutor;
+
+	/** Callbacks for resource (de-)allocations. */
+	private ResourceActions resourceActions;
+
+	/** True iff the component has been started. */
+	private boolean started;
+
+	public DeclarativeSlotManager(
+			ScheduledExecutor scheduledExecutor,
+			SlotManagerConfiguration slotManagerConfiguration,
+			SlotManagerMetricGroup slotManagerMetricGroup,
+			ResourceTracker resourceTracker,
+			SlotTracker slotTracker) {
+
+		Preconditions.checkNotNull(slotManagerConfiguration);
+		this.taskManagerRequestTimeout = slotManagerConfiguration.getTaskManagerRequestTimeout();
+		this.slotManagerMetricGroup = Preconditions.checkNotNull(slotManagerMetricGroup);
+		this.resourceTracker = Preconditions.checkNotNull(resourceTracker);
+
+		pendingSlotAllocationFutures = new HashMap<>(16);
+
+		this.slotTracker = Preconditions.checkNotNull(slotTracker);
+		slotTracker.registerSlotStatusUpdateListener(createSlotStatusUpdateListener());
+
+		slotMatchingStrategy = slotManagerConfiguration.getSlotMatchingStrategy();
+
+		taskExecutorManagerFactory = (executor, resourceActions) -> new TaskExecutorManager(
+			slotManagerConfiguration.getDefaultWorkerResourceSpec(),
+			slotManagerConfiguration.getNumSlotsPerWorker(),
+			slotManagerConfiguration.getMaxSlotNum(),
+			slotManagerConfiguration.isWaitResultConsumedBeforeRelease(),
+			slotManagerConfiguration.getRedundantTaskManagerNum(),
+			slotManagerConfiguration.getTaskManagerTimeout(),
+			scheduledExecutor,
+			executor,
+			resourceActions);
+
+		resourceManagerId = null;
+		resourceActions = null;
+		mainThreadExecutor = null;
+		slotRequestTimeoutCheck = null;
+		taskExecutorManager = null;
+
+		started = false;
+	}
+
+	private SlotStatusUpdateListener createSlotStatusUpdateListener() {
+		return (taskManagerSlot, previous, current, jobId) -> {
+			if (previous == SlotState.PENDING) {
+				cancelAllocationFuture(taskManagerSlot.getSlotId());
+			}
+
+			if (current == SlotState.PENDING) {
+				resourceTracker.notifyAcquiredResource(jobId, taskManagerSlot.getResourceProfile());
+			}
+			if (current == SlotState.FREE) {
+				resourceTracker.notifyLostResource(jobId, taskManagerSlot.getResourceProfile());
+			}
+
+			if (current == SlotState.ALLOCATED) {
+				taskExecutorManager.occupySlot(taskManagerSlot.getInstanceId());
+			}
+			if (previous == SlotState.ALLOCATED && current == SlotState.FREE) {
+				taskExecutorManager.freeSlot(taskManagerSlot.getInstanceId());
+			}
+		};
+	}
+
+	private void cancelAllocationFuture(SlotID slotId) {
+		final CompletableFuture<Acknowledge> acknowledgeCompletableFuture = pendingSlotAllocationFutures.remove(slotId);
+		// the future may be null if we are just re-playing the state transitions due to a slot report
+		if (acknowledgeCompletableFuture != null) {
+			acknowledgeCompletableFuture.cancel(false);
+		}
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Component lifecycle methods
+	// ---------------------------------------------------------------------------------------------
+
+	/**
+	 * Starts the slot manager with the given leader id and resource manager actions.
+	 *
+	 * @param newResourceManagerId to use for communication with the task managers
+	 * @param newMainThreadExecutor to use to run code in the ResourceManager's main thread
+	 * @param newResourceActions to use for resource (de-)allocations
+	 */
+	@Override
+	public void start(ResourceManagerId newResourceManagerId, Executor newMainThreadExecutor, ResourceActions newResourceActions) {
+		LOG.info("Starting the slot manager.");
+
+		this.resourceManagerId = Preconditions.checkNotNull(newResourceManagerId);
+		mainThreadExecutor = Preconditions.checkNotNull(newMainThreadExecutor);
+		resourceActions = Preconditions.checkNotNull(newResourceActions);
+		taskExecutorManager = taskExecutorManagerFactory.apply(newMainThreadExecutor, newResourceActions);
+
+		started = true;
+
+		registerSlotManagerMetrics();
+	}
+
+	private void registerSlotManagerMetrics() {
+		slotManagerMetricGroup.gauge(
+			MetricNames.TASK_SLOTS_AVAILABLE,
+			() -> (long) getNumberFreeSlots());
+		slotManagerMetricGroup.gauge(
+			MetricNames.TASK_SLOTS_TOTAL,
+			() -> (long) getNumberRegisteredSlots());
+	}
+
+	/**
+	 * Suspends the component. This clears the internal state of the slot manager.
+	 */
+	@Override
+	public void suspend() {
+		LOG.info("Suspending the slot manager.");
+
+		if (slotRequestTimeoutCheck != null) {
+			slotRequestTimeoutCheck.cancel(false);
+			slotRequestTimeoutCheck = null;
+		}
+
+		resourceTracker.clear();
+		taskExecutorManager.close();
+
+		for (InstanceID registeredTaskManager : taskExecutorManager.getTaskExecutors()) {
+			unregisterTaskManager(registeredTaskManager, new SlotManagerException("The slot manager is being suspended."));
+		}
+
+		taskExecutorManager = null;
+		resourceManagerId = null;
+		resourceActions = null;
+		started = false;
+	}
+
+	/**
+	 * Closes the slot manager.
+	 *
+	 * @throws Exception if the close operation fails
+	 */
+	@Override
+	public void close() throws Exception {
+		LOG.info("Closing the slot manager.");
+
+		suspend();
+		slotManagerMetricGroup.close();
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Public API
+	// ---------------------------------------------------------------------------------------------
+
+	@Override
+	public void processResourceRequirements(ResourceRequirements resourceRequirements) {
+		checkInit();
+		LOG.debug("Received resource requirements from job {}: {}", resourceRequirements.getJobId(), resourceRequirements.getResourceRequirements());
+
+		if (resourceRequirements.getResourceRequirements().isEmpty()) {
+			jobMasterTargetAddresses.remove(resourceRequirements.getJobId());
+		} else {
+			jobMasterTargetAddresses.put(resourceRequirements.getJobId(), resourceRequirements.getTargetAddress());
+		}
+		resourceTracker.notifyResourceRequirements(resourceRequirements.getJobId(), resourceRequirements.getResourceRequirements());
+		checkResourceRequirements();
+	}
+
+	/**
+	 * 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
+	 * @return True if the task manager has not been registered before and is registered successfully; otherwise false
+	 */
+	@Override
+	public boolean registerTaskManager(final TaskExecutorConnection taskExecutorConnection, SlotReport initialSlotReport) {
+		checkInit();
+		LOG.debug("Registering task executor {} under {} at the slot manager.", taskExecutorConnection.getResourceID(), taskExecutorConnection.getInstanceID());
+
+		// we identify task managers by their instance id
+		if (taskExecutorManager.isTaskManagerRegistered(taskExecutorConnection.getInstanceID())) {
+			LOG.debug("Task executor {} was already registered.", taskExecutorConnection.getResourceID());
+			reportSlotStatus(taskExecutorConnection.getInstanceID(), initialSlotReport);
+			return false;
+		} else {
+			if (!taskExecutorManager.registerTaskManager(taskExecutorConnection, initialSlotReport)) {
+				LOG.debug("Task executor {} could not be registered.", taskExecutorConnection.getResourceID());
+				return false;
+			}
+
+			// register the new slots
+			for (SlotStatus slotStatus : initialSlotReport) {
+				slotTracker.addSlot(
+					slotStatus.getSlotID(),
+					slotStatus.getResourceProfile(),
+					taskExecutorConnection,
+					slotStatus.getJobID());
+			}
+
+			checkResourceRequirements();
+			return true;
+		}
+	}
+
+	@Override
+	public boolean unregisterTaskManager(InstanceID instanceId, Exception cause) {
+		checkInit();
+
+		LOG.debug("Unregistering task executor {} from the slot manager.", instanceId);
+
+		if (taskExecutorManager.isTaskManagerRegistered(instanceId)) {
+			slotTracker.removeSlots(taskExecutorManager.getSlotsOf(instanceId));
+			taskExecutorManager.unregisterTaskExecutor(instanceId);
+			checkResourceRequirements();
+
+			return true;
+		} else {
+			LOG.debug("There is no task executor registered with instance ID {}. Ignoring this message.", instanceId);
+
+			return false;
+		}
+	}
+
+	/**
+	 * Reports the current slot allocations for a task manager identified by the given instance id.
+	 *
+	 * @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
+	 */
+	@Override
+	public boolean reportSlotStatus(InstanceID instanceId, SlotReport slotReport) {
+		checkInit();
+
+		LOG.debug("Received slot report from instance {}: {}.", instanceId, slotReport);
+
+		if (taskExecutorManager.isTaskManagerRegistered(instanceId)) {
+			slotTracker.notifySlotStatus(slotReport);
+			checkResourceRequirements();
+			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
+	 */
+	@Override
+	public void freeSlot(SlotID slotId, AllocationID allocationId) {
+		checkInit();
+		LOG.debug("Freeing slot {}.", slotId);
+
+		slotTracker.notifyFree(slotId);
+		checkResourceRequirements();
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Requirement matching
+	// ---------------------------------------------------------------------------------------------
+
+	private void checkResourceRequirements() {
+		final Map<JobID, Collection<ResourceRequirement>> missingResources = resourceTracker.getMissingResources();
+		if (missingResources.isEmpty()) {
+			return;
+		}
+
+		final Map<JobID, ResourceCounter> outstandingRequirements = new LinkedHashMap<>();
+		for (Map.Entry<JobID, Collection<ResourceRequirement>> resourceRequirements : missingResources.entrySet()) {
+			JobID jobId = resourceRequirements.getKey();
+
+			for (ResourceRequirement resourceRequirement : resourceRequirements.getValue()) {
+				int numMissingSlots = internalRequestSlots(jobId, jobMasterTargetAddresses.get(jobId), resourceRequirement);
+				if (numMissingSlots > 0) {
+					outstandingRequirements
+						.computeIfAbsent(jobId, ignored -> new ResourceCounter())
+						.incrementCount(resourceRequirement.getResourceProfile(), numMissingSlots);
+				}
+			}
+		}
+
+		final ResourceCounter pendingSlots = new ResourceCounter(taskExecutorManager.getPendingTaskManagerSlots().stream().collect(
+			Collectors.groupingBy(
+				PendingTaskManagerSlot::getResourceProfile,
+				Collectors.summingInt(x -> 1))));
+
+		for (Map.Entry<JobID, ResourceCounter> unfulfilledRequirement : outstandingRequirements.entrySet()) {
+			tryFulfillRequirementsWithPendingOrNewSlots(
+				unfulfilledRequirement.getKey(),
+				unfulfilledRequirement.getValue().getResourceProfilesWithCount(),
+				pendingSlots);
+		}
+	}
+
+	/**
+	 * Tries to allocate slots for the given requirement. If there are not enough slots available, the
+	 * resource manager is informed to allocate more resources.
+	 *
+	 * @param jobId job to allocate slots for
+	 * @param targetAddress address of the jobmaster
+	 * @param resourceRequirement required slots
+	 * @return the number of missing slots
+	 */
+	private int internalRequestSlots(JobID jobId, String targetAddress, ResourceRequirement resourceRequirement) {
+		final ResourceProfile requiredResource = resourceRequirement.getResourceProfile();
+		Collection<TaskManagerSlotInformation> freeSlots = slotTracker.getFreeSlots();
+
+		int numUnfulfilled = 0;
+		for (int x = 0; x < resourceRequirement.getNumberOfRequiredSlots(); x++) {
+
+			final Optional<TaskManagerSlotInformation> reservedSlot = slotMatchingStrategy.findMatchingSlot(requiredResource, freeSlots, this::getNumberRegisteredSlotsOf);
+			if (reservedSlot.isPresent()) {
+				// we do not need to modify freeSlots because it is indirectly modified by the allocation
+				allocateSlot(reservedSlot.get(), jobId, targetAddress, requiredResource);
+			} else {
+				numUnfulfilled++;
+			}
+		}
+		return numUnfulfilled;
+	}
+
+	/**
+	 * Allocates the given slot. This entails sending a registration message to the task manager and treating failures.
+	 *
+	 * @param taskManagerSlot slot to allocate
+	 * @param jobId job for which the slot should be allocated for
+	 * @param targetAddress address of the job master
+	 * @param resourceProfile resource profile for the requirement for which the slot is used
+	 */
+	private void allocateSlot(TaskManagerSlotInformation taskManagerSlot, JobID jobId, String targetAddress, ResourceProfile resourceProfile) {
+		TaskExecutorConnection taskExecutorConnection = taskManagerSlot.getTaskManagerConnection();
+		TaskExecutorGateway gateway = taskExecutorConnection.getTaskExecutorGateway();
+
+		final CompletableFuture<Acknowledge> completableFuture = new CompletableFuture<>();
+		final SlotID slotId = taskManagerSlot.getSlotId();
+		final InstanceID instanceId = taskManagerSlot.getInstanceId();
+
+		LOG.debug("Starting allocation of slot {} for job {} with resource profile {}.", taskManagerSlot.getSlotId(), jobId, resourceProfile);
+
+		slotTracker.notifyAllocationStart(slotId, jobId);
+
+		if (!taskExecutorManager.isTaskManagerRegistered(instanceId)) {
+			throw new IllegalStateException("Could not find a registered task manager for instance id " +
+				instanceId + '.');
+		}
+
+		taskExecutorManager.markUsed(instanceId);
+
+		// RPC call to the task manager
+		CompletableFuture<Acknowledge> requestFuture = gateway.requestSlot(
+			slotId,
+			jobId,
+			new AllocationID(),
+			resourceProfile,
+			targetAddress,
+			resourceManagerId,
+			taskManagerRequestTimeout);
+
+		requestFuture.whenComplete(
+			(Acknowledge acknowledge, Throwable throwable) -> {
+				if (acknowledge != null) {
+					completableFuture.complete(acknowledge);
+				} else {
+					completableFuture.completeExceptionally(throwable);
+				}
+			});
+
+		completableFuture.whenCompleteAsync(
+			(Acknowledge acknowledge, Throwable throwable) -> {
+				try {
+					if (acknowledge != null) {
+						LOG.trace("Completed allocation of slot {} for job {}.", slotId, jobId);
+						slotTracker.notifyAllocationComplete(slotId, jobId);
+					} else {
+						if (throwable instanceof SlotOccupiedException) {
+							SlotOccupiedException exception = (SlotOccupiedException) throwable;
+							LOG.debug("Tried allocating slot {} for job {}, but it was already allocated for job {}.", slotId, jobId, exception.getJobId());
+							// report as a slot status to force the state transition
+							// this could be a problem if we ever assume that the task executor always reports about all slots
+							slotTracker.notifySlotStatus(Collections.singleton(new SlotStatus(slotId, taskManagerSlot.getResourceProfile(), exception.getJobId(), exception.getAllocationId())));
+						} else {
+							if (throwable instanceof CancellationException) {
+								LOG.debug("Cancelled allocation of slot {} for job {}.", slotId, jobId, throwable);
+							} else {
+								LOG.warn("Slot allocation for slot {} for job {} failed.", slotId, jobId, throwable);
+								slotTracker.notifyFree(slotId);
+							}
+						}
+					}
+					checkResourceRequirements();
+				} catch (Exception e) {
+					LOG.error("Error while completing the slot allocation.", e);
+				}
+			},
+			mainThreadExecutor);
+	}
+
+	private void tryFulfillRequirementsWithPendingOrNewSlots(JobID jobId, Map<ResourceProfile, Integer> missingResources, ResourceCounter pendingSlots) {

Review comment:
       Aren't we calling this method always with pending slots and no new slots?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java
##########
@@ -861,6 +884,8 @@ protected void closeJobManagerConnection(JobID jobId, Exception cause) {
 
 			jmResourceIdRegistrations.remove(jobManagerResourceId);
 
+			slotManager.processResourceRequirements(ResourceRequirements.empty(jobId, jobMasterGateway.getAddress()));

Review comment:
       On the other hand a job will only be removed after `resourcemanager.job.timeout` which defaults to 5 minutes. Then I guess it is fine that we don't try to actively allocated new resources for a job whose job manager has died/timed out.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManager.java
##########
@@ -0,0 +1,609 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.metrics.MetricNames;
+import org.apache.flink.runtime.metrics.groups.SlotManagerMetricGroup;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerId;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.slots.ResourceCounter;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+import org.apache.flink.runtime.slots.ResourceRequirements;
+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.SlotOccupiedException;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ScheduledFuture;
+import java.util.function.BiFunction;
+import java.util.stream.Collectors;
+
+/**
+ * Implementation of {@link SlotManager} supporting declarative slot management.
+ */
+public class DeclarativeSlotManager implements SlotManager {
+	private static final Logger LOG = LoggerFactory.getLogger(DeclarativeSlotManager.class);
+
+	private final SlotTracker slotTracker;
+	private final ResourceTracker resourceTracker;
+	private final BiFunction<Executor, ResourceActions, TaskExecutorManager> taskExecutorManagerFactory;
+	private TaskExecutorManager taskExecutorManager;
+
+	/** Timeout for slot requests to the task manager. */
+	private final Time taskManagerRequestTimeout;
+	private ScheduledFuture<?> slotRequestTimeoutCheck;
+
+	private final SlotMatchingStrategy slotMatchingStrategy;
+
+	private final SlotManagerMetricGroup slotManagerMetricGroup;
+
+	private final Map<JobID, String> jobMasterTargetAddresses = new HashMap<>();
+	private final HashMap<SlotID, CompletableFuture<Acknowledge>> pendingSlotAllocationFutures;
+
+	/** ResourceManager's id. */
+	private ResourceManagerId resourceManagerId;
+
+	/** Executor for future callbacks which have to be "synchronized". */
+	private Executor mainThreadExecutor;
+
+	/** Callbacks for resource (de-)allocations. */
+	private ResourceActions resourceActions;
+
+	/** True iff the component has been started. */
+	private boolean started;
+
+	public DeclarativeSlotManager(
+			ScheduledExecutor scheduledExecutor,
+			SlotManagerConfiguration slotManagerConfiguration,
+			SlotManagerMetricGroup slotManagerMetricGroup,
+			ResourceTracker resourceTracker,
+			SlotTracker slotTracker) {
+
+		Preconditions.checkNotNull(slotManagerConfiguration);
+		this.taskManagerRequestTimeout = slotManagerConfiguration.getTaskManagerRequestTimeout();
+		this.slotManagerMetricGroup = Preconditions.checkNotNull(slotManagerMetricGroup);
+		this.resourceTracker = Preconditions.checkNotNull(resourceTracker);
+
+		pendingSlotAllocationFutures = new HashMap<>(16);
+
+		this.slotTracker = Preconditions.checkNotNull(slotTracker);
+		slotTracker.registerSlotStatusUpdateListener(createSlotStatusUpdateListener());
+
+		slotMatchingStrategy = slotManagerConfiguration.getSlotMatchingStrategy();
+
+		taskExecutorManagerFactory = (executor, resourceActions) -> new TaskExecutorManager(
+			slotManagerConfiguration.getDefaultWorkerResourceSpec(),
+			slotManagerConfiguration.getNumSlotsPerWorker(),
+			slotManagerConfiguration.getMaxSlotNum(),
+			slotManagerConfiguration.isWaitResultConsumedBeforeRelease(),
+			slotManagerConfiguration.getRedundantTaskManagerNum(),
+			slotManagerConfiguration.getTaskManagerTimeout(),
+			scheduledExecutor,
+			executor,
+			resourceActions);
+
+		resourceManagerId = null;
+		resourceActions = null;
+		mainThreadExecutor = null;
+		slotRequestTimeoutCheck = null;
+		taskExecutorManager = null;
+
+		started = false;
+	}
+
+	private SlotStatusUpdateListener createSlotStatusUpdateListener() {
+		return (taskManagerSlot, previous, current, jobId) -> {
+			if (previous == SlotState.PENDING) {
+				cancelAllocationFuture(taskManagerSlot.getSlotId());
+			}
+
+			if (current == SlotState.PENDING) {
+				resourceTracker.notifyAcquiredResource(jobId, taskManagerSlot.getResourceProfile());
+			}
+			if (current == SlotState.FREE) {
+				resourceTracker.notifyLostResource(jobId, taskManagerSlot.getResourceProfile());
+			}
+
+			if (current == SlotState.ALLOCATED) {
+				taskExecutorManager.occupySlot(taskManagerSlot.getInstanceId());
+			}
+			if (previous == SlotState.ALLOCATED && current == SlotState.FREE) {
+				taskExecutorManager.freeSlot(taskManagerSlot.getInstanceId());
+			}
+		};
+	}
+
+	private void cancelAllocationFuture(SlotID slotId) {
+		final CompletableFuture<Acknowledge> acknowledgeCompletableFuture = pendingSlotAllocationFutures.remove(slotId);
+		// the future may be null if we are just re-playing the state transitions due to a slot report
+		if (acknowledgeCompletableFuture != null) {
+			acknowledgeCompletableFuture.cancel(false);
+		}
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Component lifecycle methods
+	// ---------------------------------------------------------------------------------------------
+
+	/**
+	 * Starts the slot manager with the given leader id and resource manager actions.
+	 *
+	 * @param newResourceManagerId to use for communication with the task managers
+	 * @param newMainThreadExecutor to use to run code in the ResourceManager's main thread
+	 * @param newResourceActions to use for resource (de-)allocations
+	 */
+	@Override
+	public void start(ResourceManagerId newResourceManagerId, Executor newMainThreadExecutor, ResourceActions newResourceActions) {
+		LOG.info("Starting the slot manager.");
+
+		this.resourceManagerId = Preconditions.checkNotNull(newResourceManagerId);
+		mainThreadExecutor = Preconditions.checkNotNull(newMainThreadExecutor);
+		resourceActions = Preconditions.checkNotNull(newResourceActions);
+		taskExecutorManager = taskExecutorManagerFactory.apply(newMainThreadExecutor, newResourceActions);
+
+		started = true;
+
+		registerSlotManagerMetrics();
+	}
+
+	private void registerSlotManagerMetrics() {
+		slotManagerMetricGroup.gauge(
+			MetricNames.TASK_SLOTS_AVAILABLE,
+			() -> (long) getNumberFreeSlots());
+		slotManagerMetricGroup.gauge(
+			MetricNames.TASK_SLOTS_TOTAL,
+			() -> (long) getNumberRegisteredSlots());
+	}
+
+	/**
+	 * Suspends the component. This clears the internal state of the slot manager.
+	 */
+	@Override
+	public void suspend() {
+		LOG.info("Suspending the slot manager.");
+
+		if (slotRequestTimeoutCheck != null) {
+			slotRequestTimeoutCheck.cancel(false);
+			slotRequestTimeoutCheck = null;
+		}
+
+		resourceTracker.clear();
+		taskExecutorManager.close();
+
+		for (InstanceID registeredTaskManager : taskExecutorManager.getTaskExecutors()) {
+			unregisterTaskManager(registeredTaskManager, new SlotManagerException("The slot manager is being suspended."));
+		}
+
+		taskExecutorManager = null;
+		resourceManagerId = null;
+		resourceActions = null;
+		started = false;
+	}
+
+	/**
+	 * Closes the slot manager.
+	 *
+	 * @throws Exception if the close operation fails
+	 */
+	@Override
+	public void close() throws Exception {
+		LOG.info("Closing the slot manager.");
+
+		suspend();
+		slotManagerMetricGroup.close();
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Public API
+	// ---------------------------------------------------------------------------------------------
+
+	@Override
+	public void processResourceRequirements(ResourceRequirements resourceRequirements) {
+		checkInit();
+		LOG.debug("Received resource requirements from job {}: {}", resourceRequirements.getJobId(), resourceRequirements.getResourceRequirements());
+
+		if (resourceRequirements.getResourceRequirements().isEmpty()) {
+			jobMasterTargetAddresses.remove(resourceRequirements.getJobId());
+		} else {
+			jobMasterTargetAddresses.put(resourceRequirements.getJobId(), resourceRequirements.getTargetAddress());
+		}
+		resourceTracker.notifyResourceRequirements(resourceRequirements.getJobId(), resourceRequirements.getResourceRequirements());
+		checkResourceRequirements();
+	}
+
+	/**
+	 * 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
+	 * @return True if the task manager has not been registered before and is registered successfully; otherwise false
+	 */
+	@Override
+	public boolean registerTaskManager(final TaskExecutorConnection taskExecutorConnection, SlotReport initialSlotReport) {
+		checkInit();
+		LOG.debug("Registering task executor {} under {} at the slot manager.", taskExecutorConnection.getResourceID(), taskExecutorConnection.getInstanceID());
+
+		// we identify task managers by their instance id
+		if (taskExecutorManager.isTaskManagerRegistered(taskExecutorConnection.getInstanceID())) {
+			LOG.debug("Task executor {} was already registered.", taskExecutorConnection.getResourceID());
+			reportSlotStatus(taskExecutorConnection.getInstanceID(), initialSlotReport);
+			return false;
+		} else {
+			if (!taskExecutorManager.registerTaskManager(taskExecutorConnection, initialSlotReport)) {
+				LOG.debug("Task executor {} could not be registered.", taskExecutorConnection.getResourceID());
+				return false;
+			}
+
+			// register the new slots
+			for (SlotStatus slotStatus : initialSlotReport) {
+				slotTracker.addSlot(
+					slotStatus.getSlotID(),
+					slotStatus.getResourceProfile(),
+					taskExecutorConnection,
+					slotStatus.getJobID());
+			}
+
+			checkResourceRequirements();
+			return true;
+		}
+	}
+
+	@Override
+	public boolean unregisterTaskManager(InstanceID instanceId, Exception cause) {
+		checkInit();
+
+		LOG.debug("Unregistering task executor {} from the slot manager.", instanceId);
+
+		if (taskExecutorManager.isTaskManagerRegistered(instanceId)) {
+			slotTracker.removeSlots(taskExecutorManager.getSlotsOf(instanceId));
+			taskExecutorManager.unregisterTaskExecutor(instanceId);
+			checkResourceRequirements();
+
+			return true;
+		} else {
+			LOG.debug("There is no task executor registered with instance ID {}. Ignoring this message.", instanceId);
+
+			return false;
+		}
+	}
+
+	/**
+	 * Reports the current slot allocations for a task manager identified by the given instance id.
+	 *
+	 * @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
+	 */
+	@Override
+	public boolean reportSlotStatus(InstanceID instanceId, SlotReport slotReport) {
+		checkInit();
+
+		LOG.debug("Received slot report from instance {}: {}.", instanceId, slotReport);
+
+		if (taskExecutorManager.isTaskManagerRegistered(instanceId)) {
+			slotTracker.notifySlotStatus(slotReport);
+			checkResourceRequirements();
+			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
+	 */
+	@Override
+	public void freeSlot(SlotID slotId, AllocationID allocationId) {
+		checkInit();
+		LOG.debug("Freeing slot {}.", slotId);
+
+		slotTracker.notifyFree(slotId);
+		checkResourceRequirements();
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Requirement matching
+	// ---------------------------------------------------------------------------------------------
+
+	private void checkResourceRequirements() {
+		final Map<JobID, Collection<ResourceRequirement>> missingResources = resourceTracker.getMissingResources();
+		if (missingResources.isEmpty()) {
+			return;
+		}
+
+		final Map<JobID, ResourceCounter> outstandingRequirements = new LinkedHashMap<>();
+		for (Map.Entry<JobID, Collection<ResourceRequirement>> resourceRequirements : missingResources.entrySet()) {
+			JobID jobId = resourceRequirements.getKey();
+
+			for (ResourceRequirement resourceRequirement : resourceRequirements.getValue()) {
+				int numMissingSlots = internalRequestSlots(jobId, jobMasterTargetAddresses.get(jobId), resourceRequirement);
+				if (numMissingSlots > 0) {
+					outstandingRequirements
+						.computeIfAbsent(jobId, ignored -> new ResourceCounter())
+						.incrementCount(resourceRequirement.getResourceProfile(), numMissingSlots);
+				}
+			}
+		}
+
+		final ResourceCounter pendingSlots = new ResourceCounter(taskExecutorManager.getPendingTaskManagerSlots().stream().collect(
+			Collectors.groupingBy(
+				PendingTaskManagerSlot::getResourceProfile,
+				Collectors.summingInt(x -> 1))));
+
+		for (Map.Entry<JobID, ResourceCounter> unfulfilledRequirement : outstandingRequirements.entrySet()) {
+			tryFulfillRequirementsWithPendingOrNewSlots(
+				unfulfilledRequirement.getKey(),
+				unfulfilledRequirement.getValue().getResourceProfilesWithCount(),
+				pendingSlots);
+		}
+	}
+
+	/**
+	 * Tries to allocate slots for the given requirement. If there are not enough slots available, the
+	 * resource manager is informed to allocate more resources.
+	 *
+	 * @param jobId job to allocate slots for
+	 * @param targetAddress address of the jobmaster
+	 * @param resourceRequirement required slots
+	 * @return the number of missing slots
+	 */
+	private int internalRequestSlots(JobID jobId, String targetAddress, ResourceRequirement resourceRequirement) {
+		final ResourceProfile requiredResource = resourceRequirement.getResourceProfile();
+		Collection<TaskManagerSlotInformation> freeSlots = slotTracker.getFreeSlots();
+
+		int numUnfulfilled = 0;
+		for (int x = 0; x < resourceRequirement.getNumberOfRequiredSlots(); x++) {
+
+			final Optional<TaskManagerSlotInformation> reservedSlot = slotMatchingStrategy.findMatchingSlot(requiredResource, freeSlots, this::getNumberRegisteredSlotsOf);
+			if (reservedSlot.isPresent()) {
+				// we do not need to modify freeSlots because it is indirectly modified by the allocation
+				allocateSlot(reservedSlot.get(), jobId, targetAddress, requiredResource);
+			} else {
+				numUnfulfilled++;
+			}
+		}
+		return numUnfulfilled;
+	}
+
+	/**
+	 * Allocates the given slot. This entails sending a registration message to the task manager and treating failures.
+	 *
+	 * @param taskManagerSlot slot to allocate
+	 * @param jobId job for which the slot should be allocated for
+	 * @param targetAddress address of the job master
+	 * @param resourceProfile resource profile for the requirement for which the slot is used
+	 */
+	private void allocateSlot(TaskManagerSlotInformation taskManagerSlot, JobID jobId, String targetAddress, ResourceProfile resourceProfile) {
+		TaskExecutorConnection taskExecutorConnection = taskManagerSlot.getTaskManagerConnection();
+		TaskExecutorGateway gateway = taskExecutorConnection.getTaskExecutorGateway();
+
+		final CompletableFuture<Acknowledge> completableFuture = new CompletableFuture<>();
+		final SlotID slotId = taskManagerSlot.getSlotId();
+		final InstanceID instanceId = taskManagerSlot.getInstanceId();
+
+		LOG.debug("Starting allocation of slot {} for job {} with resource profile {}.", taskManagerSlot.getSlotId(), jobId, resourceProfile);
+
+		slotTracker.notifyAllocationStart(slotId, jobId);
+
+		if (!taskExecutorManager.isTaskManagerRegistered(instanceId)) {
+			throw new IllegalStateException("Could not find a registered task manager for instance id " +
+				instanceId + '.');
+		}
+
+		taskExecutorManager.markUsed(instanceId);
+
+		// RPC call to the task manager
+		CompletableFuture<Acknowledge> requestFuture = gateway.requestSlot(
+			slotId,
+			jobId,
+			new AllocationID(),
+			resourceProfile,
+			targetAddress,
+			resourceManagerId,
+			taskManagerRequestTimeout);
+
+		requestFuture.whenComplete(
+			(Acknowledge acknowledge, Throwable throwable) -> {
+				if (acknowledge != null) {
+					completableFuture.complete(acknowledge);
+				} else {
+					completableFuture.completeExceptionally(throwable);
+				}
+			});
+
+		completableFuture.whenCompleteAsync(
+			(Acknowledge acknowledge, Throwable throwable) -> {
+				try {
+					if (acknowledge != null) {
+						LOG.trace("Completed allocation of slot {} for job {}.", slotId, jobId);
+						slotTracker.notifyAllocationComplete(slotId, jobId);
+					} else {
+						if (throwable instanceof SlotOccupiedException) {
+							SlotOccupiedException exception = (SlotOccupiedException) throwable;
+							LOG.debug("Tried allocating slot {} for job {}, but it was already allocated for job {}.", slotId, jobId, exception.getJobId());
+							// report as a slot status to force the state transition
+							// this could be a problem if we ever assume that the task executor always reports about all slots
+							slotTracker.notifySlotStatus(Collections.singleton(new SlotStatus(slotId, taskManagerSlot.getResourceProfile(), exception.getJobId(), exception.getAllocationId())));
+						} else {
+							if (throwable instanceof CancellationException) {
+								LOG.debug("Cancelled allocation of slot {} for job {}.", slotId, jobId, throwable);
+							} else {
+								LOG.warn("Slot allocation for slot {} for job {} failed.", slotId, jobId, throwable);
+								slotTracker.notifyFree(slotId);
+							}
+						}
+					}
+					checkResourceRequirements();
+				} catch (Exception e) {
+					LOG.error("Error while completing the slot allocation.", e);
+				}

Review comment:
       Where does the exception come from? If this exception is the result of an illegal state, then we should instead use handleAsync and wrap it with `FutureUtils.assertNoException` to fail hard in case of an illegal state.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManager.java
##########
@@ -0,0 +1,609 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.metrics.MetricNames;
+import org.apache.flink.runtime.metrics.groups.SlotManagerMetricGroup;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerId;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.slots.ResourceCounter;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+import org.apache.flink.runtime.slots.ResourceRequirements;
+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.SlotOccupiedException;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ScheduledFuture;
+import java.util.function.BiFunction;
+import java.util.stream.Collectors;
+
+/**
+ * Implementation of {@link SlotManager} supporting declarative slot management.
+ */
+public class DeclarativeSlotManager implements SlotManager {
+	private static final Logger LOG = LoggerFactory.getLogger(DeclarativeSlotManager.class);
+
+	private final SlotTracker slotTracker;
+	private final ResourceTracker resourceTracker;
+	private final BiFunction<Executor, ResourceActions, TaskExecutorManager> taskExecutorManagerFactory;
+	private TaskExecutorManager taskExecutorManager;
+
+	/** Timeout for slot requests to the task manager. */
+	private final Time taskManagerRequestTimeout;
+	private ScheduledFuture<?> slotRequestTimeoutCheck;

Review comment:
       For what do we need the `slotRequestTimeoutCheck`?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManager.java
##########
@@ -0,0 +1,609 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.metrics.MetricNames;
+import org.apache.flink.runtime.metrics.groups.SlotManagerMetricGroup;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerId;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.slots.ResourceCounter;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+import org.apache.flink.runtime.slots.ResourceRequirements;
+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.SlotOccupiedException;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ScheduledFuture;
+import java.util.function.BiFunction;
+import java.util.stream.Collectors;
+
+/**
+ * Implementation of {@link SlotManager} supporting declarative slot management.
+ */
+public class DeclarativeSlotManager implements SlotManager {
+	private static final Logger LOG = LoggerFactory.getLogger(DeclarativeSlotManager.class);
+
+	private final SlotTracker slotTracker;
+	private final ResourceTracker resourceTracker;
+	private final BiFunction<Executor, ResourceActions, TaskExecutorManager> taskExecutorManagerFactory;
+	private TaskExecutorManager taskExecutorManager;
+
+	/** Timeout for slot requests to the task manager. */
+	private final Time taskManagerRequestTimeout;
+	private ScheduledFuture<?> slotRequestTimeoutCheck;
+
+	private final SlotMatchingStrategy slotMatchingStrategy;
+
+	private final SlotManagerMetricGroup slotManagerMetricGroup;
+
+	private final Map<JobID, String> jobMasterTargetAddresses = new HashMap<>();
+	private final HashMap<SlotID, CompletableFuture<Acknowledge>> pendingSlotAllocationFutures;
+
+	/** ResourceManager's id. */
+	private ResourceManagerId resourceManagerId;
+
+	/** Executor for future callbacks which have to be "synchronized". */
+	private Executor mainThreadExecutor;
+
+	/** Callbacks for resource (de-)allocations. */
+	private ResourceActions resourceActions;
+
+	/** True iff the component has been started. */
+	private boolean started;
+
+	public DeclarativeSlotManager(
+			ScheduledExecutor scheduledExecutor,
+			SlotManagerConfiguration slotManagerConfiguration,
+			SlotManagerMetricGroup slotManagerMetricGroup,
+			ResourceTracker resourceTracker,
+			SlotTracker slotTracker) {
+
+		Preconditions.checkNotNull(slotManagerConfiguration);
+		this.taskManagerRequestTimeout = slotManagerConfiguration.getTaskManagerRequestTimeout();
+		this.slotManagerMetricGroup = Preconditions.checkNotNull(slotManagerMetricGroup);
+		this.resourceTracker = Preconditions.checkNotNull(resourceTracker);
+
+		pendingSlotAllocationFutures = new HashMap<>(16);
+
+		this.slotTracker = Preconditions.checkNotNull(slotTracker);
+		slotTracker.registerSlotStatusUpdateListener(createSlotStatusUpdateListener());
+
+		slotMatchingStrategy = slotManagerConfiguration.getSlotMatchingStrategy();
+
+		taskExecutorManagerFactory = (executor, resourceActions) -> new TaskExecutorManager(
+			slotManagerConfiguration.getDefaultWorkerResourceSpec(),
+			slotManagerConfiguration.getNumSlotsPerWorker(),
+			slotManagerConfiguration.getMaxSlotNum(),
+			slotManagerConfiguration.isWaitResultConsumedBeforeRelease(),
+			slotManagerConfiguration.getRedundantTaskManagerNum(),
+			slotManagerConfiguration.getTaskManagerTimeout(),
+			scheduledExecutor,
+			executor,
+			resourceActions);
+
+		resourceManagerId = null;
+		resourceActions = null;
+		mainThreadExecutor = null;
+		slotRequestTimeoutCheck = null;
+		taskExecutorManager = null;
+
+		started = false;
+	}
+
+	private SlotStatusUpdateListener createSlotStatusUpdateListener() {
+		return (taskManagerSlot, previous, current, jobId) -> {
+			if (previous == SlotState.PENDING) {
+				cancelAllocationFuture(taskManagerSlot.getSlotId());
+			}
+
+			if (current == SlotState.PENDING) {
+				resourceTracker.notifyAcquiredResource(jobId, taskManagerSlot.getResourceProfile());
+			}
+			if (current == SlotState.FREE) {
+				resourceTracker.notifyLostResource(jobId, taskManagerSlot.getResourceProfile());
+			}
+
+			if (current == SlotState.ALLOCATED) {
+				taskExecutorManager.occupySlot(taskManagerSlot.getInstanceId());
+			}
+			if (previous == SlotState.ALLOCATED && current == SlotState.FREE) {
+				taskExecutorManager.freeSlot(taskManagerSlot.getInstanceId());
+			}
+		};
+	}
+
+	private void cancelAllocationFuture(SlotID slotId) {
+		final CompletableFuture<Acknowledge> acknowledgeCompletableFuture = pendingSlotAllocationFutures.remove(slotId);
+		// the future may be null if we are just re-playing the state transitions due to a slot report
+		if (acknowledgeCompletableFuture != null) {
+			acknowledgeCompletableFuture.cancel(false);
+		}
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Component lifecycle methods
+	// ---------------------------------------------------------------------------------------------
+
+	/**
+	 * Starts the slot manager with the given leader id and resource manager actions.
+	 *
+	 * @param newResourceManagerId to use for communication with the task managers
+	 * @param newMainThreadExecutor to use to run code in the ResourceManager's main thread
+	 * @param newResourceActions to use for resource (de-)allocations
+	 */
+	@Override
+	public void start(ResourceManagerId newResourceManagerId, Executor newMainThreadExecutor, ResourceActions newResourceActions) {
+		LOG.info("Starting the slot manager.");
+
+		this.resourceManagerId = Preconditions.checkNotNull(newResourceManagerId);
+		mainThreadExecutor = Preconditions.checkNotNull(newMainThreadExecutor);
+		resourceActions = Preconditions.checkNotNull(newResourceActions);
+		taskExecutorManager = taskExecutorManagerFactory.apply(newMainThreadExecutor, newResourceActions);
+
+		started = true;
+
+		registerSlotManagerMetrics();
+	}
+
+	private void registerSlotManagerMetrics() {
+		slotManagerMetricGroup.gauge(
+			MetricNames.TASK_SLOTS_AVAILABLE,
+			() -> (long) getNumberFreeSlots());
+		slotManagerMetricGroup.gauge(
+			MetricNames.TASK_SLOTS_TOTAL,
+			() -> (long) getNumberRegisteredSlots());
+	}
+
+	/**
+	 * Suspends the component. This clears the internal state of the slot manager.
+	 */
+	@Override
+	public void suspend() {
+		LOG.info("Suspending the slot manager.");
+
+		if (slotRequestTimeoutCheck != null) {
+			slotRequestTimeoutCheck.cancel(false);
+			slotRequestTimeoutCheck = null;
+		}
+
+		resourceTracker.clear();
+		taskExecutorManager.close();
+
+		for (InstanceID registeredTaskManager : taskExecutorManager.getTaskExecutors()) {
+			unregisterTaskManager(registeredTaskManager, new SlotManagerException("The slot manager is being suspended."));
+		}
+
+		taskExecutorManager = null;
+		resourceManagerId = null;
+		resourceActions = null;
+		started = false;
+	}
+
+	/**
+	 * Closes the slot manager.
+	 *
+	 * @throws Exception if the close operation fails
+	 */
+	@Override
+	public void close() throws Exception {
+		LOG.info("Closing the slot manager.");
+
+		suspend();
+		slotManagerMetricGroup.close();
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Public API
+	// ---------------------------------------------------------------------------------------------
+
+	@Override
+	public void processResourceRequirements(ResourceRequirements resourceRequirements) {
+		checkInit();
+		LOG.debug("Received resource requirements from job {}: {}", resourceRequirements.getJobId(), resourceRequirements.getResourceRequirements());
+
+		if (resourceRequirements.getResourceRequirements().isEmpty()) {
+			jobMasterTargetAddresses.remove(resourceRequirements.getJobId());
+		} else {
+			jobMasterTargetAddresses.put(resourceRequirements.getJobId(), resourceRequirements.getTargetAddress());
+		}
+		resourceTracker.notifyResourceRequirements(resourceRequirements.getJobId(), resourceRequirements.getResourceRequirements());
+		checkResourceRequirements();
+	}
+
+	/**
+	 * 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
+	 * @return True if the task manager has not been registered before and is registered successfully; otherwise false
+	 */
+	@Override
+	public boolean registerTaskManager(final TaskExecutorConnection taskExecutorConnection, SlotReport initialSlotReport) {
+		checkInit();
+		LOG.debug("Registering task executor {} under {} at the slot manager.", taskExecutorConnection.getResourceID(), taskExecutorConnection.getInstanceID());
+
+		// we identify task managers by their instance id
+		if (taskExecutorManager.isTaskManagerRegistered(taskExecutorConnection.getInstanceID())) {
+			LOG.debug("Task executor {} was already registered.", taskExecutorConnection.getResourceID());
+			reportSlotStatus(taskExecutorConnection.getInstanceID(), initialSlotReport);
+			return false;
+		} else {
+			if (!taskExecutorManager.registerTaskManager(taskExecutorConnection, initialSlotReport)) {
+				LOG.debug("Task executor {} could not be registered.", taskExecutorConnection.getResourceID());
+				return false;
+			}
+
+			// register the new slots
+			for (SlotStatus slotStatus : initialSlotReport) {
+				slotTracker.addSlot(
+					slotStatus.getSlotID(),
+					slotStatus.getResourceProfile(),
+					taskExecutorConnection,
+					slotStatus.getJobID());
+			}
+
+			checkResourceRequirements();
+			return true;
+		}
+	}
+
+	@Override
+	public boolean unregisterTaskManager(InstanceID instanceId, Exception cause) {
+		checkInit();
+
+		LOG.debug("Unregistering task executor {} from the slot manager.", instanceId);
+
+		if (taskExecutorManager.isTaskManagerRegistered(instanceId)) {
+			slotTracker.removeSlots(taskExecutorManager.getSlotsOf(instanceId));
+			taskExecutorManager.unregisterTaskExecutor(instanceId);
+			checkResourceRequirements();
+
+			return true;
+		} else {
+			LOG.debug("There is no task executor registered with instance ID {}. Ignoring this message.", instanceId);
+
+			return false;
+		}
+	}
+
+	/**
+	 * Reports the current slot allocations for a task manager identified by the given instance id.
+	 *
+	 * @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
+	 */
+	@Override
+	public boolean reportSlotStatus(InstanceID instanceId, SlotReport slotReport) {
+		checkInit();
+
+		LOG.debug("Received slot report from instance {}: {}.", instanceId, slotReport);
+
+		if (taskExecutorManager.isTaskManagerRegistered(instanceId)) {
+			slotTracker.notifySlotStatus(slotReport);
+			checkResourceRequirements();
+			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
+	 */
+	@Override
+	public void freeSlot(SlotID slotId, AllocationID allocationId) {
+		checkInit();
+		LOG.debug("Freeing slot {}.", slotId);
+
+		slotTracker.notifyFree(slotId);
+		checkResourceRequirements();
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Requirement matching
+	// ---------------------------------------------------------------------------------------------
+
+	private void checkResourceRequirements() {
+		final Map<JobID, Collection<ResourceRequirement>> missingResources = resourceTracker.getMissingResources();
+		if (missingResources.isEmpty()) {
+			return;
+		}
+
+		final Map<JobID, ResourceCounter> outstandingRequirements = new LinkedHashMap<>();
+		for (Map.Entry<JobID, Collection<ResourceRequirement>> resourceRequirements : missingResources.entrySet()) {
+			JobID jobId = resourceRequirements.getKey();
+
+			for (ResourceRequirement resourceRequirement : resourceRequirements.getValue()) {
+				int numMissingSlots = internalRequestSlots(jobId, jobMasterTargetAddresses.get(jobId), resourceRequirement);
+				if (numMissingSlots > 0) {
+					outstandingRequirements
+						.computeIfAbsent(jobId, ignored -> new ResourceCounter())
+						.incrementCount(resourceRequirement.getResourceProfile(), numMissingSlots);
+				}
+			}
+		}
+
+		final ResourceCounter pendingSlots = new ResourceCounter(taskExecutorManager.getPendingTaskManagerSlots().stream().collect(
+			Collectors.groupingBy(
+				PendingTaskManagerSlot::getResourceProfile,
+				Collectors.summingInt(x -> 1))));
+
+		for (Map.Entry<JobID, ResourceCounter> unfulfilledRequirement : outstandingRequirements.entrySet()) {
+			tryFulfillRequirementsWithPendingOrNewSlots(
+				unfulfilledRequirement.getKey(),
+				unfulfilledRequirement.getValue().getResourceProfilesWithCount(),
+				pendingSlots);
+		}
+	}
+
+	/**
+	 * Tries to allocate slots for the given requirement. If there are not enough slots available, the
+	 * resource manager is informed to allocate more resources.
+	 *
+	 * @param jobId job to allocate slots for
+	 * @param targetAddress address of the jobmaster
+	 * @param resourceRequirement required slots
+	 * @return the number of missing slots
+	 */
+	private int internalRequestSlots(JobID jobId, String targetAddress, ResourceRequirement resourceRequirement) {
+		final ResourceProfile requiredResource = resourceRequirement.getResourceProfile();
+		Collection<TaskManagerSlotInformation> freeSlots = slotTracker.getFreeSlots();
+
+		int numUnfulfilled = 0;
+		for (int x = 0; x < resourceRequirement.getNumberOfRequiredSlots(); x++) {
+
+			final Optional<TaskManagerSlotInformation> reservedSlot = slotMatchingStrategy.findMatchingSlot(requiredResource, freeSlots, this::getNumberRegisteredSlotsOf);
+			if (reservedSlot.isPresent()) {
+				// we do not need to modify freeSlots because it is indirectly modified by the allocation
+				allocateSlot(reservedSlot.get(), jobId, targetAddress, requiredResource);
+			} else {
+				numUnfulfilled++;
+			}
+		}
+		return numUnfulfilled;
+	}
+
+	/**
+	 * Allocates the given slot. This entails sending a registration message to the task manager and treating failures.
+	 *
+	 * @param taskManagerSlot slot to allocate
+	 * @param jobId job for which the slot should be allocated for
+	 * @param targetAddress address of the job master
+	 * @param resourceProfile resource profile for the requirement for which the slot is used
+	 */
+	private void allocateSlot(TaskManagerSlotInformation taskManagerSlot, JobID jobId, String targetAddress, ResourceProfile resourceProfile) {
+		TaskExecutorConnection taskExecutorConnection = taskManagerSlot.getTaskManagerConnection();
+		TaskExecutorGateway gateway = taskExecutorConnection.getTaskExecutorGateway();
+
+		final CompletableFuture<Acknowledge> completableFuture = new CompletableFuture<>();
+		final SlotID slotId = taskManagerSlot.getSlotId();
+		final InstanceID instanceId = taskManagerSlot.getInstanceId();
+
+		LOG.debug("Starting allocation of slot {} for job {} with resource profile {}.", taskManagerSlot.getSlotId(), jobId, resourceProfile);
+
+		slotTracker.notifyAllocationStart(slotId, jobId);
+
+		if (!taskExecutorManager.isTaskManagerRegistered(instanceId)) {
+			throw new IllegalStateException("Could not find a registered task manager for instance id " +
+				instanceId + '.');
+		}
+
+		taskExecutorManager.markUsed(instanceId);
+
+		// RPC call to the task manager
+		CompletableFuture<Acknowledge> requestFuture = gateway.requestSlot(
+			slotId,
+			jobId,
+			new AllocationID(),
+			resourceProfile,
+			targetAddress,
+			resourceManagerId,
+			taskManagerRequestTimeout);
+
+		requestFuture.whenComplete(
+			(Acknowledge acknowledge, Throwable throwable) -> {
+				if (acknowledge != null) {
+					completableFuture.complete(acknowledge);
+				} else {
+					completableFuture.completeExceptionally(throwable);
+				}
+			});
+
+		completableFuture.whenCompleteAsync(
+			(Acknowledge acknowledge, Throwable throwable) -> {
+				try {
+					if (acknowledge != null) {
+						LOG.trace("Completed allocation of slot {} for job {}.", slotId, jobId);
+						slotTracker.notifyAllocationComplete(slotId, jobId);
+					} else {
+						if (throwable instanceof SlotOccupiedException) {
+							SlotOccupiedException exception = (SlotOccupiedException) throwable;
+							LOG.debug("Tried allocating slot {} for job {}, but it was already allocated for job {}.", slotId, jobId, exception.getJobId());
+							// report as a slot status to force the state transition
+							// this could be a problem if we ever assume that the task executor always reports about all slots
+							slotTracker.notifySlotStatus(Collections.singleton(new SlotStatus(slotId, taskManagerSlot.getResourceProfile(), exception.getJobId(), exception.getAllocationId())));
+						} else {
+							if (throwable instanceof CancellationException) {
+								LOG.debug("Cancelled allocation of slot {} for job {}.", slotId, jobId, throwable);

Review comment:
       Where are we cancelling `completableFuture`?

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManagerTest.java
##########
@@ -0,0 +1,1092 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple6;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.concurrent.ManuallyTriggeredScheduledExecutor;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerId;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+import org.apache.flink.runtime.slots.ResourceRequirements;
+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.TestingTaskExecutorGateway;
+import org.apache.flink.runtime.taskexecutor.TestingTaskExecutorGatewayBuilder;
+import org.apache.flink.runtime.taskexecutor.exceptions.SlotAllocationException;
+import org.apache.flink.runtime.taskexecutor.exceptions.SlotOccupiedException;
+import org.apache.flink.runtime.testingUtils.TestingUtils;
+import org.apache.flink.util.FlinkException;
+import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.function.FunctionUtils;
+
+import akka.pattern.AskTimeoutException;
+import org.junit.Ignore;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.hamcrest.CoreMatchers.hasItem;
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.empty;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.hasSize;
+import static org.hamcrest.Matchers.is;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertSame;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Tests for the {@link DeclarativeSlotManager}.
+ */
+public class DeclarativeSlotManagerTest extends TestLogger {
+
+	private static final FlinkException TEST_EXCEPTION = new FlinkException("Test exception");
+
+	private static final WorkerResourceSpec WORKER_RESOURCE_SPEC = new WorkerResourceSpec.Builder()
+		.setCpuCores(100.0)
+		.setTaskHeapMemoryMB(10000)
+		.setTaskOffHeapMemoryMB(10000)
+		.setNetworkMemoryMB(10000)
+		.setManagedMemoryMB(10000)
+		.build();
+
+	/**
+	 * Tests that we can register task manager and their slots at the slot manager.
+	 */
+	@Test
+	public void testTaskManagerRegistration() throws Exception {
+		final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder().createTestingTaskExecutorGateway();
+		final ResourceID resourceId = ResourceID.generate();
+		final TaskExecutorConnection taskManagerConnection = new TaskExecutorConnection(resourceId, taskExecutorGateway);
+
+		final SlotID slotId1 = new SlotID(resourceId, 0);
+		final SlotID slotId2 = new SlotID(resourceId, 1);
+		final SlotReport slotReport = new SlotReport(Arrays.asList(createFreeSlotStatus(slotId1), createFreeSlotStatus(slotId2)));
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport);
+
+			assertThat("The number registered slots does not equal the expected number.", slotManager.getNumberRegisteredSlots(), is(2));
+
+			assertNotNull(slotTracker.getSlot(slotId1));
+			assertNotNull(slotTracker.getSlot(slotId2));
+		}
+	}
+
+	/**
+	 * Tests that un-registration of task managers will free and remove all registered slots.
+	 */
+	@Test
+	public void testTaskManagerUnregistration() throws Exception {
+		final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder()
+			.setRequestSlotFunction(tuple6 -> new CompletableFuture<>())
+			.createTestingTaskExecutorGateway();
+		final TaskExecutorConnection taskManagerConnection = createTaskExecutorConnection(taskExecutorGateway);
+		final ResourceID resourceId = taskManagerConnection.getResourceID();
+
+		final SlotID slotId1 = new SlotID(resourceId, 0);
+		final SlotID slotId2 = new SlotID(resourceId, 1);
+		final SlotReport slotReport = new SlotReport(Arrays.asList(createAllocatedSlotStatus(slotId1), createFreeSlotStatus(slotId2)));
+
+		final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot();
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport);
+
+			assertEquals("The number registered slots does not equal the expected number.", 2, slotManager.getNumberRegisteredSlots());
+
+			DeclarativeTaskManagerSlot slot1 = slotTracker.getSlot(slotId1);
+			DeclarativeTaskManagerSlot slot2 = slotTracker.getSlot(slotId2);
+
+			assertSame(SlotState.ALLOCATED, slot1.getState());
+			assertSame(SlotState.FREE, slot2.getState());

Review comment:
       Do we really have to look at the internal state of the slots in order to ensure that after a TM gets unregistered that there are no more slots? One could simply use `slotManager.getNumberRegisteredSlots()` for this.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManager.java
##########
@@ -0,0 +1,609 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.metrics.MetricNames;
+import org.apache.flink.runtime.metrics.groups.SlotManagerMetricGroup;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerId;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.slots.ResourceCounter;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+import org.apache.flink.runtime.slots.ResourceRequirements;
+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.SlotOccupiedException;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ScheduledFuture;
+import java.util.function.BiFunction;
+import java.util.stream.Collectors;
+
+/**
+ * Implementation of {@link SlotManager} supporting declarative slot management.
+ */
+public class DeclarativeSlotManager implements SlotManager {
+	private static final Logger LOG = LoggerFactory.getLogger(DeclarativeSlotManager.class);
+
+	private final SlotTracker slotTracker;
+	private final ResourceTracker resourceTracker;
+	private final BiFunction<Executor, ResourceActions, TaskExecutorManager> taskExecutorManagerFactory;
+	private TaskExecutorManager taskExecutorManager;
+
+	/** Timeout for slot requests to the task manager. */
+	private final Time taskManagerRequestTimeout;
+	private ScheduledFuture<?> slotRequestTimeoutCheck;
+
+	private final SlotMatchingStrategy slotMatchingStrategy;
+
+	private final SlotManagerMetricGroup slotManagerMetricGroup;
+
+	private final Map<JobID, String> jobMasterTargetAddresses = new HashMap<>();
+	private final HashMap<SlotID, CompletableFuture<Acknowledge>> pendingSlotAllocationFutures;
+
+	/** ResourceManager's id. */
+	private ResourceManagerId resourceManagerId;
+
+	/** Executor for future callbacks which have to be "synchronized". */
+	private Executor mainThreadExecutor;
+
+	/** Callbacks for resource (de-)allocations. */
+	private ResourceActions resourceActions;
+
+	/** True iff the component has been started. */
+	private boolean started;
+
+	public DeclarativeSlotManager(
+			ScheduledExecutor scheduledExecutor,
+			SlotManagerConfiguration slotManagerConfiguration,
+			SlotManagerMetricGroup slotManagerMetricGroup,
+			ResourceTracker resourceTracker,
+			SlotTracker slotTracker) {
+
+		Preconditions.checkNotNull(slotManagerConfiguration);
+		this.taskManagerRequestTimeout = slotManagerConfiguration.getTaskManagerRequestTimeout();
+		this.slotManagerMetricGroup = Preconditions.checkNotNull(slotManagerMetricGroup);
+		this.resourceTracker = Preconditions.checkNotNull(resourceTracker);
+
+		pendingSlotAllocationFutures = new HashMap<>(16);
+
+		this.slotTracker = Preconditions.checkNotNull(slotTracker);
+		slotTracker.registerSlotStatusUpdateListener(createSlotStatusUpdateListener());
+
+		slotMatchingStrategy = slotManagerConfiguration.getSlotMatchingStrategy();
+
+		taskExecutorManagerFactory = (executor, resourceActions) -> new TaskExecutorManager(
+			slotManagerConfiguration.getDefaultWorkerResourceSpec(),
+			slotManagerConfiguration.getNumSlotsPerWorker(),
+			slotManagerConfiguration.getMaxSlotNum(),
+			slotManagerConfiguration.isWaitResultConsumedBeforeRelease(),
+			slotManagerConfiguration.getRedundantTaskManagerNum(),
+			slotManagerConfiguration.getTaskManagerTimeout(),
+			scheduledExecutor,
+			executor,
+			resourceActions);
+
+		resourceManagerId = null;
+		resourceActions = null;
+		mainThreadExecutor = null;
+		slotRequestTimeoutCheck = null;
+		taskExecutorManager = null;
+
+		started = false;
+	}
+
+	private SlotStatusUpdateListener createSlotStatusUpdateListener() {
+		return (taskManagerSlot, previous, current, jobId) -> {
+			if (previous == SlotState.PENDING) {
+				cancelAllocationFuture(taskManagerSlot.getSlotId());
+			}
+
+			if (current == SlotState.PENDING) {
+				resourceTracker.notifyAcquiredResource(jobId, taskManagerSlot.getResourceProfile());
+			}
+			if (current == SlotState.FREE) {
+				resourceTracker.notifyLostResource(jobId, taskManagerSlot.getResourceProfile());
+			}
+
+			if (current == SlotState.ALLOCATED) {
+				taskExecutorManager.occupySlot(taskManagerSlot.getInstanceId());
+			}
+			if (previous == SlotState.ALLOCATED && current == SlotState.FREE) {
+				taskExecutorManager.freeSlot(taskManagerSlot.getInstanceId());
+			}
+		};
+	}
+
+	private void cancelAllocationFuture(SlotID slotId) {
+		final CompletableFuture<Acknowledge> acknowledgeCompletableFuture = pendingSlotAllocationFutures.remove(slotId);
+		// the future may be null if we are just re-playing the state transitions due to a slot report
+		if (acknowledgeCompletableFuture != null) {
+			acknowledgeCompletableFuture.cancel(false);
+		}
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Component lifecycle methods
+	// ---------------------------------------------------------------------------------------------
+
+	/**
+	 * Starts the slot manager with the given leader id and resource manager actions.
+	 *
+	 * @param newResourceManagerId to use for communication with the task managers
+	 * @param newMainThreadExecutor to use to run code in the ResourceManager's main thread
+	 * @param newResourceActions to use for resource (de-)allocations
+	 */
+	@Override
+	public void start(ResourceManagerId newResourceManagerId, Executor newMainThreadExecutor, ResourceActions newResourceActions) {
+		LOG.info("Starting the slot manager.");
+
+		this.resourceManagerId = Preconditions.checkNotNull(newResourceManagerId);
+		mainThreadExecutor = Preconditions.checkNotNull(newMainThreadExecutor);
+		resourceActions = Preconditions.checkNotNull(newResourceActions);
+		taskExecutorManager = taskExecutorManagerFactory.apply(newMainThreadExecutor, newResourceActions);
+
+		started = true;
+
+		registerSlotManagerMetrics();
+	}
+
+	private void registerSlotManagerMetrics() {
+		slotManagerMetricGroup.gauge(
+			MetricNames.TASK_SLOTS_AVAILABLE,
+			() -> (long) getNumberFreeSlots());
+		slotManagerMetricGroup.gauge(
+			MetricNames.TASK_SLOTS_TOTAL,
+			() -> (long) getNumberRegisteredSlots());
+	}
+
+	/**
+	 * Suspends the component. This clears the internal state of the slot manager.
+	 */
+	@Override
+	public void suspend() {
+		LOG.info("Suspending the slot manager.");
+
+		if (slotRequestTimeoutCheck != null) {
+			slotRequestTimeoutCheck.cancel(false);
+			slotRequestTimeoutCheck = null;
+		}
+
+		resourceTracker.clear();
+		taskExecutorManager.close();
+
+		for (InstanceID registeredTaskManager : taskExecutorManager.getTaskExecutors()) {
+			unregisterTaskManager(registeredTaskManager, new SlotManagerException("The slot manager is being suspended."));
+		}
+
+		taskExecutorManager = null;
+		resourceManagerId = null;
+		resourceActions = null;
+		started = false;
+	}
+
+	/**
+	 * Closes the slot manager.
+	 *
+	 * @throws Exception if the close operation fails
+	 */
+	@Override
+	public void close() throws Exception {
+		LOG.info("Closing the slot manager.");
+
+		suspend();
+		slotManagerMetricGroup.close();
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Public API
+	// ---------------------------------------------------------------------------------------------
+
+	@Override
+	public void processResourceRequirements(ResourceRequirements resourceRequirements) {
+		checkInit();
+		LOG.debug("Received resource requirements from job {}: {}", resourceRequirements.getJobId(), resourceRequirements.getResourceRequirements());
+
+		if (resourceRequirements.getResourceRequirements().isEmpty()) {
+			jobMasterTargetAddresses.remove(resourceRequirements.getJobId());
+		} else {
+			jobMasterTargetAddresses.put(resourceRequirements.getJobId(), resourceRequirements.getTargetAddress());
+		}
+		resourceTracker.notifyResourceRequirements(resourceRequirements.getJobId(), resourceRequirements.getResourceRequirements());
+		checkResourceRequirements();
+	}
+
+	/**
+	 * 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
+	 * @return True if the task manager has not been registered before and is registered successfully; otherwise false
+	 */
+	@Override
+	public boolean registerTaskManager(final TaskExecutorConnection taskExecutorConnection, SlotReport initialSlotReport) {
+		checkInit();
+		LOG.debug("Registering task executor {} under {} at the slot manager.", taskExecutorConnection.getResourceID(), taskExecutorConnection.getInstanceID());
+
+		// we identify task managers by their instance id
+		if (taskExecutorManager.isTaskManagerRegistered(taskExecutorConnection.getInstanceID())) {
+			LOG.debug("Task executor {} was already registered.", taskExecutorConnection.getResourceID());
+			reportSlotStatus(taskExecutorConnection.getInstanceID(), initialSlotReport);
+			return false;
+		} else {
+			if (!taskExecutorManager.registerTaskManager(taskExecutorConnection, initialSlotReport)) {
+				LOG.debug("Task executor {} could not be registered.", taskExecutorConnection.getResourceID());
+				return false;
+			}
+
+			// register the new slots
+			for (SlotStatus slotStatus : initialSlotReport) {
+				slotTracker.addSlot(
+					slotStatus.getSlotID(),
+					slotStatus.getResourceProfile(),
+					taskExecutorConnection,
+					slotStatus.getJobID());
+			}
+
+			checkResourceRequirements();
+			return true;
+		}
+	}
+
+	@Override
+	public boolean unregisterTaskManager(InstanceID instanceId, Exception cause) {
+		checkInit();
+
+		LOG.debug("Unregistering task executor {} from the slot manager.", instanceId);
+
+		if (taskExecutorManager.isTaskManagerRegistered(instanceId)) {
+			slotTracker.removeSlots(taskExecutorManager.getSlotsOf(instanceId));
+			taskExecutorManager.unregisterTaskExecutor(instanceId);
+			checkResourceRequirements();
+
+			return true;
+		} else {
+			LOG.debug("There is no task executor registered with instance ID {}. Ignoring this message.", instanceId);
+
+			return false;
+		}
+	}
+
+	/**
+	 * Reports the current slot allocations for a task manager identified by the given instance id.
+	 *
+	 * @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
+	 */
+	@Override
+	public boolean reportSlotStatus(InstanceID instanceId, SlotReport slotReport) {
+		checkInit();
+
+		LOG.debug("Received slot report from instance {}: {}.", instanceId, slotReport);
+
+		if (taskExecutorManager.isTaskManagerRegistered(instanceId)) {
+			slotTracker.notifySlotStatus(slotReport);
+			checkResourceRequirements();
+			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
+	 */
+	@Override
+	public void freeSlot(SlotID slotId, AllocationID allocationId) {
+		checkInit();
+		LOG.debug("Freeing slot {}.", slotId);
+
+		slotTracker.notifyFree(slotId);
+		checkResourceRequirements();
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Requirement matching
+	// ---------------------------------------------------------------------------------------------
+
+	private void checkResourceRequirements() {
+		final Map<JobID, Collection<ResourceRequirement>> missingResources = resourceTracker.getMissingResources();
+		if (missingResources.isEmpty()) {
+			return;
+		}
+
+		final Map<JobID, ResourceCounter> outstandingRequirements = new LinkedHashMap<>();
+		for (Map.Entry<JobID, Collection<ResourceRequirement>> resourceRequirements : missingResources.entrySet()) {
+			JobID jobId = resourceRequirements.getKey();
+
+			for (ResourceRequirement resourceRequirement : resourceRequirements.getValue()) {
+				int numMissingSlots = internalRequestSlots(jobId, jobMasterTargetAddresses.get(jobId), resourceRequirement);
+				if (numMissingSlots > 0) {
+					outstandingRequirements
+						.computeIfAbsent(jobId, ignored -> new ResourceCounter())
+						.incrementCount(resourceRequirement.getResourceProfile(), numMissingSlots);
+				}
+			}
+		}
+
+		final ResourceCounter pendingSlots = new ResourceCounter(taskExecutorManager.getPendingTaskManagerSlots().stream().collect(
+			Collectors.groupingBy(
+				PendingTaskManagerSlot::getResourceProfile,
+				Collectors.summingInt(x -> 1))));
+
+		for (Map.Entry<JobID, ResourceCounter> unfulfilledRequirement : outstandingRequirements.entrySet()) {
+			tryFulfillRequirementsWithPendingOrNewSlots(
+				unfulfilledRequirement.getKey(),
+				unfulfilledRequirement.getValue().getResourceProfilesWithCount(),
+				pendingSlots);
+		}
+	}
+
+	/**
+	 * Tries to allocate slots for the given requirement. If there are not enough slots available, the
+	 * resource manager is informed to allocate more resources.
+	 *
+	 * @param jobId job to allocate slots for
+	 * @param targetAddress address of the jobmaster
+	 * @param resourceRequirement required slots
+	 * @return the number of missing slots
+	 */
+	private int internalRequestSlots(JobID jobId, String targetAddress, ResourceRequirement resourceRequirement) {
+		final ResourceProfile requiredResource = resourceRequirement.getResourceProfile();
+		Collection<TaskManagerSlotInformation> freeSlots = slotTracker.getFreeSlots();
+
+		int numUnfulfilled = 0;
+		for (int x = 0; x < resourceRequirement.getNumberOfRequiredSlots(); x++) {
+
+			final Optional<TaskManagerSlotInformation> reservedSlot = slotMatchingStrategy.findMatchingSlot(requiredResource, freeSlots, this::getNumberRegisteredSlotsOf);
+			if (reservedSlot.isPresent()) {
+				// we do not need to modify freeSlots because it is indirectly modified by the allocation
+				allocateSlot(reservedSlot.get(), jobId, targetAddress, requiredResource);
+			} else {
+				numUnfulfilled++;
+			}
+		}
+		return numUnfulfilled;
+	}
+
+	/**
+	 * Allocates the given slot. This entails sending a registration message to the task manager and treating failures.
+	 *
+	 * @param taskManagerSlot slot to allocate
+	 * @param jobId job for which the slot should be allocated for
+	 * @param targetAddress address of the job master
+	 * @param resourceProfile resource profile for the requirement for which the slot is used
+	 */
+	private void allocateSlot(TaskManagerSlotInformation taskManagerSlot, JobID jobId, String targetAddress, ResourceProfile resourceProfile) {
+		TaskExecutorConnection taskExecutorConnection = taskManagerSlot.getTaskManagerConnection();
+		TaskExecutorGateway gateway = taskExecutorConnection.getTaskExecutorGateway();
+
+		final CompletableFuture<Acknowledge> completableFuture = new CompletableFuture<>();
+		final SlotID slotId = taskManagerSlot.getSlotId();
+		final InstanceID instanceId = taskManagerSlot.getInstanceId();
+
+		LOG.debug("Starting allocation of slot {} for job {} with resource profile {}.", taskManagerSlot.getSlotId(), jobId, resourceProfile);
+
+		slotTracker.notifyAllocationStart(slotId, jobId);
+
+		if (!taskExecutorManager.isTaskManagerRegistered(instanceId)) {
+			throw new IllegalStateException("Could not find a registered task manager for instance id " +
+				instanceId + '.');
+		}
+
+		taskExecutorManager.markUsed(instanceId);
+
+		// RPC call to the task manager
+		CompletableFuture<Acknowledge> requestFuture = gateway.requestSlot(
+			slotId,
+			jobId,
+			new AllocationID(),
+			resourceProfile,
+			targetAddress,
+			resourceManagerId,
+			taskManagerRequestTimeout);
+
+		requestFuture.whenComplete(
+			(Acknowledge acknowledge, Throwable throwable) -> {
+				if (acknowledge != null) {
+					completableFuture.complete(acknowledge);
+				} else {
+					completableFuture.completeExceptionally(throwable);
+				}
+			});
+
+		completableFuture.whenCompleteAsync(
+			(Acknowledge acknowledge, Throwable throwable) -> {
+				try {
+					if (acknowledge != null) {
+						LOG.trace("Completed allocation of slot {} for job {}.", slotId, jobId);
+						slotTracker.notifyAllocationComplete(slotId, jobId);
+					} else {
+						if (throwable instanceof SlotOccupiedException) {
+							SlotOccupiedException exception = (SlotOccupiedException) throwable;
+							LOG.debug("Tried allocating slot {} for job {}, but it was already allocated for job {}.", slotId, jobId, exception.getJobId());
+							// report as a slot status to force the state transition
+							// this could be a problem if we ever assume that the task executor always reports about all slots
+							slotTracker.notifySlotStatus(Collections.singleton(new SlotStatus(slotId, taskManagerSlot.getResourceProfile(), exception.getJobId(), exception.getAllocationId())));
+						} else {
+							if (throwable instanceof CancellationException) {
+								LOG.debug("Cancelled allocation of slot {} for job {}.", slotId, jobId, throwable);
+							} else {
+								LOG.warn("Slot allocation for slot {} for job {} failed.", slotId, jobId, throwable);
+								slotTracker.notifyFree(slotId);
+							}
+						}
+					}
+					checkResourceRequirements();
+				} catch (Exception e) {
+					LOG.error("Error while completing the slot allocation.", e);
+				}
+			},
+			mainThreadExecutor);
+	}
+
+	private void tryFulfillRequirementsWithPendingOrNewSlots(JobID jobId, Map<ResourceProfile, Integer> missingResources, ResourceCounter pendingSlots) {
+		for (Map.Entry<ResourceProfile, Integer> missingResource : missingResources.entrySet()) {
+			ResourceProfile profile = missingResource.getKey();
+			for (int i = 0; i < missingResource.getValue(); i++) {
+				if (!tryFulfillWithPendingSlots(profile, pendingSlots)) {
+					Optional<ResourceRequirement> newlyFulfillableRequirements = taskExecutorManager.allocateWorker(profile);
+					if (newlyFulfillableRequirements.isPresent()) {
+						ResourceRequirement newSlots = newlyFulfillableRequirements.get();
+						// reserve one of the new slots
+						if (newSlots.getNumberOfRequiredSlots() > 1) {
+							pendingSlots.incrementCount(newSlots.getResourceProfile(), newSlots.getNumberOfRequiredSlots() - 1);
+						}
+					} else {
+						LOG.warn("Could not fulfill resource requirements of job {}.", jobId);
+						resourceActions.notifyNotEnoughResourcesAvailable(jobId, resourceTracker.getAcquiredResources(jobId));
+						return;
+					}

Review comment:
       One could factor this part out into a separate method which might make it a bit less complex.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManagerTest.java
##########
@@ -0,0 +1,1092 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple6;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.concurrent.ManuallyTriggeredScheduledExecutor;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerId;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+import org.apache.flink.runtime.slots.ResourceRequirements;
+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.TestingTaskExecutorGateway;
+import org.apache.flink.runtime.taskexecutor.TestingTaskExecutorGatewayBuilder;
+import org.apache.flink.runtime.taskexecutor.exceptions.SlotAllocationException;
+import org.apache.flink.runtime.taskexecutor.exceptions.SlotOccupiedException;
+import org.apache.flink.runtime.testingUtils.TestingUtils;
+import org.apache.flink.util.FlinkException;
+import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.function.FunctionUtils;
+
+import akka.pattern.AskTimeoutException;
+import org.junit.Ignore;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.hamcrest.CoreMatchers.hasItem;
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.empty;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.hasSize;
+import static org.hamcrest.Matchers.is;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertSame;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Tests for the {@link DeclarativeSlotManager}.
+ */
+public class DeclarativeSlotManagerTest extends TestLogger {
+
+	private static final FlinkException TEST_EXCEPTION = new FlinkException("Test exception");
+
+	private static final WorkerResourceSpec WORKER_RESOURCE_SPEC = new WorkerResourceSpec.Builder()
+		.setCpuCores(100.0)
+		.setTaskHeapMemoryMB(10000)
+		.setTaskOffHeapMemoryMB(10000)
+		.setNetworkMemoryMB(10000)
+		.setManagedMemoryMB(10000)
+		.build();
+
+	/**
+	 * Tests that we can register task manager and their slots at the slot manager.
+	 */
+	@Test
+	public void testTaskManagerRegistration() throws Exception {
+		final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder().createTestingTaskExecutorGateway();
+		final ResourceID resourceId = ResourceID.generate();
+		final TaskExecutorConnection taskManagerConnection = new TaskExecutorConnection(resourceId, taskExecutorGateway);
+
+		final SlotID slotId1 = new SlotID(resourceId, 0);
+		final SlotID slotId2 = new SlotID(resourceId, 1);
+		final SlotReport slotReport = new SlotReport(Arrays.asList(createFreeSlotStatus(slotId1), createFreeSlotStatus(slotId2)));
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport);
+
+			assertThat("The number registered slots does not equal the expected number.", slotManager.getNumberRegisteredSlots(), is(2));
+
+			assertNotNull(slotTracker.getSlot(slotId1));
+			assertNotNull(slotTracker.getSlot(slotId2));
+		}
+	}
+
+	/**
+	 * Tests that un-registration of task managers will free and remove all registered slots.
+	 */
+	@Test
+	public void testTaskManagerUnregistration() throws Exception {
+		final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder()
+			.setRequestSlotFunction(tuple6 -> new CompletableFuture<>())
+			.createTestingTaskExecutorGateway();
+		final TaskExecutorConnection taskManagerConnection = createTaskExecutorConnection(taskExecutorGateway);
+		final ResourceID resourceId = taskManagerConnection.getResourceID();
+
+		final SlotID slotId1 = new SlotID(resourceId, 0);
+		final SlotID slotId2 = new SlotID(resourceId, 1);
+		final SlotReport slotReport = new SlotReport(Arrays.asList(createAllocatedSlotStatus(slotId1), createFreeSlotStatus(slotId2)));
+
+		final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot();
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport);
+
+			assertEquals("The number registered slots does not equal the expected number.", 2, slotManager.getNumberRegisteredSlots());
+
+			DeclarativeTaskManagerSlot slot1 = slotTracker.getSlot(slotId1);
+			DeclarativeTaskManagerSlot slot2 = slotTracker.getSlot(slotId2);
+
+			assertSame(SlotState.ALLOCATED, slot1.getState());
+			assertSame(SlotState.FREE, slot2.getState());
+
+			slotManager.processResourceRequirements(resourceRequirements);
+
+			assertSame(SlotState.PENDING, slot2.getState());
+
+			slotManager.unregisterTaskManager(taskManagerConnection.getInstanceID(), TEST_EXCEPTION);
+
+			assertEquals(0, slotManager.getNumberRegisteredSlots());
+		}
+	}
+
+	/**
+	 * Tests that a slot request with no free slots will trigger the resource allocation.
+	 */
+	@Test
+	public void testSlotRequestWithoutFreeSlots() throws Exception {
+		final ResourceManagerId resourceManagerId = ResourceManagerId.generate();
+
+		final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot();
+
+		CompletableFuture<WorkerResourceSpec> allocateResourceFuture = new CompletableFuture<>();
+		ResourceActions resourceManagerActions = new TestingResourceActionsBuilder()
+			.setAllocateResourceConsumer(allocateResourceFuture::complete)
+			.build();
+
+		try (SlotManager slotManager = createSlotManager(resourceManagerId, resourceManagerActions)) {
+
+			slotManager.processResourceRequirements(resourceRequirements);
+
+			allocateResourceFuture.get();

Review comment:
       Maybe check that the right sized resource is requested.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13553: [FLINK-19306][coordination] Add DeclarativeSlotManager

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13553:
URL: https://github.com/apache/flink/pull/13553#issuecomment-704936985


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "15d4c353c4c0a646b2bdf34bbedbd81359cb5b5f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7264",
       "triggerID" : "15d4c353c4c0a646b2bdf34bbedbd81359cb5b5f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15d4c353c4c0a646b2bdf34bbedbd81359cb5b5f",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7283",
       "triggerID" : "705384887",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "15d4c353c4c0a646b2bdf34bbedbd81359cb5b5f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7264",
       "triggerID" : "705384887",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 15d4c353c4c0a646b2bdf34bbedbd81359cb5b5f Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7283) Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7264) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13553: [FLINK-19306][coordination] Add DeclarativeSlotManager

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13553:
URL: https://github.com/apache/flink/pull/13553#issuecomment-704936985


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "15d4c353c4c0a646b2bdf34bbedbd81359cb5b5f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7264",
       "triggerID" : "15d4c353c4c0a646b2bdf34bbedbd81359cb5b5f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15d4c353c4c0a646b2bdf34bbedbd81359cb5b5f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7283",
       "triggerID" : "705384887",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "15d4c353c4c0a646b2bdf34bbedbd81359cb5b5f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7264",
       "triggerID" : "705384887",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "484cbbdfcc7e371ca51de4d88bab086a299795a6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7514",
       "triggerID" : "484cbbdfcc7e371ca51de4d88bab086a299795a6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7a4c350c9514dc81429de2d9887837045b5894e0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7527",
       "triggerID" : "7a4c350c9514dc81429de2d9887837045b5894e0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4b9288681085da65fc2c3f2742c898d13bf4af46",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7580",
       "triggerID" : "4b9288681085da65fc2c3f2742c898d13bf4af46",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f89b7098fb826ded9b17b0f2eb48c747451623d9",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7667",
       "triggerID" : "f89b7098fb826ded9b17b0f2eb48c747451623d9",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * f89b7098fb826ded9b17b0f2eb48c747451623d9 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7667) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13553: [FLINK-19306][coordination] Add DeclarativeSlotManager

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13553:
URL: https://github.com/apache/flink/pull/13553#issuecomment-704936985


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "15d4c353c4c0a646b2bdf34bbedbd81359cb5b5f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7264",
       "triggerID" : "15d4c353c4c0a646b2bdf34bbedbd81359cb5b5f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15d4c353c4c0a646b2bdf34bbedbd81359cb5b5f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7283",
       "triggerID" : "705384887",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "15d4c353c4c0a646b2bdf34bbedbd81359cb5b5f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7264",
       "triggerID" : "705384887",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "484cbbdfcc7e371ca51de4d88bab086a299795a6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7514",
       "triggerID" : "484cbbdfcc7e371ca51de4d88bab086a299795a6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7a4c350c9514dc81429de2d9887837045b5894e0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7527",
       "triggerID" : "7a4c350c9514dc81429de2d9887837045b5894e0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4b9288681085da65fc2c3f2742c898d13bf4af46",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7580",
       "triggerID" : "4b9288681085da65fc2c3f2742c898d13bf4af46",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 4b9288681085da65fc2c3f2742c898d13bf4af46 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7580) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] zentol commented on a change in pull request #13553: [FLINK-19306][coordination] Add DeclarativeSlotManager

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #13553:
URL: https://github.com/apache/flink/pull/13553#discussion_r503168361



##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManagerTest.java
##########
@@ -0,0 +1,1092 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple6;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.concurrent.ManuallyTriggeredScheduledExecutor;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerId;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+import org.apache.flink.runtime.slots.ResourceRequirements;
+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.TestingTaskExecutorGateway;
+import org.apache.flink.runtime.taskexecutor.TestingTaskExecutorGatewayBuilder;
+import org.apache.flink.runtime.taskexecutor.exceptions.SlotAllocationException;
+import org.apache.flink.runtime.taskexecutor.exceptions.SlotOccupiedException;
+import org.apache.flink.runtime.testingUtils.TestingUtils;
+import org.apache.flink.util.FlinkException;
+import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.function.FunctionUtils;
+
+import akka.pattern.AskTimeoutException;
+import org.junit.Ignore;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.hamcrest.CoreMatchers.hasItem;
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.empty;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.hasSize;
+import static org.hamcrest.Matchers.is;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertSame;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Tests for the {@link DeclarativeSlotManager}.
+ */
+public class DeclarativeSlotManagerTest extends TestLogger {
+
+	private static final FlinkException TEST_EXCEPTION = new FlinkException("Test exception");
+
+	private static final WorkerResourceSpec WORKER_RESOURCE_SPEC = new WorkerResourceSpec.Builder()
+		.setCpuCores(100.0)
+		.setTaskHeapMemoryMB(10000)
+		.setTaskOffHeapMemoryMB(10000)
+		.setNetworkMemoryMB(10000)
+		.setManagedMemoryMB(10000)
+		.build();
+
+	/**
+	 * Tests that we can register task manager and their slots at the slot manager.
+	 */
+	@Test
+	public void testTaskManagerRegistration() throws Exception {
+		final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder().createTestingTaskExecutorGateway();
+		final ResourceID resourceId = ResourceID.generate();
+		final TaskExecutorConnection taskManagerConnection = new TaskExecutorConnection(resourceId, taskExecutorGateway);
+
+		final SlotID slotId1 = new SlotID(resourceId, 0);
+		final SlotID slotId2 = new SlotID(resourceId, 1);
+		final SlotReport slotReport = new SlotReport(Arrays.asList(createFreeSlotStatus(slotId1), createFreeSlotStatus(slotId2)));
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport);
+
+			assertThat("The number registered slots does not equal the expected number.", slotManager.getNumberRegisteredSlots(), is(2));
+
+			assertNotNull(slotTracker.getSlot(slotId1));
+			assertNotNull(slotTracker.getSlot(slotId2));
+		}
+	}
+
+	/**
+	 * Tests that un-registration of task managers will free and remove all registered slots.
+	 */
+	@Test
+	public void testTaskManagerUnregistration() throws Exception {
+		final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder()
+			.setRequestSlotFunction(tuple6 -> new CompletableFuture<>())
+			.createTestingTaskExecutorGateway();
+		final TaskExecutorConnection taskManagerConnection = createTaskExecutorConnection(taskExecutorGateway);
+		final ResourceID resourceId = taskManagerConnection.getResourceID();
+
+		final SlotID slotId1 = new SlotID(resourceId, 0);
+		final SlotID slotId2 = new SlotID(resourceId, 1);
+		final SlotReport slotReport = new SlotReport(Arrays.asList(createAllocatedSlotStatus(slotId1), createFreeSlotStatus(slotId2)));
+
+		final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot();
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport);
+
+			assertEquals("The number registered slots does not equal the expected number.", 2, slotManager.getNumberRegisteredSlots());
+
+			DeclarativeTaskManagerSlot slot1 = slotTracker.getSlot(slotId1);
+			DeclarativeTaskManagerSlot slot2 = slotTracker.getSlot(slotId2);
+
+			assertSame(SlotState.ALLOCATED, slot1.getState());
+			assertSame(SlotState.FREE, slot2.getState());
+
+			slotManager.processResourceRequirements(resourceRequirements);
+
+			assertSame(SlotState.PENDING, slot2.getState());
+
+			slotManager.unregisterTaskManager(taskManagerConnection.getInstanceID(), TEST_EXCEPTION);
+
+			assertEquals(0, slotManager.getNumberRegisteredSlots());
+		}
+	}
+
+	/**
+	 * Tests that a slot request with no free slots will trigger the resource allocation.
+	 */
+	@Test
+	public void testSlotRequestWithoutFreeSlots() throws Exception {
+		final ResourceManagerId resourceManagerId = ResourceManagerId.generate();
+
+		final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot();
+
+		CompletableFuture<WorkerResourceSpec> allocateResourceFuture = new CompletableFuture<>();
+		ResourceActions resourceManagerActions = new TestingResourceActionsBuilder()
+			.setAllocateResourceConsumer(allocateResourceFuture::complete)
+			.build();
+
+		try (SlotManager slotManager = createSlotManager(resourceManagerId, resourceManagerActions)) {
+
+			slotManager.processResourceRequirements(resourceRequirements);
+
+			allocateResourceFuture.get();
+		}
+	}
+
+	/**
+	 * Tests that resources continue to be considered missing if we cannot allocate more resources.
+	 */
+	@Test
+	public void testResourceDeclarationWithResourceAllocationFailure() throws Exception {
+		final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot();
+
+		ResourceActions resourceManagerActions = new TestingResourceActionsBuilder()
+			.setAllocateResourceFunction(value -> false)
+			.build();
+
+		final ResourceTracker resourceTracker = new DefaultResourceTracker();
+
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setResourceTracker(resourceTracker)
+			.buildAndStartWithDirectExec(ResourceManagerId.generate(), resourceManagerActions)) {
+
+			slotManager.processResourceRequirements(resourceRequirements);
+
+			final JobID jobId = resourceRequirements.getJobId();
+			assertThat(getTotalResourceCount(resourceTracker.getMissingResources().get(jobId)), is(1));
+		}
+	}
+
+	/**
+	 * Tests that a slot request which can be fulfilled will trigger a slot allocation.
+	 */
+	@Test
+	public void testSlotRequestWithFreeSlot() throws Exception {
+		final ResourceManagerId resourceManagerId = ResourceManagerId.generate();
+		final ResourceID resourceID = ResourceID.generate();
+		final JobID jobId = new JobID();
+		final SlotID slotId = new SlotID(resourceID, 0);
+		final String targetAddress = "localhost";
+		final ResourceProfile resourceProfile = ResourceProfile.fromResources(42.0, 1337);
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec(resourceManagerId, new TestingResourceActionsBuilder().build())) {
+
+			final CompletableFuture<Tuple6<SlotID, JobID, AllocationID, ResourceProfile, String, ResourceManagerId>> requestFuture = new CompletableFuture<>();
+			// accept an incoming slot request
+			final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder()
+				.setRequestSlotFunction(tuple6 -> {
+					requestFuture.complete(Tuple6.of(tuple6.f0, tuple6.f1, tuple6.f2, tuple6.f3, tuple6.f4, tuple6.f5));
+					return CompletableFuture.completedFuture(Acknowledge.get());
+				})
+				.createTestingTaskExecutorGateway();
+
+			final TaskExecutorConnection taskExecutorConnection = new TaskExecutorConnection(resourceID, taskExecutorGateway);
+
+			final SlotStatus slotStatus = new SlotStatus(slotId, resourceProfile);
+			final SlotReport slotReport = new SlotReport(slotStatus);
+
+			slotManager.registerTaskManager(
+				taskExecutorConnection,
+				slotReport);
+
+			ResourceRequirements requirements = ResourceRequirements.create(
+				jobId,
+				targetAddress,
+				Collections.singleton(ResourceRequirement.create(resourceProfile, 1)));
+			slotManager.processResourceRequirements(requirements);
+
+			assertThat(requestFuture.get(), is(equalTo(Tuple6.of(slotId, jobId, requestFuture.get().f2, resourceProfile, targetAddress, resourceManagerId))));
+
+			DeclarativeTaskManagerSlot slot = slotTracker.getSlot(slotId);
+
+			assertEquals("The slot has not been allocated to the expected allocation id.", jobId, slot.getJobId());
+		}
+	}
+
+	/**
+	 * Checks that reducing resource requirements while a slot allocation is in-progress results in the slot being freed
+	 * one the allocation completes.
+	 */
+	@Test
+	@Ignore
+	public void testResourceRequirementReductionDuringAllocation() throws Exception {

Review comment:
       No, that's why it I added `@Ignore` initially, but I never got around to deleting it.
   
   Once the requirements are reduced the SlotManager knows that the pending slots exceeds the requirements, but takes no further action. The TaskExecutor will offer the slot to the JM, which may or may not fail. In any case the TM will inform the SM about the resulting state, so we're good in regards to correctness. (And we'd introduce an odd edge-case where the SM actually frees a slot on the TM)




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13553: [FLINK-19306][coordination] Add DeclarativeSlotManager

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13553:
URL: https://github.com/apache/flink/pull/13553#issuecomment-704936985


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "15d4c353c4c0a646b2bdf34bbedbd81359cb5b5f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7264",
       "triggerID" : "15d4c353c4c0a646b2bdf34bbedbd81359cb5b5f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15d4c353c4c0a646b2bdf34bbedbd81359cb5b5f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7283",
       "triggerID" : "705384887",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "15d4c353c4c0a646b2bdf34bbedbd81359cb5b5f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7264",
       "triggerID" : "705384887",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 15d4c353c4c0a646b2bdf34bbedbd81359cb5b5f Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7283) Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7264) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] tillrohrmann commented on a change in pull request #13553: [FLINK-19306][coordination] Add DeclarativeSlotManager

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #13553:
URL: https://github.com/apache/flink/pull/13553#discussion_r502529140



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManager.java
##########
@@ -0,0 +1,609 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.metrics.MetricNames;
+import org.apache.flink.runtime.metrics.groups.SlotManagerMetricGroup;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerId;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.slots.ResourceCounter;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+import org.apache.flink.runtime.slots.ResourceRequirements;
+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.SlotOccupiedException;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ScheduledFuture;
+import java.util.function.BiFunction;
+import java.util.stream.Collectors;
+
+/**
+ * Implementation of {@link SlotManager} supporting declarative slot management.
+ */
+public class DeclarativeSlotManager implements SlotManager {
+	private static final Logger LOG = LoggerFactory.getLogger(DeclarativeSlotManager.class);
+
+	private final SlotTracker slotTracker;
+	private final ResourceTracker resourceTracker;
+	private final BiFunction<Executor, ResourceActions, TaskExecutorManager> taskExecutorManagerFactory;
+	private TaskExecutorManager taskExecutorManager;
+
+	/** Timeout for slot requests to the task manager. */
+	private final Time taskManagerRequestTimeout;
+	private ScheduledFuture<?> slotRequestTimeoutCheck;
+
+	private final SlotMatchingStrategy slotMatchingStrategy;
+
+	private final SlotManagerMetricGroup slotManagerMetricGroup;
+
+	private final Map<JobID, String> jobMasterTargetAddresses = new HashMap<>();
+	private final HashMap<SlotID, CompletableFuture<Acknowledge>> pendingSlotAllocationFutures;
+
+	/** ResourceManager's id. */
+	private ResourceManagerId resourceManagerId;
+
+	/** Executor for future callbacks which have to be "synchronized". */
+	private Executor mainThreadExecutor;
+
+	/** Callbacks for resource (de-)allocations. */
+	private ResourceActions resourceActions;
+
+	/** True iff the component has been started. */
+	private boolean started;
+
+	public DeclarativeSlotManager(
+			ScheduledExecutor scheduledExecutor,
+			SlotManagerConfiguration slotManagerConfiguration,
+			SlotManagerMetricGroup slotManagerMetricGroup,
+			ResourceTracker resourceTracker,
+			SlotTracker slotTracker) {
+
+		Preconditions.checkNotNull(slotManagerConfiguration);
+		this.taskManagerRequestTimeout = slotManagerConfiguration.getTaskManagerRequestTimeout();
+		this.slotManagerMetricGroup = Preconditions.checkNotNull(slotManagerMetricGroup);
+		this.resourceTracker = Preconditions.checkNotNull(resourceTracker);
+
+		pendingSlotAllocationFutures = new HashMap<>(16);
+
+		this.slotTracker = Preconditions.checkNotNull(slotTracker);
+		slotTracker.registerSlotStatusUpdateListener(createSlotStatusUpdateListener());
+
+		slotMatchingStrategy = slotManagerConfiguration.getSlotMatchingStrategy();
+
+		taskExecutorManagerFactory = (executor, resourceActions) -> new TaskExecutorManager(
+			slotManagerConfiguration.getDefaultWorkerResourceSpec(),
+			slotManagerConfiguration.getNumSlotsPerWorker(),
+			slotManagerConfiguration.getMaxSlotNum(),
+			slotManagerConfiguration.isWaitResultConsumedBeforeRelease(),
+			slotManagerConfiguration.getRedundantTaskManagerNum(),
+			slotManagerConfiguration.getTaskManagerTimeout(),
+			scheduledExecutor,
+			executor,
+			resourceActions);
+
+		resourceManagerId = null;
+		resourceActions = null;
+		mainThreadExecutor = null;
+		slotRequestTimeoutCheck = null;
+		taskExecutorManager = null;
+
+		started = false;
+	}
+
+	private SlotStatusUpdateListener createSlotStatusUpdateListener() {
+		return (taskManagerSlot, previous, current, jobId) -> {
+			if (previous == SlotState.PENDING) {
+				cancelAllocationFuture(taskManagerSlot.getSlotId());
+			}
+
+			if (current == SlotState.PENDING) {
+				resourceTracker.notifyAcquiredResource(jobId, taskManagerSlot.getResourceProfile());
+			}
+			if (current == SlotState.FREE) {
+				resourceTracker.notifyLostResource(jobId, taskManagerSlot.getResourceProfile());
+			}
+
+			if (current == SlotState.ALLOCATED) {
+				taskExecutorManager.occupySlot(taskManagerSlot.getInstanceId());
+			}
+			if (previous == SlotState.ALLOCATED && current == SlotState.FREE) {
+				taskExecutorManager.freeSlot(taskManagerSlot.getInstanceId());
+			}
+		};
+	}
+
+	private void cancelAllocationFuture(SlotID slotId) {
+		final CompletableFuture<Acknowledge> acknowledgeCompletableFuture = pendingSlotAllocationFutures.remove(slotId);
+		// the future may be null if we are just re-playing the state transitions due to a slot report
+		if (acknowledgeCompletableFuture != null) {
+			acknowledgeCompletableFuture.cancel(false);
+		}
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Component lifecycle methods
+	// ---------------------------------------------------------------------------------------------
+
+	/**
+	 * Starts the slot manager with the given leader id and resource manager actions.
+	 *
+	 * @param newResourceManagerId to use for communication with the task managers
+	 * @param newMainThreadExecutor to use to run code in the ResourceManager's main thread
+	 * @param newResourceActions to use for resource (de-)allocations
+	 */
+	@Override
+	public void start(ResourceManagerId newResourceManagerId, Executor newMainThreadExecutor, ResourceActions newResourceActions) {
+		LOG.info("Starting the slot manager.");
+
+		this.resourceManagerId = Preconditions.checkNotNull(newResourceManagerId);
+		mainThreadExecutor = Preconditions.checkNotNull(newMainThreadExecutor);
+		resourceActions = Preconditions.checkNotNull(newResourceActions);
+		taskExecutorManager = taskExecutorManagerFactory.apply(newMainThreadExecutor, newResourceActions);
+
+		started = true;
+
+		registerSlotManagerMetrics();
+	}
+
+	private void registerSlotManagerMetrics() {
+		slotManagerMetricGroup.gauge(
+			MetricNames.TASK_SLOTS_AVAILABLE,
+			() -> (long) getNumberFreeSlots());
+		slotManagerMetricGroup.gauge(
+			MetricNames.TASK_SLOTS_TOTAL,
+			() -> (long) getNumberRegisteredSlots());
+	}
+
+	/**
+	 * Suspends the component. This clears the internal state of the slot manager.
+	 */
+	@Override
+	public void suspend() {
+		LOG.info("Suspending the slot manager.");
+
+		if (slotRequestTimeoutCheck != null) {
+			slotRequestTimeoutCheck.cancel(false);
+			slotRequestTimeoutCheck = null;
+		}
+
+		resourceTracker.clear();
+		taskExecutorManager.close();
+
+		for (InstanceID registeredTaskManager : taskExecutorManager.getTaskExecutors()) {
+			unregisterTaskManager(registeredTaskManager, new SlotManagerException("The slot manager is being suspended."));
+		}
+
+		taskExecutorManager = null;
+		resourceManagerId = null;
+		resourceActions = null;
+		started = false;
+	}
+
+	/**
+	 * Closes the slot manager.
+	 *
+	 * @throws Exception if the close operation fails
+	 */
+	@Override
+	public void close() throws Exception {
+		LOG.info("Closing the slot manager.");
+
+		suspend();
+		slotManagerMetricGroup.close();
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Public API
+	// ---------------------------------------------------------------------------------------------
+
+	@Override
+	public void processResourceRequirements(ResourceRequirements resourceRequirements) {
+		checkInit();
+		LOG.debug("Received resource requirements from job {}: {}", resourceRequirements.getJobId(), resourceRequirements.getResourceRequirements());
+
+		if (resourceRequirements.getResourceRequirements().isEmpty()) {
+			jobMasterTargetAddresses.remove(resourceRequirements.getJobId());
+		} else {
+			jobMasterTargetAddresses.put(resourceRequirements.getJobId(), resourceRequirements.getTargetAddress());
+		}
+		resourceTracker.notifyResourceRequirements(resourceRequirements.getJobId(), resourceRequirements.getResourceRequirements());
+		checkResourceRequirements();
+	}
+
+	/**
+	 * 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
+	 * @return True if the task manager has not been registered before and is registered successfully; otherwise false
+	 */
+	@Override
+	public boolean registerTaskManager(final TaskExecutorConnection taskExecutorConnection, SlotReport initialSlotReport) {
+		checkInit();
+		LOG.debug("Registering task executor {} under {} at the slot manager.", taskExecutorConnection.getResourceID(), taskExecutorConnection.getInstanceID());
+
+		// we identify task managers by their instance id
+		if (taskExecutorManager.isTaskManagerRegistered(taskExecutorConnection.getInstanceID())) {
+			LOG.debug("Task executor {} was already registered.", taskExecutorConnection.getResourceID());
+			reportSlotStatus(taskExecutorConnection.getInstanceID(), initialSlotReport);
+			return false;
+		} else {
+			if (!taskExecutorManager.registerTaskManager(taskExecutorConnection, initialSlotReport)) {
+				LOG.debug("Task executor {} could not be registered.", taskExecutorConnection.getResourceID());
+				return false;
+			}
+
+			// register the new slots
+			for (SlotStatus slotStatus : initialSlotReport) {
+				slotTracker.addSlot(
+					slotStatus.getSlotID(),
+					slotStatus.getResourceProfile(),
+					taskExecutorConnection,
+					slotStatus.getJobID());
+			}
+
+			checkResourceRequirements();
+			return true;
+		}
+	}
+
+	@Override
+	public boolean unregisterTaskManager(InstanceID instanceId, Exception cause) {
+		checkInit();
+
+		LOG.debug("Unregistering task executor {} from the slot manager.", instanceId);
+
+		if (taskExecutorManager.isTaskManagerRegistered(instanceId)) {
+			slotTracker.removeSlots(taskExecutorManager.getSlotsOf(instanceId));
+			taskExecutorManager.unregisterTaskExecutor(instanceId);
+			checkResourceRequirements();
+
+			return true;
+		} else {
+			LOG.debug("There is no task executor registered with instance ID {}. Ignoring this message.", instanceId);
+
+			return false;
+		}
+	}
+
+	/**
+	 * Reports the current slot allocations for a task manager identified by the given instance id.
+	 *
+	 * @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
+	 */
+	@Override
+	public boolean reportSlotStatus(InstanceID instanceId, SlotReport slotReport) {
+		checkInit();
+
+		LOG.debug("Received slot report from instance {}: {}.", instanceId, slotReport);
+
+		if (taskExecutorManager.isTaskManagerRegistered(instanceId)) {
+			slotTracker.notifySlotStatus(slotReport);
+			checkResourceRequirements();
+			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
+	 */
+	@Override
+	public void freeSlot(SlotID slotId, AllocationID allocationId) {
+		checkInit();
+		LOG.debug("Freeing slot {}.", slotId);
+
+		slotTracker.notifyFree(slotId);
+		checkResourceRequirements();
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Requirement matching
+	// ---------------------------------------------------------------------------------------------
+
+	private void checkResourceRequirements() {
+		final Map<JobID, Collection<ResourceRequirement>> missingResources = resourceTracker.getMissingResources();
+		if (missingResources.isEmpty()) {
+			return;
+		}
+
+		final Map<JobID, ResourceCounter> outstandingRequirements = new LinkedHashMap<>();
+		for (Map.Entry<JobID, Collection<ResourceRequirement>> resourceRequirements : missingResources.entrySet()) {
+			JobID jobId = resourceRequirements.getKey();
+
+			for (ResourceRequirement resourceRequirement : resourceRequirements.getValue()) {
+				int numMissingSlots = internalRequestSlots(jobId, jobMasterTargetAddresses.get(jobId), resourceRequirement);
+				if (numMissingSlots > 0) {
+					outstandingRequirements
+						.computeIfAbsent(jobId, ignored -> new ResourceCounter())
+						.incrementCount(resourceRequirement.getResourceProfile(), numMissingSlots);
+				}
+			}
+		}
+
+		final ResourceCounter pendingSlots = new ResourceCounter(taskExecutorManager.getPendingTaskManagerSlots().stream().collect(
+			Collectors.groupingBy(
+				PendingTaskManagerSlot::getResourceProfile,
+				Collectors.summingInt(x -> 1))));
+
+		for (Map.Entry<JobID, ResourceCounter> unfulfilledRequirement : outstandingRequirements.entrySet()) {
+			tryFulfillRequirementsWithPendingOrNewSlots(
+				unfulfilledRequirement.getKey(),
+				unfulfilledRequirement.getValue().getResourceProfilesWithCount(),
+				pendingSlots);
+		}
+	}
+
+	/**
+	 * Tries to allocate slots for the given requirement. If there are not enough slots available, the
+	 * resource manager is informed to allocate more resources.
+	 *
+	 * @param jobId job to allocate slots for
+	 * @param targetAddress address of the jobmaster
+	 * @param resourceRequirement required slots
+	 * @return the number of missing slots
+	 */
+	private int internalRequestSlots(JobID jobId, String targetAddress, ResourceRequirement resourceRequirement) {
+		final ResourceProfile requiredResource = resourceRequirement.getResourceProfile();
+		Collection<TaskManagerSlotInformation> freeSlots = slotTracker.getFreeSlots();
+
+		int numUnfulfilled = 0;
+		for (int x = 0; x < resourceRequirement.getNumberOfRequiredSlots(); x++) {
+
+			final Optional<TaskManagerSlotInformation> reservedSlot = slotMatchingStrategy.findMatchingSlot(requiredResource, freeSlots, this::getNumberRegisteredSlotsOf);
+			if (reservedSlot.isPresent()) {
+				// we do not need to modify freeSlots because it is indirectly modified by the allocation
+				allocateSlot(reservedSlot.get(), jobId, targetAddress, requiredResource);
+			} else {
+				numUnfulfilled++;
+			}
+		}
+		return numUnfulfilled;
+	}
+
+	/**
+	 * Allocates the given slot. This entails sending a registration message to the task manager and treating failures.
+	 *
+	 * @param taskManagerSlot slot to allocate
+	 * @param jobId job for which the slot should be allocated for
+	 * @param targetAddress address of the job master
+	 * @param resourceProfile resource profile for the requirement for which the slot is used
+	 */
+	private void allocateSlot(TaskManagerSlotInformation taskManagerSlot, JobID jobId, String targetAddress, ResourceProfile resourceProfile) {
+		TaskExecutorConnection taskExecutorConnection = taskManagerSlot.getTaskManagerConnection();
+		TaskExecutorGateway gateway = taskExecutorConnection.getTaskExecutorGateway();
+
+		final CompletableFuture<Acknowledge> completableFuture = new CompletableFuture<>();
+		final SlotID slotId = taskManagerSlot.getSlotId();
+		final InstanceID instanceId = taskManagerSlot.getInstanceId();
+
+		LOG.debug("Starting allocation of slot {} for job {} with resource profile {}.", taskManagerSlot.getSlotId(), jobId, resourceProfile);
+
+		slotTracker.notifyAllocationStart(slotId, jobId);
+
+		if (!taskExecutorManager.isTaskManagerRegistered(instanceId)) {
+			throw new IllegalStateException("Could not find a registered task manager for instance id " +
+				instanceId + '.');
+		}
+
+		taskExecutorManager.markUsed(instanceId);
+
+		// RPC call to the task manager
+		CompletableFuture<Acknowledge> requestFuture = gateway.requestSlot(
+			slotId,
+			jobId,
+			new AllocationID(),
+			resourceProfile,
+			targetAddress,
+			resourceManagerId,
+			taskManagerRequestTimeout);
+
+		requestFuture.whenComplete(
+			(Acknowledge acknowledge, Throwable throwable) -> {
+				if (acknowledge != null) {
+					completableFuture.complete(acknowledge);
+				} else {
+					completableFuture.completeExceptionally(throwable);
+				}
+			});
+
+		completableFuture.whenCompleteAsync(
+			(Acknowledge acknowledge, Throwable throwable) -> {
+				try {
+					if (acknowledge != null) {
+						LOG.trace("Completed allocation of slot {} for job {}.", slotId, jobId);
+						slotTracker.notifyAllocationComplete(slotId, jobId);
+					} else {
+						if (throwable instanceof SlotOccupiedException) {
+							SlotOccupiedException exception = (SlotOccupiedException) throwable;
+							LOG.debug("Tried allocating slot {} for job {}, but it was already allocated for job {}.", slotId, jobId, exception.getJobId());
+							// report as a slot status to force the state transition
+							// this could be a problem if we ever assume that the task executor always reports about all slots
+							slotTracker.notifySlotStatus(Collections.singleton(new SlotStatus(slotId, taskManagerSlot.getResourceProfile(), exception.getJobId(), exception.getAllocationId())));
+						} else {
+							if (throwable instanceof CancellationException) {
+								LOG.debug("Cancelled allocation of slot {} for job {}.", slotId, jobId, throwable);
+							} else {
+								LOG.warn("Slot allocation for slot {} for job {} failed.", slotId, jobId, throwable);
+								slotTracker.notifyFree(slotId);
+							}
+						}
+					}
+					checkResourceRequirements();
+				} catch (Exception e) {
+					LOG.error("Error while completing the slot allocation.", e);
+				}
+			},
+			mainThreadExecutor);
+	}
+
+	private void tryFulfillRequirementsWithPendingOrNewSlots(JobID jobId, Map<ResourceProfile, Integer> missingResources, ResourceCounter pendingSlots) {

Review comment:
       Then let's change the name of the method.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] zentol commented on a change in pull request #13553: [FLINK-19306][coordination] Add DeclarativeSlotManager

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #13553:
URL: https://github.com/apache/flink/pull/13553#discussion_r503793724



##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManagerTest.java
##########
@@ -0,0 +1,1092 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple6;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.concurrent.ManuallyTriggeredScheduledExecutor;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerId;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+import org.apache.flink.runtime.slots.ResourceRequirements;
+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.TestingTaskExecutorGateway;
+import org.apache.flink.runtime.taskexecutor.TestingTaskExecutorGatewayBuilder;
+import org.apache.flink.runtime.taskexecutor.exceptions.SlotAllocationException;
+import org.apache.flink.runtime.taskexecutor.exceptions.SlotOccupiedException;
+import org.apache.flink.runtime.testingUtils.TestingUtils;
+import org.apache.flink.util.FlinkException;
+import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.function.FunctionUtils;
+
+import akka.pattern.AskTimeoutException;
+import org.junit.Ignore;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.hamcrest.CoreMatchers.hasItem;
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.empty;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.hasSize;
+import static org.hamcrest.Matchers.is;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertSame;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Tests for the {@link DeclarativeSlotManager}.
+ */
+public class DeclarativeSlotManagerTest extends TestLogger {
+
+	private static final FlinkException TEST_EXCEPTION = new FlinkException("Test exception");
+
+	private static final WorkerResourceSpec WORKER_RESOURCE_SPEC = new WorkerResourceSpec.Builder()
+		.setCpuCores(100.0)
+		.setTaskHeapMemoryMB(10000)
+		.setTaskOffHeapMemoryMB(10000)
+		.setNetworkMemoryMB(10000)
+		.setManagedMemoryMB(10000)
+		.build();
+
+	/**
+	 * Tests that we can register task manager and their slots at the slot manager.
+	 */
+	@Test
+	public void testTaskManagerRegistration() throws Exception {
+		final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder().createTestingTaskExecutorGateway();
+		final ResourceID resourceId = ResourceID.generate();
+		final TaskExecutorConnection taskManagerConnection = new TaskExecutorConnection(resourceId, taskExecutorGateway);
+
+		final SlotID slotId1 = new SlotID(resourceId, 0);
+		final SlotID slotId2 = new SlotID(resourceId, 1);
+		final SlotReport slotReport = new SlotReport(Arrays.asList(createFreeSlotStatus(slotId1), createFreeSlotStatus(slotId2)));
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport);
+
+			assertThat("The number registered slots does not equal the expected number.", slotManager.getNumberRegisteredSlots(), is(2));
+
+			assertNotNull(slotTracker.getSlot(slotId1));
+			assertNotNull(slotTracker.getSlot(slotId2));
+		}
+	}
+
+	/**
+	 * Tests that un-registration of task managers will free and remove all registered slots.
+	 */
+	@Test
+	public void testTaskManagerUnregistration() throws Exception {
+		final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder()
+			.setRequestSlotFunction(tuple6 -> new CompletableFuture<>())
+			.createTestingTaskExecutorGateway();
+		final TaskExecutorConnection taskManagerConnection = createTaskExecutorConnection(taskExecutorGateway);
+		final ResourceID resourceId = taskManagerConnection.getResourceID();
+
+		final SlotID slotId1 = new SlotID(resourceId, 0);
+		final SlotID slotId2 = new SlotID(resourceId, 1);
+		final SlotReport slotReport = new SlotReport(Arrays.asList(createAllocatedSlotStatus(slotId1), createFreeSlotStatus(slotId2)));
+
+		final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot();
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport);
+
+			assertEquals("The number registered slots does not equal the expected number.", 2, slotManager.getNumberRegisteredSlots());
+
+			DeclarativeTaskManagerSlot slot1 = slotTracker.getSlot(slotId1);
+			DeclarativeTaskManagerSlot slot2 = slotTracker.getSlot(slotId2);
+
+			assertSame(SlotState.ALLOCATED, slot1.getState());
+			assertSame(SlotState.FREE, slot2.getState());
+
+			slotManager.processResourceRequirements(resourceRequirements);
+
+			assertSame(SlotState.PENDING, slot2.getState());
+
+			slotManager.unregisterTaskManager(taskManagerConnection.getInstanceID(), TEST_EXCEPTION);
+
+			assertEquals(0, slotManager.getNumberRegisteredSlots());
+		}
+	}
+
+	/**
+	 * Tests that a slot request with no free slots will trigger the resource allocation.
+	 */
+	@Test
+	public void testSlotRequestWithoutFreeSlots() throws Exception {
+		final ResourceManagerId resourceManagerId = ResourceManagerId.generate();
+
+		final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot();
+
+		CompletableFuture<WorkerResourceSpec> allocateResourceFuture = new CompletableFuture<>();
+		ResourceActions resourceManagerActions = new TestingResourceActionsBuilder()
+			.setAllocateResourceConsumer(allocateResourceFuture::complete)
+			.build();
+
+		try (SlotManager slotManager = createSlotManager(resourceManagerId, resourceManagerActions)) {
+
+			slotManager.processResourceRequirements(resourceRequirements);
+
+			allocateResourceFuture.get();
+		}
+	}
+
+	/**
+	 * Tests that resources continue to be considered missing if we cannot allocate more resources.
+	 */
+	@Test
+	public void testResourceDeclarationWithResourceAllocationFailure() throws Exception {
+		final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot();
+
+		ResourceActions resourceManagerActions = new TestingResourceActionsBuilder()
+			.setAllocateResourceFunction(value -> false)
+			.build();
+
+		final ResourceTracker resourceTracker = new DefaultResourceTracker();
+
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setResourceTracker(resourceTracker)
+			.buildAndStartWithDirectExec(ResourceManagerId.generate(), resourceManagerActions)) {
+
+			slotManager.processResourceRequirements(resourceRequirements);
+
+			final JobID jobId = resourceRequirements.getJobId();
+			assertThat(getTotalResourceCount(resourceTracker.getMissingResources().get(jobId)), is(1));
+		}
+	}
+
+	/**
+	 * Tests that a slot request which can be fulfilled will trigger a slot allocation.
+	 */
+	@Test
+	public void testSlotRequestWithFreeSlot() throws Exception {
+		final ResourceManagerId resourceManagerId = ResourceManagerId.generate();
+		final ResourceID resourceID = ResourceID.generate();
+		final JobID jobId = new JobID();
+		final SlotID slotId = new SlotID(resourceID, 0);
+		final String targetAddress = "localhost";
+		final ResourceProfile resourceProfile = ResourceProfile.fromResources(42.0, 1337);
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec(resourceManagerId, new TestingResourceActionsBuilder().build())) {
+
+			final CompletableFuture<Tuple6<SlotID, JobID, AllocationID, ResourceProfile, String, ResourceManagerId>> requestFuture = new CompletableFuture<>();
+			// accept an incoming slot request
+			final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder()
+				.setRequestSlotFunction(tuple6 -> {
+					requestFuture.complete(Tuple6.of(tuple6.f0, tuple6.f1, tuple6.f2, tuple6.f3, tuple6.f4, tuple6.f5));
+					return CompletableFuture.completedFuture(Acknowledge.get());
+				})
+				.createTestingTaskExecutorGateway();
+
+			final TaskExecutorConnection taskExecutorConnection = new TaskExecutorConnection(resourceID, taskExecutorGateway);
+
+			final SlotStatus slotStatus = new SlotStatus(slotId, resourceProfile);
+			final SlotReport slotReport = new SlotReport(slotStatus);
+
+			slotManager.registerTaskManager(
+				taskExecutorConnection,
+				slotReport);
+
+			ResourceRequirements requirements = ResourceRequirements.create(
+				jobId,
+				targetAddress,
+				Collections.singleton(ResourceRequirement.create(resourceProfile, 1)));
+			slotManager.processResourceRequirements(requirements);
+
+			assertThat(requestFuture.get(), is(equalTo(Tuple6.of(slotId, jobId, requestFuture.get().f2, resourceProfile, targetAddress, resourceManagerId))));
+
+			DeclarativeTaskManagerSlot slot = slotTracker.getSlot(slotId);
+
+			assertEquals("The slot has not been allocated to the expected allocation id.", jobId, slot.getJobId());
+		}
+	}
+
+	/**
+	 * Checks that reducing resource requirements while a slot allocation is in-progress results in the slot being freed
+	 * one the allocation completes.
+	 */
+	@Test
+	@Ignore
+	public void testResourceRequirementReductionDuringAllocation() throws Exception {
+		final TaskExecutorConnection taskManagerConnection = createTaskExecutorConnection();
+		final ResourceID resourceID = taskManagerConnection.getResourceID();
+		final SlotID slotId = new SlotID(resourceID, 0);
+
+		final SlotReport slotReport = new SlotReport(createFreeSlotStatus(slotId));
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport);
+
+			DeclarativeTaskManagerSlot slot = slotTracker.getSlot(slotId);
+
+			final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot();
+			slotManager.processResourceRequirements(resourceRequirements);
+
+			assertSame(SlotState.PENDING, slot.getState());
+
+			slotManager.processResourceRequirements(ResourceRequirements.create(resourceRequirements.getJobId(), "foobar", Collections.emptyList()));
+
+			slot = slotTracker.getSlot(slotId);
+
+			assertSame(SlotState.FREE, slot.getState());
+		}
+	}
+
+	/**
+	 * Tests that pending slot requests are tried to be fulfilled upon new slot registrations.
+	 */
+	@Test
+	public void testFulfillingPendingSlotRequest() throws Exception {
+		final ResourceManagerId resourceManagerId = ResourceManagerId.generate();
+		final ResourceID resourceID = ResourceID.generate();
+		final JobID jobId = new JobID();
+		final SlotID slotId = new SlotID(resourceID, 0);
+		final String targetAddress = "localhost";
+		final ResourceProfile resourceProfile = ResourceProfile.fromResources(42.0, 1337);
+
+		final AtomicInteger numberAllocateResourceCalls = new AtomicInteger(0);
+		ResourceActions resourceManagerActions = new TestingResourceActionsBuilder()
+			.setAllocateResourceConsumer(ignored -> numberAllocateResourceCalls.incrementAndGet())
+			.build();
+
+		final CompletableFuture<Tuple6<SlotID, JobID, AllocationID, ResourceProfile, String, ResourceManagerId>> requestFuture = new CompletableFuture<>();
+		// accept an incoming slot request
+		final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder()
+			.setRequestSlotFunction(tuple6 -> {
+				requestFuture.complete(Tuple6.of(tuple6.f0, tuple6.f1, tuple6.f2, tuple6.f3, tuple6.f4, tuple6.f5));
+				return CompletableFuture.completedFuture(Acknowledge.get());
+			})
+			.createTestingTaskExecutorGateway();
+
+		final TaskExecutorConnection taskExecutorConnection = new TaskExecutorConnection(resourceID, taskExecutorGateway);
+
+		final SlotStatus slotStatus = new SlotStatus(slotId, resourceProfile);
+		final SlotReport slotReport = new SlotReport(slotStatus);

Review comment:
       They are; the only difference is the order in which requirements are declared / the task executor is registered. I'll check whether we can't de-duplicate things a bit.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] zentol commented on a change in pull request #13553: [FLINK-19306][coordination] Add DeclarativeSlotManager

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #13553:
URL: https://github.com/apache/flink/pull/13553#discussion_r502340101



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManager.java
##########
@@ -0,0 +1,609 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.metrics.MetricNames;
+import org.apache.flink.runtime.metrics.groups.SlotManagerMetricGroup;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerId;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.slots.ResourceCounter;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+import org.apache.flink.runtime.slots.ResourceRequirements;
+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.SlotOccupiedException;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ScheduledFuture;
+import java.util.function.BiFunction;
+import java.util.stream.Collectors;
+
+/**
+ * Implementation of {@link SlotManager} supporting declarative slot management.
+ */
+public class DeclarativeSlotManager implements SlotManager {
+	private static final Logger LOG = LoggerFactory.getLogger(DeclarativeSlotManager.class);
+
+	private final SlotTracker slotTracker;
+	private final ResourceTracker resourceTracker;
+	private final BiFunction<Executor, ResourceActions, TaskExecutorManager> taskExecutorManagerFactory;
+	private TaskExecutorManager taskExecutorManager;
+
+	/** Timeout for slot requests to the task manager. */
+	private final Time taskManagerRequestTimeout;
+	private ScheduledFuture<?> slotRequestTimeoutCheck;
+
+	private final SlotMatchingStrategy slotMatchingStrategy;
+
+	private final SlotManagerMetricGroup slotManagerMetricGroup;
+
+	private final Map<JobID, String> jobMasterTargetAddresses = new HashMap<>();
+	private final HashMap<SlotID, CompletableFuture<Acknowledge>> pendingSlotAllocationFutures;
+
+	/** ResourceManager's id. */
+	private ResourceManagerId resourceManagerId;
+
+	/** Executor for future callbacks which have to be "synchronized". */
+	private Executor mainThreadExecutor;
+
+	/** Callbacks for resource (de-)allocations. */
+	private ResourceActions resourceActions;
+
+	/** True iff the component has been started. */
+	private boolean started;
+
+	public DeclarativeSlotManager(
+			ScheduledExecutor scheduledExecutor,
+			SlotManagerConfiguration slotManagerConfiguration,
+			SlotManagerMetricGroup slotManagerMetricGroup,
+			ResourceTracker resourceTracker,
+			SlotTracker slotTracker) {
+
+		Preconditions.checkNotNull(slotManagerConfiguration);
+		this.taskManagerRequestTimeout = slotManagerConfiguration.getTaskManagerRequestTimeout();
+		this.slotManagerMetricGroup = Preconditions.checkNotNull(slotManagerMetricGroup);
+		this.resourceTracker = Preconditions.checkNotNull(resourceTracker);
+
+		pendingSlotAllocationFutures = new HashMap<>(16);
+
+		this.slotTracker = Preconditions.checkNotNull(slotTracker);
+		slotTracker.registerSlotStatusUpdateListener(createSlotStatusUpdateListener());
+
+		slotMatchingStrategy = slotManagerConfiguration.getSlotMatchingStrategy();
+
+		taskExecutorManagerFactory = (executor, resourceActions) -> new TaskExecutorManager(
+			slotManagerConfiguration.getDefaultWorkerResourceSpec(),
+			slotManagerConfiguration.getNumSlotsPerWorker(),
+			slotManagerConfiguration.getMaxSlotNum(),
+			slotManagerConfiguration.isWaitResultConsumedBeforeRelease(),
+			slotManagerConfiguration.getRedundantTaskManagerNum(),
+			slotManagerConfiguration.getTaskManagerTimeout(),
+			scheduledExecutor,
+			executor,
+			resourceActions);
+
+		resourceManagerId = null;
+		resourceActions = null;
+		mainThreadExecutor = null;
+		slotRequestTimeoutCheck = null;
+		taskExecutorManager = null;
+
+		started = false;
+	}
+
+	private SlotStatusUpdateListener createSlotStatusUpdateListener() {
+		return (taskManagerSlot, previous, current, jobId) -> {
+			if (previous == SlotState.PENDING) {
+				cancelAllocationFuture(taskManagerSlot.getSlotId());
+			}
+
+			if (current == SlotState.PENDING) {
+				resourceTracker.notifyAcquiredResource(jobId, taskManagerSlot.getResourceProfile());
+			}
+			if (current == SlotState.FREE) {
+				resourceTracker.notifyLostResource(jobId, taskManagerSlot.getResourceProfile());
+			}
+
+			if (current == SlotState.ALLOCATED) {
+				taskExecutorManager.occupySlot(taskManagerSlot.getInstanceId());
+			}
+			if (previous == SlotState.ALLOCATED && current == SlotState.FREE) {
+				taskExecutorManager.freeSlot(taskManagerSlot.getInstanceId());
+			}
+		};
+	}
+
+	private void cancelAllocationFuture(SlotID slotId) {
+		final CompletableFuture<Acknowledge> acknowledgeCompletableFuture = pendingSlotAllocationFutures.remove(slotId);
+		// the future may be null if we are just re-playing the state transitions due to a slot report
+		if (acknowledgeCompletableFuture != null) {
+			acknowledgeCompletableFuture.cancel(false);
+		}
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Component lifecycle methods
+	// ---------------------------------------------------------------------------------------------
+
+	/**
+	 * Starts the slot manager with the given leader id and resource manager actions.
+	 *
+	 * @param newResourceManagerId to use for communication with the task managers
+	 * @param newMainThreadExecutor to use to run code in the ResourceManager's main thread
+	 * @param newResourceActions to use for resource (de-)allocations
+	 */
+	@Override
+	public void start(ResourceManagerId newResourceManagerId, Executor newMainThreadExecutor, ResourceActions newResourceActions) {
+		LOG.info("Starting the slot manager.");
+
+		this.resourceManagerId = Preconditions.checkNotNull(newResourceManagerId);
+		mainThreadExecutor = Preconditions.checkNotNull(newMainThreadExecutor);
+		resourceActions = Preconditions.checkNotNull(newResourceActions);
+		taskExecutorManager = taskExecutorManagerFactory.apply(newMainThreadExecutor, newResourceActions);
+
+		started = true;
+
+		registerSlotManagerMetrics();
+	}
+
+	private void registerSlotManagerMetrics() {
+		slotManagerMetricGroup.gauge(
+			MetricNames.TASK_SLOTS_AVAILABLE,
+			() -> (long) getNumberFreeSlots());
+		slotManagerMetricGroup.gauge(
+			MetricNames.TASK_SLOTS_TOTAL,
+			() -> (long) getNumberRegisteredSlots());
+	}
+
+	/**
+	 * Suspends the component. This clears the internal state of the slot manager.
+	 */
+	@Override
+	public void suspend() {
+		LOG.info("Suspending the slot manager.");
+
+		if (slotRequestTimeoutCheck != null) {
+			slotRequestTimeoutCheck.cancel(false);
+			slotRequestTimeoutCheck = null;
+		}
+
+		resourceTracker.clear();
+		taskExecutorManager.close();
+
+		for (InstanceID registeredTaskManager : taskExecutorManager.getTaskExecutors()) {
+			unregisterTaskManager(registeredTaskManager, new SlotManagerException("The slot manager is being suspended."));
+		}
+
+		taskExecutorManager = null;
+		resourceManagerId = null;
+		resourceActions = null;
+		started = false;
+	}
+
+	/**
+	 * Closes the slot manager.
+	 *
+	 * @throws Exception if the close operation fails
+	 */
+	@Override
+	public void close() throws Exception {
+		LOG.info("Closing the slot manager.");
+
+		suspend();
+		slotManagerMetricGroup.close();
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Public API
+	// ---------------------------------------------------------------------------------------------
+
+	@Override
+	public void processResourceRequirements(ResourceRequirements resourceRequirements) {
+		checkInit();
+		LOG.debug("Received resource requirements from job {}: {}", resourceRequirements.getJobId(), resourceRequirements.getResourceRequirements());
+
+		if (resourceRequirements.getResourceRequirements().isEmpty()) {
+			jobMasterTargetAddresses.remove(resourceRequirements.getJobId());
+		} else {
+			jobMasterTargetAddresses.put(resourceRequirements.getJobId(), resourceRequirements.getTargetAddress());
+		}
+		resourceTracker.notifyResourceRequirements(resourceRequirements.getJobId(), resourceRequirements.getResourceRequirements());
+		checkResourceRequirements();
+	}
+
+	/**
+	 * 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
+	 * @return True if the task manager has not been registered before and is registered successfully; otherwise false
+	 */
+	@Override
+	public boolean registerTaskManager(final TaskExecutorConnection taskExecutorConnection, SlotReport initialSlotReport) {
+		checkInit();
+		LOG.debug("Registering task executor {} under {} at the slot manager.", taskExecutorConnection.getResourceID(), taskExecutorConnection.getInstanceID());
+
+		// we identify task managers by their instance id
+		if (taskExecutorManager.isTaskManagerRegistered(taskExecutorConnection.getInstanceID())) {
+			LOG.debug("Task executor {} was already registered.", taskExecutorConnection.getResourceID());
+			reportSlotStatus(taskExecutorConnection.getInstanceID(), initialSlotReport);
+			return false;
+		} else {
+			if (!taskExecutorManager.registerTaskManager(taskExecutorConnection, initialSlotReport)) {
+				LOG.debug("Task executor {} could not be registered.", taskExecutorConnection.getResourceID());
+				return false;
+			}
+
+			// register the new slots
+			for (SlotStatus slotStatus : initialSlotReport) {
+				slotTracker.addSlot(
+					slotStatus.getSlotID(),
+					slotStatus.getResourceProfile(),
+					taskExecutorConnection,
+					slotStatus.getJobID());
+			}
+
+			checkResourceRequirements();
+			return true;
+		}
+	}
+
+	@Override
+	public boolean unregisterTaskManager(InstanceID instanceId, Exception cause) {
+		checkInit();
+
+		LOG.debug("Unregistering task executor {} from the slot manager.", instanceId);
+
+		if (taskExecutorManager.isTaskManagerRegistered(instanceId)) {
+			slotTracker.removeSlots(taskExecutorManager.getSlotsOf(instanceId));
+			taskExecutorManager.unregisterTaskExecutor(instanceId);
+			checkResourceRequirements();
+
+			return true;
+		} else {
+			LOG.debug("There is no task executor registered with instance ID {}. Ignoring this message.", instanceId);
+
+			return false;
+		}
+	}
+
+	/**
+	 * Reports the current slot allocations for a task manager identified by the given instance id.
+	 *
+	 * @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
+	 */
+	@Override
+	public boolean reportSlotStatus(InstanceID instanceId, SlotReport slotReport) {
+		checkInit();
+
+		LOG.debug("Received slot report from instance {}: {}.", instanceId, slotReport);
+
+		if (taskExecutorManager.isTaskManagerRegistered(instanceId)) {
+			slotTracker.notifySlotStatus(slotReport);
+			checkResourceRequirements();
+			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
+	 */
+	@Override
+	public void freeSlot(SlotID slotId, AllocationID allocationId) {
+		checkInit();
+		LOG.debug("Freeing slot {}.", slotId);
+
+		slotTracker.notifyFree(slotId);
+		checkResourceRequirements();
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Requirement matching
+	// ---------------------------------------------------------------------------------------------
+
+	private void checkResourceRequirements() {
+		final Map<JobID, Collection<ResourceRequirement>> missingResources = resourceTracker.getMissingResources();
+		if (missingResources.isEmpty()) {
+			return;
+		}
+
+		final Map<JobID, ResourceCounter> outstandingRequirements = new LinkedHashMap<>();
+		for (Map.Entry<JobID, Collection<ResourceRequirement>> resourceRequirements : missingResources.entrySet()) {
+			JobID jobId = resourceRequirements.getKey();
+
+			for (ResourceRequirement resourceRequirement : resourceRequirements.getValue()) {
+				int numMissingSlots = internalRequestSlots(jobId, jobMasterTargetAddresses.get(jobId), resourceRequirement);
+				if (numMissingSlots > 0) {
+					outstandingRequirements
+						.computeIfAbsent(jobId, ignored -> new ResourceCounter())
+						.incrementCount(resourceRequirement.getResourceProfile(), numMissingSlots);
+				}
+			}
+		}
+
+		final ResourceCounter pendingSlots = new ResourceCounter(taskExecutorManager.getPendingTaskManagerSlots().stream().collect(
+			Collectors.groupingBy(
+				PendingTaskManagerSlot::getResourceProfile,
+				Collectors.summingInt(x -> 1))));
+
+		for (Map.Entry<JobID, ResourceCounter> unfulfilledRequirement : outstandingRequirements.entrySet()) {
+			tryFulfillRequirementsWithPendingOrNewSlots(
+				unfulfilledRequirement.getKey(),
+				unfulfilledRequirement.getValue().getResourceProfilesWithCount(),
+				pendingSlots);
+		}
+	}
+
+	/**
+	 * Tries to allocate slots for the given requirement. If there are not enough slots available, the
+	 * resource manager is informed to allocate more resources.
+	 *
+	 * @param jobId job to allocate slots for
+	 * @param targetAddress address of the jobmaster
+	 * @param resourceRequirement required slots
+	 * @return the number of missing slots
+	 */
+	private int internalRequestSlots(JobID jobId, String targetAddress, ResourceRequirement resourceRequirement) {
+		final ResourceProfile requiredResource = resourceRequirement.getResourceProfile();
+		Collection<TaskManagerSlotInformation> freeSlots = slotTracker.getFreeSlots();
+
+		int numUnfulfilled = 0;
+		for (int x = 0; x < resourceRequirement.getNumberOfRequiredSlots(); x++) {
+
+			final Optional<TaskManagerSlotInformation> reservedSlot = slotMatchingStrategy.findMatchingSlot(requiredResource, freeSlots, this::getNumberRegisteredSlotsOf);
+			if (reservedSlot.isPresent()) {
+				// we do not need to modify freeSlots because it is indirectly modified by the allocation
+				allocateSlot(reservedSlot.get(), jobId, targetAddress, requiredResource);
+			} else {
+				numUnfulfilled++;
+			}
+		}
+		return numUnfulfilled;
+	}
+
+	/**
+	 * Allocates the given slot. This entails sending a registration message to the task manager and treating failures.
+	 *
+	 * @param taskManagerSlot slot to allocate
+	 * @param jobId job for which the slot should be allocated for
+	 * @param targetAddress address of the job master
+	 * @param resourceProfile resource profile for the requirement for which the slot is used
+	 */
+	private void allocateSlot(TaskManagerSlotInformation taskManagerSlot, JobID jobId, String targetAddress, ResourceProfile resourceProfile) {
+		TaskExecutorConnection taskExecutorConnection = taskManagerSlot.getTaskManagerConnection();
+		TaskExecutorGateway gateway = taskExecutorConnection.getTaskExecutorGateway();
+
+		final CompletableFuture<Acknowledge> completableFuture = new CompletableFuture<>();
+		final SlotID slotId = taskManagerSlot.getSlotId();
+		final InstanceID instanceId = taskManagerSlot.getInstanceId();
+
+		LOG.debug("Starting allocation of slot {} for job {} with resource profile {}.", taskManagerSlot.getSlotId(), jobId, resourceProfile);
+
+		slotTracker.notifyAllocationStart(slotId, jobId);
+
+		if (!taskExecutorManager.isTaskManagerRegistered(instanceId)) {
+			throw new IllegalStateException("Could not find a registered task manager for instance id " +
+				instanceId + '.');
+		}
+
+		taskExecutorManager.markUsed(instanceId);
+
+		// RPC call to the task manager
+		CompletableFuture<Acknowledge> requestFuture = gateway.requestSlot(
+			slotId,
+			jobId,
+			new AllocationID(),
+			resourceProfile,
+			targetAddress,
+			resourceManagerId,
+			taskManagerRequestTimeout);
+
+		requestFuture.whenComplete(
+			(Acknowledge acknowledge, Throwable throwable) -> {
+				if (acknowledge != null) {
+					completableFuture.complete(acknowledge);
+				} else {
+					completableFuture.completeExceptionally(throwable);
+				}
+			});
+
+		completableFuture.whenCompleteAsync(
+			(Acknowledge acknowledge, Throwable throwable) -> {
+				try {
+					if (acknowledge != null) {
+						LOG.trace("Completed allocation of slot {} for job {}.", slotId, jobId);
+						slotTracker.notifyAllocationComplete(slotId, jobId);
+					} else {
+						if (throwable instanceof SlotOccupiedException) {
+							SlotOccupiedException exception = (SlotOccupiedException) throwable;
+							LOG.debug("Tried allocating slot {} for job {}, but it was already allocated for job {}.", slotId, jobId, exception.getJobId());
+							// report as a slot status to force the state transition
+							// this could be a problem if we ever assume that the task executor always reports about all slots
+							slotTracker.notifySlotStatus(Collections.singleton(new SlotStatus(slotId, taskManagerSlot.getResourceProfile(), exception.getJobId(), exception.getAllocationId())));
+						} else {
+							if (throwable instanceof CancellationException) {
+								LOG.debug("Cancelled allocation of slot {} for job {}.", slotId, jobId, throwable);
+							} else {
+								LOG.warn("Slot allocation for slot {} for job {} failed.", slotId, jobId, throwable);
+								slotTracker.notifyFree(slotId);
+							}
+						}
+					}
+					checkResourceRequirements();
+				} catch (Exception e) {
+					LOG.error("Error while completing the slot allocation.", e);
+				}
+			},
+			mainThreadExecutor);
+	}
+
+	private void tryFulfillRequirementsWithPendingOrNewSlots(JobID jobId, Map<ResourceProfile, Integer> missingResources, ResourceCounter pendingSlots) {
+		for (Map.Entry<ResourceProfile, Integer> missingResource : missingResources.entrySet()) {
+			ResourceProfile profile = missingResource.getKey();
+			for (int i = 0; i < missingResource.getValue(); i++) {
+				if (!tryFulfillWithPendingSlots(profile, pendingSlots)) {
+					Optional<ResourceRequirement> newlyFulfillableRequirements = taskExecutorManager.allocateWorker(profile);
+					if (newlyFulfillableRequirements.isPresent()) {
+						ResourceRequirement newSlots = newlyFulfillableRequirements.get();
+						// reserve one of the new slots
+						if (newSlots.getNumberOfRequiredSlots() > 1) {
+							pendingSlots.incrementCount(newSlots.getResourceProfile(), newSlots.getNumberOfRequiredSlots() - 1);
+						}
+					} else {
+						LOG.warn("Could not fulfill resource requirements of job {}.", jobId);
+						resourceActions.notifyNotEnoughResourcesAvailable(jobId, resourceTracker.getAcquiredResources(jobId));
+						return;
+					}
+				}
+			}
+		}
+	}
+
+	private boolean tryFulfillWithPendingSlots(ResourceProfile resourceProfile, ResourceCounter pendingSlots) {
+		Set<ResourceProfile> pendingSlotProfiles = pendingSlots.getResourceProfiles();
+
+		// short-cut, pretty much only applicable to fine-grained resource management
+		if (pendingSlotProfiles.contains(resourceProfile)) {
+			pendingSlots.decrementCount(resourceProfile, 1);
+			return true;
+		}
+
+		for (ResourceProfile pendingSlotProfile : pendingSlotProfiles) {
+			if (pendingSlotProfile.isMatching(resourceProfile)) {
+				pendingSlots.decrementCount(pendingSlotProfile, 1);
+				return true;
+			}
+		}
+
+		return false;
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Legacy APIs
+	// ---------------------------------------------------------------------------------------------
+
+	@Override
+	public int getNumberRegisteredSlots() {
+		return taskExecutorManager.getNumberRegisteredSlots();
+	}
+
+	@Override
+	public int getNumberRegisteredSlotsOf(InstanceID instanceId) {
+		return taskExecutorManager.getNumberRegisteredSlotsOf(instanceId);
+	}
+
+	@Override
+	public int getNumberFreeSlots() {
+		return taskExecutorManager.getNumberFreeSlots();
+	}
+
+	@Override
+	public int getNumberFreeSlotsOf(InstanceID instanceId) {
+		return taskExecutorManager.getNumberFreeSlotsOf(instanceId);
+	}
+
+	@Override
+	public Map<WorkerResourceSpec, Integer> getRequiredResources() {
+		return taskExecutorManager.getRequiredWorkers();
+	}
+
+	@Override
+	public ResourceProfile getRegisteredResource() {
+		return taskExecutorManager.getTotalRegisteredResources();
+	}
+
+	@Override
+	public ResourceProfile getRegisteredResourceOf(InstanceID instanceID) {
+		return taskExecutorManager.getTotalRegisteredResourcesOf(instanceID);
+	}
+
+	@Override
+	public ResourceProfile getFreeResource() {
+		return taskExecutorManager.getTotalFreeResources();
+	}
+
+	@Override
+	public ResourceProfile getFreeResourceOf(InstanceID instanceID) {
+		return taskExecutorManager.getTotalFreeResourcesOf(instanceID);
+	}
+
+	@Override
+	public void setFailUnfulfillableRequest(boolean failUnfulfillableRequest) {
+		// we always send notifications if we cannot fulfill requests, and it is the responsibility of the JobManager
+		// to handle it (e.g., by reducing requirements and failing outright)
+	}
+
+	@Override
+	public int getNumberPendingSlotRequests() {
+		// only exists for testing purposes
+		throw new UnsupportedOperationException();
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Internal utility methods
+	// ---------------------------------------------------------------------------------------------
+
+	private void checkInit() {
+		Preconditions.checkState(started, "The slot manager has not been started.");
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Testing methods
+	// ---------------------------------------------------------------------------------------------
+
+	@Override
+	@VisibleForTesting
+	public void unregisterTaskManagersAndReleaseResources() {

Review comment:
       This method only exists for a single test case in the `YarnResourceManagerTest`. I think it is quite questionable because it not only relies on RM-internal components (the SlotManager) to trigger the unregistration/release (instead of unregistering the TE at the RM), but also introduces a custom codepath into the SlotManager for releasing resources that in this form doesn't appear to exist in production.
   The test asserts that the container is released upon calling this method, but it begs the question what we are even testing. If we are just interested in whether the Yarn RM resource actions can properly shut down the container, then we shouldn't need the slot manager for that.
   




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13553: [FLINK-19306][coordination] Add DeclarativeSlotManager

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13553:
URL: https://github.com/apache/flink/pull/13553#issuecomment-704936985


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "15d4c353c4c0a646b2bdf34bbedbd81359cb5b5f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7264",
       "triggerID" : "15d4c353c4c0a646b2bdf34bbedbd81359cb5b5f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15d4c353c4c0a646b2bdf34bbedbd81359cb5b5f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7283",
       "triggerID" : "705384887",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "15d4c353c4c0a646b2bdf34bbedbd81359cb5b5f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7264",
       "triggerID" : "705384887",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "484cbbdfcc7e371ca51de4d88bab086a299795a6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7514",
       "triggerID" : "484cbbdfcc7e371ca51de4d88bab086a299795a6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7a4c350c9514dc81429de2d9887837045b5894e0",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7527",
       "triggerID" : "7a4c350c9514dc81429de2d9887837045b5894e0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4b9288681085da65fc2c3f2742c898d13bf4af46",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7580",
       "triggerID" : "4b9288681085da65fc2c3f2742c898d13bf4af46",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7a4c350c9514dc81429de2d9887837045b5894e0 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7527) 
   * 4b9288681085da65fc2c3f2742c898d13bf4af46 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7580) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] tillrohrmann commented on a change in pull request #13553: [FLINK-19306][coordination] Add DeclarativeSlotManager

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #13553:
URL: https://github.com/apache/flink/pull/13553#discussion_r502527695



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManager.java
##########
@@ -0,0 +1,609 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.metrics.MetricNames;
+import org.apache.flink.runtime.metrics.groups.SlotManagerMetricGroup;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerId;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.slots.ResourceCounter;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+import org.apache.flink.runtime.slots.ResourceRequirements;
+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.SlotOccupiedException;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ScheduledFuture;
+import java.util.function.BiFunction;
+import java.util.stream.Collectors;
+
+/**
+ * Implementation of {@link SlotManager} supporting declarative slot management.
+ */
+public class DeclarativeSlotManager implements SlotManager {
+	private static final Logger LOG = LoggerFactory.getLogger(DeclarativeSlotManager.class);
+
+	private final SlotTracker slotTracker;
+	private final ResourceTracker resourceTracker;
+	private final BiFunction<Executor, ResourceActions, TaskExecutorManager> taskExecutorManagerFactory;
+	private TaskExecutorManager taskExecutorManager;
+
+	/** Timeout for slot requests to the task manager. */
+	private final Time taskManagerRequestTimeout;
+	private ScheduledFuture<?> slotRequestTimeoutCheck;
+
+	private final SlotMatchingStrategy slotMatchingStrategy;
+
+	private final SlotManagerMetricGroup slotManagerMetricGroup;
+
+	private final Map<JobID, String> jobMasterTargetAddresses = new HashMap<>();
+	private final HashMap<SlotID, CompletableFuture<Acknowledge>> pendingSlotAllocationFutures;
+
+	/** ResourceManager's id. */
+	private ResourceManagerId resourceManagerId;
+
+	/** Executor for future callbacks which have to be "synchronized". */
+	private Executor mainThreadExecutor;
+
+	/** Callbacks for resource (de-)allocations. */
+	private ResourceActions resourceActions;
+
+	/** True iff the component has been started. */
+	private boolean started;
+
+	public DeclarativeSlotManager(
+			ScheduledExecutor scheduledExecutor,
+			SlotManagerConfiguration slotManagerConfiguration,
+			SlotManagerMetricGroup slotManagerMetricGroup,
+			ResourceTracker resourceTracker,
+			SlotTracker slotTracker) {
+
+		Preconditions.checkNotNull(slotManagerConfiguration);
+		this.taskManagerRequestTimeout = slotManagerConfiguration.getTaskManagerRequestTimeout();
+		this.slotManagerMetricGroup = Preconditions.checkNotNull(slotManagerMetricGroup);
+		this.resourceTracker = Preconditions.checkNotNull(resourceTracker);
+
+		pendingSlotAllocationFutures = new HashMap<>(16);
+
+		this.slotTracker = Preconditions.checkNotNull(slotTracker);
+		slotTracker.registerSlotStatusUpdateListener(createSlotStatusUpdateListener());
+
+		slotMatchingStrategy = slotManagerConfiguration.getSlotMatchingStrategy();
+
+		taskExecutorManagerFactory = (executor, resourceActions) -> new TaskExecutorManager(
+			slotManagerConfiguration.getDefaultWorkerResourceSpec(),
+			slotManagerConfiguration.getNumSlotsPerWorker(),
+			slotManagerConfiguration.getMaxSlotNum(),
+			slotManagerConfiguration.isWaitResultConsumedBeforeRelease(),
+			slotManagerConfiguration.getRedundantTaskManagerNum(),
+			slotManagerConfiguration.getTaskManagerTimeout(),
+			scheduledExecutor,
+			executor,
+			resourceActions);
+
+		resourceManagerId = null;
+		resourceActions = null;
+		mainThreadExecutor = null;
+		slotRequestTimeoutCheck = null;
+		taskExecutorManager = null;
+
+		started = false;
+	}
+
+	private SlotStatusUpdateListener createSlotStatusUpdateListener() {
+		return (taskManagerSlot, previous, current, jobId) -> {
+			if (previous == SlotState.PENDING) {
+				cancelAllocationFuture(taskManagerSlot.getSlotId());
+			}
+
+			if (current == SlotState.PENDING) {
+				resourceTracker.notifyAcquiredResource(jobId, taskManagerSlot.getResourceProfile());
+			}
+			if (current == SlotState.FREE) {
+				resourceTracker.notifyLostResource(jobId, taskManagerSlot.getResourceProfile());
+			}
+
+			if (current == SlotState.ALLOCATED) {
+				taskExecutorManager.occupySlot(taskManagerSlot.getInstanceId());
+			}
+			if (previous == SlotState.ALLOCATED && current == SlotState.FREE) {
+				taskExecutorManager.freeSlot(taskManagerSlot.getInstanceId());
+			}
+		};
+	}
+
+	private void cancelAllocationFuture(SlotID slotId) {
+		final CompletableFuture<Acknowledge> acknowledgeCompletableFuture = pendingSlotAllocationFutures.remove(slotId);
+		// the future may be null if we are just re-playing the state transitions due to a slot report
+		if (acknowledgeCompletableFuture != null) {
+			acknowledgeCompletableFuture.cancel(false);
+		}
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Component lifecycle methods
+	// ---------------------------------------------------------------------------------------------
+
+	/**
+	 * Starts the slot manager with the given leader id and resource manager actions.
+	 *
+	 * @param newResourceManagerId to use for communication with the task managers
+	 * @param newMainThreadExecutor to use to run code in the ResourceManager's main thread
+	 * @param newResourceActions to use for resource (de-)allocations
+	 */
+	@Override
+	public void start(ResourceManagerId newResourceManagerId, Executor newMainThreadExecutor, ResourceActions newResourceActions) {
+		LOG.info("Starting the slot manager.");
+
+		this.resourceManagerId = Preconditions.checkNotNull(newResourceManagerId);
+		mainThreadExecutor = Preconditions.checkNotNull(newMainThreadExecutor);
+		resourceActions = Preconditions.checkNotNull(newResourceActions);
+		taskExecutorManager = taskExecutorManagerFactory.apply(newMainThreadExecutor, newResourceActions);
+
+		started = true;
+
+		registerSlotManagerMetrics();
+	}
+
+	private void registerSlotManagerMetrics() {
+		slotManagerMetricGroup.gauge(
+			MetricNames.TASK_SLOTS_AVAILABLE,
+			() -> (long) getNumberFreeSlots());
+		slotManagerMetricGroup.gauge(
+			MetricNames.TASK_SLOTS_TOTAL,
+			() -> (long) getNumberRegisteredSlots());
+	}

Review comment:
       I guess the same problem already exists in the `SlotManagerImpl`.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManager.java
##########
@@ -0,0 +1,609 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.metrics.MetricNames;
+import org.apache.flink.runtime.metrics.groups.SlotManagerMetricGroup;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerId;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.slots.ResourceCounter;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+import org.apache.flink.runtime.slots.ResourceRequirements;
+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.SlotOccupiedException;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ScheduledFuture;
+import java.util.function.BiFunction;
+import java.util.stream.Collectors;
+
+/**
+ * Implementation of {@link SlotManager} supporting declarative slot management.
+ */
+public class DeclarativeSlotManager implements SlotManager {
+	private static final Logger LOG = LoggerFactory.getLogger(DeclarativeSlotManager.class);
+
+	private final SlotTracker slotTracker;
+	private final ResourceTracker resourceTracker;
+	private final BiFunction<Executor, ResourceActions, TaskExecutorManager> taskExecutorManagerFactory;
+	private TaskExecutorManager taskExecutorManager;
+
+	/** Timeout for slot requests to the task manager. */
+	private final Time taskManagerRequestTimeout;
+	private ScheduledFuture<?> slotRequestTimeoutCheck;
+
+	private final SlotMatchingStrategy slotMatchingStrategy;
+
+	private final SlotManagerMetricGroup slotManagerMetricGroup;
+
+	private final Map<JobID, String> jobMasterTargetAddresses = new HashMap<>();
+	private final HashMap<SlotID, CompletableFuture<Acknowledge>> pendingSlotAllocationFutures;
+
+	/** ResourceManager's id. */
+	private ResourceManagerId resourceManagerId;
+
+	/** Executor for future callbacks which have to be "synchronized". */
+	private Executor mainThreadExecutor;
+
+	/** Callbacks for resource (de-)allocations. */
+	private ResourceActions resourceActions;
+
+	/** True iff the component has been started. */
+	private boolean started;
+
+	public DeclarativeSlotManager(
+			ScheduledExecutor scheduledExecutor,
+			SlotManagerConfiguration slotManagerConfiguration,
+			SlotManagerMetricGroup slotManagerMetricGroup,
+			ResourceTracker resourceTracker,
+			SlotTracker slotTracker) {
+
+		Preconditions.checkNotNull(slotManagerConfiguration);
+		this.taskManagerRequestTimeout = slotManagerConfiguration.getTaskManagerRequestTimeout();
+		this.slotManagerMetricGroup = Preconditions.checkNotNull(slotManagerMetricGroup);
+		this.resourceTracker = Preconditions.checkNotNull(resourceTracker);
+
+		pendingSlotAllocationFutures = new HashMap<>(16);
+
+		this.slotTracker = Preconditions.checkNotNull(slotTracker);
+		slotTracker.registerSlotStatusUpdateListener(createSlotStatusUpdateListener());
+
+		slotMatchingStrategy = slotManagerConfiguration.getSlotMatchingStrategy();
+
+		taskExecutorManagerFactory = (executor, resourceActions) -> new TaskExecutorManager(
+			slotManagerConfiguration.getDefaultWorkerResourceSpec(),
+			slotManagerConfiguration.getNumSlotsPerWorker(),
+			slotManagerConfiguration.getMaxSlotNum(),
+			slotManagerConfiguration.isWaitResultConsumedBeforeRelease(),
+			slotManagerConfiguration.getRedundantTaskManagerNum(),
+			slotManagerConfiguration.getTaskManagerTimeout(),
+			scheduledExecutor,
+			executor,
+			resourceActions);
+
+		resourceManagerId = null;
+		resourceActions = null;
+		mainThreadExecutor = null;
+		slotRequestTimeoutCheck = null;
+		taskExecutorManager = null;
+
+		started = false;
+	}
+
+	private SlotStatusUpdateListener createSlotStatusUpdateListener() {
+		return (taskManagerSlot, previous, current, jobId) -> {
+			if (previous == SlotState.PENDING) {
+				cancelAllocationFuture(taskManagerSlot.getSlotId());
+			}
+
+			if (current == SlotState.PENDING) {
+				resourceTracker.notifyAcquiredResource(jobId, taskManagerSlot.getResourceProfile());
+			}
+			if (current == SlotState.FREE) {
+				resourceTracker.notifyLostResource(jobId, taskManagerSlot.getResourceProfile());
+			}
+
+			if (current == SlotState.ALLOCATED) {
+				taskExecutorManager.occupySlot(taskManagerSlot.getInstanceId());
+			}
+			if (previous == SlotState.ALLOCATED && current == SlotState.FREE) {
+				taskExecutorManager.freeSlot(taskManagerSlot.getInstanceId());
+			}
+		};
+	}
+
+	private void cancelAllocationFuture(SlotID slotId) {
+		final CompletableFuture<Acknowledge> acknowledgeCompletableFuture = pendingSlotAllocationFutures.remove(slotId);
+		// the future may be null if we are just re-playing the state transitions due to a slot report
+		if (acknowledgeCompletableFuture != null) {
+			acknowledgeCompletableFuture.cancel(false);
+		}
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Component lifecycle methods
+	// ---------------------------------------------------------------------------------------------
+
+	/**
+	 * Starts the slot manager with the given leader id and resource manager actions.
+	 *
+	 * @param newResourceManagerId to use for communication with the task managers
+	 * @param newMainThreadExecutor to use to run code in the ResourceManager's main thread
+	 * @param newResourceActions to use for resource (de-)allocations
+	 */
+	@Override
+	public void start(ResourceManagerId newResourceManagerId, Executor newMainThreadExecutor, ResourceActions newResourceActions) {
+		LOG.info("Starting the slot manager.");
+
+		this.resourceManagerId = Preconditions.checkNotNull(newResourceManagerId);
+		mainThreadExecutor = Preconditions.checkNotNull(newMainThreadExecutor);
+		resourceActions = Preconditions.checkNotNull(newResourceActions);
+		taskExecutorManager = taskExecutorManagerFactory.apply(newMainThreadExecutor, newResourceActions);
+
+		started = true;
+
+		registerSlotManagerMetrics();
+	}
+
+	private void registerSlotManagerMetrics() {
+		slotManagerMetricGroup.gauge(
+			MetricNames.TASK_SLOTS_AVAILABLE,
+			() -> (long) getNumberFreeSlots());
+		slotManagerMetricGroup.gauge(
+			MetricNames.TASK_SLOTS_TOTAL,
+			() -> (long) getNumberRegisteredSlots());
+	}

Review comment:
       I guess the same problem already exists in the `SlotManagerImpl`. Hence, we can tackle it as a follow up.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] tillrohrmann commented on a change in pull request #13553: [FLINK-19306][coordination] Add DeclarativeSlotManager

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #13553:
URL: https://github.com/apache/flink/pull/13553#discussion_r502529728



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManager.java
##########
@@ -0,0 +1,609 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.metrics.MetricNames;
+import org.apache.flink.runtime.metrics.groups.SlotManagerMetricGroup;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerId;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.slots.ResourceCounter;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+import org.apache.flink.runtime.slots.ResourceRequirements;
+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.SlotOccupiedException;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ScheduledFuture;
+import java.util.function.BiFunction;
+import java.util.stream.Collectors;
+
+/**
+ * Implementation of {@link SlotManager} supporting declarative slot management.
+ */
+public class DeclarativeSlotManager implements SlotManager {
+	private static final Logger LOG = LoggerFactory.getLogger(DeclarativeSlotManager.class);
+
+	private final SlotTracker slotTracker;
+	private final ResourceTracker resourceTracker;
+	private final BiFunction<Executor, ResourceActions, TaskExecutorManager> taskExecutorManagerFactory;
+	private TaskExecutorManager taskExecutorManager;
+
+	/** Timeout for slot requests to the task manager. */
+	private final Time taskManagerRequestTimeout;
+	private ScheduledFuture<?> slotRequestTimeoutCheck;
+
+	private final SlotMatchingStrategy slotMatchingStrategy;
+
+	private final SlotManagerMetricGroup slotManagerMetricGroup;
+
+	private final Map<JobID, String> jobMasterTargetAddresses = new HashMap<>();
+	private final HashMap<SlotID, CompletableFuture<Acknowledge>> pendingSlotAllocationFutures;
+
+	/** ResourceManager's id. */
+	private ResourceManagerId resourceManagerId;
+
+	/** Executor for future callbacks which have to be "synchronized". */
+	private Executor mainThreadExecutor;
+
+	/** Callbacks for resource (de-)allocations. */
+	private ResourceActions resourceActions;
+
+	/** True iff the component has been started. */
+	private boolean started;
+
+	public DeclarativeSlotManager(
+			ScheduledExecutor scheduledExecutor,
+			SlotManagerConfiguration slotManagerConfiguration,
+			SlotManagerMetricGroup slotManagerMetricGroup,
+			ResourceTracker resourceTracker,
+			SlotTracker slotTracker) {
+
+		Preconditions.checkNotNull(slotManagerConfiguration);
+		this.taskManagerRequestTimeout = slotManagerConfiguration.getTaskManagerRequestTimeout();
+		this.slotManagerMetricGroup = Preconditions.checkNotNull(slotManagerMetricGroup);
+		this.resourceTracker = Preconditions.checkNotNull(resourceTracker);
+
+		pendingSlotAllocationFutures = new HashMap<>(16);
+
+		this.slotTracker = Preconditions.checkNotNull(slotTracker);
+		slotTracker.registerSlotStatusUpdateListener(createSlotStatusUpdateListener());
+
+		slotMatchingStrategy = slotManagerConfiguration.getSlotMatchingStrategy();
+
+		taskExecutorManagerFactory = (executor, resourceActions) -> new TaskExecutorManager(
+			slotManagerConfiguration.getDefaultWorkerResourceSpec(),
+			slotManagerConfiguration.getNumSlotsPerWorker(),
+			slotManagerConfiguration.getMaxSlotNum(),
+			slotManagerConfiguration.isWaitResultConsumedBeforeRelease(),
+			slotManagerConfiguration.getRedundantTaskManagerNum(),
+			slotManagerConfiguration.getTaskManagerTimeout(),
+			scheduledExecutor,
+			executor,
+			resourceActions);
+
+		resourceManagerId = null;
+		resourceActions = null;
+		mainThreadExecutor = null;
+		slotRequestTimeoutCheck = null;
+		taskExecutorManager = null;
+
+		started = false;
+	}
+
+	private SlotStatusUpdateListener createSlotStatusUpdateListener() {
+		return (taskManagerSlot, previous, current, jobId) -> {
+			if (previous == SlotState.PENDING) {
+				cancelAllocationFuture(taskManagerSlot.getSlotId());
+			}
+
+			if (current == SlotState.PENDING) {
+				resourceTracker.notifyAcquiredResource(jobId, taskManagerSlot.getResourceProfile());
+			}
+			if (current == SlotState.FREE) {
+				resourceTracker.notifyLostResource(jobId, taskManagerSlot.getResourceProfile());
+			}
+
+			if (current == SlotState.ALLOCATED) {
+				taskExecutorManager.occupySlot(taskManagerSlot.getInstanceId());
+			}
+			if (previous == SlotState.ALLOCATED && current == SlotState.FREE) {
+				taskExecutorManager.freeSlot(taskManagerSlot.getInstanceId());
+			}
+		};
+	}
+
+	private void cancelAllocationFuture(SlotID slotId) {
+		final CompletableFuture<Acknowledge> acknowledgeCompletableFuture = pendingSlotAllocationFutures.remove(slotId);
+		// the future may be null if we are just re-playing the state transitions due to a slot report
+		if (acknowledgeCompletableFuture != null) {
+			acknowledgeCompletableFuture.cancel(false);
+		}
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Component lifecycle methods
+	// ---------------------------------------------------------------------------------------------
+
+	/**
+	 * Starts the slot manager with the given leader id and resource manager actions.
+	 *
+	 * @param newResourceManagerId to use for communication with the task managers
+	 * @param newMainThreadExecutor to use to run code in the ResourceManager's main thread
+	 * @param newResourceActions to use for resource (de-)allocations
+	 */
+	@Override
+	public void start(ResourceManagerId newResourceManagerId, Executor newMainThreadExecutor, ResourceActions newResourceActions) {
+		LOG.info("Starting the slot manager.");
+
+		this.resourceManagerId = Preconditions.checkNotNull(newResourceManagerId);
+		mainThreadExecutor = Preconditions.checkNotNull(newMainThreadExecutor);
+		resourceActions = Preconditions.checkNotNull(newResourceActions);
+		taskExecutorManager = taskExecutorManagerFactory.apply(newMainThreadExecutor, newResourceActions);
+
+		started = true;
+
+		registerSlotManagerMetrics();
+	}
+
+	private void registerSlotManagerMetrics() {
+		slotManagerMetricGroup.gauge(
+			MetricNames.TASK_SLOTS_AVAILABLE,
+			() -> (long) getNumberFreeSlots());
+		slotManagerMetricGroup.gauge(
+			MetricNames.TASK_SLOTS_TOTAL,
+			() -> (long) getNumberRegisteredSlots());
+	}
+
+	/**
+	 * Suspends the component. This clears the internal state of the slot manager.
+	 */
+	@Override
+	public void suspend() {
+		LOG.info("Suspending the slot manager.");
+
+		if (slotRequestTimeoutCheck != null) {
+			slotRequestTimeoutCheck.cancel(false);
+			slotRequestTimeoutCheck = null;
+		}
+
+		resourceTracker.clear();
+		taskExecutorManager.close();
+
+		for (InstanceID registeredTaskManager : taskExecutorManager.getTaskExecutors()) {
+			unregisterTaskManager(registeredTaskManager, new SlotManagerException("The slot manager is being suspended."));
+		}
+
+		taskExecutorManager = null;
+		resourceManagerId = null;
+		resourceActions = null;
+		started = false;
+	}
+
+	/**
+	 * Closes the slot manager.
+	 *
+	 * @throws Exception if the close operation fails
+	 */
+	@Override
+	public void close() throws Exception {
+		LOG.info("Closing the slot manager.");
+
+		suspend();
+		slotManagerMetricGroup.close();
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Public API
+	// ---------------------------------------------------------------------------------------------
+
+	@Override
+	public void processResourceRequirements(ResourceRequirements resourceRequirements) {
+		checkInit();
+		LOG.debug("Received resource requirements from job {}: {}", resourceRequirements.getJobId(), resourceRequirements.getResourceRequirements());
+
+		if (resourceRequirements.getResourceRequirements().isEmpty()) {
+			jobMasterTargetAddresses.remove(resourceRequirements.getJobId());
+		} else {
+			jobMasterTargetAddresses.put(resourceRequirements.getJobId(), resourceRequirements.getTargetAddress());
+		}
+		resourceTracker.notifyResourceRequirements(resourceRequirements.getJobId(), resourceRequirements.getResourceRequirements());
+		checkResourceRequirements();
+	}
+
+	/**
+	 * 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
+	 * @return True if the task manager has not been registered before and is registered successfully; otherwise false
+	 */
+	@Override
+	public boolean registerTaskManager(final TaskExecutorConnection taskExecutorConnection, SlotReport initialSlotReport) {
+		checkInit();
+		LOG.debug("Registering task executor {} under {} at the slot manager.", taskExecutorConnection.getResourceID(), taskExecutorConnection.getInstanceID());
+
+		// we identify task managers by their instance id
+		if (taskExecutorManager.isTaskManagerRegistered(taskExecutorConnection.getInstanceID())) {
+			LOG.debug("Task executor {} was already registered.", taskExecutorConnection.getResourceID());
+			reportSlotStatus(taskExecutorConnection.getInstanceID(), initialSlotReport);
+			return false;
+		} else {
+			if (!taskExecutorManager.registerTaskManager(taskExecutorConnection, initialSlotReport)) {
+				LOG.debug("Task executor {} could not be registered.", taskExecutorConnection.getResourceID());
+				return false;
+			}
+
+			// register the new slots
+			for (SlotStatus slotStatus : initialSlotReport) {
+				slotTracker.addSlot(
+					slotStatus.getSlotID(),
+					slotStatus.getResourceProfile(),
+					taskExecutorConnection,
+					slotStatus.getJobID());
+			}
+
+			checkResourceRequirements();
+			return true;
+		}
+	}
+
+	@Override
+	public boolean unregisterTaskManager(InstanceID instanceId, Exception cause) {
+		checkInit();
+
+		LOG.debug("Unregistering task executor {} from the slot manager.", instanceId);
+
+		if (taskExecutorManager.isTaskManagerRegistered(instanceId)) {
+			slotTracker.removeSlots(taskExecutorManager.getSlotsOf(instanceId));
+			taskExecutorManager.unregisterTaskExecutor(instanceId);
+			checkResourceRequirements();
+
+			return true;
+		} else {
+			LOG.debug("There is no task executor registered with instance ID {}. Ignoring this message.", instanceId);
+
+			return false;
+		}
+	}
+
+	/**
+	 * Reports the current slot allocations for a task manager identified by the given instance id.
+	 *
+	 * @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
+	 */
+	@Override
+	public boolean reportSlotStatus(InstanceID instanceId, SlotReport slotReport) {
+		checkInit();
+
+		LOG.debug("Received slot report from instance {}: {}.", instanceId, slotReport);
+
+		if (taskExecutorManager.isTaskManagerRegistered(instanceId)) {
+			slotTracker.notifySlotStatus(slotReport);
+			checkResourceRequirements();
+			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
+	 */
+	@Override
+	public void freeSlot(SlotID slotId, AllocationID allocationId) {
+		checkInit();
+		LOG.debug("Freeing slot {}.", slotId);
+
+		slotTracker.notifyFree(slotId);
+		checkResourceRequirements();
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Requirement matching
+	// ---------------------------------------------------------------------------------------------
+
+	private void checkResourceRequirements() {
+		final Map<JobID, Collection<ResourceRequirement>> missingResources = resourceTracker.getMissingResources();
+		if (missingResources.isEmpty()) {
+			return;
+		}
+
+		final Map<JobID, ResourceCounter> outstandingRequirements = new LinkedHashMap<>();
+		for (Map.Entry<JobID, Collection<ResourceRequirement>> resourceRequirements : missingResources.entrySet()) {
+			JobID jobId = resourceRequirements.getKey();
+
+			for (ResourceRequirement resourceRequirement : resourceRequirements.getValue()) {
+				int numMissingSlots = internalRequestSlots(jobId, jobMasterTargetAddresses.get(jobId), resourceRequirement);
+				if (numMissingSlots > 0) {
+					outstandingRequirements
+						.computeIfAbsent(jobId, ignored -> new ResourceCounter())
+						.incrementCount(resourceRequirement.getResourceProfile(), numMissingSlots);
+				}
+			}
+		}
+
+		final ResourceCounter pendingSlots = new ResourceCounter(taskExecutorManager.getPendingTaskManagerSlots().stream().collect(
+			Collectors.groupingBy(
+				PendingTaskManagerSlot::getResourceProfile,
+				Collectors.summingInt(x -> 1))));
+
+		for (Map.Entry<JobID, ResourceCounter> unfulfilledRequirement : outstandingRequirements.entrySet()) {
+			tryFulfillRequirementsWithPendingOrNewSlots(
+				unfulfilledRequirement.getKey(),
+				unfulfilledRequirement.getValue().getResourceProfilesWithCount(),
+				pendingSlots);
+		}
+	}
+
+	/**
+	 * Tries to allocate slots for the given requirement. If there are not enough slots available, the
+	 * resource manager is informed to allocate more resources.
+	 *
+	 * @param jobId job to allocate slots for
+	 * @param targetAddress address of the jobmaster
+	 * @param resourceRequirement required slots
+	 * @return the number of missing slots
+	 */
+	private int internalRequestSlots(JobID jobId, String targetAddress, ResourceRequirement resourceRequirement) {
+		final ResourceProfile requiredResource = resourceRequirement.getResourceProfile();
+		Collection<TaskManagerSlotInformation> freeSlots = slotTracker.getFreeSlots();
+
+		int numUnfulfilled = 0;
+		for (int x = 0; x < resourceRequirement.getNumberOfRequiredSlots(); x++) {
+
+			final Optional<TaskManagerSlotInformation> reservedSlot = slotMatchingStrategy.findMatchingSlot(requiredResource, freeSlots, this::getNumberRegisteredSlotsOf);
+			if (reservedSlot.isPresent()) {
+				// we do not need to modify freeSlots because it is indirectly modified by the allocation
+				allocateSlot(reservedSlot.get(), jobId, targetAddress, requiredResource);
+			} else {
+				numUnfulfilled++;
+			}
+		}
+		return numUnfulfilled;
+	}
+
+	/**
+	 * Allocates the given slot. This entails sending a registration message to the task manager and treating failures.
+	 *
+	 * @param taskManagerSlot slot to allocate
+	 * @param jobId job for which the slot should be allocated for
+	 * @param targetAddress address of the job master
+	 * @param resourceProfile resource profile for the requirement for which the slot is used
+	 */
+	private void allocateSlot(TaskManagerSlotInformation taskManagerSlot, JobID jobId, String targetAddress, ResourceProfile resourceProfile) {
+		TaskExecutorConnection taskExecutorConnection = taskManagerSlot.getTaskManagerConnection();
+		TaskExecutorGateway gateway = taskExecutorConnection.getTaskExecutorGateway();
+
+		final CompletableFuture<Acknowledge> completableFuture = new CompletableFuture<>();
+		final SlotID slotId = taskManagerSlot.getSlotId();
+		final InstanceID instanceId = taskManagerSlot.getInstanceId();
+
+		LOG.debug("Starting allocation of slot {} for job {} with resource profile {}.", taskManagerSlot.getSlotId(), jobId, resourceProfile);
+
+		slotTracker.notifyAllocationStart(slotId, jobId);
+
+		if (!taskExecutorManager.isTaskManagerRegistered(instanceId)) {
+			throw new IllegalStateException("Could not find a registered task manager for instance id " +
+				instanceId + '.');
+		}
+
+		taskExecutorManager.markUsed(instanceId);
+
+		// RPC call to the task manager
+		CompletableFuture<Acknowledge> requestFuture = gateway.requestSlot(
+			slotId,
+			jobId,
+			new AllocationID(),
+			resourceProfile,
+			targetAddress,
+			resourceManagerId,
+			taskManagerRequestTimeout);
+
+		requestFuture.whenComplete(
+			(Acknowledge acknowledge, Throwable throwable) -> {
+				if (acknowledge != null) {
+					completableFuture.complete(acknowledge);
+				} else {
+					completableFuture.completeExceptionally(throwable);
+				}
+			});
+
+		completableFuture.whenCompleteAsync(
+			(Acknowledge acknowledge, Throwable throwable) -> {
+				try {
+					if (acknowledge != null) {
+						LOG.trace("Completed allocation of slot {} for job {}.", slotId, jobId);
+						slotTracker.notifyAllocationComplete(slotId, jobId);
+					} else {
+						if (throwable instanceof SlotOccupiedException) {
+							SlotOccupiedException exception = (SlotOccupiedException) throwable;
+							LOG.debug("Tried allocating slot {} for job {}, but it was already allocated for job {}.", slotId, jobId, exception.getJobId());
+							// report as a slot status to force the state transition
+							// this could be a problem if we ever assume that the task executor always reports about all slots
+							slotTracker.notifySlotStatus(Collections.singleton(new SlotStatus(slotId, taskManagerSlot.getResourceProfile(), exception.getJobId(), exception.getAllocationId())));
+						} else {
+							if (throwable instanceof CancellationException) {
+								LOG.debug("Cancelled allocation of slot {} for job {}.", slotId, jobId, throwable);
+							} else {
+								LOG.warn("Slot allocation for slot {} for job {} failed.", slotId, jobId, throwable);
+								slotTracker.notifyFree(slotId);
+							}
+						}
+					}
+					checkResourceRequirements();
+				} catch (Exception e) {
+					LOG.error("Error while completing the slot allocation.", e);
+				}
+			},
+			mainThreadExecutor);
+	}
+
+	private void tryFulfillRequirementsWithPendingOrNewSlots(JobID jobId, Map<ResourceProfile, Integer> missingResources, ResourceCounter pendingSlots) {
+		for (Map.Entry<ResourceProfile, Integer> missingResource : missingResources.entrySet()) {
+			ResourceProfile profile = missingResource.getKey();
+			for (int i = 0; i < missingResource.getValue(); i++) {
+				if (!tryFulfillWithPendingSlots(profile, pendingSlots)) {
+					Optional<ResourceRequirement> newlyFulfillableRequirements = taskExecutorManager.allocateWorker(profile);
+					if (newlyFulfillableRequirements.isPresent()) {
+						ResourceRequirement newSlots = newlyFulfillableRequirements.get();
+						// reserve one of the new slots
+						if (newSlots.getNumberOfRequiredSlots() > 1) {
+							pendingSlots.incrementCount(newSlots.getResourceProfile(), newSlots.getNumberOfRequiredSlots() - 1);
+						}
+					} else {
+						LOG.warn("Could not fulfill resource requirements of job {}.", jobId);
+						resourceActions.notifyNotEnoughResourcesAvailable(jobId, resourceTracker.getAcquiredResources(jobId));
+						return;
+					}
+				}
+			}
+		}
+	}
+
+	private boolean tryFulfillWithPendingSlots(ResourceProfile resourceProfile, ResourceCounter pendingSlots) {
+		Set<ResourceProfile> pendingSlotProfiles = pendingSlots.getResourceProfiles();
+
+		// short-cut, pretty much only applicable to fine-grained resource management
+		if (pendingSlotProfiles.contains(resourceProfile)) {
+			pendingSlots.decrementCount(resourceProfile, 1);
+			return true;
+		}
+
+		for (ResourceProfile pendingSlotProfile : pendingSlotProfiles) {
+			if (pendingSlotProfile.isMatching(resourceProfile)) {
+				pendingSlots.decrementCount(pendingSlotProfile, 1);
+				return true;
+			}
+		}
+
+		return false;
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Legacy APIs
+	// ---------------------------------------------------------------------------------------------
+
+	@Override
+	public int getNumberRegisteredSlots() {
+		return taskExecutorManager.getNumberRegisteredSlots();
+	}
+
+	@Override
+	public int getNumberRegisteredSlotsOf(InstanceID instanceId) {
+		return taskExecutorManager.getNumberRegisteredSlotsOf(instanceId);
+	}
+
+	@Override
+	public int getNumberFreeSlots() {
+		return taskExecutorManager.getNumberFreeSlots();
+	}
+
+	@Override
+	public int getNumberFreeSlotsOf(InstanceID instanceId) {
+		return taskExecutorManager.getNumberFreeSlotsOf(instanceId);
+	}
+
+	@Override
+	public Map<WorkerResourceSpec, Integer> getRequiredResources() {
+		return taskExecutorManager.getRequiredWorkers();
+	}
+
+	@Override
+	public ResourceProfile getRegisteredResource() {
+		return taskExecutorManager.getTotalRegisteredResources();
+	}
+
+	@Override
+	public ResourceProfile getRegisteredResourceOf(InstanceID instanceID) {
+		return taskExecutorManager.getTotalRegisteredResourcesOf(instanceID);
+	}
+
+	@Override
+	public ResourceProfile getFreeResource() {
+		return taskExecutorManager.getTotalFreeResources();
+	}
+
+	@Override
+	public ResourceProfile getFreeResourceOf(InstanceID instanceID) {
+		return taskExecutorManager.getTotalFreeResourcesOf(instanceID);
+	}
+
+	@Override
+	public void setFailUnfulfillableRequest(boolean failUnfulfillableRequest) {
+		// we always send notifications if we cannot fulfill requests, and it is the responsibility of the JobManager
+		// to handle it (e.g., by reducing requirements and failing outright)
+	}
+
+	@Override
+	public int getNumberPendingSlotRequests() {
+		// only exists for testing purposes
+		throw new UnsupportedOperationException();
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Internal utility methods
+	// ---------------------------------------------------------------------------------------------
+
+	private void checkInit() {
+		Preconditions.checkState(started, "The slot manager has not been started.");
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Testing methods
+	// ---------------------------------------------------------------------------------------------
+
+	@Override
+	@VisibleForTesting
+	public void unregisterTaskManagersAndReleaseResources() {

Review comment:
       Can we refactor the test and get rid of this method then?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] zentol commented on a change in pull request #13553: [FLINK-19306][coordination] Add DeclarativeSlotManager

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #13553:
URL: https://github.com/apache/flink/pull/13553#discussion_r502327559



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java
##########
@@ -1190,6 +1215,11 @@ public void notifyAllocationFailure(JobID jobId, AllocationID allocationId, Exce
 				jobManagerRegistration.getJobManagerGateway().notifyAllocationFailure(allocationId, cause);
 			}
 		}
+
+		@Override
+		public void notifyNotEnoughResourcesAvailable(JobID jobId, Collection<ResourceRequirement> acquiredResources) {
+			validateRunsInMainThread();
+		}

Review comment:
       The call to JobMaster#notifyNotEnoughResourcesAvailable; but that wasn't added yet because I only wanted to touch the RM side.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManager.java
##########
@@ -0,0 +1,609 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.metrics.MetricNames;
+import org.apache.flink.runtime.metrics.groups.SlotManagerMetricGroup;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerId;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.slots.ResourceCounter;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+import org.apache.flink.runtime.slots.ResourceRequirements;
+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.SlotOccupiedException;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ScheduledFuture;
+import java.util.function.BiFunction;
+import java.util.stream.Collectors;
+
+/**
+ * Implementation of {@link SlotManager} supporting declarative slot management.
+ */
+public class DeclarativeSlotManager implements SlotManager {
+	private static final Logger LOG = LoggerFactory.getLogger(DeclarativeSlotManager.class);
+
+	private final SlotTracker slotTracker;
+	private final ResourceTracker resourceTracker;
+	private final BiFunction<Executor, ResourceActions, TaskExecutorManager> taskExecutorManagerFactory;
+	private TaskExecutorManager taskExecutorManager;
+
+	/** Timeout for slot requests to the task manager. */
+	private final Time taskManagerRequestTimeout;
+	private ScheduledFuture<?> slotRequestTimeoutCheck;
+
+	private final SlotMatchingStrategy slotMatchingStrategy;
+
+	private final SlotManagerMetricGroup slotManagerMetricGroup;
+
+	private final Map<JobID, String> jobMasterTargetAddresses = new HashMap<>();
+	private final HashMap<SlotID, CompletableFuture<Acknowledge>> pendingSlotAllocationFutures;
+
+	/** ResourceManager's id. */
+	private ResourceManagerId resourceManagerId;
+
+	/** Executor for future callbacks which have to be "synchronized". */
+	private Executor mainThreadExecutor;
+
+	/** Callbacks for resource (de-)allocations. */
+	private ResourceActions resourceActions;
+
+	/** True iff the component has been started. */
+	private boolean started;
+
+	public DeclarativeSlotManager(
+			ScheduledExecutor scheduledExecutor,
+			SlotManagerConfiguration slotManagerConfiguration,
+			SlotManagerMetricGroup slotManagerMetricGroup,
+			ResourceTracker resourceTracker,
+			SlotTracker slotTracker) {
+
+		Preconditions.checkNotNull(slotManagerConfiguration);
+		this.taskManagerRequestTimeout = slotManagerConfiguration.getTaskManagerRequestTimeout();
+		this.slotManagerMetricGroup = Preconditions.checkNotNull(slotManagerMetricGroup);
+		this.resourceTracker = Preconditions.checkNotNull(resourceTracker);
+
+		pendingSlotAllocationFutures = new HashMap<>(16);
+
+		this.slotTracker = Preconditions.checkNotNull(slotTracker);
+		slotTracker.registerSlotStatusUpdateListener(createSlotStatusUpdateListener());
+
+		slotMatchingStrategy = slotManagerConfiguration.getSlotMatchingStrategy();
+
+		taskExecutorManagerFactory = (executor, resourceActions) -> new TaskExecutorManager(
+			slotManagerConfiguration.getDefaultWorkerResourceSpec(),
+			slotManagerConfiguration.getNumSlotsPerWorker(),
+			slotManagerConfiguration.getMaxSlotNum(),
+			slotManagerConfiguration.isWaitResultConsumedBeforeRelease(),
+			slotManagerConfiguration.getRedundantTaskManagerNum(),
+			slotManagerConfiguration.getTaskManagerTimeout(),
+			scheduledExecutor,
+			executor,
+			resourceActions);
+
+		resourceManagerId = null;
+		resourceActions = null;
+		mainThreadExecutor = null;
+		slotRequestTimeoutCheck = null;
+		taskExecutorManager = null;
+
+		started = false;
+	}
+
+	private SlotStatusUpdateListener createSlotStatusUpdateListener() {
+		return (taskManagerSlot, previous, current, jobId) -> {
+			if (previous == SlotState.PENDING) {
+				cancelAllocationFuture(taskManagerSlot.getSlotId());
+			}
+
+			if (current == SlotState.PENDING) {
+				resourceTracker.notifyAcquiredResource(jobId, taskManagerSlot.getResourceProfile());
+			}
+			if (current == SlotState.FREE) {
+				resourceTracker.notifyLostResource(jobId, taskManagerSlot.getResourceProfile());
+			}
+
+			if (current == SlotState.ALLOCATED) {
+				taskExecutorManager.occupySlot(taskManagerSlot.getInstanceId());
+			}
+			if (previous == SlotState.ALLOCATED && current == SlotState.FREE) {
+				taskExecutorManager.freeSlot(taskManagerSlot.getInstanceId());
+			}
+		};
+	}
+
+	private void cancelAllocationFuture(SlotID slotId) {
+		final CompletableFuture<Acknowledge> acknowledgeCompletableFuture = pendingSlotAllocationFutures.remove(slotId);

Review comment:
       It is _supposed_ to be populated in `allocateSlot`. It is mostly there since the current SM also does such book-keeping (just in the TaskManagerSlot), but it could very well be unnecessary at this point.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManagerTest.java
##########
@@ -0,0 +1,1092 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple6;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.concurrent.ManuallyTriggeredScheduledExecutor;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerId;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+import org.apache.flink.runtime.slots.ResourceRequirements;
+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.TestingTaskExecutorGateway;
+import org.apache.flink.runtime.taskexecutor.TestingTaskExecutorGatewayBuilder;
+import org.apache.flink.runtime.taskexecutor.exceptions.SlotAllocationException;
+import org.apache.flink.runtime.taskexecutor.exceptions.SlotOccupiedException;
+import org.apache.flink.runtime.testingUtils.TestingUtils;
+import org.apache.flink.util.FlinkException;
+import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.function.FunctionUtils;
+
+import akka.pattern.AskTimeoutException;
+import org.junit.Ignore;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.hamcrest.CoreMatchers.hasItem;
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.empty;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.hasSize;
+import static org.hamcrest.Matchers.is;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertSame;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Tests for the {@link DeclarativeSlotManager}.
+ */
+public class DeclarativeSlotManagerTest extends TestLogger {
+
+	private static final FlinkException TEST_EXCEPTION = new FlinkException("Test exception");
+
+	private static final WorkerResourceSpec WORKER_RESOURCE_SPEC = new WorkerResourceSpec.Builder()
+		.setCpuCores(100.0)
+		.setTaskHeapMemoryMB(10000)
+		.setTaskOffHeapMemoryMB(10000)
+		.setNetworkMemoryMB(10000)
+		.setManagedMemoryMB(10000)
+		.build();
+
+	/**
+	 * Tests that we can register task manager and their slots at the slot manager.
+	 */
+	@Test
+	public void testTaskManagerRegistration() throws Exception {
+		final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder().createTestingTaskExecutorGateway();
+		final ResourceID resourceId = ResourceID.generate();
+		final TaskExecutorConnection taskManagerConnection = new TaskExecutorConnection(resourceId, taskExecutorGateway);
+
+		final SlotID slotId1 = new SlotID(resourceId, 0);
+		final SlotID slotId2 = new SlotID(resourceId, 1);
+		final SlotReport slotReport = new SlotReport(Arrays.asList(createFreeSlotStatus(slotId1), createFreeSlotStatus(slotId2)));
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport);
+
+			assertThat("The number registered slots does not equal the expected number.", slotManager.getNumberRegisteredSlots(), is(2));
+
+			assertNotNull(slotTracker.getSlot(slotId1));
+			assertNotNull(slotTracker.getSlot(slotId2));
+		}
+	}
+
+	/**
+	 * Tests that un-registration of task managers will free and remove all registered slots.
+	 */
+	@Test
+	public void testTaskManagerUnregistration() throws Exception {
+		final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder()
+			.setRequestSlotFunction(tuple6 -> new CompletableFuture<>())
+			.createTestingTaskExecutorGateway();
+		final TaskExecutorConnection taskManagerConnection = createTaskExecutorConnection(taskExecutorGateway);
+		final ResourceID resourceId = taskManagerConnection.getResourceID();
+
+		final SlotID slotId1 = new SlotID(resourceId, 0);
+		final SlotID slotId2 = new SlotID(resourceId, 1);
+		final SlotReport slotReport = new SlotReport(Arrays.asList(createAllocatedSlotStatus(slotId1), createFreeSlotStatus(slotId2)));
+
+		final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot();
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport);
+
+			assertEquals("The number registered slots does not equal the expected number.", 2, slotManager.getNumberRegisteredSlots());
+
+			DeclarativeTaskManagerSlot slot1 = slotTracker.getSlot(slotId1);
+			DeclarativeTaskManagerSlot slot2 = slotTracker.getSlot(slotId2);
+
+			assertSame(SlotState.ALLOCATED, slot1.getState());
+			assertSame(SlotState.FREE, slot2.getState());

Review comment:
       Agreed.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManagerTest.java
##########
@@ -0,0 +1,1092 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple6;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.concurrent.ManuallyTriggeredScheduledExecutor;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerId;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+import org.apache.flink.runtime.slots.ResourceRequirements;
+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.TestingTaskExecutorGateway;
+import org.apache.flink.runtime.taskexecutor.TestingTaskExecutorGatewayBuilder;
+import org.apache.flink.runtime.taskexecutor.exceptions.SlotAllocationException;
+import org.apache.flink.runtime.taskexecutor.exceptions.SlotOccupiedException;
+import org.apache.flink.runtime.testingUtils.TestingUtils;
+import org.apache.flink.util.FlinkException;
+import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.function.FunctionUtils;
+
+import akka.pattern.AskTimeoutException;
+import org.junit.Ignore;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.hamcrest.CoreMatchers.hasItem;
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.empty;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.hasSize;
+import static org.hamcrest.Matchers.is;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertSame;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Tests for the {@link DeclarativeSlotManager}.
+ */
+public class DeclarativeSlotManagerTest extends TestLogger {
+
+	private static final FlinkException TEST_EXCEPTION = new FlinkException("Test exception");
+
+	private static final WorkerResourceSpec WORKER_RESOURCE_SPEC = new WorkerResourceSpec.Builder()
+		.setCpuCores(100.0)
+		.setTaskHeapMemoryMB(10000)
+		.setTaskOffHeapMemoryMB(10000)
+		.setNetworkMemoryMB(10000)
+		.setManagedMemoryMB(10000)
+		.build();
+
+	/**
+	 * Tests that we can register task manager and their slots at the slot manager.
+	 */
+	@Test
+	public void testTaskManagerRegistration() throws Exception {
+		final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder().createTestingTaskExecutorGateway();
+		final ResourceID resourceId = ResourceID.generate();
+		final TaskExecutorConnection taskManagerConnection = new TaskExecutorConnection(resourceId, taskExecutorGateway);
+
+		final SlotID slotId1 = new SlotID(resourceId, 0);
+		final SlotID slotId2 = new SlotID(resourceId, 1);
+		final SlotReport slotReport = new SlotReport(Arrays.asList(createFreeSlotStatus(slotId1), createFreeSlotStatus(slotId2)));
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport);
+
+			assertThat("The number registered slots does not equal the expected number.", slotManager.getNumberRegisteredSlots(), is(2));
+
+			assertNotNull(slotTracker.getSlot(slotId1));
+			assertNotNull(slotTracker.getSlot(slotId2));
+		}
+	}
+
+	/**
+	 * Tests that un-registration of task managers will free and remove all registered slots.
+	 */
+	@Test
+	public void testTaskManagerUnregistration() throws Exception {
+		final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder()
+			.setRequestSlotFunction(tuple6 -> new CompletableFuture<>())
+			.createTestingTaskExecutorGateway();
+		final TaskExecutorConnection taskManagerConnection = createTaskExecutorConnection(taskExecutorGateway);
+		final ResourceID resourceId = taskManagerConnection.getResourceID();
+
+		final SlotID slotId1 = new SlotID(resourceId, 0);
+		final SlotID slotId2 = new SlotID(resourceId, 1);
+		final SlotReport slotReport = new SlotReport(Arrays.asList(createAllocatedSlotStatus(slotId1), createFreeSlotStatus(slotId2)));
+
+		final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot();
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport);
+
+			assertEquals("The number registered slots does not equal the expected number.", 2, slotManager.getNumberRegisteredSlots());
+
+			DeclarativeTaskManagerSlot slot1 = slotTracker.getSlot(slotId1);
+			DeclarativeTaskManagerSlot slot2 = slotTracker.getSlot(slotId2);
+
+			assertSame(SlotState.ALLOCATED, slot1.getState());
+			assertSame(SlotState.FREE, slot2.getState());
+
+			slotManager.processResourceRequirements(resourceRequirements);
+
+			assertSame(SlotState.PENDING, slot2.getState());
+
+			slotManager.unregisterTaskManager(taskManagerConnection.getInstanceID(), TEST_EXCEPTION);
+
+			assertEquals(0, slotManager.getNumberRegisteredSlots());
+		}
+	}
+
+	/**
+	 * Tests that a slot request with no free slots will trigger the resource allocation.
+	 */
+	@Test
+	public void testSlotRequestWithoutFreeSlots() throws Exception {
+		final ResourceManagerId resourceManagerId = ResourceManagerId.generate();
+
+		final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot();
+
+		CompletableFuture<WorkerResourceSpec> allocateResourceFuture = new CompletableFuture<>();
+		ResourceActions resourceManagerActions = new TestingResourceActionsBuilder()
+			.setAllocateResourceConsumer(allocateResourceFuture::complete)
+			.build();
+
+		try (SlotManager slotManager = createSlotManager(resourceManagerId, resourceManagerActions)) {
+
+			slotManager.processResourceRequirements(resourceRequirements);
+
+			allocateResourceFuture.get();

Review comment:
       The size of the requested resource is not really dependent on the resource requirements; the only thing that is checked (by the TaskExecutorManager) is that the slots of said resource can fulfill the requirements.
   This is admittedly not tested, but belongs into the TaskExecutorManagerTest imo.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManager.java
##########
@@ -0,0 +1,609 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.metrics.MetricNames;
+import org.apache.flink.runtime.metrics.groups.SlotManagerMetricGroup;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerId;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.slots.ResourceCounter;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+import org.apache.flink.runtime.slots.ResourceRequirements;
+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.SlotOccupiedException;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ScheduledFuture;
+import java.util.function.BiFunction;
+import java.util.stream.Collectors;
+
+/**
+ * Implementation of {@link SlotManager} supporting declarative slot management.
+ */
+public class DeclarativeSlotManager implements SlotManager {
+	private static final Logger LOG = LoggerFactory.getLogger(DeclarativeSlotManager.class);
+
+	private final SlotTracker slotTracker;
+	private final ResourceTracker resourceTracker;
+	private final BiFunction<Executor, ResourceActions, TaskExecutorManager> taskExecutorManagerFactory;
+	private TaskExecutorManager taskExecutorManager;
+
+	/** Timeout for slot requests to the task manager. */
+	private final Time taskManagerRequestTimeout;
+	private ScheduledFuture<?> slotRequestTimeoutCheck;

Review comment:
       There should be no need for it.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManager.java
##########
@@ -0,0 +1,609 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.metrics.MetricNames;
+import org.apache.flink.runtime.metrics.groups.SlotManagerMetricGroup;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerId;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.slots.ResourceCounter;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+import org.apache.flink.runtime.slots.ResourceRequirements;
+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.SlotOccupiedException;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ScheduledFuture;
+import java.util.function.BiFunction;
+import java.util.stream.Collectors;
+
+/**
+ * Implementation of {@link SlotManager} supporting declarative slot management.
+ */
+public class DeclarativeSlotManager implements SlotManager {
+	private static final Logger LOG = LoggerFactory.getLogger(DeclarativeSlotManager.class);
+
+	private final SlotTracker slotTracker;
+	private final ResourceTracker resourceTracker;
+	private final BiFunction<Executor, ResourceActions, TaskExecutorManager> taskExecutorManagerFactory;
+	private TaskExecutorManager taskExecutorManager;
+
+	/** Timeout for slot requests to the task manager. */
+	private final Time taskManagerRequestTimeout;
+	private ScheduledFuture<?> slotRequestTimeoutCheck;
+
+	private final SlotMatchingStrategy slotMatchingStrategy;
+
+	private final SlotManagerMetricGroup slotManagerMetricGroup;
+
+	private final Map<JobID, String> jobMasterTargetAddresses = new HashMap<>();
+	private final HashMap<SlotID, CompletableFuture<Acknowledge>> pendingSlotAllocationFutures;
+
+	/** ResourceManager's id. */
+	private ResourceManagerId resourceManagerId;
+
+	/** Executor for future callbacks which have to be "synchronized". */
+	private Executor mainThreadExecutor;
+
+	/** Callbacks for resource (de-)allocations. */
+	private ResourceActions resourceActions;
+
+	/** True iff the component has been started. */
+	private boolean started;
+
+	public DeclarativeSlotManager(
+			ScheduledExecutor scheduledExecutor,
+			SlotManagerConfiguration slotManagerConfiguration,
+			SlotManagerMetricGroup slotManagerMetricGroup,
+			ResourceTracker resourceTracker,
+			SlotTracker slotTracker) {
+
+		Preconditions.checkNotNull(slotManagerConfiguration);
+		this.taskManagerRequestTimeout = slotManagerConfiguration.getTaskManagerRequestTimeout();
+		this.slotManagerMetricGroup = Preconditions.checkNotNull(slotManagerMetricGroup);
+		this.resourceTracker = Preconditions.checkNotNull(resourceTracker);
+
+		pendingSlotAllocationFutures = new HashMap<>(16);
+
+		this.slotTracker = Preconditions.checkNotNull(slotTracker);
+		slotTracker.registerSlotStatusUpdateListener(createSlotStatusUpdateListener());
+
+		slotMatchingStrategy = slotManagerConfiguration.getSlotMatchingStrategy();
+
+		taskExecutorManagerFactory = (executor, resourceActions) -> new TaskExecutorManager(
+			slotManagerConfiguration.getDefaultWorkerResourceSpec(),
+			slotManagerConfiguration.getNumSlotsPerWorker(),
+			slotManagerConfiguration.getMaxSlotNum(),
+			slotManagerConfiguration.isWaitResultConsumedBeforeRelease(),
+			slotManagerConfiguration.getRedundantTaskManagerNum(),
+			slotManagerConfiguration.getTaskManagerTimeout(),
+			scheduledExecutor,
+			executor,
+			resourceActions);
+
+		resourceManagerId = null;
+		resourceActions = null;
+		mainThreadExecutor = null;
+		slotRequestTimeoutCheck = null;
+		taskExecutorManager = null;
+
+		started = false;
+	}
+
+	private SlotStatusUpdateListener createSlotStatusUpdateListener() {
+		return (taskManagerSlot, previous, current, jobId) -> {
+			if (previous == SlotState.PENDING) {
+				cancelAllocationFuture(taskManagerSlot.getSlotId());
+			}
+
+			if (current == SlotState.PENDING) {
+				resourceTracker.notifyAcquiredResource(jobId, taskManagerSlot.getResourceProfile());
+			}
+			if (current == SlotState.FREE) {
+				resourceTracker.notifyLostResource(jobId, taskManagerSlot.getResourceProfile());
+			}
+
+			if (current == SlotState.ALLOCATED) {
+				taskExecutorManager.occupySlot(taskManagerSlot.getInstanceId());
+			}
+			if (previous == SlotState.ALLOCATED && current == SlotState.FREE) {
+				taskExecutorManager.freeSlot(taskManagerSlot.getInstanceId());
+			}
+		};
+	}
+
+	private void cancelAllocationFuture(SlotID slotId) {
+		final CompletableFuture<Acknowledge> acknowledgeCompletableFuture = pendingSlotAllocationFutures.remove(slotId);

Review comment:
       After all we aren't really doing anything on cancellation, although it might be useful for debugging purposes?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManager.java
##########
@@ -0,0 +1,609 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.metrics.MetricNames;
+import org.apache.flink.runtime.metrics.groups.SlotManagerMetricGroup;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerId;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.slots.ResourceCounter;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+import org.apache.flink.runtime.slots.ResourceRequirements;
+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.SlotOccupiedException;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ScheduledFuture;
+import java.util.function.BiFunction;
+import java.util.stream.Collectors;
+
+/**
+ * Implementation of {@link SlotManager} supporting declarative slot management.
+ */
+public class DeclarativeSlotManager implements SlotManager {
+	private static final Logger LOG = LoggerFactory.getLogger(DeclarativeSlotManager.class);
+
+	private final SlotTracker slotTracker;
+	private final ResourceTracker resourceTracker;
+	private final BiFunction<Executor, ResourceActions, TaskExecutorManager> taskExecutorManagerFactory;
+	private TaskExecutorManager taskExecutorManager;
+
+	/** Timeout for slot requests to the task manager. */
+	private final Time taskManagerRequestTimeout;
+	private ScheduledFuture<?> slotRequestTimeoutCheck;
+
+	private final SlotMatchingStrategy slotMatchingStrategy;
+
+	private final SlotManagerMetricGroup slotManagerMetricGroup;
+
+	private final Map<JobID, String> jobMasterTargetAddresses = new HashMap<>();
+	private final HashMap<SlotID, CompletableFuture<Acknowledge>> pendingSlotAllocationFutures;
+
+	/** ResourceManager's id. */
+	private ResourceManagerId resourceManagerId;
+
+	/** Executor for future callbacks which have to be "synchronized". */
+	private Executor mainThreadExecutor;
+
+	/** Callbacks for resource (de-)allocations. */
+	private ResourceActions resourceActions;
+
+	/** True iff the component has been started. */
+	private boolean started;
+
+	public DeclarativeSlotManager(
+			ScheduledExecutor scheduledExecutor,
+			SlotManagerConfiguration slotManagerConfiguration,
+			SlotManagerMetricGroup slotManagerMetricGroup,
+			ResourceTracker resourceTracker,
+			SlotTracker slotTracker) {
+
+		Preconditions.checkNotNull(slotManagerConfiguration);
+		this.taskManagerRequestTimeout = slotManagerConfiguration.getTaskManagerRequestTimeout();
+		this.slotManagerMetricGroup = Preconditions.checkNotNull(slotManagerMetricGroup);
+		this.resourceTracker = Preconditions.checkNotNull(resourceTracker);
+
+		pendingSlotAllocationFutures = new HashMap<>(16);
+
+		this.slotTracker = Preconditions.checkNotNull(slotTracker);
+		slotTracker.registerSlotStatusUpdateListener(createSlotStatusUpdateListener());
+
+		slotMatchingStrategy = slotManagerConfiguration.getSlotMatchingStrategy();
+
+		taskExecutorManagerFactory = (executor, resourceActions) -> new TaskExecutorManager(
+			slotManagerConfiguration.getDefaultWorkerResourceSpec(),
+			slotManagerConfiguration.getNumSlotsPerWorker(),
+			slotManagerConfiguration.getMaxSlotNum(),
+			slotManagerConfiguration.isWaitResultConsumedBeforeRelease(),
+			slotManagerConfiguration.getRedundantTaskManagerNum(),
+			slotManagerConfiguration.getTaskManagerTimeout(),
+			scheduledExecutor,
+			executor,
+			resourceActions);
+
+		resourceManagerId = null;
+		resourceActions = null;
+		mainThreadExecutor = null;
+		slotRequestTimeoutCheck = null;
+		taskExecutorManager = null;
+
+		started = false;
+	}
+
+	private SlotStatusUpdateListener createSlotStatusUpdateListener() {
+		return (taskManagerSlot, previous, current, jobId) -> {
+			if (previous == SlotState.PENDING) {
+				cancelAllocationFuture(taskManagerSlot.getSlotId());
+			}
+
+			if (current == SlotState.PENDING) {
+				resourceTracker.notifyAcquiredResource(jobId, taskManagerSlot.getResourceProfile());
+			}
+			if (current == SlotState.FREE) {
+				resourceTracker.notifyLostResource(jobId, taskManagerSlot.getResourceProfile());
+			}
+
+			if (current == SlotState.ALLOCATED) {
+				taskExecutorManager.occupySlot(taskManagerSlot.getInstanceId());
+			}
+			if (previous == SlotState.ALLOCATED && current == SlotState.FREE) {
+				taskExecutorManager.freeSlot(taskManagerSlot.getInstanceId());
+			}
+		};
+	}
+
+	private void cancelAllocationFuture(SlotID slotId) {
+		final CompletableFuture<Acknowledge> acknowledgeCompletableFuture = pendingSlotAllocationFutures.remove(slotId);
+		// the future may be null if we are just re-playing the state transitions due to a slot report
+		if (acknowledgeCompletableFuture != null) {
+			acknowledgeCompletableFuture.cancel(false);
+		}
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Component lifecycle methods
+	// ---------------------------------------------------------------------------------------------
+
+	/**
+	 * Starts the slot manager with the given leader id and resource manager actions.
+	 *
+	 * @param newResourceManagerId to use for communication with the task managers
+	 * @param newMainThreadExecutor to use to run code in the ResourceManager's main thread
+	 * @param newResourceActions to use for resource (de-)allocations
+	 */
+	@Override
+	public void start(ResourceManagerId newResourceManagerId, Executor newMainThreadExecutor, ResourceActions newResourceActions) {
+		LOG.info("Starting the slot manager.");
+
+		this.resourceManagerId = Preconditions.checkNotNull(newResourceManagerId);
+		mainThreadExecutor = Preconditions.checkNotNull(newMainThreadExecutor);
+		resourceActions = Preconditions.checkNotNull(newResourceActions);
+		taskExecutorManager = taskExecutorManagerFactory.apply(newMainThreadExecutor, newResourceActions);
+
+		started = true;
+
+		registerSlotManagerMetrics();
+	}
+
+	private void registerSlotManagerMetrics() {
+		slotManagerMetricGroup.gauge(
+			MetricNames.TASK_SLOTS_AVAILABLE,
+			() -> (long) getNumberFreeSlots());
+		slotManagerMetricGroup.gauge(
+			MetricNames.TASK_SLOTS_TOTAL,
+			() -> (long) getNumberRegisteredSlots());
+	}
+
+	/**
+	 * Suspends the component. This clears the internal state of the slot manager.
+	 */
+	@Override
+	public void suspend() {
+		LOG.info("Suspending the slot manager.");
+
+		if (slotRequestTimeoutCheck != null) {
+			slotRequestTimeoutCheck.cancel(false);
+			slotRequestTimeoutCheck = null;
+		}
+
+		resourceTracker.clear();
+		taskExecutorManager.close();
+
+		for (InstanceID registeredTaskManager : taskExecutorManager.getTaskExecutors()) {
+			unregisterTaskManager(registeredTaskManager, new SlotManagerException("The slot manager is being suspended."));
+		}
+
+		taskExecutorManager = null;
+		resourceManagerId = null;
+		resourceActions = null;
+		started = false;
+	}
+
+	/**
+	 * Closes the slot manager.
+	 *
+	 * @throws Exception if the close operation fails
+	 */
+	@Override
+	public void close() throws Exception {
+		LOG.info("Closing the slot manager.");
+
+		suspend();
+		slotManagerMetricGroup.close();
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Public API
+	// ---------------------------------------------------------------------------------------------
+
+	@Override
+	public void processResourceRequirements(ResourceRequirements resourceRequirements) {
+		checkInit();
+		LOG.debug("Received resource requirements from job {}: {}", resourceRequirements.getJobId(), resourceRequirements.getResourceRequirements());
+
+		if (resourceRequirements.getResourceRequirements().isEmpty()) {
+			jobMasterTargetAddresses.remove(resourceRequirements.getJobId());
+		} else {
+			jobMasterTargetAddresses.put(resourceRequirements.getJobId(), resourceRequirements.getTargetAddress());
+		}
+		resourceTracker.notifyResourceRequirements(resourceRequirements.getJobId(), resourceRequirements.getResourceRequirements());
+		checkResourceRequirements();
+	}
+
+	/**
+	 * 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
+	 * @return True if the task manager has not been registered before and is registered successfully; otherwise false
+	 */
+	@Override
+	public boolean registerTaskManager(final TaskExecutorConnection taskExecutorConnection, SlotReport initialSlotReport) {
+		checkInit();
+		LOG.debug("Registering task executor {} under {} at the slot manager.", taskExecutorConnection.getResourceID(), taskExecutorConnection.getInstanceID());
+
+		// we identify task managers by their instance id
+		if (taskExecutorManager.isTaskManagerRegistered(taskExecutorConnection.getInstanceID())) {
+			LOG.debug("Task executor {} was already registered.", taskExecutorConnection.getResourceID());
+			reportSlotStatus(taskExecutorConnection.getInstanceID(), initialSlotReport);
+			return false;
+		} else {
+			if (!taskExecutorManager.registerTaskManager(taskExecutorConnection, initialSlotReport)) {
+				LOG.debug("Task executor {} could not be registered.", taskExecutorConnection.getResourceID());
+				return false;
+			}
+
+			// register the new slots
+			for (SlotStatus slotStatus : initialSlotReport) {
+				slotTracker.addSlot(
+					slotStatus.getSlotID(),
+					slotStatus.getResourceProfile(),
+					taskExecutorConnection,
+					slotStatus.getJobID());
+			}
+
+			checkResourceRequirements();
+			return true;
+		}
+	}
+
+	@Override
+	public boolean unregisterTaskManager(InstanceID instanceId, Exception cause) {
+		checkInit();
+
+		LOG.debug("Unregistering task executor {} from the slot manager.", instanceId);
+
+		if (taskExecutorManager.isTaskManagerRegistered(instanceId)) {
+			slotTracker.removeSlots(taskExecutorManager.getSlotsOf(instanceId));
+			taskExecutorManager.unregisterTaskExecutor(instanceId);
+			checkResourceRequirements();
+
+			return true;
+		} else {
+			LOG.debug("There is no task executor registered with instance ID {}. Ignoring this message.", instanceId);
+
+			return false;
+		}
+	}
+
+	/**
+	 * Reports the current slot allocations for a task manager identified by the given instance id.
+	 *
+	 * @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
+	 */
+	@Override
+	public boolean reportSlotStatus(InstanceID instanceId, SlotReport slotReport) {
+		checkInit();
+
+		LOG.debug("Received slot report from instance {}: {}.", instanceId, slotReport);
+
+		if (taskExecutorManager.isTaskManagerRegistered(instanceId)) {
+			slotTracker.notifySlotStatus(slotReport);
+			checkResourceRequirements();
+			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
+	 */
+	@Override
+	public void freeSlot(SlotID slotId, AllocationID allocationId) {
+		checkInit();
+		LOG.debug("Freeing slot {}.", slotId);
+
+		slotTracker.notifyFree(slotId);
+		checkResourceRequirements();
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Requirement matching
+	// ---------------------------------------------------------------------------------------------
+
+	private void checkResourceRequirements() {
+		final Map<JobID, Collection<ResourceRequirement>> missingResources = resourceTracker.getMissingResources();
+		if (missingResources.isEmpty()) {
+			return;
+		}
+
+		final Map<JobID, ResourceCounter> outstandingRequirements = new LinkedHashMap<>();
+		for (Map.Entry<JobID, Collection<ResourceRequirement>> resourceRequirements : missingResources.entrySet()) {
+			JobID jobId = resourceRequirements.getKey();
+
+			for (ResourceRequirement resourceRequirement : resourceRequirements.getValue()) {
+				int numMissingSlots = internalRequestSlots(jobId, jobMasterTargetAddresses.get(jobId), resourceRequirement);
+				if (numMissingSlots > 0) {
+					outstandingRequirements
+						.computeIfAbsent(jobId, ignored -> new ResourceCounter())
+						.incrementCount(resourceRequirement.getResourceProfile(), numMissingSlots);
+				}
+			}
+		}
+
+		final ResourceCounter pendingSlots = new ResourceCounter(taskExecutorManager.getPendingTaskManagerSlots().stream().collect(
+			Collectors.groupingBy(
+				PendingTaskManagerSlot::getResourceProfile,
+				Collectors.summingInt(x -> 1))));
+
+		for (Map.Entry<JobID, ResourceCounter> unfulfilledRequirement : outstandingRequirements.entrySet()) {
+			tryFulfillRequirementsWithPendingOrNewSlots(
+				unfulfilledRequirement.getKey(),
+				unfulfilledRequirement.getValue().getResourceProfilesWithCount(),
+				pendingSlots);
+		}
+	}
+
+	/**
+	 * Tries to allocate slots for the given requirement. If there are not enough slots available, the
+	 * resource manager is informed to allocate more resources.
+	 *
+	 * @param jobId job to allocate slots for
+	 * @param targetAddress address of the jobmaster
+	 * @param resourceRequirement required slots
+	 * @return the number of missing slots
+	 */
+	private int internalRequestSlots(JobID jobId, String targetAddress, ResourceRequirement resourceRequirement) {
+		final ResourceProfile requiredResource = resourceRequirement.getResourceProfile();
+		Collection<TaskManagerSlotInformation> freeSlots = slotTracker.getFreeSlots();
+
+		int numUnfulfilled = 0;
+		for (int x = 0; x < resourceRequirement.getNumberOfRequiredSlots(); x++) {
+
+			final Optional<TaskManagerSlotInformation> reservedSlot = slotMatchingStrategy.findMatchingSlot(requiredResource, freeSlots, this::getNumberRegisteredSlotsOf);
+			if (reservedSlot.isPresent()) {
+				// we do not need to modify freeSlots because it is indirectly modified by the allocation
+				allocateSlot(reservedSlot.get(), jobId, targetAddress, requiredResource);
+			} else {
+				numUnfulfilled++;
+			}
+		}
+		return numUnfulfilled;
+	}
+
+	/**
+	 * Allocates the given slot. This entails sending a registration message to the task manager and treating failures.
+	 *
+	 * @param taskManagerSlot slot to allocate
+	 * @param jobId job for which the slot should be allocated for
+	 * @param targetAddress address of the job master
+	 * @param resourceProfile resource profile for the requirement for which the slot is used
+	 */
+	private void allocateSlot(TaskManagerSlotInformation taskManagerSlot, JobID jobId, String targetAddress, ResourceProfile resourceProfile) {
+		TaskExecutorConnection taskExecutorConnection = taskManagerSlot.getTaskManagerConnection();
+		TaskExecutorGateway gateway = taskExecutorConnection.getTaskExecutorGateway();
+
+		final CompletableFuture<Acknowledge> completableFuture = new CompletableFuture<>();
+		final SlotID slotId = taskManagerSlot.getSlotId();
+		final InstanceID instanceId = taskManagerSlot.getInstanceId();
+
+		LOG.debug("Starting allocation of slot {} for job {} with resource profile {}.", taskManagerSlot.getSlotId(), jobId, resourceProfile);
+
+		slotTracker.notifyAllocationStart(slotId, jobId);
+
+		if (!taskExecutorManager.isTaskManagerRegistered(instanceId)) {
+			throw new IllegalStateException("Could not find a registered task manager for instance id " +
+				instanceId + '.');
+		}
+
+		taskExecutorManager.markUsed(instanceId);
+
+		// RPC call to the task manager
+		CompletableFuture<Acknowledge> requestFuture = gateway.requestSlot(
+			slotId,
+			jobId,
+			new AllocationID(),
+			resourceProfile,
+			targetAddress,
+			resourceManagerId,
+			taskManagerRequestTimeout);
+
+		requestFuture.whenComplete(
+			(Acknowledge acknowledge, Throwable throwable) -> {
+				if (acknowledge != null) {
+					completableFuture.complete(acknowledge);
+				} else {
+					completableFuture.completeExceptionally(throwable);
+				}
+			});
+
+		completableFuture.whenCompleteAsync(
+			(Acknowledge acknowledge, Throwable throwable) -> {
+				try {
+					if (acknowledge != null) {
+						LOG.trace("Completed allocation of slot {} for job {}.", slotId, jobId);
+						slotTracker.notifyAllocationComplete(slotId, jobId);
+					} else {
+						if (throwable instanceof SlotOccupiedException) {
+							SlotOccupiedException exception = (SlotOccupiedException) throwable;
+							LOG.debug("Tried allocating slot {} for job {}, but it was already allocated for job {}.", slotId, jobId, exception.getJobId());
+							// report as a slot status to force the state transition
+							// this could be a problem if we ever assume that the task executor always reports about all slots
+							slotTracker.notifySlotStatus(Collections.singleton(new SlotStatus(slotId, taskManagerSlot.getResourceProfile(), exception.getJobId(), exception.getAllocationId())));
+						} else {
+							if (throwable instanceof CancellationException) {
+								LOG.debug("Cancelled allocation of slot {} for job {}.", slotId, jobId, throwable);
+							} else {
+								LOG.warn("Slot allocation for slot {} for job {} failed.", slotId, jobId, throwable);
+								slotTracker.notifyFree(slotId);
+							}
+						}
+					}
+					checkResourceRequirements();
+				} catch (Exception e) {
+					LOG.error("Error while completing the slot allocation.", e);
+				}
+			},
+			mainThreadExecutor);
+	}
+
+	private void tryFulfillRequirementsWithPendingOrNewSlots(JobID jobId, Map<ResourceProfile, Integer> missingResources, ResourceCounter pendingSlots) {
+		for (Map.Entry<ResourceProfile, Integer> missingResource : missingResources.entrySet()) {
+			ResourceProfile profile = missingResource.getKey();
+			for (int i = 0; i < missingResource.getValue(); i++) {
+				if (!tryFulfillWithPendingSlots(profile, pendingSlots)) {
+					Optional<ResourceRequirement> newlyFulfillableRequirements = taskExecutorManager.allocateWorker(profile);
+					if (newlyFulfillableRequirements.isPresent()) {
+						ResourceRequirement newSlots = newlyFulfillableRequirements.get();
+						// reserve one of the new slots
+						if (newSlots.getNumberOfRequiredSlots() > 1) {
+							pendingSlots.incrementCount(newSlots.getResourceProfile(), newSlots.getNumberOfRequiredSlots() - 1);
+						}
+					} else {
+						LOG.warn("Could not fulfill resource requirements of job {}.", jobId);
+						resourceActions.notifyNotEnoughResourcesAvailable(jobId, resourceTracker.getAcquiredResources(jobId));
+						return;
+					}
+				}
+			}
+		}
+	}
+
+	private boolean tryFulfillWithPendingSlots(ResourceProfile resourceProfile, ResourceCounter pendingSlots) {
+		Set<ResourceProfile> pendingSlotProfiles = pendingSlots.getResourceProfiles();
+
+		// short-cut, pretty much only applicable to fine-grained resource management
+		if (pendingSlotProfiles.contains(resourceProfile)) {
+			pendingSlots.decrementCount(resourceProfile, 1);
+			return true;
+		}
+
+		for (ResourceProfile pendingSlotProfile : pendingSlotProfiles) {
+			if (pendingSlotProfile.isMatching(resourceProfile)) {
+				pendingSlots.decrementCount(pendingSlotProfile, 1);
+				return true;
+			}
+		}
+
+		return false;
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Legacy APIs
+	// ---------------------------------------------------------------------------------------------
+
+	@Override
+	public int getNumberRegisteredSlots() {
+		return taskExecutorManager.getNumberRegisteredSlots();
+	}
+
+	@Override
+	public int getNumberRegisteredSlotsOf(InstanceID instanceId) {
+		return taskExecutorManager.getNumberRegisteredSlotsOf(instanceId);
+	}
+
+	@Override
+	public int getNumberFreeSlots() {
+		return taskExecutorManager.getNumberFreeSlots();
+	}
+
+	@Override
+	public int getNumberFreeSlotsOf(InstanceID instanceId) {
+		return taskExecutorManager.getNumberFreeSlotsOf(instanceId);
+	}
+
+	@Override
+	public Map<WorkerResourceSpec, Integer> getRequiredResources() {
+		return taskExecutorManager.getRequiredWorkers();
+	}
+
+	@Override
+	public ResourceProfile getRegisteredResource() {
+		return taskExecutorManager.getTotalRegisteredResources();
+	}
+
+	@Override
+	public ResourceProfile getRegisteredResourceOf(InstanceID instanceID) {
+		return taskExecutorManager.getTotalRegisteredResourcesOf(instanceID);
+	}
+
+	@Override
+	public ResourceProfile getFreeResource() {
+		return taskExecutorManager.getTotalFreeResources();
+	}
+
+	@Override
+	public ResourceProfile getFreeResourceOf(InstanceID instanceID) {
+		return taskExecutorManager.getTotalFreeResourcesOf(instanceID);
+	}
+
+	@Override
+	public void setFailUnfulfillableRequest(boolean failUnfulfillableRequest) {
+		// we always send notifications if we cannot fulfill requests, and it is the responsibility of the JobManager
+		// to handle it (e.g., by reducing requirements and failing outright)
+	}
+
+	@Override
+	public int getNumberPendingSlotRequests() {
+		// only exists for testing purposes
+		throw new UnsupportedOperationException();
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Internal utility methods
+	// ---------------------------------------------------------------------------------------------
+
+	private void checkInit() {
+		Preconditions.checkState(started, "The slot manager has not been started.");
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Testing methods
+	// ---------------------------------------------------------------------------------------------
+
+	@Override
+	@VisibleForTesting
+	public void unregisterTaskManagersAndReleaseResources() {

Review comment:
       This method only exists for a single test case in the `YarnResourceManagerTest`. I think it is quite questionable because it not only relies on RM-internal components (the SlotManager) to trigger the unregistration/release (instead of unregistering the TE at the RM), but also introduces a custom codepath into the SlotManager for releasing resources that in this form doesn't appear to exist in production.
   The test asserts that the container is released upon calling this method, but it begs the question what we are even testing. If we are just interested in whether the Yarn RM resource actions can properly shut down the container, then we shouldn't need the slot manager for that.
   

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManager.java
##########
@@ -0,0 +1,609 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.metrics.MetricNames;
+import org.apache.flink.runtime.metrics.groups.SlotManagerMetricGroup;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerId;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.slots.ResourceCounter;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+import org.apache.flink.runtime.slots.ResourceRequirements;
+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.SlotOccupiedException;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ScheduledFuture;
+import java.util.function.BiFunction;
+import java.util.stream.Collectors;
+
+/**
+ * Implementation of {@link SlotManager} supporting declarative slot management.
+ */
+public class DeclarativeSlotManager implements SlotManager {
+	private static final Logger LOG = LoggerFactory.getLogger(DeclarativeSlotManager.class);
+
+	private final SlotTracker slotTracker;
+	private final ResourceTracker resourceTracker;
+	private final BiFunction<Executor, ResourceActions, TaskExecutorManager> taskExecutorManagerFactory;
+	private TaskExecutorManager taskExecutorManager;
+
+	/** Timeout for slot requests to the task manager. */
+	private final Time taskManagerRequestTimeout;
+	private ScheduledFuture<?> slotRequestTimeoutCheck;
+
+	private final SlotMatchingStrategy slotMatchingStrategy;
+
+	private final SlotManagerMetricGroup slotManagerMetricGroup;
+
+	private final Map<JobID, String> jobMasterTargetAddresses = new HashMap<>();
+	private final HashMap<SlotID, CompletableFuture<Acknowledge>> pendingSlotAllocationFutures;
+
+	/** ResourceManager's id. */
+	private ResourceManagerId resourceManagerId;
+
+	/** Executor for future callbacks which have to be "synchronized". */
+	private Executor mainThreadExecutor;
+
+	/** Callbacks for resource (de-)allocations. */
+	private ResourceActions resourceActions;
+
+	/** True iff the component has been started. */
+	private boolean started;
+
+	public DeclarativeSlotManager(
+			ScheduledExecutor scheduledExecutor,
+			SlotManagerConfiguration slotManagerConfiguration,
+			SlotManagerMetricGroup slotManagerMetricGroup,
+			ResourceTracker resourceTracker,
+			SlotTracker slotTracker) {
+
+		Preconditions.checkNotNull(slotManagerConfiguration);
+		this.taskManagerRequestTimeout = slotManagerConfiguration.getTaskManagerRequestTimeout();
+		this.slotManagerMetricGroup = Preconditions.checkNotNull(slotManagerMetricGroup);
+		this.resourceTracker = Preconditions.checkNotNull(resourceTracker);
+
+		pendingSlotAllocationFutures = new HashMap<>(16);
+
+		this.slotTracker = Preconditions.checkNotNull(slotTracker);
+		slotTracker.registerSlotStatusUpdateListener(createSlotStatusUpdateListener());
+
+		slotMatchingStrategy = slotManagerConfiguration.getSlotMatchingStrategy();
+
+		taskExecutorManagerFactory = (executor, resourceActions) -> new TaskExecutorManager(
+			slotManagerConfiguration.getDefaultWorkerResourceSpec(),
+			slotManagerConfiguration.getNumSlotsPerWorker(),
+			slotManagerConfiguration.getMaxSlotNum(),
+			slotManagerConfiguration.isWaitResultConsumedBeforeRelease(),
+			slotManagerConfiguration.getRedundantTaskManagerNum(),
+			slotManagerConfiguration.getTaskManagerTimeout(),
+			scheduledExecutor,
+			executor,
+			resourceActions);
+
+		resourceManagerId = null;
+		resourceActions = null;
+		mainThreadExecutor = null;
+		slotRequestTimeoutCheck = null;
+		taskExecutorManager = null;
+
+		started = false;
+	}
+
+	private SlotStatusUpdateListener createSlotStatusUpdateListener() {
+		return (taskManagerSlot, previous, current, jobId) -> {
+			if (previous == SlotState.PENDING) {
+				cancelAllocationFuture(taskManagerSlot.getSlotId());
+			}
+
+			if (current == SlotState.PENDING) {
+				resourceTracker.notifyAcquiredResource(jobId, taskManagerSlot.getResourceProfile());
+			}
+			if (current == SlotState.FREE) {
+				resourceTracker.notifyLostResource(jobId, taskManagerSlot.getResourceProfile());
+			}
+
+			if (current == SlotState.ALLOCATED) {
+				taskExecutorManager.occupySlot(taskManagerSlot.getInstanceId());
+			}
+			if (previous == SlotState.ALLOCATED && current == SlotState.FREE) {
+				taskExecutorManager.freeSlot(taskManagerSlot.getInstanceId());
+			}
+		};
+	}
+
+	private void cancelAllocationFuture(SlotID slotId) {
+		final CompletableFuture<Acknowledge> acknowledgeCompletableFuture = pendingSlotAllocationFutures.remove(slotId);
+		// the future may be null if we are just re-playing the state transitions due to a slot report
+		if (acknowledgeCompletableFuture != null) {
+			acknowledgeCompletableFuture.cancel(false);
+		}
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Component lifecycle methods
+	// ---------------------------------------------------------------------------------------------
+
+	/**
+	 * Starts the slot manager with the given leader id and resource manager actions.
+	 *
+	 * @param newResourceManagerId to use for communication with the task managers
+	 * @param newMainThreadExecutor to use to run code in the ResourceManager's main thread
+	 * @param newResourceActions to use for resource (de-)allocations
+	 */
+	@Override
+	public void start(ResourceManagerId newResourceManagerId, Executor newMainThreadExecutor, ResourceActions newResourceActions) {
+		LOG.info("Starting the slot manager.");
+
+		this.resourceManagerId = Preconditions.checkNotNull(newResourceManagerId);
+		mainThreadExecutor = Preconditions.checkNotNull(newMainThreadExecutor);
+		resourceActions = Preconditions.checkNotNull(newResourceActions);
+		taskExecutorManager = taskExecutorManagerFactory.apply(newMainThreadExecutor, newResourceActions);
+
+		started = true;
+
+		registerSlotManagerMetrics();
+	}
+
+	private void registerSlotManagerMetrics() {
+		slotManagerMetricGroup.gauge(
+			MetricNames.TASK_SLOTS_AVAILABLE,
+			() -> (long) getNumberFreeSlots());
+		slotManagerMetricGroup.gauge(
+			MetricNames.TASK_SLOTS_TOTAL,
+			() -> (long) getNumberRegisteredSlots());
+	}
+
+	/**
+	 * Suspends the component. This clears the internal state of the slot manager.
+	 */
+	@Override
+	public void suspend() {
+		LOG.info("Suspending the slot manager.");
+
+		if (slotRequestTimeoutCheck != null) {
+			slotRequestTimeoutCheck.cancel(false);
+			slotRequestTimeoutCheck = null;
+		}
+
+		resourceTracker.clear();
+		taskExecutorManager.close();
+
+		for (InstanceID registeredTaskManager : taskExecutorManager.getTaskExecutors()) {
+			unregisterTaskManager(registeredTaskManager, new SlotManagerException("The slot manager is being suspended."));
+		}
+
+		taskExecutorManager = null;
+		resourceManagerId = null;
+		resourceActions = null;
+		started = false;
+	}
+
+	/**
+	 * Closes the slot manager.
+	 *
+	 * @throws Exception if the close operation fails
+	 */
+	@Override
+	public void close() throws Exception {
+		LOG.info("Closing the slot manager.");
+
+		suspend();
+		slotManagerMetricGroup.close();
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Public API
+	// ---------------------------------------------------------------------------------------------
+
+	@Override
+	public void processResourceRequirements(ResourceRequirements resourceRequirements) {
+		checkInit();
+		LOG.debug("Received resource requirements from job {}: {}", resourceRequirements.getJobId(), resourceRequirements.getResourceRequirements());
+
+		if (resourceRequirements.getResourceRequirements().isEmpty()) {
+			jobMasterTargetAddresses.remove(resourceRequirements.getJobId());
+		} else {
+			jobMasterTargetAddresses.put(resourceRequirements.getJobId(), resourceRequirements.getTargetAddress());
+		}
+		resourceTracker.notifyResourceRequirements(resourceRequirements.getJobId(), resourceRequirements.getResourceRequirements());
+		checkResourceRequirements();
+	}
+
+	/**
+	 * 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
+	 * @return True if the task manager has not been registered before and is registered successfully; otherwise false
+	 */
+	@Override
+	public boolean registerTaskManager(final TaskExecutorConnection taskExecutorConnection, SlotReport initialSlotReport) {
+		checkInit();
+		LOG.debug("Registering task executor {} under {} at the slot manager.", taskExecutorConnection.getResourceID(), taskExecutorConnection.getInstanceID());
+
+		// we identify task managers by their instance id
+		if (taskExecutorManager.isTaskManagerRegistered(taskExecutorConnection.getInstanceID())) {
+			LOG.debug("Task executor {} was already registered.", taskExecutorConnection.getResourceID());
+			reportSlotStatus(taskExecutorConnection.getInstanceID(), initialSlotReport);
+			return false;
+		} else {
+			if (!taskExecutorManager.registerTaskManager(taskExecutorConnection, initialSlotReport)) {
+				LOG.debug("Task executor {} could not be registered.", taskExecutorConnection.getResourceID());
+				return false;
+			}
+
+			// register the new slots
+			for (SlotStatus slotStatus : initialSlotReport) {
+				slotTracker.addSlot(
+					slotStatus.getSlotID(),
+					slotStatus.getResourceProfile(),
+					taskExecutorConnection,
+					slotStatus.getJobID());
+			}
+
+			checkResourceRequirements();
+			return true;
+		}
+	}
+
+	@Override
+	public boolean unregisterTaskManager(InstanceID instanceId, Exception cause) {
+		checkInit();
+
+		LOG.debug("Unregistering task executor {} from the slot manager.", instanceId);
+
+		if (taskExecutorManager.isTaskManagerRegistered(instanceId)) {
+			slotTracker.removeSlots(taskExecutorManager.getSlotsOf(instanceId));
+			taskExecutorManager.unregisterTaskExecutor(instanceId);
+			checkResourceRequirements();
+
+			return true;
+		} else {
+			LOG.debug("There is no task executor registered with instance ID {}. Ignoring this message.", instanceId);
+
+			return false;
+		}
+	}
+
+	/**
+	 * Reports the current slot allocations for a task manager identified by the given instance id.
+	 *
+	 * @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
+	 */
+	@Override
+	public boolean reportSlotStatus(InstanceID instanceId, SlotReport slotReport) {
+		checkInit();
+
+		LOG.debug("Received slot report from instance {}: {}.", instanceId, slotReport);
+
+		if (taskExecutorManager.isTaskManagerRegistered(instanceId)) {
+			slotTracker.notifySlotStatus(slotReport);
+			checkResourceRequirements();
+			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
+	 */
+	@Override
+	public void freeSlot(SlotID slotId, AllocationID allocationId) {
+		checkInit();
+		LOG.debug("Freeing slot {}.", slotId);
+
+		slotTracker.notifyFree(slotId);
+		checkResourceRequirements();
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Requirement matching
+	// ---------------------------------------------------------------------------------------------
+
+	private void checkResourceRequirements() {
+		final Map<JobID, Collection<ResourceRequirement>> missingResources = resourceTracker.getMissingResources();
+		if (missingResources.isEmpty()) {
+			return;
+		}
+
+		final Map<JobID, ResourceCounter> outstandingRequirements = new LinkedHashMap<>();
+		for (Map.Entry<JobID, Collection<ResourceRequirement>> resourceRequirements : missingResources.entrySet()) {
+			JobID jobId = resourceRequirements.getKey();
+
+			for (ResourceRequirement resourceRequirement : resourceRequirements.getValue()) {
+				int numMissingSlots = internalRequestSlots(jobId, jobMasterTargetAddresses.get(jobId), resourceRequirement);
+				if (numMissingSlots > 0) {
+					outstandingRequirements
+						.computeIfAbsent(jobId, ignored -> new ResourceCounter())
+						.incrementCount(resourceRequirement.getResourceProfile(), numMissingSlots);
+				}
+			}
+		}
+
+		final ResourceCounter pendingSlots = new ResourceCounter(taskExecutorManager.getPendingTaskManagerSlots().stream().collect(
+			Collectors.groupingBy(
+				PendingTaskManagerSlot::getResourceProfile,
+				Collectors.summingInt(x -> 1))));
+
+		for (Map.Entry<JobID, ResourceCounter> unfulfilledRequirement : outstandingRequirements.entrySet()) {
+			tryFulfillRequirementsWithPendingOrNewSlots(
+				unfulfilledRequirement.getKey(),
+				unfulfilledRequirement.getValue().getResourceProfilesWithCount(),
+				pendingSlots);
+		}
+	}
+
+	/**
+	 * Tries to allocate slots for the given requirement. If there are not enough slots available, the
+	 * resource manager is informed to allocate more resources.
+	 *
+	 * @param jobId job to allocate slots for
+	 * @param targetAddress address of the jobmaster
+	 * @param resourceRequirement required slots
+	 * @return the number of missing slots
+	 */
+	private int internalRequestSlots(JobID jobId, String targetAddress, ResourceRequirement resourceRequirement) {
+		final ResourceProfile requiredResource = resourceRequirement.getResourceProfile();
+		Collection<TaskManagerSlotInformation> freeSlots = slotTracker.getFreeSlots();
+
+		int numUnfulfilled = 0;
+		for (int x = 0; x < resourceRequirement.getNumberOfRequiredSlots(); x++) {
+
+			final Optional<TaskManagerSlotInformation> reservedSlot = slotMatchingStrategy.findMatchingSlot(requiredResource, freeSlots, this::getNumberRegisteredSlotsOf);
+			if (reservedSlot.isPresent()) {
+				// we do not need to modify freeSlots because it is indirectly modified by the allocation
+				allocateSlot(reservedSlot.get(), jobId, targetAddress, requiredResource);
+			} else {
+				numUnfulfilled++;
+			}
+		}
+		return numUnfulfilled;
+	}
+
+	/**
+	 * Allocates the given slot. This entails sending a registration message to the task manager and treating failures.
+	 *
+	 * @param taskManagerSlot slot to allocate
+	 * @param jobId job for which the slot should be allocated for
+	 * @param targetAddress address of the job master
+	 * @param resourceProfile resource profile for the requirement for which the slot is used
+	 */
+	private void allocateSlot(TaskManagerSlotInformation taskManagerSlot, JobID jobId, String targetAddress, ResourceProfile resourceProfile) {
+		TaskExecutorConnection taskExecutorConnection = taskManagerSlot.getTaskManagerConnection();
+		TaskExecutorGateway gateway = taskExecutorConnection.getTaskExecutorGateway();
+
+		final CompletableFuture<Acknowledge> completableFuture = new CompletableFuture<>();
+		final SlotID slotId = taskManagerSlot.getSlotId();
+		final InstanceID instanceId = taskManagerSlot.getInstanceId();
+
+		LOG.debug("Starting allocation of slot {} for job {} with resource profile {}.", taskManagerSlot.getSlotId(), jobId, resourceProfile);
+
+		slotTracker.notifyAllocationStart(slotId, jobId);
+
+		if (!taskExecutorManager.isTaskManagerRegistered(instanceId)) {
+			throw new IllegalStateException("Could not find a registered task manager for instance id " +
+				instanceId + '.');
+		}
+
+		taskExecutorManager.markUsed(instanceId);
+
+		// RPC call to the task manager
+		CompletableFuture<Acknowledge> requestFuture = gateway.requestSlot(
+			slotId,
+			jobId,
+			new AllocationID(),
+			resourceProfile,
+			targetAddress,
+			resourceManagerId,
+			taskManagerRequestTimeout);
+
+		requestFuture.whenComplete(
+			(Acknowledge acknowledge, Throwable throwable) -> {
+				if (acknowledge != null) {
+					completableFuture.complete(acknowledge);
+				} else {
+					completableFuture.completeExceptionally(throwable);
+				}
+			});
+
+		completableFuture.whenCompleteAsync(
+			(Acknowledge acknowledge, Throwable throwable) -> {
+				try {
+					if (acknowledge != null) {
+						LOG.trace("Completed allocation of slot {} for job {}.", slotId, jobId);
+						slotTracker.notifyAllocationComplete(slotId, jobId);
+					} else {
+						if (throwable instanceof SlotOccupiedException) {
+							SlotOccupiedException exception = (SlotOccupiedException) throwable;
+							LOG.debug("Tried allocating slot {} for job {}, but it was already allocated for job {}.", slotId, jobId, exception.getJobId());
+							// report as a slot status to force the state transition
+							// this could be a problem if we ever assume that the task executor always reports about all slots
+							slotTracker.notifySlotStatus(Collections.singleton(new SlotStatus(slotId, taskManagerSlot.getResourceProfile(), exception.getJobId(), exception.getAllocationId())));
+						} else {
+							if (throwable instanceof CancellationException) {
+								LOG.debug("Cancelled allocation of slot {} for job {}.", slotId, jobId, throwable);
+							} else {
+								LOG.warn("Slot allocation for slot {} for job {} failed.", slotId, jobId, throwable);
+								slotTracker.notifyFree(slotId);
+							}
+						}
+					}
+					checkResourceRequirements();
+				} catch (Exception e) {
+					LOG.error("Error while completing the slot allocation.", e);
+				}
+			},
+			mainThreadExecutor);
+	}
+
+	private void tryFulfillRequirementsWithPendingOrNewSlots(JobID jobId, Map<ResourceProfile, Integer> missingResources, ResourceCounter pendingSlots) {

Review comment:
       yes

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManager.java
##########
@@ -0,0 +1,609 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.metrics.MetricNames;
+import org.apache.flink.runtime.metrics.groups.SlotManagerMetricGroup;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerId;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.slots.ResourceCounter;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+import org.apache.flink.runtime.slots.ResourceRequirements;
+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.SlotOccupiedException;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ScheduledFuture;
+import java.util.function.BiFunction;
+import java.util.stream.Collectors;
+
+/**
+ * Implementation of {@link SlotManager} supporting declarative slot management.
+ */
+public class DeclarativeSlotManager implements SlotManager {
+	private static final Logger LOG = LoggerFactory.getLogger(DeclarativeSlotManager.class);
+
+	private final SlotTracker slotTracker;
+	private final ResourceTracker resourceTracker;
+	private final BiFunction<Executor, ResourceActions, TaskExecutorManager> taskExecutorManagerFactory;
+	private TaskExecutorManager taskExecutorManager;
+
+	/** Timeout for slot requests to the task manager. */
+	private final Time taskManagerRequestTimeout;
+	private ScheduledFuture<?> slotRequestTimeoutCheck;
+
+	private final SlotMatchingStrategy slotMatchingStrategy;
+
+	private final SlotManagerMetricGroup slotManagerMetricGroup;
+
+	private final Map<JobID, String> jobMasterTargetAddresses = new HashMap<>();
+	private final HashMap<SlotID, CompletableFuture<Acknowledge>> pendingSlotAllocationFutures;
+
+	/** ResourceManager's id. */
+	private ResourceManagerId resourceManagerId;
+
+	/** Executor for future callbacks which have to be "synchronized". */
+	private Executor mainThreadExecutor;
+
+	/** Callbacks for resource (de-)allocations. */
+	private ResourceActions resourceActions;
+
+	/** True iff the component has been started. */
+	private boolean started;
+
+	public DeclarativeSlotManager(
+			ScheduledExecutor scheduledExecutor,
+			SlotManagerConfiguration slotManagerConfiguration,
+			SlotManagerMetricGroup slotManagerMetricGroup,
+			ResourceTracker resourceTracker,
+			SlotTracker slotTracker) {
+
+		Preconditions.checkNotNull(slotManagerConfiguration);
+		this.taskManagerRequestTimeout = slotManagerConfiguration.getTaskManagerRequestTimeout();
+		this.slotManagerMetricGroup = Preconditions.checkNotNull(slotManagerMetricGroup);
+		this.resourceTracker = Preconditions.checkNotNull(resourceTracker);
+
+		pendingSlotAllocationFutures = new HashMap<>(16);
+
+		this.slotTracker = Preconditions.checkNotNull(slotTracker);
+		slotTracker.registerSlotStatusUpdateListener(createSlotStatusUpdateListener());
+
+		slotMatchingStrategy = slotManagerConfiguration.getSlotMatchingStrategy();
+
+		taskExecutorManagerFactory = (executor, resourceActions) -> new TaskExecutorManager(
+			slotManagerConfiguration.getDefaultWorkerResourceSpec(),
+			slotManagerConfiguration.getNumSlotsPerWorker(),
+			slotManagerConfiguration.getMaxSlotNum(),
+			slotManagerConfiguration.isWaitResultConsumedBeforeRelease(),
+			slotManagerConfiguration.getRedundantTaskManagerNum(),
+			slotManagerConfiguration.getTaskManagerTimeout(),
+			scheduledExecutor,
+			executor,
+			resourceActions);
+
+		resourceManagerId = null;
+		resourceActions = null;
+		mainThreadExecutor = null;
+		slotRequestTimeoutCheck = null;
+		taskExecutorManager = null;
+
+		started = false;
+	}
+
+	private SlotStatusUpdateListener createSlotStatusUpdateListener() {
+		return (taskManagerSlot, previous, current, jobId) -> {
+			if (previous == SlotState.PENDING) {
+				cancelAllocationFuture(taskManagerSlot.getSlotId());
+			}
+
+			if (current == SlotState.PENDING) {
+				resourceTracker.notifyAcquiredResource(jobId, taskManagerSlot.getResourceProfile());
+			}
+			if (current == SlotState.FREE) {
+				resourceTracker.notifyLostResource(jobId, taskManagerSlot.getResourceProfile());
+			}
+
+			if (current == SlotState.ALLOCATED) {
+				taskExecutorManager.occupySlot(taskManagerSlot.getInstanceId());
+			}
+			if (previous == SlotState.ALLOCATED && current == SlotState.FREE) {
+				taskExecutorManager.freeSlot(taskManagerSlot.getInstanceId());
+			}
+		};
+	}
+
+	private void cancelAllocationFuture(SlotID slotId) {
+		final CompletableFuture<Acknowledge> acknowledgeCompletableFuture = pendingSlotAllocationFutures.remove(slotId);
+		// the future may be null if we are just re-playing the state transitions due to a slot report
+		if (acknowledgeCompletableFuture != null) {
+			acknowledgeCompletableFuture.cancel(false);
+		}
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Component lifecycle methods
+	// ---------------------------------------------------------------------------------------------
+
+	/**
+	 * Starts the slot manager with the given leader id and resource manager actions.
+	 *
+	 * @param newResourceManagerId to use for communication with the task managers
+	 * @param newMainThreadExecutor to use to run code in the ResourceManager's main thread
+	 * @param newResourceActions to use for resource (de-)allocations
+	 */
+	@Override
+	public void start(ResourceManagerId newResourceManagerId, Executor newMainThreadExecutor, ResourceActions newResourceActions) {
+		LOG.info("Starting the slot manager.");
+
+		this.resourceManagerId = Preconditions.checkNotNull(newResourceManagerId);
+		mainThreadExecutor = Preconditions.checkNotNull(newMainThreadExecutor);
+		resourceActions = Preconditions.checkNotNull(newResourceActions);
+		taskExecutorManager = taskExecutorManagerFactory.apply(newMainThreadExecutor, newResourceActions);
+
+		started = true;
+
+		registerSlotManagerMetrics();
+	}
+
+	private void registerSlotManagerMetrics() {
+		slotManagerMetricGroup.gauge(
+			MetricNames.TASK_SLOTS_AVAILABLE,
+			() -> (long) getNumberFreeSlots());
+		slotManagerMetricGroup.gauge(
+			MetricNames.TASK_SLOTS_TOTAL,
+			() -> (long) getNumberRegisteredSlots());
+	}
+
+	/**
+	 * Suspends the component. This clears the internal state of the slot manager.
+	 */
+	@Override
+	public void suspend() {
+		LOG.info("Suspending the slot manager.");
+
+		if (slotRequestTimeoutCheck != null) {
+			slotRequestTimeoutCheck.cancel(false);
+			slotRequestTimeoutCheck = null;
+		}
+
+		resourceTracker.clear();
+		taskExecutorManager.close();
+
+		for (InstanceID registeredTaskManager : taskExecutorManager.getTaskExecutors()) {
+			unregisterTaskManager(registeredTaskManager, new SlotManagerException("The slot manager is being suspended."));
+		}
+
+		taskExecutorManager = null;
+		resourceManagerId = null;
+		resourceActions = null;
+		started = false;
+	}
+
+	/**
+	 * Closes the slot manager.
+	 *
+	 * @throws Exception if the close operation fails
+	 */
+	@Override
+	public void close() throws Exception {
+		LOG.info("Closing the slot manager.");
+
+		suspend();
+		slotManagerMetricGroup.close();
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Public API
+	// ---------------------------------------------------------------------------------------------
+
+	@Override
+	public void processResourceRequirements(ResourceRequirements resourceRequirements) {
+		checkInit();
+		LOG.debug("Received resource requirements from job {}: {}", resourceRequirements.getJobId(), resourceRequirements.getResourceRequirements());
+
+		if (resourceRequirements.getResourceRequirements().isEmpty()) {
+			jobMasterTargetAddresses.remove(resourceRequirements.getJobId());
+		} else {
+			jobMasterTargetAddresses.put(resourceRequirements.getJobId(), resourceRequirements.getTargetAddress());
+		}
+		resourceTracker.notifyResourceRequirements(resourceRequirements.getJobId(), resourceRequirements.getResourceRequirements());
+		checkResourceRequirements();
+	}
+
+	/**
+	 * 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
+	 * @return True if the task manager has not been registered before and is registered successfully; otherwise false
+	 */
+	@Override
+	public boolean registerTaskManager(final TaskExecutorConnection taskExecutorConnection, SlotReport initialSlotReport) {
+		checkInit();
+		LOG.debug("Registering task executor {} under {} at the slot manager.", taskExecutorConnection.getResourceID(), taskExecutorConnection.getInstanceID());
+
+		// we identify task managers by their instance id
+		if (taskExecutorManager.isTaskManagerRegistered(taskExecutorConnection.getInstanceID())) {
+			LOG.debug("Task executor {} was already registered.", taskExecutorConnection.getResourceID());
+			reportSlotStatus(taskExecutorConnection.getInstanceID(), initialSlotReport);
+			return false;
+		} else {
+			if (!taskExecutorManager.registerTaskManager(taskExecutorConnection, initialSlotReport)) {
+				LOG.debug("Task executor {} could not be registered.", taskExecutorConnection.getResourceID());
+				return false;
+			}
+
+			// register the new slots
+			for (SlotStatus slotStatus : initialSlotReport) {
+				slotTracker.addSlot(
+					slotStatus.getSlotID(),
+					slotStatus.getResourceProfile(),
+					taskExecutorConnection,
+					slotStatus.getJobID());
+			}
+
+			checkResourceRequirements();
+			return true;
+		}
+	}
+
+	@Override
+	public boolean unregisterTaskManager(InstanceID instanceId, Exception cause) {
+		checkInit();
+
+		LOG.debug("Unregistering task executor {} from the slot manager.", instanceId);
+
+		if (taskExecutorManager.isTaskManagerRegistered(instanceId)) {
+			slotTracker.removeSlots(taskExecutorManager.getSlotsOf(instanceId));
+			taskExecutorManager.unregisterTaskExecutor(instanceId);
+			checkResourceRequirements();
+
+			return true;
+		} else {
+			LOG.debug("There is no task executor registered with instance ID {}. Ignoring this message.", instanceId);
+
+			return false;
+		}
+	}
+
+	/**
+	 * Reports the current slot allocations for a task manager identified by the given instance id.
+	 *
+	 * @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
+	 */
+	@Override
+	public boolean reportSlotStatus(InstanceID instanceId, SlotReport slotReport) {
+		checkInit();
+
+		LOG.debug("Received slot report from instance {}: {}.", instanceId, slotReport);
+
+		if (taskExecutorManager.isTaskManagerRegistered(instanceId)) {
+			slotTracker.notifySlotStatus(slotReport);
+			checkResourceRequirements();
+			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
+	 */
+	@Override
+	public void freeSlot(SlotID slotId, AllocationID allocationId) {
+		checkInit();
+		LOG.debug("Freeing slot {}.", slotId);
+
+		slotTracker.notifyFree(slotId);
+		checkResourceRequirements();
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Requirement matching
+	// ---------------------------------------------------------------------------------------------
+
+	private void checkResourceRequirements() {
+		final Map<JobID, Collection<ResourceRequirement>> missingResources = resourceTracker.getMissingResources();
+		if (missingResources.isEmpty()) {
+			return;
+		}
+
+		final Map<JobID, ResourceCounter> outstandingRequirements = new LinkedHashMap<>();
+		for (Map.Entry<JobID, Collection<ResourceRequirement>> resourceRequirements : missingResources.entrySet()) {
+			JobID jobId = resourceRequirements.getKey();
+
+			for (ResourceRequirement resourceRequirement : resourceRequirements.getValue()) {
+				int numMissingSlots = internalRequestSlots(jobId, jobMasterTargetAddresses.get(jobId), resourceRequirement);
+				if (numMissingSlots > 0) {
+					outstandingRequirements
+						.computeIfAbsent(jobId, ignored -> new ResourceCounter())
+						.incrementCount(resourceRequirement.getResourceProfile(), numMissingSlots);
+				}
+			}
+		}
+
+		final ResourceCounter pendingSlots = new ResourceCounter(taskExecutorManager.getPendingTaskManagerSlots().stream().collect(
+			Collectors.groupingBy(
+				PendingTaskManagerSlot::getResourceProfile,
+				Collectors.summingInt(x -> 1))));
+
+		for (Map.Entry<JobID, ResourceCounter> unfulfilledRequirement : outstandingRequirements.entrySet()) {
+			tryFulfillRequirementsWithPendingOrNewSlots(
+				unfulfilledRequirement.getKey(),
+				unfulfilledRequirement.getValue().getResourceProfilesWithCount(),
+				pendingSlots);
+		}
+	}
+
+	/**
+	 * Tries to allocate slots for the given requirement. If there are not enough slots available, the
+	 * resource manager is informed to allocate more resources.
+	 *
+	 * @param jobId job to allocate slots for
+	 * @param targetAddress address of the jobmaster
+	 * @param resourceRequirement required slots
+	 * @return the number of missing slots
+	 */
+	private int internalRequestSlots(JobID jobId, String targetAddress, ResourceRequirement resourceRequirement) {
+		final ResourceProfile requiredResource = resourceRequirement.getResourceProfile();
+		Collection<TaskManagerSlotInformation> freeSlots = slotTracker.getFreeSlots();
+
+		int numUnfulfilled = 0;
+		for (int x = 0; x < resourceRequirement.getNumberOfRequiredSlots(); x++) {
+
+			final Optional<TaskManagerSlotInformation> reservedSlot = slotMatchingStrategy.findMatchingSlot(requiredResource, freeSlots, this::getNumberRegisteredSlotsOf);
+			if (reservedSlot.isPresent()) {
+				// we do not need to modify freeSlots because it is indirectly modified by the allocation
+				allocateSlot(reservedSlot.get(), jobId, targetAddress, requiredResource);
+			} else {
+				numUnfulfilled++;
+			}
+		}
+		return numUnfulfilled;
+	}
+
+	/**
+	 * Allocates the given slot. This entails sending a registration message to the task manager and treating failures.
+	 *
+	 * @param taskManagerSlot slot to allocate
+	 * @param jobId job for which the slot should be allocated for
+	 * @param targetAddress address of the job master
+	 * @param resourceProfile resource profile for the requirement for which the slot is used
+	 */
+	private void allocateSlot(TaskManagerSlotInformation taskManagerSlot, JobID jobId, String targetAddress, ResourceProfile resourceProfile) {
+		TaskExecutorConnection taskExecutorConnection = taskManagerSlot.getTaskManagerConnection();
+		TaskExecutorGateway gateway = taskExecutorConnection.getTaskExecutorGateway();
+
+		final CompletableFuture<Acknowledge> completableFuture = new CompletableFuture<>();
+		final SlotID slotId = taskManagerSlot.getSlotId();
+		final InstanceID instanceId = taskManagerSlot.getInstanceId();
+
+		LOG.debug("Starting allocation of slot {} for job {} with resource profile {}.", taskManagerSlot.getSlotId(), jobId, resourceProfile);
+
+		slotTracker.notifyAllocationStart(slotId, jobId);
+
+		if (!taskExecutorManager.isTaskManagerRegistered(instanceId)) {
+			throw new IllegalStateException("Could not find a registered task manager for instance id " +
+				instanceId + '.');
+		}
+
+		taskExecutorManager.markUsed(instanceId);
+
+		// RPC call to the task manager
+		CompletableFuture<Acknowledge> requestFuture = gateway.requestSlot(
+			slotId,
+			jobId,
+			new AllocationID(),
+			resourceProfile,
+			targetAddress,
+			resourceManagerId,
+			taskManagerRequestTimeout);
+
+		requestFuture.whenComplete(
+			(Acknowledge acknowledge, Throwable throwable) -> {
+				if (acknowledge != null) {
+					completableFuture.complete(acknowledge);
+				} else {
+					completableFuture.completeExceptionally(throwable);
+				}
+			});
+
+		completableFuture.whenCompleteAsync(
+			(Acknowledge acknowledge, Throwable throwable) -> {
+				try {
+					if (acknowledge != null) {
+						LOG.trace("Completed allocation of slot {} for job {}.", slotId, jobId);
+						slotTracker.notifyAllocationComplete(slotId, jobId);
+					} else {
+						if (throwable instanceof SlotOccupiedException) {
+							SlotOccupiedException exception = (SlotOccupiedException) throwable;
+							LOG.debug("Tried allocating slot {} for job {}, but it was already allocated for job {}.", slotId, jobId, exception.getJobId());
+							// report as a slot status to force the state transition
+							// this could be a problem if we ever assume that the task executor always reports about all slots
+							slotTracker.notifySlotStatus(Collections.singleton(new SlotStatus(slotId, taskManagerSlot.getResourceProfile(), exception.getJobId(), exception.getAllocationId())));
+						} else {
+							if (throwable instanceof CancellationException) {
+								LOG.debug("Cancelled allocation of slot {} for job {}.", slotId, jobId, throwable);

Review comment:
       Whenever a pending slot switches to another state (see `createSlotStatusUpdateListener`)

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManager.java
##########
@@ -0,0 +1,609 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.metrics.MetricNames;
+import org.apache.flink.runtime.metrics.groups.SlotManagerMetricGroup;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerId;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.slots.ResourceCounter;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+import org.apache.flink.runtime.slots.ResourceRequirements;
+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.SlotOccupiedException;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ScheduledFuture;
+import java.util.function.BiFunction;
+import java.util.stream.Collectors;
+
+/**
+ * Implementation of {@link SlotManager} supporting declarative slot management.
+ */
+public class DeclarativeSlotManager implements SlotManager {
+	private static final Logger LOG = LoggerFactory.getLogger(DeclarativeSlotManager.class);
+
+	private final SlotTracker slotTracker;
+	private final ResourceTracker resourceTracker;
+	private final BiFunction<Executor, ResourceActions, TaskExecutorManager> taskExecutorManagerFactory;
+	private TaskExecutorManager taskExecutorManager;
+
+	/** Timeout for slot requests to the task manager. */
+	private final Time taskManagerRequestTimeout;
+	private ScheduledFuture<?> slotRequestTimeoutCheck;
+
+	private final SlotMatchingStrategy slotMatchingStrategy;
+
+	private final SlotManagerMetricGroup slotManagerMetricGroup;
+
+	private final Map<JobID, String> jobMasterTargetAddresses = new HashMap<>();
+	private final HashMap<SlotID, CompletableFuture<Acknowledge>> pendingSlotAllocationFutures;
+
+	/** ResourceManager's id. */
+	private ResourceManagerId resourceManagerId;
+
+	/** Executor for future callbacks which have to be "synchronized". */
+	private Executor mainThreadExecutor;
+
+	/** Callbacks for resource (de-)allocations. */
+	private ResourceActions resourceActions;
+
+	/** True iff the component has been started. */
+	private boolean started;
+
+	public DeclarativeSlotManager(
+			ScheduledExecutor scheduledExecutor,
+			SlotManagerConfiguration slotManagerConfiguration,
+			SlotManagerMetricGroup slotManagerMetricGroup,
+			ResourceTracker resourceTracker,
+			SlotTracker slotTracker) {
+
+		Preconditions.checkNotNull(slotManagerConfiguration);
+		this.taskManagerRequestTimeout = slotManagerConfiguration.getTaskManagerRequestTimeout();
+		this.slotManagerMetricGroup = Preconditions.checkNotNull(slotManagerMetricGroup);
+		this.resourceTracker = Preconditions.checkNotNull(resourceTracker);
+
+		pendingSlotAllocationFutures = new HashMap<>(16);
+
+		this.slotTracker = Preconditions.checkNotNull(slotTracker);
+		slotTracker.registerSlotStatusUpdateListener(createSlotStatusUpdateListener());
+
+		slotMatchingStrategy = slotManagerConfiguration.getSlotMatchingStrategy();
+
+		taskExecutorManagerFactory = (executor, resourceActions) -> new TaskExecutorManager(
+			slotManagerConfiguration.getDefaultWorkerResourceSpec(),
+			slotManagerConfiguration.getNumSlotsPerWorker(),
+			slotManagerConfiguration.getMaxSlotNum(),
+			slotManagerConfiguration.isWaitResultConsumedBeforeRelease(),
+			slotManagerConfiguration.getRedundantTaskManagerNum(),
+			slotManagerConfiguration.getTaskManagerTimeout(),
+			scheduledExecutor,
+			executor,
+			resourceActions);
+
+		resourceManagerId = null;
+		resourceActions = null;
+		mainThreadExecutor = null;
+		slotRequestTimeoutCheck = null;
+		taskExecutorManager = null;
+
+		started = false;
+	}
+
+	private SlotStatusUpdateListener createSlotStatusUpdateListener() {
+		return (taskManagerSlot, previous, current, jobId) -> {
+			if (previous == SlotState.PENDING) {
+				cancelAllocationFuture(taskManagerSlot.getSlotId());
+			}
+
+			if (current == SlotState.PENDING) {
+				resourceTracker.notifyAcquiredResource(jobId, taskManagerSlot.getResourceProfile());
+			}
+			if (current == SlotState.FREE) {
+				resourceTracker.notifyLostResource(jobId, taskManagerSlot.getResourceProfile());
+			}
+
+			if (current == SlotState.ALLOCATED) {
+				taskExecutorManager.occupySlot(taskManagerSlot.getInstanceId());
+			}
+			if (previous == SlotState.ALLOCATED && current == SlotState.FREE) {
+				taskExecutorManager.freeSlot(taskManagerSlot.getInstanceId());
+			}
+		};
+	}
+
+	private void cancelAllocationFuture(SlotID slotId) {
+		final CompletableFuture<Acknowledge> acknowledgeCompletableFuture = pendingSlotAllocationFutures.remove(slotId);
+		// the future may be null if we are just re-playing the state transitions due to a slot report
+		if (acknowledgeCompletableFuture != null) {
+			acknowledgeCompletableFuture.cancel(false);
+		}
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Component lifecycle methods
+	// ---------------------------------------------------------------------------------------------
+
+	/**
+	 * Starts the slot manager with the given leader id and resource manager actions.
+	 *
+	 * @param newResourceManagerId to use for communication with the task managers
+	 * @param newMainThreadExecutor to use to run code in the ResourceManager's main thread
+	 * @param newResourceActions to use for resource (de-)allocations
+	 */
+	@Override
+	public void start(ResourceManagerId newResourceManagerId, Executor newMainThreadExecutor, ResourceActions newResourceActions) {
+		LOG.info("Starting the slot manager.");
+
+		this.resourceManagerId = Preconditions.checkNotNull(newResourceManagerId);
+		mainThreadExecutor = Preconditions.checkNotNull(newMainThreadExecutor);
+		resourceActions = Preconditions.checkNotNull(newResourceActions);
+		taskExecutorManager = taskExecutorManagerFactory.apply(newMainThreadExecutor, newResourceActions);
+
+		started = true;
+
+		registerSlotManagerMetrics();
+	}
+
+	private void registerSlotManagerMetrics() {
+		slotManagerMetricGroup.gauge(
+			MetricNames.TASK_SLOTS_AVAILABLE,
+			() -> (long) getNumberFreeSlots());
+		slotManagerMetricGroup.gauge(
+			MetricNames.TASK_SLOTS_TOTAL,
+			() -> (long) getNumberRegisteredSlots());
+	}
+
+	/**
+	 * Suspends the component. This clears the internal state of the slot manager.
+	 */
+	@Override
+	public void suspend() {
+		LOG.info("Suspending the slot manager.");
+
+		if (slotRequestTimeoutCheck != null) {
+			slotRequestTimeoutCheck.cancel(false);
+			slotRequestTimeoutCheck = null;
+		}
+
+		resourceTracker.clear();
+		taskExecutorManager.close();
+
+		for (InstanceID registeredTaskManager : taskExecutorManager.getTaskExecutors()) {
+			unregisterTaskManager(registeredTaskManager, new SlotManagerException("The slot manager is being suspended."));
+		}
+
+		taskExecutorManager = null;
+		resourceManagerId = null;
+		resourceActions = null;
+		started = false;
+	}
+
+	/**
+	 * Closes the slot manager.
+	 *
+	 * @throws Exception if the close operation fails
+	 */
+	@Override
+	public void close() throws Exception {
+		LOG.info("Closing the slot manager.");
+
+		suspend();
+		slotManagerMetricGroup.close();
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Public API
+	// ---------------------------------------------------------------------------------------------
+
+	@Override
+	public void processResourceRequirements(ResourceRequirements resourceRequirements) {
+		checkInit();
+		LOG.debug("Received resource requirements from job {}: {}", resourceRequirements.getJobId(), resourceRequirements.getResourceRequirements());
+
+		if (resourceRequirements.getResourceRequirements().isEmpty()) {
+			jobMasterTargetAddresses.remove(resourceRequirements.getJobId());
+		} else {
+			jobMasterTargetAddresses.put(resourceRequirements.getJobId(), resourceRequirements.getTargetAddress());
+		}
+		resourceTracker.notifyResourceRequirements(resourceRequirements.getJobId(), resourceRequirements.getResourceRequirements());
+		checkResourceRequirements();
+	}
+
+	/**
+	 * 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
+	 * @return True if the task manager has not been registered before and is registered successfully; otherwise false
+	 */
+	@Override
+	public boolean registerTaskManager(final TaskExecutorConnection taskExecutorConnection, SlotReport initialSlotReport) {
+		checkInit();
+		LOG.debug("Registering task executor {} under {} at the slot manager.", taskExecutorConnection.getResourceID(), taskExecutorConnection.getInstanceID());
+
+		// we identify task managers by their instance id
+		if (taskExecutorManager.isTaskManagerRegistered(taskExecutorConnection.getInstanceID())) {
+			LOG.debug("Task executor {} was already registered.", taskExecutorConnection.getResourceID());
+			reportSlotStatus(taskExecutorConnection.getInstanceID(), initialSlotReport);
+			return false;
+		} else {
+			if (!taskExecutorManager.registerTaskManager(taskExecutorConnection, initialSlotReport)) {
+				LOG.debug("Task executor {} could not be registered.", taskExecutorConnection.getResourceID());
+				return false;
+			}
+
+			// register the new slots
+			for (SlotStatus slotStatus : initialSlotReport) {
+				slotTracker.addSlot(
+					slotStatus.getSlotID(),
+					slotStatus.getResourceProfile(),
+					taskExecutorConnection,
+					slotStatus.getJobID());
+			}
+
+			checkResourceRequirements();
+			return true;
+		}
+	}
+
+	@Override
+	public boolean unregisterTaskManager(InstanceID instanceId, Exception cause) {
+		checkInit();
+
+		LOG.debug("Unregistering task executor {} from the slot manager.", instanceId);
+
+		if (taskExecutorManager.isTaskManagerRegistered(instanceId)) {
+			slotTracker.removeSlots(taskExecutorManager.getSlotsOf(instanceId));
+			taskExecutorManager.unregisterTaskExecutor(instanceId);
+			checkResourceRequirements();
+
+			return true;
+		} else {
+			LOG.debug("There is no task executor registered with instance ID {}. Ignoring this message.", instanceId);
+
+			return false;
+		}
+	}
+
+	/**
+	 * Reports the current slot allocations for a task manager identified by the given instance id.
+	 *
+	 * @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
+	 */
+	@Override
+	public boolean reportSlotStatus(InstanceID instanceId, SlotReport slotReport) {
+		checkInit();
+
+		LOG.debug("Received slot report from instance {}: {}.", instanceId, slotReport);
+
+		if (taskExecutorManager.isTaskManagerRegistered(instanceId)) {
+			slotTracker.notifySlotStatus(slotReport);
+			checkResourceRequirements();
+			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
+	 */
+	@Override
+	public void freeSlot(SlotID slotId, AllocationID allocationId) {
+		checkInit();
+		LOG.debug("Freeing slot {}.", slotId);
+
+		slotTracker.notifyFree(slotId);
+		checkResourceRequirements();
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Requirement matching
+	// ---------------------------------------------------------------------------------------------
+
+	private void checkResourceRequirements() {
+		final Map<JobID, Collection<ResourceRequirement>> missingResources = resourceTracker.getMissingResources();
+		if (missingResources.isEmpty()) {
+			return;
+		}
+
+		final Map<JobID, ResourceCounter> outstandingRequirements = new LinkedHashMap<>();
+		for (Map.Entry<JobID, Collection<ResourceRequirement>> resourceRequirements : missingResources.entrySet()) {
+			JobID jobId = resourceRequirements.getKey();
+
+			for (ResourceRequirement resourceRequirement : resourceRequirements.getValue()) {
+				int numMissingSlots = internalRequestSlots(jobId, jobMasterTargetAddresses.get(jobId), resourceRequirement);
+				if (numMissingSlots > 0) {
+					outstandingRequirements
+						.computeIfAbsent(jobId, ignored -> new ResourceCounter())
+						.incrementCount(resourceRequirement.getResourceProfile(), numMissingSlots);
+				}
+			}
+		}
+
+		final ResourceCounter pendingSlots = new ResourceCounter(taskExecutorManager.getPendingTaskManagerSlots().stream().collect(
+			Collectors.groupingBy(
+				PendingTaskManagerSlot::getResourceProfile,
+				Collectors.summingInt(x -> 1))));
+
+		for (Map.Entry<JobID, ResourceCounter> unfulfilledRequirement : outstandingRequirements.entrySet()) {
+			tryFulfillRequirementsWithPendingOrNewSlots(
+				unfulfilledRequirement.getKey(),
+				unfulfilledRequirement.getValue().getResourceProfilesWithCount(),
+				pendingSlots);
+		}
+	}
+
+	/**
+	 * Tries to allocate slots for the given requirement. If there are not enough slots available, the
+	 * resource manager is informed to allocate more resources.
+	 *
+	 * @param jobId job to allocate slots for
+	 * @param targetAddress address of the jobmaster
+	 * @param resourceRequirement required slots
+	 * @return the number of missing slots
+	 */
+	private int internalRequestSlots(JobID jobId, String targetAddress, ResourceRequirement resourceRequirement) {
+		final ResourceProfile requiredResource = resourceRequirement.getResourceProfile();
+		Collection<TaskManagerSlotInformation> freeSlots = slotTracker.getFreeSlots();
+
+		int numUnfulfilled = 0;
+		for (int x = 0; x < resourceRequirement.getNumberOfRequiredSlots(); x++) {
+
+			final Optional<TaskManagerSlotInformation> reservedSlot = slotMatchingStrategy.findMatchingSlot(requiredResource, freeSlots, this::getNumberRegisteredSlotsOf);
+			if (reservedSlot.isPresent()) {
+				// we do not need to modify freeSlots because it is indirectly modified by the allocation
+				allocateSlot(reservedSlot.get(), jobId, targetAddress, requiredResource);
+			} else {
+				numUnfulfilled++;
+			}
+		}
+		return numUnfulfilled;
+	}
+
+	/**
+	 * Allocates the given slot. This entails sending a registration message to the task manager and treating failures.
+	 *
+	 * @param taskManagerSlot slot to allocate
+	 * @param jobId job for which the slot should be allocated for
+	 * @param targetAddress address of the job master
+	 * @param resourceProfile resource profile for the requirement for which the slot is used
+	 */
+	private void allocateSlot(TaskManagerSlotInformation taskManagerSlot, JobID jobId, String targetAddress, ResourceProfile resourceProfile) {
+		TaskExecutorConnection taskExecutorConnection = taskManagerSlot.getTaskManagerConnection();
+		TaskExecutorGateway gateway = taskExecutorConnection.getTaskExecutorGateway();
+
+		final CompletableFuture<Acknowledge> completableFuture = new CompletableFuture<>();
+		final SlotID slotId = taskManagerSlot.getSlotId();
+		final InstanceID instanceId = taskManagerSlot.getInstanceId();
+
+		LOG.debug("Starting allocation of slot {} for job {} with resource profile {}.", taskManagerSlot.getSlotId(), jobId, resourceProfile);
+
+		slotTracker.notifyAllocationStart(slotId, jobId);
+
+		if (!taskExecutorManager.isTaskManagerRegistered(instanceId)) {
+			throw new IllegalStateException("Could not find a registered task manager for instance id " +
+				instanceId + '.');
+		}
+
+		taskExecutorManager.markUsed(instanceId);
+
+		// RPC call to the task manager
+		CompletableFuture<Acknowledge> requestFuture = gateway.requestSlot(
+			slotId,
+			jobId,
+			new AllocationID(),
+			resourceProfile,
+			targetAddress,
+			resourceManagerId,
+			taskManagerRequestTimeout);
+
+		requestFuture.whenComplete(
+			(Acknowledge acknowledge, Throwable throwable) -> {
+				if (acknowledge != null) {
+					completableFuture.complete(acknowledge);
+				} else {
+					completableFuture.completeExceptionally(throwable);
+				}
+			});
+
+		completableFuture.whenCompleteAsync(
+			(Acknowledge acknowledge, Throwable throwable) -> {
+				try {
+					if (acknowledge != null) {
+						LOG.trace("Completed allocation of slot {} for job {}.", slotId, jobId);
+						slotTracker.notifyAllocationComplete(slotId, jobId);
+					} else {
+						if (throwable instanceof SlotOccupiedException) {
+							SlotOccupiedException exception = (SlotOccupiedException) throwable;
+							LOG.debug("Tried allocating slot {} for job {}, but it was already allocated for job {}.", slotId, jobId, exception.getJobId());
+							// report as a slot status to force the state transition
+							// this could be a problem if we ever assume that the task executor always reports about all slots
+							slotTracker.notifySlotStatus(Collections.singleton(new SlotStatus(slotId, taskManagerSlot.getResourceProfile(), exception.getJobId(), exception.getAllocationId())));
+						} else {
+							if (throwable instanceof CancellationException) {
+								LOG.debug("Cancelled allocation of slot {} for job {}.", slotId, jobId, throwable);
+							} else {
+								LOG.warn("Slot allocation for slot {} for job {} failed.", slotId, jobId, throwable);
+								slotTracker.notifyFree(slotId);
+							}
+						}
+					}
+					checkResourceRequirements();
+				} catch (Exception e) {
+					LOG.error("Error while completing the slot allocation.", e);
+				}

Review comment:
       This was just taken as is from the existing `SlotManagerImpl`. I can change it though.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManager.java
##########
@@ -0,0 +1,609 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.metrics.MetricNames;
+import org.apache.flink.runtime.metrics.groups.SlotManagerMetricGroup;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerId;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.slots.ResourceCounter;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+import org.apache.flink.runtime.slots.ResourceRequirements;
+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.SlotOccupiedException;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ScheduledFuture;
+import java.util.function.BiFunction;
+import java.util.stream.Collectors;
+
+/**
+ * Implementation of {@link SlotManager} supporting declarative slot management.
+ */
+public class DeclarativeSlotManager implements SlotManager {
+	private static final Logger LOG = LoggerFactory.getLogger(DeclarativeSlotManager.class);
+
+	private final SlotTracker slotTracker;
+	private final ResourceTracker resourceTracker;
+	private final BiFunction<Executor, ResourceActions, TaskExecutorManager> taskExecutorManagerFactory;
+	private TaskExecutorManager taskExecutorManager;
+
+	/** Timeout for slot requests to the task manager. */
+	private final Time taskManagerRequestTimeout;
+	private ScheduledFuture<?> slotRequestTimeoutCheck;
+
+	private final SlotMatchingStrategy slotMatchingStrategy;
+
+	private final SlotManagerMetricGroup slotManagerMetricGroup;
+
+	private final Map<JobID, String> jobMasterTargetAddresses = new HashMap<>();
+	private final HashMap<SlotID, CompletableFuture<Acknowledge>> pendingSlotAllocationFutures;
+
+	/** ResourceManager's id. */
+	private ResourceManagerId resourceManagerId;
+
+	/** Executor for future callbacks which have to be "synchronized". */
+	private Executor mainThreadExecutor;
+
+	/** Callbacks for resource (de-)allocations. */
+	private ResourceActions resourceActions;
+
+	/** True iff the component has been started. */
+	private boolean started;
+
+	public DeclarativeSlotManager(
+			ScheduledExecutor scheduledExecutor,
+			SlotManagerConfiguration slotManagerConfiguration,
+			SlotManagerMetricGroup slotManagerMetricGroup,
+			ResourceTracker resourceTracker,
+			SlotTracker slotTracker) {
+
+		Preconditions.checkNotNull(slotManagerConfiguration);
+		this.taskManagerRequestTimeout = slotManagerConfiguration.getTaskManagerRequestTimeout();
+		this.slotManagerMetricGroup = Preconditions.checkNotNull(slotManagerMetricGroup);
+		this.resourceTracker = Preconditions.checkNotNull(resourceTracker);
+
+		pendingSlotAllocationFutures = new HashMap<>(16);
+
+		this.slotTracker = Preconditions.checkNotNull(slotTracker);
+		slotTracker.registerSlotStatusUpdateListener(createSlotStatusUpdateListener());
+
+		slotMatchingStrategy = slotManagerConfiguration.getSlotMatchingStrategy();
+
+		taskExecutorManagerFactory = (executor, resourceActions) -> new TaskExecutorManager(
+			slotManagerConfiguration.getDefaultWorkerResourceSpec(),
+			slotManagerConfiguration.getNumSlotsPerWorker(),
+			slotManagerConfiguration.getMaxSlotNum(),
+			slotManagerConfiguration.isWaitResultConsumedBeforeRelease(),
+			slotManagerConfiguration.getRedundantTaskManagerNum(),
+			slotManagerConfiguration.getTaskManagerTimeout(),
+			scheduledExecutor,
+			executor,
+			resourceActions);
+
+		resourceManagerId = null;
+		resourceActions = null;
+		mainThreadExecutor = null;
+		slotRequestTimeoutCheck = null;
+		taskExecutorManager = null;
+
+		started = false;
+	}
+
+	private SlotStatusUpdateListener createSlotStatusUpdateListener() {
+		return (taskManagerSlot, previous, current, jobId) -> {
+			if (previous == SlotState.PENDING) {
+				cancelAllocationFuture(taskManagerSlot.getSlotId());
+			}
+
+			if (current == SlotState.PENDING) {
+				resourceTracker.notifyAcquiredResource(jobId, taskManagerSlot.getResourceProfile());
+			}
+			if (current == SlotState.FREE) {
+				resourceTracker.notifyLostResource(jobId, taskManagerSlot.getResourceProfile());
+			}
+
+			if (current == SlotState.ALLOCATED) {
+				taskExecutorManager.occupySlot(taskManagerSlot.getInstanceId());
+			}
+			if (previous == SlotState.ALLOCATED && current == SlotState.FREE) {
+				taskExecutorManager.freeSlot(taskManagerSlot.getInstanceId());
+			}
+		};
+	}
+
+	private void cancelAllocationFuture(SlotID slotId) {
+		final CompletableFuture<Acknowledge> acknowledgeCompletableFuture = pendingSlotAllocationFutures.remove(slotId);
+		// the future may be null if we are just re-playing the state transitions due to a slot report
+		if (acknowledgeCompletableFuture != null) {
+			acknowledgeCompletableFuture.cancel(false);
+		}
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Component lifecycle methods
+	// ---------------------------------------------------------------------------------------------
+
+	/**
+	 * Starts the slot manager with the given leader id and resource manager actions.
+	 *
+	 * @param newResourceManagerId to use for communication with the task managers
+	 * @param newMainThreadExecutor to use to run code in the ResourceManager's main thread
+	 * @param newResourceActions to use for resource (de-)allocations
+	 */
+	@Override
+	public void start(ResourceManagerId newResourceManagerId, Executor newMainThreadExecutor, ResourceActions newResourceActions) {
+		LOG.info("Starting the slot manager.");
+
+		this.resourceManagerId = Preconditions.checkNotNull(newResourceManagerId);
+		mainThreadExecutor = Preconditions.checkNotNull(newMainThreadExecutor);
+		resourceActions = Preconditions.checkNotNull(newResourceActions);
+		taskExecutorManager = taskExecutorManagerFactory.apply(newMainThreadExecutor, newResourceActions);
+
+		started = true;
+
+		registerSlotManagerMetrics();
+	}
+
+	private void registerSlotManagerMetrics() {
+		slotManagerMetricGroup.gauge(
+			MetricNames.TASK_SLOTS_AVAILABLE,
+			() -> (long) getNumberFreeSlots());
+		slotManagerMetricGroup.gauge(
+			MetricNames.TASK_SLOTS_TOTAL,
+			() -> (long) getNumberRegisteredSlots());
+	}
+
+	/**
+	 * Suspends the component. This clears the internal state of the slot manager.
+	 */
+	@Override
+	public void suspend() {
+		LOG.info("Suspending the slot manager.");
+
+		if (slotRequestTimeoutCheck != null) {
+			slotRequestTimeoutCheck.cancel(false);
+			slotRequestTimeoutCheck = null;
+		}
+
+		resourceTracker.clear();
+		taskExecutorManager.close();
+
+		for (InstanceID registeredTaskManager : taskExecutorManager.getTaskExecutors()) {
+			unregisterTaskManager(registeredTaskManager, new SlotManagerException("The slot manager is being suspended."));
+		}
+
+		taskExecutorManager = null;
+		resourceManagerId = null;
+		resourceActions = null;
+		started = false;
+	}
+
+	/**
+	 * Closes the slot manager.
+	 *
+	 * @throws Exception if the close operation fails
+	 */
+	@Override
+	public void close() throws Exception {
+		LOG.info("Closing the slot manager.");
+
+		suspend();
+		slotManagerMetricGroup.close();
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Public API
+	// ---------------------------------------------------------------------------------------------
+
+	@Override
+	public void processResourceRequirements(ResourceRequirements resourceRequirements) {
+		checkInit();
+		LOG.debug("Received resource requirements from job {}: {}", resourceRequirements.getJobId(), resourceRequirements.getResourceRequirements());
+
+		if (resourceRequirements.getResourceRequirements().isEmpty()) {
+			jobMasterTargetAddresses.remove(resourceRequirements.getJobId());
+		} else {
+			jobMasterTargetAddresses.put(resourceRequirements.getJobId(), resourceRequirements.getTargetAddress());
+		}
+		resourceTracker.notifyResourceRequirements(resourceRequirements.getJobId(), resourceRequirements.getResourceRequirements());
+		checkResourceRequirements();
+	}
+
+	/**
+	 * 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
+	 * @return True if the task manager has not been registered before and is registered successfully; otherwise false
+	 */
+	@Override
+	public boolean registerTaskManager(final TaskExecutorConnection taskExecutorConnection, SlotReport initialSlotReport) {
+		checkInit();
+		LOG.debug("Registering task executor {} under {} at the slot manager.", taskExecutorConnection.getResourceID(), taskExecutorConnection.getInstanceID());
+
+		// we identify task managers by their instance id
+		if (taskExecutorManager.isTaskManagerRegistered(taskExecutorConnection.getInstanceID())) {
+			LOG.debug("Task executor {} was already registered.", taskExecutorConnection.getResourceID());
+			reportSlotStatus(taskExecutorConnection.getInstanceID(), initialSlotReport);
+			return false;
+		} else {
+			if (!taskExecutorManager.registerTaskManager(taskExecutorConnection, initialSlotReport)) {
+				LOG.debug("Task executor {} could not be registered.", taskExecutorConnection.getResourceID());
+				return false;
+			}
+
+			// register the new slots
+			for (SlotStatus slotStatus : initialSlotReport) {
+				slotTracker.addSlot(
+					slotStatus.getSlotID(),
+					slotStatus.getResourceProfile(),
+					taskExecutorConnection,
+					slotStatus.getJobID());
+			}
+
+			checkResourceRequirements();
+			return true;
+		}
+	}
+
+	@Override
+	public boolean unregisterTaskManager(InstanceID instanceId, Exception cause) {
+		checkInit();
+
+		LOG.debug("Unregistering task executor {} from the slot manager.", instanceId);
+
+		if (taskExecutorManager.isTaskManagerRegistered(instanceId)) {
+			slotTracker.removeSlots(taskExecutorManager.getSlotsOf(instanceId));
+			taskExecutorManager.unregisterTaskExecutor(instanceId);
+			checkResourceRequirements();
+
+			return true;
+		} else {
+			LOG.debug("There is no task executor registered with instance ID {}. Ignoring this message.", instanceId);
+
+			return false;
+		}
+	}
+
+	/**
+	 * Reports the current slot allocations for a task manager identified by the given instance id.
+	 *
+	 * @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
+	 */
+	@Override
+	public boolean reportSlotStatus(InstanceID instanceId, SlotReport slotReport) {
+		checkInit();
+
+		LOG.debug("Received slot report from instance {}: {}.", instanceId, slotReport);
+
+		if (taskExecutorManager.isTaskManagerRegistered(instanceId)) {
+			slotTracker.notifySlotStatus(slotReport);
+			checkResourceRequirements();
+			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
+	 */
+	@Override
+	public void freeSlot(SlotID slotId, AllocationID allocationId) {
+		checkInit();
+		LOG.debug("Freeing slot {}.", slotId);
+
+		slotTracker.notifyFree(slotId);
+		checkResourceRequirements();
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Requirement matching
+	// ---------------------------------------------------------------------------------------------
+
+	private void checkResourceRequirements() {
+		final Map<JobID, Collection<ResourceRequirement>> missingResources = resourceTracker.getMissingResources();
+		if (missingResources.isEmpty()) {
+			return;
+		}
+
+		final Map<JobID, ResourceCounter> outstandingRequirements = new LinkedHashMap<>();
+		for (Map.Entry<JobID, Collection<ResourceRequirement>> resourceRequirements : missingResources.entrySet()) {
+			JobID jobId = resourceRequirements.getKey();
+
+			for (ResourceRequirement resourceRequirement : resourceRequirements.getValue()) {
+				int numMissingSlots = internalRequestSlots(jobId, jobMasterTargetAddresses.get(jobId), resourceRequirement);
+				if (numMissingSlots > 0) {
+					outstandingRequirements
+						.computeIfAbsent(jobId, ignored -> new ResourceCounter())
+						.incrementCount(resourceRequirement.getResourceProfile(), numMissingSlots);
+				}
+			}
+		}
+
+		final ResourceCounter pendingSlots = new ResourceCounter(taskExecutorManager.getPendingTaskManagerSlots().stream().collect(
+			Collectors.groupingBy(
+				PendingTaskManagerSlot::getResourceProfile,
+				Collectors.summingInt(x -> 1))));
+
+		for (Map.Entry<JobID, ResourceCounter> unfulfilledRequirement : outstandingRequirements.entrySet()) {
+			tryFulfillRequirementsWithPendingOrNewSlots(
+				unfulfilledRequirement.getKey(),
+				unfulfilledRequirement.getValue().getResourceProfilesWithCount(),
+				pendingSlots);
+		}
+	}
+
+	/**
+	 * Tries to allocate slots for the given requirement. If there are not enough slots available, the
+	 * resource manager is informed to allocate more resources.
+	 *
+	 * @param jobId job to allocate slots for
+	 * @param targetAddress address of the jobmaster
+	 * @param resourceRequirement required slots
+	 * @return the number of missing slots
+	 */
+	private int internalRequestSlots(JobID jobId, String targetAddress, ResourceRequirement resourceRequirement) {
+		final ResourceProfile requiredResource = resourceRequirement.getResourceProfile();
+		Collection<TaskManagerSlotInformation> freeSlots = slotTracker.getFreeSlots();
+
+		int numUnfulfilled = 0;
+		for (int x = 0; x < resourceRequirement.getNumberOfRequiredSlots(); x++) {
+
+			final Optional<TaskManagerSlotInformation> reservedSlot = slotMatchingStrategy.findMatchingSlot(requiredResource, freeSlots, this::getNumberRegisteredSlotsOf);
+			if (reservedSlot.isPresent()) {
+				// we do not need to modify freeSlots because it is indirectly modified by the allocation
+				allocateSlot(reservedSlot.get(), jobId, targetAddress, requiredResource);
+			} else {
+				numUnfulfilled++;
+			}
+		}
+		return numUnfulfilled;
+	}
+
+	/**
+	 * Allocates the given slot. This entails sending a registration message to the task manager and treating failures.
+	 *
+	 * @param taskManagerSlot slot to allocate
+	 * @param jobId job for which the slot should be allocated for
+	 * @param targetAddress address of the job master
+	 * @param resourceProfile resource profile for the requirement for which the slot is used
+	 */
+	private void allocateSlot(TaskManagerSlotInformation taskManagerSlot, JobID jobId, String targetAddress, ResourceProfile resourceProfile) {
+		TaskExecutorConnection taskExecutorConnection = taskManagerSlot.getTaskManagerConnection();
+		TaskExecutorGateway gateway = taskExecutorConnection.getTaskExecutorGateway();
+
+		final CompletableFuture<Acknowledge> completableFuture = new CompletableFuture<>();
+		final SlotID slotId = taskManagerSlot.getSlotId();
+		final InstanceID instanceId = taskManagerSlot.getInstanceId();
+
+		LOG.debug("Starting allocation of slot {} for job {} with resource profile {}.", taskManagerSlot.getSlotId(), jobId, resourceProfile);
+
+		slotTracker.notifyAllocationStart(slotId, jobId);
+
+		if (!taskExecutorManager.isTaskManagerRegistered(instanceId)) {
+			throw new IllegalStateException("Could not find a registered task manager for instance id " +
+				instanceId + '.');
+		}
+
+		taskExecutorManager.markUsed(instanceId);
+
+		// RPC call to the task manager
+		CompletableFuture<Acknowledge> requestFuture = gateway.requestSlot(
+			slotId,
+			jobId,
+			new AllocationID(),
+			resourceProfile,
+			targetAddress,
+			resourceManagerId,
+			taskManagerRequestTimeout);
+
+		requestFuture.whenComplete(
+			(Acknowledge acknowledge, Throwable throwable) -> {
+				if (acknowledge != null) {
+					completableFuture.complete(acknowledge);
+				} else {
+					completableFuture.completeExceptionally(throwable);
+				}
+			});
+
+		completableFuture.whenCompleteAsync(
+			(Acknowledge acknowledge, Throwable throwable) -> {
+				try {
+					if (acknowledge != null) {
+						LOG.trace("Completed allocation of slot {} for job {}.", slotId, jobId);
+						slotTracker.notifyAllocationComplete(slotId, jobId);
+					} else {
+						if (throwable instanceof SlotOccupiedException) {
+							SlotOccupiedException exception = (SlotOccupiedException) throwable;
+							LOG.debug("Tried allocating slot {} for job {}, but it was already allocated for job {}.", slotId, jobId, exception.getJobId());
+							// report as a slot status to force the state transition
+							// this could be a problem if we ever assume that the task executor always reports about all slots
+							slotTracker.notifySlotStatus(Collections.singleton(new SlotStatus(slotId, taskManagerSlot.getResourceProfile(), exception.getJobId(), exception.getAllocationId())));
+						} else {
+							if (throwable instanceof CancellationException) {
+								LOG.debug("Cancelled allocation of slot {} for job {}.", slotId, jobId, throwable);
+							} else {
+								LOG.warn("Slot allocation for slot {} for job {} failed.", slotId, jobId, throwable);
+								slotTracker.notifyFree(slotId);
+							}
+						}
+					}
+					checkResourceRequirements();

Review comment:
       I can't quite think of a reason at the moment to check the requirements after a successful allocation

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManager.java
##########
@@ -0,0 +1,609 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.metrics.MetricNames;
+import org.apache.flink.runtime.metrics.groups.SlotManagerMetricGroup;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerId;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.slots.ResourceCounter;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+import org.apache.flink.runtime.slots.ResourceRequirements;
+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.SlotOccupiedException;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ScheduledFuture;
+import java.util.function.BiFunction;
+import java.util.stream.Collectors;
+
+/**
+ * Implementation of {@link SlotManager} supporting declarative slot management.
+ */
+public class DeclarativeSlotManager implements SlotManager {
+	private static final Logger LOG = LoggerFactory.getLogger(DeclarativeSlotManager.class);
+
+	private final SlotTracker slotTracker;
+	private final ResourceTracker resourceTracker;
+	private final BiFunction<Executor, ResourceActions, TaskExecutorManager> taskExecutorManagerFactory;
+	private TaskExecutorManager taskExecutorManager;
+
+	/** Timeout for slot requests to the task manager. */
+	private final Time taskManagerRequestTimeout;
+	private ScheduledFuture<?> slotRequestTimeoutCheck;
+
+	private final SlotMatchingStrategy slotMatchingStrategy;
+
+	private final SlotManagerMetricGroup slotManagerMetricGroup;
+
+	private final Map<JobID, String> jobMasterTargetAddresses = new HashMap<>();
+	private final HashMap<SlotID, CompletableFuture<Acknowledge>> pendingSlotAllocationFutures;
+
+	/** ResourceManager's id. */
+	private ResourceManagerId resourceManagerId;
+
+	/** Executor for future callbacks which have to be "synchronized". */
+	private Executor mainThreadExecutor;
+
+	/** Callbacks for resource (de-)allocations. */
+	private ResourceActions resourceActions;
+
+	/** True iff the component has been started. */
+	private boolean started;
+
+	public DeclarativeSlotManager(
+			ScheduledExecutor scheduledExecutor,
+			SlotManagerConfiguration slotManagerConfiguration,
+			SlotManagerMetricGroup slotManagerMetricGroup,
+			ResourceTracker resourceTracker,
+			SlotTracker slotTracker) {
+
+		Preconditions.checkNotNull(slotManagerConfiguration);
+		this.taskManagerRequestTimeout = slotManagerConfiguration.getTaskManagerRequestTimeout();
+		this.slotManagerMetricGroup = Preconditions.checkNotNull(slotManagerMetricGroup);
+		this.resourceTracker = Preconditions.checkNotNull(resourceTracker);
+
+		pendingSlotAllocationFutures = new HashMap<>(16);
+
+		this.slotTracker = Preconditions.checkNotNull(slotTracker);
+		slotTracker.registerSlotStatusUpdateListener(createSlotStatusUpdateListener());
+
+		slotMatchingStrategy = slotManagerConfiguration.getSlotMatchingStrategy();
+
+		taskExecutorManagerFactory = (executor, resourceActions) -> new TaskExecutorManager(
+			slotManagerConfiguration.getDefaultWorkerResourceSpec(),
+			slotManagerConfiguration.getNumSlotsPerWorker(),
+			slotManagerConfiguration.getMaxSlotNum(),
+			slotManagerConfiguration.isWaitResultConsumedBeforeRelease(),
+			slotManagerConfiguration.getRedundantTaskManagerNum(),
+			slotManagerConfiguration.getTaskManagerTimeout(),
+			scheduledExecutor,
+			executor,
+			resourceActions);
+
+		resourceManagerId = null;
+		resourceActions = null;
+		mainThreadExecutor = null;
+		slotRequestTimeoutCheck = null;
+		taskExecutorManager = null;
+
+		started = false;
+	}
+
+	private SlotStatusUpdateListener createSlotStatusUpdateListener() {
+		return (taskManagerSlot, previous, current, jobId) -> {
+			if (previous == SlotState.PENDING) {
+				cancelAllocationFuture(taskManagerSlot.getSlotId());
+			}
+
+			if (current == SlotState.PENDING) {
+				resourceTracker.notifyAcquiredResource(jobId, taskManagerSlot.getResourceProfile());
+			}
+			if (current == SlotState.FREE) {
+				resourceTracker.notifyLostResource(jobId, taskManagerSlot.getResourceProfile());
+			}
+
+			if (current == SlotState.ALLOCATED) {
+				taskExecutorManager.occupySlot(taskManagerSlot.getInstanceId());
+			}
+			if (previous == SlotState.ALLOCATED && current == SlotState.FREE) {
+				taskExecutorManager.freeSlot(taskManagerSlot.getInstanceId());
+			}
+		};
+	}
+
+	private void cancelAllocationFuture(SlotID slotId) {
+		final CompletableFuture<Acknowledge> acknowledgeCompletableFuture = pendingSlotAllocationFutures.remove(slotId);
+		// the future may be null if we are just re-playing the state transitions due to a slot report
+		if (acknowledgeCompletableFuture != null) {
+			acknowledgeCompletableFuture.cancel(false);
+		}
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Component lifecycle methods
+	// ---------------------------------------------------------------------------------------------
+
+	/**
+	 * Starts the slot manager with the given leader id and resource manager actions.
+	 *
+	 * @param newResourceManagerId to use for communication with the task managers
+	 * @param newMainThreadExecutor to use to run code in the ResourceManager's main thread
+	 * @param newResourceActions to use for resource (de-)allocations
+	 */
+	@Override
+	public void start(ResourceManagerId newResourceManagerId, Executor newMainThreadExecutor, ResourceActions newResourceActions) {
+		LOG.info("Starting the slot manager.");
+
+		this.resourceManagerId = Preconditions.checkNotNull(newResourceManagerId);
+		mainThreadExecutor = Preconditions.checkNotNull(newMainThreadExecutor);
+		resourceActions = Preconditions.checkNotNull(newResourceActions);
+		taskExecutorManager = taskExecutorManagerFactory.apply(newMainThreadExecutor, newResourceActions);
+
+		started = true;
+
+		registerSlotManagerMetrics();
+	}
+
+	private void registerSlotManagerMetrics() {
+		slotManagerMetricGroup.gauge(
+			MetricNames.TASK_SLOTS_AVAILABLE,
+			() -> (long) getNumberFreeSlots());
+		slotManagerMetricGroup.gauge(
+			MetricNames.TASK_SLOTS_TOTAL,
+			() -> (long) getNumberRegisteredSlots());
+	}

Review comment:
       not a problem in the sense that things will still work. But we are attempting to register metrics multiple which will cause warnings.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManager.java
##########
@@ -0,0 +1,609 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.metrics.MetricNames;
+import org.apache.flink.runtime.metrics.groups.SlotManagerMetricGroup;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerId;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.slots.ResourceCounter;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+import org.apache.flink.runtime.slots.ResourceRequirements;
+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.SlotOccupiedException;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ScheduledFuture;
+import java.util.function.BiFunction;
+import java.util.stream.Collectors;
+
+/**
+ * Implementation of {@link SlotManager} supporting declarative slot management.
+ */
+public class DeclarativeSlotManager implements SlotManager {
+	private static final Logger LOG = LoggerFactory.getLogger(DeclarativeSlotManager.class);
+
+	private final SlotTracker slotTracker;
+	private final ResourceTracker resourceTracker;
+	private final BiFunction<Executor, ResourceActions, TaskExecutorManager> taskExecutorManagerFactory;
+	private TaskExecutorManager taskExecutorManager;
+
+	/** Timeout for slot requests to the task manager. */
+	private final Time taskManagerRequestTimeout;
+	private ScheduledFuture<?> slotRequestTimeoutCheck;
+
+	private final SlotMatchingStrategy slotMatchingStrategy;
+
+	private final SlotManagerMetricGroup slotManagerMetricGroup;
+
+	private final Map<JobID, String> jobMasterTargetAddresses = new HashMap<>();
+	private final HashMap<SlotID, CompletableFuture<Acknowledge>> pendingSlotAllocationFutures;
+
+	/** ResourceManager's id. */
+	private ResourceManagerId resourceManagerId;

Review comment:
       I did not want to add nullable since we enforce via checkInit() that these fields are not null usually. Otherwise we'll get a whole bunch of IDE warnings about potential null pointer exceptions.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java
##########
@@ -861,6 +884,8 @@ protected void closeJobManagerConnection(JobID jobId, Exception cause) {
 
 			jmResourceIdRegistrations.remove(jobManagerResourceId);
 
+			slotManager.processResourceRequirements(ResourceRequirements.empty(jobId, jobMasterGateway.getAddress()));

Review comment:
       hmm...i think the current behavior is fine for the case where a JM just crashes, but when we have HA _I think_ we are also clearing the requirements when the leader lost leadership.
   Soo...`removeJob` may be a better place to put it.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] zentol commented on a change in pull request #13553: [FLINK-19306][coordination] Add DeclarativeSlotManager

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #13553:
URL: https://github.com/apache/flink/pull/13553#discussion_r502331874



##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManagerTest.java
##########
@@ -0,0 +1,1092 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple6;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.concurrent.ManuallyTriggeredScheduledExecutor;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerId;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+import org.apache.flink.runtime.slots.ResourceRequirements;
+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.TestingTaskExecutorGateway;
+import org.apache.flink.runtime.taskexecutor.TestingTaskExecutorGatewayBuilder;
+import org.apache.flink.runtime.taskexecutor.exceptions.SlotAllocationException;
+import org.apache.flink.runtime.taskexecutor.exceptions.SlotOccupiedException;
+import org.apache.flink.runtime.testingUtils.TestingUtils;
+import org.apache.flink.util.FlinkException;
+import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.function.FunctionUtils;
+
+import akka.pattern.AskTimeoutException;
+import org.junit.Ignore;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.hamcrest.CoreMatchers.hasItem;
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.empty;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.hasSize;
+import static org.hamcrest.Matchers.is;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertSame;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Tests for the {@link DeclarativeSlotManager}.
+ */
+public class DeclarativeSlotManagerTest extends TestLogger {
+
+	private static final FlinkException TEST_EXCEPTION = new FlinkException("Test exception");
+
+	private static final WorkerResourceSpec WORKER_RESOURCE_SPEC = new WorkerResourceSpec.Builder()
+		.setCpuCores(100.0)
+		.setTaskHeapMemoryMB(10000)
+		.setTaskOffHeapMemoryMB(10000)
+		.setNetworkMemoryMB(10000)
+		.setManagedMemoryMB(10000)
+		.build();
+
+	/**
+	 * Tests that we can register task manager and their slots at the slot manager.
+	 */
+	@Test
+	public void testTaskManagerRegistration() throws Exception {
+		final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder().createTestingTaskExecutorGateway();
+		final ResourceID resourceId = ResourceID.generate();
+		final TaskExecutorConnection taskManagerConnection = new TaskExecutorConnection(resourceId, taskExecutorGateway);
+
+		final SlotID slotId1 = new SlotID(resourceId, 0);
+		final SlotID slotId2 = new SlotID(resourceId, 1);
+		final SlotReport slotReport = new SlotReport(Arrays.asList(createFreeSlotStatus(slotId1), createFreeSlotStatus(slotId2)));
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport);
+
+			assertThat("The number registered slots does not equal the expected number.", slotManager.getNumberRegisteredSlots(), is(2));
+
+			assertNotNull(slotTracker.getSlot(slotId1));
+			assertNotNull(slotTracker.getSlot(slotId2));
+		}
+	}
+
+	/**
+	 * Tests that un-registration of task managers will free and remove all registered slots.
+	 */
+	@Test
+	public void testTaskManagerUnregistration() throws Exception {
+		final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder()
+			.setRequestSlotFunction(tuple6 -> new CompletableFuture<>())
+			.createTestingTaskExecutorGateway();
+		final TaskExecutorConnection taskManagerConnection = createTaskExecutorConnection(taskExecutorGateway);
+		final ResourceID resourceId = taskManagerConnection.getResourceID();
+
+		final SlotID slotId1 = new SlotID(resourceId, 0);
+		final SlotID slotId2 = new SlotID(resourceId, 1);
+		final SlotReport slotReport = new SlotReport(Arrays.asList(createAllocatedSlotStatus(slotId1), createFreeSlotStatus(slotId2)));
+
+		final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot();
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport);
+
+			assertEquals("The number registered slots does not equal the expected number.", 2, slotManager.getNumberRegisteredSlots());
+
+			DeclarativeTaskManagerSlot slot1 = slotTracker.getSlot(slotId1);
+			DeclarativeTaskManagerSlot slot2 = slotTracker.getSlot(slotId2);
+
+			assertSame(SlotState.ALLOCATED, slot1.getState());
+			assertSame(SlotState.FREE, slot2.getState());
+
+			slotManager.processResourceRequirements(resourceRequirements);
+
+			assertSame(SlotState.PENDING, slot2.getState());
+
+			slotManager.unregisterTaskManager(taskManagerConnection.getInstanceID(), TEST_EXCEPTION);
+
+			assertEquals(0, slotManager.getNumberRegisteredSlots());
+		}
+	}
+
+	/**
+	 * Tests that a slot request with no free slots will trigger the resource allocation.
+	 */
+	@Test
+	public void testSlotRequestWithoutFreeSlots() throws Exception {
+		final ResourceManagerId resourceManagerId = ResourceManagerId.generate();
+
+		final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot();
+
+		CompletableFuture<WorkerResourceSpec> allocateResourceFuture = new CompletableFuture<>();
+		ResourceActions resourceManagerActions = new TestingResourceActionsBuilder()
+			.setAllocateResourceConsumer(allocateResourceFuture::complete)
+			.build();
+
+		try (SlotManager slotManager = createSlotManager(resourceManagerId, resourceManagerActions)) {
+
+			slotManager.processResourceRequirements(resourceRequirements);
+
+			allocateResourceFuture.get();

Review comment:
       The size of the requested resource is not really dependent on the resource requirements; the only thing that is checked (by the TaskExecutorManager) is that the slots of said resource can fulfill the requirements.
   This is admittedly not tested, but belongs into the TaskExecutorManagerTest imo.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] zentol commented on a change in pull request #13553: [FLINK-19306][coordination] Add DeclarativeSlotManager

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #13553:
URL: https://github.com/apache/flink/pull/13553#discussion_r502362147



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManager.java
##########
@@ -0,0 +1,609 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.metrics.MetricNames;
+import org.apache.flink.runtime.metrics.groups.SlotManagerMetricGroup;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerId;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.slots.ResourceCounter;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+import org.apache.flink.runtime.slots.ResourceRequirements;
+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.SlotOccupiedException;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ScheduledFuture;
+import java.util.function.BiFunction;
+import java.util.stream.Collectors;
+
+/**
+ * Implementation of {@link SlotManager} supporting declarative slot management.
+ */
+public class DeclarativeSlotManager implements SlotManager {
+	private static final Logger LOG = LoggerFactory.getLogger(DeclarativeSlotManager.class);
+
+	private final SlotTracker slotTracker;
+	private final ResourceTracker resourceTracker;
+	private final BiFunction<Executor, ResourceActions, TaskExecutorManager> taskExecutorManagerFactory;
+	private TaskExecutorManager taskExecutorManager;
+
+	/** Timeout for slot requests to the task manager. */
+	private final Time taskManagerRequestTimeout;
+	private ScheduledFuture<?> slotRequestTimeoutCheck;
+
+	private final SlotMatchingStrategy slotMatchingStrategy;
+
+	private final SlotManagerMetricGroup slotManagerMetricGroup;
+
+	private final Map<JobID, String> jobMasterTargetAddresses = new HashMap<>();
+	private final HashMap<SlotID, CompletableFuture<Acknowledge>> pendingSlotAllocationFutures;
+
+	/** ResourceManager's id. */
+	private ResourceManagerId resourceManagerId;

Review comment:
       I did not want to add nullable since we enforce via checkInit() that these fields are not null usually. Otherwise we'll get a whole bunch of IDE warnings about potential null pointer exceptions.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] zentol commented on a change in pull request #13553: [FLINK-19306][coordination] Add DeclarativeSlotManager

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #13553:
URL: https://github.com/apache/flink/pull/13553#discussion_r502329434



##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManagerTest.java
##########
@@ -0,0 +1,1092 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple6;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.concurrent.ManuallyTriggeredScheduledExecutor;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerId;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+import org.apache.flink.runtime.slots.ResourceRequirements;
+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.TestingTaskExecutorGateway;
+import org.apache.flink.runtime.taskexecutor.TestingTaskExecutorGatewayBuilder;
+import org.apache.flink.runtime.taskexecutor.exceptions.SlotAllocationException;
+import org.apache.flink.runtime.taskexecutor.exceptions.SlotOccupiedException;
+import org.apache.flink.runtime.testingUtils.TestingUtils;
+import org.apache.flink.util.FlinkException;
+import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.function.FunctionUtils;
+
+import akka.pattern.AskTimeoutException;
+import org.junit.Ignore;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.hamcrest.CoreMatchers.hasItem;
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.empty;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.hasSize;
+import static org.hamcrest.Matchers.is;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertSame;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Tests for the {@link DeclarativeSlotManager}.
+ */
+public class DeclarativeSlotManagerTest extends TestLogger {
+
+	private static final FlinkException TEST_EXCEPTION = new FlinkException("Test exception");
+
+	private static final WorkerResourceSpec WORKER_RESOURCE_SPEC = new WorkerResourceSpec.Builder()
+		.setCpuCores(100.0)
+		.setTaskHeapMemoryMB(10000)
+		.setTaskOffHeapMemoryMB(10000)
+		.setNetworkMemoryMB(10000)
+		.setManagedMemoryMB(10000)
+		.build();
+
+	/**
+	 * Tests that we can register task manager and their slots at the slot manager.
+	 */
+	@Test
+	public void testTaskManagerRegistration() throws Exception {
+		final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder().createTestingTaskExecutorGateway();
+		final ResourceID resourceId = ResourceID.generate();
+		final TaskExecutorConnection taskManagerConnection = new TaskExecutorConnection(resourceId, taskExecutorGateway);
+
+		final SlotID slotId1 = new SlotID(resourceId, 0);
+		final SlotID slotId2 = new SlotID(resourceId, 1);
+		final SlotReport slotReport = new SlotReport(Arrays.asList(createFreeSlotStatus(slotId1), createFreeSlotStatus(slotId2)));
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport);
+
+			assertThat("The number registered slots does not equal the expected number.", slotManager.getNumberRegisteredSlots(), is(2));
+
+			assertNotNull(slotTracker.getSlot(slotId1));
+			assertNotNull(slotTracker.getSlot(slotId2));
+		}
+	}
+
+	/**
+	 * Tests that un-registration of task managers will free and remove all registered slots.
+	 */
+	@Test
+	public void testTaskManagerUnregistration() throws Exception {
+		final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder()
+			.setRequestSlotFunction(tuple6 -> new CompletableFuture<>())
+			.createTestingTaskExecutorGateway();
+		final TaskExecutorConnection taskManagerConnection = createTaskExecutorConnection(taskExecutorGateway);
+		final ResourceID resourceId = taskManagerConnection.getResourceID();
+
+		final SlotID slotId1 = new SlotID(resourceId, 0);
+		final SlotID slotId2 = new SlotID(resourceId, 1);
+		final SlotReport slotReport = new SlotReport(Arrays.asList(createAllocatedSlotStatus(slotId1), createFreeSlotStatus(slotId2)));
+
+		final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot();
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport);
+
+			assertEquals("The number registered slots does not equal the expected number.", 2, slotManager.getNumberRegisteredSlots());
+
+			DeclarativeTaskManagerSlot slot1 = slotTracker.getSlot(slotId1);
+			DeclarativeTaskManagerSlot slot2 = slotTracker.getSlot(slotId2);
+
+			assertSame(SlotState.ALLOCATED, slot1.getState());
+			assertSame(SlotState.FREE, slot2.getState());

Review comment:
       Agreed.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] zentol commented on a change in pull request #13553: [FLINK-19306][coordination] Add DeclarativeSlotManager

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #13553:
URL: https://github.com/apache/flink/pull/13553#discussion_r502332396



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManager.java
##########
@@ -0,0 +1,609 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.metrics.MetricNames;
+import org.apache.flink.runtime.metrics.groups.SlotManagerMetricGroup;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerId;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.slots.ResourceCounter;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+import org.apache.flink.runtime.slots.ResourceRequirements;
+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.SlotOccupiedException;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ScheduledFuture;
+import java.util.function.BiFunction;
+import java.util.stream.Collectors;
+
+/**
+ * Implementation of {@link SlotManager} supporting declarative slot management.
+ */
+public class DeclarativeSlotManager implements SlotManager {
+	private static final Logger LOG = LoggerFactory.getLogger(DeclarativeSlotManager.class);
+
+	private final SlotTracker slotTracker;
+	private final ResourceTracker resourceTracker;
+	private final BiFunction<Executor, ResourceActions, TaskExecutorManager> taskExecutorManagerFactory;
+	private TaskExecutorManager taskExecutorManager;
+
+	/** Timeout for slot requests to the task manager. */
+	private final Time taskManagerRequestTimeout;
+	private ScheduledFuture<?> slotRequestTimeoutCheck;

Review comment:
       There should be no need for it.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13553: [FLINK-19306][coordination] Add DeclarativeSlotManager

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13553:
URL: https://github.com/apache/flink/pull/13553#issuecomment-704936985


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "15d4c353c4c0a646b2bdf34bbedbd81359cb5b5f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7264",
       "triggerID" : "15d4c353c4c0a646b2bdf34bbedbd81359cb5b5f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 15d4c353c4c0a646b2bdf34bbedbd81359cb5b5f Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7264) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] zentol commented on pull request #13553: [FLINK-19306][coordination] Add DeclarativeSlotManager

Posted by GitBox <gi...@apache.org>.
zentol commented on pull request #13553:
URL: https://github.com/apache/flink/pull/13553#issuecomment-705384887


   @flinkbot run azure


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13553: [FLINK-19306][coordination] Add DeclarativeSlotManager

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13553:
URL: https://github.com/apache/flink/pull/13553#issuecomment-704936985


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "15d4c353c4c0a646b2bdf34bbedbd81359cb5b5f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7264",
       "triggerID" : "15d4c353c4c0a646b2bdf34bbedbd81359cb5b5f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15d4c353c4c0a646b2bdf34bbedbd81359cb5b5f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7283",
       "triggerID" : "705384887",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "15d4c353c4c0a646b2bdf34bbedbd81359cb5b5f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7264",
       "triggerID" : "705384887",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "484cbbdfcc7e371ca51de4d88bab086a299795a6",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7514",
       "triggerID" : "484cbbdfcc7e371ca51de4d88bab086a299795a6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7a4c350c9514dc81429de2d9887837045b5894e0",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7527",
       "triggerID" : "7a4c350c9514dc81429de2d9887837045b5894e0",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 15d4c353c4c0a646b2bdf34bbedbd81359cb5b5f Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7283) Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7264) 
   * 484cbbdfcc7e371ca51de4d88bab086a299795a6 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7514) 
   * 7a4c350c9514dc81429de2d9887837045b5894e0 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7527) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] zentol commented on a change in pull request #13553: [FLINK-19306][coordination] Add DeclarativeSlotManager

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #13553:
URL: https://github.com/apache/flink/pull/13553#discussion_r503342632



##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManagerTest.java
##########
@@ -0,0 +1,1092 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple6;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.concurrent.ManuallyTriggeredScheduledExecutor;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerId;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+import org.apache.flink.runtime.slots.ResourceRequirements;
+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.TestingTaskExecutorGateway;
+import org.apache.flink.runtime.taskexecutor.TestingTaskExecutorGatewayBuilder;
+import org.apache.flink.runtime.taskexecutor.exceptions.SlotAllocationException;
+import org.apache.flink.runtime.taskexecutor.exceptions.SlotOccupiedException;
+import org.apache.flink.runtime.testingUtils.TestingUtils;
+import org.apache.flink.util.FlinkException;
+import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.function.FunctionUtils;
+
+import akka.pattern.AskTimeoutException;
+import org.junit.Ignore;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.hamcrest.CoreMatchers.hasItem;
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.empty;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.hasSize;
+import static org.hamcrest.Matchers.is;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertSame;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Tests for the {@link DeclarativeSlotManager}.
+ */
+public class DeclarativeSlotManagerTest extends TestLogger {
+
+	private static final FlinkException TEST_EXCEPTION = new FlinkException("Test exception");
+
+	private static final WorkerResourceSpec WORKER_RESOURCE_SPEC = new WorkerResourceSpec.Builder()
+		.setCpuCores(100.0)
+		.setTaskHeapMemoryMB(10000)
+		.setTaskOffHeapMemoryMB(10000)
+		.setNetworkMemoryMB(10000)
+		.setManagedMemoryMB(10000)
+		.build();
+
+	/**
+	 * Tests that we can register task manager and their slots at the slot manager.
+	 */
+	@Test
+	public void testTaskManagerRegistration() throws Exception {
+		final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder().createTestingTaskExecutorGateway();
+		final ResourceID resourceId = ResourceID.generate();
+		final TaskExecutorConnection taskManagerConnection = new TaskExecutorConnection(resourceId, taskExecutorGateway);
+
+		final SlotID slotId1 = new SlotID(resourceId, 0);
+		final SlotID slotId2 = new SlotID(resourceId, 1);
+		final SlotReport slotReport = new SlotReport(Arrays.asList(createFreeSlotStatus(slotId1), createFreeSlotStatus(slotId2)));
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport);
+
+			assertThat("The number registered slots does not equal the expected number.", slotManager.getNumberRegisteredSlots(), is(2));
+
+			assertNotNull(slotTracker.getSlot(slotId1));
+			assertNotNull(slotTracker.getSlot(slotId2));
+		}
+	}
+
+	/**
+	 * Tests that un-registration of task managers will free and remove all registered slots.
+	 */
+	@Test
+	public void testTaskManagerUnregistration() throws Exception {
+		final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder()
+			.setRequestSlotFunction(tuple6 -> new CompletableFuture<>())
+			.createTestingTaskExecutorGateway();
+		final TaskExecutorConnection taskManagerConnection = createTaskExecutorConnection(taskExecutorGateway);
+		final ResourceID resourceId = taskManagerConnection.getResourceID();
+
+		final SlotID slotId1 = new SlotID(resourceId, 0);
+		final SlotID slotId2 = new SlotID(resourceId, 1);
+		final SlotReport slotReport = new SlotReport(Arrays.asList(createAllocatedSlotStatus(slotId1), createFreeSlotStatus(slotId2)));
+
+		final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot();
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport);
+
+			assertEquals("The number registered slots does not equal the expected number.", 2, slotManager.getNumberRegisteredSlots());
+
+			DeclarativeTaskManagerSlot slot1 = slotTracker.getSlot(slotId1);
+			DeclarativeTaskManagerSlot slot2 = slotTracker.getSlot(slotId2);
+
+			assertSame(SlotState.ALLOCATED, slot1.getState());
+			assertSame(SlotState.FREE, slot2.getState());
+
+			slotManager.processResourceRequirements(resourceRequirements);
+
+			assertSame(SlotState.PENDING, slot2.getState());
+
+			slotManager.unregisterTaskManager(taskManagerConnection.getInstanceID(), TEST_EXCEPTION);
+
+			assertEquals(0, slotManager.getNumberRegisteredSlots());
+		}
+	}
+
+	/**
+	 * Tests that a slot request with no free slots will trigger the resource allocation.
+	 */
+	@Test
+	public void testSlotRequestWithoutFreeSlots() throws Exception {
+		final ResourceManagerId resourceManagerId = ResourceManagerId.generate();
+
+		final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot();
+
+		CompletableFuture<WorkerResourceSpec> allocateResourceFuture = new CompletableFuture<>();
+		ResourceActions resourceManagerActions = new TestingResourceActionsBuilder()
+			.setAllocateResourceConsumer(allocateResourceFuture::complete)
+			.build();
+
+		try (SlotManager slotManager = createSlotManager(resourceManagerId, resourceManagerActions)) {
+
+			slotManager.processResourceRequirements(resourceRequirements);
+
+			allocateResourceFuture.get();
+		}
+	}
+
+	/**
+	 * Tests that resources continue to be considered missing if we cannot allocate more resources.
+	 */
+	@Test
+	public void testResourceDeclarationWithResourceAllocationFailure() throws Exception {
+		final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot();
+
+		ResourceActions resourceManagerActions = new TestingResourceActionsBuilder()
+			.setAllocateResourceFunction(value -> false)
+			.build();
+
+		final ResourceTracker resourceTracker = new DefaultResourceTracker();
+
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setResourceTracker(resourceTracker)
+			.buildAndStartWithDirectExec(ResourceManagerId.generate(), resourceManagerActions)) {
+
+			slotManager.processResourceRequirements(resourceRequirements);
+
+			final JobID jobId = resourceRequirements.getJobId();
+			assertThat(getTotalResourceCount(resourceTracker.getMissingResources().get(jobId)), is(1));
+		}
+	}
+
+	/**
+	 * Tests that a slot request which can be fulfilled will trigger a slot allocation.
+	 */
+	@Test
+	public void testSlotRequestWithFreeSlot() throws Exception {
+		final ResourceManagerId resourceManagerId = ResourceManagerId.generate();
+		final ResourceID resourceID = ResourceID.generate();
+		final JobID jobId = new JobID();
+		final SlotID slotId = new SlotID(resourceID, 0);
+		final String targetAddress = "localhost";
+		final ResourceProfile resourceProfile = ResourceProfile.fromResources(42.0, 1337);
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec(resourceManagerId, new TestingResourceActionsBuilder().build())) {
+
+			final CompletableFuture<Tuple6<SlotID, JobID, AllocationID, ResourceProfile, String, ResourceManagerId>> requestFuture = new CompletableFuture<>();
+			// accept an incoming slot request
+			final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder()
+				.setRequestSlotFunction(tuple6 -> {
+					requestFuture.complete(Tuple6.of(tuple6.f0, tuple6.f1, tuple6.f2, tuple6.f3, tuple6.f4, tuple6.f5));
+					return CompletableFuture.completedFuture(Acknowledge.get());
+				})
+				.createTestingTaskExecutorGateway();
+
+			final TaskExecutorConnection taskExecutorConnection = new TaskExecutorConnection(resourceID, taskExecutorGateway);
+
+			final SlotStatus slotStatus = new SlotStatus(slotId, resourceProfile);
+			final SlotReport slotReport = new SlotReport(slotStatus);
+
+			slotManager.registerTaskManager(
+				taskExecutorConnection,
+				slotReport);
+
+			ResourceRequirements requirements = ResourceRequirements.create(
+				jobId,
+				targetAddress,
+				Collections.singleton(ResourceRequirement.create(resourceProfile, 1)));
+			slotManager.processResourceRequirements(requirements);
+
+			assertThat(requestFuture.get(), is(equalTo(Tuple6.of(slotId, jobId, requestFuture.get().f2, resourceProfile, targetAddress, resourceManagerId))));
+
+			DeclarativeTaskManagerSlot slot = slotTracker.getSlot(slotId);
+
+			assertEquals("The slot has not been allocated to the expected allocation id.", jobId, slot.getJobId());
+		}
+	}
+
+	/**
+	 * Checks that reducing resource requirements while a slot allocation is in-progress results in the slot being freed
+	 * one the allocation completes.
+	 */
+	@Test
+	@Ignore
+	public void testResourceRequirementReductionDuringAllocation() throws Exception {
+		final TaskExecutorConnection taskManagerConnection = createTaskExecutorConnection();
+		final ResourceID resourceID = taskManagerConnection.getResourceID();
+		final SlotID slotId = new SlotID(resourceID, 0);
+
+		final SlotReport slotReport = new SlotReport(createFreeSlotStatus(slotId));
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport);
+
+			DeclarativeTaskManagerSlot slot = slotTracker.getSlot(slotId);
+
+			final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot();
+			slotManager.processResourceRequirements(resourceRequirements);
+
+			assertSame(SlotState.PENDING, slot.getState());
+
+			slotManager.processResourceRequirements(ResourceRequirements.create(resourceRequirements.getJobId(), "foobar", Collections.emptyList()));
+
+			slot = slotTracker.getSlot(slotId);
+
+			assertSame(SlotState.FREE, slot.getState());
+		}
+	}
+
+	/**
+	 * Tests that pending slot requests are tried to be fulfilled upon new slot registrations.
+	 */
+	@Test
+	public void testFulfillingPendingSlotRequest() throws Exception {
+		final ResourceManagerId resourceManagerId = ResourceManagerId.generate();
+		final ResourceID resourceID = ResourceID.generate();
+		final JobID jobId = new JobID();
+		final SlotID slotId = new SlotID(resourceID, 0);
+		final String targetAddress = "localhost";
+		final ResourceProfile resourceProfile = ResourceProfile.fromResources(42.0, 1337);
+
+		final AtomicInteger numberAllocateResourceCalls = new AtomicInteger(0);
+		ResourceActions resourceManagerActions = new TestingResourceActionsBuilder()
+			.setAllocateResourceConsumer(ignored -> numberAllocateResourceCalls.incrementAndGet())
+			.build();
+
+		final CompletableFuture<Tuple6<SlotID, JobID, AllocationID, ResourceProfile, String, ResourceManagerId>> requestFuture = new CompletableFuture<>();
+		// accept an incoming slot request
+		final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder()
+			.setRequestSlotFunction(tuple6 -> {
+				requestFuture.complete(Tuple6.of(tuple6.f0, tuple6.f1, tuple6.f2, tuple6.f3, tuple6.f4, tuple6.f5));
+				return CompletableFuture.completedFuture(Acknowledge.get());
+			})
+			.createTestingTaskExecutorGateway();
+
+		final TaskExecutorConnection taskExecutorConnection = new TaskExecutorConnection(resourceID, taskExecutorGateway);
+
+		final SlotStatus slotStatus = new SlotStatus(slotId, resourceProfile);
+		final SlotReport slotReport = new SlotReport(slotStatus);
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec(resourceManagerId, resourceManagerActions)) {
+
+			final ResourceRequirements resourceRequirements = ResourceRequirements.create(
+				jobId,
+				targetAddress,
+				Collections.singleton(ResourceRequirement.create(resourceProfile, 1)));
+			slotManager.processResourceRequirements(resourceRequirements);
+
+			assertThat(numberAllocateResourceCalls.get(), is(1));
+
+			slotManager.registerTaskManager(
+				taskExecutorConnection,
+				slotReport);
+
+			assertThat(requestFuture.get(), is(equalTo(Tuple6.of(slotId, jobId, requestFuture.get().f2, resourceProfile, targetAddress, resourceManagerId))));
+
+			DeclarativeTaskManagerSlot slot = slotTracker.getSlot(slotId);
+
+			assertEquals("The slot has not been allocated to the expected job id.", jobId, slot.getJobId());
+		}
+	}
+
+	/**
+	 * Tests that freeing a slot will correctly reset the slot and mark it as a free slot.
+	 */
+	@Test
+	public void testFreeSlot() throws Exception {
+		final TaskExecutorConnection taskExecutorConnection = createTaskExecutorConnection();
+		final ResourceID resourceID = taskExecutorConnection.getResourceID();
+		final SlotID slotId = new SlotID(resourceID, 0);
+
+		final SlotReport slotReport = new SlotReport(createAllocatedSlotStatus(slotId));
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			slotManager.registerTaskManager(
+				taskExecutorConnection,
+				slotReport);
+
+			DeclarativeTaskManagerSlot slot = slotTracker.getSlot(slotId);
+
+			assertSame(SlotState.ALLOCATED, slot.getState());
+
+			slotManager.freeSlot(slotId, new AllocationID());
+
+			assertSame(SlotState.FREE, slot.getState());
+
+			assertEquals(1, slotManager.getNumberFreeSlots());
+		}
+	}
+
+	/**
+	 * Tests that duplicate resource requirement declaration do not result in additional slots being allocated after a
+	 * pending slot request has been fulfilled but not yet freed.
+	 */
+	@Test
+	public void testDuplicateResourceRequirementDeclarationAfterSuccessfulAllocation() throws Exception {
+		final ResourceManagerId resourceManagerId = ResourceManagerId.generate();
+		final AtomicInteger allocateResourceCalls = new AtomicInteger(0);
+		final ResourceActions resourceManagerActions = new TestingResourceActionsBuilder()
+			.setAllocateResourceConsumer(ignored -> allocateResourceCalls.incrementAndGet())
+			.build();
+		ResourceRequirements requirements = createResourceRequirementsForSingleSlot();
+
+		final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder().createTestingTaskExecutorGateway();
+
+		final ResourceID resourceID = ResourceID.generate();
+
+		final TaskExecutorConnection taskManagerConnection = new TaskExecutorConnection(resourceID, taskExecutorGateway);
+
+		final SlotID slotId = new SlotID(resourceID, 0);
+		final SlotReport slotReport = new SlotReport(createFreeSlotStatus(slotId));
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec(resourceManagerId, resourceManagerActions)) {
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport);
+
+			slotManager.processResourceRequirements(requirements);
+
+			DeclarativeTaskManagerSlot slot = slotTracker.getSlot(slotId);
+
+			assertThat(slot.getState(), is(SlotState.ALLOCATED));
+
+			slotManager.processResourceRequirements(requirements);
+		}
+
+		// check that we have only called the resource allocation only for the first slot request,
+		// since the second request is a duplicate
+		assertThat(allocateResourceCalls.get(), is(0));
+	}
+
+	/**
+	 * Tests that a slot allocated for one job can be allocated for another job after being freed.
+	 */
+	@Test
+	public void testSlotCanBeAllocatedForDifferentJobAfterFree() throws Exception {
+		final AllocationID allocationId = new AllocationID();
+		final ResourceRequirements resourceRequirements1 = createResourceRequirementsForSingleSlot();
+		final ResourceRequirements resourceRequirements2 = createResourceRequirementsForSingleSlot();
+
+		final TaskExecutorConnection taskManagerConnection = createTaskExecutorConnection();
+		final ResourceID resourceID = taskManagerConnection.getResourceID();
+
+		final SlotID slotId = new SlotID(resourceID, 0);
+		final SlotReport slotReport = new SlotReport(createFreeSlotStatus(slotId));
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport);
+
+			slotManager.processResourceRequirements(resourceRequirements1);
+
+			DeclarativeTaskManagerSlot slot = slotTracker.getSlot(slotId);
+
+			assertEquals("The slot has not been allocated to the expected job id.", resourceRequirements1.getJobId(), slot.getJobId());
+
+			// clear resource requirements, so that the slot isn't immediately re-assigned
+			slotManager.processResourceRequirements(ResourceRequirements.create(resourceRequirements1.getJobId(), resourceRequirements1.getTargetAddress(), Collections.emptyList()));
+			slotManager.freeSlot(slotId, allocationId);
+
+			// check that the slot has been freed
+			assertSame(SlotState.FREE, slot.getState());
+
+			slotManager.processResourceRequirements(resourceRequirements2);
+
+			assertEquals("The slot has not been allocated to the expected job id.", resourceRequirements2.getJobId(), slot.getJobId());
+		}
+	}
+
+	/**
+	 * Tests that the slot manager ignores slot reports of unknown origin (not registered
+	 * task managers).
+	 */
+	@Test
+	public void testReceivingUnknownSlotReport() throws Exception {
+		final ResourceManagerId resourceManagerId = ResourceManagerId.generate();
+		final ResourceActions resourceManagerActions = new TestingResourceActionsBuilder().build();
+
+		final InstanceID unknownInstanceID = new InstanceID();
+		final SlotID unknownSlotId = new SlotID(ResourceID.generate(), 0);
+		final SlotReport unknownSlotReport = new SlotReport(createFreeSlotStatus(unknownSlotId));
+
+		try (SlotManager slotManager = createSlotManager(resourceManagerId, resourceManagerActions)) {
+			// check that we don't have any slots registered
+			assertThat(slotManager.getNumberRegisteredSlots(), is(0));
+
+			// this should not update anything since the instance id is not known to the slot manager
+			assertFalse(slotManager.reportSlotStatus(unknownInstanceID, unknownSlotReport));
+
+			assertThat(slotManager.getNumberRegisteredSlots(), is(0));
+		}
+	}
+
+	/**
+	 * Tests that slots are updated with respect to the latest incoming slot report. This means that
+	 * slots for which a report was received are updated accordingly.
+	 */
+	@Test
+	public void testUpdateSlotReport() throws Exception {
+		final TaskExecutorConnection taskManagerConnection = createTaskExecutorConnection();
+		final ResourceID resourceId = taskManagerConnection.getResourceID();
+
+		final SlotID slotId1 = new SlotID(resourceId, 0);
+		final SlotID slotId2 = new SlotID(resourceId, 1);
+
+		final SlotStatus slotStatus1 = createFreeSlotStatus(slotId1);
+		final SlotStatus slotStatus2 = createFreeSlotStatus(slotId2);
+
+		final SlotStatus newSlotStatus2 = createAllocatedSlotStatus(slotId2);
+		final JobID jobId = newSlotStatus2.getJobID();
+
+		final SlotReport slotReport1 = new SlotReport(Arrays.asList(slotStatus1, slotStatus2));
+		final SlotReport slotReport2 = new SlotReport(Arrays.asList(newSlotStatus2, slotStatus1));
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			// check that we don't have any slots registered
+			assertEquals(0, slotManager.getNumberRegisteredSlots());
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport1);
+
+			DeclarativeTaskManagerSlot slot1 = slotTracker.getSlot(slotId1);
+			DeclarativeTaskManagerSlot slot2 = slotTracker.getSlot(slotId2);
+
+			assertEquals(2, slotManager.getNumberRegisteredSlots());
+
+			assertSame(SlotState.FREE, slot1.getState());
+			assertSame(SlotState.FREE, slot2.getState());
+
+			assertTrue(slotManager.reportSlotStatus(taskManagerConnection.getInstanceID(), slotReport2));
+
+			assertEquals(2, slotManager.getNumberRegisteredSlots());
+
+			assertNotNull(slotTracker.getSlot(slotId1));
+			assertNotNull(slotTracker.getSlot(slotId2));
+
+			// slotId2 should have been allocated for jobiD
+			assertEquals(jobId, slotTracker.getSlot(slotId2).getJobId());
+		}
+	}
+
+	/**
+	 * Tests that if a slot allocation times out we try to allocate another slot.
+	 */
+	@Test
+	public void testSlotAllocationTimeout() throws Exception {
+		final CompletableFuture<Void> secondSlotRequestFuture = new CompletableFuture<>();
+		final AtomicInteger slotRequestsCount = new AtomicInteger();
+		final TaskExecutorConnection taskManagerConnection = createTaskExecutorConnection(new TestingTaskExecutorGatewayBuilder()
+			.setRequestSlotFunction(ignored -> {
+				if (slotRequestsCount.getAndAdd(1) == 1) {
+					secondSlotRequestFuture.complete(null);
+				} else {
+					// mimic RPC timeout
+					return FutureUtils.completedExceptionally(new AskTimeoutException("timeout"));
+				}
+				return new CompletableFuture<>();
+			})
+			.createTestingTaskExecutorGateway());
+		final SlotReport slotReport = createSlotReport(taskManagerConnection.getResourceID(), 2);
+
+		final Executor mainThreadExecutor = TestingUtils.defaultExecutor();
+
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.build()) {
+
+			slotManager.start(ResourceManagerId.generate(), mainThreadExecutor, new TestingResourceActionsBuilder().build());
+
+			CompletableFuture.runAsync(() -> slotManager.registerTaskManager(taskManagerConnection, slotReport), mainThreadExecutor)
+				.thenRun(() -> slotManager.processResourceRequirements(createResourceRequirementsForSingleSlot()))
+				.get(5, TimeUnit.SECONDS);
+
+			// a second request is only sent if the first request timed out
+			secondSlotRequestFuture.get();
+		}
+	}
+
+	/**
+	 * Tests that a slot allocation is retried if it times out on the task manager side.
+	 */
+	@Test
+	public void testTaskManagerSlotAllocationTimeoutHandling() throws Exception {
+		final JobID jobId = new JobID();
+		final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot(jobId);
+		final CompletableFuture<Acknowledge> slotRequestFuture1 = new CompletableFuture<>();
+		final CompletableFuture<Acknowledge> slotRequestFuture2 = new CompletableFuture<>();
+		final Iterator<CompletableFuture<Acknowledge>> slotRequestFutureIterator = Arrays.asList(slotRequestFuture1, slotRequestFuture2).iterator();
+		final ArrayBlockingQueue<SlotID> slotIds = new ArrayBlockingQueue<>(2);
+
+		final TestingTaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder()
+			.setRequestSlotFunction(FunctionUtils.uncheckedFunction(
+				requestSlotParameters -> {
+					slotIds.put(requestSlotParameters.f0);
+					return slotRequestFutureIterator.next();
+				}))
+			.createTestingTaskExecutorGateway();
+
+		final ResourceID resourceId = ResourceID.generate();
+		final TaskExecutorConnection taskManagerConnection = new TaskExecutorConnection(resourceId, taskExecutorGateway);
+
+		final SlotID slotId1 = new SlotID(resourceId, 0);
+		final SlotID slotId2 = new SlotID(resourceId, 1);
+		final SlotReport slotReport = new SlotReport(Arrays.asList(createFreeSlotStatus(slotId1), createFreeSlotStatus(slotId2)));
+
+		final ResourceTracker resourceTracker = new DefaultResourceTracker();
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setResourceTracker(resourceTracker)
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport);
+
+			slotManager.processResourceRequirements(resourceRequirements);
+
+			final SlotID firstSlotId = slotIds.take();
+			assertThat(slotIds, is(empty()));
+
+			DeclarativeTaskManagerSlot failedSlot = slotTracker.getSlot(firstSlotId);
+
+			// let the first attempt fail --> this should trigger a second attempt
+			slotRequestFuture1.completeExceptionally(new SlotAllocationException("Test exception."));
+
+			assertThat(getTotalResourceCount(resourceTracker.getAcquiredResources(jobId)), is(1));
+
+			// the second attempt succeeds
+			slotRequestFuture2.complete(Acknowledge.get());
+
+			final SlotID secondSlotId = slotIds.take();
+			assertThat(slotIds, is(empty()));
+
+			DeclarativeTaskManagerSlot slot = slotTracker.getSlot(secondSlotId);
+
+			assertThat(slot.getState(), is(SlotState.ALLOCATED));
+			assertEquals(jobId, slot.getJobId());
+
+			if (!failedSlot.getSlotId().equals(slot.getSlotId())) {
+				assertThat(failedSlot.getState(), is(SlotState.FREE));
+			}

Review comment:
       yes; when the allocation fails we go through the same process as on the first attempt, and neither the `SlotTracker#getFreeSlots` nor `SlotMatchingStrategy#findMatchingSlot` give any guarantees on the order in which slots are returned/matched.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] tillrohrmann commented on a change in pull request #13553: [FLINK-19306][coordination] Add DeclarativeSlotManager

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #13553:
URL: https://github.com/apache/flink/pull/13553#discussion_r502542502



##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManagerTest.java
##########
@@ -0,0 +1,1092 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple6;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.concurrent.ManuallyTriggeredScheduledExecutor;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerId;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+import org.apache.flink.runtime.slots.ResourceRequirements;
+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.TestingTaskExecutorGateway;
+import org.apache.flink.runtime.taskexecutor.TestingTaskExecutorGatewayBuilder;
+import org.apache.flink.runtime.taskexecutor.exceptions.SlotAllocationException;
+import org.apache.flink.runtime.taskexecutor.exceptions.SlotOccupiedException;
+import org.apache.flink.runtime.testingUtils.TestingUtils;
+import org.apache.flink.util.FlinkException;
+import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.function.FunctionUtils;
+
+import akka.pattern.AskTimeoutException;
+import org.junit.Ignore;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.hamcrest.CoreMatchers.hasItem;
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.empty;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.hasSize;
+import static org.hamcrest.Matchers.is;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertSame;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Tests for the {@link DeclarativeSlotManager}.
+ */
+public class DeclarativeSlotManagerTest extends TestLogger {
+
+	private static final FlinkException TEST_EXCEPTION = new FlinkException("Test exception");
+
+	private static final WorkerResourceSpec WORKER_RESOURCE_SPEC = new WorkerResourceSpec.Builder()
+		.setCpuCores(100.0)
+		.setTaskHeapMemoryMB(10000)
+		.setTaskOffHeapMemoryMB(10000)
+		.setNetworkMemoryMB(10000)
+		.setManagedMemoryMB(10000)
+		.build();
+
+	/**
+	 * Tests that we can register task manager and their slots at the slot manager.
+	 */
+	@Test
+	public void testTaskManagerRegistration() throws Exception {
+		final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder().createTestingTaskExecutorGateway();
+		final ResourceID resourceId = ResourceID.generate();
+		final TaskExecutorConnection taskManagerConnection = new TaskExecutorConnection(resourceId, taskExecutorGateway);
+
+		final SlotID slotId1 = new SlotID(resourceId, 0);
+		final SlotID slotId2 = new SlotID(resourceId, 1);
+		final SlotReport slotReport = new SlotReport(Arrays.asList(createFreeSlotStatus(slotId1), createFreeSlotStatus(slotId2)));
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport);
+
+			assertThat("The number registered slots does not equal the expected number.", slotManager.getNumberRegisteredSlots(), is(2));
+
+			assertNotNull(slotTracker.getSlot(slotId1));
+			assertNotNull(slotTracker.getSlot(slotId2));
+		}
+	}
+
+	/**
+	 * Tests that un-registration of task managers will free and remove all registered slots.
+	 */
+	@Test
+	public void testTaskManagerUnregistration() throws Exception {
+		final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder()
+			.setRequestSlotFunction(tuple6 -> new CompletableFuture<>())
+			.createTestingTaskExecutorGateway();
+		final TaskExecutorConnection taskManagerConnection = createTaskExecutorConnection(taskExecutorGateway);
+		final ResourceID resourceId = taskManagerConnection.getResourceID();
+
+		final SlotID slotId1 = new SlotID(resourceId, 0);
+		final SlotID slotId2 = new SlotID(resourceId, 1);
+		final SlotReport slotReport = new SlotReport(Arrays.asList(createAllocatedSlotStatus(slotId1), createFreeSlotStatus(slotId2)));
+
+		final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot();
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport);
+
+			assertEquals("The number registered slots does not equal the expected number.", 2, slotManager.getNumberRegisteredSlots());
+
+			DeclarativeTaskManagerSlot slot1 = slotTracker.getSlot(slotId1);
+			DeclarativeTaskManagerSlot slot2 = slotTracker.getSlot(slotId2);
+
+			assertSame(SlotState.ALLOCATED, slot1.getState());
+			assertSame(SlotState.FREE, slot2.getState());
+
+			slotManager.processResourceRequirements(resourceRequirements);
+
+			assertSame(SlotState.PENDING, slot2.getState());
+
+			slotManager.unregisterTaskManager(taskManagerConnection.getInstanceID(), TEST_EXCEPTION);
+
+			assertEquals(0, slotManager.getNumberRegisteredSlots());
+		}
+	}
+
+	/**
+	 * Tests that a slot request with no free slots will trigger the resource allocation.
+	 */
+	@Test
+	public void testSlotRequestWithoutFreeSlots() throws Exception {
+		final ResourceManagerId resourceManagerId = ResourceManagerId.generate();
+
+		final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot();
+
+		CompletableFuture<WorkerResourceSpec> allocateResourceFuture = new CompletableFuture<>();
+		ResourceActions resourceManagerActions = new TestingResourceActionsBuilder()
+			.setAllocateResourceConsumer(allocateResourceFuture::complete)
+			.build();
+
+		try (SlotManager slotManager = createSlotManager(resourceManagerId, resourceManagerActions)) {
+
+			slotManager.processResourceRequirements(resourceRequirements);
+
+			allocateResourceFuture.get();
+		}
+	}
+
+	/**
+	 * Tests that resources continue to be considered missing if we cannot allocate more resources.
+	 */
+	@Test
+	public void testResourceDeclarationWithResourceAllocationFailure() throws Exception {
+		final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot();
+
+		ResourceActions resourceManagerActions = new TestingResourceActionsBuilder()
+			.setAllocateResourceFunction(value -> false)
+			.build();
+
+		final ResourceTracker resourceTracker = new DefaultResourceTracker();
+
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setResourceTracker(resourceTracker)
+			.buildAndStartWithDirectExec(ResourceManagerId.generate(), resourceManagerActions)) {
+
+			slotManager.processResourceRequirements(resourceRequirements);
+
+			final JobID jobId = resourceRequirements.getJobId();
+			assertThat(getTotalResourceCount(resourceTracker.getMissingResources().get(jobId)), is(1));
+		}
+	}
+
+	/**
+	 * Tests that a slot request which can be fulfilled will trigger a slot allocation.
+	 */
+	@Test
+	public void testSlotRequestWithFreeSlot() throws Exception {
+		final ResourceManagerId resourceManagerId = ResourceManagerId.generate();
+		final ResourceID resourceID = ResourceID.generate();
+		final JobID jobId = new JobID();
+		final SlotID slotId = new SlotID(resourceID, 0);
+		final String targetAddress = "localhost";
+		final ResourceProfile resourceProfile = ResourceProfile.fromResources(42.0, 1337);
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec(resourceManagerId, new TestingResourceActionsBuilder().build())) {
+
+			final CompletableFuture<Tuple6<SlotID, JobID, AllocationID, ResourceProfile, String, ResourceManagerId>> requestFuture = new CompletableFuture<>();
+			// accept an incoming slot request
+			final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder()
+				.setRequestSlotFunction(tuple6 -> {
+					requestFuture.complete(Tuple6.of(tuple6.f0, tuple6.f1, tuple6.f2, tuple6.f3, tuple6.f4, tuple6.f5));
+					return CompletableFuture.completedFuture(Acknowledge.get());
+				})
+				.createTestingTaskExecutorGateway();
+
+			final TaskExecutorConnection taskExecutorConnection = new TaskExecutorConnection(resourceID, taskExecutorGateway);
+
+			final SlotStatus slotStatus = new SlotStatus(slotId, resourceProfile);
+			final SlotReport slotReport = new SlotReport(slotStatus);
+
+			slotManager.registerTaskManager(
+				taskExecutorConnection,
+				slotReport);
+
+			ResourceRequirements requirements = ResourceRequirements.create(
+				jobId,
+				targetAddress,
+				Collections.singleton(ResourceRequirement.create(resourceProfile, 1)));
+			slotManager.processResourceRequirements(requirements);
+
+			assertThat(requestFuture.get(), is(equalTo(Tuple6.of(slotId, jobId, requestFuture.get().f2, resourceProfile, targetAddress, resourceManagerId))));
+
+			DeclarativeTaskManagerSlot slot = slotTracker.getSlot(slotId);
+
+			assertEquals("The slot has not been allocated to the expected allocation id.", jobId, slot.getJobId());
+		}
+	}
+
+	/**
+	 * Checks that reducing resource requirements while a slot allocation is in-progress results in the slot being freed
+	 * one the allocation completes.
+	 */
+	@Test
+	@Ignore
+	public void testResourceRequirementReductionDuringAllocation() throws Exception {

Review comment:
       Is this really supported by the current `DeclarativeSlotManager` implementation?

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManagerTest.java
##########
@@ -0,0 +1,1092 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple6;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.concurrent.ManuallyTriggeredScheduledExecutor;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerId;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+import org.apache.flink.runtime.slots.ResourceRequirements;
+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.TestingTaskExecutorGateway;
+import org.apache.flink.runtime.taskexecutor.TestingTaskExecutorGatewayBuilder;
+import org.apache.flink.runtime.taskexecutor.exceptions.SlotAllocationException;
+import org.apache.flink.runtime.taskexecutor.exceptions.SlotOccupiedException;
+import org.apache.flink.runtime.testingUtils.TestingUtils;
+import org.apache.flink.util.FlinkException;
+import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.function.FunctionUtils;
+
+import akka.pattern.AskTimeoutException;
+import org.junit.Ignore;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.hamcrest.CoreMatchers.hasItem;
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.empty;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.hasSize;
+import static org.hamcrest.Matchers.is;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertSame;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Tests for the {@link DeclarativeSlotManager}.
+ */
+public class DeclarativeSlotManagerTest extends TestLogger {
+
+	private static final FlinkException TEST_EXCEPTION = new FlinkException("Test exception");
+
+	private static final WorkerResourceSpec WORKER_RESOURCE_SPEC = new WorkerResourceSpec.Builder()
+		.setCpuCores(100.0)
+		.setTaskHeapMemoryMB(10000)
+		.setTaskOffHeapMemoryMB(10000)
+		.setNetworkMemoryMB(10000)
+		.setManagedMemoryMB(10000)
+		.build();
+
+	/**
+	 * Tests that we can register task manager and their slots at the slot manager.
+	 */
+	@Test
+	public void testTaskManagerRegistration() throws Exception {
+		final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder().createTestingTaskExecutorGateway();
+		final ResourceID resourceId = ResourceID.generate();
+		final TaskExecutorConnection taskManagerConnection = new TaskExecutorConnection(resourceId, taskExecutorGateway);
+
+		final SlotID slotId1 = new SlotID(resourceId, 0);
+		final SlotID slotId2 = new SlotID(resourceId, 1);
+		final SlotReport slotReport = new SlotReport(Arrays.asList(createFreeSlotStatus(slotId1), createFreeSlotStatus(slotId2)));
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport);
+
+			assertThat("The number registered slots does not equal the expected number.", slotManager.getNumberRegisteredSlots(), is(2));
+
+			assertNotNull(slotTracker.getSlot(slotId1));
+			assertNotNull(slotTracker.getSlot(slotId2));
+		}
+	}
+
+	/**
+	 * Tests that un-registration of task managers will free and remove all registered slots.
+	 */
+	@Test
+	public void testTaskManagerUnregistration() throws Exception {
+		final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder()
+			.setRequestSlotFunction(tuple6 -> new CompletableFuture<>())
+			.createTestingTaskExecutorGateway();
+		final TaskExecutorConnection taskManagerConnection = createTaskExecutorConnection(taskExecutorGateway);
+		final ResourceID resourceId = taskManagerConnection.getResourceID();
+
+		final SlotID slotId1 = new SlotID(resourceId, 0);
+		final SlotID slotId2 = new SlotID(resourceId, 1);
+		final SlotReport slotReport = new SlotReport(Arrays.asList(createAllocatedSlotStatus(slotId1), createFreeSlotStatus(slotId2)));
+
+		final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot();
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport);
+
+			assertEquals("The number registered slots does not equal the expected number.", 2, slotManager.getNumberRegisteredSlots());
+
+			DeclarativeTaskManagerSlot slot1 = slotTracker.getSlot(slotId1);
+			DeclarativeTaskManagerSlot slot2 = slotTracker.getSlot(slotId2);
+
+			assertSame(SlotState.ALLOCATED, slot1.getState());
+			assertSame(SlotState.FREE, slot2.getState());
+
+			slotManager.processResourceRequirements(resourceRequirements);
+
+			assertSame(SlotState.PENDING, slot2.getState());
+
+			slotManager.unregisterTaskManager(taskManagerConnection.getInstanceID(), TEST_EXCEPTION);
+
+			assertEquals(0, slotManager.getNumberRegisteredSlots());
+		}
+	}
+
+	/**
+	 * Tests that a slot request with no free slots will trigger the resource allocation.
+	 */
+	@Test
+	public void testSlotRequestWithoutFreeSlots() throws Exception {
+		final ResourceManagerId resourceManagerId = ResourceManagerId.generate();
+
+		final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot();
+
+		CompletableFuture<WorkerResourceSpec> allocateResourceFuture = new CompletableFuture<>();
+		ResourceActions resourceManagerActions = new TestingResourceActionsBuilder()
+			.setAllocateResourceConsumer(allocateResourceFuture::complete)
+			.build();
+
+		try (SlotManager slotManager = createSlotManager(resourceManagerId, resourceManagerActions)) {
+
+			slotManager.processResourceRequirements(resourceRequirements);
+
+			allocateResourceFuture.get();
+		}
+	}
+
+	/**
+	 * Tests that resources continue to be considered missing if we cannot allocate more resources.
+	 */
+	@Test
+	public void testResourceDeclarationWithResourceAllocationFailure() throws Exception {
+		final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot();
+
+		ResourceActions resourceManagerActions = new TestingResourceActionsBuilder()
+			.setAllocateResourceFunction(value -> false)
+			.build();
+
+		final ResourceTracker resourceTracker = new DefaultResourceTracker();
+
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setResourceTracker(resourceTracker)
+			.buildAndStartWithDirectExec(ResourceManagerId.generate(), resourceManagerActions)) {
+
+			slotManager.processResourceRequirements(resourceRequirements);
+
+			final JobID jobId = resourceRequirements.getJobId();
+			assertThat(getTotalResourceCount(resourceTracker.getMissingResources().get(jobId)), is(1));
+		}
+	}
+
+	/**
+	 * Tests that a slot request which can be fulfilled will trigger a slot allocation.
+	 */
+	@Test
+	public void testSlotRequestWithFreeSlot() throws Exception {
+		final ResourceManagerId resourceManagerId = ResourceManagerId.generate();
+		final ResourceID resourceID = ResourceID.generate();
+		final JobID jobId = new JobID();
+		final SlotID slotId = new SlotID(resourceID, 0);
+		final String targetAddress = "localhost";
+		final ResourceProfile resourceProfile = ResourceProfile.fromResources(42.0, 1337);
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec(resourceManagerId, new TestingResourceActionsBuilder().build())) {
+
+			final CompletableFuture<Tuple6<SlotID, JobID, AllocationID, ResourceProfile, String, ResourceManagerId>> requestFuture = new CompletableFuture<>();
+			// accept an incoming slot request
+			final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder()
+				.setRequestSlotFunction(tuple6 -> {
+					requestFuture.complete(Tuple6.of(tuple6.f0, tuple6.f1, tuple6.f2, tuple6.f3, tuple6.f4, tuple6.f5));
+					return CompletableFuture.completedFuture(Acknowledge.get());
+				})
+				.createTestingTaskExecutorGateway();
+
+			final TaskExecutorConnection taskExecutorConnection = new TaskExecutorConnection(resourceID, taskExecutorGateway);
+
+			final SlotStatus slotStatus = new SlotStatus(slotId, resourceProfile);
+			final SlotReport slotReport = new SlotReport(slotStatus);
+
+			slotManager.registerTaskManager(
+				taskExecutorConnection,
+				slotReport);
+
+			ResourceRequirements requirements = ResourceRequirements.create(
+				jobId,
+				targetAddress,
+				Collections.singleton(ResourceRequirement.create(resourceProfile, 1)));
+			slotManager.processResourceRequirements(requirements);
+
+			assertThat(requestFuture.get(), is(equalTo(Tuple6.of(slotId, jobId, requestFuture.get().f2, resourceProfile, targetAddress, resourceManagerId))));
+
+			DeclarativeTaskManagerSlot slot = slotTracker.getSlot(slotId);
+
+			assertEquals("The slot has not been allocated to the expected allocation id.", jobId, slot.getJobId());
+		}
+	}
+
+	/**
+	 * Checks that reducing resource requirements while a slot allocation is in-progress results in the slot being freed
+	 * one the allocation completes.

Review comment:
       ```suggestion
   	 * once the allocation completes.
   ```

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManagerTest.java
##########
@@ -0,0 +1,1092 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple6;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.concurrent.ManuallyTriggeredScheduledExecutor;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerId;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+import org.apache.flink.runtime.slots.ResourceRequirements;
+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.TestingTaskExecutorGateway;
+import org.apache.flink.runtime.taskexecutor.TestingTaskExecutorGatewayBuilder;
+import org.apache.flink.runtime.taskexecutor.exceptions.SlotAllocationException;
+import org.apache.flink.runtime.taskexecutor.exceptions.SlotOccupiedException;
+import org.apache.flink.runtime.testingUtils.TestingUtils;
+import org.apache.flink.util.FlinkException;
+import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.function.FunctionUtils;
+
+import akka.pattern.AskTimeoutException;
+import org.junit.Ignore;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.hamcrest.CoreMatchers.hasItem;
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.empty;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.hasSize;
+import static org.hamcrest.Matchers.is;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertSame;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Tests for the {@link DeclarativeSlotManager}.
+ */
+public class DeclarativeSlotManagerTest extends TestLogger {
+
+	private static final FlinkException TEST_EXCEPTION = new FlinkException("Test exception");
+
+	private static final WorkerResourceSpec WORKER_RESOURCE_SPEC = new WorkerResourceSpec.Builder()
+		.setCpuCores(100.0)
+		.setTaskHeapMemoryMB(10000)
+		.setTaskOffHeapMemoryMB(10000)
+		.setNetworkMemoryMB(10000)
+		.setManagedMemoryMB(10000)
+		.build();
+
+	/**
+	 * Tests that we can register task manager and their slots at the slot manager.
+	 */
+	@Test
+	public void testTaskManagerRegistration() throws Exception {
+		final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder().createTestingTaskExecutorGateway();
+		final ResourceID resourceId = ResourceID.generate();
+		final TaskExecutorConnection taskManagerConnection = new TaskExecutorConnection(resourceId, taskExecutorGateway);
+
+		final SlotID slotId1 = new SlotID(resourceId, 0);
+		final SlotID slotId2 = new SlotID(resourceId, 1);
+		final SlotReport slotReport = new SlotReport(Arrays.asList(createFreeSlotStatus(slotId1), createFreeSlotStatus(slotId2)));
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport);
+
+			assertThat("The number registered slots does not equal the expected number.", slotManager.getNumberRegisteredSlots(), is(2));
+
+			assertNotNull(slotTracker.getSlot(slotId1));
+			assertNotNull(slotTracker.getSlot(slotId2));
+		}
+	}
+
+	/**
+	 * Tests that un-registration of task managers will free and remove all registered slots.
+	 */
+	@Test
+	public void testTaskManagerUnregistration() throws Exception {
+		final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder()
+			.setRequestSlotFunction(tuple6 -> new CompletableFuture<>())
+			.createTestingTaskExecutorGateway();
+		final TaskExecutorConnection taskManagerConnection = createTaskExecutorConnection(taskExecutorGateway);
+		final ResourceID resourceId = taskManagerConnection.getResourceID();
+
+		final SlotID slotId1 = new SlotID(resourceId, 0);
+		final SlotID slotId2 = new SlotID(resourceId, 1);
+		final SlotReport slotReport = new SlotReport(Arrays.asList(createAllocatedSlotStatus(slotId1), createFreeSlotStatus(slotId2)));
+
+		final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot();
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport);
+
+			assertEquals("The number registered slots does not equal the expected number.", 2, slotManager.getNumberRegisteredSlots());
+
+			DeclarativeTaskManagerSlot slot1 = slotTracker.getSlot(slotId1);
+			DeclarativeTaskManagerSlot slot2 = slotTracker.getSlot(slotId2);
+
+			assertSame(SlotState.ALLOCATED, slot1.getState());
+			assertSame(SlotState.FREE, slot2.getState());
+
+			slotManager.processResourceRequirements(resourceRequirements);
+
+			assertSame(SlotState.PENDING, slot2.getState());
+
+			slotManager.unregisterTaskManager(taskManagerConnection.getInstanceID(), TEST_EXCEPTION);
+
+			assertEquals(0, slotManager.getNumberRegisteredSlots());
+		}
+	}
+
+	/**
+	 * Tests that a slot request with no free slots will trigger the resource allocation.
+	 */
+	@Test
+	public void testSlotRequestWithoutFreeSlots() throws Exception {
+		final ResourceManagerId resourceManagerId = ResourceManagerId.generate();
+
+		final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot();
+
+		CompletableFuture<WorkerResourceSpec> allocateResourceFuture = new CompletableFuture<>();
+		ResourceActions resourceManagerActions = new TestingResourceActionsBuilder()
+			.setAllocateResourceConsumer(allocateResourceFuture::complete)
+			.build();
+
+		try (SlotManager slotManager = createSlotManager(resourceManagerId, resourceManagerActions)) {
+
+			slotManager.processResourceRequirements(resourceRequirements);
+
+			allocateResourceFuture.get();
+		}
+	}
+
+	/**
+	 * Tests that resources continue to be considered missing if we cannot allocate more resources.
+	 */
+	@Test
+	public void testResourceDeclarationWithResourceAllocationFailure() throws Exception {
+		final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot();
+
+		ResourceActions resourceManagerActions = new TestingResourceActionsBuilder()
+			.setAllocateResourceFunction(value -> false)
+			.build();
+
+		final ResourceTracker resourceTracker = new DefaultResourceTracker();
+
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setResourceTracker(resourceTracker)
+			.buildAndStartWithDirectExec(ResourceManagerId.generate(), resourceManagerActions)) {
+
+			slotManager.processResourceRequirements(resourceRequirements);
+
+			final JobID jobId = resourceRequirements.getJobId();
+			assertThat(getTotalResourceCount(resourceTracker.getMissingResources().get(jobId)), is(1));
+		}
+	}
+
+	/**
+	 * Tests that a slot request which can be fulfilled will trigger a slot allocation.
+	 */
+	@Test
+	public void testSlotRequestWithFreeSlot() throws Exception {
+		final ResourceManagerId resourceManagerId = ResourceManagerId.generate();
+		final ResourceID resourceID = ResourceID.generate();
+		final JobID jobId = new JobID();
+		final SlotID slotId = new SlotID(resourceID, 0);
+		final String targetAddress = "localhost";
+		final ResourceProfile resourceProfile = ResourceProfile.fromResources(42.0, 1337);
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec(resourceManagerId, new TestingResourceActionsBuilder().build())) {
+
+			final CompletableFuture<Tuple6<SlotID, JobID, AllocationID, ResourceProfile, String, ResourceManagerId>> requestFuture = new CompletableFuture<>();
+			// accept an incoming slot request
+			final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder()
+				.setRequestSlotFunction(tuple6 -> {
+					requestFuture.complete(Tuple6.of(tuple6.f0, tuple6.f1, tuple6.f2, tuple6.f3, tuple6.f4, tuple6.f5));
+					return CompletableFuture.completedFuture(Acknowledge.get());
+				})
+				.createTestingTaskExecutorGateway();
+
+			final TaskExecutorConnection taskExecutorConnection = new TaskExecutorConnection(resourceID, taskExecutorGateway);
+
+			final SlotStatus slotStatus = new SlotStatus(slotId, resourceProfile);
+			final SlotReport slotReport = new SlotReport(slotStatus);
+
+			slotManager.registerTaskManager(
+				taskExecutorConnection,
+				slotReport);
+
+			ResourceRequirements requirements = ResourceRequirements.create(
+				jobId,
+				targetAddress,
+				Collections.singleton(ResourceRequirement.create(resourceProfile, 1)));
+			slotManager.processResourceRequirements(requirements);
+
+			assertThat(requestFuture.get(), is(equalTo(Tuple6.of(slotId, jobId, requestFuture.get().f2, resourceProfile, targetAddress, resourceManagerId))));
+
+			DeclarativeTaskManagerSlot slot = slotTracker.getSlot(slotId);
+
+			assertEquals("The slot has not been allocated to the expected allocation id.", jobId, slot.getJobId());
+		}
+	}
+
+	/**
+	 * Checks that reducing resource requirements while a slot allocation is in-progress results in the slot being freed
+	 * one the allocation completes.
+	 */
+	@Test
+	@Ignore
+	public void testResourceRequirementReductionDuringAllocation() throws Exception {
+		final TaskExecutorConnection taskManagerConnection = createTaskExecutorConnection();
+		final ResourceID resourceID = taskManagerConnection.getResourceID();
+		final SlotID slotId = new SlotID(resourceID, 0);
+
+		final SlotReport slotReport = new SlotReport(createFreeSlotStatus(slotId));
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport);
+
+			DeclarativeTaskManagerSlot slot = slotTracker.getSlot(slotId);
+
+			final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot();
+			slotManager.processResourceRequirements(resourceRequirements);
+
+			assertSame(SlotState.PENDING, slot.getState());
+
+			slotManager.processResourceRequirements(ResourceRequirements.create(resourceRequirements.getJobId(), "foobar", Collections.emptyList()));
+
+			slot = slotTracker.getSlot(slotId);
+
+			assertSame(SlotState.FREE, slot.getState());
+		}
+	}
+
+	/**
+	 * Tests that pending slot requests are tried to be fulfilled upon new slot registrations.
+	 */
+	@Test
+	public void testFulfillingPendingSlotRequest() throws Exception {
+		final ResourceManagerId resourceManagerId = ResourceManagerId.generate();
+		final ResourceID resourceID = ResourceID.generate();
+		final JobID jobId = new JobID();
+		final SlotID slotId = new SlotID(resourceID, 0);
+		final String targetAddress = "localhost";
+		final ResourceProfile resourceProfile = ResourceProfile.fromResources(42.0, 1337);
+
+		final AtomicInteger numberAllocateResourceCalls = new AtomicInteger(0);
+		ResourceActions resourceManagerActions = new TestingResourceActionsBuilder()
+			.setAllocateResourceConsumer(ignored -> numberAllocateResourceCalls.incrementAndGet())
+			.build();
+
+		final CompletableFuture<Tuple6<SlotID, JobID, AllocationID, ResourceProfile, String, ResourceManagerId>> requestFuture = new CompletableFuture<>();
+		// accept an incoming slot request
+		final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder()
+			.setRequestSlotFunction(tuple6 -> {
+				requestFuture.complete(Tuple6.of(tuple6.f0, tuple6.f1, tuple6.f2, tuple6.f3, tuple6.f4, tuple6.f5));
+				return CompletableFuture.completedFuture(Acknowledge.get());
+			})
+			.createTestingTaskExecutorGateway();
+
+		final TaskExecutorConnection taskExecutorConnection = new TaskExecutorConnection(resourceID, taskExecutorGateway);
+
+		final SlotStatus slotStatus = new SlotStatus(slotId, resourceProfile);
+		final SlotReport slotReport = new SlotReport(slotStatus);
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec(resourceManagerId, resourceManagerActions)) {
+
+			final ResourceRequirements resourceRequirements = ResourceRequirements.create(
+				jobId,
+				targetAddress,
+				Collections.singleton(ResourceRequirement.create(resourceProfile, 1)));
+			slotManager.processResourceRequirements(resourceRequirements);
+
+			assertThat(numberAllocateResourceCalls.get(), is(1));
+
+			slotManager.registerTaskManager(
+				taskExecutorConnection,
+				slotReport);
+
+			assertThat(requestFuture.get(), is(equalTo(Tuple6.of(slotId, jobId, requestFuture.get().f2, resourceProfile, targetAddress, resourceManagerId))));
+
+			DeclarativeTaskManagerSlot slot = slotTracker.getSlot(slotId);
+
+			assertEquals("The slot has not been allocated to the expected job id.", jobId, slot.getJobId());
+		}
+	}
+
+	/**
+	 * Tests that freeing a slot will correctly reset the slot and mark it as a free slot.
+	 */
+	@Test
+	public void testFreeSlot() throws Exception {
+		final TaskExecutorConnection taskExecutorConnection = createTaskExecutorConnection();
+		final ResourceID resourceID = taskExecutorConnection.getResourceID();
+		final SlotID slotId = new SlotID(resourceID, 0);
+
+		final SlotReport slotReport = new SlotReport(createAllocatedSlotStatus(slotId));
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			slotManager.registerTaskManager(
+				taskExecutorConnection,
+				slotReport);
+
+			DeclarativeTaskManagerSlot slot = slotTracker.getSlot(slotId);
+
+			assertSame(SlotState.ALLOCATED, slot.getState());
+
+			slotManager.freeSlot(slotId, new AllocationID());
+
+			assertSame(SlotState.FREE, slot.getState());
+
+			assertEquals(1, slotManager.getNumberFreeSlots());
+		}
+	}
+
+	/**
+	 * Tests that duplicate resource requirement declaration do not result in additional slots being allocated after a
+	 * pending slot request has been fulfilled but not yet freed.
+	 */
+	@Test
+	public void testDuplicateResourceRequirementDeclarationAfterSuccessfulAllocation() throws Exception {
+		final ResourceManagerId resourceManagerId = ResourceManagerId.generate();
+		final AtomicInteger allocateResourceCalls = new AtomicInteger(0);
+		final ResourceActions resourceManagerActions = new TestingResourceActionsBuilder()
+			.setAllocateResourceConsumer(ignored -> allocateResourceCalls.incrementAndGet())
+			.build();
+		ResourceRequirements requirements = createResourceRequirementsForSingleSlot();
+
+		final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder().createTestingTaskExecutorGateway();
+
+		final ResourceID resourceID = ResourceID.generate();
+
+		final TaskExecutorConnection taskManagerConnection = new TaskExecutorConnection(resourceID, taskExecutorGateway);
+
+		final SlotID slotId = new SlotID(resourceID, 0);
+		final SlotReport slotReport = new SlotReport(createFreeSlotStatus(slotId));
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec(resourceManagerId, resourceManagerActions)) {
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport);
+
+			slotManager.processResourceRequirements(requirements);
+
+			DeclarativeTaskManagerSlot slot = slotTracker.getSlot(slotId);
+
+			assertThat(slot.getState(), is(SlotState.ALLOCATED));
+
+			slotManager.processResourceRequirements(requirements);
+		}
+
+		// check that we have only called the resource allocation only for the first slot request,
+		// since the second request is a duplicate
+		assertThat(allocateResourceCalls.get(), is(0));
+	}
+
+	/**
+	 * Tests that a slot allocated for one job can be allocated for another job after being freed.
+	 */
+	@Test
+	public void testSlotCanBeAllocatedForDifferentJobAfterFree() throws Exception {
+		final AllocationID allocationId = new AllocationID();
+		final ResourceRequirements resourceRequirements1 = createResourceRequirementsForSingleSlot();
+		final ResourceRequirements resourceRequirements2 = createResourceRequirementsForSingleSlot();
+
+		final TaskExecutorConnection taskManagerConnection = createTaskExecutorConnection();
+		final ResourceID resourceID = taskManagerConnection.getResourceID();
+
+		final SlotID slotId = new SlotID(resourceID, 0);
+		final SlotReport slotReport = new SlotReport(createFreeSlotStatus(slotId));
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport);
+
+			slotManager.processResourceRequirements(resourceRequirements1);
+
+			DeclarativeTaskManagerSlot slot = slotTracker.getSlot(slotId);
+
+			assertEquals("The slot has not been allocated to the expected job id.", resourceRequirements1.getJobId(), slot.getJobId());
+
+			// clear resource requirements, so that the slot isn't immediately re-assigned
+			slotManager.processResourceRequirements(ResourceRequirements.create(resourceRequirements1.getJobId(), resourceRequirements1.getTargetAddress(), Collections.emptyList()));
+			slotManager.freeSlot(slotId, allocationId);
+
+			// check that the slot has been freed
+			assertSame(SlotState.FREE, slot.getState());
+
+			slotManager.processResourceRequirements(resourceRequirements2);
+
+			assertEquals("The slot has not been allocated to the expected job id.", resourceRequirements2.getJobId(), slot.getJobId());
+		}
+	}

Review comment:
       Should we also test that the slo is being used to fulfill requirements of another job if the resource requirements of this job were already registered?

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManagerTest.java
##########
@@ -0,0 +1,1092 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple6;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.concurrent.ManuallyTriggeredScheduledExecutor;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerId;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+import org.apache.flink.runtime.slots.ResourceRequirements;
+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.TestingTaskExecutorGateway;
+import org.apache.flink.runtime.taskexecutor.TestingTaskExecutorGatewayBuilder;
+import org.apache.flink.runtime.taskexecutor.exceptions.SlotAllocationException;
+import org.apache.flink.runtime.taskexecutor.exceptions.SlotOccupiedException;
+import org.apache.flink.runtime.testingUtils.TestingUtils;
+import org.apache.flink.util.FlinkException;
+import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.function.FunctionUtils;
+
+import akka.pattern.AskTimeoutException;
+import org.junit.Ignore;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.hamcrest.CoreMatchers.hasItem;
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.empty;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.hasSize;
+import static org.hamcrest.Matchers.is;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertSame;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Tests for the {@link DeclarativeSlotManager}.
+ */
+public class DeclarativeSlotManagerTest extends TestLogger {
+
+	private static final FlinkException TEST_EXCEPTION = new FlinkException("Test exception");
+
+	private static final WorkerResourceSpec WORKER_RESOURCE_SPEC = new WorkerResourceSpec.Builder()
+		.setCpuCores(100.0)
+		.setTaskHeapMemoryMB(10000)
+		.setTaskOffHeapMemoryMB(10000)
+		.setNetworkMemoryMB(10000)
+		.setManagedMemoryMB(10000)
+		.build();
+
+	/**
+	 * Tests that we can register task manager and their slots at the slot manager.
+	 */
+	@Test
+	public void testTaskManagerRegistration() throws Exception {
+		final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder().createTestingTaskExecutorGateway();
+		final ResourceID resourceId = ResourceID.generate();
+		final TaskExecutorConnection taskManagerConnection = new TaskExecutorConnection(resourceId, taskExecutorGateway);
+
+		final SlotID slotId1 = new SlotID(resourceId, 0);
+		final SlotID slotId2 = new SlotID(resourceId, 1);
+		final SlotReport slotReport = new SlotReport(Arrays.asList(createFreeSlotStatus(slotId1), createFreeSlotStatus(slotId2)));
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport);
+
+			assertThat("The number registered slots does not equal the expected number.", slotManager.getNumberRegisteredSlots(), is(2));
+
+			assertNotNull(slotTracker.getSlot(slotId1));
+			assertNotNull(slotTracker.getSlot(slotId2));
+		}
+	}
+
+	/**
+	 * Tests that un-registration of task managers will free and remove all registered slots.
+	 */
+	@Test
+	public void testTaskManagerUnregistration() throws Exception {
+		final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder()
+			.setRequestSlotFunction(tuple6 -> new CompletableFuture<>())
+			.createTestingTaskExecutorGateway();
+		final TaskExecutorConnection taskManagerConnection = createTaskExecutorConnection(taskExecutorGateway);
+		final ResourceID resourceId = taskManagerConnection.getResourceID();
+
+		final SlotID slotId1 = new SlotID(resourceId, 0);
+		final SlotID slotId2 = new SlotID(resourceId, 1);
+		final SlotReport slotReport = new SlotReport(Arrays.asList(createAllocatedSlotStatus(slotId1), createFreeSlotStatus(slotId2)));
+
+		final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot();
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport);
+
+			assertEquals("The number registered slots does not equal the expected number.", 2, slotManager.getNumberRegisteredSlots());
+
+			DeclarativeTaskManagerSlot slot1 = slotTracker.getSlot(slotId1);
+			DeclarativeTaskManagerSlot slot2 = slotTracker.getSlot(slotId2);
+
+			assertSame(SlotState.ALLOCATED, slot1.getState());
+			assertSame(SlotState.FREE, slot2.getState());
+
+			slotManager.processResourceRequirements(resourceRequirements);
+
+			assertSame(SlotState.PENDING, slot2.getState());
+
+			slotManager.unregisterTaskManager(taskManagerConnection.getInstanceID(), TEST_EXCEPTION);
+
+			assertEquals(0, slotManager.getNumberRegisteredSlots());
+		}
+	}
+
+	/**
+	 * Tests that a slot request with no free slots will trigger the resource allocation.
+	 */
+	@Test
+	public void testSlotRequestWithoutFreeSlots() throws Exception {
+		final ResourceManagerId resourceManagerId = ResourceManagerId.generate();
+
+		final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot();
+
+		CompletableFuture<WorkerResourceSpec> allocateResourceFuture = new CompletableFuture<>();
+		ResourceActions resourceManagerActions = new TestingResourceActionsBuilder()
+			.setAllocateResourceConsumer(allocateResourceFuture::complete)
+			.build();
+
+		try (SlotManager slotManager = createSlotManager(resourceManagerId, resourceManagerActions)) {
+
+			slotManager.processResourceRequirements(resourceRequirements);
+
+			allocateResourceFuture.get();
+		}
+	}
+
+	/**
+	 * Tests that resources continue to be considered missing if we cannot allocate more resources.
+	 */
+	@Test
+	public void testResourceDeclarationWithResourceAllocationFailure() throws Exception {
+		final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot();
+
+		ResourceActions resourceManagerActions = new TestingResourceActionsBuilder()
+			.setAllocateResourceFunction(value -> false)
+			.build();
+
+		final ResourceTracker resourceTracker = new DefaultResourceTracker();
+
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setResourceTracker(resourceTracker)
+			.buildAndStartWithDirectExec(ResourceManagerId.generate(), resourceManagerActions)) {
+
+			slotManager.processResourceRequirements(resourceRequirements);
+
+			final JobID jobId = resourceRequirements.getJobId();
+			assertThat(getTotalResourceCount(resourceTracker.getMissingResources().get(jobId)), is(1));
+		}
+	}
+
+	/**
+	 * Tests that a slot request which can be fulfilled will trigger a slot allocation.
+	 */
+	@Test
+	public void testSlotRequestWithFreeSlot() throws Exception {
+		final ResourceManagerId resourceManagerId = ResourceManagerId.generate();
+		final ResourceID resourceID = ResourceID.generate();
+		final JobID jobId = new JobID();
+		final SlotID slotId = new SlotID(resourceID, 0);
+		final String targetAddress = "localhost";
+		final ResourceProfile resourceProfile = ResourceProfile.fromResources(42.0, 1337);
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec(resourceManagerId, new TestingResourceActionsBuilder().build())) {
+
+			final CompletableFuture<Tuple6<SlotID, JobID, AllocationID, ResourceProfile, String, ResourceManagerId>> requestFuture = new CompletableFuture<>();
+			// accept an incoming slot request
+			final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder()
+				.setRequestSlotFunction(tuple6 -> {
+					requestFuture.complete(Tuple6.of(tuple6.f0, tuple6.f1, tuple6.f2, tuple6.f3, tuple6.f4, tuple6.f5));
+					return CompletableFuture.completedFuture(Acknowledge.get());
+				})
+				.createTestingTaskExecutorGateway();
+
+			final TaskExecutorConnection taskExecutorConnection = new TaskExecutorConnection(resourceID, taskExecutorGateway);
+
+			final SlotStatus slotStatus = new SlotStatus(slotId, resourceProfile);
+			final SlotReport slotReport = new SlotReport(slotStatus);
+
+			slotManager.registerTaskManager(
+				taskExecutorConnection,
+				slotReport);
+
+			ResourceRequirements requirements = ResourceRequirements.create(
+				jobId,
+				targetAddress,
+				Collections.singleton(ResourceRequirement.create(resourceProfile, 1)));
+			slotManager.processResourceRequirements(requirements);
+
+			assertThat(requestFuture.get(), is(equalTo(Tuple6.of(slotId, jobId, requestFuture.get().f2, resourceProfile, targetAddress, resourceManagerId))));
+
+			DeclarativeTaskManagerSlot slot = slotTracker.getSlot(slotId);
+
+			assertEquals("The slot has not been allocated to the expected allocation id.", jobId, slot.getJobId());
+		}
+	}
+
+	/**
+	 * Checks that reducing resource requirements while a slot allocation is in-progress results in the slot being freed
+	 * one the allocation completes.
+	 */
+	@Test
+	@Ignore
+	public void testResourceRequirementReductionDuringAllocation() throws Exception {
+		final TaskExecutorConnection taskManagerConnection = createTaskExecutorConnection();
+		final ResourceID resourceID = taskManagerConnection.getResourceID();
+		final SlotID slotId = new SlotID(resourceID, 0);
+
+		final SlotReport slotReport = new SlotReport(createFreeSlotStatus(slotId));
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport);
+
+			DeclarativeTaskManagerSlot slot = slotTracker.getSlot(slotId);
+
+			final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot();
+			slotManager.processResourceRequirements(resourceRequirements);
+
+			assertSame(SlotState.PENDING, slot.getState());
+
+			slotManager.processResourceRequirements(ResourceRequirements.create(resourceRequirements.getJobId(), "foobar", Collections.emptyList()));
+
+			slot = slotTracker.getSlot(slotId);
+
+			assertSame(SlotState.FREE, slot.getState());
+		}
+	}
+
+	/**
+	 * Tests that pending slot requests are tried to be fulfilled upon new slot registrations.
+	 */
+	@Test
+	public void testFulfillingPendingSlotRequest() throws Exception {
+		final ResourceManagerId resourceManagerId = ResourceManagerId.generate();
+		final ResourceID resourceID = ResourceID.generate();
+		final JobID jobId = new JobID();
+		final SlotID slotId = new SlotID(resourceID, 0);
+		final String targetAddress = "localhost";
+		final ResourceProfile resourceProfile = ResourceProfile.fromResources(42.0, 1337);
+
+		final AtomicInteger numberAllocateResourceCalls = new AtomicInteger(0);
+		ResourceActions resourceManagerActions = new TestingResourceActionsBuilder()
+			.setAllocateResourceConsumer(ignored -> numberAllocateResourceCalls.incrementAndGet())
+			.build();
+
+		final CompletableFuture<Tuple6<SlotID, JobID, AllocationID, ResourceProfile, String, ResourceManagerId>> requestFuture = new CompletableFuture<>();
+		// accept an incoming slot request
+		final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder()
+			.setRequestSlotFunction(tuple6 -> {
+				requestFuture.complete(Tuple6.of(tuple6.f0, tuple6.f1, tuple6.f2, tuple6.f3, tuple6.f4, tuple6.f5));
+				return CompletableFuture.completedFuture(Acknowledge.get());
+			})
+			.createTestingTaskExecutorGateway();
+
+		final TaskExecutorConnection taskExecutorConnection = new TaskExecutorConnection(resourceID, taskExecutorGateway);
+
+		final SlotStatus slotStatus = new SlotStatus(slotId, resourceProfile);
+		final SlotReport slotReport = new SlotReport(slotStatus);
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec(resourceManagerId, resourceManagerActions)) {
+
+			final ResourceRequirements resourceRequirements = ResourceRequirements.create(
+				jobId,
+				targetAddress,
+				Collections.singleton(ResourceRequirement.create(resourceProfile, 1)));
+			slotManager.processResourceRequirements(resourceRequirements);
+
+			assertThat(numberAllocateResourceCalls.get(), is(1));
+
+			slotManager.registerTaskManager(
+				taskExecutorConnection,
+				slotReport);
+
+			assertThat(requestFuture.get(), is(equalTo(Tuple6.of(slotId, jobId, requestFuture.get().f2, resourceProfile, targetAddress, resourceManagerId))));
+
+			DeclarativeTaskManagerSlot slot = slotTracker.getSlot(slotId);
+
+			assertEquals("The slot has not been allocated to the expected job id.", jobId, slot.getJobId());
+		}
+	}
+
+	/**
+	 * Tests that freeing a slot will correctly reset the slot and mark it as a free slot.
+	 */
+	@Test
+	public void testFreeSlot() throws Exception {
+		final TaskExecutorConnection taskExecutorConnection = createTaskExecutorConnection();
+		final ResourceID resourceID = taskExecutorConnection.getResourceID();
+		final SlotID slotId = new SlotID(resourceID, 0);
+
+		final SlotReport slotReport = new SlotReport(createAllocatedSlotStatus(slotId));
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			slotManager.registerTaskManager(
+				taskExecutorConnection,
+				slotReport);
+
+			DeclarativeTaskManagerSlot slot = slotTracker.getSlot(slotId);
+
+			assertSame(SlotState.ALLOCATED, slot.getState());
+
+			slotManager.freeSlot(slotId, new AllocationID());
+
+			assertSame(SlotState.FREE, slot.getState());
+
+			assertEquals(1, slotManager.getNumberFreeSlots());
+		}
+	}
+
+	/**
+	 * Tests that duplicate resource requirement declaration do not result in additional slots being allocated after a
+	 * pending slot request has been fulfilled but not yet freed.
+	 */
+	@Test
+	public void testDuplicateResourceRequirementDeclarationAfterSuccessfulAllocation() throws Exception {
+		final ResourceManagerId resourceManagerId = ResourceManagerId.generate();
+		final AtomicInteger allocateResourceCalls = new AtomicInteger(0);
+		final ResourceActions resourceManagerActions = new TestingResourceActionsBuilder()
+			.setAllocateResourceConsumer(ignored -> allocateResourceCalls.incrementAndGet())
+			.build();
+		ResourceRequirements requirements = createResourceRequirementsForSingleSlot();
+
+		final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder().createTestingTaskExecutorGateway();
+
+		final ResourceID resourceID = ResourceID.generate();
+
+		final TaskExecutorConnection taskManagerConnection = new TaskExecutorConnection(resourceID, taskExecutorGateway);
+
+		final SlotID slotId = new SlotID(resourceID, 0);
+		final SlotReport slotReport = new SlotReport(createFreeSlotStatus(slotId));
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec(resourceManagerId, resourceManagerActions)) {
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport);
+
+			slotManager.processResourceRequirements(requirements);
+
+			DeclarativeTaskManagerSlot slot = slotTracker.getSlot(slotId);
+
+			assertThat(slot.getState(), is(SlotState.ALLOCATED));
+
+			slotManager.processResourceRequirements(requirements);
+		}
+
+		// check that we have only called the resource allocation only for the first slot request,
+		// since the second request is a duplicate
+		assertThat(allocateResourceCalls.get(), is(0));
+	}
+
+	/**
+	 * Tests that a slot allocated for one job can be allocated for another job after being freed.
+	 */
+	@Test
+	public void testSlotCanBeAllocatedForDifferentJobAfterFree() throws Exception {
+		final AllocationID allocationId = new AllocationID();
+		final ResourceRequirements resourceRequirements1 = createResourceRequirementsForSingleSlot();
+		final ResourceRequirements resourceRequirements2 = createResourceRequirementsForSingleSlot();
+
+		final TaskExecutorConnection taskManagerConnection = createTaskExecutorConnection();
+		final ResourceID resourceID = taskManagerConnection.getResourceID();
+
+		final SlotID slotId = new SlotID(resourceID, 0);
+		final SlotReport slotReport = new SlotReport(createFreeSlotStatus(slotId));
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport);
+
+			slotManager.processResourceRequirements(resourceRequirements1);
+
+			DeclarativeTaskManagerSlot slot = slotTracker.getSlot(slotId);
+
+			assertEquals("The slot has not been allocated to the expected job id.", resourceRequirements1.getJobId(), slot.getJobId());
+
+			// clear resource requirements, so that the slot isn't immediately re-assigned
+			slotManager.processResourceRequirements(ResourceRequirements.create(resourceRequirements1.getJobId(), resourceRequirements1.getTargetAddress(), Collections.emptyList()));
+			slotManager.freeSlot(slotId, allocationId);
+
+			// check that the slot has been freed
+			assertSame(SlotState.FREE, slot.getState());
+
+			slotManager.processResourceRequirements(resourceRequirements2);
+
+			assertEquals("The slot has not been allocated to the expected job id.", resourceRequirements2.getJobId(), slot.getJobId());
+		}
+	}
+
+	/**
+	 * Tests that the slot manager ignores slot reports of unknown origin (not registered
+	 * task managers).
+	 */
+	@Test
+	public void testReceivingUnknownSlotReport() throws Exception {
+		final ResourceManagerId resourceManagerId = ResourceManagerId.generate();
+		final ResourceActions resourceManagerActions = new TestingResourceActionsBuilder().build();
+
+		final InstanceID unknownInstanceID = new InstanceID();
+		final SlotID unknownSlotId = new SlotID(ResourceID.generate(), 0);
+		final SlotReport unknownSlotReport = new SlotReport(createFreeSlotStatus(unknownSlotId));
+
+		try (SlotManager slotManager = createSlotManager(resourceManagerId, resourceManagerActions)) {
+			// check that we don't have any slots registered
+			assertThat(slotManager.getNumberRegisteredSlots(), is(0));
+
+			// this should not update anything since the instance id is not known to the slot manager
+			assertFalse(slotManager.reportSlotStatus(unknownInstanceID, unknownSlotReport));
+
+			assertThat(slotManager.getNumberRegisteredSlots(), is(0));
+		}
+	}
+
+	/**
+	 * Tests that slots are updated with respect to the latest incoming slot report. This means that
+	 * slots for which a report was received are updated accordingly.
+	 */
+	@Test
+	public void testUpdateSlotReport() throws Exception {
+		final TaskExecutorConnection taskManagerConnection = createTaskExecutorConnection();
+		final ResourceID resourceId = taskManagerConnection.getResourceID();
+
+		final SlotID slotId1 = new SlotID(resourceId, 0);
+		final SlotID slotId2 = new SlotID(resourceId, 1);
+
+		final SlotStatus slotStatus1 = createFreeSlotStatus(slotId1);
+		final SlotStatus slotStatus2 = createFreeSlotStatus(slotId2);
+
+		final SlotStatus newSlotStatus2 = createAllocatedSlotStatus(slotId2);
+		final JobID jobId = newSlotStatus2.getJobID();
+
+		final SlotReport slotReport1 = new SlotReport(Arrays.asList(slotStatus1, slotStatus2));
+		final SlotReport slotReport2 = new SlotReport(Arrays.asList(newSlotStatus2, slotStatus1));
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			// check that we don't have any slots registered
+			assertEquals(0, slotManager.getNumberRegisteredSlots());
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport1);
+
+			DeclarativeTaskManagerSlot slot1 = slotTracker.getSlot(slotId1);
+			DeclarativeTaskManagerSlot slot2 = slotTracker.getSlot(slotId2);
+
+			assertEquals(2, slotManager.getNumberRegisteredSlots());
+
+			assertSame(SlotState.FREE, slot1.getState());
+			assertSame(SlotState.FREE, slot2.getState());
+
+			assertTrue(slotManager.reportSlotStatus(taskManagerConnection.getInstanceID(), slotReport2));
+
+			assertEquals(2, slotManager.getNumberRegisteredSlots());
+
+			assertNotNull(slotTracker.getSlot(slotId1));
+			assertNotNull(slotTracker.getSlot(slotId2));
+
+			// slotId2 should have been allocated for jobiD
+			assertEquals(jobId, slotTracker.getSlot(slotId2).getJobId());
+		}
+	}
+
+	/**
+	 * Tests that if a slot allocation times out we try to allocate another slot.
+	 */
+	@Test
+	public void testSlotAllocationTimeout() throws Exception {
+		final CompletableFuture<Void> secondSlotRequestFuture = new CompletableFuture<>();
+		final AtomicInteger slotRequestsCount = new AtomicInteger();
+		final TaskExecutorConnection taskManagerConnection = createTaskExecutorConnection(new TestingTaskExecutorGatewayBuilder()
+			.setRequestSlotFunction(ignored -> {
+				if (slotRequestsCount.getAndAdd(1) == 1) {
+					secondSlotRequestFuture.complete(null);
+				} else {
+					// mimic RPC timeout
+					return FutureUtils.completedExceptionally(new AskTimeoutException("timeout"));
+				}
+				return new CompletableFuture<>();
+			})
+			.createTestingTaskExecutorGateway());
+		final SlotReport slotReport = createSlotReport(taskManagerConnection.getResourceID(), 2);
+
+		final Executor mainThreadExecutor = TestingUtils.defaultExecutor();
+
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.build()) {
+
+			slotManager.start(ResourceManagerId.generate(), mainThreadExecutor, new TestingResourceActionsBuilder().build());
+
+			CompletableFuture.runAsync(() -> slotManager.registerTaskManager(taskManagerConnection, slotReport), mainThreadExecutor)
+				.thenRun(() -> slotManager.processResourceRequirements(createResourceRequirementsForSingleSlot()))
+				.get(5, TimeUnit.SECONDS);
+
+			// a second request is only sent if the first request timed out
+			secondSlotRequestFuture.get();
+		}
+	}
+
+	/**
+	 * Tests that a slot allocation is retried if it times out on the task manager side.
+	 */
+	@Test
+	public void testTaskManagerSlotAllocationTimeoutHandling() throws Exception {
+		final JobID jobId = new JobID();
+		final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot(jobId);
+		final CompletableFuture<Acknowledge> slotRequestFuture1 = new CompletableFuture<>();
+		final CompletableFuture<Acknowledge> slotRequestFuture2 = new CompletableFuture<>();
+		final Iterator<CompletableFuture<Acknowledge>> slotRequestFutureIterator = Arrays.asList(slotRequestFuture1, slotRequestFuture2).iterator();
+		final ArrayBlockingQueue<SlotID> slotIds = new ArrayBlockingQueue<>(2);
+
+		final TestingTaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder()
+			.setRequestSlotFunction(FunctionUtils.uncheckedFunction(
+				requestSlotParameters -> {
+					slotIds.put(requestSlotParameters.f0);
+					return slotRequestFutureIterator.next();
+				}))
+			.createTestingTaskExecutorGateway();
+
+		final ResourceID resourceId = ResourceID.generate();
+		final TaskExecutorConnection taskManagerConnection = new TaskExecutorConnection(resourceId, taskExecutorGateway);
+
+		final SlotID slotId1 = new SlotID(resourceId, 0);
+		final SlotID slotId2 = new SlotID(resourceId, 1);
+		final SlotReport slotReport = new SlotReport(Arrays.asList(createFreeSlotStatus(slotId1), createFreeSlotStatus(slotId2)));
+
+		final ResourceTracker resourceTracker = new DefaultResourceTracker();
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setResourceTracker(resourceTracker)
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport);
+
+			slotManager.processResourceRequirements(resourceRequirements);
+
+			final SlotID firstSlotId = slotIds.take();
+			assertThat(slotIds, is(empty()));
+
+			DeclarativeTaskManagerSlot failedSlot = slotTracker.getSlot(firstSlotId);
+
+			// let the first attempt fail --> this should trigger a second attempt
+			slotRequestFuture1.completeExceptionally(new SlotAllocationException("Test exception."));
+
+			assertThat(getTotalResourceCount(resourceTracker.getAcquiredResources(jobId)), is(1));
+
+			// the second attempt succeeds
+			slotRequestFuture2.complete(Acknowledge.get());
+
+			final SlotID secondSlotId = slotIds.take();
+			assertThat(slotIds, is(empty()));
+
+			DeclarativeTaskManagerSlot slot = slotTracker.getSlot(secondSlotId);
+
+			assertThat(slot.getState(), is(SlotState.ALLOCATED));
+			assertEquals(jobId, slot.getJobId());
+
+			if (!failedSlot.getSlotId().equals(slot.getSlotId())) {
+				assertThat(failedSlot.getState(), is(SlotState.FREE));
+			}

Review comment:
       It can happen that we retry with `firstSlotId`?

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManagerTest.java
##########
@@ -0,0 +1,1092 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple6;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.concurrent.ManuallyTriggeredScheduledExecutor;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerId;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+import org.apache.flink.runtime.slots.ResourceRequirements;
+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.TestingTaskExecutorGateway;
+import org.apache.flink.runtime.taskexecutor.TestingTaskExecutorGatewayBuilder;
+import org.apache.flink.runtime.taskexecutor.exceptions.SlotAllocationException;
+import org.apache.flink.runtime.taskexecutor.exceptions.SlotOccupiedException;
+import org.apache.flink.runtime.testingUtils.TestingUtils;
+import org.apache.flink.util.FlinkException;
+import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.function.FunctionUtils;
+
+import akka.pattern.AskTimeoutException;
+import org.junit.Ignore;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.hamcrest.CoreMatchers.hasItem;
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.empty;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.hasSize;
+import static org.hamcrest.Matchers.is;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertSame;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Tests for the {@link DeclarativeSlotManager}.
+ */
+public class DeclarativeSlotManagerTest extends TestLogger {
+
+	private static final FlinkException TEST_EXCEPTION = new FlinkException("Test exception");
+
+	private static final WorkerResourceSpec WORKER_RESOURCE_SPEC = new WorkerResourceSpec.Builder()
+		.setCpuCores(100.0)
+		.setTaskHeapMemoryMB(10000)
+		.setTaskOffHeapMemoryMB(10000)
+		.setNetworkMemoryMB(10000)
+		.setManagedMemoryMB(10000)
+		.build();
+
+	/**
+	 * Tests that we can register task manager and their slots at the slot manager.
+	 */
+	@Test
+	public void testTaskManagerRegistration() throws Exception {
+		final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder().createTestingTaskExecutorGateway();
+		final ResourceID resourceId = ResourceID.generate();
+		final TaskExecutorConnection taskManagerConnection = new TaskExecutorConnection(resourceId, taskExecutorGateway);
+
+		final SlotID slotId1 = new SlotID(resourceId, 0);
+		final SlotID slotId2 = new SlotID(resourceId, 1);
+		final SlotReport slotReport = new SlotReport(Arrays.asList(createFreeSlotStatus(slotId1), createFreeSlotStatus(slotId2)));
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport);
+
+			assertThat("The number registered slots does not equal the expected number.", slotManager.getNumberRegisteredSlots(), is(2));
+
+			assertNotNull(slotTracker.getSlot(slotId1));
+			assertNotNull(slotTracker.getSlot(slotId2));
+		}
+	}
+
+	/**
+	 * Tests that un-registration of task managers will free and remove all registered slots.
+	 */
+	@Test
+	public void testTaskManagerUnregistration() throws Exception {
+		final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder()
+			.setRequestSlotFunction(tuple6 -> new CompletableFuture<>())
+			.createTestingTaskExecutorGateway();
+		final TaskExecutorConnection taskManagerConnection = createTaskExecutorConnection(taskExecutorGateway);
+		final ResourceID resourceId = taskManagerConnection.getResourceID();
+
+		final SlotID slotId1 = new SlotID(resourceId, 0);
+		final SlotID slotId2 = new SlotID(resourceId, 1);
+		final SlotReport slotReport = new SlotReport(Arrays.asList(createAllocatedSlotStatus(slotId1), createFreeSlotStatus(slotId2)));
+
+		final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot();
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport);
+
+			assertEquals("The number registered slots does not equal the expected number.", 2, slotManager.getNumberRegisteredSlots());
+
+			DeclarativeTaskManagerSlot slot1 = slotTracker.getSlot(slotId1);
+			DeclarativeTaskManagerSlot slot2 = slotTracker.getSlot(slotId2);
+
+			assertSame(SlotState.ALLOCATED, slot1.getState());
+			assertSame(SlotState.FREE, slot2.getState());
+
+			slotManager.processResourceRequirements(resourceRequirements);
+
+			assertSame(SlotState.PENDING, slot2.getState());
+
+			slotManager.unregisterTaskManager(taskManagerConnection.getInstanceID(), TEST_EXCEPTION);
+
+			assertEquals(0, slotManager.getNumberRegisteredSlots());
+		}
+	}
+
+	/**
+	 * Tests that a slot request with no free slots will trigger the resource allocation.
+	 */
+	@Test
+	public void testSlotRequestWithoutFreeSlots() throws Exception {
+		final ResourceManagerId resourceManagerId = ResourceManagerId.generate();
+
+		final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot();
+
+		CompletableFuture<WorkerResourceSpec> allocateResourceFuture = new CompletableFuture<>();
+		ResourceActions resourceManagerActions = new TestingResourceActionsBuilder()
+			.setAllocateResourceConsumer(allocateResourceFuture::complete)
+			.build();
+
+		try (SlotManager slotManager = createSlotManager(resourceManagerId, resourceManagerActions)) {
+
+			slotManager.processResourceRequirements(resourceRequirements);
+
+			allocateResourceFuture.get();
+		}
+	}
+
+	/**
+	 * Tests that resources continue to be considered missing if we cannot allocate more resources.
+	 */
+	@Test
+	public void testResourceDeclarationWithResourceAllocationFailure() throws Exception {
+		final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot();
+
+		ResourceActions resourceManagerActions = new TestingResourceActionsBuilder()
+			.setAllocateResourceFunction(value -> false)
+			.build();
+
+		final ResourceTracker resourceTracker = new DefaultResourceTracker();
+
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setResourceTracker(resourceTracker)
+			.buildAndStartWithDirectExec(ResourceManagerId.generate(), resourceManagerActions)) {
+
+			slotManager.processResourceRequirements(resourceRequirements);
+
+			final JobID jobId = resourceRequirements.getJobId();
+			assertThat(getTotalResourceCount(resourceTracker.getMissingResources().get(jobId)), is(1));
+		}
+	}
+
+	/**
+	 * Tests that a slot request which can be fulfilled will trigger a slot allocation.
+	 */
+	@Test
+	public void testSlotRequestWithFreeSlot() throws Exception {
+		final ResourceManagerId resourceManagerId = ResourceManagerId.generate();
+		final ResourceID resourceID = ResourceID.generate();
+		final JobID jobId = new JobID();
+		final SlotID slotId = new SlotID(resourceID, 0);
+		final String targetAddress = "localhost";
+		final ResourceProfile resourceProfile = ResourceProfile.fromResources(42.0, 1337);
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec(resourceManagerId, new TestingResourceActionsBuilder().build())) {
+
+			final CompletableFuture<Tuple6<SlotID, JobID, AllocationID, ResourceProfile, String, ResourceManagerId>> requestFuture = new CompletableFuture<>();
+			// accept an incoming slot request
+			final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder()
+				.setRequestSlotFunction(tuple6 -> {
+					requestFuture.complete(Tuple6.of(tuple6.f0, tuple6.f1, tuple6.f2, tuple6.f3, tuple6.f4, tuple6.f5));
+					return CompletableFuture.completedFuture(Acknowledge.get());
+				})
+				.createTestingTaskExecutorGateway();
+
+			final TaskExecutorConnection taskExecutorConnection = new TaskExecutorConnection(resourceID, taskExecutorGateway);
+
+			final SlotStatus slotStatus = new SlotStatus(slotId, resourceProfile);
+			final SlotReport slotReport = new SlotReport(slotStatus);
+
+			slotManager.registerTaskManager(
+				taskExecutorConnection,
+				slotReport);
+
+			ResourceRequirements requirements = ResourceRequirements.create(
+				jobId,
+				targetAddress,
+				Collections.singleton(ResourceRequirement.create(resourceProfile, 1)));
+			slotManager.processResourceRequirements(requirements);
+
+			assertThat(requestFuture.get(), is(equalTo(Tuple6.of(slotId, jobId, requestFuture.get().f2, resourceProfile, targetAddress, resourceManagerId))));
+
+			DeclarativeTaskManagerSlot slot = slotTracker.getSlot(slotId);
+
+			assertEquals("The slot has not been allocated to the expected allocation id.", jobId, slot.getJobId());
+		}
+	}
+
+	/**
+	 * Checks that reducing resource requirements while a slot allocation is in-progress results in the slot being freed
+	 * one the allocation completes.
+	 */
+	@Test
+	@Ignore
+	public void testResourceRequirementReductionDuringAllocation() throws Exception {
+		final TaskExecutorConnection taskManagerConnection = createTaskExecutorConnection();
+		final ResourceID resourceID = taskManagerConnection.getResourceID();
+		final SlotID slotId = new SlotID(resourceID, 0);
+
+		final SlotReport slotReport = new SlotReport(createFreeSlotStatus(slotId));
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport);
+
+			DeclarativeTaskManagerSlot slot = slotTracker.getSlot(slotId);
+
+			final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot();
+			slotManager.processResourceRequirements(resourceRequirements);
+
+			assertSame(SlotState.PENDING, slot.getState());
+
+			slotManager.processResourceRequirements(ResourceRequirements.create(resourceRequirements.getJobId(), "foobar", Collections.emptyList()));
+
+			slot = slotTracker.getSlot(slotId);
+
+			assertSame(SlotState.FREE, slot.getState());
+		}
+	}
+
+	/**
+	 * Tests that pending slot requests are tried to be fulfilled upon new slot registrations.
+	 */
+	@Test
+	public void testFulfillingPendingSlotRequest() throws Exception {
+		final ResourceManagerId resourceManagerId = ResourceManagerId.generate();
+		final ResourceID resourceID = ResourceID.generate();
+		final JobID jobId = new JobID();
+		final SlotID slotId = new SlotID(resourceID, 0);
+		final String targetAddress = "localhost";
+		final ResourceProfile resourceProfile = ResourceProfile.fromResources(42.0, 1337);
+
+		final AtomicInteger numberAllocateResourceCalls = new AtomicInteger(0);
+		ResourceActions resourceManagerActions = new TestingResourceActionsBuilder()
+			.setAllocateResourceConsumer(ignored -> numberAllocateResourceCalls.incrementAndGet())
+			.build();
+
+		final CompletableFuture<Tuple6<SlotID, JobID, AllocationID, ResourceProfile, String, ResourceManagerId>> requestFuture = new CompletableFuture<>();
+		// accept an incoming slot request
+		final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder()
+			.setRequestSlotFunction(tuple6 -> {
+				requestFuture.complete(Tuple6.of(tuple6.f0, tuple6.f1, tuple6.f2, tuple6.f3, tuple6.f4, tuple6.f5));
+				return CompletableFuture.completedFuture(Acknowledge.get());
+			})
+			.createTestingTaskExecutorGateway();
+
+		final TaskExecutorConnection taskExecutorConnection = new TaskExecutorConnection(resourceID, taskExecutorGateway);
+
+		final SlotStatus slotStatus = new SlotStatus(slotId, resourceProfile);
+		final SlotReport slotReport = new SlotReport(slotStatus);
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec(resourceManagerId, resourceManagerActions)) {
+
+			final ResourceRequirements resourceRequirements = ResourceRequirements.create(
+				jobId,
+				targetAddress,
+				Collections.singleton(ResourceRequirement.create(resourceProfile, 1)));
+			slotManager.processResourceRequirements(resourceRequirements);
+
+			assertThat(numberAllocateResourceCalls.get(), is(1));
+
+			slotManager.registerTaskManager(
+				taskExecutorConnection,
+				slotReport);
+
+			assertThat(requestFuture.get(), is(equalTo(Tuple6.of(slotId, jobId, requestFuture.get().f2, resourceProfile, targetAddress, resourceManagerId))));
+
+			DeclarativeTaskManagerSlot slot = slotTracker.getSlot(slotId);
+
+			assertEquals("The slot has not been allocated to the expected job id.", jobId, slot.getJobId());
+		}
+	}
+
+	/**
+	 * Tests that freeing a slot will correctly reset the slot and mark it as a free slot.
+	 */
+	@Test
+	public void testFreeSlot() throws Exception {
+		final TaskExecutorConnection taskExecutorConnection = createTaskExecutorConnection();
+		final ResourceID resourceID = taskExecutorConnection.getResourceID();
+		final SlotID slotId = new SlotID(resourceID, 0);
+
+		final SlotReport slotReport = new SlotReport(createAllocatedSlotStatus(slotId));
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			slotManager.registerTaskManager(
+				taskExecutorConnection,
+				slotReport);
+
+			DeclarativeTaskManagerSlot slot = slotTracker.getSlot(slotId);
+
+			assertSame(SlotState.ALLOCATED, slot.getState());
+
+			slotManager.freeSlot(slotId, new AllocationID());
+
+			assertSame(SlotState.FREE, slot.getState());
+
+			assertEquals(1, slotManager.getNumberFreeSlots());
+		}
+	}
+
+	/**
+	 * Tests that duplicate resource requirement declaration do not result in additional slots being allocated after a
+	 * pending slot request has been fulfilled but not yet freed.
+	 */
+	@Test
+	public void testDuplicateResourceRequirementDeclarationAfterSuccessfulAllocation() throws Exception {
+		final ResourceManagerId resourceManagerId = ResourceManagerId.generate();
+		final AtomicInteger allocateResourceCalls = new AtomicInteger(0);
+		final ResourceActions resourceManagerActions = new TestingResourceActionsBuilder()
+			.setAllocateResourceConsumer(ignored -> allocateResourceCalls.incrementAndGet())
+			.build();
+		ResourceRequirements requirements = createResourceRequirementsForSingleSlot();
+
+		final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder().createTestingTaskExecutorGateway();
+
+		final ResourceID resourceID = ResourceID.generate();
+
+		final TaskExecutorConnection taskManagerConnection = new TaskExecutorConnection(resourceID, taskExecutorGateway);
+
+		final SlotID slotId = new SlotID(resourceID, 0);
+		final SlotReport slotReport = new SlotReport(createFreeSlotStatus(slotId));
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec(resourceManagerId, resourceManagerActions)) {
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport);
+
+			slotManager.processResourceRequirements(requirements);
+
+			DeclarativeTaskManagerSlot slot = slotTracker.getSlot(slotId);
+
+			assertThat(slot.getState(), is(SlotState.ALLOCATED));
+
+			slotManager.processResourceRequirements(requirements);
+		}
+
+		// check that we have only called the resource allocation only for the first slot request,
+		// since the second request is a duplicate
+		assertThat(allocateResourceCalls.get(), is(0));
+	}
+
+	/**
+	 * Tests that a slot allocated for one job can be allocated for another job after being freed.
+	 */
+	@Test
+	public void testSlotCanBeAllocatedForDifferentJobAfterFree() throws Exception {
+		final AllocationID allocationId = new AllocationID();
+		final ResourceRequirements resourceRequirements1 = createResourceRequirementsForSingleSlot();
+		final ResourceRequirements resourceRequirements2 = createResourceRequirementsForSingleSlot();
+
+		final TaskExecutorConnection taskManagerConnection = createTaskExecutorConnection();
+		final ResourceID resourceID = taskManagerConnection.getResourceID();
+
+		final SlotID slotId = new SlotID(resourceID, 0);
+		final SlotReport slotReport = new SlotReport(createFreeSlotStatus(slotId));
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport);
+
+			slotManager.processResourceRequirements(resourceRequirements1);
+
+			DeclarativeTaskManagerSlot slot = slotTracker.getSlot(slotId);
+
+			assertEquals("The slot has not been allocated to the expected job id.", resourceRequirements1.getJobId(), slot.getJobId());
+
+			// clear resource requirements, so that the slot isn't immediately re-assigned
+			slotManager.processResourceRequirements(ResourceRequirements.create(resourceRequirements1.getJobId(), resourceRequirements1.getTargetAddress(), Collections.emptyList()));
+			slotManager.freeSlot(slotId, allocationId);
+
+			// check that the slot has been freed
+			assertSame(SlotState.FREE, slot.getState());
+
+			slotManager.processResourceRequirements(resourceRequirements2);
+
+			assertEquals("The slot has not been allocated to the expected job id.", resourceRequirements2.getJobId(), slot.getJobId());
+		}
+	}
+
+	/**
+	 * Tests that the slot manager ignores slot reports of unknown origin (not registered
+	 * task managers).
+	 */
+	@Test
+	public void testReceivingUnknownSlotReport() throws Exception {
+		final ResourceManagerId resourceManagerId = ResourceManagerId.generate();
+		final ResourceActions resourceManagerActions = new TestingResourceActionsBuilder().build();
+
+		final InstanceID unknownInstanceID = new InstanceID();
+		final SlotID unknownSlotId = new SlotID(ResourceID.generate(), 0);
+		final SlotReport unknownSlotReport = new SlotReport(createFreeSlotStatus(unknownSlotId));
+
+		try (SlotManager slotManager = createSlotManager(resourceManagerId, resourceManagerActions)) {
+			// check that we don't have any slots registered
+			assertThat(slotManager.getNumberRegisteredSlots(), is(0));
+
+			// this should not update anything since the instance id is not known to the slot manager
+			assertFalse(slotManager.reportSlotStatus(unknownInstanceID, unknownSlotReport));
+
+			assertThat(slotManager.getNumberRegisteredSlots(), is(0));
+		}
+	}
+
+	/**
+	 * Tests that slots are updated with respect to the latest incoming slot report. This means that
+	 * slots for which a report was received are updated accordingly.
+	 */
+	@Test
+	public void testUpdateSlotReport() throws Exception {
+		final TaskExecutorConnection taskManagerConnection = createTaskExecutorConnection();
+		final ResourceID resourceId = taskManagerConnection.getResourceID();
+
+		final SlotID slotId1 = new SlotID(resourceId, 0);
+		final SlotID slotId2 = new SlotID(resourceId, 1);
+
+		final SlotStatus slotStatus1 = createFreeSlotStatus(slotId1);
+		final SlotStatus slotStatus2 = createFreeSlotStatus(slotId2);
+
+		final SlotStatus newSlotStatus2 = createAllocatedSlotStatus(slotId2);
+		final JobID jobId = newSlotStatus2.getJobID();
+
+		final SlotReport slotReport1 = new SlotReport(Arrays.asList(slotStatus1, slotStatus2));
+		final SlotReport slotReport2 = new SlotReport(Arrays.asList(newSlotStatus2, slotStatus1));
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			// check that we don't have any slots registered
+			assertEquals(0, slotManager.getNumberRegisteredSlots());
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport1);
+
+			DeclarativeTaskManagerSlot slot1 = slotTracker.getSlot(slotId1);
+			DeclarativeTaskManagerSlot slot2 = slotTracker.getSlot(slotId2);
+
+			assertEquals(2, slotManager.getNumberRegisteredSlots());
+
+			assertSame(SlotState.FREE, slot1.getState());
+			assertSame(SlotState.FREE, slot2.getState());
+
+			assertTrue(slotManager.reportSlotStatus(taskManagerConnection.getInstanceID(), slotReport2));
+
+			assertEquals(2, slotManager.getNumberRegisteredSlots());
+
+			assertNotNull(slotTracker.getSlot(slotId1));
+			assertNotNull(slotTracker.getSlot(slotId2));
+
+			// slotId2 should have been allocated for jobiD
+			assertEquals(jobId, slotTracker.getSlot(slotId2).getJobId());
+		}
+	}
+
+	/**
+	 * Tests that if a slot allocation times out we try to allocate another slot.
+	 */
+	@Test
+	public void testSlotAllocationTimeout() throws Exception {
+		final CompletableFuture<Void> secondSlotRequestFuture = new CompletableFuture<>();
+		final AtomicInteger slotRequestsCount = new AtomicInteger();
+		final TaskExecutorConnection taskManagerConnection = createTaskExecutorConnection(new TestingTaskExecutorGatewayBuilder()
+			.setRequestSlotFunction(ignored -> {
+				if (slotRequestsCount.getAndAdd(1) == 1) {
+					secondSlotRequestFuture.complete(null);
+				} else {
+					// mimic RPC timeout
+					return FutureUtils.completedExceptionally(new AskTimeoutException("timeout"));
+				}
+				return new CompletableFuture<>();
+			})
+			.createTestingTaskExecutorGateway());
+		final SlotReport slotReport = createSlotReport(taskManagerConnection.getResourceID(), 2);
+
+		final Executor mainThreadExecutor = TestingUtils.defaultExecutor();
+
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.build()) {
+
+			slotManager.start(ResourceManagerId.generate(), mainThreadExecutor, new TestingResourceActionsBuilder().build());
+
+			CompletableFuture.runAsync(() -> slotManager.registerTaskManager(taskManagerConnection, slotReport), mainThreadExecutor)
+				.thenRun(() -> slotManager.processResourceRequirements(createResourceRequirementsForSingleSlot()))
+				.get(5, TimeUnit.SECONDS);
+
+			// a second request is only sent if the first request timed out
+			secondSlotRequestFuture.get();
+		}
+	}
+
+	/**
+	 * Tests that a slot allocation is retried if it times out on the task manager side.
+	 */
+	@Test
+	public void testTaskManagerSlotAllocationTimeoutHandling() throws Exception {
+		final JobID jobId = new JobID();
+		final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot(jobId);
+		final CompletableFuture<Acknowledge> slotRequestFuture1 = new CompletableFuture<>();
+		final CompletableFuture<Acknowledge> slotRequestFuture2 = new CompletableFuture<>();
+		final Iterator<CompletableFuture<Acknowledge>> slotRequestFutureIterator = Arrays.asList(slotRequestFuture1, slotRequestFuture2).iterator();
+		final ArrayBlockingQueue<SlotID> slotIds = new ArrayBlockingQueue<>(2);
+
+		final TestingTaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder()
+			.setRequestSlotFunction(FunctionUtils.uncheckedFunction(
+				requestSlotParameters -> {
+					slotIds.put(requestSlotParameters.f0);
+					return slotRequestFutureIterator.next();
+				}))
+			.createTestingTaskExecutorGateway();
+
+		final ResourceID resourceId = ResourceID.generate();
+		final TaskExecutorConnection taskManagerConnection = new TaskExecutorConnection(resourceId, taskExecutorGateway);
+
+		final SlotID slotId1 = new SlotID(resourceId, 0);
+		final SlotID slotId2 = new SlotID(resourceId, 1);
+		final SlotReport slotReport = new SlotReport(Arrays.asList(createFreeSlotStatus(slotId1), createFreeSlotStatus(slotId2)));
+
+		final ResourceTracker resourceTracker = new DefaultResourceTracker();
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setResourceTracker(resourceTracker)
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport);
+
+			slotManager.processResourceRequirements(resourceRequirements);
+
+			final SlotID firstSlotId = slotIds.take();
+			assertThat(slotIds, is(empty()));
+
+			DeclarativeTaskManagerSlot failedSlot = slotTracker.getSlot(firstSlotId);
+
+			// let the first attempt fail --> this should trigger a second attempt
+			slotRequestFuture1.completeExceptionally(new SlotAllocationException("Test exception."));
+
+			assertThat(getTotalResourceCount(resourceTracker.getAcquiredResources(jobId)), is(1));
+
+			// the second attempt succeeds
+			slotRequestFuture2.complete(Acknowledge.get());
+
+			final SlotID secondSlotId = slotIds.take();
+			assertThat(slotIds, is(empty()));
+
+			DeclarativeTaskManagerSlot slot = slotTracker.getSlot(secondSlotId);
+
+			assertThat(slot.getState(), is(SlotState.ALLOCATED));
+			assertEquals(jobId, slot.getJobId());
+
+			if (!failedSlot.getSlotId().equals(slot.getSlotId())) {
+				assertThat(failedSlot.getState(), is(SlotState.FREE));
+			}
+		}
+	}
+
+	/**
+	 * Tests that a pending slot allocation is cancelled if a slot report indicates that the slot is already allocated
+	 * by another job.
+	 */
+	@Test
+	public void testSlotReportWithConflictingJobIdDuringSlotAllocation() throws Exception {
+		final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot();
+		final ArrayBlockingQueue<SlotID> requestedSlotIds = new ArrayBlockingQueue<>(2);
+
+		final TestingTaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder()
+			.setRequestSlotFunction(FunctionUtils.uncheckedFunction(
+				requestSlotParameters -> {
+					requestedSlotIds.put(requestSlotParameters.f0);
+					return new CompletableFuture<>();
+				}))
+			.createTestingTaskExecutorGateway();
+
+		final TaskExecutorConnection taskExecutorConnection = createTaskExecutorConnection(taskExecutorGateway);
+		final ResourceID resourceId = taskExecutorConnection.getResourceID();
+
+		final SlotID slotId1 = new SlotID(resourceId, 0);
+		final SlotID slotId2 = new SlotID(resourceId, 1);
+		final SlotReport slotReport = new SlotReport(Arrays.asList(createFreeSlotStatus(slotId1), createFreeSlotStatus(slotId2)));
+
+		final ScheduledExecutor mainThreadExecutor = new ManuallyTriggeredScheduledExecutor();
+
+		try (final DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setScheduledExecutor(mainThreadExecutor)
+			.build()) {
+
+			slotManager.start(ResourceManagerId.generate(), mainThreadExecutor, new TestingResourceActionsBuilder().build());
+
+			slotManager.registerTaskManager(taskExecutorConnection, slotReport);
+			slotManager.processResourceRequirements(resourceRequirements);
+
+			final SlotID firstRequestedSlotId = requestedSlotIds.take();
+			final SlotID freeSlotId = firstRequestedSlotId.equals(slotId1) ? slotId2 : slotId1;
+
+			final SlotReport newSlotReport = new SlotReport(Arrays.asList(createAllocatedSlotStatus(firstRequestedSlotId), createFreeSlotStatus(freeSlotId)));
+
+			slotManager.reportSlotStatus(taskExecutorConnection.getInstanceID(), newSlotReport);
+
+			final SlotID secondRequestedSlotId = requestedSlotIds.take();
+
+			assertEquals(freeSlotId, secondRequestedSlotId);
+		}
+	}
+
+	/**
+	 * Tests that free slots which are reported as allocated won't be considered for fulfilling
+	 * other pending slot requests.
+	 *
+	 * <p>See: FLINK-8505
+	 */
+	@Test
+	public void testReportAllocatedSlot() throws Exception {
+		final ResourceID taskManagerId = ResourceID.generate();
+		final TestingTaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder().createTestingTaskExecutorGateway();
+		final TaskExecutorConnection taskExecutorConnection = new TaskExecutorConnection(taskManagerId, taskExecutorGateway);
+
+		final ResourceTracker resourceTracker = new DefaultResourceTracker();
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setResourceTracker(resourceTracker)
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			// initially report a single slot as free
+			final SlotID slotId = new SlotID(taskManagerId, 0);
+			final SlotReport initialSlotReport = new SlotReport(createFreeSlotStatus(slotId));
+
+			slotManager.registerTaskManager(taskExecutorConnection, initialSlotReport);
+
+			assertThat(slotManager.getNumberRegisteredSlots(), is(equalTo(1)));
+
+			// Now report this slot as allocated
+			final SlotStatus slotStatus = createAllocatedSlotStatus(slotId);
+			final SlotReport slotReport = new SlotReport(slotStatus);
+
+			slotManager.reportSlotStatus(
+				taskExecutorConnection.getInstanceID(),
+				slotReport);
+
+			final JobID jobId = new JobID();
+			// this resource requirement should not be fulfilled
+			ResourceRequirements requirements = createResourceRequirementsForSingleSlot(jobId);
+
+			slotManager.processResourceRequirements(requirements);
+
+			assertThat(slotTracker.getSlot(slotId).getJobId(), is(slotStatus.getJobID()));
+			assertThat(getTotalResourceCount(resourceTracker.getMissingResources().get(jobId)), is(1));
+		}
+	}
+
+	/**
+	 * Tests that the SlotManager retries allocating a slot if the TaskExecutor#requestSlot call
+	 * fails.
+	 */
+	@Test
+	public void testSlotRequestFailure() throws Exception {
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (final DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			ResourceRequirements requirements = createResourceRequirementsForSingleSlot();
+			slotManager.processResourceRequirements(requirements);
+
+			final BlockingQueue<Tuple6<SlotID, JobID, AllocationID, ResourceProfile, String, ResourceManagerId>> requestSlotQueue = new ArrayBlockingQueue<>(1);
+			final BlockingQueue<CompletableFuture<Acknowledge>> responseQueue = new ArrayBlockingQueue<>(1);
+
+			final TestingTaskExecutorGateway testingTaskExecutorGateway = new TestingTaskExecutorGatewayBuilder()
+				.setRequestSlotFunction(slotIDJobIDAllocationIDStringResourceManagerIdTuple6 -> {
+					requestSlotQueue.offer(slotIDJobIDAllocationIDStringResourceManagerIdTuple6);
+					try {
+						return responseQueue.take();
+					} catch (InterruptedException ignored) {
+						return FutureUtils.completedExceptionally(new FlinkException("Response queue was interrupted."));
+					}
+				})
+				.createTestingTaskExecutorGateway();
+
+			final ResourceID taskExecutorResourceId = ResourceID.generate();
+			final TaskExecutorConnection taskExecutionConnection = new TaskExecutorConnection(taskExecutorResourceId, testingTaskExecutorGateway);
+			final SlotReport slotReport = new SlotReport(createFreeSlotStatus(new SlotID(taskExecutorResourceId, 0)));
+
+			final CompletableFuture<Acknowledge> firstManualSlotRequestResponse = new CompletableFuture<>();
+			responseQueue.offer(firstManualSlotRequestResponse);
+
+			slotManager.registerTaskManager(taskExecutionConnection, slotReport);
+
+			final Tuple6<SlotID, JobID, AllocationID, ResourceProfile, String, ResourceManagerId> firstRequest = requestSlotQueue.take();
+
+			final CompletableFuture<Acknowledge> secondManualSlotRequestResponse = new CompletableFuture<>();
+			responseQueue.offer(secondManualSlotRequestResponse);

Review comment:
       Could this be simplified by inserting the futures at creation time of the `responseQueue`?

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManagerTest.java
##########
@@ -0,0 +1,1092 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple6;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.concurrent.ManuallyTriggeredScheduledExecutor;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerId;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+import org.apache.flink.runtime.slots.ResourceRequirements;
+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.TestingTaskExecutorGateway;
+import org.apache.flink.runtime.taskexecutor.TestingTaskExecutorGatewayBuilder;
+import org.apache.flink.runtime.taskexecutor.exceptions.SlotAllocationException;
+import org.apache.flink.runtime.taskexecutor.exceptions.SlotOccupiedException;
+import org.apache.flink.runtime.testingUtils.TestingUtils;
+import org.apache.flink.util.FlinkException;
+import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.function.FunctionUtils;
+
+import akka.pattern.AskTimeoutException;
+import org.junit.Ignore;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.hamcrest.CoreMatchers.hasItem;
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.empty;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.hasSize;
+import static org.hamcrest.Matchers.is;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertSame;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Tests for the {@link DeclarativeSlotManager}.
+ */
+public class DeclarativeSlotManagerTest extends TestLogger {
+
+	private static final FlinkException TEST_EXCEPTION = new FlinkException("Test exception");
+
+	private static final WorkerResourceSpec WORKER_RESOURCE_SPEC = new WorkerResourceSpec.Builder()
+		.setCpuCores(100.0)
+		.setTaskHeapMemoryMB(10000)
+		.setTaskOffHeapMemoryMB(10000)
+		.setNetworkMemoryMB(10000)
+		.setManagedMemoryMB(10000)
+		.build();
+
+	/**
+	 * Tests that we can register task manager and their slots at the slot manager.
+	 */
+	@Test
+	public void testTaskManagerRegistration() throws Exception {
+		final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder().createTestingTaskExecutorGateway();
+		final ResourceID resourceId = ResourceID.generate();
+		final TaskExecutorConnection taskManagerConnection = new TaskExecutorConnection(resourceId, taskExecutorGateway);
+
+		final SlotID slotId1 = new SlotID(resourceId, 0);
+		final SlotID slotId2 = new SlotID(resourceId, 1);
+		final SlotReport slotReport = new SlotReport(Arrays.asList(createFreeSlotStatus(slotId1), createFreeSlotStatus(slotId2)));
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport);
+
+			assertThat("The number registered slots does not equal the expected number.", slotManager.getNumberRegisteredSlots(), is(2));
+
+			assertNotNull(slotTracker.getSlot(slotId1));
+			assertNotNull(slotTracker.getSlot(slotId2));
+		}
+	}
+
+	/**
+	 * Tests that un-registration of task managers will free and remove all registered slots.
+	 */
+	@Test
+	public void testTaskManagerUnregistration() throws Exception {
+		final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder()
+			.setRequestSlotFunction(tuple6 -> new CompletableFuture<>())
+			.createTestingTaskExecutorGateway();
+		final TaskExecutorConnection taskManagerConnection = createTaskExecutorConnection(taskExecutorGateway);
+		final ResourceID resourceId = taskManagerConnection.getResourceID();
+
+		final SlotID slotId1 = new SlotID(resourceId, 0);
+		final SlotID slotId2 = new SlotID(resourceId, 1);
+		final SlotReport slotReport = new SlotReport(Arrays.asList(createAllocatedSlotStatus(slotId1), createFreeSlotStatus(slotId2)));
+
+		final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot();
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport);
+
+			assertEquals("The number registered slots does not equal the expected number.", 2, slotManager.getNumberRegisteredSlots());
+
+			DeclarativeTaskManagerSlot slot1 = slotTracker.getSlot(slotId1);
+			DeclarativeTaskManagerSlot slot2 = slotTracker.getSlot(slotId2);
+
+			assertSame(SlotState.ALLOCATED, slot1.getState());
+			assertSame(SlotState.FREE, slot2.getState());
+
+			slotManager.processResourceRequirements(resourceRequirements);
+
+			assertSame(SlotState.PENDING, slot2.getState());
+
+			slotManager.unregisterTaskManager(taskManagerConnection.getInstanceID(), TEST_EXCEPTION);
+
+			assertEquals(0, slotManager.getNumberRegisteredSlots());
+		}
+	}
+
+	/**
+	 * Tests that a slot request with no free slots will trigger the resource allocation.
+	 */
+	@Test
+	public void testSlotRequestWithoutFreeSlots() throws Exception {
+		final ResourceManagerId resourceManagerId = ResourceManagerId.generate();
+
+		final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot();
+
+		CompletableFuture<WorkerResourceSpec> allocateResourceFuture = new CompletableFuture<>();
+		ResourceActions resourceManagerActions = new TestingResourceActionsBuilder()
+			.setAllocateResourceConsumer(allocateResourceFuture::complete)
+			.build();
+
+		try (SlotManager slotManager = createSlotManager(resourceManagerId, resourceManagerActions)) {
+
+			slotManager.processResourceRequirements(resourceRequirements);
+
+			allocateResourceFuture.get();
+		}
+	}
+
+	/**
+	 * Tests that resources continue to be considered missing if we cannot allocate more resources.
+	 */
+	@Test
+	public void testResourceDeclarationWithResourceAllocationFailure() throws Exception {
+		final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot();
+
+		ResourceActions resourceManagerActions = new TestingResourceActionsBuilder()
+			.setAllocateResourceFunction(value -> false)
+			.build();
+
+		final ResourceTracker resourceTracker = new DefaultResourceTracker();
+
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setResourceTracker(resourceTracker)
+			.buildAndStartWithDirectExec(ResourceManagerId.generate(), resourceManagerActions)) {
+
+			slotManager.processResourceRequirements(resourceRequirements);
+
+			final JobID jobId = resourceRequirements.getJobId();
+			assertThat(getTotalResourceCount(resourceTracker.getMissingResources().get(jobId)), is(1));
+		}
+	}
+
+	/**
+	 * Tests that a slot request which can be fulfilled will trigger a slot allocation.
+	 */
+	@Test
+	public void testSlotRequestWithFreeSlot() throws Exception {
+		final ResourceManagerId resourceManagerId = ResourceManagerId.generate();
+		final ResourceID resourceID = ResourceID.generate();
+		final JobID jobId = new JobID();
+		final SlotID slotId = new SlotID(resourceID, 0);
+		final String targetAddress = "localhost";
+		final ResourceProfile resourceProfile = ResourceProfile.fromResources(42.0, 1337);
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec(resourceManagerId, new TestingResourceActionsBuilder().build())) {
+
+			final CompletableFuture<Tuple6<SlotID, JobID, AllocationID, ResourceProfile, String, ResourceManagerId>> requestFuture = new CompletableFuture<>();
+			// accept an incoming slot request
+			final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder()
+				.setRequestSlotFunction(tuple6 -> {
+					requestFuture.complete(Tuple6.of(tuple6.f0, tuple6.f1, tuple6.f2, tuple6.f3, tuple6.f4, tuple6.f5));
+					return CompletableFuture.completedFuture(Acknowledge.get());
+				})
+				.createTestingTaskExecutorGateway();
+
+			final TaskExecutorConnection taskExecutorConnection = new TaskExecutorConnection(resourceID, taskExecutorGateway);
+
+			final SlotStatus slotStatus = new SlotStatus(slotId, resourceProfile);
+			final SlotReport slotReport = new SlotReport(slotStatus);
+
+			slotManager.registerTaskManager(
+				taskExecutorConnection,
+				slotReport);
+
+			ResourceRequirements requirements = ResourceRequirements.create(
+				jobId,
+				targetAddress,
+				Collections.singleton(ResourceRequirement.create(resourceProfile, 1)));
+			slotManager.processResourceRequirements(requirements);
+
+			assertThat(requestFuture.get(), is(equalTo(Tuple6.of(slotId, jobId, requestFuture.get().f2, resourceProfile, targetAddress, resourceManagerId))));
+
+			DeclarativeTaskManagerSlot slot = slotTracker.getSlot(slotId);
+
+			assertEquals("The slot has not been allocated to the expected allocation id.", jobId, slot.getJobId());
+		}
+	}
+
+	/**
+	 * Checks that reducing resource requirements while a slot allocation is in-progress results in the slot being freed
+	 * one the allocation completes.
+	 */
+	@Test
+	@Ignore
+	public void testResourceRequirementReductionDuringAllocation() throws Exception {
+		final TaskExecutorConnection taskManagerConnection = createTaskExecutorConnection();
+		final ResourceID resourceID = taskManagerConnection.getResourceID();
+		final SlotID slotId = new SlotID(resourceID, 0);
+
+		final SlotReport slotReport = new SlotReport(createFreeSlotStatus(slotId));
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport);
+
+			DeclarativeTaskManagerSlot slot = slotTracker.getSlot(slotId);
+
+			final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot();
+			slotManager.processResourceRequirements(resourceRequirements);
+
+			assertSame(SlotState.PENDING, slot.getState());
+
+			slotManager.processResourceRequirements(ResourceRequirements.create(resourceRequirements.getJobId(), "foobar", Collections.emptyList()));
+
+			slot = slotTracker.getSlot(slotId);
+
+			assertSame(SlotState.FREE, slot.getState());
+		}
+	}
+
+	/**
+	 * Tests that pending slot requests are tried to be fulfilled upon new slot registrations.
+	 */
+	@Test
+	public void testFulfillingPendingSlotRequest() throws Exception {
+		final ResourceManagerId resourceManagerId = ResourceManagerId.generate();
+		final ResourceID resourceID = ResourceID.generate();
+		final JobID jobId = new JobID();
+		final SlotID slotId = new SlotID(resourceID, 0);
+		final String targetAddress = "localhost";
+		final ResourceProfile resourceProfile = ResourceProfile.fromResources(42.0, 1337);
+
+		final AtomicInteger numberAllocateResourceCalls = new AtomicInteger(0);
+		ResourceActions resourceManagerActions = new TestingResourceActionsBuilder()
+			.setAllocateResourceConsumer(ignored -> numberAllocateResourceCalls.incrementAndGet())
+			.build();
+
+		final CompletableFuture<Tuple6<SlotID, JobID, AllocationID, ResourceProfile, String, ResourceManagerId>> requestFuture = new CompletableFuture<>();
+		// accept an incoming slot request
+		final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder()
+			.setRequestSlotFunction(tuple6 -> {
+				requestFuture.complete(Tuple6.of(tuple6.f0, tuple6.f1, tuple6.f2, tuple6.f3, tuple6.f4, tuple6.f5));
+				return CompletableFuture.completedFuture(Acknowledge.get());
+			})
+			.createTestingTaskExecutorGateway();
+
+		final TaskExecutorConnection taskExecutorConnection = new TaskExecutorConnection(resourceID, taskExecutorGateway);
+
+		final SlotStatus slotStatus = new SlotStatus(slotId, resourceProfile);
+		final SlotReport slotReport = new SlotReport(slotStatus);

Review comment:
       This logic looks quite similar to the logic in `testSlotRequestWithFreeSlot `.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManagerTest.java
##########
@@ -0,0 +1,1092 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple6;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.concurrent.ManuallyTriggeredScheduledExecutor;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerId;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+import org.apache.flink.runtime.slots.ResourceRequirements;
+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.TestingTaskExecutorGateway;
+import org.apache.flink.runtime.taskexecutor.TestingTaskExecutorGatewayBuilder;
+import org.apache.flink.runtime.taskexecutor.exceptions.SlotAllocationException;
+import org.apache.flink.runtime.taskexecutor.exceptions.SlotOccupiedException;
+import org.apache.flink.runtime.testingUtils.TestingUtils;
+import org.apache.flink.util.FlinkException;
+import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.function.FunctionUtils;
+
+import akka.pattern.AskTimeoutException;
+import org.junit.Ignore;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.hamcrest.CoreMatchers.hasItem;
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.empty;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.hasSize;
+import static org.hamcrest.Matchers.is;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertSame;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Tests for the {@link DeclarativeSlotManager}.
+ */
+public class DeclarativeSlotManagerTest extends TestLogger {
+
+	private static final FlinkException TEST_EXCEPTION = new FlinkException("Test exception");
+
+	private static final WorkerResourceSpec WORKER_RESOURCE_SPEC = new WorkerResourceSpec.Builder()
+		.setCpuCores(100.0)
+		.setTaskHeapMemoryMB(10000)
+		.setTaskOffHeapMemoryMB(10000)
+		.setNetworkMemoryMB(10000)
+		.setManagedMemoryMB(10000)
+		.build();
+
+	/**
+	 * Tests that we can register task manager and their slots at the slot manager.
+	 */
+	@Test
+	public void testTaskManagerRegistration() throws Exception {
+		final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder().createTestingTaskExecutorGateway();
+		final ResourceID resourceId = ResourceID.generate();
+		final TaskExecutorConnection taskManagerConnection = new TaskExecutorConnection(resourceId, taskExecutorGateway);
+
+		final SlotID slotId1 = new SlotID(resourceId, 0);
+		final SlotID slotId2 = new SlotID(resourceId, 1);
+		final SlotReport slotReport = new SlotReport(Arrays.asList(createFreeSlotStatus(slotId1), createFreeSlotStatus(slotId2)));
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport);
+
+			assertThat("The number registered slots does not equal the expected number.", slotManager.getNumberRegisteredSlots(), is(2));
+
+			assertNotNull(slotTracker.getSlot(slotId1));
+			assertNotNull(slotTracker.getSlot(slotId2));
+		}
+	}
+
+	/**
+	 * Tests that un-registration of task managers will free and remove all registered slots.
+	 */
+	@Test
+	public void testTaskManagerUnregistration() throws Exception {
+		final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder()
+			.setRequestSlotFunction(tuple6 -> new CompletableFuture<>())
+			.createTestingTaskExecutorGateway();
+		final TaskExecutorConnection taskManagerConnection = createTaskExecutorConnection(taskExecutorGateway);
+		final ResourceID resourceId = taskManagerConnection.getResourceID();
+
+		final SlotID slotId1 = new SlotID(resourceId, 0);
+		final SlotID slotId2 = new SlotID(resourceId, 1);
+		final SlotReport slotReport = new SlotReport(Arrays.asList(createAllocatedSlotStatus(slotId1), createFreeSlotStatus(slotId2)));
+
+		final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot();
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport);
+
+			assertEquals("The number registered slots does not equal the expected number.", 2, slotManager.getNumberRegisteredSlots());
+
+			DeclarativeTaskManagerSlot slot1 = slotTracker.getSlot(slotId1);
+			DeclarativeTaskManagerSlot slot2 = slotTracker.getSlot(slotId2);
+
+			assertSame(SlotState.ALLOCATED, slot1.getState());
+			assertSame(SlotState.FREE, slot2.getState());
+
+			slotManager.processResourceRequirements(resourceRequirements);
+
+			assertSame(SlotState.PENDING, slot2.getState());
+
+			slotManager.unregisterTaskManager(taskManagerConnection.getInstanceID(), TEST_EXCEPTION);
+
+			assertEquals(0, slotManager.getNumberRegisteredSlots());
+		}
+	}
+
+	/**
+	 * Tests that a slot request with no free slots will trigger the resource allocation.
+	 */
+	@Test
+	public void testSlotRequestWithoutFreeSlots() throws Exception {
+		final ResourceManagerId resourceManagerId = ResourceManagerId.generate();
+
+		final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot();
+
+		CompletableFuture<WorkerResourceSpec> allocateResourceFuture = new CompletableFuture<>();
+		ResourceActions resourceManagerActions = new TestingResourceActionsBuilder()
+			.setAllocateResourceConsumer(allocateResourceFuture::complete)
+			.build();
+
+		try (SlotManager slotManager = createSlotManager(resourceManagerId, resourceManagerActions)) {
+
+			slotManager.processResourceRequirements(resourceRequirements);
+
+			allocateResourceFuture.get();
+		}
+	}
+
+	/**
+	 * Tests that resources continue to be considered missing if we cannot allocate more resources.
+	 */
+	@Test
+	public void testResourceDeclarationWithResourceAllocationFailure() throws Exception {
+		final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot();
+
+		ResourceActions resourceManagerActions = new TestingResourceActionsBuilder()
+			.setAllocateResourceFunction(value -> false)
+			.build();
+
+		final ResourceTracker resourceTracker = new DefaultResourceTracker();
+
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setResourceTracker(resourceTracker)
+			.buildAndStartWithDirectExec(ResourceManagerId.generate(), resourceManagerActions)) {
+
+			slotManager.processResourceRequirements(resourceRequirements);
+
+			final JobID jobId = resourceRequirements.getJobId();
+			assertThat(getTotalResourceCount(resourceTracker.getMissingResources().get(jobId)), is(1));
+		}
+	}
+
+	/**
+	 * Tests that a slot request which can be fulfilled will trigger a slot allocation.
+	 */
+	@Test
+	public void testSlotRequestWithFreeSlot() throws Exception {
+		final ResourceManagerId resourceManagerId = ResourceManagerId.generate();
+		final ResourceID resourceID = ResourceID.generate();
+		final JobID jobId = new JobID();
+		final SlotID slotId = new SlotID(resourceID, 0);
+		final String targetAddress = "localhost";
+		final ResourceProfile resourceProfile = ResourceProfile.fromResources(42.0, 1337);
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec(resourceManagerId, new TestingResourceActionsBuilder().build())) {
+
+			final CompletableFuture<Tuple6<SlotID, JobID, AllocationID, ResourceProfile, String, ResourceManagerId>> requestFuture = new CompletableFuture<>();
+			// accept an incoming slot request
+			final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder()
+				.setRequestSlotFunction(tuple6 -> {
+					requestFuture.complete(Tuple6.of(tuple6.f0, tuple6.f1, tuple6.f2, tuple6.f3, tuple6.f4, tuple6.f5));
+					return CompletableFuture.completedFuture(Acknowledge.get());
+				})
+				.createTestingTaskExecutorGateway();
+
+			final TaskExecutorConnection taskExecutorConnection = new TaskExecutorConnection(resourceID, taskExecutorGateway);
+
+			final SlotStatus slotStatus = new SlotStatus(slotId, resourceProfile);
+			final SlotReport slotReport = new SlotReport(slotStatus);
+
+			slotManager.registerTaskManager(
+				taskExecutorConnection,
+				slotReport);
+
+			ResourceRequirements requirements = ResourceRequirements.create(
+				jobId,
+				targetAddress,
+				Collections.singleton(ResourceRequirement.create(resourceProfile, 1)));
+			slotManager.processResourceRequirements(requirements);
+
+			assertThat(requestFuture.get(), is(equalTo(Tuple6.of(slotId, jobId, requestFuture.get().f2, resourceProfile, targetAddress, resourceManagerId))));
+
+			DeclarativeTaskManagerSlot slot = slotTracker.getSlot(slotId);
+
+			assertEquals("The slot has not been allocated to the expected allocation id.", jobId, slot.getJobId());
+		}
+	}
+
+	/**
+	 * Checks that reducing resource requirements while a slot allocation is in-progress results in the slot being freed
+	 * one the allocation completes.
+	 */
+	@Test
+	@Ignore

Review comment:
       Why is this test ignored?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManager.java
##########
@@ -0,0 +1,609 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.metrics.MetricNames;
+import org.apache.flink.runtime.metrics.groups.SlotManagerMetricGroup;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerId;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.slots.ResourceCounter;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+import org.apache.flink.runtime.slots.ResourceRequirements;
+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.SlotOccupiedException;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ScheduledFuture;
+import java.util.function.BiFunction;
+import java.util.stream.Collectors;
+
+/**
+ * Implementation of {@link SlotManager} supporting declarative slot management.
+ */
+public class DeclarativeSlotManager implements SlotManager {
+	private static final Logger LOG = LoggerFactory.getLogger(DeclarativeSlotManager.class);
+
+	private final SlotTracker slotTracker;
+	private final ResourceTracker resourceTracker;
+	private final BiFunction<Executor, ResourceActions, TaskExecutorManager> taskExecutorManagerFactory;
+	private TaskExecutorManager taskExecutorManager;
+
+	/** Timeout for slot requests to the task manager. */
+	private final Time taskManagerRequestTimeout;
+	private ScheduledFuture<?> slotRequestTimeoutCheck;
+
+	private final SlotMatchingStrategy slotMatchingStrategy;
+
+	private final SlotManagerMetricGroup slotManagerMetricGroup;
+
+	private final Map<JobID, String> jobMasterTargetAddresses = new HashMap<>();
+	private final HashMap<SlotID, CompletableFuture<Acknowledge>> pendingSlotAllocationFutures;
+
+	/** ResourceManager's id. */
+	private ResourceManagerId resourceManagerId;
+
+	/** Executor for future callbacks which have to be "synchronized". */
+	private Executor mainThreadExecutor;
+
+	/** Callbacks for resource (de-)allocations. */
+	private ResourceActions resourceActions;
+
+	/** True iff the component has been started. */
+	private boolean started;
+
+	public DeclarativeSlotManager(
+			ScheduledExecutor scheduledExecutor,
+			SlotManagerConfiguration slotManagerConfiguration,
+			SlotManagerMetricGroup slotManagerMetricGroup,
+			ResourceTracker resourceTracker,
+			SlotTracker slotTracker) {
+
+		Preconditions.checkNotNull(slotManagerConfiguration);
+		this.taskManagerRequestTimeout = slotManagerConfiguration.getTaskManagerRequestTimeout();
+		this.slotManagerMetricGroup = Preconditions.checkNotNull(slotManagerMetricGroup);
+		this.resourceTracker = Preconditions.checkNotNull(resourceTracker);
+
+		pendingSlotAllocationFutures = new HashMap<>(16);
+
+		this.slotTracker = Preconditions.checkNotNull(slotTracker);
+		slotTracker.registerSlotStatusUpdateListener(createSlotStatusUpdateListener());
+
+		slotMatchingStrategy = slotManagerConfiguration.getSlotMatchingStrategy();
+
+		taskExecutorManagerFactory = (executor, resourceActions) -> new TaskExecutorManager(
+			slotManagerConfiguration.getDefaultWorkerResourceSpec(),
+			slotManagerConfiguration.getNumSlotsPerWorker(),
+			slotManagerConfiguration.getMaxSlotNum(),
+			slotManagerConfiguration.isWaitResultConsumedBeforeRelease(),
+			slotManagerConfiguration.getRedundantTaskManagerNum(),
+			slotManagerConfiguration.getTaskManagerTimeout(),
+			scheduledExecutor,
+			executor,
+			resourceActions);
+
+		resourceManagerId = null;
+		resourceActions = null;
+		mainThreadExecutor = null;
+		slotRequestTimeoutCheck = null;
+		taskExecutorManager = null;
+
+		started = false;
+	}
+
+	private SlotStatusUpdateListener createSlotStatusUpdateListener() {
+		return (taskManagerSlot, previous, current, jobId) -> {
+			if (previous == SlotState.PENDING) {
+				cancelAllocationFuture(taskManagerSlot.getSlotId());
+			}
+
+			if (current == SlotState.PENDING) {
+				resourceTracker.notifyAcquiredResource(jobId, taskManagerSlot.getResourceProfile());
+			}
+			if (current == SlotState.FREE) {
+				resourceTracker.notifyLostResource(jobId, taskManagerSlot.getResourceProfile());
+			}
+
+			if (current == SlotState.ALLOCATED) {
+				taskExecutorManager.occupySlot(taskManagerSlot.getInstanceId());
+			}
+			if (previous == SlotState.ALLOCATED && current == SlotState.FREE) {
+				taskExecutorManager.freeSlot(taskManagerSlot.getInstanceId());
+			}
+		};
+	}
+
+	private void cancelAllocationFuture(SlotID slotId) {
+		final CompletableFuture<Acknowledge> acknowledgeCompletableFuture = pendingSlotAllocationFutures.remove(slotId);
+		// the future may be null if we are just re-playing the state transitions due to a slot report
+		if (acknowledgeCompletableFuture != null) {
+			acknowledgeCompletableFuture.cancel(false);
+		}
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Component lifecycle methods
+	// ---------------------------------------------------------------------------------------------
+
+	/**
+	 * Starts the slot manager with the given leader id and resource manager actions.
+	 *
+	 * @param newResourceManagerId to use for communication with the task managers
+	 * @param newMainThreadExecutor to use to run code in the ResourceManager's main thread
+	 * @param newResourceActions to use for resource (de-)allocations
+	 */
+	@Override
+	public void start(ResourceManagerId newResourceManagerId, Executor newMainThreadExecutor, ResourceActions newResourceActions) {
+		LOG.info("Starting the slot manager.");
+
+		this.resourceManagerId = Preconditions.checkNotNull(newResourceManagerId);
+		mainThreadExecutor = Preconditions.checkNotNull(newMainThreadExecutor);
+		resourceActions = Preconditions.checkNotNull(newResourceActions);
+		taskExecutorManager = taskExecutorManagerFactory.apply(newMainThreadExecutor, newResourceActions);
+
+		started = true;
+
+		registerSlotManagerMetrics();
+	}
+
+	private void registerSlotManagerMetrics() {
+		slotManagerMetricGroup.gauge(
+			MetricNames.TASK_SLOTS_AVAILABLE,
+			() -> (long) getNumberFreeSlots());
+		slotManagerMetricGroup.gauge(
+			MetricNames.TASK_SLOTS_TOTAL,
+			() -> (long) getNumberRegisteredSlots());
+	}
+
+	/**
+	 * Suspends the component. This clears the internal state of the slot manager.
+	 */
+	@Override
+	public void suspend() {
+		LOG.info("Suspending the slot manager.");
+
+		if (slotRequestTimeoutCheck != null) {
+			slotRequestTimeoutCheck.cancel(false);
+			slotRequestTimeoutCheck = null;
+		}
+
+		resourceTracker.clear();
+		taskExecutorManager.close();
+
+		for (InstanceID registeredTaskManager : taskExecutorManager.getTaskExecutors()) {
+			unregisterTaskManager(registeredTaskManager, new SlotManagerException("The slot manager is being suspended."));
+		}
+
+		taskExecutorManager = null;
+		resourceManagerId = null;
+		resourceActions = null;
+		started = false;
+	}
+
+	/**
+	 * Closes the slot manager.
+	 *
+	 * @throws Exception if the close operation fails
+	 */
+	@Override
+	public void close() throws Exception {
+		LOG.info("Closing the slot manager.");
+
+		suspend();
+		slotManagerMetricGroup.close();
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Public API
+	// ---------------------------------------------------------------------------------------------
+
+	@Override
+	public void processResourceRequirements(ResourceRequirements resourceRequirements) {
+		checkInit();
+		LOG.debug("Received resource requirements from job {}: {}", resourceRequirements.getJobId(), resourceRequirements.getResourceRequirements());
+
+		if (resourceRequirements.getResourceRequirements().isEmpty()) {
+			jobMasterTargetAddresses.remove(resourceRequirements.getJobId());
+		} else {
+			jobMasterTargetAddresses.put(resourceRequirements.getJobId(), resourceRequirements.getTargetAddress());
+		}
+		resourceTracker.notifyResourceRequirements(resourceRequirements.getJobId(), resourceRequirements.getResourceRequirements());
+		checkResourceRequirements();
+	}
+
+	/**
+	 * 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
+	 * @return True if the task manager has not been registered before and is registered successfully; otherwise false
+	 */
+	@Override
+	public boolean registerTaskManager(final TaskExecutorConnection taskExecutorConnection, SlotReport initialSlotReport) {
+		checkInit();
+		LOG.debug("Registering task executor {} under {} at the slot manager.", taskExecutorConnection.getResourceID(), taskExecutorConnection.getInstanceID());
+
+		// we identify task managers by their instance id
+		if (taskExecutorManager.isTaskManagerRegistered(taskExecutorConnection.getInstanceID())) {
+			LOG.debug("Task executor {} was already registered.", taskExecutorConnection.getResourceID());
+			reportSlotStatus(taskExecutorConnection.getInstanceID(), initialSlotReport);
+			return false;
+		} else {
+			if (!taskExecutorManager.registerTaskManager(taskExecutorConnection, initialSlotReport)) {
+				LOG.debug("Task executor {} could not be registered.", taskExecutorConnection.getResourceID());
+				return false;
+			}
+
+			// register the new slots
+			for (SlotStatus slotStatus : initialSlotReport) {
+				slotTracker.addSlot(
+					slotStatus.getSlotID(),
+					slotStatus.getResourceProfile(),
+					taskExecutorConnection,
+					slotStatus.getJobID());
+			}
+
+			checkResourceRequirements();
+			return true;
+		}
+	}
+
+	@Override
+	public boolean unregisterTaskManager(InstanceID instanceId, Exception cause) {
+		checkInit();
+
+		LOG.debug("Unregistering task executor {} from the slot manager.", instanceId);
+
+		if (taskExecutorManager.isTaskManagerRegistered(instanceId)) {
+			slotTracker.removeSlots(taskExecutorManager.getSlotsOf(instanceId));
+			taskExecutorManager.unregisterTaskExecutor(instanceId);
+			checkResourceRequirements();
+
+			return true;
+		} else {
+			LOG.debug("There is no task executor registered with instance ID {}. Ignoring this message.", instanceId);
+
+			return false;
+		}
+	}
+
+	/**
+	 * Reports the current slot allocations for a task manager identified by the given instance id.
+	 *
+	 * @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
+	 */
+	@Override
+	public boolean reportSlotStatus(InstanceID instanceId, SlotReport slotReport) {
+		checkInit();
+
+		LOG.debug("Received slot report from instance {}: {}.", instanceId, slotReport);
+
+		if (taskExecutorManager.isTaskManagerRegistered(instanceId)) {
+			slotTracker.notifySlotStatus(slotReport);
+			checkResourceRequirements();
+			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
+	 */
+	@Override
+	public void freeSlot(SlotID slotId, AllocationID allocationId) {
+		checkInit();
+		LOG.debug("Freeing slot {}.", slotId);
+
+		slotTracker.notifyFree(slotId);
+		checkResourceRequirements();
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Requirement matching
+	// ---------------------------------------------------------------------------------------------
+
+	private void checkResourceRequirements() {

Review comment:
       We should state the computational complexity of this method and add a comment that we might need to revisit this logic in case that the matching takes too much time. An alternative approach would be to remember which pending slot was assigned to which job. This could decrease the complexity a bit.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManagerTest.java
##########
@@ -0,0 +1,1092 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple6;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.concurrent.ManuallyTriggeredScheduledExecutor;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerId;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+import org.apache.flink.runtime.slots.ResourceRequirements;
+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.TestingTaskExecutorGateway;
+import org.apache.flink.runtime.taskexecutor.TestingTaskExecutorGatewayBuilder;
+import org.apache.flink.runtime.taskexecutor.exceptions.SlotAllocationException;
+import org.apache.flink.runtime.taskexecutor.exceptions.SlotOccupiedException;
+import org.apache.flink.runtime.testingUtils.TestingUtils;
+import org.apache.flink.util.FlinkException;
+import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.function.FunctionUtils;
+
+import akka.pattern.AskTimeoutException;
+import org.junit.Ignore;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.hamcrest.CoreMatchers.hasItem;
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.empty;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.hasSize;
+import static org.hamcrest.Matchers.is;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertSame;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Tests for the {@link DeclarativeSlotManager}.
+ */
+public class DeclarativeSlotManagerTest extends TestLogger {
+
+	private static final FlinkException TEST_EXCEPTION = new FlinkException("Test exception");
+
+	private static final WorkerResourceSpec WORKER_RESOURCE_SPEC = new WorkerResourceSpec.Builder()
+		.setCpuCores(100.0)
+		.setTaskHeapMemoryMB(10000)
+		.setTaskOffHeapMemoryMB(10000)
+		.setNetworkMemoryMB(10000)
+		.setManagedMemoryMB(10000)
+		.build();
+
+	/**
+	 * Tests that we can register task manager and their slots at the slot manager.
+	 */
+	@Test
+	public void testTaskManagerRegistration() throws Exception {
+		final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder().createTestingTaskExecutorGateway();
+		final ResourceID resourceId = ResourceID.generate();
+		final TaskExecutorConnection taskManagerConnection = new TaskExecutorConnection(resourceId, taskExecutorGateway);
+
+		final SlotID slotId1 = new SlotID(resourceId, 0);
+		final SlotID slotId2 = new SlotID(resourceId, 1);
+		final SlotReport slotReport = new SlotReport(Arrays.asList(createFreeSlotStatus(slotId1), createFreeSlotStatus(slotId2)));
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport);
+
+			assertThat("The number registered slots does not equal the expected number.", slotManager.getNumberRegisteredSlots(), is(2));
+
+			assertNotNull(slotTracker.getSlot(slotId1));
+			assertNotNull(slotTracker.getSlot(slotId2));
+		}
+	}
+
+	/**
+	 * Tests that un-registration of task managers will free and remove all registered slots.
+	 */
+	@Test
+	public void testTaskManagerUnregistration() throws Exception {
+		final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder()
+			.setRequestSlotFunction(tuple6 -> new CompletableFuture<>())
+			.createTestingTaskExecutorGateway();
+		final TaskExecutorConnection taskManagerConnection = createTaskExecutorConnection(taskExecutorGateway);
+		final ResourceID resourceId = taskManagerConnection.getResourceID();
+
+		final SlotID slotId1 = new SlotID(resourceId, 0);
+		final SlotID slotId2 = new SlotID(resourceId, 1);
+		final SlotReport slotReport = new SlotReport(Arrays.asList(createAllocatedSlotStatus(slotId1), createFreeSlotStatus(slotId2)));
+
+		final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot();
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport);
+
+			assertEquals("The number registered slots does not equal the expected number.", 2, slotManager.getNumberRegisteredSlots());
+
+			DeclarativeTaskManagerSlot slot1 = slotTracker.getSlot(slotId1);
+			DeclarativeTaskManagerSlot slot2 = slotTracker.getSlot(slotId2);
+
+			assertSame(SlotState.ALLOCATED, slot1.getState());
+			assertSame(SlotState.FREE, slot2.getState());
+
+			slotManager.processResourceRequirements(resourceRequirements);
+
+			assertSame(SlotState.PENDING, slot2.getState());
+
+			slotManager.unregisterTaskManager(taskManagerConnection.getInstanceID(), TEST_EXCEPTION);
+
+			assertEquals(0, slotManager.getNumberRegisteredSlots());
+		}
+	}
+
+	/**
+	 * Tests that a slot request with no free slots will trigger the resource allocation.
+	 */
+	@Test
+	public void testSlotRequestWithoutFreeSlots() throws Exception {
+		final ResourceManagerId resourceManagerId = ResourceManagerId.generate();
+
+		final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot();
+
+		CompletableFuture<WorkerResourceSpec> allocateResourceFuture = new CompletableFuture<>();
+		ResourceActions resourceManagerActions = new TestingResourceActionsBuilder()
+			.setAllocateResourceConsumer(allocateResourceFuture::complete)
+			.build();
+
+		try (SlotManager slotManager = createSlotManager(resourceManagerId, resourceManagerActions)) {
+
+			slotManager.processResourceRequirements(resourceRequirements);
+
+			allocateResourceFuture.get();
+		}
+	}
+
+	/**
+	 * Tests that resources continue to be considered missing if we cannot allocate more resources.
+	 */
+	@Test
+	public void testResourceDeclarationWithResourceAllocationFailure() throws Exception {
+		final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot();
+
+		ResourceActions resourceManagerActions = new TestingResourceActionsBuilder()
+			.setAllocateResourceFunction(value -> false)
+			.build();
+
+		final ResourceTracker resourceTracker = new DefaultResourceTracker();
+
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setResourceTracker(resourceTracker)
+			.buildAndStartWithDirectExec(ResourceManagerId.generate(), resourceManagerActions)) {
+
+			slotManager.processResourceRequirements(resourceRequirements);
+
+			final JobID jobId = resourceRequirements.getJobId();
+			assertThat(getTotalResourceCount(resourceTracker.getMissingResources().get(jobId)), is(1));
+		}
+	}
+
+	/**
+	 * Tests that a slot request which can be fulfilled will trigger a slot allocation.
+	 */
+	@Test
+	public void testSlotRequestWithFreeSlot() throws Exception {
+		final ResourceManagerId resourceManagerId = ResourceManagerId.generate();
+		final ResourceID resourceID = ResourceID.generate();
+		final JobID jobId = new JobID();
+		final SlotID slotId = new SlotID(resourceID, 0);
+		final String targetAddress = "localhost";
+		final ResourceProfile resourceProfile = ResourceProfile.fromResources(42.0, 1337);
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec(resourceManagerId, new TestingResourceActionsBuilder().build())) {
+
+			final CompletableFuture<Tuple6<SlotID, JobID, AllocationID, ResourceProfile, String, ResourceManagerId>> requestFuture = new CompletableFuture<>();
+			// accept an incoming slot request
+			final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder()
+				.setRequestSlotFunction(tuple6 -> {
+					requestFuture.complete(Tuple6.of(tuple6.f0, tuple6.f1, tuple6.f2, tuple6.f3, tuple6.f4, tuple6.f5));
+					return CompletableFuture.completedFuture(Acknowledge.get());
+				})
+				.createTestingTaskExecutorGateway();
+
+			final TaskExecutorConnection taskExecutorConnection = new TaskExecutorConnection(resourceID, taskExecutorGateway);
+
+			final SlotStatus slotStatus = new SlotStatus(slotId, resourceProfile);
+			final SlotReport slotReport = new SlotReport(slotStatus);
+
+			slotManager.registerTaskManager(
+				taskExecutorConnection,
+				slotReport);
+
+			ResourceRequirements requirements = ResourceRequirements.create(
+				jobId,
+				targetAddress,
+				Collections.singleton(ResourceRequirement.create(resourceProfile, 1)));
+			slotManager.processResourceRequirements(requirements);
+
+			assertThat(requestFuture.get(), is(equalTo(Tuple6.of(slotId, jobId, requestFuture.get().f2, resourceProfile, targetAddress, resourceManagerId))));
+
+			DeclarativeTaskManagerSlot slot = slotTracker.getSlot(slotId);
+
+			assertEquals("The slot has not been allocated to the expected allocation id.", jobId, slot.getJobId());
+		}
+	}
+
+	/**
+	 * Checks that reducing resource requirements while a slot allocation is in-progress results in the slot being freed
+	 * one the allocation completes.
+	 */
+	@Test
+	@Ignore
+	public void testResourceRequirementReductionDuringAllocation() throws Exception {
+		final TaskExecutorConnection taskManagerConnection = createTaskExecutorConnection();
+		final ResourceID resourceID = taskManagerConnection.getResourceID();
+		final SlotID slotId = new SlotID(resourceID, 0);
+
+		final SlotReport slotReport = new SlotReport(createFreeSlotStatus(slotId));
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport);
+
+			DeclarativeTaskManagerSlot slot = slotTracker.getSlot(slotId);
+
+			final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot();
+			slotManager.processResourceRequirements(resourceRequirements);
+
+			assertSame(SlotState.PENDING, slot.getState());
+
+			slotManager.processResourceRequirements(ResourceRequirements.create(resourceRequirements.getJobId(), "foobar", Collections.emptyList()));
+
+			slot = slotTracker.getSlot(slotId);
+
+			assertSame(SlotState.FREE, slot.getState());
+		}
+	}
+
+	/**
+	 * Tests that pending slot requests are tried to be fulfilled upon new slot registrations.
+	 */
+	@Test
+	public void testFulfillingPendingSlotRequest() throws Exception {
+		final ResourceManagerId resourceManagerId = ResourceManagerId.generate();
+		final ResourceID resourceID = ResourceID.generate();
+		final JobID jobId = new JobID();
+		final SlotID slotId = new SlotID(resourceID, 0);
+		final String targetAddress = "localhost";
+		final ResourceProfile resourceProfile = ResourceProfile.fromResources(42.0, 1337);
+
+		final AtomicInteger numberAllocateResourceCalls = new AtomicInteger(0);
+		ResourceActions resourceManagerActions = new TestingResourceActionsBuilder()
+			.setAllocateResourceConsumer(ignored -> numberAllocateResourceCalls.incrementAndGet())
+			.build();
+
+		final CompletableFuture<Tuple6<SlotID, JobID, AllocationID, ResourceProfile, String, ResourceManagerId>> requestFuture = new CompletableFuture<>();
+		// accept an incoming slot request
+		final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder()
+			.setRequestSlotFunction(tuple6 -> {
+				requestFuture.complete(Tuple6.of(tuple6.f0, tuple6.f1, tuple6.f2, tuple6.f3, tuple6.f4, tuple6.f5));
+				return CompletableFuture.completedFuture(Acknowledge.get());
+			})
+			.createTestingTaskExecutorGateway();
+
+		final TaskExecutorConnection taskExecutorConnection = new TaskExecutorConnection(resourceID, taskExecutorGateway);
+
+		final SlotStatus slotStatus = new SlotStatus(slotId, resourceProfile);
+		final SlotReport slotReport = new SlotReport(slotStatus);
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec(resourceManagerId, resourceManagerActions)) {
+
+			final ResourceRequirements resourceRequirements = ResourceRequirements.create(
+				jobId,
+				targetAddress,
+				Collections.singleton(ResourceRequirement.create(resourceProfile, 1)));
+			slotManager.processResourceRequirements(resourceRequirements);
+
+			assertThat(numberAllocateResourceCalls.get(), is(1));
+
+			slotManager.registerTaskManager(
+				taskExecutorConnection,
+				slotReport);
+
+			assertThat(requestFuture.get(), is(equalTo(Tuple6.of(slotId, jobId, requestFuture.get().f2, resourceProfile, targetAddress, resourceManagerId))));
+
+			DeclarativeTaskManagerSlot slot = slotTracker.getSlot(slotId);
+
+			assertEquals("The slot has not been allocated to the expected job id.", jobId, slot.getJobId());
+		}
+	}
+
+	/**
+	 * Tests that freeing a slot will correctly reset the slot and mark it as a free slot.
+	 */
+	@Test
+	public void testFreeSlot() throws Exception {
+		final TaskExecutorConnection taskExecutorConnection = createTaskExecutorConnection();
+		final ResourceID resourceID = taskExecutorConnection.getResourceID();
+		final SlotID slotId = new SlotID(resourceID, 0);
+
+		final SlotReport slotReport = new SlotReport(createAllocatedSlotStatus(slotId));
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			slotManager.registerTaskManager(
+				taskExecutorConnection,
+				slotReport);
+
+			DeclarativeTaskManagerSlot slot = slotTracker.getSlot(slotId);
+
+			assertSame(SlotState.ALLOCATED, slot.getState());
+
+			slotManager.freeSlot(slotId, new AllocationID());
+
+			assertSame(SlotState.FREE, slot.getState());
+
+			assertEquals(1, slotManager.getNumberFreeSlots());
+		}
+	}
+
+	/**
+	 * Tests that duplicate resource requirement declaration do not result in additional slots being allocated after a
+	 * pending slot request has been fulfilled but not yet freed.
+	 */
+	@Test
+	public void testDuplicateResourceRequirementDeclarationAfterSuccessfulAllocation() throws Exception {
+		final ResourceManagerId resourceManagerId = ResourceManagerId.generate();
+		final AtomicInteger allocateResourceCalls = new AtomicInteger(0);
+		final ResourceActions resourceManagerActions = new TestingResourceActionsBuilder()
+			.setAllocateResourceConsumer(ignored -> allocateResourceCalls.incrementAndGet())
+			.build();
+		ResourceRequirements requirements = createResourceRequirementsForSingleSlot();
+
+		final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder().createTestingTaskExecutorGateway();
+
+		final ResourceID resourceID = ResourceID.generate();
+
+		final TaskExecutorConnection taskManagerConnection = new TaskExecutorConnection(resourceID, taskExecutorGateway);
+
+		final SlotID slotId = new SlotID(resourceID, 0);
+		final SlotReport slotReport = new SlotReport(createFreeSlotStatus(slotId));
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec(resourceManagerId, resourceManagerActions)) {
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport);
+
+			slotManager.processResourceRequirements(requirements);
+
+			DeclarativeTaskManagerSlot slot = slotTracker.getSlot(slotId);
+
+			assertThat(slot.getState(), is(SlotState.ALLOCATED));
+
+			slotManager.processResourceRequirements(requirements);
+		}
+
+		// check that we have only called the resource allocation only for the first slot request,
+		// since the second request is a duplicate
+		assertThat(allocateResourceCalls.get(), is(0));
+	}
+
+	/**
+	 * Tests that a slot allocated for one job can be allocated for another job after being freed.
+	 */
+	@Test
+	public void testSlotCanBeAllocatedForDifferentJobAfterFree() throws Exception {
+		final AllocationID allocationId = new AllocationID();
+		final ResourceRequirements resourceRequirements1 = createResourceRequirementsForSingleSlot();
+		final ResourceRequirements resourceRequirements2 = createResourceRequirementsForSingleSlot();
+
+		final TaskExecutorConnection taskManagerConnection = createTaskExecutorConnection();
+		final ResourceID resourceID = taskManagerConnection.getResourceID();
+
+		final SlotID slotId = new SlotID(resourceID, 0);
+		final SlotReport slotReport = new SlotReport(createFreeSlotStatus(slotId));
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport);
+
+			slotManager.processResourceRequirements(resourceRequirements1);
+
+			DeclarativeTaskManagerSlot slot = slotTracker.getSlot(slotId);
+
+			assertEquals("The slot has not been allocated to the expected job id.", resourceRequirements1.getJobId(), slot.getJobId());
+
+			// clear resource requirements, so that the slot isn't immediately re-assigned
+			slotManager.processResourceRequirements(ResourceRequirements.create(resourceRequirements1.getJobId(), resourceRequirements1.getTargetAddress(), Collections.emptyList()));
+			slotManager.freeSlot(slotId, allocationId);
+
+			// check that the slot has been freed
+			assertSame(SlotState.FREE, slot.getState());
+
+			slotManager.processResourceRequirements(resourceRequirements2);
+
+			assertEquals("The slot has not been allocated to the expected job id.", resourceRequirements2.getJobId(), slot.getJobId());
+		}
+	}
+
+	/**
+	 * Tests that the slot manager ignores slot reports of unknown origin (not registered
+	 * task managers).
+	 */
+	@Test
+	public void testReceivingUnknownSlotReport() throws Exception {
+		final ResourceManagerId resourceManagerId = ResourceManagerId.generate();
+		final ResourceActions resourceManagerActions = new TestingResourceActionsBuilder().build();
+
+		final InstanceID unknownInstanceID = new InstanceID();
+		final SlotID unknownSlotId = new SlotID(ResourceID.generate(), 0);
+		final SlotReport unknownSlotReport = new SlotReport(createFreeSlotStatus(unknownSlotId));
+
+		try (SlotManager slotManager = createSlotManager(resourceManagerId, resourceManagerActions)) {
+			// check that we don't have any slots registered
+			assertThat(slotManager.getNumberRegisteredSlots(), is(0));
+
+			// this should not update anything since the instance id is not known to the slot manager
+			assertFalse(slotManager.reportSlotStatus(unknownInstanceID, unknownSlotReport));
+
+			assertThat(slotManager.getNumberRegisteredSlots(), is(0));
+		}
+	}
+
+	/**
+	 * Tests that slots are updated with respect to the latest incoming slot report. This means that
+	 * slots for which a report was received are updated accordingly.
+	 */
+	@Test
+	public void testUpdateSlotReport() throws Exception {
+		final TaskExecutorConnection taskManagerConnection = createTaskExecutorConnection();
+		final ResourceID resourceId = taskManagerConnection.getResourceID();
+
+		final SlotID slotId1 = new SlotID(resourceId, 0);
+		final SlotID slotId2 = new SlotID(resourceId, 1);
+
+		final SlotStatus slotStatus1 = createFreeSlotStatus(slotId1);
+		final SlotStatus slotStatus2 = createFreeSlotStatus(slotId2);
+
+		final SlotStatus newSlotStatus2 = createAllocatedSlotStatus(slotId2);
+		final JobID jobId = newSlotStatus2.getJobID();
+
+		final SlotReport slotReport1 = new SlotReport(Arrays.asList(slotStatus1, slotStatus2));
+		final SlotReport slotReport2 = new SlotReport(Arrays.asList(newSlotStatus2, slotStatus1));
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			// check that we don't have any slots registered
+			assertEquals(0, slotManager.getNumberRegisteredSlots());
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport1);
+
+			DeclarativeTaskManagerSlot slot1 = slotTracker.getSlot(slotId1);
+			DeclarativeTaskManagerSlot slot2 = slotTracker.getSlot(slotId2);
+
+			assertEquals(2, slotManager.getNumberRegisteredSlots());
+
+			assertSame(SlotState.FREE, slot1.getState());
+			assertSame(SlotState.FREE, slot2.getState());
+
+			assertTrue(slotManager.reportSlotStatus(taskManagerConnection.getInstanceID(), slotReport2));
+
+			assertEquals(2, slotManager.getNumberRegisteredSlots());
+
+			assertNotNull(slotTracker.getSlot(slotId1));
+			assertNotNull(slotTracker.getSlot(slotId2));
+
+			// slotId2 should have been allocated for jobiD
+			assertEquals(jobId, slotTracker.getSlot(slotId2).getJobId());
+		}
+	}
+
+	/**
+	 * Tests that if a slot allocation times out we try to allocate another slot.
+	 */
+	@Test
+	public void testSlotAllocationTimeout() throws Exception {
+		final CompletableFuture<Void> secondSlotRequestFuture = new CompletableFuture<>();
+		final AtomicInteger slotRequestsCount = new AtomicInteger();
+		final TaskExecutorConnection taskManagerConnection = createTaskExecutorConnection(new TestingTaskExecutorGatewayBuilder()
+			.setRequestSlotFunction(ignored -> {
+				if (slotRequestsCount.getAndAdd(1) == 1) {
+					secondSlotRequestFuture.complete(null);
+				} else {
+					// mimic RPC timeout
+					return FutureUtils.completedExceptionally(new AskTimeoutException("timeout"));
+				}
+				return new CompletableFuture<>();
+			})
+			.createTestingTaskExecutorGateway());
+		final SlotReport slotReport = createSlotReport(taskManagerConnection.getResourceID(), 2);
+
+		final Executor mainThreadExecutor = TestingUtils.defaultExecutor();
+
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.build()) {
+
+			slotManager.start(ResourceManagerId.generate(), mainThreadExecutor, new TestingResourceActionsBuilder().build());
+
+			CompletableFuture.runAsync(() -> slotManager.registerTaskManager(taskManagerConnection, slotReport), mainThreadExecutor)
+				.thenRun(() -> slotManager.processResourceRequirements(createResourceRequirementsForSingleSlot()))
+				.get(5, TimeUnit.SECONDS);
+
+			// a second request is only sent if the first request timed out
+			secondSlotRequestFuture.get();
+		}
+	}
+
+	/**
+	 * Tests that a slot allocation is retried if it times out on the task manager side.
+	 */
+	@Test
+	public void testTaskManagerSlotAllocationTimeoutHandling() throws Exception {
+		final JobID jobId = new JobID();
+		final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot(jobId);
+		final CompletableFuture<Acknowledge> slotRequestFuture1 = new CompletableFuture<>();
+		final CompletableFuture<Acknowledge> slotRequestFuture2 = new CompletableFuture<>();
+		final Iterator<CompletableFuture<Acknowledge>> slotRequestFutureIterator = Arrays.asList(slotRequestFuture1, slotRequestFuture2).iterator();
+		final ArrayBlockingQueue<SlotID> slotIds = new ArrayBlockingQueue<>(2);
+
+		final TestingTaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder()
+			.setRequestSlotFunction(FunctionUtils.uncheckedFunction(
+				requestSlotParameters -> {
+					slotIds.put(requestSlotParameters.f0);
+					return slotRequestFutureIterator.next();
+				}))
+			.createTestingTaskExecutorGateway();
+
+		final ResourceID resourceId = ResourceID.generate();
+		final TaskExecutorConnection taskManagerConnection = new TaskExecutorConnection(resourceId, taskExecutorGateway);
+
+		final SlotID slotId1 = new SlotID(resourceId, 0);
+		final SlotID slotId2 = new SlotID(resourceId, 1);
+		final SlotReport slotReport = new SlotReport(Arrays.asList(createFreeSlotStatus(slotId1), createFreeSlotStatus(slotId2)));
+
+		final ResourceTracker resourceTracker = new DefaultResourceTracker();
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setResourceTracker(resourceTracker)
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport);
+
+			slotManager.processResourceRequirements(resourceRequirements);
+
+			final SlotID firstSlotId = slotIds.take();
+			assertThat(slotIds, is(empty()));
+
+			DeclarativeTaskManagerSlot failedSlot = slotTracker.getSlot(firstSlotId);
+
+			// let the first attempt fail --> this should trigger a second attempt
+			slotRequestFuture1.completeExceptionally(new SlotAllocationException("Test exception."));
+
+			assertThat(getTotalResourceCount(resourceTracker.getAcquiredResources(jobId)), is(1));
+
+			// the second attempt succeeds
+			slotRequestFuture2.complete(Acknowledge.get());
+
+			final SlotID secondSlotId = slotIds.take();
+			assertThat(slotIds, is(empty()));
+
+			DeclarativeTaskManagerSlot slot = slotTracker.getSlot(secondSlotId);
+
+			assertThat(slot.getState(), is(SlotState.ALLOCATED));
+			assertEquals(jobId, slot.getJobId());
+
+			if (!failedSlot.getSlotId().equals(slot.getSlotId())) {
+				assertThat(failedSlot.getState(), is(SlotState.FREE));
+			}
+		}
+	}
+
+	/**
+	 * Tests that a pending slot allocation is cancelled if a slot report indicates that the slot is already allocated
+	 * by another job.
+	 */
+	@Test
+	public void testSlotReportWithConflictingJobIdDuringSlotAllocation() throws Exception {
+		final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot();
+		final ArrayBlockingQueue<SlotID> requestedSlotIds = new ArrayBlockingQueue<>(2);
+
+		final TestingTaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder()
+			.setRequestSlotFunction(FunctionUtils.uncheckedFunction(
+				requestSlotParameters -> {
+					requestedSlotIds.put(requestSlotParameters.f0);
+					return new CompletableFuture<>();
+				}))
+			.createTestingTaskExecutorGateway();
+
+		final TaskExecutorConnection taskExecutorConnection = createTaskExecutorConnection(taskExecutorGateway);
+		final ResourceID resourceId = taskExecutorConnection.getResourceID();
+
+		final SlotID slotId1 = new SlotID(resourceId, 0);
+		final SlotID slotId2 = new SlotID(resourceId, 1);
+		final SlotReport slotReport = new SlotReport(Arrays.asList(createFreeSlotStatus(slotId1), createFreeSlotStatus(slotId2)));
+
+		final ScheduledExecutor mainThreadExecutor = new ManuallyTriggeredScheduledExecutor();
+
+		try (final DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setScheduledExecutor(mainThreadExecutor)
+			.build()) {
+
+			slotManager.start(ResourceManagerId.generate(), mainThreadExecutor, new TestingResourceActionsBuilder().build());
+
+			slotManager.registerTaskManager(taskExecutorConnection, slotReport);
+			slotManager.processResourceRequirements(resourceRequirements);
+
+			final SlotID firstRequestedSlotId = requestedSlotIds.take();
+			final SlotID freeSlotId = firstRequestedSlotId.equals(slotId1) ? slotId2 : slotId1;
+
+			final SlotReport newSlotReport = new SlotReport(Arrays.asList(createAllocatedSlotStatus(firstRequestedSlotId), createFreeSlotStatus(freeSlotId)));
+
+			slotManager.reportSlotStatus(taskExecutorConnection.getInstanceID(), newSlotReport);
+
+			final SlotID secondRequestedSlotId = requestedSlotIds.take();
+
+			assertEquals(freeSlotId, secondRequestedSlotId);
+		}
+	}
+
+	/**
+	 * Tests that free slots which are reported as allocated won't be considered for fulfilling
+	 * other pending slot requests.
+	 *
+	 * <p>See: FLINK-8505
+	 */
+	@Test
+	public void testReportAllocatedSlot() throws Exception {
+		final ResourceID taskManagerId = ResourceID.generate();
+		final TestingTaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder().createTestingTaskExecutorGateway();
+		final TaskExecutorConnection taskExecutorConnection = new TaskExecutorConnection(taskManagerId, taskExecutorGateway);
+
+		final ResourceTracker resourceTracker = new DefaultResourceTracker();
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setResourceTracker(resourceTracker)
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			// initially report a single slot as free
+			final SlotID slotId = new SlotID(taskManagerId, 0);
+			final SlotReport initialSlotReport = new SlotReport(createFreeSlotStatus(slotId));
+
+			slotManager.registerTaskManager(taskExecutorConnection, initialSlotReport);
+
+			assertThat(slotManager.getNumberRegisteredSlots(), is(equalTo(1)));
+
+			// Now report this slot as allocated
+			final SlotStatus slotStatus = createAllocatedSlotStatus(slotId);
+			final SlotReport slotReport = new SlotReport(slotStatus);
+
+			slotManager.reportSlotStatus(
+				taskExecutorConnection.getInstanceID(),
+				slotReport);
+
+			final JobID jobId = new JobID();
+			// this resource requirement should not be fulfilled
+			ResourceRequirements requirements = createResourceRequirementsForSingleSlot(jobId);
+
+			slotManager.processResourceRequirements(requirements);
+
+			assertThat(slotTracker.getSlot(slotId).getJobId(), is(slotStatus.getJobID()));
+			assertThat(getTotalResourceCount(resourceTracker.getMissingResources().get(jobId)), is(1));
+		}
+	}
+
+	/**
+	 * Tests that the SlotManager retries allocating a slot if the TaskExecutor#requestSlot call
+	 * fails.
+	 */
+	@Test
+	public void testSlotRequestFailure() throws Exception {
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (final DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			ResourceRequirements requirements = createResourceRequirementsForSingleSlot();
+			slotManager.processResourceRequirements(requirements);
+
+			final BlockingQueue<Tuple6<SlotID, JobID, AllocationID, ResourceProfile, String, ResourceManagerId>> requestSlotQueue = new ArrayBlockingQueue<>(1);
+			final BlockingQueue<CompletableFuture<Acknowledge>> responseQueue = new ArrayBlockingQueue<>(1);
+
+			final TestingTaskExecutorGateway testingTaskExecutorGateway = new TestingTaskExecutorGatewayBuilder()
+				.setRequestSlotFunction(slotIDJobIDAllocationIDStringResourceManagerIdTuple6 -> {
+					requestSlotQueue.offer(slotIDJobIDAllocationIDStringResourceManagerIdTuple6);
+					try {
+						return responseQueue.take();
+					} catch (InterruptedException ignored) {
+						return FutureUtils.completedExceptionally(new FlinkException("Response queue was interrupted."));
+					}
+				})
+				.createTestingTaskExecutorGateway();
+
+			final ResourceID taskExecutorResourceId = ResourceID.generate();
+			final TaskExecutorConnection taskExecutionConnection = new TaskExecutorConnection(taskExecutorResourceId, testingTaskExecutorGateway);
+			final SlotReport slotReport = new SlotReport(createFreeSlotStatus(new SlotID(taskExecutorResourceId, 0)));
+
+			final CompletableFuture<Acknowledge> firstManualSlotRequestResponse = new CompletableFuture<>();
+			responseQueue.offer(firstManualSlotRequestResponse);
+
+			slotManager.registerTaskManager(taskExecutionConnection, slotReport);
+
+			final Tuple6<SlotID, JobID, AllocationID, ResourceProfile, String, ResourceManagerId> firstRequest = requestSlotQueue.take();
+
+			final CompletableFuture<Acknowledge> secondManualSlotRequestResponse = new CompletableFuture<>();
+			responseQueue.offer(secondManualSlotRequestResponse);
+
+			// fail first request
+			firstManualSlotRequestResponse.completeExceptionally(new SlotAllocationException("Test exception"));
+
+			final Tuple6<SlotID, JobID, AllocationID, ResourceProfile, String, ResourceManagerId> secondRequest = requestSlotQueue.take();
+
+			assertThat(secondRequest.f1, equalTo(firstRequest.f1));
+			assertThat(secondRequest.f0, equalTo(firstRequest.f0));
+
+			secondManualSlotRequestResponse.complete(Acknowledge.get());
+
+			final DeclarativeTaskManagerSlot slot = slotTracker.getSlot(secondRequest.f0);
+			assertThat(slot.getState(), equalTo(SlotState.ALLOCATED));
+			assertThat(slot.getJobId(), equalTo(secondRequest.f1));
+		}
+	}
+
+	/**
+	 * Tests that pending request is removed if task executor reports a slot with the same job id.
+	 */
+	@Test
+	public void testSlotRequestRemovedIfTMReportAllocation() throws Exception {
+		final ResourceTracker resourceTracker = new DefaultResourceTracker();
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+
+		try (final DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setResourceTracker(resourceTracker)
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			final JobID jobID = new JobID();
+			slotManager.processResourceRequirements(createResourceRequirementsForSingleSlot(jobID));
+
+			final BlockingQueue<Tuple6<SlotID, JobID, AllocationID, ResourceProfile, String, ResourceManagerId>> requestSlotQueue = new ArrayBlockingQueue<>(1);
+			final BlockingQueue<CompletableFuture<Acknowledge>> responseQueue = new ArrayBlockingQueue<>(1);
+
+			final TestingTaskExecutorGateway testingTaskExecutorGateway = new TestingTaskExecutorGatewayBuilder()
+				.setRequestSlotFunction(slotIDJobIDAllocationIDStringResourceManagerIdTuple6 -> {
+					requestSlotQueue.offer(slotIDJobIDAllocationIDStringResourceManagerIdTuple6);
+					try {
+						return responseQueue.take();
+					} catch (InterruptedException ignored) {
+						return FutureUtils.completedExceptionally(new FlinkException("Response queue was interrupted."));
+					}
+				})
+				.createTestingTaskExecutorGateway();
+
+			final ResourceID taskExecutorResourceId = ResourceID.generate();
+			final TaskExecutorConnection taskExecutionConnection = new TaskExecutorConnection(taskExecutorResourceId, testingTaskExecutorGateway);
+			final SlotReport slotReport = new SlotReport(createFreeSlotStatus(new SlotID(taskExecutorResourceId, 0)));
+
+			final CompletableFuture<Acknowledge> firstManualSlotRequestResponse = new CompletableFuture<>();
+			responseQueue.offer(firstManualSlotRequestResponse);
+
+			slotManager.registerTaskManager(taskExecutionConnection, slotReport);
+
+			final Tuple6<SlotID, JobID, AllocationID, ResourceProfile, String, ResourceManagerId> firstRequest = requestSlotQueue.take();
+
+			final CompletableFuture<Acknowledge> secondManualSlotRequestResponse = new CompletableFuture<>();
+			responseQueue.offer(secondManualSlotRequestResponse);

Review comment:
       Same here with the `responseQueue`.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManagerTest.java
##########
@@ -0,0 +1,1092 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple6;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.concurrent.ManuallyTriggeredScheduledExecutor;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerId;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+import org.apache.flink.runtime.slots.ResourceRequirements;
+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.TestingTaskExecutorGateway;
+import org.apache.flink.runtime.taskexecutor.TestingTaskExecutorGatewayBuilder;
+import org.apache.flink.runtime.taskexecutor.exceptions.SlotAllocationException;
+import org.apache.flink.runtime.taskexecutor.exceptions.SlotOccupiedException;
+import org.apache.flink.runtime.testingUtils.TestingUtils;
+import org.apache.flink.util.FlinkException;
+import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.function.FunctionUtils;
+
+import akka.pattern.AskTimeoutException;
+import org.junit.Ignore;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.hamcrest.CoreMatchers.hasItem;
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.empty;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.hasSize;
+import static org.hamcrest.Matchers.is;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertSame;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Tests for the {@link DeclarativeSlotManager}.
+ */
+public class DeclarativeSlotManagerTest extends TestLogger {
+
+	private static final FlinkException TEST_EXCEPTION = new FlinkException("Test exception");
+
+	private static final WorkerResourceSpec WORKER_RESOURCE_SPEC = new WorkerResourceSpec.Builder()
+		.setCpuCores(100.0)
+		.setTaskHeapMemoryMB(10000)
+		.setTaskOffHeapMemoryMB(10000)
+		.setNetworkMemoryMB(10000)
+		.setManagedMemoryMB(10000)
+		.build();
+
+	/**
+	 * Tests that we can register task manager and their slots at the slot manager.
+	 */
+	@Test
+	public void testTaskManagerRegistration() throws Exception {
+		final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder().createTestingTaskExecutorGateway();
+		final ResourceID resourceId = ResourceID.generate();
+		final TaskExecutorConnection taskManagerConnection = new TaskExecutorConnection(resourceId, taskExecutorGateway);
+
+		final SlotID slotId1 = new SlotID(resourceId, 0);
+		final SlotID slotId2 = new SlotID(resourceId, 1);
+		final SlotReport slotReport = new SlotReport(Arrays.asList(createFreeSlotStatus(slotId1), createFreeSlotStatus(slotId2)));
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport);
+
+			assertThat("The number registered slots does not equal the expected number.", slotManager.getNumberRegisteredSlots(), is(2));
+
+			assertNotNull(slotTracker.getSlot(slotId1));
+			assertNotNull(slotTracker.getSlot(slotId2));
+		}
+	}
+
+	/**
+	 * Tests that un-registration of task managers will free and remove all registered slots.
+	 */
+	@Test
+	public void testTaskManagerUnregistration() throws Exception {
+		final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder()
+			.setRequestSlotFunction(tuple6 -> new CompletableFuture<>())
+			.createTestingTaskExecutorGateway();
+		final TaskExecutorConnection taskManagerConnection = createTaskExecutorConnection(taskExecutorGateway);
+		final ResourceID resourceId = taskManagerConnection.getResourceID();
+
+		final SlotID slotId1 = new SlotID(resourceId, 0);
+		final SlotID slotId2 = new SlotID(resourceId, 1);
+		final SlotReport slotReport = new SlotReport(Arrays.asList(createAllocatedSlotStatus(slotId1), createFreeSlotStatus(slotId2)));
+
+		final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot();
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport);
+
+			assertEquals("The number registered slots does not equal the expected number.", 2, slotManager.getNumberRegisteredSlots());
+
+			DeclarativeTaskManagerSlot slot1 = slotTracker.getSlot(slotId1);
+			DeclarativeTaskManagerSlot slot2 = slotTracker.getSlot(slotId2);
+
+			assertSame(SlotState.ALLOCATED, slot1.getState());
+			assertSame(SlotState.FREE, slot2.getState());
+
+			slotManager.processResourceRequirements(resourceRequirements);
+
+			assertSame(SlotState.PENDING, slot2.getState());
+
+			slotManager.unregisterTaskManager(taskManagerConnection.getInstanceID(), TEST_EXCEPTION);
+
+			assertEquals(0, slotManager.getNumberRegisteredSlots());
+		}
+	}
+
+	/**
+	 * Tests that a slot request with no free slots will trigger the resource allocation.
+	 */
+	@Test
+	public void testSlotRequestWithoutFreeSlots() throws Exception {
+		final ResourceManagerId resourceManagerId = ResourceManagerId.generate();
+
+		final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot();
+
+		CompletableFuture<WorkerResourceSpec> allocateResourceFuture = new CompletableFuture<>();
+		ResourceActions resourceManagerActions = new TestingResourceActionsBuilder()
+			.setAllocateResourceConsumer(allocateResourceFuture::complete)
+			.build();
+
+		try (SlotManager slotManager = createSlotManager(resourceManagerId, resourceManagerActions)) {
+
+			slotManager.processResourceRequirements(resourceRequirements);
+
+			allocateResourceFuture.get();
+		}
+	}
+
+	/**
+	 * Tests that resources continue to be considered missing if we cannot allocate more resources.
+	 */
+	@Test
+	public void testResourceDeclarationWithResourceAllocationFailure() throws Exception {
+		final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot();
+
+		ResourceActions resourceManagerActions = new TestingResourceActionsBuilder()
+			.setAllocateResourceFunction(value -> false)
+			.build();
+
+		final ResourceTracker resourceTracker = new DefaultResourceTracker();
+
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setResourceTracker(resourceTracker)
+			.buildAndStartWithDirectExec(ResourceManagerId.generate(), resourceManagerActions)) {
+
+			slotManager.processResourceRequirements(resourceRequirements);
+
+			final JobID jobId = resourceRequirements.getJobId();
+			assertThat(getTotalResourceCount(resourceTracker.getMissingResources().get(jobId)), is(1));
+		}
+	}
+
+	/**
+	 * Tests that a slot request which can be fulfilled will trigger a slot allocation.
+	 */
+	@Test
+	public void testSlotRequestWithFreeSlot() throws Exception {
+		final ResourceManagerId resourceManagerId = ResourceManagerId.generate();
+		final ResourceID resourceID = ResourceID.generate();
+		final JobID jobId = new JobID();
+		final SlotID slotId = new SlotID(resourceID, 0);
+		final String targetAddress = "localhost";
+		final ResourceProfile resourceProfile = ResourceProfile.fromResources(42.0, 1337);
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec(resourceManagerId, new TestingResourceActionsBuilder().build())) {
+
+			final CompletableFuture<Tuple6<SlotID, JobID, AllocationID, ResourceProfile, String, ResourceManagerId>> requestFuture = new CompletableFuture<>();
+			// accept an incoming slot request
+			final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder()
+				.setRequestSlotFunction(tuple6 -> {
+					requestFuture.complete(Tuple6.of(tuple6.f0, tuple6.f1, tuple6.f2, tuple6.f3, tuple6.f4, tuple6.f5));
+					return CompletableFuture.completedFuture(Acknowledge.get());
+				})
+				.createTestingTaskExecutorGateway();
+
+			final TaskExecutorConnection taskExecutorConnection = new TaskExecutorConnection(resourceID, taskExecutorGateway);
+
+			final SlotStatus slotStatus = new SlotStatus(slotId, resourceProfile);
+			final SlotReport slotReport = new SlotReport(slotStatus);
+
+			slotManager.registerTaskManager(
+				taskExecutorConnection,
+				slotReport);
+
+			ResourceRequirements requirements = ResourceRequirements.create(
+				jobId,
+				targetAddress,
+				Collections.singleton(ResourceRequirement.create(resourceProfile, 1)));
+			slotManager.processResourceRequirements(requirements);
+
+			assertThat(requestFuture.get(), is(equalTo(Tuple6.of(slotId, jobId, requestFuture.get().f2, resourceProfile, targetAddress, resourceManagerId))));
+
+			DeclarativeTaskManagerSlot slot = slotTracker.getSlot(slotId);
+
+			assertEquals("The slot has not been allocated to the expected allocation id.", jobId, slot.getJobId());
+		}
+	}
+
+	/**
+	 * Checks that reducing resource requirements while a slot allocation is in-progress results in the slot being freed
+	 * one the allocation completes.
+	 */
+	@Test
+	@Ignore
+	public void testResourceRequirementReductionDuringAllocation() throws Exception {
+		final TaskExecutorConnection taskManagerConnection = createTaskExecutorConnection();
+		final ResourceID resourceID = taskManagerConnection.getResourceID();
+		final SlotID slotId = new SlotID(resourceID, 0);
+
+		final SlotReport slotReport = new SlotReport(createFreeSlotStatus(slotId));
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport);
+
+			DeclarativeTaskManagerSlot slot = slotTracker.getSlot(slotId);
+
+			final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot();
+			slotManager.processResourceRequirements(resourceRequirements);
+
+			assertSame(SlotState.PENDING, slot.getState());
+
+			slotManager.processResourceRequirements(ResourceRequirements.create(resourceRequirements.getJobId(), "foobar", Collections.emptyList()));
+
+			slot = slotTracker.getSlot(slotId);
+
+			assertSame(SlotState.FREE, slot.getState());
+		}
+	}
+
+	/**
+	 * Tests that pending slot requests are tried to be fulfilled upon new slot registrations.
+	 */
+	@Test
+	public void testFulfillingPendingSlotRequest() throws Exception {
+		final ResourceManagerId resourceManagerId = ResourceManagerId.generate();
+		final ResourceID resourceID = ResourceID.generate();
+		final JobID jobId = new JobID();
+		final SlotID slotId = new SlotID(resourceID, 0);
+		final String targetAddress = "localhost";
+		final ResourceProfile resourceProfile = ResourceProfile.fromResources(42.0, 1337);
+
+		final AtomicInteger numberAllocateResourceCalls = new AtomicInteger(0);
+		ResourceActions resourceManagerActions = new TestingResourceActionsBuilder()
+			.setAllocateResourceConsumer(ignored -> numberAllocateResourceCalls.incrementAndGet())
+			.build();
+
+		final CompletableFuture<Tuple6<SlotID, JobID, AllocationID, ResourceProfile, String, ResourceManagerId>> requestFuture = new CompletableFuture<>();
+		// accept an incoming slot request
+		final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder()
+			.setRequestSlotFunction(tuple6 -> {
+				requestFuture.complete(Tuple6.of(tuple6.f0, tuple6.f1, tuple6.f2, tuple6.f3, tuple6.f4, tuple6.f5));
+				return CompletableFuture.completedFuture(Acknowledge.get());
+			})
+			.createTestingTaskExecutorGateway();
+
+		final TaskExecutorConnection taskExecutorConnection = new TaskExecutorConnection(resourceID, taskExecutorGateway);
+
+		final SlotStatus slotStatus = new SlotStatus(slotId, resourceProfile);
+		final SlotReport slotReport = new SlotReport(slotStatus);
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec(resourceManagerId, resourceManagerActions)) {
+
+			final ResourceRequirements resourceRequirements = ResourceRequirements.create(
+				jobId,
+				targetAddress,
+				Collections.singleton(ResourceRequirement.create(resourceProfile, 1)));
+			slotManager.processResourceRequirements(resourceRequirements);
+
+			assertThat(numberAllocateResourceCalls.get(), is(1));
+
+			slotManager.registerTaskManager(
+				taskExecutorConnection,
+				slotReport);
+
+			assertThat(requestFuture.get(), is(equalTo(Tuple6.of(slotId, jobId, requestFuture.get().f2, resourceProfile, targetAddress, resourceManagerId))));
+
+			DeclarativeTaskManagerSlot slot = slotTracker.getSlot(slotId);
+
+			assertEquals("The slot has not been allocated to the expected job id.", jobId, slot.getJobId());
+		}
+	}
+
+	/**
+	 * Tests that freeing a slot will correctly reset the slot and mark it as a free slot.
+	 */
+	@Test
+	public void testFreeSlot() throws Exception {
+		final TaskExecutorConnection taskExecutorConnection = createTaskExecutorConnection();
+		final ResourceID resourceID = taskExecutorConnection.getResourceID();
+		final SlotID slotId = new SlotID(resourceID, 0);
+
+		final SlotReport slotReport = new SlotReport(createAllocatedSlotStatus(slotId));
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			slotManager.registerTaskManager(
+				taskExecutorConnection,
+				slotReport);
+
+			DeclarativeTaskManagerSlot slot = slotTracker.getSlot(slotId);
+
+			assertSame(SlotState.ALLOCATED, slot.getState());
+
+			slotManager.freeSlot(slotId, new AllocationID());
+
+			assertSame(SlotState.FREE, slot.getState());
+
+			assertEquals(1, slotManager.getNumberFreeSlots());
+		}
+	}
+
+	/**
+	 * Tests that duplicate resource requirement declaration do not result in additional slots being allocated after a
+	 * pending slot request has been fulfilled but not yet freed.
+	 */
+	@Test
+	public void testDuplicateResourceRequirementDeclarationAfterSuccessfulAllocation() throws Exception {
+		final ResourceManagerId resourceManagerId = ResourceManagerId.generate();
+		final AtomicInteger allocateResourceCalls = new AtomicInteger(0);
+		final ResourceActions resourceManagerActions = new TestingResourceActionsBuilder()
+			.setAllocateResourceConsumer(ignored -> allocateResourceCalls.incrementAndGet())
+			.build();
+		ResourceRequirements requirements = createResourceRequirementsForSingleSlot();
+
+		final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder().createTestingTaskExecutorGateway();
+
+		final ResourceID resourceID = ResourceID.generate();
+
+		final TaskExecutorConnection taskManagerConnection = new TaskExecutorConnection(resourceID, taskExecutorGateway);
+
+		final SlotID slotId = new SlotID(resourceID, 0);
+		final SlotReport slotReport = new SlotReport(createFreeSlotStatus(slotId));
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec(resourceManagerId, resourceManagerActions)) {
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport);
+
+			slotManager.processResourceRequirements(requirements);
+
+			DeclarativeTaskManagerSlot slot = slotTracker.getSlot(slotId);
+
+			assertThat(slot.getState(), is(SlotState.ALLOCATED));
+
+			slotManager.processResourceRequirements(requirements);
+		}
+
+		// check that we have only called the resource allocation only for the first slot request,
+		// since the second request is a duplicate
+		assertThat(allocateResourceCalls.get(), is(0));
+	}
+
+	/**
+	 * Tests that a slot allocated for one job can be allocated for another job after being freed.
+	 */
+	@Test
+	public void testSlotCanBeAllocatedForDifferentJobAfterFree() throws Exception {
+		final AllocationID allocationId = new AllocationID();
+		final ResourceRequirements resourceRequirements1 = createResourceRequirementsForSingleSlot();
+		final ResourceRequirements resourceRequirements2 = createResourceRequirementsForSingleSlot();
+
+		final TaskExecutorConnection taskManagerConnection = createTaskExecutorConnection();
+		final ResourceID resourceID = taskManagerConnection.getResourceID();
+
+		final SlotID slotId = new SlotID(resourceID, 0);
+		final SlotReport slotReport = new SlotReport(createFreeSlotStatus(slotId));
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport);
+
+			slotManager.processResourceRequirements(resourceRequirements1);
+
+			DeclarativeTaskManagerSlot slot = slotTracker.getSlot(slotId);
+
+			assertEquals("The slot has not been allocated to the expected job id.", resourceRequirements1.getJobId(), slot.getJobId());
+
+			// clear resource requirements, so that the slot isn't immediately re-assigned
+			slotManager.processResourceRequirements(ResourceRequirements.create(resourceRequirements1.getJobId(), resourceRequirements1.getTargetAddress(), Collections.emptyList()));
+			slotManager.freeSlot(slotId, allocationId);
+
+			// check that the slot has been freed
+			assertSame(SlotState.FREE, slot.getState());
+
+			slotManager.processResourceRequirements(resourceRequirements2);
+
+			assertEquals("The slot has not been allocated to the expected job id.", resourceRequirements2.getJobId(), slot.getJobId());
+		}
+	}
+
+	/**
+	 * Tests that the slot manager ignores slot reports of unknown origin (not registered
+	 * task managers).
+	 */
+	@Test
+	public void testReceivingUnknownSlotReport() throws Exception {
+		final ResourceManagerId resourceManagerId = ResourceManagerId.generate();
+		final ResourceActions resourceManagerActions = new TestingResourceActionsBuilder().build();
+
+		final InstanceID unknownInstanceID = new InstanceID();
+		final SlotID unknownSlotId = new SlotID(ResourceID.generate(), 0);
+		final SlotReport unknownSlotReport = new SlotReport(createFreeSlotStatus(unknownSlotId));
+
+		try (SlotManager slotManager = createSlotManager(resourceManagerId, resourceManagerActions)) {
+			// check that we don't have any slots registered
+			assertThat(slotManager.getNumberRegisteredSlots(), is(0));
+
+			// this should not update anything since the instance id is not known to the slot manager
+			assertFalse(slotManager.reportSlotStatus(unknownInstanceID, unknownSlotReport));
+
+			assertThat(slotManager.getNumberRegisteredSlots(), is(0));
+		}
+	}
+
+	/**
+	 * Tests that slots are updated with respect to the latest incoming slot report. This means that
+	 * slots for which a report was received are updated accordingly.
+	 */
+	@Test
+	public void testUpdateSlotReport() throws Exception {
+		final TaskExecutorConnection taskManagerConnection = createTaskExecutorConnection();
+		final ResourceID resourceId = taskManagerConnection.getResourceID();
+
+		final SlotID slotId1 = new SlotID(resourceId, 0);
+		final SlotID slotId2 = new SlotID(resourceId, 1);
+
+		final SlotStatus slotStatus1 = createFreeSlotStatus(slotId1);
+		final SlotStatus slotStatus2 = createFreeSlotStatus(slotId2);
+
+		final SlotStatus newSlotStatus2 = createAllocatedSlotStatus(slotId2);
+		final JobID jobId = newSlotStatus2.getJobID();
+
+		final SlotReport slotReport1 = new SlotReport(Arrays.asList(slotStatus1, slotStatus2));
+		final SlotReport slotReport2 = new SlotReport(Arrays.asList(newSlotStatus2, slotStatus1));
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			// check that we don't have any slots registered
+			assertEquals(0, slotManager.getNumberRegisteredSlots());
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport1);
+
+			DeclarativeTaskManagerSlot slot1 = slotTracker.getSlot(slotId1);
+			DeclarativeTaskManagerSlot slot2 = slotTracker.getSlot(slotId2);
+
+			assertEquals(2, slotManager.getNumberRegisteredSlots());
+
+			assertSame(SlotState.FREE, slot1.getState());
+			assertSame(SlotState.FREE, slot2.getState());
+
+			assertTrue(slotManager.reportSlotStatus(taskManagerConnection.getInstanceID(), slotReport2));
+
+			assertEquals(2, slotManager.getNumberRegisteredSlots());
+
+			assertNotNull(slotTracker.getSlot(slotId1));
+			assertNotNull(slotTracker.getSlot(slotId2));
+
+			// slotId2 should have been allocated for jobiD
+			assertEquals(jobId, slotTracker.getSlot(slotId2).getJobId());

Review comment:
       and `slot1` should still be free, right?

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManagerTest.java
##########
@@ -0,0 +1,1092 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple6;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.concurrent.ManuallyTriggeredScheduledExecutor;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerId;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+import org.apache.flink.runtime.slots.ResourceRequirements;
+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.TestingTaskExecutorGateway;
+import org.apache.flink.runtime.taskexecutor.TestingTaskExecutorGatewayBuilder;
+import org.apache.flink.runtime.taskexecutor.exceptions.SlotAllocationException;
+import org.apache.flink.runtime.taskexecutor.exceptions.SlotOccupiedException;
+import org.apache.flink.runtime.testingUtils.TestingUtils;
+import org.apache.flink.util.FlinkException;
+import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.function.FunctionUtils;
+
+import akka.pattern.AskTimeoutException;
+import org.junit.Ignore;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.hamcrest.CoreMatchers.hasItem;
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.empty;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.hasSize;
+import static org.hamcrest.Matchers.is;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertSame;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Tests for the {@link DeclarativeSlotManager}.
+ */
+public class DeclarativeSlotManagerTest extends TestLogger {
+
+	private static final FlinkException TEST_EXCEPTION = new FlinkException("Test exception");
+
+	private static final WorkerResourceSpec WORKER_RESOURCE_SPEC = new WorkerResourceSpec.Builder()
+		.setCpuCores(100.0)
+		.setTaskHeapMemoryMB(10000)
+		.setTaskOffHeapMemoryMB(10000)
+		.setNetworkMemoryMB(10000)
+		.setManagedMemoryMB(10000)
+		.build();
+
+	/**
+	 * Tests that we can register task manager and their slots at the slot manager.
+	 */
+	@Test
+	public void testTaskManagerRegistration() throws Exception {
+		final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder().createTestingTaskExecutorGateway();
+		final ResourceID resourceId = ResourceID.generate();
+		final TaskExecutorConnection taskManagerConnection = new TaskExecutorConnection(resourceId, taskExecutorGateway);
+
+		final SlotID slotId1 = new SlotID(resourceId, 0);
+		final SlotID slotId2 = new SlotID(resourceId, 1);
+		final SlotReport slotReport = new SlotReport(Arrays.asList(createFreeSlotStatus(slotId1), createFreeSlotStatus(slotId2)));
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport);
+
+			assertThat("The number registered slots does not equal the expected number.", slotManager.getNumberRegisteredSlots(), is(2));
+
+			assertNotNull(slotTracker.getSlot(slotId1));
+			assertNotNull(slotTracker.getSlot(slotId2));
+		}
+	}
+
+	/**
+	 * Tests that un-registration of task managers will free and remove all registered slots.
+	 */
+	@Test
+	public void testTaskManagerUnregistration() throws Exception {
+		final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder()
+			.setRequestSlotFunction(tuple6 -> new CompletableFuture<>())
+			.createTestingTaskExecutorGateway();
+		final TaskExecutorConnection taskManagerConnection = createTaskExecutorConnection(taskExecutorGateway);
+		final ResourceID resourceId = taskManagerConnection.getResourceID();
+
+		final SlotID slotId1 = new SlotID(resourceId, 0);
+		final SlotID slotId2 = new SlotID(resourceId, 1);
+		final SlotReport slotReport = new SlotReport(Arrays.asList(createAllocatedSlotStatus(slotId1), createFreeSlotStatus(slotId2)));
+
+		final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot();
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport);
+
+			assertEquals("The number registered slots does not equal the expected number.", 2, slotManager.getNumberRegisteredSlots());
+
+			DeclarativeTaskManagerSlot slot1 = slotTracker.getSlot(slotId1);
+			DeclarativeTaskManagerSlot slot2 = slotTracker.getSlot(slotId2);
+
+			assertSame(SlotState.ALLOCATED, slot1.getState());
+			assertSame(SlotState.FREE, slot2.getState());
+
+			slotManager.processResourceRequirements(resourceRequirements);
+
+			assertSame(SlotState.PENDING, slot2.getState());
+
+			slotManager.unregisterTaskManager(taskManagerConnection.getInstanceID(), TEST_EXCEPTION);
+
+			assertEquals(0, slotManager.getNumberRegisteredSlots());
+		}
+	}
+
+	/**
+	 * Tests that a slot request with no free slots will trigger the resource allocation.
+	 */
+	@Test
+	public void testSlotRequestWithoutFreeSlots() throws Exception {
+		final ResourceManagerId resourceManagerId = ResourceManagerId.generate();
+
+		final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot();
+
+		CompletableFuture<WorkerResourceSpec> allocateResourceFuture = new CompletableFuture<>();
+		ResourceActions resourceManagerActions = new TestingResourceActionsBuilder()
+			.setAllocateResourceConsumer(allocateResourceFuture::complete)
+			.build();
+
+		try (SlotManager slotManager = createSlotManager(resourceManagerId, resourceManagerActions)) {
+
+			slotManager.processResourceRequirements(resourceRequirements);
+
+			allocateResourceFuture.get();
+		}
+	}
+
+	/**
+	 * Tests that resources continue to be considered missing if we cannot allocate more resources.
+	 */
+	@Test
+	public void testResourceDeclarationWithResourceAllocationFailure() throws Exception {
+		final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot();
+
+		ResourceActions resourceManagerActions = new TestingResourceActionsBuilder()
+			.setAllocateResourceFunction(value -> false)
+			.build();
+
+		final ResourceTracker resourceTracker = new DefaultResourceTracker();
+
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setResourceTracker(resourceTracker)
+			.buildAndStartWithDirectExec(ResourceManagerId.generate(), resourceManagerActions)) {
+
+			slotManager.processResourceRequirements(resourceRequirements);
+
+			final JobID jobId = resourceRequirements.getJobId();
+			assertThat(getTotalResourceCount(resourceTracker.getMissingResources().get(jobId)), is(1));
+		}
+	}
+
+	/**
+	 * Tests that a slot request which can be fulfilled will trigger a slot allocation.
+	 */
+	@Test
+	public void testSlotRequestWithFreeSlot() throws Exception {
+		final ResourceManagerId resourceManagerId = ResourceManagerId.generate();
+		final ResourceID resourceID = ResourceID.generate();
+		final JobID jobId = new JobID();
+		final SlotID slotId = new SlotID(resourceID, 0);
+		final String targetAddress = "localhost";
+		final ResourceProfile resourceProfile = ResourceProfile.fromResources(42.0, 1337);
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec(resourceManagerId, new TestingResourceActionsBuilder().build())) {
+
+			final CompletableFuture<Tuple6<SlotID, JobID, AllocationID, ResourceProfile, String, ResourceManagerId>> requestFuture = new CompletableFuture<>();
+			// accept an incoming slot request
+			final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder()
+				.setRequestSlotFunction(tuple6 -> {
+					requestFuture.complete(Tuple6.of(tuple6.f0, tuple6.f1, tuple6.f2, tuple6.f3, tuple6.f4, tuple6.f5));
+					return CompletableFuture.completedFuture(Acknowledge.get());
+				})
+				.createTestingTaskExecutorGateway();
+
+			final TaskExecutorConnection taskExecutorConnection = new TaskExecutorConnection(resourceID, taskExecutorGateway);
+
+			final SlotStatus slotStatus = new SlotStatus(slotId, resourceProfile);
+			final SlotReport slotReport = new SlotReport(slotStatus);
+
+			slotManager.registerTaskManager(
+				taskExecutorConnection,
+				slotReport);
+
+			ResourceRequirements requirements = ResourceRequirements.create(
+				jobId,
+				targetAddress,
+				Collections.singleton(ResourceRequirement.create(resourceProfile, 1)));
+			slotManager.processResourceRequirements(requirements);
+
+			assertThat(requestFuture.get(), is(equalTo(Tuple6.of(slotId, jobId, requestFuture.get().f2, resourceProfile, targetAddress, resourceManagerId))));
+
+			DeclarativeTaskManagerSlot slot = slotTracker.getSlot(slotId);
+
+			assertEquals("The slot has not been allocated to the expected allocation id.", jobId, slot.getJobId());
+		}
+	}
+
+	/**
+	 * Checks that reducing resource requirements while a slot allocation is in-progress results in the slot being freed
+	 * one the allocation completes.
+	 */
+	@Test
+	@Ignore
+	public void testResourceRequirementReductionDuringAllocation() throws Exception {
+		final TaskExecutorConnection taskManagerConnection = createTaskExecutorConnection();
+		final ResourceID resourceID = taskManagerConnection.getResourceID();
+		final SlotID slotId = new SlotID(resourceID, 0);
+
+		final SlotReport slotReport = new SlotReport(createFreeSlotStatus(slotId));
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport);
+
+			DeclarativeTaskManagerSlot slot = slotTracker.getSlot(slotId);
+
+			final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot();
+			slotManager.processResourceRequirements(resourceRequirements);
+
+			assertSame(SlotState.PENDING, slot.getState());
+
+			slotManager.processResourceRequirements(ResourceRequirements.create(resourceRequirements.getJobId(), "foobar", Collections.emptyList()));
+
+			slot = slotTracker.getSlot(slotId);
+
+			assertSame(SlotState.FREE, slot.getState());
+		}
+	}
+
+	/**
+	 * Tests that pending slot requests are tried to be fulfilled upon new slot registrations.
+	 */
+	@Test
+	public void testFulfillingPendingSlotRequest() throws Exception {
+		final ResourceManagerId resourceManagerId = ResourceManagerId.generate();
+		final ResourceID resourceID = ResourceID.generate();
+		final JobID jobId = new JobID();
+		final SlotID slotId = new SlotID(resourceID, 0);
+		final String targetAddress = "localhost";
+		final ResourceProfile resourceProfile = ResourceProfile.fromResources(42.0, 1337);
+
+		final AtomicInteger numberAllocateResourceCalls = new AtomicInteger(0);
+		ResourceActions resourceManagerActions = new TestingResourceActionsBuilder()
+			.setAllocateResourceConsumer(ignored -> numberAllocateResourceCalls.incrementAndGet())
+			.build();
+
+		final CompletableFuture<Tuple6<SlotID, JobID, AllocationID, ResourceProfile, String, ResourceManagerId>> requestFuture = new CompletableFuture<>();
+		// accept an incoming slot request
+		final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder()
+			.setRequestSlotFunction(tuple6 -> {
+				requestFuture.complete(Tuple6.of(tuple6.f0, tuple6.f1, tuple6.f2, tuple6.f3, tuple6.f4, tuple6.f5));
+				return CompletableFuture.completedFuture(Acknowledge.get());
+			})
+			.createTestingTaskExecutorGateway();
+
+		final TaskExecutorConnection taskExecutorConnection = new TaskExecutorConnection(resourceID, taskExecutorGateway);
+
+		final SlotStatus slotStatus = new SlotStatus(slotId, resourceProfile);
+		final SlotReport slotReport = new SlotReport(slotStatus);
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec(resourceManagerId, resourceManagerActions)) {
+
+			final ResourceRequirements resourceRequirements = ResourceRequirements.create(
+				jobId,
+				targetAddress,
+				Collections.singleton(ResourceRequirement.create(resourceProfile, 1)));
+			slotManager.processResourceRequirements(resourceRequirements);
+
+			assertThat(numberAllocateResourceCalls.get(), is(1));
+
+			slotManager.registerTaskManager(
+				taskExecutorConnection,
+				slotReport);
+
+			assertThat(requestFuture.get(), is(equalTo(Tuple6.of(slotId, jobId, requestFuture.get().f2, resourceProfile, targetAddress, resourceManagerId))));
+
+			DeclarativeTaskManagerSlot slot = slotTracker.getSlot(slotId);
+
+			assertEquals("The slot has not been allocated to the expected job id.", jobId, slot.getJobId());
+		}
+	}
+
+	/**
+	 * Tests that freeing a slot will correctly reset the slot and mark it as a free slot.
+	 */
+	@Test
+	public void testFreeSlot() throws Exception {
+		final TaskExecutorConnection taskExecutorConnection = createTaskExecutorConnection();
+		final ResourceID resourceID = taskExecutorConnection.getResourceID();
+		final SlotID slotId = new SlotID(resourceID, 0);
+
+		final SlotReport slotReport = new SlotReport(createAllocatedSlotStatus(slotId));
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			slotManager.registerTaskManager(
+				taskExecutorConnection,
+				slotReport);
+
+			DeclarativeTaskManagerSlot slot = slotTracker.getSlot(slotId);
+
+			assertSame(SlotState.ALLOCATED, slot.getState());
+
+			slotManager.freeSlot(slotId, new AllocationID());
+
+			assertSame(SlotState.FREE, slot.getState());
+
+			assertEquals(1, slotManager.getNumberFreeSlots());
+		}
+	}
+
+	/**
+	 * Tests that duplicate resource requirement declaration do not result in additional slots being allocated after a
+	 * pending slot request has been fulfilled but not yet freed.
+	 */
+	@Test
+	public void testDuplicateResourceRequirementDeclarationAfterSuccessfulAllocation() throws Exception {
+		final ResourceManagerId resourceManagerId = ResourceManagerId.generate();
+		final AtomicInteger allocateResourceCalls = new AtomicInteger(0);
+		final ResourceActions resourceManagerActions = new TestingResourceActionsBuilder()
+			.setAllocateResourceConsumer(ignored -> allocateResourceCalls.incrementAndGet())
+			.build();
+		ResourceRequirements requirements = createResourceRequirementsForSingleSlot();
+
+		final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder().createTestingTaskExecutorGateway();
+
+		final ResourceID resourceID = ResourceID.generate();
+
+		final TaskExecutorConnection taskManagerConnection = new TaskExecutorConnection(resourceID, taskExecutorGateway);
+
+		final SlotID slotId = new SlotID(resourceID, 0);
+		final SlotReport slotReport = new SlotReport(createFreeSlotStatus(slotId));
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec(resourceManagerId, resourceManagerActions)) {
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport);
+
+			slotManager.processResourceRequirements(requirements);
+
+			DeclarativeTaskManagerSlot slot = slotTracker.getSlot(slotId);
+
+			assertThat(slot.getState(), is(SlotState.ALLOCATED));
+
+			slotManager.processResourceRequirements(requirements);
+		}
+
+		// check that we have only called the resource allocation only for the first slot request,
+		// since the second request is a duplicate
+		assertThat(allocateResourceCalls.get(), is(0));
+	}
+
+	/**
+	 * Tests that a slot allocated for one job can be allocated for another job after being freed.
+	 */
+	@Test
+	public void testSlotCanBeAllocatedForDifferentJobAfterFree() throws Exception {
+		final AllocationID allocationId = new AllocationID();
+		final ResourceRequirements resourceRequirements1 = createResourceRequirementsForSingleSlot();
+		final ResourceRequirements resourceRequirements2 = createResourceRequirementsForSingleSlot();
+
+		final TaskExecutorConnection taskManagerConnection = createTaskExecutorConnection();
+		final ResourceID resourceID = taskManagerConnection.getResourceID();
+
+		final SlotID slotId = new SlotID(resourceID, 0);
+		final SlotReport slotReport = new SlotReport(createFreeSlotStatus(slotId));
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport);
+
+			slotManager.processResourceRequirements(resourceRequirements1);
+
+			DeclarativeTaskManagerSlot slot = slotTracker.getSlot(slotId);
+
+			assertEquals("The slot has not been allocated to the expected job id.", resourceRequirements1.getJobId(), slot.getJobId());
+
+			// clear resource requirements, so that the slot isn't immediately re-assigned
+			slotManager.processResourceRequirements(ResourceRequirements.create(resourceRequirements1.getJobId(), resourceRequirements1.getTargetAddress(), Collections.emptyList()));
+			slotManager.freeSlot(slotId, allocationId);
+
+			// check that the slot has been freed
+			assertSame(SlotState.FREE, slot.getState());
+
+			slotManager.processResourceRequirements(resourceRequirements2);
+
+			assertEquals("The slot has not been allocated to the expected job id.", resourceRequirements2.getJobId(), slot.getJobId());
+		}
+	}
+
+	/**
+	 * Tests that the slot manager ignores slot reports of unknown origin (not registered
+	 * task managers).
+	 */
+	@Test
+	public void testReceivingUnknownSlotReport() throws Exception {
+		final ResourceManagerId resourceManagerId = ResourceManagerId.generate();
+		final ResourceActions resourceManagerActions = new TestingResourceActionsBuilder().build();
+
+		final InstanceID unknownInstanceID = new InstanceID();
+		final SlotID unknownSlotId = new SlotID(ResourceID.generate(), 0);
+		final SlotReport unknownSlotReport = new SlotReport(createFreeSlotStatus(unknownSlotId));
+
+		try (SlotManager slotManager = createSlotManager(resourceManagerId, resourceManagerActions)) {
+			// check that we don't have any slots registered
+			assertThat(slotManager.getNumberRegisteredSlots(), is(0));
+
+			// this should not update anything since the instance id is not known to the slot manager
+			assertFalse(slotManager.reportSlotStatus(unknownInstanceID, unknownSlotReport));
+
+			assertThat(slotManager.getNumberRegisteredSlots(), is(0));
+		}
+	}
+
+	/**
+	 * Tests that slots are updated with respect to the latest incoming slot report. This means that
+	 * slots for which a report was received are updated accordingly.
+	 */
+	@Test
+	public void testUpdateSlotReport() throws Exception {
+		final TaskExecutorConnection taskManagerConnection = createTaskExecutorConnection();
+		final ResourceID resourceId = taskManagerConnection.getResourceID();
+
+		final SlotID slotId1 = new SlotID(resourceId, 0);
+		final SlotID slotId2 = new SlotID(resourceId, 1);
+
+		final SlotStatus slotStatus1 = createFreeSlotStatus(slotId1);
+		final SlotStatus slotStatus2 = createFreeSlotStatus(slotId2);
+
+		final SlotStatus newSlotStatus2 = createAllocatedSlotStatus(slotId2);
+		final JobID jobId = newSlotStatus2.getJobID();
+
+		final SlotReport slotReport1 = new SlotReport(Arrays.asList(slotStatus1, slotStatus2));
+		final SlotReport slotReport2 = new SlotReport(Arrays.asList(newSlotStatus2, slotStatus1));
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			// check that we don't have any slots registered
+			assertEquals(0, slotManager.getNumberRegisteredSlots());
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport1);
+
+			DeclarativeTaskManagerSlot slot1 = slotTracker.getSlot(slotId1);
+			DeclarativeTaskManagerSlot slot2 = slotTracker.getSlot(slotId2);
+
+			assertEquals(2, slotManager.getNumberRegisteredSlots());
+
+			assertSame(SlotState.FREE, slot1.getState());
+			assertSame(SlotState.FREE, slot2.getState());
+
+			assertTrue(slotManager.reportSlotStatus(taskManagerConnection.getInstanceID(), slotReport2));
+
+			assertEquals(2, slotManager.getNumberRegisteredSlots());
+
+			assertNotNull(slotTracker.getSlot(slotId1));
+			assertNotNull(slotTracker.getSlot(slotId2));
+
+			// slotId2 should have been allocated for jobiD
+			assertEquals(jobId, slotTracker.getSlot(slotId2).getJobId());
+		}
+	}
+
+	/**
+	 * Tests that if a slot allocation times out we try to allocate another slot.
+	 */
+	@Test
+	public void testSlotAllocationTimeout() throws Exception {
+		final CompletableFuture<Void> secondSlotRequestFuture = new CompletableFuture<>();
+		final AtomicInteger slotRequestsCount = new AtomicInteger();
+		final TaskExecutorConnection taskManagerConnection = createTaskExecutorConnection(new TestingTaskExecutorGatewayBuilder()
+			.setRequestSlotFunction(ignored -> {
+				if (slotRequestsCount.getAndAdd(1) == 1) {
+					secondSlotRequestFuture.complete(null);
+				} else {
+					// mimic RPC timeout
+					return FutureUtils.completedExceptionally(new AskTimeoutException("timeout"));
+				}
+				return new CompletableFuture<>();
+			})
+			.createTestingTaskExecutorGateway());
+		final SlotReport slotReport = createSlotReport(taskManagerConnection.getResourceID(), 2);
+
+		final Executor mainThreadExecutor = TestingUtils.defaultExecutor();
+
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.build()) {
+
+			slotManager.start(ResourceManagerId.generate(), mainThreadExecutor, new TestingResourceActionsBuilder().build());
+
+			CompletableFuture.runAsync(() -> slotManager.registerTaskManager(taskManagerConnection, slotReport), mainThreadExecutor)
+				.thenRun(() -> slotManager.processResourceRequirements(createResourceRequirementsForSingleSlot()))
+				.get(5, TimeUnit.SECONDS);
+
+			// a second request is only sent if the first request timed out
+			secondSlotRequestFuture.get();
+		}
+	}
+
+	/**
+	 * Tests that a slot allocation is retried if it times out on the task manager side.
+	 */
+	@Test
+	public void testTaskManagerSlotAllocationTimeoutHandling() throws Exception {
+		final JobID jobId = new JobID();
+		final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot(jobId);
+		final CompletableFuture<Acknowledge> slotRequestFuture1 = new CompletableFuture<>();
+		final CompletableFuture<Acknowledge> slotRequestFuture2 = new CompletableFuture<>();
+		final Iterator<CompletableFuture<Acknowledge>> slotRequestFutureIterator = Arrays.asList(slotRequestFuture1, slotRequestFuture2).iterator();
+		final ArrayBlockingQueue<SlotID> slotIds = new ArrayBlockingQueue<>(2);
+
+		final TestingTaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder()
+			.setRequestSlotFunction(FunctionUtils.uncheckedFunction(
+				requestSlotParameters -> {
+					slotIds.put(requestSlotParameters.f0);
+					return slotRequestFutureIterator.next();
+				}))
+			.createTestingTaskExecutorGateway();
+
+		final ResourceID resourceId = ResourceID.generate();
+		final TaskExecutorConnection taskManagerConnection = new TaskExecutorConnection(resourceId, taskExecutorGateway);
+
+		final SlotID slotId1 = new SlotID(resourceId, 0);
+		final SlotID slotId2 = new SlotID(resourceId, 1);
+		final SlotReport slotReport = new SlotReport(Arrays.asList(createFreeSlotStatus(slotId1), createFreeSlotStatus(slotId2)));
+
+		final ResourceTracker resourceTracker = new DefaultResourceTracker();
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setResourceTracker(resourceTracker)
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport);
+
+			slotManager.processResourceRequirements(resourceRequirements);
+
+			final SlotID firstSlotId = slotIds.take();
+			assertThat(slotIds, is(empty()));
+
+			DeclarativeTaskManagerSlot failedSlot = slotTracker.getSlot(firstSlotId);
+
+			// let the first attempt fail --> this should trigger a second attempt
+			slotRequestFuture1.completeExceptionally(new SlotAllocationException("Test exception."));
+
+			assertThat(getTotalResourceCount(resourceTracker.getAcquiredResources(jobId)), is(1));
+
+			// the second attempt succeeds
+			slotRequestFuture2.complete(Acknowledge.get());
+
+			final SlotID secondSlotId = slotIds.take();
+			assertThat(slotIds, is(empty()));
+
+			DeclarativeTaskManagerSlot slot = slotTracker.getSlot(secondSlotId);
+
+			assertThat(slot.getState(), is(SlotState.ALLOCATED));
+			assertEquals(jobId, slot.getJobId());
+
+			if (!failedSlot.getSlotId().equals(slot.getSlotId())) {
+				assertThat(failedSlot.getState(), is(SlotState.FREE));
+			}
+		}
+	}
+
+	/**
+	 * Tests that a pending slot allocation is cancelled if a slot report indicates that the slot is already allocated
+	 * by another job.
+	 */
+	@Test
+	public void testSlotReportWithConflictingJobIdDuringSlotAllocation() throws Exception {
+		final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot();
+		final ArrayBlockingQueue<SlotID> requestedSlotIds = new ArrayBlockingQueue<>(2);
+
+		final TestingTaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder()
+			.setRequestSlotFunction(FunctionUtils.uncheckedFunction(
+				requestSlotParameters -> {
+					requestedSlotIds.put(requestSlotParameters.f0);
+					return new CompletableFuture<>();
+				}))
+			.createTestingTaskExecutorGateway();
+
+		final TaskExecutorConnection taskExecutorConnection = createTaskExecutorConnection(taskExecutorGateway);
+		final ResourceID resourceId = taskExecutorConnection.getResourceID();
+
+		final SlotID slotId1 = new SlotID(resourceId, 0);
+		final SlotID slotId2 = new SlotID(resourceId, 1);
+		final SlotReport slotReport = new SlotReport(Arrays.asList(createFreeSlotStatus(slotId1), createFreeSlotStatus(slotId2)));
+
+		final ScheduledExecutor mainThreadExecutor = new ManuallyTriggeredScheduledExecutor();
+
+		try (final DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setScheduledExecutor(mainThreadExecutor)
+			.build()) {
+
+			slotManager.start(ResourceManagerId.generate(), mainThreadExecutor, new TestingResourceActionsBuilder().build());
+
+			slotManager.registerTaskManager(taskExecutorConnection, slotReport);
+			slotManager.processResourceRequirements(resourceRequirements);
+
+			final SlotID firstRequestedSlotId = requestedSlotIds.take();
+			final SlotID freeSlotId = firstRequestedSlotId.equals(slotId1) ? slotId2 : slotId1;
+
+			final SlotReport newSlotReport = new SlotReport(Arrays.asList(createAllocatedSlotStatus(firstRequestedSlotId), createFreeSlotStatus(freeSlotId)));
+
+			slotManager.reportSlotStatus(taskExecutorConnection.getInstanceID(), newSlotReport);
+
+			final SlotID secondRequestedSlotId = requestedSlotIds.take();
+
+			assertEquals(freeSlotId, secondRequestedSlotId);
+		}
+	}
+
+	/**
+	 * Tests that free slots which are reported as allocated won't be considered for fulfilling
+	 * other pending slot requests.
+	 *
+	 * <p>See: FLINK-8505
+	 */
+	@Test
+	public void testReportAllocatedSlot() throws Exception {
+		final ResourceID taskManagerId = ResourceID.generate();
+		final TestingTaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder().createTestingTaskExecutorGateway();
+		final TaskExecutorConnection taskExecutorConnection = new TaskExecutorConnection(taskManagerId, taskExecutorGateway);
+
+		final ResourceTracker resourceTracker = new DefaultResourceTracker();
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setResourceTracker(resourceTracker)
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			// initially report a single slot as free
+			final SlotID slotId = new SlotID(taskManagerId, 0);
+			final SlotReport initialSlotReport = new SlotReport(createFreeSlotStatus(slotId));
+
+			slotManager.registerTaskManager(taskExecutorConnection, initialSlotReport);
+
+			assertThat(slotManager.getNumberRegisteredSlots(), is(equalTo(1)));
+
+			// Now report this slot as allocated
+			final SlotStatus slotStatus = createAllocatedSlotStatus(slotId);
+			final SlotReport slotReport = new SlotReport(slotStatus);
+
+			slotManager.reportSlotStatus(
+				taskExecutorConnection.getInstanceID(),
+				slotReport);
+
+			final JobID jobId = new JobID();
+			// this resource requirement should not be fulfilled
+			ResourceRequirements requirements = createResourceRequirementsForSingleSlot(jobId);
+
+			slotManager.processResourceRequirements(requirements);
+
+			assertThat(slotTracker.getSlot(slotId).getJobId(), is(slotStatus.getJobID()));
+			assertThat(getTotalResourceCount(resourceTracker.getMissingResources().get(jobId)), is(1));
+		}
+	}
+
+	/**
+	 * Tests that the SlotManager retries allocating a slot if the TaskExecutor#requestSlot call
+	 * fails.
+	 */
+	@Test
+	public void testSlotRequestFailure() throws Exception {
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (final DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			ResourceRequirements requirements = createResourceRequirementsForSingleSlot();
+			slotManager.processResourceRequirements(requirements);
+
+			final BlockingQueue<Tuple6<SlotID, JobID, AllocationID, ResourceProfile, String, ResourceManagerId>> requestSlotQueue = new ArrayBlockingQueue<>(1);
+			final BlockingQueue<CompletableFuture<Acknowledge>> responseQueue = new ArrayBlockingQueue<>(1);
+
+			final TestingTaskExecutorGateway testingTaskExecutorGateway = new TestingTaskExecutorGatewayBuilder()
+				.setRequestSlotFunction(slotIDJobIDAllocationIDStringResourceManagerIdTuple6 -> {
+					requestSlotQueue.offer(slotIDJobIDAllocationIDStringResourceManagerIdTuple6);
+					try {
+						return responseQueue.take();
+					} catch (InterruptedException ignored) {
+						return FutureUtils.completedExceptionally(new FlinkException("Response queue was interrupted."));
+					}
+				})
+				.createTestingTaskExecutorGateway();
+
+			final ResourceID taskExecutorResourceId = ResourceID.generate();
+			final TaskExecutorConnection taskExecutionConnection = new TaskExecutorConnection(taskExecutorResourceId, testingTaskExecutorGateway);
+			final SlotReport slotReport = new SlotReport(createFreeSlotStatus(new SlotID(taskExecutorResourceId, 0)));
+
+			final CompletableFuture<Acknowledge> firstManualSlotRequestResponse = new CompletableFuture<>();
+			responseQueue.offer(firstManualSlotRequestResponse);
+
+			slotManager.registerTaskManager(taskExecutionConnection, slotReport);
+
+			final Tuple6<SlotID, JobID, AllocationID, ResourceProfile, String, ResourceManagerId> firstRequest = requestSlotQueue.take();
+
+			final CompletableFuture<Acknowledge> secondManualSlotRequestResponse = new CompletableFuture<>();
+			responseQueue.offer(secondManualSlotRequestResponse);
+
+			// fail first request
+			firstManualSlotRequestResponse.completeExceptionally(new SlotAllocationException("Test exception"));
+
+			final Tuple6<SlotID, JobID, AllocationID, ResourceProfile, String, ResourceManagerId> secondRequest = requestSlotQueue.take();
+
+			assertThat(secondRequest.f1, equalTo(firstRequest.f1));
+			assertThat(secondRequest.f0, equalTo(firstRequest.f0));
+
+			secondManualSlotRequestResponse.complete(Acknowledge.get());
+
+			final DeclarativeTaskManagerSlot slot = slotTracker.getSlot(secondRequest.f0);
+			assertThat(slot.getState(), equalTo(SlotState.ALLOCATED));
+			assertThat(slot.getJobId(), equalTo(secondRequest.f1));
+		}
+	}
+
+	/**
+	 * Tests that pending request is removed if task executor reports a slot with the same job id.
+	 */
+	@Test
+	public void testSlotRequestRemovedIfTMReportAllocation() throws Exception {

Review comment:
       ```suggestion
   	public void testSlotRequestRemovedIfTMReportsAllocation() throws Exception {
   ```

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManagerTest.java
##########
@@ -0,0 +1,1092 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple6;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.concurrent.ManuallyTriggeredScheduledExecutor;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerId;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+import org.apache.flink.runtime.slots.ResourceRequirements;
+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.TestingTaskExecutorGateway;
+import org.apache.flink.runtime.taskexecutor.TestingTaskExecutorGatewayBuilder;
+import org.apache.flink.runtime.taskexecutor.exceptions.SlotAllocationException;
+import org.apache.flink.runtime.taskexecutor.exceptions.SlotOccupiedException;
+import org.apache.flink.runtime.testingUtils.TestingUtils;
+import org.apache.flink.util.FlinkException;
+import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.function.FunctionUtils;
+
+import akka.pattern.AskTimeoutException;
+import org.junit.Ignore;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.hamcrest.CoreMatchers.hasItem;
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.empty;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.hasSize;
+import static org.hamcrest.Matchers.is;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertSame;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Tests for the {@link DeclarativeSlotManager}.
+ */
+public class DeclarativeSlotManagerTest extends TestLogger {
+
+	private static final FlinkException TEST_EXCEPTION = new FlinkException("Test exception");
+
+	private static final WorkerResourceSpec WORKER_RESOURCE_SPEC = new WorkerResourceSpec.Builder()
+		.setCpuCores(100.0)
+		.setTaskHeapMemoryMB(10000)
+		.setTaskOffHeapMemoryMB(10000)
+		.setNetworkMemoryMB(10000)
+		.setManagedMemoryMB(10000)
+		.build();
+
+	/**
+	 * Tests that we can register task manager and their slots at the slot manager.
+	 */
+	@Test
+	public void testTaskManagerRegistration() throws Exception {
+		final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder().createTestingTaskExecutorGateway();
+		final ResourceID resourceId = ResourceID.generate();
+		final TaskExecutorConnection taskManagerConnection = new TaskExecutorConnection(resourceId, taskExecutorGateway);
+
+		final SlotID slotId1 = new SlotID(resourceId, 0);
+		final SlotID slotId2 = new SlotID(resourceId, 1);
+		final SlotReport slotReport = new SlotReport(Arrays.asList(createFreeSlotStatus(slotId1), createFreeSlotStatus(slotId2)));
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport);
+
+			assertThat("The number registered slots does not equal the expected number.", slotManager.getNumberRegisteredSlots(), is(2));
+
+			assertNotNull(slotTracker.getSlot(slotId1));
+			assertNotNull(slotTracker.getSlot(slotId2));
+		}
+	}
+
+	/**
+	 * Tests that un-registration of task managers will free and remove all registered slots.
+	 */
+	@Test
+	public void testTaskManagerUnregistration() throws Exception {
+		final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder()
+			.setRequestSlotFunction(tuple6 -> new CompletableFuture<>())
+			.createTestingTaskExecutorGateway();
+		final TaskExecutorConnection taskManagerConnection = createTaskExecutorConnection(taskExecutorGateway);
+		final ResourceID resourceId = taskManagerConnection.getResourceID();
+
+		final SlotID slotId1 = new SlotID(resourceId, 0);
+		final SlotID slotId2 = new SlotID(resourceId, 1);
+		final SlotReport slotReport = new SlotReport(Arrays.asList(createAllocatedSlotStatus(slotId1), createFreeSlotStatus(slotId2)));
+
+		final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot();
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport);
+
+			assertEquals("The number registered slots does not equal the expected number.", 2, slotManager.getNumberRegisteredSlots());
+
+			DeclarativeTaskManagerSlot slot1 = slotTracker.getSlot(slotId1);
+			DeclarativeTaskManagerSlot slot2 = slotTracker.getSlot(slotId2);
+
+			assertSame(SlotState.ALLOCATED, slot1.getState());
+			assertSame(SlotState.FREE, slot2.getState());
+
+			slotManager.processResourceRequirements(resourceRequirements);
+
+			assertSame(SlotState.PENDING, slot2.getState());
+
+			slotManager.unregisterTaskManager(taskManagerConnection.getInstanceID(), TEST_EXCEPTION);
+
+			assertEquals(0, slotManager.getNumberRegisteredSlots());
+		}
+	}
+
+	/**
+	 * Tests that a slot request with no free slots will trigger the resource allocation.
+	 */
+	@Test
+	public void testSlotRequestWithoutFreeSlots() throws Exception {
+		final ResourceManagerId resourceManagerId = ResourceManagerId.generate();
+
+		final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot();
+
+		CompletableFuture<WorkerResourceSpec> allocateResourceFuture = new CompletableFuture<>();
+		ResourceActions resourceManagerActions = new TestingResourceActionsBuilder()
+			.setAllocateResourceConsumer(allocateResourceFuture::complete)
+			.build();
+
+		try (SlotManager slotManager = createSlotManager(resourceManagerId, resourceManagerActions)) {
+
+			slotManager.processResourceRequirements(resourceRequirements);
+
+			allocateResourceFuture.get();
+		}
+	}
+
+	/**
+	 * Tests that resources continue to be considered missing if we cannot allocate more resources.
+	 */
+	@Test
+	public void testResourceDeclarationWithResourceAllocationFailure() throws Exception {
+		final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot();
+
+		ResourceActions resourceManagerActions = new TestingResourceActionsBuilder()
+			.setAllocateResourceFunction(value -> false)
+			.build();
+
+		final ResourceTracker resourceTracker = new DefaultResourceTracker();
+
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setResourceTracker(resourceTracker)
+			.buildAndStartWithDirectExec(ResourceManagerId.generate(), resourceManagerActions)) {
+
+			slotManager.processResourceRequirements(resourceRequirements);
+
+			final JobID jobId = resourceRequirements.getJobId();
+			assertThat(getTotalResourceCount(resourceTracker.getMissingResources().get(jobId)), is(1));
+		}
+	}
+
+	/**
+	 * Tests that a slot request which can be fulfilled will trigger a slot allocation.
+	 */
+	@Test
+	public void testSlotRequestWithFreeSlot() throws Exception {
+		final ResourceManagerId resourceManagerId = ResourceManagerId.generate();
+		final ResourceID resourceID = ResourceID.generate();
+		final JobID jobId = new JobID();
+		final SlotID slotId = new SlotID(resourceID, 0);
+		final String targetAddress = "localhost";
+		final ResourceProfile resourceProfile = ResourceProfile.fromResources(42.0, 1337);
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec(resourceManagerId, new TestingResourceActionsBuilder().build())) {
+
+			final CompletableFuture<Tuple6<SlotID, JobID, AllocationID, ResourceProfile, String, ResourceManagerId>> requestFuture = new CompletableFuture<>();
+			// accept an incoming slot request
+			final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder()
+				.setRequestSlotFunction(tuple6 -> {
+					requestFuture.complete(Tuple6.of(tuple6.f0, tuple6.f1, tuple6.f2, tuple6.f3, tuple6.f4, tuple6.f5));
+					return CompletableFuture.completedFuture(Acknowledge.get());
+				})
+				.createTestingTaskExecutorGateway();
+
+			final TaskExecutorConnection taskExecutorConnection = new TaskExecutorConnection(resourceID, taskExecutorGateway);
+
+			final SlotStatus slotStatus = new SlotStatus(slotId, resourceProfile);
+			final SlotReport slotReport = new SlotReport(slotStatus);
+
+			slotManager.registerTaskManager(
+				taskExecutorConnection,
+				slotReport);
+
+			ResourceRequirements requirements = ResourceRequirements.create(
+				jobId,
+				targetAddress,
+				Collections.singleton(ResourceRequirement.create(resourceProfile, 1)));
+			slotManager.processResourceRequirements(requirements);
+
+			assertThat(requestFuture.get(), is(equalTo(Tuple6.of(slotId, jobId, requestFuture.get().f2, resourceProfile, targetAddress, resourceManagerId))));
+
+			DeclarativeTaskManagerSlot slot = slotTracker.getSlot(slotId);
+
+			assertEquals("The slot has not been allocated to the expected allocation id.", jobId, slot.getJobId());
+		}
+	}
+
+	/**
+	 * Checks that reducing resource requirements while a slot allocation is in-progress results in the slot being freed
+	 * one the allocation completes.
+	 */
+	@Test
+	@Ignore
+	public void testResourceRequirementReductionDuringAllocation() throws Exception {
+		final TaskExecutorConnection taskManagerConnection = createTaskExecutorConnection();
+		final ResourceID resourceID = taskManagerConnection.getResourceID();
+		final SlotID slotId = new SlotID(resourceID, 0);
+
+		final SlotReport slotReport = new SlotReport(createFreeSlotStatus(slotId));
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport);
+
+			DeclarativeTaskManagerSlot slot = slotTracker.getSlot(slotId);
+
+			final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot();
+			slotManager.processResourceRequirements(resourceRequirements);
+
+			assertSame(SlotState.PENDING, slot.getState());
+
+			slotManager.processResourceRequirements(ResourceRequirements.create(resourceRequirements.getJobId(), "foobar", Collections.emptyList()));
+
+			slot = slotTracker.getSlot(slotId);
+
+			assertSame(SlotState.FREE, slot.getState());
+		}
+	}
+
+	/**
+	 * Tests that pending slot requests are tried to be fulfilled upon new slot registrations.
+	 */
+	@Test
+	public void testFulfillingPendingSlotRequest() throws Exception {
+		final ResourceManagerId resourceManagerId = ResourceManagerId.generate();
+		final ResourceID resourceID = ResourceID.generate();
+		final JobID jobId = new JobID();
+		final SlotID slotId = new SlotID(resourceID, 0);
+		final String targetAddress = "localhost";
+		final ResourceProfile resourceProfile = ResourceProfile.fromResources(42.0, 1337);
+
+		final AtomicInteger numberAllocateResourceCalls = new AtomicInteger(0);
+		ResourceActions resourceManagerActions = new TestingResourceActionsBuilder()
+			.setAllocateResourceConsumer(ignored -> numberAllocateResourceCalls.incrementAndGet())
+			.build();
+
+		final CompletableFuture<Tuple6<SlotID, JobID, AllocationID, ResourceProfile, String, ResourceManagerId>> requestFuture = new CompletableFuture<>();
+		// accept an incoming slot request
+		final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder()
+			.setRequestSlotFunction(tuple6 -> {
+				requestFuture.complete(Tuple6.of(tuple6.f0, tuple6.f1, tuple6.f2, tuple6.f3, tuple6.f4, tuple6.f5));
+				return CompletableFuture.completedFuture(Acknowledge.get());
+			})
+			.createTestingTaskExecutorGateway();
+
+		final TaskExecutorConnection taskExecutorConnection = new TaskExecutorConnection(resourceID, taskExecutorGateway);
+
+		final SlotStatus slotStatus = new SlotStatus(slotId, resourceProfile);
+		final SlotReport slotReport = new SlotReport(slotStatus);
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec(resourceManagerId, resourceManagerActions)) {
+
+			final ResourceRequirements resourceRequirements = ResourceRequirements.create(
+				jobId,
+				targetAddress,
+				Collections.singleton(ResourceRequirement.create(resourceProfile, 1)));
+			slotManager.processResourceRequirements(resourceRequirements);
+
+			assertThat(numberAllocateResourceCalls.get(), is(1));
+
+			slotManager.registerTaskManager(
+				taskExecutorConnection,
+				slotReport);
+
+			assertThat(requestFuture.get(), is(equalTo(Tuple6.of(slotId, jobId, requestFuture.get().f2, resourceProfile, targetAddress, resourceManagerId))));
+
+			DeclarativeTaskManagerSlot slot = slotTracker.getSlot(slotId);
+
+			assertEquals("The slot has not been allocated to the expected job id.", jobId, slot.getJobId());
+		}
+	}
+
+	/**
+	 * Tests that freeing a slot will correctly reset the slot and mark it as a free slot.
+	 */
+	@Test
+	public void testFreeSlot() throws Exception {
+		final TaskExecutorConnection taskExecutorConnection = createTaskExecutorConnection();
+		final ResourceID resourceID = taskExecutorConnection.getResourceID();
+		final SlotID slotId = new SlotID(resourceID, 0);
+
+		final SlotReport slotReport = new SlotReport(createAllocatedSlotStatus(slotId));
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			slotManager.registerTaskManager(
+				taskExecutorConnection,
+				slotReport);
+
+			DeclarativeTaskManagerSlot slot = slotTracker.getSlot(slotId);
+
+			assertSame(SlotState.ALLOCATED, slot.getState());
+
+			slotManager.freeSlot(slotId, new AllocationID());
+
+			assertSame(SlotState.FREE, slot.getState());
+
+			assertEquals(1, slotManager.getNumberFreeSlots());
+		}
+	}
+
+	/**
+	 * Tests that duplicate resource requirement declaration do not result in additional slots being allocated after a
+	 * pending slot request has been fulfilled but not yet freed.
+	 */
+	@Test
+	public void testDuplicateResourceRequirementDeclarationAfterSuccessfulAllocation() throws Exception {
+		final ResourceManagerId resourceManagerId = ResourceManagerId.generate();
+		final AtomicInteger allocateResourceCalls = new AtomicInteger(0);
+		final ResourceActions resourceManagerActions = new TestingResourceActionsBuilder()
+			.setAllocateResourceConsumer(ignored -> allocateResourceCalls.incrementAndGet())
+			.build();
+		ResourceRequirements requirements = createResourceRequirementsForSingleSlot();
+
+		final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder().createTestingTaskExecutorGateway();
+
+		final ResourceID resourceID = ResourceID.generate();
+
+		final TaskExecutorConnection taskManagerConnection = new TaskExecutorConnection(resourceID, taskExecutorGateway);
+
+		final SlotID slotId = new SlotID(resourceID, 0);
+		final SlotReport slotReport = new SlotReport(createFreeSlotStatus(slotId));
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec(resourceManagerId, resourceManagerActions)) {
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport);
+
+			slotManager.processResourceRequirements(requirements);
+
+			DeclarativeTaskManagerSlot slot = slotTracker.getSlot(slotId);
+
+			assertThat(slot.getState(), is(SlotState.ALLOCATED));
+
+			slotManager.processResourceRequirements(requirements);
+		}
+
+		// check that we have only called the resource allocation only for the first slot request,
+		// since the second request is a duplicate
+		assertThat(allocateResourceCalls.get(), is(0));
+	}
+
+	/**
+	 * Tests that a slot allocated for one job can be allocated for another job after being freed.
+	 */
+	@Test
+	public void testSlotCanBeAllocatedForDifferentJobAfterFree() throws Exception {
+		final AllocationID allocationId = new AllocationID();
+		final ResourceRequirements resourceRequirements1 = createResourceRequirementsForSingleSlot();
+		final ResourceRequirements resourceRequirements2 = createResourceRequirementsForSingleSlot();
+
+		final TaskExecutorConnection taskManagerConnection = createTaskExecutorConnection();
+		final ResourceID resourceID = taskManagerConnection.getResourceID();
+
+		final SlotID slotId = new SlotID(resourceID, 0);
+		final SlotReport slotReport = new SlotReport(createFreeSlotStatus(slotId));
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport);
+
+			slotManager.processResourceRequirements(resourceRequirements1);
+
+			DeclarativeTaskManagerSlot slot = slotTracker.getSlot(slotId);
+
+			assertEquals("The slot has not been allocated to the expected job id.", resourceRequirements1.getJobId(), slot.getJobId());
+
+			// clear resource requirements, so that the slot isn't immediately re-assigned
+			slotManager.processResourceRequirements(ResourceRequirements.create(resourceRequirements1.getJobId(), resourceRequirements1.getTargetAddress(), Collections.emptyList()));
+			slotManager.freeSlot(slotId, allocationId);
+
+			// check that the slot has been freed
+			assertSame(SlotState.FREE, slot.getState());
+
+			slotManager.processResourceRequirements(resourceRequirements2);
+
+			assertEquals("The slot has not been allocated to the expected job id.", resourceRequirements2.getJobId(), slot.getJobId());
+		}
+	}
+
+	/**
+	 * Tests that the slot manager ignores slot reports of unknown origin (not registered
+	 * task managers).
+	 */
+	@Test
+	public void testReceivingUnknownSlotReport() throws Exception {
+		final ResourceManagerId resourceManagerId = ResourceManagerId.generate();
+		final ResourceActions resourceManagerActions = new TestingResourceActionsBuilder().build();
+
+		final InstanceID unknownInstanceID = new InstanceID();
+		final SlotID unknownSlotId = new SlotID(ResourceID.generate(), 0);
+		final SlotReport unknownSlotReport = new SlotReport(createFreeSlotStatus(unknownSlotId));
+
+		try (SlotManager slotManager = createSlotManager(resourceManagerId, resourceManagerActions)) {
+			// check that we don't have any slots registered
+			assertThat(slotManager.getNumberRegisteredSlots(), is(0));
+
+			// this should not update anything since the instance id is not known to the slot manager
+			assertFalse(slotManager.reportSlotStatus(unknownInstanceID, unknownSlotReport));
+
+			assertThat(slotManager.getNumberRegisteredSlots(), is(0));
+		}
+	}
+
+	/**
+	 * Tests that slots are updated with respect to the latest incoming slot report. This means that
+	 * slots for which a report was received are updated accordingly.
+	 */
+	@Test
+	public void testUpdateSlotReport() throws Exception {
+		final TaskExecutorConnection taskManagerConnection = createTaskExecutorConnection();
+		final ResourceID resourceId = taskManagerConnection.getResourceID();
+
+		final SlotID slotId1 = new SlotID(resourceId, 0);
+		final SlotID slotId2 = new SlotID(resourceId, 1);
+
+		final SlotStatus slotStatus1 = createFreeSlotStatus(slotId1);
+		final SlotStatus slotStatus2 = createFreeSlotStatus(slotId2);
+
+		final SlotStatus newSlotStatus2 = createAllocatedSlotStatus(slotId2);
+		final JobID jobId = newSlotStatus2.getJobID();
+
+		final SlotReport slotReport1 = new SlotReport(Arrays.asList(slotStatus1, slotStatus2));
+		final SlotReport slotReport2 = new SlotReport(Arrays.asList(newSlotStatus2, slotStatus1));
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			// check that we don't have any slots registered
+			assertEquals(0, slotManager.getNumberRegisteredSlots());
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport1);
+
+			DeclarativeTaskManagerSlot slot1 = slotTracker.getSlot(slotId1);
+			DeclarativeTaskManagerSlot slot2 = slotTracker.getSlot(slotId2);
+
+			assertEquals(2, slotManager.getNumberRegisteredSlots());
+
+			assertSame(SlotState.FREE, slot1.getState());
+			assertSame(SlotState.FREE, slot2.getState());
+
+			assertTrue(slotManager.reportSlotStatus(taskManagerConnection.getInstanceID(), slotReport2));
+
+			assertEquals(2, slotManager.getNumberRegisteredSlots());
+
+			assertNotNull(slotTracker.getSlot(slotId1));
+			assertNotNull(slotTracker.getSlot(slotId2));
+
+			// slotId2 should have been allocated for jobiD
+			assertEquals(jobId, slotTracker.getSlot(slotId2).getJobId());
+		}
+	}
+
+	/**
+	 * Tests that if a slot allocation times out we try to allocate another slot.
+	 */
+	@Test
+	public void testSlotAllocationTimeout() throws Exception {
+		final CompletableFuture<Void> secondSlotRequestFuture = new CompletableFuture<>();
+		final AtomicInteger slotRequestsCount = new AtomicInteger();
+		final TaskExecutorConnection taskManagerConnection = createTaskExecutorConnection(new TestingTaskExecutorGatewayBuilder()
+			.setRequestSlotFunction(ignored -> {
+				if (slotRequestsCount.getAndAdd(1) == 1) {
+					secondSlotRequestFuture.complete(null);
+				} else {
+					// mimic RPC timeout
+					return FutureUtils.completedExceptionally(new AskTimeoutException("timeout"));
+				}
+				return new CompletableFuture<>();
+			})
+			.createTestingTaskExecutorGateway());
+		final SlotReport slotReport = createSlotReport(taskManagerConnection.getResourceID(), 2);
+
+		final Executor mainThreadExecutor = TestingUtils.defaultExecutor();
+
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.build()) {
+
+			slotManager.start(ResourceManagerId.generate(), mainThreadExecutor, new TestingResourceActionsBuilder().build());
+
+			CompletableFuture.runAsync(() -> slotManager.registerTaskManager(taskManagerConnection, slotReport), mainThreadExecutor)
+				.thenRun(() -> slotManager.processResourceRequirements(createResourceRequirementsForSingleSlot()))
+				.get(5, TimeUnit.SECONDS);
+
+			// a second request is only sent if the first request timed out
+			secondSlotRequestFuture.get();
+		}
+	}
+
+	/**
+	 * Tests that a slot allocation is retried if it times out on the task manager side.
+	 */
+	@Test
+	public void testTaskManagerSlotAllocationTimeoutHandling() throws Exception {
+		final JobID jobId = new JobID();
+		final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot(jobId);
+		final CompletableFuture<Acknowledge> slotRequestFuture1 = new CompletableFuture<>();
+		final CompletableFuture<Acknowledge> slotRequestFuture2 = new CompletableFuture<>();
+		final Iterator<CompletableFuture<Acknowledge>> slotRequestFutureIterator = Arrays.asList(slotRequestFuture1, slotRequestFuture2).iterator();
+		final ArrayBlockingQueue<SlotID> slotIds = new ArrayBlockingQueue<>(2);
+
+		final TestingTaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder()
+			.setRequestSlotFunction(FunctionUtils.uncheckedFunction(
+				requestSlotParameters -> {
+					slotIds.put(requestSlotParameters.f0);
+					return slotRequestFutureIterator.next();
+				}))
+			.createTestingTaskExecutorGateway();
+
+		final ResourceID resourceId = ResourceID.generate();
+		final TaskExecutorConnection taskManagerConnection = new TaskExecutorConnection(resourceId, taskExecutorGateway);
+
+		final SlotID slotId1 = new SlotID(resourceId, 0);
+		final SlotID slotId2 = new SlotID(resourceId, 1);
+		final SlotReport slotReport = new SlotReport(Arrays.asList(createFreeSlotStatus(slotId1), createFreeSlotStatus(slotId2)));
+
+		final ResourceTracker resourceTracker = new DefaultResourceTracker();
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setResourceTracker(resourceTracker)
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport);
+
+			slotManager.processResourceRequirements(resourceRequirements);
+
+			final SlotID firstSlotId = slotIds.take();
+			assertThat(slotIds, is(empty()));
+
+			DeclarativeTaskManagerSlot failedSlot = slotTracker.getSlot(firstSlotId);
+
+			// let the first attempt fail --> this should trigger a second attempt
+			slotRequestFuture1.completeExceptionally(new SlotAllocationException("Test exception."));
+
+			assertThat(getTotalResourceCount(resourceTracker.getAcquiredResources(jobId)), is(1));
+
+			// the second attempt succeeds
+			slotRequestFuture2.complete(Acknowledge.get());
+
+			final SlotID secondSlotId = slotIds.take();
+			assertThat(slotIds, is(empty()));
+
+			DeclarativeTaskManagerSlot slot = slotTracker.getSlot(secondSlotId);
+
+			assertThat(slot.getState(), is(SlotState.ALLOCATED));
+			assertEquals(jobId, slot.getJobId());
+
+			if (!failedSlot.getSlotId().equals(slot.getSlotId())) {
+				assertThat(failedSlot.getState(), is(SlotState.FREE));
+			}
+		}
+	}
+
+	/**
+	 * Tests that a pending slot allocation is cancelled if a slot report indicates that the slot is already allocated
+	 * by another job.
+	 */
+	@Test
+	public void testSlotReportWithConflictingJobIdDuringSlotAllocation() throws Exception {
+		final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot();
+		final ArrayBlockingQueue<SlotID> requestedSlotIds = new ArrayBlockingQueue<>(2);
+
+		final TestingTaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder()
+			.setRequestSlotFunction(FunctionUtils.uncheckedFunction(
+				requestSlotParameters -> {
+					requestedSlotIds.put(requestSlotParameters.f0);
+					return new CompletableFuture<>();
+				}))
+			.createTestingTaskExecutorGateway();
+
+		final TaskExecutorConnection taskExecutorConnection = createTaskExecutorConnection(taskExecutorGateway);
+		final ResourceID resourceId = taskExecutorConnection.getResourceID();
+
+		final SlotID slotId1 = new SlotID(resourceId, 0);
+		final SlotID slotId2 = new SlotID(resourceId, 1);
+		final SlotReport slotReport = new SlotReport(Arrays.asList(createFreeSlotStatus(slotId1), createFreeSlotStatus(slotId2)));
+
+		final ScheduledExecutor mainThreadExecutor = new ManuallyTriggeredScheduledExecutor();
+
+		try (final DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setScheduledExecutor(mainThreadExecutor)
+			.build()) {
+
+			slotManager.start(ResourceManagerId.generate(), mainThreadExecutor, new TestingResourceActionsBuilder().build());
+
+			slotManager.registerTaskManager(taskExecutorConnection, slotReport);
+			slotManager.processResourceRequirements(resourceRequirements);
+
+			final SlotID firstRequestedSlotId = requestedSlotIds.take();
+			final SlotID freeSlotId = firstRequestedSlotId.equals(slotId1) ? slotId2 : slotId1;
+
+			final SlotReport newSlotReport = new SlotReport(Arrays.asList(createAllocatedSlotStatus(firstRequestedSlotId), createFreeSlotStatus(freeSlotId)));
+
+			slotManager.reportSlotStatus(taskExecutorConnection.getInstanceID(), newSlotReport);
+
+			final SlotID secondRequestedSlotId = requestedSlotIds.take();
+
+			assertEquals(freeSlotId, secondRequestedSlotId);
+		}
+	}
+
+	/**
+	 * Tests that free slots which are reported as allocated won't be considered for fulfilling
+	 * other pending slot requests.
+	 *
+	 * <p>See: FLINK-8505
+	 */
+	@Test
+	public void testReportAllocatedSlot() throws Exception {
+		final ResourceID taskManagerId = ResourceID.generate();
+		final TestingTaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder().createTestingTaskExecutorGateway();
+		final TaskExecutorConnection taskExecutorConnection = new TaskExecutorConnection(taskManagerId, taskExecutorGateway);
+
+		final ResourceTracker resourceTracker = new DefaultResourceTracker();
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setResourceTracker(resourceTracker)
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			// initially report a single slot as free
+			final SlotID slotId = new SlotID(taskManagerId, 0);
+			final SlotReport initialSlotReport = new SlotReport(createFreeSlotStatus(slotId));
+
+			slotManager.registerTaskManager(taskExecutorConnection, initialSlotReport);
+
+			assertThat(slotManager.getNumberRegisteredSlots(), is(equalTo(1)));
+
+			// Now report this slot as allocated
+			final SlotStatus slotStatus = createAllocatedSlotStatus(slotId);
+			final SlotReport slotReport = new SlotReport(slotStatus);
+
+			slotManager.reportSlotStatus(
+				taskExecutorConnection.getInstanceID(),
+				slotReport);
+
+			final JobID jobId = new JobID();
+			// this resource requirement should not be fulfilled
+			ResourceRequirements requirements = createResourceRequirementsForSingleSlot(jobId);
+
+			slotManager.processResourceRequirements(requirements);
+
+			assertThat(slotTracker.getSlot(slotId).getJobId(), is(slotStatus.getJobID()));
+			assertThat(getTotalResourceCount(resourceTracker.getMissingResources().get(jobId)), is(1));
+		}
+	}
+
+	/**
+	 * Tests that the SlotManager retries allocating a slot if the TaskExecutor#requestSlot call
+	 * fails.
+	 */
+	@Test
+	public void testSlotRequestFailure() throws Exception {
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (final DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			ResourceRequirements requirements = createResourceRequirementsForSingleSlot();
+			slotManager.processResourceRequirements(requirements);
+
+			final BlockingQueue<Tuple6<SlotID, JobID, AllocationID, ResourceProfile, String, ResourceManagerId>> requestSlotQueue = new ArrayBlockingQueue<>(1);
+			final BlockingQueue<CompletableFuture<Acknowledge>> responseQueue = new ArrayBlockingQueue<>(1);
+
+			final TestingTaskExecutorGateway testingTaskExecutorGateway = new TestingTaskExecutorGatewayBuilder()
+				.setRequestSlotFunction(slotIDJobIDAllocationIDStringResourceManagerIdTuple6 -> {
+					requestSlotQueue.offer(slotIDJobIDAllocationIDStringResourceManagerIdTuple6);
+					try {
+						return responseQueue.take();
+					} catch (InterruptedException ignored) {
+						return FutureUtils.completedExceptionally(new FlinkException("Response queue was interrupted."));
+					}
+				})
+				.createTestingTaskExecutorGateway();
+
+			final ResourceID taskExecutorResourceId = ResourceID.generate();
+			final TaskExecutorConnection taskExecutionConnection = new TaskExecutorConnection(taskExecutorResourceId, testingTaskExecutorGateway);
+			final SlotReport slotReport = new SlotReport(createFreeSlotStatus(new SlotID(taskExecutorResourceId, 0)));
+
+			final CompletableFuture<Acknowledge> firstManualSlotRequestResponse = new CompletableFuture<>();
+			responseQueue.offer(firstManualSlotRequestResponse);
+
+			slotManager.registerTaskManager(taskExecutionConnection, slotReport);
+
+			final Tuple6<SlotID, JobID, AllocationID, ResourceProfile, String, ResourceManagerId> firstRequest = requestSlotQueue.take();
+
+			final CompletableFuture<Acknowledge> secondManualSlotRequestResponse = new CompletableFuture<>();
+			responseQueue.offer(secondManualSlotRequestResponse);
+
+			// fail first request
+			firstManualSlotRequestResponse.completeExceptionally(new SlotAllocationException("Test exception"));
+
+			final Tuple6<SlotID, JobID, AllocationID, ResourceProfile, String, ResourceManagerId> secondRequest = requestSlotQueue.take();
+
+			assertThat(secondRequest.f1, equalTo(firstRequest.f1));
+			assertThat(secondRequest.f0, equalTo(firstRequest.f0));
+
+			secondManualSlotRequestResponse.complete(Acknowledge.get());
+
+			final DeclarativeTaskManagerSlot slot = slotTracker.getSlot(secondRequest.f0);
+			assertThat(slot.getState(), equalTo(SlotState.ALLOCATED));
+			assertThat(slot.getJobId(), equalTo(secondRequest.f1));
+		}
+	}
+
+	/**
+	 * Tests that pending request is removed if task executor reports a slot with the same job id.
+	 */
+	@Test
+	public void testSlotRequestRemovedIfTMReportAllocation() throws Exception {
+		final ResourceTracker resourceTracker = new DefaultResourceTracker();
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+
+		try (final DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setResourceTracker(resourceTracker)
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			final JobID jobID = new JobID();
+			slotManager.processResourceRequirements(createResourceRequirementsForSingleSlot(jobID));
+
+			final BlockingQueue<Tuple6<SlotID, JobID, AllocationID, ResourceProfile, String, ResourceManagerId>> requestSlotQueue = new ArrayBlockingQueue<>(1);
+			final BlockingQueue<CompletableFuture<Acknowledge>> responseQueue = new ArrayBlockingQueue<>(1);
+
+			final TestingTaskExecutorGateway testingTaskExecutorGateway = new TestingTaskExecutorGatewayBuilder()
+				.setRequestSlotFunction(slotIDJobIDAllocationIDStringResourceManagerIdTuple6 -> {
+					requestSlotQueue.offer(slotIDJobIDAllocationIDStringResourceManagerIdTuple6);
+					try {
+						return responseQueue.take();
+					} catch (InterruptedException ignored) {
+						return FutureUtils.completedExceptionally(new FlinkException("Response queue was interrupted."));
+					}
+				})
+				.createTestingTaskExecutorGateway();
+
+			final ResourceID taskExecutorResourceId = ResourceID.generate();
+			final TaskExecutorConnection taskExecutionConnection = new TaskExecutorConnection(taskExecutorResourceId, testingTaskExecutorGateway);
+			final SlotReport slotReport = new SlotReport(createFreeSlotStatus(new SlotID(taskExecutorResourceId, 0)));
+
+			final CompletableFuture<Acknowledge> firstManualSlotRequestResponse = new CompletableFuture<>();
+			responseQueue.offer(firstManualSlotRequestResponse);
+
+			slotManager.registerTaskManager(taskExecutionConnection, slotReport);
+
+			final Tuple6<SlotID, JobID, AllocationID, ResourceProfile, String, ResourceManagerId> firstRequest = requestSlotQueue.take();
+
+			final CompletableFuture<Acknowledge> secondManualSlotRequestResponse = new CompletableFuture<>();
+			responseQueue.offer(secondManualSlotRequestResponse);
+
+			// fail first request
+			firstManualSlotRequestResponse.completeExceptionally(new TimeoutException("Test exception to fail first allocation"));
+
+			final Tuple6<SlotID, JobID, AllocationID, ResourceProfile, String, ResourceManagerId> secondRequest = requestSlotQueue.take();
+
+			// fail second request
+			secondManualSlotRequestResponse.completeExceptionally(new SlotOccupiedException("Test exception", new AllocationID(), jobID));
+
+			assertThat(firstRequest.f1, equalTo(jobID));
+			assertThat(secondRequest.f1, equalTo(jobID));
+			assertThat(secondRequest.f0, equalTo(firstRequest.f0));
+
+			final DeclarativeTaskManagerSlot slot = slotTracker.getSlot(secondRequest.f0);
+			assertThat(slot.getState(), equalTo(SlotState.ALLOCATED));
+			assertThat(slot.getJobId(), equalTo(firstRequest.f1));
+
+			assertThat(slotManager.getNumberRegisteredSlots(), is(1));
+			assertThat(getTotalResourceCount(resourceTracker.getAcquiredResources(jobID)), is(1));
+		}
+	}
+
+	/**
+	 * Tests notify the job manager of the allocations when the task manager is failed/killed.
+	 */
+	@Test
+	public void testNotifyFailedAllocationWhenTaskManagerTerminated() throws Exception {
+		final ResourceTracker resourceTracker = new DefaultResourceTracker();
+
+		try (final DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setResourceTracker(resourceTracker)
+			.buildAndStartWithDirectExec()) {
+
+			// register slot request for job1.
+			JobID jobId1 = new JobID();
+			slotManager.processResourceRequirements(createResourceRequirements(jobId1, 2));
+
+			// create task-manager-1 with 2 slots.
+			final ResourceID taskExecutorResourceId1 = ResourceID.generate();
+			final TestingTaskExecutorGateway testingTaskExecutorGateway1 = new TestingTaskExecutorGatewayBuilder().createTestingTaskExecutorGateway();
+			final TaskExecutorConnection taskExecutionConnection1 = new TaskExecutorConnection(taskExecutorResourceId1, testingTaskExecutorGateway1);
+			final SlotReport slotReport1 = createSlotReport(taskExecutorResourceId1, 2);
+
+			// register the task-manager-1 to the slot manager, this will trigger the slot allocation for job1.
+			slotManager.registerTaskManager(taskExecutionConnection1, slotReport1);
+
+			// register slot request for job2.
+			JobID jobId2 = new JobID();
+			slotManager.processResourceRequirements(createResourceRequirements(jobId2, 2));
+
+			// register slot request for job3.
+			JobID jobId3 = new JobID();
+			slotManager.processResourceRequirements(createResourceRequirements(jobId3, 1));
+
+			// create task-manager-2 with 3 slots.
+			final ResourceID taskExecutorResourceId2 = ResourceID.generate();
+			final TestingTaskExecutorGateway testingTaskExecutorGateway2 = new TestingTaskExecutorGatewayBuilder().createTestingTaskExecutorGateway();
+			final TaskExecutorConnection taskExecutionConnection2 = new TaskExecutorConnection(taskExecutorResourceId2, testingTaskExecutorGateway2);
+			final SlotReport slotReport2 = createSlotReport(taskExecutorResourceId2, 3);
+
+			// register the task-manager-2 to the slot manager, this will trigger the slot allocation for job2 and job3.
+			slotManager.registerTaskManager(taskExecutionConnection2, slotReport2);
+
+			// validate for job1.
+			slotManager.unregisterTaskManager(taskExecutionConnection1.getInstanceID(), TEST_EXCEPTION);
+
+			assertThat(getTotalResourceCount(resourceTracker.getMissingResources().get(jobId1)), is(2));
+
+			// validate the result for job2 and job3.
+			slotManager.unregisterTaskManager(taskExecutionConnection2.getInstanceID(), TEST_EXCEPTION);
+
+			assertThat(getTotalResourceCount(resourceTracker.getMissingResources().get(jobId2)), is(2));
+			assertThat(getTotalResourceCount(resourceTracker.getMissingResources().get(jobId3)), is(1));

Review comment:
       Not sure whether this test needs to be so complex. I guess it would be good enough to test it only with a single TM.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManagerTest.java
##########
@@ -0,0 +1,1092 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple6;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.concurrent.ManuallyTriggeredScheduledExecutor;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerId;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+import org.apache.flink.runtime.slots.ResourceRequirements;
+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.TestingTaskExecutorGateway;
+import org.apache.flink.runtime.taskexecutor.TestingTaskExecutorGatewayBuilder;
+import org.apache.flink.runtime.taskexecutor.exceptions.SlotAllocationException;
+import org.apache.flink.runtime.taskexecutor.exceptions.SlotOccupiedException;
+import org.apache.flink.runtime.testingUtils.TestingUtils;
+import org.apache.flink.util.FlinkException;
+import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.function.FunctionUtils;
+
+import akka.pattern.AskTimeoutException;
+import org.junit.Ignore;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.hamcrest.CoreMatchers.hasItem;
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.empty;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.hasSize;
+import static org.hamcrest.Matchers.is;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertSame;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Tests for the {@link DeclarativeSlotManager}.
+ */
+public class DeclarativeSlotManagerTest extends TestLogger {
+
+	private static final FlinkException TEST_EXCEPTION = new FlinkException("Test exception");
+
+	private static final WorkerResourceSpec WORKER_RESOURCE_SPEC = new WorkerResourceSpec.Builder()
+		.setCpuCores(100.0)
+		.setTaskHeapMemoryMB(10000)
+		.setTaskOffHeapMemoryMB(10000)
+		.setNetworkMemoryMB(10000)
+		.setManagedMemoryMB(10000)
+		.build();
+
+	/**
+	 * Tests that we can register task manager and their slots at the slot manager.
+	 */
+	@Test
+	public void testTaskManagerRegistration() throws Exception {
+		final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder().createTestingTaskExecutorGateway();
+		final ResourceID resourceId = ResourceID.generate();
+		final TaskExecutorConnection taskManagerConnection = new TaskExecutorConnection(resourceId, taskExecutorGateway);
+
+		final SlotID slotId1 = new SlotID(resourceId, 0);
+		final SlotID slotId2 = new SlotID(resourceId, 1);
+		final SlotReport slotReport = new SlotReport(Arrays.asList(createFreeSlotStatus(slotId1), createFreeSlotStatus(slotId2)));
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport);
+
+			assertThat("The number registered slots does not equal the expected number.", slotManager.getNumberRegisteredSlots(), is(2));
+
+			assertNotNull(slotTracker.getSlot(slotId1));
+			assertNotNull(slotTracker.getSlot(slotId2));
+		}
+	}
+
+	/**
+	 * Tests that un-registration of task managers will free and remove all registered slots.
+	 */
+	@Test
+	public void testTaskManagerUnregistration() throws Exception {
+		final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder()
+			.setRequestSlotFunction(tuple6 -> new CompletableFuture<>())
+			.createTestingTaskExecutorGateway();
+		final TaskExecutorConnection taskManagerConnection = createTaskExecutorConnection(taskExecutorGateway);
+		final ResourceID resourceId = taskManagerConnection.getResourceID();
+
+		final SlotID slotId1 = new SlotID(resourceId, 0);
+		final SlotID slotId2 = new SlotID(resourceId, 1);
+		final SlotReport slotReport = new SlotReport(Arrays.asList(createAllocatedSlotStatus(slotId1), createFreeSlotStatus(slotId2)));
+
+		final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot();
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport);
+
+			assertEquals("The number registered slots does not equal the expected number.", 2, slotManager.getNumberRegisteredSlots());
+
+			DeclarativeTaskManagerSlot slot1 = slotTracker.getSlot(slotId1);
+			DeclarativeTaskManagerSlot slot2 = slotTracker.getSlot(slotId2);
+
+			assertSame(SlotState.ALLOCATED, slot1.getState());
+			assertSame(SlotState.FREE, slot2.getState());
+
+			slotManager.processResourceRequirements(resourceRequirements);
+
+			assertSame(SlotState.PENDING, slot2.getState());
+
+			slotManager.unregisterTaskManager(taskManagerConnection.getInstanceID(), TEST_EXCEPTION);
+
+			assertEquals(0, slotManager.getNumberRegisteredSlots());
+		}
+	}
+
+	/**
+	 * Tests that a slot request with no free slots will trigger the resource allocation.
+	 */
+	@Test
+	public void testSlotRequestWithoutFreeSlots() throws Exception {
+		final ResourceManagerId resourceManagerId = ResourceManagerId.generate();
+
+		final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot();
+
+		CompletableFuture<WorkerResourceSpec> allocateResourceFuture = new CompletableFuture<>();
+		ResourceActions resourceManagerActions = new TestingResourceActionsBuilder()
+			.setAllocateResourceConsumer(allocateResourceFuture::complete)
+			.build();
+
+		try (SlotManager slotManager = createSlotManager(resourceManagerId, resourceManagerActions)) {
+
+			slotManager.processResourceRequirements(resourceRequirements);
+
+			allocateResourceFuture.get();
+		}
+	}
+
+	/**
+	 * Tests that resources continue to be considered missing if we cannot allocate more resources.
+	 */
+	@Test
+	public void testResourceDeclarationWithResourceAllocationFailure() throws Exception {
+		final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot();
+
+		ResourceActions resourceManagerActions = new TestingResourceActionsBuilder()
+			.setAllocateResourceFunction(value -> false)
+			.build();
+
+		final ResourceTracker resourceTracker = new DefaultResourceTracker();
+
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setResourceTracker(resourceTracker)
+			.buildAndStartWithDirectExec(ResourceManagerId.generate(), resourceManagerActions)) {
+
+			slotManager.processResourceRequirements(resourceRequirements);
+
+			final JobID jobId = resourceRequirements.getJobId();
+			assertThat(getTotalResourceCount(resourceTracker.getMissingResources().get(jobId)), is(1));
+		}
+	}
+
+	/**
+	 * Tests that a slot request which can be fulfilled will trigger a slot allocation.
+	 */
+	@Test
+	public void testSlotRequestWithFreeSlot() throws Exception {
+		final ResourceManagerId resourceManagerId = ResourceManagerId.generate();
+		final ResourceID resourceID = ResourceID.generate();
+		final JobID jobId = new JobID();
+		final SlotID slotId = new SlotID(resourceID, 0);
+		final String targetAddress = "localhost";
+		final ResourceProfile resourceProfile = ResourceProfile.fromResources(42.0, 1337);
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec(resourceManagerId, new TestingResourceActionsBuilder().build())) {
+
+			final CompletableFuture<Tuple6<SlotID, JobID, AllocationID, ResourceProfile, String, ResourceManagerId>> requestFuture = new CompletableFuture<>();
+			// accept an incoming slot request
+			final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder()
+				.setRequestSlotFunction(tuple6 -> {
+					requestFuture.complete(Tuple6.of(tuple6.f0, tuple6.f1, tuple6.f2, tuple6.f3, tuple6.f4, tuple6.f5));
+					return CompletableFuture.completedFuture(Acknowledge.get());
+				})
+				.createTestingTaskExecutorGateway();
+
+			final TaskExecutorConnection taskExecutorConnection = new TaskExecutorConnection(resourceID, taskExecutorGateway);
+
+			final SlotStatus slotStatus = new SlotStatus(slotId, resourceProfile);
+			final SlotReport slotReport = new SlotReport(slotStatus);
+
+			slotManager.registerTaskManager(
+				taskExecutorConnection,
+				slotReport);
+
+			ResourceRequirements requirements = ResourceRequirements.create(
+				jobId,
+				targetAddress,
+				Collections.singleton(ResourceRequirement.create(resourceProfile, 1)));
+			slotManager.processResourceRequirements(requirements);
+
+			assertThat(requestFuture.get(), is(equalTo(Tuple6.of(slotId, jobId, requestFuture.get().f2, resourceProfile, targetAddress, resourceManagerId))));
+
+			DeclarativeTaskManagerSlot slot = slotTracker.getSlot(slotId);
+
+			assertEquals("The slot has not been allocated to the expected allocation id.", jobId, slot.getJobId());
+		}
+	}
+
+	/**
+	 * Checks that reducing resource requirements while a slot allocation is in-progress results in the slot being freed
+	 * one the allocation completes.
+	 */
+	@Test
+	@Ignore
+	public void testResourceRequirementReductionDuringAllocation() throws Exception {
+		final TaskExecutorConnection taskManagerConnection = createTaskExecutorConnection();
+		final ResourceID resourceID = taskManagerConnection.getResourceID();
+		final SlotID slotId = new SlotID(resourceID, 0);
+
+		final SlotReport slotReport = new SlotReport(createFreeSlotStatus(slotId));
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport);
+
+			DeclarativeTaskManagerSlot slot = slotTracker.getSlot(slotId);
+
+			final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot();
+			slotManager.processResourceRequirements(resourceRequirements);
+
+			assertSame(SlotState.PENDING, slot.getState());
+
+			slotManager.processResourceRequirements(ResourceRequirements.create(resourceRequirements.getJobId(), "foobar", Collections.emptyList()));
+
+			slot = slotTracker.getSlot(slotId);
+
+			assertSame(SlotState.FREE, slot.getState());
+		}
+	}
+
+	/**
+	 * Tests that pending slot requests are tried to be fulfilled upon new slot registrations.
+	 */
+	@Test
+	public void testFulfillingPendingSlotRequest() throws Exception {
+		final ResourceManagerId resourceManagerId = ResourceManagerId.generate();
+		final ResourceID resourceID = ResourceID.generate();
+		final JobID jobId = new JobID();
+		final SlotID slotId = new SlotID(resourceID, 0);
+		final String targetAddress = "localhost";
+		final ResourceProfile resourceProfile = ResourceProfile.fromResources(42.0, 1337);
+
+		final AtomicInteger numberAllocateResourceCalls = new AtomicInteger(0);
+		ResourceActions resourceManagerActions = new TestingResourceActionsBuilder()
+			.setAllocateResourceConsumer(ignored -> numberAllocateResourceCalls.incrementAndGet())
+			.build();
+
+		final CompletableFuture<Tuple6<SlotID, JobID, AllocationID, ResourceProfile, String, ResourceManagerId>> requestFuture = new CompletableFuture<>();
+		// accept an incoming slot request
+		final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder()
+			.setRequestSlotFunction(tuple6 -> {
+				requestFuture.complete(Tuple6.of(tuple6.f0, tuple6.f1, tuple6.f2, tuple6.f3, tuple6.f4, tuple6.f5));
+				return CompletableFuture.completedFuture(Acknowledge.get());
+			})
+			.createTestingTaskExecutorGateway();
+
+		final TaskExecutorConnection taskExecutorConnection = new TaskExecutorConnection(resourceID, taskExecutorGateway);
+
+		final SlotStatus slotStatus = new SlotStatus(slotId, resourceProfile);
+		final SlotReport slotReport = new SlotReport(slotStatus);
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec(resourceManagerId, resourceManagerActions)) {
+
+			final ResourceRequirements resourceRequirements = ResourceRequirements.create(
+				jobId,
+				targetAddress,
+				Collections.singleton(ResourceRequirement.create(resourceProfile, 1)));
+			slotManager.processResourceRequirements(resourceRequirements);
+
+			assertThat(numberAllocateResourceCalls.get(), is(1));
+
+			slotManager.registerTaskManager(
+				taskExecutorConnection,
+				slotReport);
+
+			assertThat(requestFuture.get(), is(equalTo(Tuple6.of(slotId, jobId, requestFuture.get().f2, resourceProfile, targetAddress, resourceManagerId))));
+
+			DeclarativeTaskManagerSlot slot = slotTracker.getSlot(slotId);
+
+			assertEquals("The slot has not been allocated to the expected job id.", jobId, slot.getJobId());
+		}
+	}
+
+	/**
+	 * Tests that freeing a slot will correctly reset the slot and mark it as a free slot.
+	 */
+	@Test
+	public void testFreeSlot() throws Exception {
+		final TaskExecutorConnection taskExecutorConnection = createTaskExecutorConnection();
+		final ResourceID resourceID = taskExecutorConnection.getResourceID();
+		final SlotID slotId = new SlotID(resourceID, 0);
+
+		final SlotReport slotReport = new SlotReport(createAllocatedSlotStatus(slotId));
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			slotManager.registerTaskManager(
+				taskExecutorConnection,
+				slotReport);
+
+			DeclarativeTaskManagerSlot slot = slotTracker.getSlot(slotId);
+
+			assertSame(SlotState.ALLOCATED, slot.getState());
+
+			slotManager.freeSlot(slotId, new AllocationID());
+
+			assertSame(SlotState.FREE, slot.getState());
+
+			assertEquals(1, slotManager.getNumberFreeSlots());
+		}
+	}
+
+	/**
+	 * Tests that duplicate resource requirement declaration do not result in additional slots being allocated after a
+	 * pending slot request has been fulfilled but not yet freed.
+	 */
+	@Test
+	public void testDuplicateResourceRequirementDeclarationAfterSuccessfulAllocation() throws Exception {
+		final ResourceManagerId resourceManagerId = ResourceManagerId.generate();
+		final AtomicInteger allocateResourceCalls = new AtomicInteger(0);
+		final ResourceActions resourceManagerActions = new TestingResourceActionsBuilder()
+			.setAllocateResourceConsumer(ignored -> allocateResourceCalls.incrementAndGet())
+			.build();
+		ResourceRequirements requirements = createResourceRequirementsForSingleSlot();
+
+		final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder().createTestingTaskExecutorGateway();
+
+		final ResourceID resourceID = ResourceID.generate();
+
+		final TaskExecutorConnection taskManagerConnection = new TaskExecutorConnection(resourceID, taskExecutorGateway);
+
+		final SlotID slotId = new SlotID(resourceID, 0);
+		final SlotReport slotReport = new SlotReport(createFreeSlotStatus(slotId));
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec(resourceManagerId, resourceManagerActions)) {
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport);
+
+			slotManager.processResourceRequirements(requirements);
+
+			DeclarativeTaskManagerSlot slot = slotTracker.getSlot(slotId);
+
+			assertThat(slot.getState(), is(SlotState.ALLOCATED));
+
+			slotManager.processResourceRequirements(requirements);
+		}
+
+		// check that we have only called the resource allocation only for the first slot request,
+		// since the second request is a duplicate
+		assertThat(allocateResourceCalls.get(), is(0));
+	}
+
+	/**
+	 * Tests that a slot allocated for one job can be allocated for another job after being freed.
+	 */
+	@Test
+	public void testSlotCanBeAllocatedForDifferentJobAfterFree() throws Exception {
+		final AllocationID allocationId = new AllocationID();
+		final ResourceRequirements resourceRequirements1 = createResourceRequirementsForSingleSlot();
+		final ResourceRequirements resourceRequirements2 = createResourceRequirementsForSingleSlot();
+
+		final TaskExecutorConnection taskManagerConnection = createTaskExecutorConnection();
+		final ResourceID resourceID = taskManagerConnection.getResourceID();
+
+		final SlotID slotId = new SlotID(resourceID, 0);
+		final SlotReport slotReport = new SlotReport(createFreeSlotStatus(slotId));
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport);
+
+			slotManager.processResourceRequirements(resourceRequirements1);
+
+			DeclarativeTaskManagerSlot slot = slotTracker.getSlot(slotId);
+
+			assertEquals("The slot has not been allocated to the expected job id.", resourceRequirements1.getJobId(), slot.getJobId());
+
+			// clear resource requirements, so that the slot isn't immediately re-assigned
+			slotManager.processResourceRequirements(ResourceRequirements.create(resourceRequirements1.getJobId(), resourceRequirements1.getTargetAddress(), Collections.emptyList()));
+			slotManager.freeSlot(slotId, allocationId);
+
+			// check that the slot has been freed
+			assertSame(SlotState.FREE, slot.getState());
+
+			slotManager.processResourceRequirements(resourceRequirements2);
+
+			assertEquals("The slot has not been allocated to the expected job id.", resourceRequirements2.getJobId(), slot.getJobId());
+		}
+	}
+
+	/**
+	 * Tests that the slot manager ignores slot reports of unknown origin (not registered
+	 * task managers).
+	 */
+	@Test
+	public void testReceivingUnknownSlotReport() throws Exception {
+		final ResourceManagerId resourceManagerId = ResourceManagerId.generate();
+		final ResourceActions resourceManagerActions = new TestingResourceActionsBuilder().build();
+
+		final InstanceID unknownInstanceID = new InstanceID();
+		final SlotID unknownSlotId = new SlotID(ResourceID.generate(), 0);
+		final SlotReport unknownSlotReport = new SlotReport(createFreeSlotStatus(unknownSlotId));
+
+		try (SlotManager slotManager = createSlotManager(resourceManagerId, resourceManagerActions)) {
+			// check that we don't have any slots registered
+			assertThat(slotManager.getNumberRegisteredSlots(), is(0));
+
+			// this should not update anything since the instance id is not known to the slot manager
+			assertFalse(slotManager.reportSlotStatus(unknownInstanceID, unknownSlotReport));
+
+			assertThat(slotManager.getNumberRegisteredSlots(), is(0));
+		}
+	}
+
+	/**
+	 * Tests that slots are updated with respect to the latest incoming slot report. This means that
+	 * slots for which a report was received are updated accordingly.
+	 */
+	@Test
+	public void testUpdateSlotReport() throws Exception {
+		final TaskExecutorConnection taskManagerConnection = createTaskExecutorConnection();
+		final ResourceID resourceId = taskManagerConnection.getResourceID();
+
+		final SlotID slotId1 = new SlotID(resourceId, 0);
+		final SlotID slotId2 = new SlotID(resourceId, 1);
+
+		final SlotStatus slotStatus1 = createFreeSlotStatus(slotId1);
+		final SlotStatus slotStatus2 = createFreeSlotStatus(slotId2);
+
+		final SlotStatus newSlotStatus2 = createAllocatedSlotStatus(slotId2);
+		final JobID jobId = newSlotStatus2.getJobID();
+
+		final SlotReport slotReport1 = new SlotReport(Arrays.asList(slotStatus1, slotStatus2));
+		final SlotReport slotReport2 = new SlotReport(Arrays.asList(newSlotStatus2, slotStatus1));
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			// check that we don't have any slots registered
+			assertEquals(0, slotManager.getNumberRegisteredSlots());
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport1);
+
+			DeclarativeTaskManagerSlot slot1 = slotTracker.getSlot(slotId1);
+			DeclarativeTaskManagerSlot slot2 = slotTracker.getSlot(slotId2);
+
+			assertEquals(2, slotManager.getNumberRegisteredSlots());
+
+			assertSame(SlotState.FREE, slot1.getState());
+			assertSame(SlotState.FREE, slot2.getState());
+
+			assertTrue(slotManager.reportSlotStatus(taskManagerConnection.getInstanceID(), slotReport2));
+
+			assertEquals(2, slotManager.getNumberRegisteredSlots());
+
+			assertNotNull(slotTracker.getSlot(slotId1));
+			assertNotNull(slotTracker.getSlot(slotId2));
+
+			// slotId2 should have been allocated for jobiD
+			assertEquals(jobId, slotTracker.getSlot(slotId2).getJobId());
+		}
+	}
+
+	/**
+	 * Tests that if a slot allocation times out we try to allocate another slot.
+	 */
+	@Test
+	public void testSlotAllocationTimeout() throws Exception {
+		final CompletableFuture<Void> secondSlotRequestFuture = new CompletableFuture<>();
+		final AtomicInteger slotRequestsCount = new AtomicInteger();
+		final TaskExecutorConnection taskManagerConnection = createTaskExecutorConnection(new TestingTaskExecutorGatewayBuilder()
+			.setRequestSlotFunction(ignored -> {
+				if (slotRequestsCount.getAndAdd(1) == 1) {
+					secondSlotRequestFuture.complete(null);
+				} else {
+					// mimic RPC timeout
+					return FutureUtils.completedExceptionally(new AskTimeoutException("timeout"));
+				}
+				return new CompletableFuture<>();

Review comment:
       A slightly easier way to express this logic would be to use a queue from which you pop and you populate the queue with the answers you want to have.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot commented on pull request #13553: [FLINK-19306][coordination] Add DeclarativeSlotManager

Posted by GitBox <gi...@apache.org>.
flinkbot commented on pull request #13553:
URL: https://github.com/apache/flink/pull/13553#issuecomment-704923575


   Thanks a lot for your contribution to the Apache Flink project. I'm the @flinkbot. I help the community
   to review your pull request. We will use this comment to track the progress of the review.
   
   
   ## Automated Checks
   Last check on commit 43a01ab430e9fd3070c8c9f6eb064fafa415e9d5 (Wed Oct 07 13:08:09 UTC 2020)
   
   **Warnings:**
    * No documentation files were touched! Remember to keep the Flink docs up to date!
   
   
   <sub>Mention the bot in a comment to re-run the automated checks.</sub>
   ## Review Progress
   
   * ❓ 1. The [description] looks good.
   * ❓ 2. There is [consensus] that the contribution should go into to Flink.
   * ❓ 3. Needs [attention] from.
   * ❓ 4. The change fits into the overall [architecture].
   * ❓ 5. Overall code [quality] is good.
   
   Please see the [Pull Request Review Guide](https://flink.apache.org/contributing/reviewing-prs.html) for a full explanation of the review process.<details>
    The Bot is tracking the review progress through labels. Labels are applied according to the order of the review items. For consensus, approval by a Flink committer of PMC member is required <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot approve description` to approve one or more aspects (aspects: `description`, `consensus`, `architecture` and `quality`)
    - `@flinkbot approve all` to approve all aspects
    - `@flinkbot approve-until architecture` to approve everything until `architecture`
    - `@flinkbot attention @username1 [@username2 ..]` to require somebody's attention
    - `@flinkbot disapprove architecture` to remove an approval you gave earlier
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot commented on pull request #13553: [FLINK-19306][coordination] Add DeclarativeSlotManager

Posted by GitBox <gi...@apache.org>.
flinkbot commented on pull request #13553:
URL: https://github.com/apache/flink/pull/13553#issuecomment-704936985


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "15d4c353c4c0a646b2bdf34bbedbd81359cb5b5f",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "15d4c353c4c0a646b2bdf34bbedbd81359cb5b5f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 15d4c353c4c0a646b2bdf34bbedbd81359cb5b5f UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] zentol commented on a change in pull request #13553: [FLINK-19306][coordination] Add DeclarativeSlotManager

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #13553:
URL: https://github.com/apache/flink/pull/13553#discussion_r502340350



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManager.java
##########
@@ -0,0 +1,609 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.metrics.MetricNames;
+import org.apache.flink.runtime.metrics.groups.SlotManagerMetricGroup;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerId;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.slots.ResourceCounter;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+import org.apache.flink.runtime.slots.ResourceRequirements;
+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.SlotOccupiedException;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ScheduledFuture;
+import java.util.function.BiFunction;
+import java.util.stream.Collectors;
+
+/**
+ * Implementation of {@link SlotManager} supporting declarative slot management.
+ */
+public class DeclarativeSlotManager implements SlotManager {
+	private static final Logger LOG = LoggerFactory.getLogger(DeclarativeSlotManager.class);
+
+	private final SlotTracker slotTracker;
+	private final ResourceTracker resourceTracker;
+	private final BiFunction<Executor, ResourceActions, TaskExecutorManager> taskExecutorManagerFactory;
+	private TaskExecutorManager taskExecutorManager;
+
+	/** Timeout for slot requests to the task manager. */
+	private final Time taskManagerRequestTimeout;
+	private ScheduledFuture<?> slotRequestTimeoutCheck;
+
+	private final SlotMatchingStrategy slotMatchingStrategy;
+
+	private final SlotManagerMetricGroup slotManagerMetricGroup;
+
+	private final Map<JobID, String> jobMasterTargetAddresses = new HashMap<>();
+	private final HashMap<SlotID, CompletableFuture<Acknowledge>> pendingSlotAllocationFutures;
+
+	/** ResourceManager's id. */
+	private ResourceManagerId resourceManagerId;
+
+	/** Executor for future callbacks which have to be "synchronized". */
+	private Executor mainThreadExecutor;
+
+	/** Callbacks for resource (de-)allocations. */
+	private ResourceActions resourceActions;
+
+	/** True iff the component has been started. */
+	private boolean started;
+
+	public DeclarativeSlotManager(
+			ScheduledExecutor scheduledExecutor,
+			SlotManagerConfiguration slotManagerConfiguration,
+			SlotManagerMetricGroup slotManagerMetricGroup,
+			ResourceTracker resourceTracker,
+			SlotTracker slotTracker) {
+
+		Preconditions.checkNotNull(slotManagerConfiguration);
+		this.taskManagerRequestTimeout = slotManagerConfiguration.getTaskManagerRequestTimeout();
+		this.slotManagerMetricGroup = Preconditions.checkNotNull(slotManagerMetricGroup);
+		this.resourceTracker = Preconditions.checkNotNull(resourceTracker);
+
+		pendingSlotAllocationFutures = new HashMap<>(16);
+
+		this.slotTracker = Preconditions.checkNotNull(slotTracker);
+		slotTracker.registerSlotStatusUpdateListener(createSlotStatusUpdateListener());
+
+		slotMatchingStrategy = slotManagerConfiguration.getSlotMatchingStrategy();
+
+		taskExecutorManagerFactory = (executor, resourceActions) -> new TaskExecutorManager(
+			slotManagerConfiguration.getDefaultWorkerResourceSpec(),
+			slotManagerConfiguration.getNumSlotsPerWorker(),
+			slotManagerConfiguration.getMaxSlotNum(),
+			slotManagerConfiguration.isWaitResultConsumedBeforeRelease(),
+			slotManagerConfiguration.getRedundantTaskManagerNum(),
+			slotManagerConfiguration.getTaskManagerTimeout(),
+			scheduledExecutor,
+			executor,
+			resourceActions);
+
+		resourceManagerId = null;
+		resourceActions = null;
+		mainThreadExecutor = null;
+		slotRequestTimeoutCheck = null;
+		taskExecutorManager = null;
+
+		started = false;
+	}
+
+	private SlotStatusUpdateListener createSlotStatusUpdateListener() {
+		return (taskManagerSlot, previous, current, jobId) -> {
+			if (previous == SlotState.PENDING) {
+				cancelAllocationFuture(taskManagerSlot.getSlotId());
+			}
+
+			if (current == SlotState.PENDING) {
+				resourceTracker.notifyAcquiredResource(jobId, taskManagerSlot.getResourceProfile());
+			}
+			if (current == SlotState.FREE) {
+				resourceTracker.notifyLostResource(jobId, taskManagerSlot.getResourceProfile());
+			}
+
+			if (current == SlotState.ALLOCATED) {
+				taskExecutorManager.occupySlot(taskManagerSlot.getInstanceId());
+			}
+			if (previous == SlotState.ALLOCATED && current == SlotState.FREE) {
+				taskExecutorManager.freeSlot(taskManagerSlot.getInstanceId());
+			}
+		};
+	}
+
+	private void cancelAllocationFuture(SlotID slotId) {
+		final CompletableFuture<Acknowledge> acknowledgeCompletableFuture = pendingSlotAllocationFutures.remove(slotId);
+		// the future may be null if we are just re-playing the state transitions due to a slot report
+		if (acknowledgeCompletableFuture != null) {
+			acknowledgeCompletableFuture.cancel(false);
+		}
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Component lifecycle methods
+	// ---------------------------------------------------------------------------------------------
+
+	/**
+	 * Starts the slot manager with the given leader id and resource manager actions.
+	 *
+	 * @param newResourceManagerId to use for communication with the task managers
+	 * @param newMainThreadExecutor to use to run code in the ResourceManager's main thread
+	 * @param newResourceActions to use for resource (de-)allocations
+	 */
+	@Override
+	public void start(ResourceManagerId newResourceManagerId, Executor newMainThreadExecutor, ResourceActions newResourceActions) {
+		LOG.info("Starting the slot manager.");
+
+		this.resourceManagerId = Preconditions.checkNotNull(newResourceManagerId);
+		mainThreadExecutor = Preconditions.checkNotNull(newMainThreadExecutor);
+		resourceActions = Preconditions.checkNotNull(newResourceActions);
+		taskExecutorManager = taskExecutorManagerFactory.apply(newMainThreadExecutor, newResourceActions);
+
+		started = true;
+
+		registerSlotManagerMetrics();
+	}
+
+	private void registerSlotManagerMetrics() {
+		slotManagerMetricGroup.gauge(
+			MetricNames.TASK_SLOTS_AVAILABLE,
+			() -> (long) getNumberFreeSlots());
+		slotManagerMetricGroup.gauge(
+			MetricNames.TASK_SLOTS_TOTAL,
+			() -> (long) getNumberRegisteredSlots());
+	}
+
+	/**
+	 * Suspends the component. This clears the internal state of the slot manager.
+	 */
+	@Override
+	public void suspend() {
+		LOG.info("Suspending the slot manager.");
+
+		if (slotRequestTimeoutCheck != null) {
+			slotRequestTimeoutCheck.cancel(false);
+			slotRequestTimeoutCheck = null;
+		}
+
+		resourceTracker.clear();
+		taskExecutorManager.close();
+
+		for (InstanceID registeredTaskManager : taskExecutorManager.getTaskExecutors()) {
+			unregisterTaskManager(registeredTaskManager, new SlotManagerException("The slot manager is being suspended."));
+		}
+
+		taskExecutorManager = null;
+		resourceManagerId = null;
+		resourceActions = null;
+		started = false;
+	}
+
+	/**
+	 * Closes the slot manager.
+	 *
+	 * @throws Exception if the close operation fails
+	 */
+	@Override
+	public void close() throws Exception {
+		LOG.info("Closing the slot manager.");
+
+		suspend();
+		slotManagerMetricGroup.close();
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Public API
+	// ---------------------------------------------------------------------------------------------
+
+	@Override
+	public void processResourceRequirements(ResourceRequirements resourceRequirements) {
+		checkInit();
+		LOG.debug("Received resource requirements from job {}: {}", resourceRequirements.getJobId(), resourceRequirements.getResourceRequirements());
+
+		if (resourceRequirements.getResourceRequirements().isEmpty()) {
+			jobMasterTargetAddresses.remove(resourceRequirements.getJobId());
+		} else {
+			jobMasterTargetAddresses.put(resourceRequirements.getJobId(), resourceRequirements.getTargetAddress());
+		}
+		resourceTracker.notifyResourceRequirements(resourceRequirements.getJobId(), resourceRequirements.getResourceRequirements());
+		checkResourceRequirements();
+	}
+
+	/**
+	 * 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
+	 * @return True if the task manager has not been registered before and is registered successfully; otherwise false
+	 */
+	@Override
+	public boolean registerTaskManager(final TaskExecutorConnection taskExecutorConnection, SlotReport initialSlotReport) {
+		checkInit();
+		LOG.debug("Registering task executor {} under {} at the slot manager.", taskExecutorConnection.getResourceID(), taskExecutorConnection.getInstanceID());
+
+		// we identify task managers by their instance id
+		if (taskExecutorManager.isTaskManagerRegistered(taskExecutorConnection.getInstanceID())) {
+			LOG.debug("Task executor {} was already registered.", taskExecutorConnection.getResourceID());
+			reportSlotStatus(taskExecutorConnection.getInstanceID(), initialSlotReport);
+			return false;
+		} else {
+			if (!taskExecutorManager.registerTaskManager(taskExecutorConnection, initialSlotReport)) {
+				LOG.debug("Task executor {} could not be registered.", taskExecutorConnection.getResourceID());
+				return false;
+			}
+
+			// register the new slots
+			for (SlotStatus slotStatus : initialSlotReport) {
+				slotTracker.addSlot(
+					slotStatus.getSlotID(),
+					slotStatus.getResourceProfile(),
+					taskExecutorConnection,
+					slotStatus.getJobID());
+			}
+
+			checkResourceRequirements();
+			return true;
+		}
+	}
+
+	@Override
+	public boolean unregisterTaskManager(InstanceID instanceId, Exception cause) {
+		checkInit();
+
+		LOG.debug("Unregistering task executor {} from the slot manager.", instanceId);
+
+		if (taskExecutorManager.isTaskManagerRegistered(instanceId)) {
+			slotTracker.removeSlots(taskExecutorManager.getSlotsOf(instanceId));
+			taskExecutorManager.unregisterTaskExecutor(instanceId);
+			checkResourceRequirements();
+
+			return true;
+		} else {
+			LOG.debug("There is no task executor registered with instance ID {}. Ignoring this message.", instanceId);
+
+			return false;
+		}
+	}
+
+	/**
+	 * Reports the current slot allocations for a task manager identified by the given instance id.
+	 *
+	 * @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
+	 */
+	@Override
+	public boolean reportSlotStatus(InstanceID instanceId, SlotReport slotReport) {
+		checkInit();
+
+		LOG.debug("Received slot report from instance {}: {}.", instanceId, slotReport);
+
+		if (taskExecutorManager.isTaskManagerRegistered(instanceId)) {
+			slotTracker.notifySlotStatus(slotReport);
+			checkResourceRequirements();
+			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
+	 */
+	@Override
+	public void freeSlot(SlotID slotId, AllocationID allocationId) {
+		checkInit();
+		LOG.debug("Freeing slot {}.", slotId);
+
+		slotTracker.notifyFree(slotId);
+		checkResourceRequirements();
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Requirement matching
+	// ---------------------------------------------------------------------------------------------
+
+	private void checkResourceRequirements() {
+		final Map<JobID, Collection<ResourceRequirement>> missingResources = resourceTracker.getMissingResources();
+		if (missingResources.isEmpty()) {
+			return;
+		}
+
+		final Map<JobID, ResourceCounter> outstandingRequirements = new LinkedHashMap<>();
+		for (Map.Entry<JobID, Collection<ResourceRequirement>> resourceRequirements : missingResources.entrySet()) {
+			JobID jobId = resourceRequirements.getKey();
+
+			for (ResourceRequirement resourceRequirement : resourceRequirements.getValue()) {
+				int numMissingSlots = internalRequestSlots(jobId, jobMasterTargetAddresses.get(jobId), resourceRequirement);
+				if (numMissingSlots > 0) {
+					outstandingRequirements
+						.computeIfAbsent(jobId, ignored -> new ResourceCounter())
+						.incrementCount(resourceRequirement.getResourceProfile(), numMissingSlots);
+				}
+			}
+		}
+
+		final ResourceCounter pendingSlots = new ResourceCounter(taskExecutorManager.getPendingTaskManagerSlots().stream().collect(
+			Collectors.groupingBy(
+				PendingTaskManagerSlot::getResourceProfile,
+				Collectors.summingInt(x -> 1))));
+
+		for (Map.Entry<JobID, ResourceCounter> unfulfilledRequirement : outstandingRequirements.entrySet()) {
+			tryFulfillRequirementsWithPendingOrNewSlots(
+				unfulfilledRequirement.getKey(),
+				unfulfilledRequirement.getValue().getResourceProfilesWithCount(),
+				pendingSlots);
+		}
+	}
+
+	/**
+	 * Tries to allocate slots for the given requirement. If there are not enough slots available, the
+	 * resource manager is informed to allocate more resources.
+	 *
+	 * @param jobId job to allocate slots for
+	 * @param targetAddress address of the jobmaster
+	 * @param resourceRequirement required slots
+	 * @return the number of missing slots
+	 */
+	private int internalRequestSlots(JobID jobId, String targetAddress, ResourceRequirement resourceRequirement) {
+		final ResourceProfile requiredResource = resourceRequirement.getResourceProfile();
+		Collection<TaskManagerSlotInformation> freeSlots = slotTracker.getFreeSlots();
+
+		int numUnfulfilled = 0;
+		for (int x = 0; x < resourceRequirement.getNumberOfRequiredSlots(); x++) {
+
+			final Optional<TaskManagerSlotInformation> reservedSlot = slotMatchingStrategy.findMatchingSlot(requiredResource, freeSlots, this::getNumberRegisteredSlotsOf);
+			if (reservedSlot.isPresent()) {
+				// we do not need to modify freeSlots because it is indirectly modified by the allocation
+				allocateSlot(reservedSlot.get(), jobId, targetAddress, requiredResource);
+			} else {
+				numUnfulfilled++;
+			}
+		}
+		return numUnfulfilled;
+	}
+
+	/**
+	 * Allocates the given slot. This entails sending a registration message to the task manager and treating failures.
+	 *
+	 * @param taskManagerSlot slot to allocate
+	 * @param jobId job for which the slot should be allocated for
+	 * @param targetAddress address of the job master
+	 * @param resourceProfile resource profile for the requirement for which the slot is used
+	 */
+	private void allocateSlot(TaskManagerSlotInformation taskManagerSlot, JobID jobId, String targetAddress, ResourceProfile resourceProfile) {
+		TaskExecutorConnection taskExecutorConnection = taskManagerSlot.getTaskManagerConnection();
+		TaskExecutorGateway gateway = taskExecutorConnection.getTaskExecutorGateway();
+
+		final CompletableFuture<Acknowledge> completableFuture = new CompletableFuture<>();
+		final SlotID slotId = taskManagerSlot.getSlotId();
+		final InstanceID instanceId = taskManagerSlot.getInstanceId();
+
+		LOG.debug("Starting allocation of slot {} for job {} with resource profile {}.", taskManagerSlot.getSlotId(), jobId, resourceProfile);
+
+		slotTracker.notifyAllocationStart(slotId, jobId);
+
+		if (!taskExecutorManager.isTaskManagerRegistered(instanceId)) {
+			throw new IllegalStateException("Could not find a registered task manager for instance id " +
+				instanceId + '.');
+		}
+
+		taskExecutorManager.markUsed(instanceId);
+
+		// RPC call to the task manager
+		CompletableFuture<Acknowledge> requestFuture = gateway.requestSlot(
+			slotId,
+			jobId,
+			new AllocationID(),
+			resourceProfile,
+			targetAddress,
+			resourceManagerId,
+			taskManagerRequestTimeout);
+
+		requestFuture.whenComplete(
+			(Acknowledge acknowledge, Throwable throwable) -> {
+				if (acknowledge != null) {
+					completableFuture.complete(acknowledge);
+				} else {
+					completableFuture.completeExceptionally(throwable);
+				}
+			});
+
+		completableFuture.whenCompleteAsync(
+			(Acknowledge acknowledge, Throwable throwable) -> {
+				try {
+					if (acknowledge != null) {
+						LOG.trace("Completed allocation of slot {} for job {}.", slotId, jobId);
+						slotTracker.notifyAllocationComplete(slotId, jobId);
+					} else {
+						if (throwable instanceof SlotOccupiedException) {
+							SlotOccupiedException exception = (SlotOccupiedException) throwable;
+							LOG.debug("Tried allocating slot {} for job {}, but it was already allocated for job {}.", slotId, jobId, exception.getJobId());
+							// report as a slot status to force the state transition
+							// this could be a problem if we ever assume that the task executor always reports about all slots
+							slotTracker.notifySlotStatus(Collections.singleton(new SlotStatus(slotId, taskManagerSlot.getResourceProfile(), exception.getJobId(), exception.getAllocationId())));
+						} else {
+							if (throwable instanceof CancellationException) {
+								LOG.debug("Cancelled allocation of slot {} for job {}.", slotId, jobId, throwable);
+							} else {
+								LOG.warn("Slot allocation for slot {} for job {} failed.", slotId, jobId, throwable);
+								slotTracker.notifyFree(slotId);
+							}
+						}
+					}
+					checkResourceRequirements();
+				} catch (Exception e) {
+					LOG.error("Error while completing the slot allocation.", e);
+				}
+			},
+			mainThreadExecutor);
+	}
+
+	private void tryFulfillRequirementsWithPendingOrNewSlots(JobID jobId, Map<ResourceProfile, Integer> missingResources, ResourceCounter pendingSlots) {

Review comment:
       yes




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] zentol commented on a change in pull request #13553: [FLINK-19306][coordination] Add DeclarativeSlotManager

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #13553:
URL: https://github.com/apache/flink/pull/13553#discussion_r502332686



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManager.java
##########
@@ -0,0 +1,609 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.metrics.MetricNames;
+import org.apache.flink.runtime.metrics.groups.SlotManagerMetricGroup;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerId;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.slots.ResourceCounter;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+import org.apache.flink.runtime.slots.ResourceRequirements;
+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.SlotOccupiedException;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ScheduledFuture;
+import java.util.function.BiFunction;
+import java.util.stream.Collectors;
+
+/**
+ * Implementation of {@link SlotManager} supporting declarative slot management.
+ */
+public class DeclarativeSlotManager implements SlotManager {
+	private static final Logger LOG = LoggerFactory.getLogger(DeclarativeSlotManager.class);
+
+	private final SlotTracker slotTracker;
+	private final ResourceTracker resourceTracker;
+	private final BiFunction<Executor, ResourceActions, TaskExecutorManager> taskExecutorManagerFactory;
+	private TaskExecutorManager taskExecutorManager;
+
+	/** Timeout for slot requests to the task manager. */
+	private final Time taskManagerRequestTimeout;
+	private ScheduledFuture<?> slotRequestTimeoutCheck;
+
+	private final SlotMatchingStrategy slotMatchingStrategy;
+
+	private final SlotManagerMetricGroup slotManagerMetricGroup;
+
+	private final Map<JobID, String> jobMasterTargetAddresses = new HashMap<>();
+	private final HashMap<SlotID, CompletableFuture<Acknowledge>> pendingSlotAllocationFutures;
+
+	/** ResourceManager's id. */
+	private ResourceManagerId resourceManagerId;
+
+	/** Executor for future callbacks which have to be "synchronized". */
+	private Executor mainThreadExecutor;
+
+	/** Callbacks for resource (de-)allocations. */
+	private ResourceActions resourceActions;
+
+	/** True iff the component has been started. */
+	private boolean started;
+
+	public DeclarativeSlotManager(
+			ScheduledExecutor scheduledExecutor,
+			SlotManagerConfiguration slotManagerConfiguration,
+			SlotManagerMetricGroup slotManagerMetricGroup,
+			ResourceTracker resourceTracker,
+			SlotTracker slotTracker) {
+
+		Preconditions.checkNotNull(slotManagerConfiguration);
+		this.taskManagerRequestTimeout = slotManagerConfiguration.getTaskManagerRequestTimeout();
+		this.slotManagerMetricGroup = Preconditions.checkNotNull(slotManagerMetricGroup);
+		this.resourceTracker = Preconditions.checkNotNull(resourceTracker);
+
+		pendingSlotAllocationFutures = new HashMap<>(16);
+
+		this.slotTracker = Preconditions.checkNotNull(slotTracker);
+		slotTracker.registerSlotStatusUpdateListener(createSlotStatusUpdateListener());
+
+		slotMatchingStrategy = slotManagerConfiguration.getSlotMatchingStrategy();
+
+		taskExecutorManagerFactory = (executor, resourceActions) -> new TaskExecutorManager(
+			slotManagerConfiguration.getDefaultWorkerResourceSpec(),
+			slotManagerConfiguration.getNumSlotsPerWorker(),
+			slotManagerConfiguration.getMaxSlotNum(),
+			slotManagerConfiguration.isWaitResultConsumedBeforeRelease(),
+			slotManagerConfiguration.getRedundantTaskManagerNum(),
+			slotManagerConfiguration.getTaskManagerTimeout(),
+			scheduledExecutor,
+			executor,
+			resourceActions);
+
+		resourceManagerId = null;
+		resourceActions = null;
+		mainThreadExecutor = null;
+		slotRequestTimeoutCheck = null;
+		taskExecutorManager = null;
+
+		started = false;
+	}
+
+	private SlotStatusUpdateListener createSlotStatusUpdateListener() {
+		return (taskManagerSlot, previous, current, jobId) -> {
+			if (previous == SlotState.PENDING) {
+				cancelAllocationFuture(taskManagerSlot.getSlotId());
+			}
+
+			if (current == SlotState.PENDING) {
+				resourceTracker.notifyAcquiredResource(jobId, taskManagerSlot.getResourceProfile());
+			}
+			if (current == SlotState.FREE) {
+				resourceTracker.notifyLostResource(jobId, taskManagerSlot.getResourceProfile());
+			}
+
+			if (current == SlotState.ALLOCATED) {
+				taskExecutorManager.occupySlot(taskManagerSlot.getInstanceId());
+			}
+			if (previous == SlotState.ALLOCATED && current == SlotState.FREE) {
+				taskExecutorManager.freeSlot(taskManagerSlot.getInstanceId());
+			}
+		};
+	}
+
+	private void cancelAllocationFuture(SlotID slotId) {
+		final CompletableFuture<Acknowledge> acknowledgeCompletableFuture = pendingSlotAllocationFutures.remove(slotId);

Review comment:
       After all we aren't really doing anything on cancellation, although it might be useful for debugging purposes?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] zentol commented on a change in pull request #13553: [FLINK-19306][coordination] Add DeclarativeSlotManager

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #13553:
URL: https://github.com/apache/flink/pull/13553#discussion_r502360960



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManager.java
##########
@@ -0,0 +1,609 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.metrics.MetricNames;
+import org.apache.flink.runtime.metrics.groups.SlotManagerMetricGroup;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerId;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.slots.ResourceCounter;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+import org.apache.flink.runtime.slots.ResourceRequirements;
+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.SlotOccupiedException;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ScheduledFuture;
+import java.util.function.BiFunction;
+import java.util.stream.Collectors;
+
+/**
+ * Implementation of {@link SlotManager} supporting declarative slot management.
+ */
+public class DeclarativeSlotManager implements SlotManager {
+	private static final Logger LOG = LoggerFactory.getLogger(DeclarativeSlotManager.class);
+
+	private final SlotTracker slotTracker;
+	private final ResourceTracker resourceTracker;
+	private final BiFunction<Executor, ResourceActions, TaskExecutorManager> taskExecutorManagerFactory;
+	private TaskExecutorManager taskExecutorManager;
+
+	/** Timeout for slot requests to the task manager. */
+	private final Time taskManagerRequestTimeout;
+	private ScheduledFuture<?> slotRequestTimeoutCheck;
+
+	private final SlotMatchingStrategy slotMatchingStrategy;
+
+	private final SlotManagerMetricGroup slotManagerMetricGroup;
+
+	private final Map<JobID, String> jobMasterTargetAddresses = new HashMap<>();
+	private final HashMap<SlotID, CompletableFuture<Acknowledge>> pendingSlotAllocationFutures;
+
+	/** ResourceManager's id. */
+	private ResourceManagerId resourceManagerId;
+
+	/** Executor for future callbacks which have to be "synchronized". */
+	private Executor mainThreadExecutor;
+
+	/** Callbacks for resource (de-)allocations. */
+	private ResourceActions resourceActions;
+
+	/** True iff the component has been started. */
+	private boolean started;
+
+	public DeclarativeSlotManager(
+			ScheduledExecutor scheduledExecutor,
+			SlotManagerConfiguration slotManagerConfiguration,
+			SlotManagerMetricGroup slotManagerMetricGroup,
+			ResourceTracker resourceTracker,
+			SlotTracker slotTracker) {
+
+		Preconditions.checkNotNull(slotManagerConfiguration);
+		this.taskManagerRequestTimeout = slotManagerConfiguration.getTaskManagerRequestTimeout();
+		this.slotManagerMetricGroup = Preconditions.checkNotNull(slotManagerMetricGroup);
+		this.resourceTracker = Preconditions.checkNotNull(resourceTracker);
+
+		pendingSlotAllocationFutures = new HashMap<>(16);
+
+		this.slotTracker = Preconditions.checkNotNull(slotTracker);
+		slotTracker.registerSlotStatusUpdateListener(createSlotStatusUpdateListener());
+
+		slotMatchingStrategy = slotManagerConfiguration.getSlotMatchingStrategy();
+
+		taskExecutorManagerFactory = (executor, resourceActions) -> new TaskExecutorManager(
+			slotManagerConfiguration.getDefaultWorkerResourceSpec(),
+			slotManagerConfiguration.getNumSlotsPerWorker(),
+			slotManagerConfiguration.getMaxSlotNum(),
+			slotManagerConfiguration.isWaitResultConsumedBeforeRelease(),
+			slotManagerConfiguration.getRedundantTaskManagerNum(),
+			slotManagerConfiguration.getTaskManagerTimeout(),
+			scheduledExecutor,
+			executor,
+			resourceActions);
+
+		resourceManagerId = null;
+		resourceActions = null;
+		mainThreadExecutor = null;
+		slotRequestTimeoutCheck = null;
+		taskExecutorManager = null;
+
+		started = false;
+	}
+
+	private SlotStatusUpdateListener createSlotStatusUpdateListener() {
+		return (taskManagerSlot, previous, current, jobId) -> {
+			if (previous == SlotState.PENDING) {
+				cancelAllocationFuture(taskManagerSlot.getSlotId());
+			}
+
+			if (current == SlotState.PENDING) {
+				resourceTracker.notifyAcquiredResource(jobId, taskManagerSlot.getResourceProfile());
+			}
+			if (current == SlotState.FREE) {
+				resourceTracker.notifyLostResource(jobId, taskManagerSlot.getResourceProfile());
+			}
+
+			if (current == SlotState.ALLOCATED) {
+				taskExecutorManager.occupySlot(taskManagerSlot.getInstanceId());
+			}
+			if (previous == SlotState.ALLOCATED && current == SlotState.FREE) {
+				taskExecutorManager.freeSlot(taskManagerSlot.getInstanceId());
+			}
+		};
+	}
+
+	private void cancelAllocationFuture(SlotID slotId) {
+		final CompletableFuture<Acknowledge> acknowledgeCompletableFuture = pendingSlotAllocationFutures.remove(slotId);
+		// the future may be null if we are just re-playing the state transitions due to a slot report
+		if (acknowledgeCompletableFuture != null) {
+			acknowledgeCompletableFuture.cancel(false);
+		}
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Component lifecycle methods
+	// ---------------------------------------------------------------------------------------------
+
+	/**
+	 * Starts the slot manager with the given leader id and resource manager actions.
+	 *
+	 * @param newResourceManagerId to use for communication with the task managers
+	 * @param newMainThreadExecutor to use to run code in the ResourceManager's main thread
+	 * @param newResourceActions to use for resource (de-)allocations
+	 */
+	@Override
+	public void start(ResourceManagerId newResourceManagerId, Executor newMainThreadExecutor, ResourceActions newResourceActions) {
+		LOG.info("Starting the slot manager.");
+
+		this.resourceManagerId = Preconditions.checkNotNull(newResourceManagerId);
+		mainThreadExecutor = Preconditions.checkNotNull(newMainThreadExecutor);
+		resourceActions = Preconditions.checkNotNull(newResourceActions);
+		taskExecutorManager = taskExecutorManagerFactory.apply(newMainThreadExecutor, newResourceActions);
+
+		started = true;
+
+		registerSlotManagerMetrics();
+	}
+
+	private void registerSlotManagerMetrics() {
+		slotManagerMetricGroup.gauge(
+			MetricNames.TASK_SLOTS_AVAILABLE,
+			() -> (long) getNumberFreeSlots());
+		slotManagerMetricGroup.gauge(
+			MetricNames.TASK_SLOTS_TOTAL,
+			() -> (long) getNumberRegisteredSlots());
+	}
+
+	/**
+	 * Suspends the component. This clears the internal state of the slot manager.
+	 */
+	@Override
+	public void suspend() {
+		LOG.info("Suspending the slot manager.");
+
+		if (slotRequestTimeoutCheck != null) {
+			slotRequestTimeoutCheck.cancel(false);
+			slotRequestTimeoutCheck = null;
+		}
+
+		resourceTracker.clear();
+		taskExecutorManager.close();
+
+		for (InstanceID registeredTaskManager : taskExecutorManager.getTaskExecutors()) {
+			unregisterTaskManager(registeredTaskManager, new SlotManagerException("The slot manager is being suspended."));
+		}
+
+		taskExecutorManager = null;
+		resourceManagerId = null;
+		resourceActions = null;
+		started = false;
+	}
+
+	/**
+	 * Closes the slot manager.
+	 *
+	 * @throws Exception if the close operation fails
+	 */
+	@Override
+	public void close() throws Exception {
+		LOG.info("Closing the slot manager.");
+
+		suspend();
+		slotManagerMetricGroup.close();
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Public API
+	// ---------------------------------------------------------------------------------------------
+
+	@Override
+	public void processResourceRequirements(ResourceRequirements resourceRequirements) {
+		checkInit();
+		LOG.debug("Received resource requirements from job {}: {}", resourceRequirements.getJobId(), resourceRequirements.getResourceRequirements());
+
+		if (resourceRequirements.getResourceRequirements().isEmpty()) {
+			jobMasterTargetAddresses.remove(resourceRequirements.getJobId());
+		} else {
+			jobMasterTargetAddresses.put(resourceRequirements.getJobId(), resourceRequirements.getTargetAddress());
+		}
+		resourceTracker.notifyResourceRequirements(resourceRequirements.getJobId(), resourceRequirements.getResourceRequirements());
+		checkResourceRequirements();
+	}
+
+	/**
+	 * 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
+	 * @return True if the task manager has not been registered before and is registered successfully; otherwise false
+	 */
+	@Override
+	public boolean registerTaskManager(final TaskExecutorConnection taskExecutorConnection, SlotReport initialSlotReport) {
+		checkInit();
+		LOG.debug("Registering task executor {} under {} at the slot manager.", taskExecutorConnection.getResourceID(), taskExecutorConnection.getInstanceID());
+
+		// we identify task managers by their instance id
+		if (taskExecutorManager.isTaskManagerRegistered(taskExecutorConnection.getInstanceID())) {
+			LOG.debug("Task executor {} was already registered.", taskExecutorConnection.getResourceID());
+			reportSlotStatus(taskExecutorConnection.getInstanceID(), initialSlotReport);
+			return false;
+		} else {
+			if (!taskExecutorManager.registerTaskManager(taskExecutorConnection, initialSlotReport)) {
+				LOG.debug("Task executor {} could not be registered.", taskExecutorConnection.getResourceID());
+				return false;
+			}
+
+			// register the new slots
+			for (SlotStatus slotStatus : initialSlotReport) {
+				slotTracker.addSlot(
+					slotStatus.getSlotID(),
+					slotStatus.getResourceProfile(),
+					taskExecutorConnection,
+					slotStatus.getJobID());
+			}
+
+			checkResourceRequirements();
+			return true;
+		}
+	}
+
+	@Override
+	public boolean unregisterTaskManager(InstanceID instanceId, Exception cause) {
+		checkInit();
+
+		LOG.debug("Unregistering task executor {} from the slot manager.", instanceId);
+
+		if (taskExecutorManager.isTaskManagerRegistered(instanceId)) {
+			slotTracker.removeSlots(taskExecutorManager.getSlotsOf(instanceId));
+			taskExecutorManager.unregisterTaskExecutor(instanceId);
+			checkResourceRequirements();
+
+			return true;
+		} else {
+			LOG.debug("There is no task executor registered with instance ID {}. Ignoring this message.", instanceId);
+
+			return false;
+		}
+	}
+
+	/**
+	 * Reports the current slot allocations for a task manager identified by the given instance id.
+	 *
+	 * @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
+	 */
+	@Override
+	public boolean reportSlotStatus(InstanceID instanceId, SlotReport slotReport) {
+		checkInit();
+
+		LOG.debug("Received slot report from instance {}: {}.", instanceId, slotReport);
+
+		if (taskExecutorManager.isTaskManagerRegistered(instanceId)) {
+			slotTracker.notifySlotStatus(slotReport);
+			checkResourceRequirements();
+			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
+	 */
+	@Override
+	public void freeSlot(SlotID slotId, AllocationID allocationId) {
+		checkInit();
+		LOG.debug("Freeing slot {}.", slotId);
+
+		slotTracker.notifyFree(slotId);
+		checkResourceRequirements();
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Requirement matching
+	// ---------------------------------------------------------------------------------------------
+
+	private void checkResourceRequirements() {
+		final Map<JobID, Collection<ResourceRequirement>> missingResources = resourceTracker.getMissingResources();
+		if (missingResources.isEmpty()) {
+			return;
+		}
+
+		final Map<JobID, ResourceCounter> outstandingRequirements = new LinkedHashMap<>();
+		for (Map.Entry<JobID, Collection<ResourceRequirement>> resourceRequirements : missingResources.entrySet()) {
+			JobID jobId = resourceRequirements.getKey();
+
+			for (ResourceRequirement resourceRequirement : resourceRequirements.getValue()) {
+				int numMissingSlots = internalRequestSlots(jobId, jobMasterTargetAddresses.get(jobId), resourceRequirement);
+				if (numMissingSlots > 0) {
+					outstandingRequirements
+						.computeIfAbsent(jobId, ignored -> new ResourceCounter())
+						.incrementCount(resourceRequirement.getResourceProfile(), numMissingSlots);
+				}
+			}
+		}
+
+		final ResourceCounter pendingSlots = new ResourceCounter(taskExecutorManager.getPendingTaskManagerSlots().stream().collect(
+			Collectors.groupingBy(
+				PendingTaskManagerSlot::getResourceProfile,
+				Collectors.summingInt(x -> 1))));
+
+		for (Map.Entry<JobID, ResourceCounter> unfulfilledRequirement : outstandingRequirements.entrySet()) {
+			tryFulfillRequirementsWithPendingOrNewSlots(
+				unfulfilledRequirement.getKey(),
+				unfulfilledRequirement.getValue().getResourceProfilesWithCount(),
+				pendingSlots);
+		}
+	}
+
+	/**
+	 * Tries to allocate slots for the given requirement. If there are not enough slots available, the
+	 * resource manager is informed to allocate more resources.
+	 *
+	 * @param jobId job to allocate slots for
+	 * @param targetAddress address of the jobmaster
+	 * @param resourceRequirement required slots
+	 * @return the number of missing slots
+	 */
+	private int internalRequestSlots(JobID jobId, String targetAddress, ResourceRequirement resourceRequirement) {
+		final ResourceProfile requiredResource = resourceRequirement.getResourceProfile();
+		Collection<TaskManagerSlotInformation> freeSlots = slotTracker.getFreeSlots();
+
+		int numUnfulfilled = 0;
+		for (int x = 0; x < resourceRequirement.getNumberOfRequiredSlots(); x++) {
+
+			final Optional<TaskManagerSlotInformation> reservedSlot = slotMatchingStrategy.findMatchingSlot(requiredResource, freeSlots, this::getNumberRegisteredSlotsOf);
+			if (reservedSlot.isPresent()) {
+				// we do not need to modify freeSlots because it is indirectly modified by the allocation
+				allocateSlot(reservedSlot.get(), jobId, targetAddress, requiredResource);
+			} else {
+				numUnfulfilled++;
+			}
+		}
+		return numUnfulfilled;
+	}
+
+	/**
+	 * Allocates the given slot. This entails sending a registration message to the task manager and treating failures.
+	 *
+	 * @param taskManagerSlot slot to allocate
+	 * @param jobId job for which the slot should be allocated for
+	 * @param targetAddress address of the job master
+	 * @param resourceProfile resource profile for the requirement for which the slot is used
+	 */
+	private void allocateSlot(TaskManagerSlotInformation taskManagerSlot, JobID jobId, String targetAddress, ResourceProfile resourceProfile) {
+		TaskExecutorConnection taskExecutorConnection = taskManagerSlot.getTaskManagerConnection();
+		TaskExecutorGateway gateway = taskExecutorConnection.getTaskExecutorGateway();
+
+		final CompletableFuture<Acknowledge> completableFuture = new CompletableFuture<>();
+		final SlotID slotId = taskManagerSlot.getSlotId();
+		final InstanceID instanceId = taskManagerSlot.getInstanceId();
+
+		LOG.debug("Starting allocation of slot {} for job {} with resource profile {}.", taskManagerSlot.getSlotId(), jobId, resourceProfile);
+
+		slotTracker.notifyAllocationStart(slotId, jobId);
+
+		if (!taskExecutorManager.isTaskManagerRegistered(instanceId)) {
+			throw new IllegalStateException("Could not find a registered task manager for instance id " +
+				instanceId + '.');
+		}
+
+		taskExecutorManager.markUsed(instanceId);
+
+		// RPC call to the task manager
+		CompletableFuture<Acknowledge> requestFuture = gateway.requestSlot(
+			slotId,
+			jobId,
+			new AllocationID(),
+			resourceProfile,
+			targetAddress,
+			resourceManagerId,
+			taskManagerRequestTimeout);
+
+		requestFuture.whenComplete(
+			(Acknowledge acknowledge, Throwable throwable) -> {
+				if (acknowledge != null) {
+					completableFuture.complete(acknowledge);
+				} else {
+					completableFuture.completeExceptionally(throwable);
+				}
+			});
+
+		completableFuture.whenCompleteAsync(
+			(Acknowledge acknowledge, Throwable throwable) -> {
+				try {
+					if (acknowledge != null) {
+						LOG.trace("Completed allocation of slot {} for job {}.", slotId, jobId);
+						slotTracker.notifyAllocationComplete(slotId, jobId);
+					} else {
+						if (throwable instanceof SlotOccupiedException) {
+							SlotOccupiedException exception = (SlotOccupiedException) throwable;
+							LOG.debug("Tried allocating slot {} for job {}, but it was already allocated for job {}.", slotId, jobId, exception.getJobId());
+							// report as a slot status to force the state transition
+							// this could be a problem if we ever assume that the task executor always reports about all slots
+							slotTracker.notifySlotStatus(Collections.singleton(new SlotStatus(slotId, taskManagerSlot.getResourceProfile(), exception.getJobId(), exception.getAllocationId())));
+						} else {
+							if (throwable instanceof CancellationException) {
+								LOG.debug("Cancelled allocation of slot {} for job {}.", slotId, jobId, throwable);
+							} else {
+								LOG.warn("Slot allocation for slot {} for job {} failed.", slotId, jobId, throwable);
+								slotTracker.notifyFree(slotId);
+							}
+						}
+					}
+					checkResourceRequirements();

Review comment:
       I can't quite think of a reason at the moment to check the requirements after a successful allocation




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] zentol commented on pull request #13553: [FLINK-19306][coordination] Add DeclarativeSlotManager

Posted by GitBox <gi...@apache.org>.
zentol commented on pull request #13553:
URL: https://github.com/apache/flink/pull/13553#issuecomment-705384887


   @flinkbot run azure


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13553: [FLINK-19306][coordination] Add DeclarativeSlotManager

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13553:
URL: https://github.com/apache/flink/pull/13553#issuecomment-704936985


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "15d4c353c4c0a646b2bdf34bbedbd81359cb5b5f",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7264",
       "triggerID" : "15d4c353c4c0a646b2bdf34bbedbd81359cb5b5f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 15d4c353c4c0a646b2bdf34bbedbd81359cb5b5f Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7264) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] tillrohrmann commented on a change in pull request #13553: [FLINK-19306][coordination] Add DeclarativeSlotManager

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #13553:
URL: https://github.com/apache/flink/pull/13553#discussion_r502532564



##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManagerTest.java
##########
@@ -0,0 +1,1092 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple6;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.concurrent.ManuallyTriggeredScheduledExecutor;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerId;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+import org.apache.flink.runtime.slots.ResourceRequirements;
+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.TestingTaskExecutorGateway;
+import org.apache.flink.runtime.taskexecutor.TestingTaskExecutorGatewayBuilder;
+import org.apache.flink.runtime.taskexecutor.exceptions.SlotAllocationException;
+import org.apache.flink.runtime.taskexecutor.exceptions.SlotOccupiedException;
+import org.apache.flink.runtime.testingUtils.TestingUtils;
+import org.apache.flink.util.FlinkException;
+import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.function.FunctionUtils;
+
+import akka.pattern.AskTimeoutException;
+import org.junit.Ignore;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.hamcrest.CoreMatchers.hasItem;
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.empty;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.hasSize;
+import static org.hamcrest.Matchers.is;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertSame;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Tests for the {@link DeclarativeSlotManager}.
+ */
+public class DeclarativeSlotManagerTest extends TestLogger {
+
+	private static final FlinkException TEST_EXCEPTION = new FlinkException("Test exception");
+
+	private static final WorkerResourceSpec WORKER_RESOURCE_SPEC = new WorkerResourceSpec.Builder()
+		.setCpuCores(100.0)
+		.setTaskHeapMemoryMB(10000)
+		.setTaskOffHeapMemoryMB(10000)
+		.setNetworkMemoryMB(10000)
+		.setManagedMemoryMB(10000)
+		.build();
+
+	/**
+	 * Tests that we can register task manager and their slots at the slot manager.
+	 */
+	@Test
+	public void testTaskManagerRegistration() throws Exception {
+		final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder().createTestingTaskExecutorGateway();
+		final ResourceID resourceId = ResourceID.generate();
+		final TaskExecutorConnection taskManagerConnection = new TaskExecutorConnection(resourceId, taskExecutorGateway);
+
+		final SlotID slotId1 = new SlotID(resourceId, 0);
+		final SlotID slotId2 = new SlotID(resourceId, 1);
+		final SlotReport slotReport = new SlotReport(Arrays.asList(createFreeSlotStatus(slotId1), createFreeSlotStatus(slotId2)));
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport);
+
+			assertThat("The number registered slots does not equal the expected number.", slotManager.getNumberRegisteredSlots(), is(2));
+
+			assertNotNull(slotTracker.getSlot(slotId1));
+			assertNotNull(slotTracker.getSlot(slotId2));
+		}
+	}
+
+	/**
+	 * Tests that un-registration of task managers will free and remove all registered slots.
+	 */
+	@Test
+	public void testTaskManagerUnregistration() throws Exception {
+		final TaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder()
+			.setRequestSlotFunction(tuple6 -> new CompletableFuture<>())
+			.createTestingTaskExecutorGateway();
+		final TaskExecutorConnection taskManagerConnection = createTaskExecutorConnection(taskExecutorGateway);
+		final ResourceID resourceId = taskManagerConnection.getResourceID();
+
+		final SlotID slotId1 = new SlotID(resourceId, 0);
+		final SlotID slotId2 = new SlotID(resourceId, 1);
+		final SlotReport slotReport = new SlotReport(Arrays.asList(createAllocatedSlotStatus(slotId1), createFreeSlotStatus(slotId2)));
+
+		final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot();
+
+		final DefaultSlotTracker slotTracker = new DefaultSlotTracker();
+
+		try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder()
+			.setSlotTracker(slotTracker)
+			.buildAndStartWithDirectExec()) {
+
+			slotManager.registerTaskManager(taskManagerConnection, slotReport);
+
+			assertEquals("The number registered slots does not equal the expected number.", 2, slotManager.getNumberRegisteredSlots());
+
+			DeclarativeTaskManagerSlot slot1 = slotTracker.getSlot(slotId1);
+			DeclarativeTaskManagerSlot slot2 = slotTracker.getSlot(slotId2);
+
+			assertSame(SlotState.ALLOCATED, slot1.getState());
+			assertSame(SlotState.FREE, slot2.getState());
+
+			slotManager.processResourceRequirements(resourceRequirements);
+
+			assertSame(SlotState.PENDING, slot2.getState());
+
+			slotManager.unregisterTaskManager(taskManagerConnection.getInstanceID(), TEST_EXCEPTION);
+
+			assertEquals(0, slotManager.getNumberRegisteredSlots());
+		}
+	}
+
+	/**
+	 * Tests that a slot request with no free slots will trigger the resource allocation.
+	 */
+	@Test
+	public void testSlotRequestWithoutFreeSlots() throws Exception {
+		final ResourceManagerId resourceManagerId = ResourceManagerId.generate();
+
+		final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot();
+
+		CompletableFuture<WorkerResourceSpec> allocateResourceFuture = new CompletableFuture<>();
+		ResourceActions resourceManagerActions = new TestingResourceActionsBuilder()
+			.setAllocateResourceConsumer(allocateResourceFuture::complete)
+			.build();
+
+		try (SlotManager slotManager = createSlotManager(resourceManagerId, resourceManagerActions)) {
+
+			slotManager.processResourceRequirements(resourceRequirements);
+
+			allocateResourceFuture.get();

Review comment:
       Ok sounds good.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] tillrohrmann commented on a change in pull request #13553: [FLINK-19306][coordination] Add DeclarativeSlotManager

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #13553:
URL: https://github.com/apache/flink/pull/13553#discussion_r502528191



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManager.java
##########
@@ -0,0 +1,609 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.metrics.MetricNames;
+import org.apache.flink.runtime.metrics.groups.SlotManagerMetricGroup;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerId;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.slots.ResourceCounter;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+import org.apache.flink.runtime.slots.ResourceRequirements;
+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.SlotOccupiedException;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ScheduledFuture;
+import java.util.function.BiFunction;
+import java.util.stream.Collectors;
+
+/**
+ * Implementation of {@link SlotManager} supporting declarative slot management.
+ */
+public class DeclarativeSlotManager implements SlotManager {
+	private static final Logger LOG = LoggerFactory.getLogger(DeclarativeSlotManager.class);
+
+	private final SlotTracker slotTracker;
+	private final ResourceTracker resourceTracker;
+	private final BiFunction<Executor, ResourceActions, TaskExecutorManager> taskExecutorManagerFactory;
+	private TaskExecutorManager taskExecutorManager;
+
+	/** Timeout for slot requests to the task manager. */
+	private final Time taskManagerRequestTimeout;
+	private ScheduledFuture<?> slotRequestTimeoutCheck;
+
+	private final SlotMatchingStrategy slotMatchingStrategy;
+
+	private final SlotManagerMetricGroup slotManagerMetricGroup;
+
+	private final Map<JobID, String> jobMasterTargetAddresses = new HashMap<>();
+	private final HashMap<SlotID, CompletableFuture<Acknowledge>> pendingSlotAllocationFutures;
+
+	/** ResourceManager's id. */
+	private ResourceManagerId resourceManagerId;
+
+	/** Executor for future callbacks which have to be "synchronized". */
+	private Executor mainThreadExecutor;
+
+	/** Callbacks for resource (de-)allocations. */
+	private ResourceActions resourceActions;
+
+	/** True iff the component has been started. */
+	private boolean started;
+
+	public DeclarativeSlotManager(
+			ScheduledExecutor scheduledExecutor,
+			SlotManagerConfiguration slotManagerConfiguration,
+			SlotManagerMetricGroup slotManagerMetricGroup,
+			ResourceTracker resourceTracker,
+			SlotTracker slotTracker) {
+
+		Preconditions.checkNotNull(slotManagerConfiguration);
+		this.taskManagerRequestTimeout = slotManagerConfiguration.getTaskManagerRequestTimeout();
+		this.slotManagerMetricGroup = Preconditions.checkNotNull(slotManagerMetricGroup);
+		this.resourceTracker = Preconditions.checkNotNull(resourceTracker);
+
+		pendingSlotAllocationFutures = new HashMap<>(16);
+
+		this.slotTracker = Preconditions.checkNotNull(slotTracker);
+		slotTracker.registerSlotStatusUpdateListener(createSlotStatusUpdateListener());
+
+		slotMatchingStrategy = slotManagerConfiguration.getSlotMatchingStrategy();
+
+		taskExecutorManagerFactory = (executor, resourceActions) -> new TaskExecutorManager(
+			slotManagerConfiguration.getDefaultWorkerResourceSpec(),
+			slotManagerConfiguration.getNumSlotsPerWorker(),
+			slotManagerConfiguration.getMaxSlotNum(),
+			slotManagerConfiguration.isWaitResultConsumedBeforeRelease(),
+			slotManagerConfiguration.getRedundantTaskManagerNum(),
+			slotManagerConfiguration.getTaskManagerTimeout(),
+			scheduledExecutor,
+			executor,
+			resourceActions);
+
+		resourceManagerId = null;
+		resourceActions = null;
+		mainThreadExecutor = null;
+		slotRequestTimeoutCheck = null;
+		taskExecutorManager = null;
+
+		started = false;
+	}
+
+	private SlotStatusUpdateListener createSlotStatusUpdateListener() {
+		return (taskManagerSlot, previous, current, jobId) -> {
+			if (previous == SlotState.PENDING) {
+				cancelAllocationFuture(taskManagerSlot.getSlotId());
+			}
+
+			if (current == SlotState.PENDING) {
+				resourceTracker.notifyAcquiredResource(jobId, taskManagerSlot.getResourceProfile());
+			}
+			if (current == SlotState.FREE) {
+				resourceTracker.notifyLostResource(jobId, taskManagerSlot.getResourceProfile());
+			}
+
+			if (current == SlotState.ALLOCATED) {
+				taskExecutorManager.occupySlot(taskManagerSlot.getInstanceId());
+			}
+			if (previous == SlotState.ALLOCATED && current == SlotState.FREE) {
+				taskExecutorManager.freeSlot(taskManagerSlot.getInstanceId());
+			}
+		};
+	}
+
+	private void cancelAllocationFuture(SlotID slotId) {
+		final CompletableFuture<Acknowledge> acknowledgeCompletableFuture = pendingSlotAllocationFutures.remove(slotId);
+		// the future may be null if we are just re-playing the state transitions due to a slot report
+		if (acknowledgeCompletableFuture != null) {
+			acknowledgeCompletableFuture.cancel(false);
+		}
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Component lifecycle methods
+	// ---------------------------------------------------------------------------------------------
+
+	/**
+	 * Starts the slot manager with the given leader id and resource manager actions.
+	 *
+	 * @param newResourceManagerId to use for communication with the task managers
+	 * @param newMainThreadExecutor to use to run code in the ResourceManager's main thread
+	 * @param newResourceActions to use for resource (de-)allocations
+	 */
+	@Override
+	public void start(ResourceManagerId newResourceManagerId, Executor newMainThreadExecutor, ResourceActions newResourceActions) {
+		LOG.info("Starting the slot manager.");
+
+		this.resourceManagerId = Preconditions.checkNotNull(newResourceManagerId);
+		mainThreadExecutor = Preconditions.checkNotNull(newMainThreadExecutor);
+		resourceActions = Preconditions.checkNotNull(newResourceActions);
+		taskExecutorManager = taskExecutorManagerFactory.apply(newMainThreadExecutor, newResourceActions);
+
+		started = true;
+
+		registerSlotManagerMetrics();
+	}
+
+	private void registerSlotManagerMetrics() {
+		slotManagerMetricGroup.gauge(
+			MetricNames.TASK_SLOTS_AVAILABLE,
+			() -> (long) getNumberFreeSlots());
+		slotManagerMetricGroup.gauge(
+			MetricNames.TASK_SLOTS_TOTAL,
+			() -> (long) getNumberRegisteredSlots());
+	}
+
+	/**
+	 * Suspends the component. This clears the internal state of the slot manager.
+	 */
+	@Override
+	public void suspend() {
+		LOG.info("Suspending the slot manager.");
+
+		if (slotRequestTimeoutCheck != null) {
+			slotRequestTimeoutCheck.cancel(false);
+			slotRequestTimeoutCheck = null;
+		}
+
+		resourceTracker.clear();
+		taskExecutorManager.close();
+
+		for (InstanceID registeredTaskManager : taskExecutorManager.getTaskExecutors()) {
+			unregisterTaskManager(registeredTaskManager, new SlotManagerException("The slot manager is being suspended."));
+		}
+
+		taskExecutorManager = null;
+		resourceManagerId = null;
+		resourceActions = null;
+		started = false;
+	}
+
+	/**
+	 * Closes the slot manager.
+	 *
+	 * @throws Exception if the close operation fails
+	 */
+	@Override
+	public void close() throws Exception {
+		LOG.info("Closing the slot manager.");
+
+		suspend();
+		slotManagerMetricGroup.close();
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Public API
+	// ---------------------------------------------------------------------------------------------
+
+	@Override
+	public void processResourceRequirements(ResourceRequirements resourceRequirements) {
+		checkInit();
+		LOG.debug("Received resource requirements from job {}: {}", resourceRequirements.getJobId(), resourceRequirements.getResourceRequirements());
+
+		if (resourceRequirements.getResourceRequirements().isEmpty()) {
+			jobMasterTargetAddresses.remove(resourceRequirements.getJobId());
+		} else {
+			jobMasterTargetAddresses.put(resourceRequirements.getJobId(), resourceRequirements.getTargetAddress());
+		}
+		resourceTracker.notifyResourceRequirements(resourceRequirements.getJobId(), resourceRequirements.getResourceRequirements());
+		checkResourceRequirements();
+	}
+
+	/**
+	 * 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
+	 * @return True if the task manager has not been registered before and is registered successfully; otherwise false
+	 */
+	@Override
+	public boolean registerTaskManager(final TaskExecutorConnection taskExecutorConnection, SlotReport initialSlotReport) {
+		checkInit();
+		LOG.debug("Registering task executor {} under {} at the slot manager.", taskExecutorConnection.getResourceID(), taskExecutorConnection.getInstanceID());
+
+		// we identify task managers by their instance id
+		if (taskExecutorManager.isTaskManagerRegistered(taskExecutorConnection.getInstanceID())) {
+			LOG.debug("Task executor {} was already registered.", taskExecutorConnection.getResourceID());
+			reportSlotStatus(taskExecutorConnection.getInstanceID(), initialSlotReport);
+			return false;
+		} else {
+			if (!taskExecutorManager.registerTaskManager(taskExecutorConnection, initialSlotReport)) {
+				LOG.debug("Task executor {} could not be registered.", taskExecutorConnection.getResourceID());
+				return false;
+			}
+
+			// register the new slots
+			for (SlotStatus slotStatus : initialSlotReport) {
+				slotTracker.addSlot(
+					slotStatus.getSlotID(),
+					slotStatus.getResourceProfile(),
+					taskExecutorConnection,
+					slotStatus.getJobID());
+			}
+
+			checkResourceRequirements();
+			return true;
+		}
+	}
+
+	@Override
+	public boolean unregisterTaskManager(InstanceID instanceId, Exception cause) {
+		checkInit();
+
+		LOG.debug("Unregistering task executor {} from the slot manager.", instanceId);
+
+		if (taskExecutorManager.isTaskManagerRegistered(instanceId)) {
+			slotTracker.removeSlots(taskExecutorManager.getSlotsOf(instanceId));
+			taskExecutorManager.unregisterTaskExecutor(instanceId);
+			checkResourceRequirements();
+
+			return true;
+		} else {
+			LOG.debug("There is no task executor registered with instance ID {}. Ignoring this message.", instanceId);
+
+			return false;
+		}
+	}
+
+	/**
+	 * Reports the current slot allocations for a task manager identified by the given instance id.
+	 *
+	 * @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
+	 */
+	@Override
+	public boolean reportSlotStatus(InstanceID instanceId, SlotReport slotReport) {
+		checkInit();
+
+		LOG.debug("Received slot report from instance {}: {}.", instanceId, slotReport);
+
+		if (taskExecutorManager.isTaskManagerRegistered(instanceId)) {
+			slotTracker.notifySlotStatus(slotReport);
+			checkResourceRequirements();
+			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
+	 */
+	@Override
+	public void freeSlot(SlotID slotId, AllocationID allocationId) {
+		checkInit();
+		LOG.debug("Freeing slot {}.", slotId);
+
+		slotTracker.notifyFree(slotId);
+		checkResourceRequirements();
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Requirement matching
+	// ---------------------------------------------------------------------------------------------
+
+	private void checkResourceRequirements() {
+		final Map<JobID, Collection<ResourceRequirement>> missingResources = resourceTracker.getMissingResources();
+		if (missingResources.isEmpty()) {
+			return;
+		}
+
+		final Map<JobID, ResourceCounter> outstandingRequirements = new LinkedHashMap<>();
+		for (Map.Entry<JobID, Collection<ResourceRequirement>> resourceRequirements : missingResources.entrySet()) {
+			JobID jobId = resourceRequirements.getKey();
+
+			for (ResourceRequirement resourceRequirement : resourceRequirements.getValue()) {
+				int numMissingSlots = internalRequestSlots(jobId, jobMasterTargetAddresses.get(jobId), resourceRequirement);
+				if (numMissingSlots > 0) {
+					outstandingRequirements
+						.computeIfAbsent(jobId, ignored -> new ResourceCounter())
+						.incrementCount(resourceRequirement.getResourceProfile(), numMissingSlots);
+				}
+			}
+		}
+
+		final ResourceCounter pendingSlots = new ResourceCounter(taskExecutorManager.getPendingTaskManagerSlots().stream().collect(
+			Collectors.groupingBy(
+				PendingTaskManagerSlot::getResourceProfile,
+				Collectors.summingInt(x -> 1))));
+
+		for (Map.Entry<JobID, ResourceCounter> unfulfilledRequirement : outstandingRequirements.entrySet()) {
+			tryFulfillRequirementsWithPendingOrNewSlots(
+				unfulfilledRequirement.getKey(),
+				unfulfilledRequirement.getValue().getResourceProfilesWithCount(),
+				pendingSlots);
+		}
+	}
+
+	/**
+	 * Tries to allocate slots for the given requirement. If there are not enough slots available, the
+	 * resource manager is informed to allocate more resources.
+	 *
+	 * @param jobId job to allocate slots for
+	 * @param targetAddress address of the jobmaster
+	 * @param resourceRequirement required slots
+	 * @return the number of missing slots
+	 */
+	private int internalRequestSlots(JobID jobId, String targetAddress, ResourceRequirement resourceRequirement) {
+		final ResourceProfile requiredResource = resourceRequirement.getResourceProfile();
+		Collection<TaskManagerSlotInformation> freeSlots = slotTracker.getFreeSlots();
+
+		int numUnfulfilled = 0;
+		for (int x = 0; x < resourceRequirement.getNumberOfRequiredSlots(); x++) {
+
+			final Optional<TaskManagerSlotInformation> reservedSlot = slotMatchingStrategy.findMatchingSlot(requiredResource, freeSlots, this::getNumberRegisteredSlotsOf);
+			if (reservedSlot.isPresent()) {
+				// we do not need to modify freeSlots because it is indirectly modified by the allocation
+				allocateSlot(reservedSlot.get(), jobId, targetAddress, requiredResource);
+			} else {
+				numUnfulfilled++;
+			}
+		}
+		return numUnfulfilled;
+	}
+
+	/**
+	 * Allocates the given slot. This entails sending a registration message to the task manager and treating failures.
+	 *
+	 * @param taskManagerSlot slot to allocate
+	 * @param jobId job for which the slot should be allocated for
+	 * @param targetAddress address of the job master
+	 * @param resourceProfile resource profile for the requirement for which the slot is used
+	 */
+	private void allocateSlot(TaskManagerSlotInformation taskManagerSlot, JobID jobId, String targetAddress, ResourceProfile resourceProfile) {
+		TaskExecutorConnection taskExecutorConnection = taskManagerSlot.getTaskManagerConnection();
+		TaskExecutorGateway gateway = taskExecutorConnection.getTaskExecutorGateway();
+
+		final CompletableFuture<Acknowledge> completableFuture = new CompletableFuture<>();
+		final SlotID slotId = taskManagerSlot.getSlotId();
+		final InstanceID instanceId = taskManagerSlot.getInstanceId();
+
+		LOG.debug("Starting allocation of slot {} for job {} with resource profile {}.", taskManagerSlot.getSlotId(), jobId, resourceProfile);
+
+		slotTracker.notifyAllocationStart(slotId, jobId);
+
+		if (!taskExecutorManager.isTaskManagerRegistered(instanceId)) {
+			throw new IllegalStateException("Could not find a registered task manager for instance id " +
+				instanceId + '.');
+		}
+
+		taskExecutorManager.markUsed(instanceId);
+
+		// RPC call to the task manager
+		CompletableFuture<Acknowledge> requestFuture = gateway.requestSlot(
+			slotId,
+			jobId,
+			new AllocationID(),
+			resourceProfile,
+			targetAddress,
+			resourceManagerId,
+			taskManagerRequestTimeout);
+
+		requestFuture.whenComplete(
+			(Acknowledge acknowledge, Throwable throwable) -> {
+				if (acknowledge != null) {
+					completableFuture.complete(acknowledge);
+				} else {
+					completableFuture.completeExceptionally(throwable);
+				}
+			});
+
+		completableFuture.whenCompleteAsync(
+			(Acknowledge acknowledge, Throwable throwable) -> {
+				try {
+					if (acknowledge != null) {
+						LOG.trace("Completed allocation of slot {} for job {}.", slotId, jobId);
+						slotTracker.notifyAllocationComplete(slotId, jobId);
+					} else {
+						if (throwable instanceof SlotOccupiedException) {
+							SlotOccupiedException exception = (SlotOccupiedException) throwable;
+							LOG.debug("Tried allocating slot {} for job {}, but it was already allocated for job {}.", slotId, jobId, exception.getJobId());
+							// report as a slot status to force the state transition
+							// this could be a problem if we ever assume that the task executor always reports about all slots
+							slotTracker.notifySlotStatus(Collections.singleton(new SlotStatus(slotId, taskManagerSlot.getResourceProfile(), exception.getJobId(), exception.getAllocationId())));
+						} else {
+							if (throwable instanceof CancellationException) {
+								LOG.debug("Cancelled allocation of slot {} for job {}.", slotId, jobId, throwable);
+							} else {
+								LOG.warn("Slot allocation for slot {} for job {} failed.", slotId, jobId, throwable);
+								slotTracker.notifyFree(slotId);
+							}
+						}
+					}
+					checkResourceRequirements();
+				} catch (Exception e) {
+					LOG.error("Error while completing the slot allocation.", e);
+				}

Review comment:
       Yes, please do so.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] zentol commented on a change in pull request #13553: [FLINK-19306][coordination] Add DeclarativeSlotManager

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #13553:
URL: https://github.com/apache/flink/pull/13553#discussion_r502327559



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java
##########
@@ -1190,6 +1215,11 @@ public void notifyAllocationFailure(JobID jobId, AllocationID allocationId, Exce
 				jobManagerRegistration.getJobManagerGateway().notifyAllocationFailure(allocationId, cause);
 			}
 		}
+
+		@Override
+		public void notifyNotEnoughResourcesAvailable(JobID jobId, Collection<ResourceRequirement> acquiredResources) {
+			validateRunsInMainThread();
+		}

Review comment:
       The call to JobMaster#notifyNotEnoughResourcesAvailable; but that wasn't added yet because I only wanted to touch the RM side.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] tillrohrmann commented on a change in pull request #13553: [FLINK-19306][coordination] Add DeclarativeSlotManager

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #13553:
URL: https://github.com/apache/flink/pull/13553#discussion_r502529338



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManager.java
##########
@@ -0,0 +1,609 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.metrics.MetricNames;
+import org.apache.flink.runtime.metrics.groups.SlotManagerMetricGroup;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerId;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.slots.ResourceCounter;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+import org.apache.flink.runtime.slots.ResourceRequirements;
+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.SlotOccupiedException;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ScheduledFuture;
+import java.util.function.BiFunction;
+import java.util.stream.Collectors;
+
+/**
+ * Implementation of {@link SlotManager} supporting declarative slot management.
+ */
+public class DeclarativeSlotManager implements SlotManager {
+	private static final Logger LOG = LoggerFactory.getLogger(DeclarativeSlotManager.class);
+
+	private final SlotTracker slotTracker;
+	private final ResourceTracker resourceTracker;
+	private final BiFunction<Executor, ResourceActions, TaskExecutorManager> taskExecutorManagerFactory;
+	private TaskExecutorManager taskExecutorManager;
+
+	/** Timeout for slot requests to the task manager. */
+	private final Time taskManagerRequestTimeout;
+	private ScheduledFuture<?> slotRequestTimeoutCheck;
+
+	private final SlotMatchingStrategy slotMatchingStrategy;
+
+	private final SlotManagerMetricGroup slotManagerMetricGroup;
+
+	private final Map<JobID, String> jobMasterTargetAddresses = new HashMap<>();
+	private final HashMap<SlotID, CompletableFuture<Acknowledge>> pendingSlotAllocationFutures;
+
+	/** ResourceManager's id. */
+	private ResourceManagerId resourceManagerId;
+
+	/** Executor for future callbacks which have to be "synchronized". */
+	private Executor mainThreadExecutor;
+
+	/** Callbacks for resource (de-)allocations. */
+	private ResourceActions resourceActions;
+
+	/** True iff the component has been started. */
+	private boolean started;
+
+	public DeclarativeSlotManager(
+			ScheduledExecutor scheduledExecutor,
+			SlotManagerConfiguration slotManagerConfiguration,
+			SlotManagerMetricGroup slotManagerMetricGroup,
+			ResourceTracker resourceTracker,
+			SlotTracker slotTracker) {
+
+		Preconditions.checkNotNull(slotManagerConfiguration);
+		this.taskManagerRequestTimeout = slotManagerConfiguration.getTaskManagerRequestTimeout();
+		this.slotManagerMetricGroup = Preconditions.checkNotNull(slotManagerMetricGroup);
+		this.resourceTracker = Preconditions.checkNotNull(resourceTracker);
+
+		pendingSlotAllocationFutures = new HashMap<>(16);
+
+		this.slotTracker = Preconditions.checkNotNull(slotTracker);
+		slotTracker.registerSlotStatusUpdateListener(createSlotStatusUpdateListener());
+
+		slotMatchingStrategy = slotManagerConfiguration.getSlotMatchingStrategy();
+
+		taskExecutorManagerFactory = (executor, resourceActions) -> new TaskExecutorManager(
+			slotManagerConfiguration.getDefaultWorkerResourceSpec(),
+			slotManagerConfiguration.getNumSlotsPerWorker(),
+			slotManagerConfiguration.getMaxSlotNum(),
+			slotManagerConfiguration.isWaitResultConsumedBeforeRelease(),
+			slotManagerConfiguration.getRedundantTaskManagerNum(),
+			slotManagerConfiguration.getTaskManagerTimeout(),
+			scheduledExecutor,
+			executor,
+			resourceActions);
+
+		resourceManagerId = null;
+		resourceActions = null;
+		mainThreadExecutor = null;
+		slotRequestTimeoutCheck = null;
+		taskExecutorManager = null;
+
+		started = false;
+	}
+
+	private SlotStatusUpdateListener createSlotStatusUpdateListener() {
+		return (taskManagerSlot, previous, current, jobId) -> {
+			if (previous == SlotState.PENDING) {
+				cancelAllocationFuture(taskManagerSlot.getSlotId());
+			}
+
+			if (current == SlotState.PENDING) {
+				resourceTracker.notifyAcquiredResource(jobId, taskManagerSlot.getResourceProfile());
+			}
+			if (current == SlotState.FREE) {
+				resourceTracker.notifyLostResource(jobId, taskManagerSlot.getResourceProfile());
+			}
+
+			if (current == SlotState.ALLOCATED) {
+				taskExecutorManager.occupySlot(taskManagerSlot.getInstanceId());
+			}
+			if (previous == SlotState.ALLOCATED && current == SlotState.FREE) {
+				taskExecutorManager.freeSlot(taskManagerSlot.getInstanceId());
+			}
+		};
+	}
+
+	private void cancelAllocationFuture(SlotID slotId) {
+		final CompletableFuture<Acknowledge> acknowledgeCompletableFuture = pendingSlotAllocationFutures.remove(slotId);
+		// the future may be null if we are just re-playing the state transitions due to a slot report
+		if (acknowledgeCompletableFuture != null) {
+			acknowledgeCompletableFuture.cancel(false);
+		}
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Component lifecycle methods
+	// ---------------------------------------------------------------------------------------------
+
+	/**
+	 * Starts the slot manager with the given leader id and resource manager actions.
+	 *
+	 * @param newResourceManagerId to use for communication with the task managers
+	 * @param newMainThreadExecutor to use to run code in the ResourceManager's main thread
+	 * @param newResourceActions to use for resource (de-)allocations
+	 */
+	@Override
+	public void start(ResourceManagerId newResourceManagerId, Executor newMainThreadExecutor, ResourceActions newResourceActions) {
+		LOG.info("Starting the slot manager.");
+
+		this.resourceManagerId = Preconditions.checkNotNull(newResourceManagerId);
+		mainThreadExecutor = Preconditions.checkNotNull(newMainThreadExecutor);
+		resourceActions = Preconditions.checkNotNull(newResourceActions);
+		taskExecutorManager = taskExecutorManagerFactory.apply(newMainThreadExecutor, newResourceActions);
+
+		started = true;
+
+		registerSlotManagerMetrics();
+	}
+
+	private void registerSlotManagerMetrics() {
+		slotManagerMetricGroup.gauge(
+			MetricNames.TASK_SLOTS_AVAILABLE,
+			() -> (long) getNumberFreeSlots());
+		slotManagerMetricGroup.gauge(
+			MetricNames.TASK_SLOTS_TOTAL,
+			() -> (long) getNumberRegisteredSlots());
+	}
+
+	/**
+	 * Suspends the component. This clears the internal state of the slot manager.
+	 */
+	@Override
+	public void suspend() {
+		LOG.info("Suspending the slot manager.");
+
+		if (slotRequestTimeoutCheck != null) {
+			slotRequestTimeoutCheck.cancel(false);
+			slotRequestTimeoutCheck = null;
+		}
+
+		resourceTracker.clear();
+		taskExecutorManager.close();
+
+		for (InstanceID registeredTaskManager : taskExecutorManager.getTaskExecutors()) {
+			unregisterTaskManager(registeredTaskManager, new SlotManagerException("The slot manager is being suspended."));
+		}
+
+		taskExecutorManager = null;
+		resourceManagerId = null;
+		resourceActions = null;
+		started = false;
+	}
+
+	/**
+	 * Closes the slot manager.
+	 *
+	 * @throws Exception if the close operation fails
+	 */
+	@Override
+	public void close() throws Exception {
+		LOG.info("Closing the slot manager.");
+
+		suspend();
+		slotManagerMetricGroup.close();
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Public API
+	// ---------------------------------------------------------------------------------------------
+
+	@Override
+	public void processResourceRequirements(ResourceRequirements resourceRequirements) {
+		checkInit();
+		LOG.debug("Received resource requirements from job {}: {}", resourceRequirements.getJobId(), resourceRequirements.getResourceRequirements());
+
+		if (resourceRequirements.getResourceRequirements().isEmpty()) {
+			jobMasterTargetAddresses.remove(resourceRequirements.getJobId());
+		} else {
+			jobMasterTargetAddresses.put(resourceRequirements.getJobId(), resourceRequirements.getTargetAddress());
+		}
+		resourceTracker.notifyResourceRequirements(resourceRequirements.getJobId(), resourceRequirements.getResourceRequirements());
+		checkResourceRequirements();
+	}
+
+	/**
+	 * 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
+	 * @return True if the task manager has not been registered before and is registered successfully; otherwise false
+	 */
+	@Override
+	public boolean registerTaskManager(final TaskExecutorConnection taskExecutorConnection, SlotReport initialSlotReport) {
+		checkInit();
+		LOG.debug("Registering task executor {} under {} at the slot manager.", taskExecutorConnection.getResourceID(), taskExecutorConnection.getInstanceID());
+
+		// we identify task managers by their instance id
+		if (taskExecutorManager.isTaskManagerRegistered(taskExecutorConnection.getInstanceID())) {
+			LOG.debug("Task executor {} was already registered.", taskExecutorConnection.getResourceID());
+			reportSlotStatus(taskExecutorConnection.getInstanceID(), initialSlotReport);
+			return false;
+		} else {
+			if (!taskExecutorManager.registerTaskManager(taskExecutorConnection, initialSlotReport)) {
+				LOG.debug("Task executor {} could not be registered.", taskExecutorConnection.getResourceID());
+				return false;
+			}
+
+			// register the new slots
+			for (SlotStatus slotStatus : initialSlotReport) {
+				slotTracker.addSlot(
+					slotStatus.getSlotID(),
+					slotStatus.getResourceProfile(),
+					taskExecutorConnection,
+					slotStatus.getJobID());
+			}
+
+			checkResourceRequirements();
+			return true;
+		}
+	}
+
+	@Override
+	public boolean unregisterTaskManager(InstanceID instanceId, Exception cause) {
+		checkInit();
+
+		LOG.debug("Unregistering task executor {} from the slot manager.", instanceId);
+
+		if (taskExecutorManager.isTaskManagerRegistered(instanceId)) {
+			slotTracker.removeSlots(taskExecutorManager.getSlotsOf(instanceId));
+			taskExecutorManager.unregisterTaskExecutor(instanceId);
+			checkResourceRequirements();
+
+			return true;
+		} else {
+			LOG.debug("There is no task executor registered with instance ID {}. Ignoring this message.", instanceId);
+
+			return false;
+		}
+	}
+
+	/**
+	 * Reports the current slot allocations for a task manager identified by the given instance id.
+	 *
+	 * @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
+	 */
+	@Override
+	public boolean reportSlotStatus(InstanceID instanceId, SlotReport slotReport) {
+		checkInit();
+
+		LOG.debug("Received slot report from instance {}: {}.", instanceId, slotReport);
+
+		if (taskExecutorManager.isTaskManagerRegistered(instanceId)) {
+			slotTracker.notifySlotStatus(slotReport);
+			checkResourceRequirements();
+			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
+	 */
+	@Override
+	public void freeSlot(SlotID slotId, AllocationID allocationId) {
+		checkInit();
+		LOG.debug("Freeing slot {}.", slotId);
+
+		slotTracker.notifyFree(slotId);
+		checkResourceRequirements();
+	}
+
+	// ---------------------------------------------------------------------------------------------
+	// Requirement matching
+	// ---------------------------------------------------------------------------------------------
+
+	private void checkResourceRequirements() {
+		final Map<JobID, Collection<ResourceRequirement>> missingResources = resourceTracker.getMissingResources();
+		if (missingResources.isEmpty()) {
+			return;
+		}
+
+		final Map<JobID, ResourceCounter> outstandingRequirements = new LinkedHashMap<>();
+		for (Map.Entry<JobID, Collection<ResourceRequirement>> resourceRequirements : missingResources.entrySet()) {
+			JobID jobId = resourceRequirements.getKey();
+
+			for (ResourceRequirement resourceRequirement : resourceRequirements.getValue()) {
+				int numMissingSlots = internalRequestSlots(jobId, jobMasterTargetAddresses.get(jobId), resourceRequirement);
+				if (numMissingSlots > 0) {
+					outstandingRequirements
+						.computeIfAbsent(jobId, ignored -> new ResourceCounter())
+						.incrementCount(resourceRequirement.getResourceProfile(), numMissingSlots);
+				}
+			}
+		}
+
+		final ResourceCounter pendingSlots = new ResourceCounter(taskExecutorManager.getPendingTaskManagerSlots().stream().collect(
+			Collectors.groupingBy(
+				PendingTaskManagerSlot::getResourceProfile,
+				Collectors.summingInt(x -> 1))));
+
+		for (Map.Entry<JobID, ResourceCounter> unfulfilledRequirement : outstandingRequirements.entrySet()) {
+			tryFulfillRequirementsWithPendingOrNewSlots(
+				unfulfilledRequirement.getKey(),
+				unfulfilledRequirement.getValue().getResourceProfilesWithCount(),
+				pendingSlots);
+		}
+	}
+
+	/**
+	 * Tries to allocate slots for the given requirement. If there are not enough slots available, the
+	 * resource manager is informed to allocate more resources.
+	 *
+	 * @param jobId job to allocate slots for
+	 * @param targetAddress address of the jobmaster
+	 * @param resourceRequirement required slots
+	 * @return the number of missing slots
+	 */
+	private int internalRequestSlots(JobID jobId, String targetAddress, ResourceRequirement resourceRequirement) {
+		final ResourceProfile requiredResource = resourceRequirement.getResourceProfile();
+		Collection<TaskManagerSlotInformation> freeSlots = slotTracker.getFreeSlots();
+
+		int numUnfulfilled = 0;
+		for (int x = 0; x < resourceRequirement.getNumberOfRequiredSlots(); x++) {
+
+			final Optional<TaskManagerSlotInformation> reservedSlot = slotMatchingStrategy.findMatchingSlot(requiredResource, freeSlots, this::getNumberRegisteredSlotsOf);
+			if (reservedSlot.isPresent()) {
+				// we do not need to modify freeSlots because it is indirectly modified by the allocation
+				allocateSlot(reservedSlot.get(), jobId, targetAddress, requiredResource);
+			} else {
+				numUnfulfilled++;
+			}
+		}
+		return numUnfulfilled;
+	}
+
+	/**
+	 * Allocates the given slot. This entails sending a registration message to the task manager and treating failures.
+	 *
+	 * @param taskManagerSlot slot to allocate
+	 * @param jobId job for which the slot should be allocated for
+	 * @param targetAddress address of the job master
+	 * @param resourceProfile resource profile for the requirement for which the slot is used
+	 */
+	private void allocateSlot(TaskManagerSlotInformation taskManagerSlot, JobID jobId, String targetAddress, ResourceProfile resourceProfile) {
+		TaskExecutorConnection taskExecutorConnection = taskManagerSlot.getTaskManagerConnection();
+		TaskExecutorGateway gateway = taskExecutorConnection.getTaskExecutorGateway();
+
+		final CompletableFuture<Acknowledge> completableFuture = new CompletableFuture<>();
+		final SlotID slotId = taskManagerSlot.getSlotId();
+		final InstanceID instanceId = taskManagerSlot.getInstanceId();
+
+		LOG.debug("Starting allocation of slot {} for job {} with resource profile {}.", taskManagerSlot.getSlotId(), jobId, resourceProfile);
+
+		slotTracker.notifyAllocationStart(slotId, jobId);
+
+		if (!taskExecutorManager.isTaskManagerRegistered(instanceId)) {
+			throw new IllegalStateException("Could not find a registered task manager for instance id " +
+				instanceId + '.');
+		}
+
+		taskExecutorManager.markUsed(instanceId);
+
+		// RPC call to the task manager
+		CompletableFuture<Acknowledge> requestFuture = gateway.requestSlot(
+			slotId,
+			jobId,
+			new AllocationID(),
+			resourceProfile,
+			targetAddress,
+			resourceManagerId,
+			taskManagerRequestTimeout);
+
+		requestFuture.whenComplete(
+			(Acknowledge acknowledge, Throwable throwable) -> {
+				if (acknowledge != null) {
+					completableFuture.complete(acknowledge);
+				} else {
+					completableFuture.completeExceptionally(throwable);
+				}
+			});
+
+		completableFuture.whenCompleteAsync(
+			(Acknowledge acknowledge, Throwable throwable) -> {
+				try {
+					if (acknowledge != null) {
+						LOG.trace("Completed allocation of slot {} for job {}.", slotId, jobId);
+						slotTracker.notifyAllocationComplete(slotId, jobId);
+					} else {
+						if (throwable instanceof SlotOccupiedException) {
+							SlotOccupiedException exception = (SlotOccupiedException) throwable;
+							LOG.debug("Tried allocating slot {} for job {}, but it was already allocated for job {}.", slotId, jobId, exception.getJobId());
+							// report as a slot status to force the state transition
+							// this could be a problem if we ever assume that the task executor always reports about all slots
+							slotTracker.notifySlotStatus(Collections.singleton(new SlotStatus(slotId, taskManagerSlot.getResourceProfile(), exception.getJobId(), exception.getAllocationId())));
+						} else {
+							if (throwable instanceof CancellationException) {
+								LOG.debug("Cancelled allocation of slot {} for job {}.", slotId, jobId, throwable);

Review comment:
       I think at the moment, we don't register these futures and, hence, it should never happen.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org