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 2021/01/14 11:27:56 UTC

[GitHub] [flink] KarmaGYZ opened a new pull request #14647: [FLINK-20835] Implement FineGrainedSlotManager

KarmaGYZ opened a new pull request #14647:
URL: https://github.com/apache/flink/pull/14647


   
   ## What is the purpose of the change
   
   Introduce SlotManager plugin for fine-grained resource management.
   - Request slot from TaskExecutor with the actual resource profiles.
   - Use ResourceTracker to bookkeep the resource requirements
   - Introduce TaskExecutorTracker, which bookkeep the total / available resource and slot assignment for registered task executor.
     - Bookkeep task manager total and available resources.
     - Bookkeep slot allocations and assignments.
     - Intorduce PendingTaskManager.
     - Map registered task executors to matching PendingTaskManager.
   
   
   ## Brief change log
   
   - af078b2 : hotfix
   - fcf8b2e : Add feature toggle for fine-grained resource management
   - 70dd92e ... ec1a427 : Code refactor and introduce utility method
   - ebc0a6e : Introduce TaskExecutorMatchingStrategy and its implementations for fine-grained resource management.
   - e7be627 : Introduce TaskExecutorTracker to track TaskExecutor's resource and slot status
   - e178fe1 : Introduce the FineGrainedSlotManager
   - 1c60554 : setup Azure build for FineGrainedSlotManager
   
   
   


----------------------------------------------------------------
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] KarmaGYZ commented on pull request #14647: [FLINK-20835] Implement FineGrainedSlotManager

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


   Thanks for the review @xintongsong . All your comments have been addressed.


----------------------------------------------------------------
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 #14647: [FLINK-20835] Implement FineGrainedSlotManager

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


   Could someone walk me through the core differences between this slot manager and the declarative slot manager?
   From a cursory glance, they appear to be quite similar. Is it just that we track resources per task executor and based on that determine which TM we ask for a slot with X resources?


----------------------------------------------------------------
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 #14647: [FLINK-20835] Implement FineGrainedSlotManager

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "1c605546a2ac429e1c475eea6f80a75d0326c7bf",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12054",
       "triggerID" : "1c605546a2ac429e1c475eea6f80a75d0326c7bf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "559cd8d09cca8082e9ca4904d11b11443f4c959f",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12226",
       "triggerID" : "559cd8d09cca8082e9ca4904d11b11443f4c959f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "05733d3630a9a72d04545e2e8aeefdf76b2e5cf1",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "05733d3630a9a72d04545e2e8aeefdf76b2e5cf1",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 1c605546a2ac429e1c475eea6f80a75d0326c7bf Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12054) 
   * 559cd8d09cca8082e9ca4904d11b11443f4c959f Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12226) 
   * 05733d3630a9a72d04545e2e8aeefdf76b2e5cf1 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 #14647: [FLINK-20835] Implement FineGrainedSlotManager

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "1c605546a2ac429e1c475eea6f80a75d0326c7bf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12054",
       "triggerID" : "1c605546a2ac429e1c475eea6f80a75d0326c7bf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "559cd8d09cca8082e9ca4904d11b11443f4c959f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12226",
       "triggerID" : "559cd8d09cca8082e9ca4904d11b11443f4c959f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "05733d3630a9a72d04545e2e8aeefdf76b2e5cf1",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12235",
       "triggerID" : "05733d3630a9a72d04545e2e8aeefdf76b2e5cf1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0a88531c02021ab6bc36c8eec483dac0ee91db97",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12540",
       "triggerID" : "0a88531c02021ab6bc36c8eec483dac0ee91db97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "45b2dc853d01aa726bdf054b3c2afa5b95a1d6fd",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "45b2dc853d01aa726bdf054b3c2afa5b95a1d6fd",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 05733d3630a9a72d04545e2e8aeefdf76b2e5cf1 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12235) 
   * 0a88531c02021ab6bc36c8eec483dac0ee91db97 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12540) 
   * 45b2dc853d01aa726bdf054b3c2afa5b95a1d6fd 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 #14647: [FLINK-20835] Implement FineGrainedSlotManager

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "1c605546a2ac429e1c475eea6f80a75d0326c7bf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12054",
       "triggerID" : "1c605546a2ac429e1c475eea6f80a75d0326c7bf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "559cd8d09cca8082e9ca4904d11b11443f4c959f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12226",
       "triggerID" : "559cd8d09cca8082e9ca4904d11b11443f4c959f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "05733d3630a9a72d04545e2e8aeefdf76b2e5cf1",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12235",
       "triggerID" : "05733d3630a9a72d04545e2e8aeefdf76b2e5cf1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0a88531c02021ab6bc36c8eec483dac0ee91db97",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "0a88531c02021ab6bc36c8eec483dac0ee91db97",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 05733d3630a9a72d04545e2e8aeefdf76b2e5cf1 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12235) 
   * 0a88531c02021ab6bc36c8eec483dac0ee91db97 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] xintongsong commented on a change in pull request #14647: [FLINK-20835] Implement FineGrainedSlotManager

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



##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/FineGrainedSlotManagerTestBase.java
##########
@@ -0,0 +1,178 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.common.time.Time;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.concurrent.ScheduledExecutorServiceAdapter;
+import org.apache.flink.runtime.metrics.groups.SlotManagerMetricGroup;
+import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerId;
+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.SlotStatus;
+import org.apache.flink.runtime.taskexecutor.TestingTaskExecutorGatewayBuilder;
+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.RunnableWithException;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Optional;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+/** Base class for the tests of {@link FineGrainedSlotManager}. */
+public abstract class FineGrainedSlotManagerTestBase extends TestLogger {
+    private static final String MAIN_THREAD_NAME = "testing-rpc-main-thread";
+    private static final ScheduledExecutor MAIN_THREAD_EXECUTOR =
+            new ScheduledExecutorServiceAdapter(
+                    Executors.newSingleThreadScheduledExecutor(
+                            runnable -> new Thread(runnable, MAIN_THREAD_NAME)));
+    static final FlinkException TEST_EXCEPTION = new FlinkException("Test exception");
+    static final long FUTURE_TIMEOUT_SECOND = 5;
+
+    /** Resource profile for the default task manager. */
+    protected abstract ResourceProfile getDefaultTaskManagerResourceProfile();
+
+    /** Resource profile for the default slot and requirement. */
+    protected abstract ResourceProfile getDefaultSlotResourceProfile();
+
+    /** The number of slot for the default task manager. */
+    protected abstract int getDefaultNumberSlotsPerWorker();
+
+    /**
+     * Resource profile for a larger task manager, which can fulfill both the larger and the default
+     * slots.
+     */
+    protected abstract ResourceProfile getLargeTaskManagerResourceProfile();
+
+    /**
+     * Resource profile for a larger slot or requirement, which can be fulfilled by the task manager
+     * and cannot be fulfilled by the default task manager.
+     */
+    protected abstract ResourceProfile getLargeSlotResourceProfile();
+
+    protected abstract Optional<ResourceAllocationStrategy> getResourceAllocationStrategy();
+
+    static SlotStatus createAllocatedSlotStatus(
+            AllocationID allocationID, ResourceProfile resourceProfile) {
+        return new SlotStatus(
+                new SlotID(ResourceID.generate(), 0), resourceProfile, new JobID(), allocationID);
+    }
+
+    static int getTotalResourceCount(Collection<ResourceRequirement> resources) {
+        if (resources == null) {
+            return 0;
+        }
+        return resources.stream()
+                .map(ResourceRequirement::getNumberOfRequiredSlots)
+                .reduce(0, Integer::sum);
+    }
+
+    static ResourceRequirements createResourceRequirementsForSingleSlot() {
+        return createResourceRequirementsForSingleSlot(new JobID());
+    }
+
+    static ResourceRequirements createResourceRequirementsForSingleSlot(JobID jobId) {
+        return createResourceRequirements(jobId, 1);
+    }
+
+    static ResourceRequirements createResourceRequirements(JobID jobId, int numRequiredSlots) {
+        return createResourceRequirements(jobId, numRequiredSlots, ResourceProfile.UNKNOWN);
+    }
+
+    static ResourceRequirements createResourceRequirements(
+            JobID jobId, int numRequiredSlots, ResourceProfile resourceProfile) {
+        return ResourceRequirements.create(
+                jobId,
+                "foobar",
+                Collections.singleton(
+                        ResourceRequirement.create(resourceProfile, numRequiredSlots)));
+    }
+
+    static TaskExecutorConnection createTaskExecutorConnection() {
+        return new TaskExecutorConnection(
+                ResourceID.generate(),
+                new TestingTaskExecutorGatewayBuilder().createTestingTaskExecutorGateway());
+    }
+
+    /** This class provides a self-contained context for each test case. */
+    protected class Context {
+        private final ResourceManagerId resourceManagerId = ResourceManagerId.generate();
+        private final ResourceTracker resourceTracker = new DefaultResourceTracker();
+        private final TaskManagerTracker taskManagerTracker = new FineGrainedTaskManagerTracker();
+        private final SlotStatusSyncer slotStatusSyncer =
+                new DefaultSlotStatusSyncer(Time.seconds(10L));
+        private final ScheduledExecutor scheduledExecutor = TestingUtils.defaultScheduledExecutor();
+        private final SlotManagerMetricGroup slotManagerMetricGroup =
+                UnregisteredMetricGroups.createUnregisteredSlotManagerMetricGroup();
+        private FineGrainedSlotManager slotManager;
+
+        protected TestingResourceAllocationStrategy.Builder resourceAllocationStrategyBuilder =

Review comment:
       We can make this `final`.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/FineGrainedSlotManagerTestBase.java
##########
@@ -0,0 +1,178 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.common.time.Time;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.concurrent.ScheduledExecutorServiceAdapter;
+import org.apache.flink.runtime.metrics.groups.SlotManagerMetricGroup;
+import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerId;
+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.SlotStatus;
+import org.apache.flink.runtime.taskexecutor.TestingTaskExecutorGatewayBuilder;
+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.RunnableWithException;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Optional;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+/** Base class for the tests of {@link FineGrainedSlotManager}. */
+public abstract class FineGrainedSlotManagerTestBase extends TestLogger {
+    private static final String MAIN_THREAD_NAME = "testing-rpc-main-thread";
+    private static final ScheduledExecutor MAIN_THREAD_EXECUTOR =
+            new ScheduledExecutorServiceAdapter(
+                    Executors.newSingleThreadScheduledExecutor(
+                            runnable -> new Thread(runnable, MAIN_THREAD_NAME)));
+    static final FlinkException TEST_EXCEPTION = new FlinkException("Test exception");
+    static final long FUTURE_TIMEOUT_SECOND = 5;
+
+    /** Resource profile for the default task manager. */
+    protected abstract ResourceProfile getDefaultTaskManagerResourceProfile();
+
+    /** Resource profile for the default slot and requirement. */
+    protected abstract ResourceProfile getDefaultSlotResourceProfile();
+
+    /** The number of slot for the default task manager. */
+    protected abstract int getDefaultNumberSlotsPerWorker();
+
+    /**
+     * Resource profile for a larger task manager, which can fulfill both the larger and the default
+     * slots.
+     */
+    protected abstract ResourceProfile getLargeTaskManagerResourceProfile();
+
+    /**
+     * Resource profile for a larger slot or requirement, which can be fulfilled by the task manager
+     * and cannot be fulfilled by the default task manager.
+     */
+    protected abstract ResourceProfile getLargeSlotResourceProfile();
+
+    protected abstract Optional<ResourceAllocationStrategy> getResourceAllocationStrategy();
+
+    static SlotStatus createAllocatedSlotStatus(
+            AllocationID allocationID, ResourceProfile resourceProfile) {
+        return new SlotStatus(
+                new SlotID(ResourceID.generate(), 0), resourceProfile, new JobID(), allocationID);
+    }
+
+    static int getTotalResourceCount(Collection<ResourceRequirement> resources) {
+        if (resources == null) {
+            return 0;
+        }
+        return resources.stream()
+                .map(ResourceRequirement::getNumberOfRequiredSlots)
+                .reduce(0, Integer::sum);
+    }
+
+    static ResourceRequirements createResourceRequirementsForSingleSlot() {
+        return createResourceRequirementsForSingleSlot(new JobID());
+    }
+
+    static ResourceRequirements createResourceRequirementsForSingleSlot(JobID jobId) {
+        return createResourceRequirements(jobId, 1);
+    }
+
+    static ResourceRequirements createResourceRequirements(JobID jobId, int numRequiredSlots) {
+        return createResourceRequirements(jobId, numRequiredSlots, ResourceProfile.UNKNOWN);
+    }
+
+    static ResourceRequirements createResourceRequirements(
+            JobID jobId, int numRequiredSlots, ResourceProfile resourceProfile) {
+        return ResourceRequirements.create(
+                jobId,
+                "foobar",
+                Collections.singleton(
+                        ResourceRequirement.create(resourceProfile, numRequiredSlots)));
+    }
+
+    static TaskExecutorConnection createTaskExecutorConnection() {
+        return new TaskExecutorConnection(
+                ResourceID.generate(),
+                new TestingTaskExecutorGatewayBuilder().createTestingTaskExecutorGateway());
+    }
+
+    /** This class provides a self-contained context for each test case. */
+    protected class Context {
+        private final ResourceManagerId resourceManagerId = ResourceManagerId.generate();
+        private final ResourceTracker resourceTracker = new DefaultResourceTracker();
+        private final TaskManagerTracker taskManagerTracker = new FineGrainedTaskManagerTracker();
+        private final SlotStatusSyncer slotStatusSyncer =
+                new DefaultSlotStatusSyncer(Time.seconds(10L));
+        private final ScheduledExecutor scheduledExecutor = TestingUtils.defaultScheduledExecutor();
+        private final SlotManagerMetricGroup slotManagerMetricGroup =
+                UnregisteredMetricGroups.createUnregisteredSlotManagerMetricGroup();
+        private FineGrainedSlotManager slotManager;
+
+        protected TestingResourceAllocationStrategy.Builder resourceAllocationStrategyBuilder =
+                TestingResourceAllocationStrategy.newBuilder();
+        protected Executor mainThreadExecutor = MAIN_THREAD_EXECUTOR;

Review comment:
       Currently, some of the events are internally scheduled on the main thread executor, while others are directly called on the slot manager in the testing thread. We should make sure the slot manager is always accessed from the same thread.




----------------------------------------------------------------
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 #14647: [FLINK-20835] Implement FineGrainedSlotManager

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "1c605546a2ac429e1c475eea6f80a75d0326c7bf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12054",
       "triggerID" : "1c605546a2ac429e1c475eea6f80a75d0326c7bf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "559cd8d09cca8082e9ca4904d11b11443f4c959f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12226",
       "triggerID" : "559cd8d09cca8082e9ca4904d11b11443f4c959f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "05733d3630a9a72d04545e2e8aeefdf76b2e5cf1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12235",
       "triggerID" : "05733d3630a9a72d04545e2e8aeefdf76b2e5cf1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0a88531c02021ab6bc36c8eec483dac0ee91db97",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12540",
       "triggerID" : "0a88531c02021ab6bc36c8eec483dac0ee91db97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "45b2dc853d01aa726bdf054b3c2afa5b95a1d6fd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12543",
       "triggerID" : "45b2dc853d01aa726bdf054b3c2afa5b95a1d6fd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "80e3121418e741d9bebf3e849817a0c4225a86dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12587",
       "triggerID" : "80e3121418e741d9bebf3e849817a0c4225a86dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f5f9f44b437922f5e0edd0b1dd3097a61351dee",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12723",
       "triggerID" : "8f5f9f44b437922f5e0edd0b1dd3097a61351dee",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0b72f53fbfe19ef426572f78ac716521bc63ab64",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12769",
       "triggerID" : "0b72f53fbfe19ef426572f78ac716521bc63ab64",
       "triggerType" : "PUSH"
     }, {
       "hash" : "970360401dcd333f1a3b29355392385262953246",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12823",
       "triggerID" : "970360401dcd333f1a3b29355392385262953246",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1b395270541b226b08653dabc1e6a6bcc3f7ed5c",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12835",
       "triggerID" : "1b395270541b226b08653dabc1e6a6bcc3f7ed5c",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 970360401dcd333f1a3b29355392385262953246 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12823) 
   * 1b395270541b226b08653dabc1e6a6bcc3f7ed5c Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12835) 
   
   <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 #14647: [FLINK-20835] Implement FineGrainedSlotManager

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "1c605546a2ac429e1c475eea6f80a75d0326c7bf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12054",
       "triggerID" : "1c605546a2ac429e1c475eea6f80a75d0326c7bf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "559cd8d09cca8082e9ca4904d11b11443f4c959f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12226",
       "triggerID" : "559cd8d09cca8082e9ca4904d11b11443f4c959f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "05733d3630a9a72d04545e2e8aeefdf76b2e5cf1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12235",
       "triggerID" : "05733d3630a9a72d04545e2e8aeefdf76b2e5cf1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0a88531c02021ab6bc36c8eec483dac0ee91db97",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12540",
       "triggerID" : "0a88531c02021ab6bc36c8eec483dac0ee91db97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "45b2dc853d01aa726bdf054b3c2afa5b95a1d6fd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12543",
       "triggerID" : "45b2dc853d01aa726bdf054b3c2afa5b95a1d6fd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "80e3121418e741d9bebf3e849817a0c4225a86dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12587",
       "triggerID" : "80e3121418e741d9bebf3e849817a0c4225a86dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f5f9f44b437922f5e0edd0b1dd3097a61351dee",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12723",
       "triggerID" : "8f5f9f44b437922f5e0edd0b1dd3097a61351dee",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0b72f53fbfe19ef426572f78ac716521bc63ab64",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12769",
       "triggerID" : "0b72f53fbfe19ef426572f78ac716521bc63ab64",
       "triggerType" : "PUSH"
     }, {
       "hash" : "970360401dcd333f1a3b29355392385262953246",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12823",
       "triggerID" : "970360401dcd333f1a3b29355392385262953246",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1b395270541b226b08653dabc1e6a6bcc3f7ed5c",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "1b395270541b226b08653dabc1e6a6bcc3f7ed5c",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 970360401dcd333f1a3b29355392385262953246 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12823) 
   * 1b395270541b226b08653dabc1e6a6bcc3f7ed5c 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] xintongsong commented on pull request #14647: [FLINK-20835] Implement FineGrainedSlotManager

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


   @zentol,
   I think the core differences are 1) as you said tracking resources per task executor and decide the TM to allocate slot accordingly, and 2) the abstraction of `ResourceAllocationStrategy` because we are targeting to support multiple fine grained allocation strategy based on this slot manager.
   
   In long term, I think there should be only one slot manager, with potentially customized strategies. Legacy `SlotManagerImpl` should be removed together with the slot request based protocol, and the `FineGrainedSlotManager` and `DeclarativeSlotManager` are indeed similar and likely to be unified.
   
   However, at the moment I'm in favor of keeping `FineGrainedSlotManager` independent from `DeclarativeSlotManager`. It should helps us to quickly deliver the feature, without considering how it should be unified with the `DeclarativeSlotManager`, and most importantly without worrying about breaking the default slot manager for changes needed by the fine grained resource management.
   
   Maybe we can revisit unifying the two slot managers after the features are stabilized.


----------------------------------------------------------------
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 #14647: [FLINK-20835] Implement FineGrainedSlotManager

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "1c605546a2ac429e1c475eea6f80a75d0326c7bf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12054",
       "triggerID" : "1c605546a2ac429e1c475eea6f80a75d0326c7bf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "559cd8d09cca8082e9ca4904d11b11443f4c959f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12226",
       "triggerID" : "559cd8d09cca8082e9ca4904d11b11443f4c959f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "05733d3630a9a72d04545e2e8aeefdf76b2e5cf1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12235",
       "triggerID" : "05733d3630a9a72d04545e2e8aeefdf76b2e5cf1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0a88531c02021ab6bc36c8eec483dac0ee91db97",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12540",
       "triggerID" : "0a88531c02021ab6bc36c8eec483dac0ee91db97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "45b2dc853d01aa726bdf054b3c2afa5b95a1d6fd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12543",
       "triggerID" : "45b2dc853d01aa726bdf054b3c2afa5b95a1d6fd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "80e3121418e741d9bebf3e849817a0c4225a86dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12587",
       "triggerID" : "80e3121418e741d9bebf3e849817a0c4225a86dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f5f9f44b437922f5e0edd0b1dd3097a61351dee",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12723",
       "triggerID" : "8f5f9f44b437922f5e0edd0b1dd3097a61351dee",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0b72f53fbfe19ef426572f78ac716521bc63ab64",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12769",
       "triggerID" : "0b72f53fbfe19ef426572f78ac716521bc63ab64",
       "triggerType" : "PUSH"
     }, {
       "hash" : "970360401dcd333f1a3b29355392385262953246",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "970360401dcd333f1a3b29355392385262953246",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 0b72f53fbfe19ef426572f78ac716521bc63ab64 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12769) 
   * 970360401dcd333f1a3b29355392385262953246 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 #14647: [FLINK-20835] Implement FineGrainedSlotManager

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "1c605546a2ac429e1c475eea6f80a75d0326c7bf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12054",
       "triggerID" : "1c605546a2ac429e1c475eea6f80a75d0326c7bf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "559cd8d09cca8082e9ca4904d11b11443f4c959f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12226",
       "triggerID" : "559cd8d09cca8082e9ca4904d11b11443f4c959f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "05733d3630a9a72d04545e2e8aeefdf76b2e5cf1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12235",
       "triggerID" : "05733d3630a9a72d04545e2e8aeefdf76b2e5cf1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0a88531c02021ab6bc36c8eec483dac0ee91db97",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12540",
       "triggerID" : "0a88531c02021ab6bc36c8eec483dac0ee91db97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "45b2dc853d01aa726bdf054b3c2afa5b95a1d6fd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12543",
       "triggerID" : "45b2dc853d01aa726bdf054b3c2afa5b95a1d6fd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "80e3121418e741d9bebf3e849817a0c4225a86dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12587",
       "triggerID" : "80e3121418e741d9bebf3e849817a0c4225a86dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f5f9f44b437922f5e0edd0b1dd3097a61351dee",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12723",
       "triggerID" : "8f5f9f44b437922f5e0edd0b1dd3097a61351dee",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0b72f53fbfe19ef426572f78ac716521bc63ab64",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12769",
       "triggerID" : "0b72f53fbfe19ef426572f78ac716521bc63ab64",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 8f5f9f44b437922f5e0edd0b1dd3097a61351dee Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12723) 
   * 0b72f53fbfe19ef426572f78ac716521bc63ab64 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12769) 
   
   <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 #14647: [FLINK-20835] Implement FineGrainedSlotManager

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






----------------------------------------------------------------
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 #14647: [FLINK-20835] Implement FineGrainedSlotManager

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "1c605546a2ac429e1c475eea6f80a75d0326c7bf",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12054",
       "triggerID" : "1c605546a2ac429e1c475eea6f80a75d0326c7bf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "559cd8d09cca8082e9ca4904d11b11443f4c959f",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "559cd8d09cca8082e9ca4904d11b11443f4c959f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 1c605546a2ac429e1c475eea6f80a75d0326c7bf Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12054) 
   * 559cd8d09cca8082e9ca4904d11b11443f4c959f 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] KarmaGYZ commented on a change in pull request #14647: [FLINK-20835] Implement FineGrainedSlotManager

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/FineGrainedTaskManagerRegistration.java
##########
@@ -0,0 +1,182 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.util.Preconditions;
+
+import java.math.BigDecimal;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * A FineGrainedTaskManagerRegistration represents a TaskManager. It records the internal state of
+ * the TaskManager, including allocated/pending slots, total/available resources.
+ *
+ * <p>This class is the fine-grained resource management version of the {@link
+ * TaskManagerRegistration}.
+ */
+public class FineGrainedTaskManagerRegistration implements TaskManagerInfo {
+    private final TaskExecutorConnection taskManagerConnection;
+
+    private final Map<AllocationID, FineGrainedTaskManagerSlot> slots;
+
+    private final ResourceProfile defaultSlotResourceProfile;
+
+    private final ResourceProfile totalResource;
+
+    private final int defaultNumSlots;
+
+    private ResourceProfile unusedResource;
+
+    private ResourceProfile pendingResource = ResourceProfile.ZERO;
+
+    /** Timestamp when the last time becoming idle. Otherwise Long.MAX_VALUE. */
+    private long idleSince;
+
+    public FineGrainedTaskManagerRegistration(
+            TaskExecutorConnection taskManagerConnection,
+            ResourceProfile totalResourceProfile,
+            ResourceProfile defaultSlotResourceProfile) {
+        this.taskManagerConnection = Preconditions.checkNotNull(taskManagerConnection);
+        this.totalResource = Preconditions.checkNotNull(totalResourceProfile);
+        this.defaultSlotResourceProfile = Preconditions.checkNotNull(defaultSlotResourceProfile);
+
+        this.slots = new HashMap<>(16);
+
+        this.defaultNumSlots =
+                totalResourceProfile
+                        .getCpuCores()
+                        .getValue()
+                        .divide(
+                                defaultSlotResourceProfile.getCpuCores().getValue(),
+                                0,
+                                BigDecimal.ROUND_DOWN)
+                        .intValue();

Review comment:
       Yes, I also agree that the number of registered/free slots might be invalid under fine-grained resource management.




----------------------------------------------------------------
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] xintongsong commented on a change in pull request #14647: [FLINK-20835] Implement FineGrainedSlotManager

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/PendingTaskManagerId.java
##########
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.util.AbstractID;
+
+/** Id of {@link PendingTaskManager}. */
+public class PendingTaskManagerId extends AbstractID {

Review comment:
       > This ID should only be used inside the slotmanager.
   
   This is an assumption rather than an explicit contract, while `Serializable` is a contract.
   
   The assumption is true now, but might be changed in future. In that case, people can easily overlook that `PendingTaskManagerId` as a `Serializable` is originally not expected to be serialized and they need to add a `serialVersionUID` to make it properly work.




----------------------------------------------------------------
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 #14647: [FLINK-20835] Implement FineGrainedSlotManager

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "1c605546a2ac429e1c475eea6f80a75d0326c7bf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12054",
       "triggerID" : "1c605546a2ac429e1c475eea6f80a75d0326c7bf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "559cd8d09cca8082e9ca4904d11b11443f4c959f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12226",
       "triggerID" : "559cd8d09cca8082e9ca4904d11b11443f4c959f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "05733d3630a9a72d04545e2e8aeefdf76b2e5cf1",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12235",
       "triggerID" : "05733d3630a9a72d04545e2e8aeefdf76b2e5cf1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0a88531c02021ab6bc36c8eec483dac0ee91db97",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12540",
       "triggerID" : "0a88531c02021ab6bc36c8eec483dac0ee91db97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "45b2dc853d01aa726bdf054b3c2afa5b95a1d6fd",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12543",
       "triggerID" : "45b2dc853d01aa726bdf054b3c2afa5b95a1d6fd",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 05733d3630a9a72d04545e2e8aeefdf76b2e5cf1 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12235) 
   * 0a88531c02021ab6bc36c8eec483dac0ee91db97 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12540) 
   * 45b2dc853d01aa726bdf054b3c2afa5b95a1d6fd Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12543) 
   
   <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] xintongsong commented on a change in pull request #14647: [FLINK-20835] Implement FineGrainedSlotManager

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/FineGrainedTaskManagerRegistration.java
##########
@@ -0,0 +1,182 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.util.Preconditions;
+
+import java.math.BigDecimal;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * A FineGrainedTaskManagerRegistration represents a TaskManager. It records the internal state of
+ * the TaskManager, including allocated/pending slots, total/available resources.
+ *
+ * <p>This class is the fine-grained resource management version of the {@link
+ * TaskManagerRegistration}.
+ */
+public class FineGrainedTaskManagerRegistration implements TaskManagerInfo {
+    private final TaskExecutorConnection taskManagerConnection;
+
+    private final Map<AllocationID, FineGrainedTaskManagerSlot> slots;
+
+    private final ResourceProfile defaultSlotResourceProfile;
+
+    private final ResourceProfile totalResource;
+
+    private final int defaultNumSlots;
+
+    private ResourceProfile unusedResource;
+
+    private ResourceProfile pendingResource = ResourceProfile.ZERO;
+
+    /** Timestamp when the last time becoming idle. Otherwise Long.MAX_VALUE. */
+    private long idleSince;
+
+    public FineGrainedTaskManagerRegistration(
+            TaskExecutorConnection taskManagerConnection,
+            ResourceProfile totalResourceProfile,
+            ResourceProfile defaultSlotResourceProfile) {
+        this.taskManagerConnection = Preconditions.checkNotNull(taskManagerConnection);
+        this.totalResource = Preconditions.checkNotNull(totalResourceProfile);
+        this.defaultSlotResourceProfile = Preconditions.checkNotNull(defaultSlotResourceProfile);
+
+        this.slots = new HashMap<>(16);

Review comment:
       > Set the initial capacity for a collection only if there is a good proven reason for that, otherwise do not clutter the code. In case of Maps it can be even deluding because the Map’s load factor effectively reduces the capacity.
   
   
   https://flink.apache.org/contributing/code-style-and-quality-java.html#collections

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/TaskManagerTracker.java
##########
@@ -0,0 +1,149 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.slots.ResourceCounter;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.Optional;
+
+/** Tracks TaskManager's resource and slot status. */
+interface TaskManagerTracker {
+
+    // ---------------------------------------------------------------------------------------------
+    // Add / Remove (pending) Resource
+    // ---------------------------------------------------------------------------------------------
+
+    /**
+     * Register a new task manager.
+     *
+     * @param taskExecutorConnection of the new task manager
+     * @param totalResourceProfile of the new task manager
+     * @param defaultSlotResourceProfile of the new task manager
+     */
+    void addTaskManager(
+            TaskExecutorConnection taskExecutorConnection,
+            ResourceProfile totalResourceProfile,
+            ResourceProfile defaultSlotResourceProfile);
+
+    /**
+     * Unregister a task manager with the given instance id.
+     *
+     * @param instanceId of the task manager
+     */
+    void removeTaskManager(InstanceID instanceId);
+
+    /**
+     * Add a new pending task manager with its resource profile.
+     *
+     * @param pendingTaskManager to be added
+     */
+    void addPendingTaskManager(PendingTaskManager pendingTaskManager);
+
+    /**
+     * Remove a pending task manager.
+     *
+     * @param pendingTaskManagerId of the pending task manager
+     */
+    void removePendingTaskManager(PendingTaskManagerId pendingTaskManagerId);
+
+    /**
+     * Record pending allocations of the strategy. This will override the previous record if
+     * present.
+     *
+     * @param pendingSlotAllocations to be recorded
+     */
+    void recordPendingAllocations(

Review comment:
       I would suggest to move this to the section `Slot status updates`.
   Also, we can make it more explicit that this method clear all previous pending allocations, by naming it to something like `replaceAllPendingAllocations`.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/FineGrainedTaskManagerRegistration.java
##########
@@ -0,0 +1,182 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.util.Preconditions;
+
+import java.math.BigDecimal;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * A FineGrainedTaskManagerRegistration represents a TaskManager. It records the internal state of
+ * the TaskManager, including allocated/pending slots, total/available resources.
+ *
+ * <p>This class is the fine-grained resource management version of the {@link
+ * TaskManagerRegistration}.
+ */
+public class FineGrainedTaskManagerRegistration implements TaskManagerInfo {
+    private final TaskExecutorConnection taskManagerConnection;
+
+    private final Map<AllocationID, FineGrainedTaskManagerSlot> slots;
+
+    private final ResourceProfile defaultSlotResourceProfile;
+
+    private final ResourceProfile totalResource;
+
+    private final int defaultNumSlots;
+
+    private ResourceProfile unusedResource;
+
+    private ResourceProfile pendingResource = ResourceProfile.ZERO;
+
+    /** Timestamp when the last time becoming idle. Otherwise Long.MAX_VALUE. */
+    private long idleSince;
+
+    public FineGrainedTaskManagerRegistration(
+            TaskExecutorConnection taskManagerConnection,
+            ResourceProfile totalResourceProfile,
+            ResourceProfile defaultSlotResourceProfile) {
+        this.taskManagerConnection = Preconditions.checkNotNull(taskManagerConnection);
+        this.totalResource = Preconditions.checkNotNull(totalResourceProfile);
+        this.defaultSlotResourceProfile = Preconditions.checkNotNull(defaultSlotResourceProfile);
+
+        this.slots = new HashMap<>(16);
+
+        this.defaultNumSlots =
+                totalResourceProfile
+                        .getCpuCores()
+                        .getValue()
+                        .divide(
+                                defaultSlotResourceProfile.getCpuCores().getValue(),
+                                0,
+                                BigDecimal.ROUND_DOWN)
+                        .intValue();

Review comment:
       Not sure about ignoring all fields other that cpu in `ResourceProfile`.
   
   I would suggest to use `ResourceProfile#subtract` and `ResourceProfile#allFieldsNoLessThan` to decide how many default slots can fit in tot total resource.
   
   Moreover, I would suggest to calculate this in `SlotManagerUtils`, so we can easily update the logic if needed later. TBH, I'm not entirely sure how we want to display the number of registered/free slots on WebUI for fine-grained resource management. It's not in scope of this PR, but we should at least make it easy to update later.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/StatusOverview.java
##########
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.util.Preconditions;
+
+import java.util.Map;
+
+/** An overview of the resource allocation of the current cluster. */
+public class StatusOverview {
+    private final Map<InstanceID, TaskManagerInfo> taskManagers;
+
+    public StatusOverview(Map<InstanceID, TaskManagerInfo> taskManagers) {
+        this.taskManagers = Preconditions.checkNotNull(taskManagers);
+    }
+
+    public int getNumberRegisteredSlots() {
+        return taskManagers.values().stream().mapToInt(TaskManagerInfo::getDefaultNumSlots).sum();
+    }
+
+    public int getNumberRegisteredSlotsOf(InstanceID instanceId) {
+        return taskManagers.containsKey(instanceId)
+                ? taskManagers.get(instanceId).getDefaultNumSlots()
+                : 0;

Review comment:
       The following avoids duplicate map lookup.
   ```
   return Optional.ofNullable(taskManagers.get(instanceId))
                   .map(TaskManagerInfo::getDefaultNumSlots)
                   .orElse(0);
   ```

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/PendingTaskManagerId.java
##########
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.util.AbstractID;
+
+/** Id of {@link PendingTaskManager}. */
+public class PendingTaskManagerId extends AbstractID {

Review comment:
       `serialVersionUID` is missing.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/DefaultResourceAllocationStrategy.java
##########
@@ -0,0 +1,233 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.slots.ResourceCounter;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.runtime.resourcemanager.slotmanager.SlotManagerUtils.getEffectiveResourceProfile;
+
+/**
+ * The default implementation of {@link ResourceAllocationStrategy}, which always allocate pending
+ * task managers with the fixed profile.
+ */
+public class DefaultResourceAllocationStrategy implements ResourceAllocationStrategy {
+    private final ResourceProfile defaultSlotResourceProfile;
+    private final ResourceProfile totalResourceProfile;
+
+    public DefaultResourceAllocationStrategy(
+            ResourceProfile defaultSlotResourceProfile, int numSlotsPerWorker) {
+        this.defaultSlotResourceProfile = defaultSlotResourceProfile;
+        this.totalResourceProfile = defaultSlotResourceProfile.multiply(numSlotsPerWorker);
+    }
+
+    /**
+     * Matches resource requirements against available and pending resources. For each job, in a
+     * first round requirements are matched against registered resources. The remaining unfulfilled
+     * requirements are matched against pending resources, allocating more workers if no matching
+     * pending resources could be found. If the requirements for a job could not be fulfilled then
+     * it will be recorded in {@link ResourceAllocationResult#getUnfulfillableJobs()}.
+     *
+     * <p>Performance notes: At it's core this method loops, for each job, over all resources for
+     * each required slot, trying to find a matching registered/pending task manager. One should
+     * generally go in with the assumption that this runs in numberOfJobsRequiringResources *
+     * numberOfRequiredSlots * numberOfFreeOrPendingTaskManagers.
+     *
+     * <p>In the absolute worst case, with J jobs, requiring R slots each with a unique resource
+     * profile such each pair of these profiles is not matching, and T registered/pending task
+     * managers that don't fulfill any requirement, then this method does a total of J*R*T resource
+     * profile comparisons.
+     */
+    @Override
+    public ResourceAllocationResult tryFulfillRequirements(
+            Map<JobID, Collection<ResourceRequirement>> missingResources,
+            Map<InstanceID, Tuple2<ResourceProfile, ResourceProfile>> registeredResources,
+            List<PendingTaskManager> pendingTaskManagers) {
+        final ResourceAllocationResult.Builder resultBuilder = ResourceAllocationResult.builder();
+        final Map<PendingTaskManagerId, ResourceProfile> pendingResources =
+                pendingTaskManagers.stream()
+                        .collect(
+                                Collectors.toMap(
+                                        PendingTaskManager::getPendingTaskManagerId,
+                                        PendingTaskManager::getTotalResourceProfile));
+        for (Map.Entry<JobID, Collection<ResourceRequirement>> resourceRequirements :
+                missingResources.entrySet()) {
+            final JobID jobId = resourceRequirements.getKey();
+
+            final ResourceCounter unfulfilledJobRequirements =
+                    tryFulfillRequirementsForJobWithRegisteredResources(
+                            jobId,
+                            resourceRequirements.getValue(),
+                            registeredResources,
+                            resultBuilder);
+
+            if (!unfulfilledJobRequirements.isEmpty()) {
+                tryFulfillRequirementsForJobWithPendingResources(
+                        jobId, unfulfilledJobRequirements, pendingResources, resultBuilder);
+            }
+        }
+        return resultBuilder.build();
+    }
+
+    private static ResourceCounter tryFulfillRequirementsForJobWithRegisteredResources(
+            JobID jobId,
+            Collection<ResourceRequirement> missingResources,
+            Map<InstanceID, Tuple2<ResourceProfile, ResourceProfile>> registeredResources,
+            ResourceAllocationResult.Builder resultBuilder) {
+        final ResourceCounter outstandingRequirements = new ResourceCounter();
+
+        for (ResourceRequirement resourceRequirement : missingResources) {
+            int numMissingRequirements =
+                    tryFindSlotsForRequirement(
+                            jobId, resourceRequirement, registeredResources, resultBuilder);
+            if (numMissingRequirements > 0) {
+                outstandingRequirements.incrementCount(
+                        resourceRequirement.getResourceProfile(), numMissingRequirements);
+            }
+        }
+        return outstandingRequirements;
+    }
+
+    private static int tryFindSlotsForRequirement(
+            JobID jobId,
+            ResourceRequirement resourceRequirement,
+            Map<InstanceID, Tuple2<ResourceProfile, ResourceProfile>> registeredResources,
+            ResourceAllocationResult.Builder resultBuilder) {
+        final ResourceProfile requiredResource = resourceRequirement.getResourceProfile();
+
+        int numUnfulfilled = 0;
+        for (int x = 0; x < resourceRequirement.getNumberOfRequiredSlots(); x++) {
+            final Optional<InstanceID> matchedTaskManager =
+                    findMatchingTaskManager(requiredResource, registeredResources);
+            if (matchedTaskManager.isPresent()) {
+                final ResourceProfile effectiveProfile =
+                        getEffectiveResourceProfile(
+                                requiredResource,
+                                registeredResources.get(matchedTaskManager.get()).f1);
+                resultBuilder.recordAllocationForRegisteredResource(
+                        jobId, matchedTaskManager.get(), effectiveProfile);
+                deductionRegisteredResource(
+                        registeredResources, matchedTaskManager.get(), effectiveProfile);
+            } else {
+                // exit loop early; we won't find a matching slot for this requirement
+                int numRemaining = resourceRequirement.getNumberOfRequiredSlots() - x;
+                numUnfulfilled += numRemaining;
+                break;
+            }
+        }
+        return numUnfulfilled;
+    }
+
+    private static Optional<InstanceID> findMatchingTaskManager(
+            ResourceProfile requirement,
+            Map<InstanceID, Tuple2<ResourceProfile, ResourceProfile>> registeredResources) {
+        return registeredResources.entrySet().stream()
+                .filter(
+                        taskManager ->
+                                canFulfillRequirement(
+                                        getEffectiveResourceProfile(
+                                                requirement, taskManager.getValue().f1),
+                                        taskManager.getValue().f0))
+                .findFirst()
+                .map(Map.Entry::getKey);
+    }
+
+    private static boolean canFulfillRequirement(
+            ResourceProfile requirement, ResourceProfile resourceProfile) {
+        return resourceProfile.allFieldsNoLessThan(requirement);
+    }
+
+    private static void deductionRegisteredResource(
+            Map<InstanceID, Tuple2<ResourceProfile, ResourceProfile>> registeredResources,
+            InstanceID instanceId,
+            ResourceProfile resourceProfile) {
+        registeredResources.computeIfPresent(
+                instanceId,
+                (id, tuple2) -> {
+                    if (tuple2.f0.subtract(resourceProfile).equals(ResourceProfile.ZERO)) {
+                        return null;
+                    } else {
+                        return Tuple2.of(tuple2.f0.subtract(resourceProfile), tuple2.f1);
+                    }
+                });
+    }
+
+    private static Optional<PendingTaskManagerId> findPendingManagerToFulfill(
+            ResourceProfile resourceProfile,
+            Map<PendingTaskManagerId, ResourceProfile> availableResources) {
+        return availableResources.entrySet().stream()
+                .filter(entry -> entry.getValue().allFieldsNoLessThan(resourceProfile))
+                .findAny()
+                .map(Map.Entry::getKey);
+    }
+
+    private void tryFulfillRequirementsForJobWithPendingResources(
+            JobID jobId,
+            ResourceCounter unfulfilledRequirements,
+            Map<PendingTaskManagerId, ResourceProfile> availableResources,
+            ResourceAllocationResult.Builder resultBuilder) {
+        for (Map.Entry<ResourceProfile, Integer> missingResource :
+                unfulfilledRequirements.getResourceProfilesWithCount().entrySet()) {
+            final ResourceProfile effectiveProfile =
+                    getEffectiveResourceProfile(
+                            missingResource.getKey(), defaultSlotResourceProfile);
+            for (int i = 0; i < missingResource.getValue(); i++) {
+                Optional<PendingTaskManagerId> matchedPendingTaskManager =
+                        findPendingManagerToFulfill(effectiveProfile, availableResources);
+                if (matchedPendingTaskManager.isPresent()) {
+                    availableResources.compute(
+                            matchedPendingTaskManager.get(),
+                            ((pendingTaskManagerId, resourceProfile) ->
+                                    resourceProfile.subtract(effectiveProfile)));

Review comment:
       IDE warns about potential NPE on `resourceProfile`.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/FineGrainedTaskManagerTracker.java
##########
@@ -0,0 +1,237 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.slots.ResourceCounter;
+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.HashSet;
+import java.util.Map;
+import java.util.Optional;
+
+/** Implementation of {@link TaskManagerTracker} supporting fine-grained resource management. */
+public class FineGrainedTaskManagerTracker implements TaskManagerTracker {
+    private static final Logger LOG = LoggerFactory.getLogger(FineGrainedTaskManagerTracker.class);
+
+    /** Map for allocated and pending slots. */
+    private final Map<AllocationID, FineGrainedTaskManagerSlot> slots;
+
+    /** All currently registered task managers. */
+    private final Map<InstanceID, FineGrainedTaskManagerRegistration> taskManagerRegistrations;
+
+    private final Map<PendingTaskManagerId, PendingTaskManager> pendingTaskManagers;
+
+    private final Map<PendingTaskManagerId, Map<JobID, ResourceCounter>>
+            pendingSlotAllocationRecords;
+
+    public FineGrainedTaskManagerTracker() {
+        slots = new HashMap<>(16);
+        taskManagerRegistrations = new HashMap<>(4);
+        pendingTaskManagers = new HashMap<>(16);
+        pendingSlotAllocationRecords = new HashMap<>();
+    }
+
+    @Override
+    public void recordPendingAllocations(
+            Map<PendingTaskManagerId, Map<JobID, ResourceCounter>> pendingSlotAllocations) {
+        LOG.trace("Record the pending allocations {}.", pendingSlotAllocations);
+        Preconditions.checkNotNull(pendingSlotAllocations);
+        pendingSlotAllocationRecords.clear();
+        pendingSlotAllocationRecords.putAll(pendingSlotAllocations);
+    }
+
+    @Override
+    public void addTaskManager(
+            TaskExecutorConnection taskExecutorConnection,
+            ResourceProfile totalResourceProfile,
+            ResourceProfile defaultSlotResourceProfile) {
+        Preconditions.checkNotNull(taskExecutorConnection);
+        Preconditions.checkNotNull(totalResourceProfile);
+        Preconditions.checkNotNull(defaultSlotResourceProfile);
+        LOG.debug(
+                "Add task manager {} with total resource {} and default slot resource {}.",
+                taskExecutorConnection.getInstanceID(),
+                totalResourceProfile,
+                defaultSlotResourceProfile);
+        final FineGrainedTaskManagerRegistration taskManagerRegistration =
+                new FineGrainedTaskManagerRegistration(
+                        taskExecutorConnection, totalResourceProfile, defaultSlotResourceProfile);
+        taskManagerRegistrations.put(
+                taskExecutorConnection.getInstanceID(), taskManagerRegistration);
+    }
+
+    @Override
+    public void removeTaskManager(InstanceID instanceId) {
+        LOG.debug("Remove task manager {}.", instanceId);
+        final FineGrainedTaskManagerRegistration taskManager =
+                Preconditions.checkNotNull(taskManagerRegistrations.get(instanceId));
+        for (AllocationID allocationId : new HashSet<>(taskManager.getAllocatedSlots().keySet())) {
+            slots.remove(allocationId);
+            taskManager.freeSlot(allocationId);
+        }
+        taskManagerRegistrations.remove(instanceId);
+    }
+
+    @Override
+    public void addPendingTaskManager(PendingTaskManager pendingTaskManager) {
+        LOG.debug("Add pending task manager {}.", pendingTaskManager);
+        Preconditions.checkNotNull(pendingTaskManager);
+        pendingTaskManagers.put(pendingTaskManager.getPendingTaskManagerId(), pendingTaskManager);
+    }
+
+    @Override
+    public void removePendingTaskManager(PendingTaskManagerId pendingTaskManagerId) {
+        Preconditions.checkNotNull(pendingTaskManagerId);
+        LOG.debug("Remove pending task manager {}.", pendingTaskManagerId);
+        pendingTaskManagers.remove(pendingTaskManagerId);
+    }
+
+    // ---------------------------------------------------------------------------------------------
+    // Core state transitions
+    // ---------------------------------------------------------------------------------------------
+
+    @Override
+    public void notifySlotStatus(
+            AllocationID allocationId,
+            JobID jobId,
+            InstanceID instanceId,
+            ResourceProfile resourceProfile,
+            SlotState slotState) {
+        Preconditions.checkNotNull(allocationId);
+        Preconditions.checkNotNull(jobId);
+        Preconditions.checkNotNull(instanceId);
+        Preconditions.checkNotNull(resourceProfile);
+        Preconditions.checkNotNull(slotState);
+        switch (slotState) {
+            case FREE:
+                freeSlot(instanceId, allocationId);
+                break;
+            case ALLOCATED:
+                allocateSlot(allocationId, jobId, instanceId, resourceProfile);
+                break;
+            case PENDING:
+                addPendingSlot(allocationId, jobId, instanceId, resourceProfile);
+                break;
+        }
+    }
+
+    private void freeSlot(InstanceID instanceId, AllocationID allocationId) {
+        Preconditions.checkState(taskManagerRegistrations.containsKey(instanceId));
+        Preconditions.checkState(slots.containsKey(allocationId));
+        LOG.debug("Free allocated slot with allocationId {}.", allocationId);
+
+        taskManagerRegistrations.get(instanceId).freeSlot(allocationId);
+        slots.remove(allocationId);
+    }
+
+    private void allocateSlot(
+            AllocationID allocationId,
+            JobID jobId,
+            InstanceID instanceId,
+            ResourceProfile resourceProfile) {
+        Preconditions.checkState(taskManagerRegistrations.containsKey(instanceId));
+        final FineGrainedTaskManagerRegistration taskManager =
+                taskManagerRegistrations.get(instanceId);
+        if (slots.containsKey(allocationId)) {

Review comment:
       Duplicate map lookup

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/ResourceAllocationResult.java
##########
@@ -0,0 +1,121 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.slots.ResourceCounter;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+/** Contains the results of the {@link ResourceAllocationStrategy}. */
+public class ResourceAllocationResult {
+    private final Set<JobID> unfulfillableJobs;
+    private final Map<JobID, Map<InstanceID, ResourceCounter>> registeredResourceAllocationResult;
+    private final List<PendingTaskManager> pendingTaskManagersToBeAllocated;
+    private final Map<PendingTaskManagerId, Map<JobID, ResourceCounter>>
+            pendingResourceAllocationResult;
+
+    private ResourceAllocationResult(
+            Set<JobID> unfulfillableJobs,
+            Map<JobID, Map<InstanceID, ResourceCounter>> registeredResourceAllocationResult,
+            List<PendingTaskManager> pendingTaskManagersToBeAllocated,
+            Map<PendingTaskManagerId, Map<JobID, ResourceCounter>>
+                    pendingResourceAllocationResult) {
+        this.unfulfillableJobs = unfulfillableJobs;
+        this.registeredResourceAllocationResult = registeredResourceAllocationResult;
+        this.pendingTaskManagersToBeAllocated = pendingTaskManagersToBeAllocated;
+        this.pendingResourceAllocationResult = pendingResourceAllocationResult;
+    }
+
+    public List<PendingTaskManager> getPendingTaskManagersToBeAllocated() {
+        return Collections.unmodifiableList(pendingTaskManagersToBeAllocated);
+    }
+
+    public Set<JobID> getUnfulfillableJobs() {
+        return Collections.unmodifiableSet(unfulfillableJobs);
+    }
+
+    public Map<JobID, Map<InstanceID, ResourceCounter>> getRegisteredResourceAllocationResult() {
+        return Collections.unmodifiableMap(registeredResourceAllocationResult);

Review comment:
       This does not prevent modifying `ResourceCounter`. Same for `getPendingResourceAllocationResult`.
   
   To strictly ensure unmodifiability, one can introduce a `ImmutableResourceCounter` that forbids modifying operations, and convert `ResourceCounter` to `ImmutableResourceCounter` in `Builder#build`. This is nice to have but not absolutely necessary for this PR, since AFAIK the `ResourceAllocationResult` is not stored anyway, thus less chance causing problems due to accidentally modifying it. 

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/DefaultSlotStatusSyncer.java
##########
@@ -0,0 +1,258 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.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.FutureUtils;
+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.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.HashSet;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+
+/** Default implementation of {@link SlotStatusSyncer} for fine-grained slot management. */
+public class DefaultSlotStatusSyncer implements SlotStatusSyncer {
+    private static final Logger LOG = LoggerFactory.getLogger(DefaultSlotStatusSyncer.class);
+
+    private final TaskManagerTracker taskManagerTracker;
+    private final ResourceTracker resourceTracker;
+    private final Set<AllocationID> pendingSlotAllocations;
+    /** Timeout for slot requests to the task manager. */
+    private final Time taskManagerRequestTimeout;
+
+    public DefaultSlotStatusSyncer(
+            TaskManagerTracker taskManagerTracker,
+            ResourceTracker resourceTracker,
+            Time taskManagerRequestTimeout) {
+        this.taskManagerTracker = Preconditions.checkNotNull(taskManagerTracker);
+        this.resourceTracker = Preconditions.checkNotNull(resourceTracker);
+        this.taskManagerRequestTimeout = Preconditions.checkNotNull(taskManagerRequestTimeout);
+
+        this.pendingSlotAllocations = new HashSet<>(16);
+    }
+
+    @Override
+    public CompletableFuture<Void> allocateSlot(
+            InstanceID instanceId,
+            JobID jobId,
+            String targetAddress,
+            ResourceProfile resourceProfile,
+            ResourceManagerId resourceManagerId,
+            Executor mainThreadExecutor) {
+        final AllocationID allocationId = new AllocationID();
+        final Optional<TaskManagerInfo> taskManager =
+                taskManagerTracker.getRegisteredTaskManager(instanceId);
+        Preconditions.checkState(
+                taskManager.isPresent(),
+                "Could not find a registered task manager for instance id " + instanceId + '.');
+        final TaskExecutorGateway gateway =
+                taskManager.get().getTaskExecutorConnection().getTaskExecutorGateway();
+
+        taskManagerTracker.notifySlotStatus(
+                allocationId, jobId, instanceId, resourceProfile, SlotState.PENDING);
+        resourceTracker.notifyAcquiredResource(jobId, resourceProfile);
+        pendingSlotAllocations.add(allocationId);
+
+        // RPC call to the task manager
+        CompletableFuture<Acknowledge> requestFuture =
+                gateway.requestSlot(
+                        SlotID.getDynamicSlotID(
+                                taskManager.get().getTaskExecutorConnection().getResourceID()),
+                        jobId,
+                        allocationId,
+                        resourceProfile,
+                        targetAddress,
+                        resourceManagerId,
+                        taskManagerRequestTimeout);
+
+        CompletableFuture<Void> returnedFuture = new CompletableFuture<>();
+
+        FutureUtils.assertNoException(
+                requestFuture.handleAsync(
+                        (Acknowledge acknowledge, Throwable throwable) -> {
+                            if (!pendingSlotAllocations.remove(allocationId)) {
+                                LOG.debug(
+                                        "Ignoring slot allocation update from task manager {} for allocation {} and job {}, because the allocation was already completed or cancelled.",
+                                        instanceId,
+                                        allocationId,
+                                        jobId);
+                                returnedFuture.complete(null);
+                                return null;
+                            }
+                            if (!taskManagerTracker
+                                    .getAllocatedOrPendingSlot(allocationId)
+                                    .isPresent()) {
+                                LOG.debug(
+                                        "The slot {} has been removed before. Ignore the future.",
+                                        allocationId);
+                                requestFuture.complete(null);
+                                return null;
+                            }
+                            if (acknowledge != null) {
+                                LOG.trace(
+                                        "Completed allocation of allocation {} for job {}.",
+                                        allocationId,
+                                        jobId);
+                                taskManagerTracker.notifySlotStatus(
+                                        allocationId,
+                                        jobId,
+                                        instanceId,
+                                        resourceProfile,
+                                        SlotState.ALLOCATED);
+                                returnedFuture.complete(null);
+                            } else {
+                                if (throwable instanceof SlotOccupiedException) {
+                                    LOG.error("Should not get this exception.", throwable);
+                                } else {
+                                    // TODO If the taskManager does not have enough resource, we
+                                    // may endlessly allocate slot on it until the next heartbeat.
+                                    LOG.warn(
+                                            "Slot allocation for allocation {} for job {} failed.",
+                                            allocationId,
+                                            jobId,
+                                            throwable);
+                                    resourceTracker.notifyLostResource(jobId, resourceProfile);
+                                    taskManagerTracker.notifySlotStatus(
+                                            allocationId,
+                                            jobId,
+                                            instanceId,
+                                            resourceProfile,
+                                            SlotState.FREE);
+                                }
+                                returnedFuture.completeExceptionally(throwable);
+                            }
+                            return null;
+                        },
+                        mainThreadExecutor));
+        return returnedFuture;
+    }
+
+    @Override
+    public void freeSlot(AllocationID allocationId) {
+        LOG.debug("Freeing slot {}.", allocationId);
+
+        Optional<TaskManagerSlotInformation> slot =
+                taskManagerTracker.getAllocatedOrPendingSlot(allocationId);
+        if (!slot.isPresent()) {
+            LOG.warn("Try to free unknown slot {}.", allocationId);
+        }
+
+        resourceTracker.notifyLostResource(slot.get().getJobId(), slot.get().getResourceProfile());
+        taskManagerTracker.notifySlotStatus(
+                allocationId,
+                slot.get().getJobId(),
+                slot.get().getInstanceId(),
+                slot.get().getResourceProfile(),
+                SlotState.FREE);
+    }

Review comment:
       Too many duplicated `slot.get()`

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/DefaultSlotStatusSyncer.java
##########
@@ -0,0 +1,258 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.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.FutureUtils;
+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.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.HashSet;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+
+/** Default implementation of {@link SlotStatusSyncer} for fine-grained slot management. */
+public class DefaultSlotStatusSyncer implements SlotStatusSyncer {
+    private static final Logger LOG = LoggerFactory.getLogger(DefaultSlotStatusSyncer.class);
+
+    private final TaskManagerTracker taskManagerTracker;
+    private final ResourceTracker resourceTracker;
+    private final Set<AllocationID> pendingSlotAllocations;
+    /** Timeout for slot requests to the task manager. */
+    private final Time taskManagerRequestTimeout;
+
+    public DefaultSlotStatusSyncer(
+            TaskManagerTracker taskManagerTracker,
+            ResourceTracker resourceTracker,
+            Time taskManagerRequestTimeout) {
+        this.taskManagerTracker = Preconditions.checkNotNull(taskManagerTracker);
+        this.resourceTracker = Preconditions.checkNotNull(resourceTracker);
+        this.taskManagerRequestTimeout = Preconditions.checkNotNull(taskManagerRequestTimeout);
+
+        this.pendingSlotAllocations = new HashSet<>(16);
+    }
+
+    @Override
+    public CompletableFuture<Void> allocateSlot(
+            InstanceID instanceId,
+            JobID jobId,
+            String targetAddress,
+            ResourceProfile resourceProfile,
+            ResourceManagerId resourceManagerId,
+            Executor mainThreadExecutor) {
+        final AllocationID allocationId = new AllocationID();
+        final Optional<TaskManagerInfo> taskManager =
+                taskManagerTracker.getRegisteredTaskManager(instanceId);
+        Preconditions.checkState(
+                taskManager.isPresent(),
+                "Could not find a registered task manager for instance id " + instanceId + '.');
+        final TaskExecutorGateway gateway =
+                taskManager.get().getTaskExecutorConnection().getTaskExecutorGateway();
+
+        taskManagerTracker.notifySlotStatus(
+                allocationId, jobId, instanceId, resourceProfile, SlotState.PENDING);
+        resourceTracker.notifyAcquiredResource(jobId, resourceProfile);
+        pendingSlotAllocations.add(allocationId);
+
+        // RPC call to the task manager
+        CompletableFuture<Acknowledge> requestFuture =
+                gateway.requestSlot(
+                        SlotID.getDynamicSlotID(
+                                taskManager.get().getTaskExecutorConnection().getResourceID()),
+                        jobId,
+                        allocationId,
+                        resourceProfile,
+                        targetAddress,
+                        resourceManagerId,
+                        taskManagerRequestTimeout);
+
+        CompletableFuture<Void> returnedFuture = new CompletableFuture<>();
+
+        FutureUtils.assertNoException(
+                requestFuture.handleAsync(
+                        (Acknowledge acknowledge, Throwable throwable) -> {
+                            if (!pendingSlotAllocations.remove(allocationId)) {
+                                LOG.debug(
+                                        "Ignoring slot allocation update from task manager {} for allocation {} and job {}, because the allocation was already completed or cancelled.",
+                                        instanceId,
+                                        allocationId,
+                                        jobId);
+                                returnedFuture.complete(null);
+                                return null;
+                            }
+                            if (!taskManagerTracker
+                                    .getAllocatedOrPendingSlot(allocationId)
+                                    .isPresent()) {
+                                LOG.debug(
+                                        "The slot {} has been removed before. Ignore the future.",
+                                        allocationId);
+                                requestFuture.complete(null);
+                                return null;
+                            }
+                            if (acknowledge != null) {
+                                LOG.trace(
+                                        "Completed allocation of allocation {} for job {}.",
+                                        allocationId,
+                                        jobId);
+                                taskManagerTracker.notifySlotStatus(
+                                        allocationId,
+                                        jobId,
+                                        instanceId,
+                                        resourceProfile,
+                                        SlotState.ALLOCATED);
+                                returnedFuture.complete(null);
+                            } else {
+                                if (throwable instanceof SlotOccupiedException) {
+                                    LOG.error("Should not get this exception.", throwable);
+                                } else {
+                                    // TODO If the taskManager does not have enough resource, we
+                                    // may endlessly allocate slot on it until the next heartbeat.
+                                    LOG.warn(
+                                            "Slot allocation for allocation {} for job {} failed.",
+                                            allocationId,
+                                            jobId,
+                                            throwable);
+                                    resourceTracker.notifyLostResource(jobId, resourceProfile);
+                                    taskManagerTracker.notifySlotStatus(
+                                            allocationId,
+                                            jobId,
+                                            instanceId,
+                                            resourceProfile,
+                                            SlotState.FREE);
+                                }
+                                returnedFuture.completeExceptionally(throwable);
+                            }
+                            return null;
+                        },
+                        mainThreadExecutor));
+        return returnedFuture;
+    }
+
+    @Override
+    public void freeSlot(AllocationID allocationId) {
+        LOG.debug("Freeing slot {}.", allocationId);
+
+        Optional<TaskManagerSlotInformation> slot =
+                taskManagerTracker.getAllocatedOrPendingSlot(allocationId);
+        if (!slot.isPresent()) {
+            LOG.warn("Try to free unknown slot {}.", allocationId);
+        }
+
+        resourceTracker.notifyLostResource(slot.get().getJobId(), slot.get().getResourceProfile());
+        taskManagerTracker.notifySlotStatus(
+                allocationId,
+                slot.get().getJobId(),
+                slot.get().getInstanceId(),
+                slot.get().getResourceProfile(),
+                SlotState.FREE);
+    }
+
+    @Override
+    public void reportSlotStatus(InstanceID instanceId, SlotReport slotReport) {
+        Preconditions.checkNotNull(slotReport);
+        Preconditions.checkNotNull(instanceId);
+        final Optional<TaskManagerInfo> taskManager =
+                taskManagerTracker.getRegisteredTaskManager(instanceId);
+
+        if (!taskManager.isPresent()) {
+            LOG.debug(
+                    "Received slot report for unknown task manager with instance id {}. Ignoring this report.",
+                    instanceId);
+            return;
+        }
+
+        LOG.debug("Received slot report from instance {}: {}.", instanceId, slotReport);
+
+        final Set<AllocationID> reportedAllocationIds = new HashSet<>();
+        slotReport
+                .iterator()
+                .forEachRemaining(
+                        slotStatus -> reportedAllocationIds.add(slotStatus.getAllocationID()));
+
+        for (TaskManagerSlotInformation slot :
+                new HashSet<>(taskManager.get().getAllocatedSlots().values())) {
+            // The slot was previously Allocated but is now freed.
+            if (!reportedAllocationIds.contains(slot.getAllocationId())
+                    && slot.getState() == SlotState.ALLOCATED) {

Review comment:
       It took me a while to understand that if the slot state is `PENDING` and is not reported, we might wait for the next slot report. Maybe this worth an explanation in comments.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/StatusOverview.java
##########
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.util.Preconditions;
+
+import java.util.Map;
+
+/** An overview of the resource allocation of the current cluster. */
+public class StatusOverview {
+    private final Map<InstanceID, TaskManagerInfo> taskManagers;
+
+    public StatusOverview(Map<InstanceID, TaskManagerInfo> taskManagers) {
+        this.taskManagers = Preconditions.checkNotNull(taskManagers);
+    }
+
+    public int getNumberRegisteredSlots() {
+        return taskManagers.values().stream().mapToInt(TaskManagerInfo::getDefaultNumSlots).sum();
+    }
+
+    public int getNumberRegisteredSlotsOf(InstanceID instanceId) {
+        return taskManagers.containsKey(instanceId)
+                ? taskManagers.get(instanceId).getDefaultNumSlots()
+                : 0;
+    }
+
+    public int getNumberFreeSlots() {
+        return taskManagers.keySet().stream().mapToInt(this::getNumberFreeSlotsOf).sum();
+    }
+
+    public int getNumberFreeSlotsOf(InstanceID instanceId) {
+        return taskManagers.containsKey(instanceId)
+                ? Math.max(
+                        taskManagers.get(instanceId).getDefaultNumSlots()
+                                - taskManagers.get(instanceId).getAllocatedSlots().size(),
+                        0)
+                : 0;
+    }
+
+    public ResourceProfile getRegisteredResource() {
+        return taskManagers.values().stream()
+                .map(TaskManagerInfo::getTotalResource)
+                .reduce(ResourceProfile.ZERO, ResourceProfile::merge);
+    }
+
+    public ResourceProfile getRegisteredResourceOf(InstanceID instanceId) {
+        return taskManagers.containsKey(instanceId)
+                ? taskManagers.get(instanceId).getTotalResource()
+                : ResourceProfile.ZERO;
+    }
+
+    public ResourceProfile getFreeResource() {
+        return taskManagers.values().stream()
+                .map(TaskManagerInfo::getAvailableResource)
+                .reduce(ResourceProfile.ZERO, ResourceProfile::merge);
+    }
+
+    public ResourceProfile getFreeResourceOf(InstanceID instanceId) {
+        return taskManagers.containsKey(instanceId)
+                ? taskManagers.get(instanceId).getAvailableResource()
+                : ResourceProfile.ZERO;
+    }

Review comment:
       Similar to `getNumberRegisteredSlotsOf`, duplicated map lookup can be avoided.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/PendingTaskManagerId.java
##########
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.util.AbstractID;
+
+/** Id of {@link PendingTaskManager}. */
+public class PendingTaskManagerId extends AbstractID {
+    private PendingTaskManagerId() {}

Review comment:
       Should call `super()`

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/FineGrainedTaskManagerRegistration.java
##########
@@ -0,0 +1,182 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.util.Preconditions;
+
+import java.math.BigDecimal;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * A FineGrainedTaskManagerRegistration represents a TaskManager. It records the internal state of
+ * the TaskManager, including allocated/pending slots, total/available resources.
+ *
+ * <p>This class is the fine-grained resource management version of the {@link
+ * TaskManagerRegistration}.
+ */
+public class FineGrainedTaskManagerRegistration implements TaskManagerInfo {
+    private final TaskExecutorConnection taskManagerConnection;
+
+    private final Map<AllocationID, FineGrainedTaskManagerSlot> slots;
+
+    private final ResourceProfile defaultSlotResourceProfile;
+
+    private final ResourceProfile totalResource;
+
+    private final int defaultNumSlots;
+
+    private ResourceProfile unusedResource;
+
+    private ResourceProfile pendingResource = ResourceProfile.ZERO;
+
+    /** Timestamp when the last time becoming idle. Otherwise Long.MAX_VALUE. */
+    private long idleSince;
+
+    public FineGrainedTaskManagerRegistration(
+            TaskExecutorConnection taskManagerConnection,
+            ResourceProfile totalResourceProfile,
+            ResourceProfile defaultSlotResourceProfile) {
+        this.taskManagerConnection = Preconditions.checkNotNull(taskManagerConnection);
+        this.totalResource = Preconditions.checkNotNull(totalResourceProfile);
+        this.defaultSlotResourceProfile = Preconditions.checkNotNull(defaultSlotResourceProfile);
+
+        this.slots = new HashMap<>(16);
+
+        this.defaultNumSlots =
+                totalResourceProfile
+                        .getCpuCores()
+                        .getValue()
+                        .divide(
+                                defaultSlotResourceProfile.getCpuCores().getValue(),
+                                0,
+                                BigDecimal.ROUND_DOWN)
+                        .intValue();
+
+        this.unusedResource =
+                ResourceProfile.newBuilder()
+                        .setCpuCores(totalResourceProfile.getCpuCores())
+                        .setTaskHeapMemory(totalResourceProfile.getTaskHeapMemory())
+                        .setTaskOffHeapMemory(totalResourceProfile.getTaskOffHeapMemory())
+                        .setNetworkMemory(totalResourceProfile.getNetworkMemory())
+                        .setManagedMemory(totalResourceProfile.getManagedMemory())
+                        .addExtendedResources(totalResourceProfile.getExtendedResources())
+                        .build();
+
+        idleSince = System.currentTimeMillis();
+    }
+
+    @Override
+    public TaskExecutorConnection getTaskExecutorConnection() {
+        return taskManagerConnection;
+    }
+
+    @Override
+    public InstanceID getInstanceId() {
+        return taskManagerConnection.getInstanceID();
+    }
+
+    @Override
+    public Map<AllocationID, TaskManagerSlotInformation> getAllocatedSlots() {
+        return Collections.unmodifiableMap(slots);
+    }
+
+    @Override
+    public ResourceProfile getAvailableResource() {
+        if (!unusedResource.allFieldsNoLessThan(pendingResource)) {
+            return ResourceProfile.ZERO;
+        }
+        return unusedResource.subtract(pendingResource);
+    }
+
+    @Override
+    public ResourceProfile getDefaultSlotResourceProfile() {
+        return defaultSlotResourceProfile;
+    }
+
+    @Override
+    public ResourceProfile getTotalResource() {
+        return totalResource;
+    }
+
+    @Override
+    public int getDefaultNumSlots() {
+        return defaultNumSlots;
+    }
+
+    @Override
+    public long getIdleSince() {
+        return idleSince;
+    }
+
+    @Override
+    public boolean isIdle() {
+        return idleSince != Long.MAX_VALUE;
+    }
+
+    public void freeSlot(AllocationID allocationId) {
+        Preconditions.checkNotNull(allocationId);
+        FineGrainedTaskManagerSlot taskManagerSlot =
+                Preconditions.checkNotNull(slots.remove(allocationId));
+
+        if (taskManagerSlot.getState() == SlotState.PENDING) {
+            pendingResource = pendingResource.subtract(taskManagerSlot.getResourceProfile());
+        } else {
+            unusedResource = unusedResource.merge(taskManagerSlot.getResourceProfile());
+        }
+
+        if (slots.isEmpty()) {
+            idleSince = System.currentTimeMillis();
+        }
+    }
+
+    public void notifyAllocationComplete(AllocationID allocationId) {
+        FineGrainedTaskManagerSlot slot = Preconditions.checkNotNull(slots.get(allocationId));
+        Preconditions.checkState(slot.getState() == SlotState.PENDING);
+        slot.completeAllocation();
+        pendingResource = pendingResource.subtract(slot.getResourceProfile());
+        unusedResource = unusedResource.subtract(slot.getResourceProfile());
+    }
+
+    public void notifyAllocationStart(
+            AllocationID allocationId, FineGrainedTaskManagerSlot taskManagerSlot) {
+        Preconditions.checkNotNull(allocationId);
+        Preconditions.checkNotNull(taskManagerSlot);
+        Preconditions.checkState(taskManagerSlot.getState() == SlotState.PENDING);
+        slots.put(allocationId, taskManagerSlot);
+        pendingResource = pendingResource.merge(taskManagerSlot.getResourceProfile());
+        idleSince = Long.MAX_VALUE;
+    }
+
+    public void occupySlot(FineGrainedTaskManagerSlot taskManagerSlot, AllocationID allocationID) {
+        Preconditions.checkNotNull(allocationID);
+        Preconditions.checkNotNull(taskManagerSlot);
+        Preconditions.checkState(taskManagerSlot.getState() == SlotState.ALLOCATED);
+
+        unusedResource = unusedResource.subtract(taskManagerSlot.getResourceProfile());
+        slots.put(allocationID, taskManagerSlot);
+
+        idleSince = Long.MAX_VALUE;
+    }

Review comment:
       IIUC, `occupySlot` is equivalent to call `notifyAllocationStart` and `notifyAllocationComplete` consecutively?
   Maybe we can combine `occupySlot` and `notifyAllocationStart` into `notifyAllocation`, and update `pendingResource` and `unusedResource` differently according to the slot state?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/StatusOverview.java
##########
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.util.Preconditions;
+
+import java.util.Map;
+
+/** An overview of the resource allocation of the current cluster. */
+public class StatusOverview {
+    private final Map<InstanceID, TaskManagerInfo> taskManagers;
+
+    public StatusOverview(Map<InstanceID, TaskManagerInfo> taskManagers) {
+        this.taskManagers = Preconditions.checkNotNull(taskManagers);
+    }
+
+    public int getNumberRegisteredSlots() {
+        return taskManagers.values().stream().mapToInt(TaskManagerInfo::getDefaultNumSlots).sum();
+    }
+
+    public int getNumberRegisteredSlotsOf(InstanceID instanceId) {
+        return taskManagers.containsKey(instanceId)
+                ? taskManagers.get(instanceId).getDefaultNumSlots()
+                : 0;
+    }
+
+    public int getNumberFreeSlots() {
+        return taskManagers.keySet().stream().mapToInt(this::getNumberFreeSlotsOf).sum();
+    }
+
+    public int getNumberFreeSlotsOf(InstanceID instanceId) {
+        return taskManagers.containsKey(instanceId)
+                ? Math.max(
+                        taskManagers.get(instanceId).getDefaultNumSlots()
+                                - taskManagers.get(instanceId).getAllocatedSlots().size(),
+                        0)
+                : 0;
+    }

Review comment:
       Similar to `getNumberRegisteredSlotsOf`, duplicated map lookup can be avoided.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/StatusOverview.java
##########
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.util.Preconditions;
+
+import java.util.Map;
+
+/** An overview of the resource allocation of the current cluster. */
+public class StatusOverview {
+    private final Map<InstanceID, TaskManagerInfo> taskManagers;
+
+    public StatusOverview(Map<InstanceID, TaskManagerInfo> taskManagers) {
+        this.taskManagers = Preconditions.checkNotNull(taskManagers);
+    }
+
+    public int getNumberRegisteredSlots() {
+        return taskManagers.values().stream().mapToInt(TaskManagerInfo::getDefaultNumSlots).sum();
+    }
+
+    public int getNumberRegisteredSlotsOf(InstanceID instanceId) {
+        return taskManagers.containsKey(instanceId)
+                ? taskManagers.get(instanceId).getDefaultNumSlots()
+                : 0;
+    }
+
+    public int getNumberFreeSlots() {
+        return taskManagers.keySet().stream().mapToInt(this::getNumberFreeSlotsOf).sum();
+    }
+
+    public int getNumberFreeSlotsOf(InstanceID instanceId) {
+        return taskManagers.containsKey(instanceId)
+                ? Math.max(
+                        taskManagers.get(instanceId).getDefaultNumSlots()
+                                - taskManagers.get(instanceId).getAllocatedSlots().size(),
+                        0)
+                : 0;
+    }
+
+    public ResourceProfile getRegisteredResource() {
+        return taskManagers.values().stream()
+                .map(TaskManagerInfo::getTotalResource)
+                .reduce(ResourceProfile.ZERO, ResourceProfile::merge);
+    }
+
+    public ResourceProfile getRegisteredResourceOf(InstanceID instanceId) {
+        return taskManagers.containsKey(instanceId)
+                ? taskManagers.get(instanceId).getTotalResource()
+                : ResourceProfile.ZERO;
+    }

Review comment:
       Similar to `getNumberRegisteredSlotsOf`, duplicated map lookup can be avoided.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/FineGrainedTaskManagerTrackerTest.java
##########
@@ -0,0 +1,254 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.slots.ResourceCounter;
+import org.apache.flink.runtime.taskexecutor.TestingTaskExecutorGatewayBuilder;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Test;
+
+import java.util.Collections;
+
+import static org.hamcrest.Matchers.empty;
+import static org.hamcrest.core.Is.is;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+
+/** Tests for the {@link FineGrainedTaskManagerTracker}. */
+public class FineGrainedTaskManagerTrackerTest extends TestLogger {

Review comment:
       I think we might need to cover more cases.
   - When `recordPendingAllocations` is called multiple times, only the last time is recorded.
   - `removePendingTaskManager` called on pending task manager where pending slots are allocated.
   And also some failure cases.
   - Trying to remove a (pending) task manager that does not exist.
   - Try to allocate slot from a (pending) task manager that does not have enough resource
   - Try to free a non-exist slot.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/FineGrainedTaskManagerSlot.java
##########
@@ -0,0 +1,113 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.util.Preconditions;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * A FineGrainedTaskManagerSlot represents a slot located in a TaskManager. It contains the
+ * necessary information for the allocation of the slot, and keeps track of the state of the slot.
+ * Note that it should in the state of {@link SlotState#FREE}.

Review comment:
       should *not* in 

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/FineGrainedTaskManagerTracker.java
##########
@@ -0,0 +1,237 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.slots.ResourceCounter;
+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.HashSet;
+import java.util.Map;
+import java.util.Optional;
+
+/** Implementation of {@link TaskManagerTracker} supporting fine-grained resource management. */
+public class FineGrainedTaskManagerTracker implements TaskManagerTracker {
+    private static final Logger LOG = LoggerFactory.getLogger(FineGrainedTaskManagerTracker.class);
+
+    /** Map for allocated and pending slots. */
+    private final Map<AllocationID, FineGrainedTaskManagerSlot> slots;
+
+    /** All currently registered task managers. */
+    private final Map<InstanceID, FineGrainedTaskManagerRegistration> taskManagerRegistrations;
+
+    private final Map<PendingTaskManagerId, PendingTaskManager> pendingTaskManagers;
+
+    private final Map<PendingTaskManagerId, Map<JobID, ResourceCounter>>
+            pendingSlotAllocationRecords;
+
+    public FineGrainedTaskManagerTracker() {
+        slots = new HashMap<>(16);
+        taskManagerRegistrations = new HashMap<>(4);
+        pendingTaskManagers = new HashMap<>(16);
+        pendingSlotAllocationRecords = new HashMap<>();
+    }
+
+    @Override
+    public void recordPendingAllocations(
+            Map<PendingTaskManagerId, Map<JobID, ResourceCounter>> pendingSlotAllocations) {
+        LOG.trace("Record the pending allocations {}.", pendingSlotAllocations);
+        Preconditions.checkNotNull(pendingSlotAllocations);
+        pendingSlotAllocationRecords.clear();
+        pendingSlotAllocationRecords.putAll(pendingSlotAllocations);
+    }
+
+    @Override
+    public void addTaskManager(
+            TaskExecutorConnection taskExecutorConnection,
+            ResourceProfile totalResourceProfile,
+            ResourceProfile defaultSlotResourceProfile) {
+        Preconditions.checkNotNull(taskExecutorConnection);
+        Preconditions.checkNotNull(totalResourceProfile);
+        Preconditions.checkNotNull(defaultSlotResourceProfile);
+        LOG.debug(
+                "Add task manager {} with total resource {} and default slot resource {}.",
+                taskExecutorConnection.getInstanceID(),
+                totalResourceProfile,
+                defaultSlotResourceProfile);
+        final FineGrainedTaskManagerRegistration taskManagerRegistration =
+                new FineGrainedTaskManagerRegistration(
+                        taskExecutorConnection, totalResourceProfile, defaultSlotResourceProfile);
+        taskManagerRegistrations.put(
+                taskExecutorConnection.getInstanceID(), taskManagerRegistration);
+    }
+
+    @Override
+    public void removeTaskManager(InstanceID instanceId) {
+        LOG.debug("Remove task manager {}.", instanceId);
+        final FineGrainedTaskManagerRegistration taskManager =
+                Preconditions.checkNotNull(taskManagerRegistrations.get(instanceId));
+        for (AllocationID allocationId : new HashSet<>(taskManager.getAllocatedSlots().keySet())) {
+            slots.remove(allocationId);
+            taskManager.freeSlot(allocationId);
+        }
+        taskManagerRegistrations.remove(instanceId);

Review comment:
       Can this be simplified as follow?
   ```
           final FineGrainedTaskManagerRegistration taskManager =
                   Preconditions.checkNotNull(taskManagerRegistrations.remove(instanceId));
           for (AllocationID allocationId : new HashSet<>(taskManager.getAllocatedSlots().keySet())) {
               slots.remove(allocationId);
           }
   ```
   Seems there's no need to call `taskManager.freeSlot` since we are removing `taskManager` anyway.
   And we avoid duplicate lookup to the map `taskManagerRegistrations`.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/FineGrainedTaskManagerTracker.java
##########
@@ -0,0 +1,237 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.slots.ResourceCounter;
+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.HashSet;
+import java.util.Map;
+import java.util.Optional;
+
+/** Implementation of {@link TaskManagerTracker} supporting fine-grained resource management. */
+public class FineGrainedTaskManagerTracker implements TaskManagerTracker {
+    private static final Logger LOG = LoggerFactory.getLogger(FineGrainedTaskManagerTracker.class);
+
+    /** Map for allocated and pending slots. */
+    private final Map<AllocationID, FineGrainedTaskManagerSlot> slots;
+
+    /** All currently registered task managers. */
+    private final Map<InstanceID, FineGrainedTaskManagerRegistration> taskManagerRegistrations;
+
+    private final Map<PendingTaskManagerId, PendingTaskManager> pendingTaskManagers;
+
+    private final Map<PendingTaskManagerId, Map<JobID, ResourceCounter>>
+            pendingSlotAllocationRecords;
+
+    public FineGrainedTaskManagerTracker() {
+        slots = new HashMap<>(16);
+        taskManagerRegistrations = new HashMap<>(4);
+        pendingTaskManagers = new HashMap<>(16);

Review comment:
       > Set the initial capacity for a collection only if there is a good proven reason for that, otherwise do not clutter the code. In case of Maps it can be even deluding because the Map’s load factor effectively reduces the capacity.
   
   
   https://flink.apache.org/contributing/code-style-and-quality-java.html#collections

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/DefaultSlotStatusSyncer.java
##########
@@ -0,0 +1,258 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.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.FutureUtils;
+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.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.HashSet;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+
+/** Default implementation of {@link SlotStatusSyncer} for fine-grained slot management. */
+public class DefaultSlotStatusSyncer implements SlotStatusSyncer {
+    private static final Logger LOG = LoggerFactory.getLogger(DefaultSlotStatusSyncer.class);
+
+    private final TaskManagerTracker taskManagerTracker;
+    private final ResourceTracker resourceTracker;
+    private final Set<AllocationID> pendingSlotAllocations;
+    /** Timeout for slot requests to the task manager. */
+    private final Time taskManagerRequestTimeout;
+
+    public DefaultSlotStatusSyncer(
+            TaskManagerTracker taskManagerTracker,
+            ResourceTracker resourceTracker,
+            Time taskManagerRequestTimeout) {
+        this.taskManagerTracker = Preconditions.checkNotNull(taskManagerTracker);
+        this.resourceTracker = Preconditions.checkNotNull(resourceTracker);
+        this.taskManagerRequestTimeout = Preconditions.checkNotNull(taskManagerRequestTimeout);
+
+        this.pendingSlotAllocations = new HashSet<>(16);
+    }
+
+    @Override
+    public CompletableFuture<Void> allocateSlot(
+            InstanceID instanceId,
+            JobID jobId,
+            String targetAddress,
+            ResourceProfile resourceProfile,
+            ResourceManagerId resourceManagerId,
+            Executor mainThreadExecutor) {
+        final AllocationID allocationId = new AllocationID();
+        final Optional<TaskManagerInfo> taskManager =
+                taskManagerTracker.getRegisteredTaskManager(instanceId);
+        Preconditions.checkState(
+                taskManager.isPresent(),
+                "Could not find a registered task manager for instance id " + instanceId + '.');
+        final TaskExecutorGateway gateway =
+                taskManager.get().getTaskExecutorConnection().getTaskExecutorGateway();
+
+        taskManagerTracker.notifySlotStatus(
+                allocationId, jobId, instanceId, resourceProfile, SlotState.PENDING);
+        resourceTracker.notifyAcquiredResource(jobId, resourceProfile);
+        pendingSlotAllocations.add(allocationId);
+
+        // RPC call to the task manager
+        CompletableFuture<Acknowledge> requestFuture =
+                gateway.requestSlot(
+                        SlotID.getDynamicSlotID(
+                                taskManager.get().getTaskExecutorConnection().getResourceID()),
+                        jobId,
+                        allocationId,
+                        resourceProfile,
+                        targetAddress,
+                        resourceManagerId,
+                        taskManagerRequestTimeout);
+
+        CompletableFuture<Void> returnedFuture = new CompletableFuture<>();
+
+        FutureUtils.assertNoException(
+                requestFuture.handleAsync(

Review comment:
       Maybe `whenComplete`, since we don't need a return value.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/FineGrainedSlotManager.java
##########
@@ -0,0 +1,790 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.api.java.tuple.Tuple2;
+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.FutureUtils;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.instance.InstanceID;
+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.util.FlinkException;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+/** Implementation of {@link SlotManager} supporting fine-grained resource management. */
+public class FineGrainedSlotManager implements SlotManager {
+    private static final Logger LOG = LoggerFactory.getLogger(FineGrainedSlotManager.class);
+
+    private final TaskManagerTracker taskManagerTracker;
+    private final ResourceTracker resourceTracker;
+    private final ResourceAllocationStrategy resourceAllocationStrategy;
+
+    private final SlotStatusSyncer slotStatusSyncer;
+
+    /** Scheduled executor for timeouts. */
+    private final ScheduledExecutor scheduledExecutor;
+
+    /** Timeout after which an unused TaskManager is released. */
+    private final Time taskManagerTimeout;
+
+    private final SlotManagerMetricGroup slotManagerMetricGroup;
+
+    private final Map<JobID, String> jobMasterTargetAddresses = new HashMap<>();
+
+    /** Defines the max limitation of the total number of task executors. */
+    private final int maxTaskManagerNum;
+
+    /** Defines the number of redundant task executors. */
+    private final int redundantTaskManagerNum;
+
+    /**
+     * Release task executor only when each produced result partition is either consumed or failed.
+     */
+    private final boolean waitResultConsumedBeforeRelease;
+
+    /** The default resource spec of workers to request. */
+    private final WorkerResourceSpec defaultWorkerResourceSpec;
+
+    /** The resource profile of default slot. */
+    private final ResourceProfile defaultSlotResourceProfile;
+
+    private boolean sendNotEnoughResourceNotifications = true;
+
+    /** ResourceManager's id. */
+    @Nullable private ResourceManagerId resourceManagerId;
+
+    /** Executor for future callbacks which have to be "synchronized". */
+    @Nullable private Executor mainThreadExecutor;
+
+    /** Callbacks for resource (de-)allocations. */
+    @Nullable private ResourceActions resourceActions;
+
+    private ScheduledFuture<?> taskManagerTimeoutsAndRedundancyCheck;
+
+    /** True iff the component has been started. */
+    private boolean started;
+
+    public FineGrainedSlotManager(
+            ScheduledExecutor scheduledExecutor,
+            SlotManagerConfiguration slotManagerConfiguration,
+            SlotManagerMetricGroup slotManagerMetricGroup,
+            ResourceTracker resourceTracker,
+            TaskManagerTracker taskManagerTracker) {
+
+        this.scheduledExecutor = Preconditions.checkNotNull(scheduledExecutor);
+
+        Preconditions.checkNotNull(slotManagerConfiguration);
+        this.taskManagerTimeout = slotManagerConfiguration.getTaskManagerTimeout();
+        this.redundantTaskManagerNum = slotManagerConfiguration.getRedundantTaskManagerNum();
+        this.waitResultConsumedBeforeRelease =
+                slotManagerConfiguration.isWaitResultConsumedBeforeRelease();
+        this.defaultWorkerResourceSpec = slotManagerConfiguration.getDefaultWorkerResourceSpec();
+        this.resourceAllocationStrategy = slotManagerConfiguration.getResourceAllocationStrategy();
+        int numSlotsPerWorker = slotManagerConfiguration.getNumSlotsPerWorker();
+        this.maxTaskManagerNum = slotManagerConfiguration.getMaxSlotNum() / numSlotsPerWorker;

Review comment:
       Not sure about converting max slot number to max TM number. This implicitly assume all TMs have the same number of slots.
   I think we should convert max slot number to max total resources <cpu, memory>, and make sure that is not exceeded.
   Given that this PR is already quite big. I would suggest to scope out the supporting for max slot number, make it a follow-up issue.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/FineGrainedTaskManagerTracker.java
##########
@@ -0,0 +1,237 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.slots.ResourceCounter;
+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.HashSet;
+import java.util.Map;
+import java.util.Optional;
+
+/** Implementation of {@link TaskManagerTracker} supporting fine-grained resource management. */
+public class FineGrainedTaskManagerTracker implements TaskManagerTracker {
+    private static final Logger LOG = LoggerFactory.getLogger(FineGrainedTaskManagerTracker.class);
+
+    /** Map for allocated and pending slots. */
+    private final Map<AllocationID, FineGrainedTaskManagerSlot> slots;
+
+    /** All currently registered task managers. */
+    private final Map<InstanceID, FineGrainedTaskManagerRegistration> taskManagerRegistrations;
+
+    private final Map<PendingTaskManagerId, PendingTaskManager> pendingTaskManagers;
+
+    private final Map<PendingTaskManagerId, Map<JobID, ResourceCounter>>
+            pendingSlotAllocationRecords;
+
+    public FineGrainedTaskManagerTracker() {
+        slots = new HashMap<>(16);
+        taskManagerRegistrations = new HashMap<>(4);
+        pendingTaskManagers = new HashMap<>(16);
+        pendingSlotAllocationRecords = new HashMap<>();
+    }
+
+    @Override
+    public void recordPendingAllocations(
+            Map<PendingTaskManagerId, Map<JobID, ResourceCounter>> pendingSlotAllocations) {
+        LOG.trace("Record the pending allocations {}.", pendingSlotAllocations);
+        Preconditions.checkNotNull(pendingSlotAllocations);
+        pendingSlotAllocationRecords.clear();
+        pendingSlotAllocationRecords.putAll(pendingSlotAllocations);
+    }
+
+    @Override
+    public void addTaskManager(
+            TaskExecutorConnection taskExecutorConnection,
+            ResourceProfile totalResourceProfile,
+            ResourceProfile defaultSlotResourceProfile) {
+        Preconditions.checkNotNull(taskExecutorConnection);
+        Preconditions.checkNotNull(totalResourceProfile);
+        Preconditions.checkNotNull(defaultSlotResourceProfile);
+        LOG.debug(
+                "Add task manager {} with total resource {} and default slot resource {}.",
+                taskExecutorConnection.getInstanceID(),
+                totalResourceProfile,
+                defaultSlotResourceProfile);
+        final FineGrainedTaskManagerRegistration taskManagerRegistration =
+                new FineGrainedTaskManagerRegistration(
+                        taskExecutorConnection, totalResourceProfile, defaultSlotResourceProfile);
+        taskManagerRegistrations.put(
+                taskExecutorConnection.getInstanceID(), taskManagerRegistration);
+    }
+
+    @Override
+    public void removeTaskManager(InstanceID instanceId) {
+        LOG.debug("Remove task manager {}.", instanceId);
+        final FineGrainedTaskManagerRegistration taskManager =
+                Preconditions.checkNotNull(taskManagerRegistrations.get(instanceId));
+        for (AllocationID allocationId : new HashSet<>(taskManager.getAllocatedSlots().keySet())) {
+            slots.remove(allocationId);
+            taskManager.freeSlot(allocationId);
+        }
+        taskManagerRegistrations.remove(instanceId);
+    }
+
+    @Override
+    public void addPendingTaskManager(PendingTaskManager pendingTaskManager) {
+        LOG.debug("Add pending task manager {}.", pendingTaskManager);
+        Preconditions.checkNotNull(pendingTaskManager);
+        pendingTaskManagers.put(pendingTaskManager.getPendingTaskManagerId(), pendingTaskManager);
+    }
+
+    @Override
+    public void removePendingTaskManager(PendingTaskManagerId pendingTaskManagerId) {
+        Preconditions.checkNotNull(pendingTaskManagerId);
+        LOG.debug("Remove pending task manager {}.", pendingTaskManagerId);
+        pendingTaskManagers.remove(pendingTaskManagerId);

Review comment:
       What happens to slot allocations that are allocated to this pending task manager?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/DefaultSlotStatusSyncer.java
##########
@@ -0,0 +1,258 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.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.FutureUtils;
+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.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.HashSet;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+
+/** Default implementation of {@link SlotStatusSyncer} for fine-grained slot management. */
+public class DefaultSlotStatusSyncer implements SlotStatusSyncer {
+    private static final Logger LOG = LoggerFactory.getLogger(DefaultSlotStatusSyncer.class);
+
+    private final TaskManagerTracker taskManagerTracker;
+    private final ResourceTracker resourceTracker;
+    private final Set<AllocationID> pendingSlotAllocations;
+    /** Timeout for slot requests to the task manager. */
+    private final Time taskManagerRequestTimeout;
+
+    public DefaultSlotStatusSyncer(
+            TaskManagerTracker taskManagerTracker,
+            ResourceTracker resourceTracker,
+            Time taskManagerRequestTimeout) {
+        this.taskManagerTracker = Preconditions.checkNotNull(taskManagerTracker);
+        this.resourceTracker = Preconditions.checkNotNull(resourceTracker);
+        this.taskManagerRequestTimeout = Preconditions.checkNotNull(taskManagerRequestTimeout);
+
+        this.pendingSlotAllocations = new HashSet<>(16);

Review comment:
       Initial capacity

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/FineGrainedTaskManagerTracker.java
##########
@@ -0,0 +1,237 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.slots.ResourceCounter;
+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.HashSet;
+import java.util.Map;
+import java.util.Optional;
+
+/** Implementation of {@link TaskManagerTracker} supporting fine-grained resource management. */
+public class FineGrainedTaskManagerTracker implements TaskManagerTracker {
+    private static final Logger LOG = LoggerFactory.getLogger(FineGrainedTaskManagerTracker.class);
+
+    /** Map for allocated and pending slots. */
+    private final Map<AllocationID, FineGrainedTaskManagerSlot> slots;
+
+    /** All currently registered task managers. */
+    private final Map<InstanceID, FineGrainedTaskManagerRegistration> taskManagerRegistrations;
+
+    private final Map<PendingTaskManagerId, PendingTaskManager> pendingTaskManagers;
+
+    private final Map<PendingTaskManagerId, Map<JobID, ResourceCounter>>
+            pendingSlotAllocationRecords;
+
+    public FineGrainedTaskManagerTracker() {
+        slots = new HashMap<>(16);
+        taskManagerRegistrations = new HashMap<>(4);
+        pendingTaskManagers = new HashMap<>(16);
+        pendingSlotAllocationRecords = new HashMap<>();
+    }
+
+    @Override
+    public void recordPendingAllocations(
+            Map<PendingTaskManagerId, Map<JobID, ResourceCounter>> pendingSlotAllocations) {
+        LOG.trace("Record the pending allocations {}.", pendingSlotAllocations);
+        Preconditions.checkNotNull(pendingSlotAllocations);
+        pendingSlotAllocationRecords.clear();
+        pendingSlotAllocationRecords.putAll(pendingSlotAllocations);
+    }
+
+    @Override
+    public void addTaskManager(
+            TaskExecutorConnection taskExecutorConnection,
+            ResourceProfile totalResourceProfile,
+            ResourceProfile defaultSlotResourceProfile) {
+        Preconditions.checkNotNull(taskExecutorConnection);
+        Preconditions.checkNotNull(totalResourceProfile);
+        Preconditions.checkNotNull(defaultSlotResourceProfile);
+        LOG.debug(
+                "Add task manager {} with total resource {} and default slot resource {}.",
+                taskExecutorConnection.getInstanceID(),
+                totalResourceProfile,
+                defaultSlotResourceProfile);
+        final FineGrainedTaskManagerRegistration taskManagerRegistration =
+                new FineGrainedTaskManagerRegistration(
+                        taskExecutorConnection, totalResourceProfile, defaultSlotResourceProfile);
+        taskManagerRegistrations.put(
+                taskExecutorConnection.getInstanceID(), taskManagerRegistration);
+    }
+
+    @Override
+    public void removeTaskManager(InstanceID instanceId) {
+        LOG.debug("Remove task manager {}.", instanceId);
+        final FineGrainedTaskManagerRegistration taskManager =
+                Preconditions.checkNotNull(taskManagerRegistrations.get(instanceId));
+        for (AllocationID allocationId : new HashSet<>(taskManager.getAllocatedSlots().keySet())) {
+            slots.remove(allocationId);
+            taskManager.freeSlot(allocationId);
+        }
+        taskManagerRegistrations.remove(instanceId);
+    }
+
+    @Override
+    public void addPendingTaskManager(PendingTaskManager pendingTaskManager) {
+        LOG.debug("Add pending task manager {}.", pendingTaskManager);
+        Preconditions.checkNotNull(pendingTaskManager);
+        pendingTaskManagers.put(pendingTaskManager.getPendingTaskManagerId(), pendingTaskManager);
+    }
+
+    @Override
+    public void removePendingTaskManager(PendingTaskManagerId pendingTaskManagerId) {
+        Preconditions.checkNotNull(pendingTaskManagerId);
+        LOG.debug("Remove pending task manager {}.", pendingTaskManagerId);
+        pendingTaskManagers.remove(pendingTaskManagerId);

Review comment:
       The current implementation relies on the caller (`FineGrainedSlotManager`) to handle the allocations on the allocations on the pending task manager. It does not remove the pending allocations, until `recordPendingAllocations` is called again.
   
   The problem is that, internal states of `FineGrainedTaskManagerTracker` can become temporary inconsistent (keeping allocation records on non-exist pending task managers), and it depends on the correctness of the caller that this inconsistency won't cause problems.
   
     I would suggest to remove and return the related pending allocation records in `removePendingTaskManager`. In this way, the caller can continue handle the pending allocations, and `FineGrainedTaskManagerTracker ` stays consistent independent from how the caller handle the pending allocations.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/FineGrainedSlotManager.java
##########
@@ -0,0 +1,790 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.api.java.tuple.Tuple2;
+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.FutureUtils;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.instance.InstanceID;
+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.util.FlinkException;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+/** Implementation of {@link SlotManager} supporting fine-grained resource management. */
+public class FineGrainedSlotManager implements SlotManager {
+    private static final Logger LOG = LoggerFactory.getLogger(FineGrainedSlotManager.class);
+
+    private final TaskManagerTracker taskManagerTracker;
+    private final ResourceTracker resourceTracker;
+    private final ResourceAllocationStrategy resourceAllocationStrategy;
+
+    private final SlotStatusSyncer slotStatusSyncer;
+
+    /** Scheduled executor for timeouts. */
+    private final ScheduledExecutor scheduledExecutor;
+
+    /** Timeout after which an unused TaskManager is released. */
+    private final Time taskManagerTimeout;
+
+    private final SlotManagerMetricGroup slotManagerMetricGroup;
+
+    private final Map<JobID, String> jobMasterTargetAddresses = new HashMap<>();
+
+    /** Defines the max limitation of the total number of task executors. */
+    private final int maxTaskManagerNum;
+
+    /** Defines the number of redundant task executors. */
+    private final int redundantTaskManagerNum;
+
+    /**
+     * Release task executor only when each produced result partition is either consumed or failed.
+     */
+    private final boolean waitResultConsumedBeforeRelease;
+
+    /** The default resource spec of workers to request. */
+    private final WorkerResourceSpec defaultWorkerResourceSpec;
+
+    /** The resource profile of default slot. */
+    private final ResourceProfile defaultSlotResourceProfile;
+
+    private boolean sendNotEnoughResourceNotifications = true;
+
+    /** ResourceManager's id. */
+    @Nullable private ResourceManagerId resourceManagerId;
+
+    /** Executor for future callbacks which have to be "synchronized". */
+    @Nullable private Executor mainThreadExecutor;
+
+    /** Callbacks for resource (de-)allocations. */
+    @Nullable private ResourceActions resourceActions;
+
+    private ScheduledFuture<?> taskManagerTimeoutsAndRedundancyCheck;
+
+    /** True iff the component has been started. */
+    private boolean started;
+
+    public FineGrainedSlotManager(
+            ScheduledExecutor scheduledExecutor,
+            SlotManagerConfiguration slotManagerConfiguration,
+            SlotManagerMetricGroup slotManagerMetricGroup,
+            ResourceTracker resourceTracker,
+            TaskManagerTracker taskManagerTracker) {
+
+        this.scheduledExecutor = Preconditions.checkNotNull(scheduledExecutor);
+
+        Preconditions.checkNotNull(slotManagerConfiguration);
+        this.taskManagerTimeout = slotManagerConfiguration.getTaskManagerTimeout();
+        this.redundantTaskManagerNum = slotManagerConfiguration.getRedundantTaskManagerNum();
+        this.waitResultConsumedBeforeRelease =
+                slotManagerConfiguration.isWaitResultConsumedBeforeRelease();
+        this.defaultWorkerResourceSpec = slotManagerConfiguration.getDefaultWorkerResourceSpec();
+        this.resourceAllocationStrategy = slotManagerConfiguration.getResourceAllocationStrategy();
+        int numSlotsPerWorker = slotManagerConfiguration.getNumSlotsPerWorker();
+        this.maxTaskManagerNum = slotManagerConfiguration.getMaxSlotNum() / numSlotsPerWorker;
+
+        this.slotManagerMetricGroup = Preconditions.checkNotNull(slotManagerMetricGroup);
+        this.defaultSlotResourceProfile =
+                SlotManagerUtils.generateDefaultSlotResourceProfile(
+                        defaultWorkerResourceSpec, numSlotsPerWorker);
+
+        this.resourceTracker = Preconditions.checkNotNull(resourceTracker);
+        this.taskManagerTracker = Preconditions.checkNotNull(taskManagerTracker);
+        this.slotStatusSyncer =

Review comment:
       I think this should also be created externally and injected as a dependency.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/DefaultSlotStatusSyncer.java
##########
@@ -0,0 +1,258 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.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.FutureUtils;
+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.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.HashSet;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+
+/** Default implementation of {@link SlotStatusSyncer} for fine-grained slot management. */
+public class DefaultSlotStatusSyncer implements SlotStatusSyncer {
+    private static final Logger LOG = LoggerFactory.getLogger(DefaultSlotStatusSyncer.class);
+
+    private final TaskManagerTracker taskManagerTracker;
+    private final ResourceTracker resourceTracker;
+    private final Set<AllocationID> pendingSlotAllocations;
+    /** Timeout for slot requests to the task manager. */
+    private final Time taskManagerRequestTimeout;
+
+    public DefaultSlotStatusSyncer(
+            TaskManagerTracker taskManagerTracker,
+            ResourceTracker resourceTracker,
+            Time taskManagerRequestTimeout) {
+        this.taskManagerTracker = Preconditions.checkNotNull(taskManagerTracker);
+        this.resourceTracker = Preconditions.checkNotNull(resourceTracker);
+        this.taskManagerRequestTimeout = Preconditions.checkNotNull(taskManagerRequestTimeout);
+
+        this.pendingSlotAllocations = new HashSet<>(16);
+    }
+
+    @Override
+    public CompletableFuture<Void> allocateSlot(
+            InstanceID instanceId,
+            JobID jobId,
+            String targetAddress,
+            ResourceProfile resourceProfile,
+            ResourceManagerId resourceManagerId,
+            Executor mainThreadExecutor) {
+        final AllocationID allocationId = new AllocationID();
+        final Optional<TaskManagerInfo> taskManager =
+                taskManagerTracker.getRegisteredTaskManager(instanceId);
+        Preconditions.checkState(
+                taskManager.isPresent(),
+                "Could not find a registered task manager for instance id " + instanceId + '.');
+        final TaskExecutorGateway gateway =
+                taskManager.get().getTaskExecutorConnection().getTaskExecutorGateway();
+
+        taskManagerTracker.notifySlotStatus(
+                allocationId, jobId, instanceId, resourceProfile, SlotState.PENDING);
+        resourceTracker.notifyAcquiredResource(jobId, resourceProfile);
+        pendingSlotAllocations.add(allocationId);
+
+        // RPC call to the task manager
+        CompletableFuture<Acknowledge> requestFuture =
+                gateway.requestSlot(
+                        SlotID.getDynamicSlotID(
+                                taskManager.get().getTaskExecutorConnection().getResourceID()),
+                        jobId,
+                        allocationId,
+                        resourceProfile,
+                        targetAddress,
+                        resourceManagerId,
+                        taskManagerRequestTimeout);
+
+        CompletableFuture<Void> returnedFuture = new CompletableFuture<>();
+
+        FutureUtils.assertNoException(
+                requestFuture.handleAsync(
+                        (Acknowledge acknowledge, Throwable throwable) -> {
+                            if (!pendingSlotAllocations.remove(allocationId)) {
+                                LOG.debug(
+                                        "Ignoring slot allocation update from task manager {} for allocation {} and job {}, because the allocation was already completed or cancelled.",
+                                        instanceId,
+                                        allocationId,
+                                        jobId);
+                                returnedFuture.complete(null);
+                                return null;
+                            }
+                            if (!taskManagerTracker
+                                    .getAllocatedOrPendingSlot(allocationId)
+                                    .isPresent()) {
+                                LOG.debug(
+                                        "The slot {} has been removed before. Ignore the future.",
+                                        allocationId);
+                                requestFuture.complete(null);
+                                return null;
+                            }
+                            if (acknowledge != null) {
+                                LOG.trace(
+                                        "Completed allocation of allocation {} for job {}.",
+                                        allocationId,
+                                        jobId);
+                                taskManagerTracker.notifySlotStatus(
+                                        allocationId,
+                                        jobId,
+                                        instanceId,
+                                        resourceProfile,
+                                        SlotState.ALLOCATED);
+                                returnedFuture.complete(null);
+                            } else {
+                                if (throwable instanceof SlotOccupiedException) {
+                                    LOG.error("Should not get this exception.", throwable);
+                                } else {
+                                    // TODO If the taskManager does not have enough resource, we
+                                    // may endlessly allocate slot on it until the next heartbeat.
+                                    LOG.warn(
+                                            "Slot allocation for allocation {} for job {} failed.",
+                                            allocationId,
+                                            jobId,
+                                            throwable);
+                                    resourceTracker.notifyLostResource(jobId, resourceProfile);
+                                    taskManagerTracker.notifySlotStatus(
+                                            allocationId,
+                                            jobId,
+                                            instanceId,
+                                            resourceProfile,
+                                            SlotState.FREE);
+                                }
+                                returnedFuture.completeExceptionally(throwable);
+                            }
+                            return null;
+                        },
+                        mainThreadExecutor));
+        return returnedFuture;
+    }
+
+    @Override
+    public void freeSlot(AllocationID allocationId) {
+        LOG.debug("Freeing slot {}.", allocationId);
+
+        Optional<TaskManagerSlotInformation> slot =
+                taskManagerTracker.getAllocatedOrPendingSlot(allocationId);
+        if (!slot.isPresent()) {
+            LOG.warn("Try to free unknown slot {}.", allocationId);

Review comment:
       Shouldn't we `return` here?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/ResourceAllocationStrategy.java
##########
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+
+/** Strategy how to allocate slots and task managers to fulfill the unfulfilled requirements. */
+public interface ResourceAllocationStrategy {
+    ResourceAllocationStrategy NO_OP_STRATEGY =

Review comment:
       `@VisibleForTesting`

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/DefaultResourceAllocationStrategy.java
##########
@@ -0,0 +1,233 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.slots.ResourceCounter;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.runtime.resourcemanager.slotmanager.SlotManagerUtils.getEffectiveResourceProfile;
+
+/**
+ * The default implementation of {@link ResourceAllocationStrategy}, which always allocate pending
+ * task managers with the fixed profile.
+ */
+public class DefaultResourceAllocationStrategy implements ResourceAllocationStrategy {
+    private final ResourceProfile defaultSlotResourceProfile;
+    private final ResourceProfile totalResourceProfile;
+
+    public DefaultResourceAllocationStrategy(
+            ResourceProfile defaultSlotResourceProfile, int numSlotsPerWorker) {
+        this.defaultSlotResourceProfile = defaultSlotResourceProfile;
+        this.totalResourceProfile = defaultSlotResourceProfile.multiply(numSlotsPerWorker);
+    }
+
+    /**
+     * Matches resource requirements against available and pending resources. For each job, in a
+     * first round requirements are matched against registered resources. The remaining unfulfilled
+     * requirements are matched against pending resources, allocating more workers if no matching
+     * pending resources could be found. If the requirements for a job could not be fulfilled then
+     * it will be recorded in {@link ResourceAllocationResult#getUnfulfillableJobs()}.
+     *
+     * <p>Performance notes: At it's core this method loops, for each job, over all resources for
+     * each required slot, trying to find a matching registered/pending task manager. One should
+     * generally go in with the assumption that this runs in numberOfJobsRequiringResources *
+     * numberOfRequiredSlots * numberOfFreeOrPendingTaskManagers.
+     *
+     * <p>In the absolute worst case, with J jobs, requiring R slots each with a unique resource
+     * profile such each pair of these profiles is not matching, and T registered/pending task
+     * managers that don't fulfill any requirement, then this method does a total of J*R*T resource
+     * profile comparisons.
+     */

Review comment:
       The computation is way too heavy at the moment.
   Given that this PR is already quite big, I'm ok with it as the first step. But we should open a follow-up ticket to optimize it.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/DefaultResourceAllocationStrategy.java
##########
@@ -0,0 +1,233 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.slots.ResourceCounter;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.runtime.resourcemanager.slotmanager.SlotManagerUtils.getEffectiveResourceProfile;
+
+/**
+ * The default implementation of {@link ResourceAllocationStrategy}, which always allocate pending
+ * task managers with the fixed profile.
+ */
+public class DefaultResourceAllocationStrategy implements ResourceAllocationStrategy {
+    private final ResourceProfile defaultSlotResourceProfile;
+    private final ResourceProfile totalResourceProfile;
+
+    public DefaultResourceAllocationStrategy(
+            ResourceProfile defaultSlotResourceProfile, int numSlotsPerWorker) {
+        this.defaultSlotResourceProfile = defaultSlotResourceProfile;
+        this.totalResourceProfile = defaultSlotResourceProfile.multiply(numSlotsPerWorker);
+    }
+
+    /**
+     * Matches resource requirements against available and pending resources. For each job, in a
+     * first round requirements are matched against registered resources. The remaining unfulfilled
+     * requirements are matched against pending resources, allocating more workers if no matching
+     * pending resources could be found. If the requirements for a job could not be fulfilled then
+     * it will be recorded in {@link ResourceAllocationResult#getUnfulfillableJobs()}.
+     *
+     * <p>Performance notes: At it's core this method loops, for each job, over all resources for
+     * each required slot, trying to find a matching registered/pending task manager. One should
+     * generally go in with the assumption that this runs in numberOfJobsRequiringResources *
+     * numberOfRequiredSlots * numberOfFreeOrPendingTaskManagers.
+     *
+     * <p>In the absolute worst case, with J jobs, requiring R slots each with a unique resource
+     * profile such each pair of these profiles is not matching, and T registered/pending task
+     * managers that don't fulfill any requirement, then this method does a total of J*R*T resource
+     * profile comparisons.
+     */
+    @Override
+    public ResourceAllocationResult tryFulfillRequirements(
+            Map<JobID, Collection<ResourceRequirement>> missingResources,
+            Map<InstanceID, Tuple2<ResourceProfile, ResourceProfile>> registeredResources,
+            List<PendingTaskManager> pendingTaskManagers) {
+        final ResourceAllocationResult.Builder resultBuilder = ResourceAllocationResult.builder();
+        final Map<PendingTaskManagerId, ResourceProfile> pendingResources =
+                pendingTaskManagers.stream()
+                        .collect(
+                                Collectors.toMap(
+                                        PendingTaskManager::getPendingTaskManagerId,
+                                        PendingTaskManager::getTotalResourceProfile));
+        for (Map.Entry<JobID, Collection<ResourceRequirement>> resourceRequirements :
+                missingResources.entrySet()) {
+            final JobID jobId = resourceRequirements.getKey();
+
+            final ResourceCounter unfulfilledJobRequirements =
+                    tryFulfillRequirementsForJobWithRegisteredResources(
+                            jobId,
+                            resourceRequirements.getValue(),
+                            registeredResources,
+                            resultBuilder);
+
+            if (!unfulfilledJobRequirements.isEmpty()) {
+                tryFulfillRequirementsForJobWithPendingResources(
+                        jobId, unfulfilledJobRequirements, pendingResources, resultBuilder);
+            }
+        }
+        return resultBuilder.build();
+    }
+
+    private static ResourceCounter tryFulfillRequirementsForJobWithRegisteredResources(
+            JobID jobId,
+            Collection<ResourceRequirement> missingResources,
+            Map<InstanceID, Tuple2<ResourceProfile, ResourceProfile>> registeredResources,
+            ResourceAllocationResult.Builder resultBuilder) {
+        final ResourceCounter outstandingRequirements = new ResourceCounter();
+
+        for (ResourceRequirement resourceRequirement : missingResources) {
+            int numMissingRequirements =
+                    tryFindSlotsForRequirement(
+                            jobId, resourceRequirement, registeredResources, resultBuilder);
+            if (numMissingRequirements > 0) {
+                outstandingRequirements.incrementCount(
+                        resourceRequirement.getResourceProfile(), numMissingRequirements);
+            }
+        }
+        return outstandingRequirements;
+    }
+
+    private static int tryFindSlotsForRequirement(
+            JobID jobId,
+            ResourceRequirement resourceRequirement,
+            Map<InstanceID, Tuple2<ResourceProfile, ResourceProfile>> registeredResources,
+            ResourceAllocationResult.Builder resultBuilder) {
+        final ResourceProfile requiredResource = resourceRequirement.getResourceProfile();
+
+        int numUnfulfilled = 0;
+        for (int x = 0; x < resourceRequirement.getNumberOfRequiredSlots(); x++) {
+            final Optional<InstanceID> matchedTaskManager =
+                    findMatchingTaskManager(requiredResource, registeredResources);
+            if (matchedTaskManager.isPresent()) {
+                final ResourceProfile effectiveProfile =
+                        getEffectiveResourceProfile(
+                                requiredResource,
+                                registeredResources.get(matchedTaskManager.get()).f1);
+                resultBuilder.recordAllocationForRegisteredResource(
+                        jobId, matchedTaskManager.get(), effectiveProfile);
+                deductionRegisteredResource(
+                        registeredResources, matchedTaskManager.get(), effectiveProfile);
+            } else {
+                // exit loop early; we won't find a matching slot for this requirement
+                int numRemaining = resourceRequirement.getNumberOfRequiredSlots() - x;
+                numUnfulfilled += numRemaining;
+                break;
+            }
+        }
+        return numUnfulfilled;
+    }
+
+    private static Optional<InstanceID> findMatchingTaskManager(
+            ResourceProfile requirement,
+            Map<InstanceID, Tuple2<ResourceProfile, ResourceProfile>> registeredResources) {
+        return registeredResources.entrySet().stream()
+                .filter(
+                        taskManager ->
+                                canFulfillRequirement(
+                                        getEffectiveResourceProfile(
+                                                requirement, taskManager.getValue().f1),
+                                        taskManager.getValue().f0))
+                .findFirst()
+                .map(Map.Entry::getKey);
+    }
+
+    private static boolean canFulfillRequirement(
+            ResourceProfile requirement, ResourceProfile resourceProfile) {
+        return resourceProfile.allFieldsNoLessThan(requirement);
+    }
+
+    private static void deductionRegisteredResource(
+            Map<InstanceID, Tuple2<ResourceProfile, ResourceProfile>> registeredResources,
+            InstanceID instanceId,
+            ResourceProfile resourceProfile) {
+        registeredResources.computeIfPresent(
+                instanceId,
+                (id, tuple2) -> {
+                    if (tuple2.f0.subtract(resourceProfile).equals(ResourceProfile.ZERO)) {
+                        return null;
+                    } else {
+                        return Tuple2.of(tuple2.f0.subtract(resourceProfile), tuple2.f1);
+                    }
+                });
+    }
+
+    private static Optional<PendingTaskManagerId> findPendingManagerToFulfill(
+            ResourceProfile resourceProfile,
+            Map<PendingTaskManagerId, ResourceProfile> availableResources) {
+        return availableResources.entrySet().stream()
+                .filter(entry -> entry.getValue().allFieldsNoLessThan(resourceProfile))
+                .findAny()
+                .map(Map.Entry::getKey);
+    }
+
+    private void tryFulfillRequirementsForJobWithPendingResources(
+            JobID jobId,
+            ResourceCounter unfulfilledRequirements,
+            Map<PendingTaskManagerId, ResourceProfile> availableResources,
+            ResourceAllocationResult.Builder resultBuilder) {
+        for (Map.Entry<ResourceProfile, Integer> missingResource :
+                unfulfilledRequirements.getResourceProfilesWithCount().entrySet()) {
+            final ResourceProfile effectiveProfile =
+                    getEffectiveResourceProfile(
+                            missingResource.getKey(), defaultSlotResourceProfile);
+            for (int i = 0; i < missingResource.getValue(); i++) {
+                Optional<PendingTaskManagerId> matchedPendingTaskManager =
+                        findPendingManagerToFulfill(effectiveProfile, availableResources);
+                if (matchedPendingTaskManager.isPresent()) {
+                    availableResources.compute(
+                            matchedPendingTaskManager.get(),
+                            ((pendingTaskManagerId, resourceProfile) ->
+                                    resourceProfile.subtract(effectiveProfile)));
+                    resultBuilder.recordAllocationForPendingResource(
+                            jobId, matchedPendingTaskManager.get(), effectiveProfile);
+                } else {
+                    if (totalResourceProfile.allFieldsNoLessThan(effectiveProfile)) {
+                        // Add new pending task manager
+                        final PendingTaskManagerId pendingTaskManagerId =
+                                PendingTaskManagerId.generate();
+                        resultBuilder.addPendingTaskManagerToBeAllocated(
+                                new PendingTaskManager(
+                                        pendingTaskManagerId,
+                                        totalResourceProfile,
+                                        defaultSlotResourceProfile));
+                        resultBuilder.recordAllocationForPendingResource(
+                                jobId, pendingTaskManagerId, effectiveProfile);
+                        availableResources.put(
+                                pendingTaskManagerId,
+                                totalResourceProfile.subtract(effectiveProfile));
+                    } else {
+                        resultBuilder.addUnfulfillableJob(jobId);

Review comment:
       If a job contains unfulfillable requirements, do we still allocate for other fulfillable requirements of the same job? An unfulfillable job will probably fail anyway, maybe we should not allocate for this job at all, save the resources for the fulfillable jobs.
   
   Not saying we should do that in this PR. I don't have a clear idea atm. Just this might makes an interesting follow-up question.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/FineGrainedTaskManagerTracker.java
##########
@@ -0,0 +1,237 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.slots.ResourceCounter;
+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.HashSet;
+import java.util.Map;
+import java.util.Optional;
+
+/** Implementation of {@link TaskManagerTracker} supporting fine-grained resource management. */
+public class FineGrainedTaskManagerTracker implements TaskManagerTracker {
+    private static final Logger LOG = LoggerFactory.getLogger(FineGrainedTaskManagerTracker.class);
+
+    /** Map for allocated and pending slots. */
+    private final Map<AllocationID, FineGrainedTaskManagerSlot> slots;
+
+    /** All currently registered task managers. */
+    private final Map<InstanceID, FineGrainedTaskManagerRegistration> taskManagerRegistrations;
+
+    private final Map<PendingTaskManagerId, PendingTaskManager> pendingTaskManagers;
+
+    private final Map<PendingTaskManagerId, Map<JobID, ResourceCounter>>
+            pendingSlotAllocationRecords;
+
+    public FineGrainedTaskManagerTracker() {
+        slots = new HashMap<>(16);
+        taskManagerRegistrations = new HashMap<>(4);
+        pendingTaskManagers = new HashMap<>(16);
+        pendingSlotAllocationRecords = new HashMap<>();
+    }
+
+    @Override
+    public void recordPendingAllocations(
+            Map<PendingTaskManagerId, Map<JobID, ResourceCounter>> pendingSlotAllocations) {
+        LOG.trace("Record the pending allocations {}.", pendingSlotAllocations);
+        Preconditions.checkNotNull(pendingSlotAllocations);
+        pendingSlotAllocationRecords.clear();
+        pendingSlotAllocationRecords.putAll(pendingSlotAllocations);
+    }
+
+    @Override
+    public void addTaskManager(
+            TaskExecutorConnection taskExecutorConnection,
+            ResourceProfile totalResourceProfile,
+            ResourceProfile defaultSlotResourceProfile) {
+        Preconditions.checkNotNull(taskExecutorConnection);
+        Preconditions.checkNotNull(totalResourceProfile);
+        Preconditions.checkNotNull(defaultSlotResourceProfile);
+        LOG.debug(
+                "Add task manager {} with total resource {} and default slot resource {}.",
+                taskExecutorConnection.getInstanceID(),
+                totalResourceProfile,
+                defaultSlotResourceProfile);
+        final FineGrainedTaskManagerRegistration taskManagerRegistration =
+                new FineGrainedTaskManagerRegistration(
+                        taskExecutorConnection, totalResourceProfile, defaultSlotResourceProfile);
+        taskManagerRegistrations.put(
+                taskExecutorConnection.getInstanceID(), taskManagerRegistration);
+    }
+
+    @Override
+    public void removeTaskManager(InstanceID instanceId) {
+        LOG.debug("Remove task manager {}.", instanceId);
+        final FineGrainedTaskManagerRegistration taskManager =
+                Preconditions.checkNotNull(taskManagerRegistrations.get(instanceId));
+        for (AllocationID allocationId : new HashSet<>(taskManager.getAllocatedSlots().keySet())) {
+            slots.remove(allocationId);
+            taskManager.freeSlot(allocationId);
+        }
+        taskManagerRegistrations.remove(instanceId);
+    }
+
+    @Override
+    public void addPendingTaskManager(PendingTaskManager pendingTaskManager) {
+        LOG.debug("Add pending task manager {}.", pendingTaskManager);
+        Preconditions.checkNotNull(pendingTaskManager);
+        pendingTaskManagers.put(pendingTaskManager.getPendingTaskManagerId(), pendingTaskManager);
+    }
+
+    @Override
+    public void removePendingTaskManager(PendingTaskManagerId pendingTaskManagerId) {
+        Preconditions.checkNotNull(pendingTaskManagerId);
+        LOG.debug("Remove pending task manager {}.", pendingTaskManagerId);
+        pendingTaskManagers.remove(pendingTaskManagerId);
+    }
+
+    // ---------------------------------------------------------------------------------------------
+    // Core state transitions
+    // ---------------------------------------------------------------------------------------------
+
+    @Override
+    public void notifySlotStatus(
+            AllocationID allocationId,
+            JobID jobId,
+            InstanceID instanceId,
+            ResourceProfile resourceProfile,
+            SlotState slotState) {
+        Preconditions.checkNotNull(allocationId);
+        Preconditions.checkNotNull(jobId);
+        Preconditions.checkNotNull(instanceId);
+        Preconditions.checkNotNull(resourceProfile);
+        Preconditions.checkNotNull(slotState);
+        switch (slotState) {
+            case FREE:
+                freeSlot(instanceId, allocationId);
+                break;
+            case ALLOCATED:
+                allocateSlot(allocationId, jobId, instanceId, resourceProfile);
+                break;
+            case PENDING:
+                addPendingSlot(allocationId, jobId, instanceId, resourceProfile);
+                break;
+        }
+    }
+
+    private void freeSlot(InstanceID instanceId, AllocationID allocationId) {
+        Preconditions.checkState(taskManagerRegistrations.containsKey(instanceId));
+        Preconditions.checkState(slots.containsKey(allocationId));

Review comment:
       Duplicate map lookup

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/DefaultSlotStatusSyncer.java
##########
@@ -0,0 +1,258 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.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.FutureUtils;
+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.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.HashSet;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+
+/** Default implementation of {@link SlotStatusSyncer} for fine-grained slot management. */
+public class DefaultSlotStatusSyncer implements SlotStatusSyncer {
+    private static final Logger LOG = LoggerFactory.getLogger(DefaultSlotStatusSyncer.class);
+
+    private final TaskManagerTracker taskManagerTracker;
+    private final ResourceTracker resourceTracker;
+    private final Set<AllocationID> pendingSlotAllocations;
+    /** Timeout for slot requests to the task manager. */
+    private final Time taskManagerRequestTimeout;
+
+    public DefaultSlotStatusSyncer(
+            TaskManagerTracker taskManagerTracker,
+            ResourceTracker resourceTracker,
+            Time taskManagerRequestTimeout) {
+        this.taskManagerTracker = Preconditions.checkNotNull(taskManagerTracker);
+        this.resourceTracker = Preconditions.checkNotNull(resourceTracker);
+        this.taskManagerRequestTimeout = Preconditions.checkNotNull(taskManagerRequestTimeout);
+
+        this.pendingSlotAllocations = new HashSet<>(16);
+    }
+
+    @Override
+    public CompletableFuture<Void> allocateSlot(
+            InstanceID instanceId,
+            JobID jobId,
+            String targetAddress,
+            ResourceProfile resourceProfile,
+            ResourceManagerId resourceManagerId,
+            Executor mainThreadExecutor) {
+        final AllocationID allocationId = new AllocationID();
+        final Optional<TaskManagerInfo> taskManager =
+                taskManagerTracker.getRegisteredTaskManager(instanceId);
+        Preconditions.checkState(
+                taskManager.isPresent(),
+                "Could not find a registered task manager for instance id " + instanceId + '.');
+        final TaskExecutorGateway gateway =
+                taskManager.get().getTaskExecutorConnection().getTaskExecutorGateway();
+
+        taskManagerTracker.notifySlotStatus(
+                allocationId, jobId, instanceId, resourceProfile, SlotState.PENDING);
+        resourceTracker.notifyAcquiredResource(jobId, resourceProfile);
+        pendingSlotAllocations.add(allocationId);
+
+        // RPC call to the task manager
+        CompletableFuture<Acknowledge> requestFuture =
+                gateway.requestSlot(
+                        SlotID.getDynamicSlotID(
+                                taskManager.get().getTaskExecutorConnection().getResourceID()),
+                        jobId,
+                        allocationId,
+                        resourceProfile,
+                        targetAddress,
+                        resourceManagerId,
+                        taskManagerRequestTimeout);
+
+        CompletableFuture<Void> returnedFuture = new CompletableFuture<>();
+
+        FutureUtils.assertNoException(
+                requestFuture.handleAsync(
+                        (Acknowledge acknowledge, Throwable throwable) -> {
+                            if (!pendingSlotAllocations.remove(allocationId)) {
+                                LOG.debug(
+                                        "Ignoring slot allocation update from task manager {} for allocation {} and job {}, because the allocation was already completed or cancelled.",
+                                        instanceId,
+                                        allocationId,
+                                        jobId);
+                                returnedFuture.complete(null);
+                                return null;
+                            }
+                            if (!taskManagerTracker
+                                    .getAllocatedOrPendingSlot(allocationId)
+                                    .isPresent()) {
+                                LOG.debug(
+                                        "The slot {} has been removed before. Ignore the future.",
+                                        allocationId);
+                                requestFuture.complete(null);
+                                return null;
+                            }
+                            if (acknowledge != null) {
+                                LOG.trace(
+                                        "Completed allocation of allocation {} for job {}.",
+                                        allocationId,
+                                        jobId);
+                                taskManagerTracker.notifySlotStatus(
+                                        allocationId,
+                                        jobId,
+                                        instanceId,
+                                        resourceProfile,
+                                        SlotState.ALLOCATED);
+                                returnedFuture.complete(null);
+                            } else {
+                                if (throwable instanceof SlotOccupiedException) {
+                                    LOG.error("Should not get this exception.", throwable);
+                                } else {
+                                    // TODO If the taskManager does not have enough resource, we
+                                    // may endlessly allocate slot on it until the next heartbeat.
+                                    LOG.warn(
+                                            "Slot allocation for allocation {} for job {} failed.",
+                                            allocationId,
+                                            jobId,
+                                            throwable);
+                                    resourceTracker.notifyLostResource(jobId, resourceProfile);
+                                    taskManagerTracker.notifySlotStatus(
+                                            allocationId,
+                                            jobId,
+                                            instanceId,
+                                            resourceProfile,
+                                            SlotState.FREE);
+                                }
+                                returnedFuture.completeExceptionally(throwable);
+                            }
+                            return null;
+                        },
+                        mainThreadExecutor));
+        return returnedFuture;
+    }
+
+    @Override
+    public void freeSlot(AllocationID allocationId) {
+        LOG.debug("Freeing slot {}.", allocationId);
+
+        Optional<TaskManagerSlotInformation> slot =
+                taskManagerTracker.getAllocatedOrPendingSlot(allocationId);
+        if (!slot.isPresent()) {
+            LOG.warn("Try to free unknown slot {}.", allocationId);
+        }
+
+        resourceTracker.notifyLostResource(slot.get().getJobId(), slot.get().getResourceProfile());
+        taskManagerTracker.notifySlotStatus(
+                allocationId,
+                slot.get().getJobId(),
+                slot.get().getInstanceId(),
+                slot.get().getResourceProfile(),
+                SlotState.FREE);
+    }

Review comment:
       And should we also try remove from `pendingSlotAllocations`? 

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/FineGrainedTaskManagerTracker.java
##########
@@ -0,0 +1,237 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.slots.ResourceCounter;
+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.HashSet;
+import java.util.Map;
+import java.util.Optional;
+
+/** Implementation of {@link TaskManagerTracker} supporting fine-grained resource management. */
+public class FineGrainedTaskManagerTracker implements TaskManagerTracker {
+    private static final Logger LOG = LoggerFactory.getLogger(FineGrainedTaskManagerTracker.class);
+
+    /** Map for allocated and pending slots. */
+    private final Map<AllocationID, FineGrainedTaskManagerSlot> slots;
+
+    /** All currently registered task managers. */
+    private final Map<InstanceID, FineGrainedTaskManagerRegistration> taskManagerRegistrations;
+
+    private final Map<PendingTaskManagerId, PendingTaskManager> pendingTaskManagers;
+
+    private final Map<PendingTaskManagerId, Map<JobID, ResourceCounter>>
+            pendingSlotAllocationRecords;
+
+    public FineGrainedTaskManagerTracker() {
+        slots = new HashMap<>(16);
+        taskManagerRegistrations = new HashMap<>(4);
+        pendingTaskManagers = new HashMap<>(16);
+        pendingSlotAllocationRecords = new HashMap<>();
+    }
+
+    @Override
+    public void recordPendingAllocations(
+            Map<PendingTaskManagerId, Map<JobID, ResourceCounter>> pendingSlotAllocations) {
+        LOG.trace("Record the pending allocations {}.", pendingSlotAllocations);
+        Preconditions.checkNotNull(pendingSlotAllocations);
+        pendingSlotAllocationRecords.clear();
+        pendingSlotAllocationRecords.putAll(pendingSlotAllocations);
+    }
+
+    @Override
+    public void addTaskManager(
+            TaskExecutorConnection taskExecutorConnection,
+            ResourceProfile totalResourceProfile,
+            ResourceProfile defaultSlotResourceProfile) {
+        Preconditions.checkNotNull(taskExecutorConnection);
+        Preconditions.checkNotNull(totalResourceProfile);
+        Preconditions.checkNotNull(defaultSlotResourceProfile);
+        LOG.debug(
+                "Add task manager {} with total resource {} and default slot resource {}.",
+                taskExecutorConnection.getInstanceID(),
+                totalResourceProfile,
+                defaultSlotResourceProfile);
+        final FineGrainedTaskManagerRegistration taskManagerRegistration =
+                new FineGrainedTaskManagerRegistration(
+                        taskExecutorConnection, totalResourceProfile, defaultSlotResourceProfile);
+        taskManagerRegistrations.put(
+                taskExecutorConnection.getInstanceID(), taskManagerRegistration);
+    }
+
+    @Override
+    public void removeTaskManager(InstanceID instanceId) {
+        LOG.debug("Remove task manager {}.", instanceId);
+        final FineGrainedTaskManagerRegistration taskManager =
+                Preconditions.checkNotNull(taskManagerRegistrations.get(instanceId));
+        for (AllocationID allocationId : new HashSet<>(taskManager.getAllocatedSlots().keySet())) {
+            slots.remove(allocationId);
+            taskManager.freeSlot(allocationId);
+        }
+        taskManagerRegistrations.remove(instanceId);
+    }
+
+    @Override
+    public void addPendingTaskManager(PendingTaskManager pendingTaskManager) {
+        LOG.debug("Add pending task manager {}.", pendingTaskManager);
+        Preconditions.checkNotNull(pendingTaskManager);
+        pendingTaskManagers.put(pendingTaskManager.getPendingTaskManagerId(), pendingTaskManager);
+    }
+
+    @Override
+    public void removePendingTaskManager(PendingTaskManagerId pendingTaskManagerId) {
+        Preconditions.checkNotNull(pendingTaskManagerId);
+        LOG.debug("Remove pending task manager {}.", pendingTaskManagerId);
+        pendingTaskManagers.remove(pendingTaskManagerId);
+    }
+
+    // ---------------------------------------------------------------------------------------------
+    // Core state transitions
+    // ---------------------------------------------------------------------------------------------
+
+    @Override
+    public void notifySlotStatus(
+            AllocationID allocationId,
+            JobID jobId,
+            InstanceID instanceId,
+            ResourceProfile resourceProfile,
+            SlotState slotState) {
+        Preconditions.checkNotNull(allocationId);
+        Preconditions.checkNotNull(jobId);
+        Preconditions.checkNotNull(instanceId);
+        Preconditions.checkNotNull(resourceProfile);
+        Preconditions.checkNotNull(slotState);
+        switch (slotState) {
+            case FREE:
+                freeSlot(instanceId, allocationId);
+                break;
+            case ALLOCATED:
+                allocateSlot(allocationId, jobId, instanceId, resourceProfile);
+                break;
+            case PENDING:
+                addPendingSlot(allocationId, jobId, instanceId, resourceProfile);
+                break;
+        }
+    }
+
+    private void freeSlot(InstanceID instanceId, AllocationID allocationId) {
+        Preconditions.checkState(taskManagerRegistrations.containsKey(instanceId));
+        Preconditions.checkState(slots.containsKey(allocationId));
+        LOG.debug("Free allocated slot with allocationId {}.", allocationId);
+
+        taskManagerRegistrations.get(instanceId).freeSlot(allocationId);
+        slots.remove(allocationId);
+    }
+
+    private void allocateSlot(
+            AllocationID allocationId,
+            JobID jobId,
+            InstanceID instanceId,
+            ResourceProfile resourceProfile) {
+        Preconditions.checkState(taskManagerRegistrations.containsKey(instanceId));

Review comment:
       Duplicate map lookup

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/FineGrainedSlotManager.java
##########
@@ -0,0 +1,790 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.api.java.tuple.Tuple2;
+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.FutureUtils;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.instance.InstanceID;
+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.util.FlinkException;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+/** Implementation of {@link SlotManager} supporting fine-grained resource management. */
+public class FineGrainedSlotManager implements SlotManager {
+    private static final Logger LOG = LoggerFactory.getLogger(FineGrainedSlotManager.class);
+
+    private final TaskManagerTracker taskManagerTracker;
+    private final ResourceTracker resourceTracker;
+    private final ResourceAllocationStrategy resourceAllocationStrategy;
+
+    private final SlotStatusSyncer slotStatusSyncer;
+
+    /** Scheduled executor for timeouts. */
+    private final ScheduledExecutor scheduledExecutor;
+
+    /** Timeout after which an unused TaskManager is released. */
+    private final Time taskManagerTimeout;
+
+    private final SlotManagerMetricGroup slotManagerMetricGroup;
+
+    private final Map<JobID, String> jobMasterTargetAddresses = new HashMap<>();
+
+    /** Defines the max limitation of the total number of task executors. */
+    private final int maxTaskManagerNum;
+
+    /** Defines the number of redundant task executors. */
+    private final int redundantTaskManagerNum;
+
+    /**
+     * Release task executor only when each produced result partition is either consumed or failed.
+     */
+    private final boolean waitResultConsumedBeforeRelease;
+
+    /** The default resource spec of workers to request. */
+    private final WorkerResourceSpec defaultWorkerResourceSpec;
+
+    /** The resource profile of default slot. */
+    private final ResourceProfile defaultSlotResourceProfile;
+
+    private boolean sendNotEnoughResourceNotifications = true;
+
+    /** ResourceManager's id. */
+    @Nullable private ResourceManagerId resourceManagerId;
+
+    /** Executor for future callbacks which have to be "synchronized". */
+    @Nullable private Executor mainThreadExecutor;
+
+    /** Callbacks for resource (de-)allocations. */
+    @Nullable private ResourceActions resourceActions;
+
+    private ScheduledFuture<?> taskManagerTimeoutsAndRedundancyCheck;

Review comment:
       `@Nullable`




----------------------------------------------------------------
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 #14647: [FLINK-20835] Implement FineGrainedSlotManager

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "1c605546a2ac429e1c475eea6f80a75d0326c7bf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12054",
       "triggerID" : "1c605546a2ac429e1c475eea6f80a75d0326c7bf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "559cd8d09cca8082e9ca4904d11b11443f4c959f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12226",
       "triggerID" : "559cd8d09cca8082e9ca4904d11b11443f4c959f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "05733d3630a9a72d04545e2e8aeefdf76b2e5cf1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12235",
       "triggerID" : "05733d3630a9a72d04545e2e8aeefdf76b2e5cf1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0a88531c02021ab6bc36c8eec483dac0ee91db97",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12540",
       "triggerID" : "0a88531c02021ab6bc36c8eec483dac0ee91db97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "45b2dc853d01aa726bdf054b3c2afa5b95a1d6fd",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12543",
       "triggerID" : "45b2dc853d01aa726bdf054b3c2afa5b95a1d6fd",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 45b2dc853d01aa726bdf054b3c2afa5b95a1d6fd Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12543) 
   
   <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] KarmaGYZ commented on a change in pull request #14647: [FLINK-20835] Implement FineGrainedSlotManager

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/DefaultResourceAllocationStrategy.java
##########
@@ -0,0 +1,233 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.slots.ResourceCounter;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.runtime.resourcemanager.slotmanager.SlotManagerUtils.getEffectiveResourceProfile;
+
+/**
+ * The default implementation of {@link ResourceAllocationStrategy}, which always allocate pending
+ * task managers with the fixed profile.
+ */
+public class DefaultResourceAllocationStrategy implements ResourceAllocationStrategy {
+    private final ResourceProfile defaultSlotResourceProfile;
+    private final ResourceProfile totalResourceProfile;
+
+    public DefaultResourceAllocationStrategy(
+            ResourceProfile defaultSlotResourceProfile, int numSlotsPerWorker) {
+        this.defaultSlotResourceProfile = defaultSlotResourceProfile;
+        this.totalResourceProfile = defaultSlotResourceProfile.multiply(numSlotsPerWorker);
+    }
+
+    /**
+     * Matches resource requirements against available and pending resources. For each job, in a
+     * first round requirements are matched against registered resources. The remaining unfulfilled
+     * requirements are matched against pending resources, allocating more workers if no matching
+     * pending resources could be found. If the requirements for a job could not be fulfilled then
+     * it will be recorded in {@link ResourceAllocationResult#getUnfulfillableJobs()}.
+     *
+     * <p>Performance notes: At it's core this method loops, for each job, over all resources for
+     * each required slot, trying to find a matching registered/pending task manager. One should
+     * generally go in with the assumption that this runs in numberOfJobsRequiringResources *
+     * numberOfRequiredSlots * numberOfFreeOrPendingTaskManagers.
+     *
+     * <p>In the absolute worst case, with J jobs, requiring R slots each with a unique resource
+     * profile such each pair of these profiles is not matching, and T registered/pending task
+     * managers that don't fulfill any requirement, then this method does a total of J*R*T resource
+     * profile comparisons.
+     */

Review comment:
       I open https://issues.apache.org/jira/browse/FLINK-21174 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 #14647: [FLINK-20835] Implement FineGrainedSlotManager

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "1c605546a2ac429e1c475eea6f80a75d0326c7bf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12054",
       "triggerID" : "1c605546a2ac429e1c475eea6f80a75d0326c7bf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "559cd8d09cca8082e9ca4904d11b11443f4c959f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12226",
       "triggerID" : "559cd8d09cca8082e9ca4904d11b11443f4c959f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "05733d3630a9a72d04545e2e8aeefdf76b2e5cf1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12235",
       "triggerID" : "05733d3630a9a72d04545e2e8aeefdf76b2e5cf1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0a88531c02021ab6bc36c8eec483dac0ee91db97",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12540",
       "triggerID" : "0a88531c02021ab6bc36c8eec483dac0ee91db97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "45b2dc853d01aa726bdf054b3c2afa5b95a1d6fd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12543",
       "triggerID" : "45b2dc853d01aa726bdf054b3c2afa5b95a1d6fd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "80e3121418e741d9bebf3e849817a0c4225a86dd",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12587",
       "triggerID" : "80e3121418e741d9bebf3e849817a0c4225a86dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f5f9f44b437922f5e0edd0b1dd3097a61351dee",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f5f9f44b437922f5e0edd0b1dd3097a61351dee",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 80e3121418e741d9bebf3e849817a0c4225a86dd Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12587) 
   * 8f5f9f44b437922f5e0edd0b1dd3097a61351dee 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 #14647: [FLINK-20835] Implement FineGrainedSlotManager

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "1c605546a2ac429e1c475eea6f80a75d0326c7bf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12054",
       "triggerID" : "1c605546a2ac429e1c475eea6f80a75d0326c7bf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "559cd8d09cca8082e9ca4904d11b11443f4c959f",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12226",
       "triggerID" : "559cd8d09cca8082e9ca4904d11b11443f4c959f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "05733d3630a9a72d04545e2e8aeefdf76b2e5cf1",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12235",
       "triggerID" : "05733d3630a9a72d04545e2e8aeefdf76b2e5cf1",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 559cd8d09cca8082e9ca4904d11b11443f4c959f Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12226) 
   * 05733d3630a9a72d04545e2e8aeefdf76b2e5cf1 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12235) 
   
   <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 #14647: [FLINK-20835] Implement FineGrainedSlotManager

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "1c605546a2ac429e1c475eea6f80a75d0326c7bf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12054",
       "triggerID" : "1c605546a2ac429e1c475eea6f80a75d0326c7bf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "559cd8d09cca8082e9ca4904d11b11443f4c959f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12226",
       "triggerID" : "559cd8d09cca8082e9ca4904d11b11443f4c959f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "05733d3630a9a72d04545e2e8aeefdf76b2e5cf1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12235",
       "triggerID" : "05733d3630a9a72d04545e2e8aeefdf76b2e5cf1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0a88531c02021ab6bc36c8eec483dac0ee91db97",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12540",
       "triggerID" : "0a88531c02021ab6bc36c8eec483dac0ee91db97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "45b2dc853d01aa726bdf054b3c2afa5b95a1d6fd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12543",
       "triggerID" : "45b2dc853d01aa726bdf054b3c2afa5b95a1d6fd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "80e3121418e741d9bebf3e849817a0c4225a86dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12587",
       "triggerID" : "80e3121418e741d9bebf3e849817a0c4225a86dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f5f9f44b437922f5e0edd0b1dd3097a61351dee",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12723",
       "triggerID" : "8f5f9f44b437922f5e0edd0b1dd3097a61351dee",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0b72f53fbfe19ef426572f78ac716521bc63ab64",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12769",
       "triggerID" : "0b72f53fbfe19ef426572f78ac716521bc63ab64",
       "triggerType" : "PUSH"
     }, {
       "hash" : "970360401dcd333f1a3b29355392385262953246",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12823",
       "triggerID" : "970360401dcd333f1a3b29355392385262953246",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 0b72f53fbfe19ef426572f78ac716521bc63ab64 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12769) 
   * 970360401dcd333f1a3b29355392385262953246 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12823) 
   
   <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] KarmaGYZ commented on a change in pull request #14647: [FLINK-20835] Implement FineGrainedSlotManager

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/FineGrainedSlotManager.java
##########
@@ -0,0 +1,790 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.api.java.tuple.Tuple2;
+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.FutureUtils;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.instance.InstanceID;
+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.util.FlinkException;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+/** Implementation of {@link SlotManager} supporting fine-grained resource management. */
+public class FineGrainedSlotManager implements SlotManager {
+    private static final Logger LOG = LoggerFactory.getLogger(FineGrainedSlotManager.class);
+
+    private final TaskManagerTracker taskManagerTracker;
+    private final ResourceTracker resourceTracker;
+    private final ResourceAllocationStrategy resourceAllocationStrategy;
+
+    private final SlotStatusSyncer slotStatusSyncer;
+
+    /** Scheduled executor for timeouts. */
+    private final ScheduledExecutor scheduledExecutor;
+
+    /** Timeout after which an unused TaskManager is released. */
+    private final Time taskManagerTimeout;
+
+    private final SlotManagerMetricGroup slotManagerMetricGroup;
+
+    private final Map<JobID, String> jobMasterTargetAddresses = new HashMap<>();
+
+    /** Defines the max limitation of the total number of task executors. */
+    private final int maxTaskManagerNum;
+
+    /** Defines the number of redundant task executors. */
+    private final int redundantTaskManagerNum;
+
+    /**
+     * Release task executor only when each produced result partition is either consumed or failed.
+     */
+    private final boolean waitResultConsumedBeforeRelease;
+
+    /** The default resource spec of workers to request. */
+    private final WorkerResourceSpec defaultWorkerResourceSpec;
+
+    /** The resource profile of default slot. */
+    private final ResourceProfile defaultSlotResourceProfile;
+
+    private boolean sendNotEnoughResourceNotifications = true;
+
+    /** ResourceManager's id. */
+    @Nullable private ResourceManagerId resourceManagerId;
+
+    /** Executor for future callbacks which have to be "synchronized". */
+    @Nullable private Executor mainThreadExecutor;
+
+    /** Callbacks for resource (de-)allocations. */
+    @Nullable private ResourceActions resourceActions;
+
+    private ScheduledFuture<?> taskManagerTimeoutsAndRedundancyCheck;
+
+    /** True iff the component has been started. */
+    private boolean started;
+
+    public FineGrainedSlotManager(
+            ScheduledExecutor scheduledExecutor,
+            SlotManagerConfiguration slotManagerConfiguration,
+            SlotManagerMetricGroup slotManagerMetricGroup,
+            ResourceTracker resourceTracker,
+            TaskManagerTracker taskManagerTracker) {
+
+        this.scheduledExecutor = Preconditions.checkNotNull(scheduledExecutor);
+
+        Preconditions.checkNotNull(slotManagerConfiguration);
+        this.taskManagerTimeout = slotManagerConfiguration.getTaskManagerTimeout();
+        this.redundantTaskManagerNum = slotManagerConfiguration.getRedundantTaskManagerNum();
+        this.waitResultConsumedBeforeRelease =
+                slotManagerConfiguration.isWaitResultConsumedBeforeRelease();
+        this.defaultWorkerResourceSpec = slotManagerConfiguration.getDefaultWorkerResourceSpec();
+        this.resourceAllocationStrategy = slotManagerConfiguration.getResourceAllocationStrategy();
+        int numSlotsPerWorker = slotManagerConfiguration.getNumSlotsPerWorker();
+        this.maxTaskManagerNum = slotManagerConfiguration.getMaxSlotNum() / numSlotsPerWorker;
+
+        this.slotManagerMetricGroup = Preconditions.checkNotNull(slotManagerMetricGroup);
+        this.defaultSlotResourceProfile =
+                SlotManagerUtils.generateDefaultSlotResourceProfile(
+                        defaultWorkerResourceSpec, numSlotsPerWorker);
+
+        this.resourceTracker = Preconditions.checkNotNull(resourceTracker);
+        this.taskManagerTracker = Preconditions.checkNotNull(taskManagerTracker);
+        this.slotStatusSyncer =
+                new DefaultSlotStatusSyncer(
+                        taskManagerTracker,
+                        resourceTracker,
+                        slotManagerConfiguration.getTaskManagerRequestTimeout());
+
+        resourceManagerId = null;
+        resourceActions = null;
+        mainThreadExecutor = null;
+        taskManagerTimeoutsAndRedundancyCheck = null;
+
+        started = false;
+    }
+
+    @Override
+    public void setFailUnfulfillableRequest(boolean failUnfulfillableRequest) {
+        // this sets up a grace period, e.g., when the cluster was started, to give task executors
+        // time to connect
+        sendNotEnoughResourceNotifications = failUnfulfillableRequest;
+
+        if (failUnfulfillableRequest) {
+            checkResourceRequirements();
+        }
+    }
+
+    // ---------------------------------------------------------------------------------------------
+    // 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.");
+
+        resourceManagerId = Preconditions.checkNotNull(newResourceManagerId);
+        mainThreadExecutor = Preconditions.checkNotNull(newMainThreadExecutor);
+        resourceActions = Preconditions.checkNotNull(newResourceActions);
+
+        started = true;
+
+        taskManagerTimeoutsAndRedundancyCheck =
+                scheduledExecutor.scheduleWithFixedDelay(
+                        () ->
+                                mainThreadExecutor.execute(
+                                        this::checkTaskManagerTimeoutsAndRedundancy),
+                        0L,
+                        taskManagerTimeout.toMilliseconds(),
+                        TimeUnit.MILLISECONDS);
+
+        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() {
+        if (!started) {
+            return;
+        }
+
+        LOG.info("Suspending the slot manager.");
+
+        resourceTracker.clear();
+
+        // stop the timeout checks for the TaskManagers
+        if (taskManagerTimeoutsAndRedundancyCheck != null) {
+            taskManagerTimeoutsAndRedundancyCheck.cancel(false);
+            taskManagerTimeoutsAndRedundancyCheck = null;
+        }
+
+        for (TaskManagerInfo registeredTaskManager :
+                new ArrayList<>(taskManagerTracker.getRegisteredTaskManagers())) {
+            unregisterTaskManager(
+                    registeredTaskManager.getInstanceId(),
+                    new SlotManagerException("The slot manager is being suspended."));
+        }
+
+        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
+     * @param totalResourceProfile of the new task manager
+     * @param defaultSlotResourceProfile of 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,
+            ResourceProfile totalResourceProfile,
+            ResourceProfile defaultSlotResourceProfile) {
+        checkInit();
+        LOG.debug(
+                "Registering task executor {} under {} at the slot manager.",
+                taskExecutorConnection.getResourceID(),
+                taskExecutorConnection.getInstanceID());
+
+        // we identify task managers by their instance id
+        if (taskManagerTracker
+                .getRegisteredTaskManager(taskExecutorConnection.getInstanceID())
+                .isPresent()) {
+            LOG.debug(
+                    "Task executor {} was already registered.",
+                    taskExecutorConnection.getResourceID());
+            reportSlotStatus(taskExecutorConnection.getInstanceID(), initialSlotReport);
+            return false;
+        } else {
+            if (isMaxSlotNumExceededAfterRegistration(
+                    initialSlotReport, totalResourceProfile, defaultSlotResourceProfile)) {
+                LOG.info(
+                        "The total number of slots exceeds the max limitation {}, releasing the excess task executor.",
+                        maxTaskManagerNum);
+                resourceActions.releaseResource(
+                        taskExecutorConnection.getInstanceID(),
+                        new FlinkException(
+                                "The total number of slots exceeds the max limitation."));
+                return false;
+            }
+            taskManagerTracker.addTaskManager(
+                    taskExecutorConnection, totalResourceProfile, defaultSlotResourceProfile);
+            final Optional<PendingTaskManagerId> matchedPendingTaskManager =
+                    findMatchingPendingTaskManager(
+                            totalResourceProfile, defaultSlotResourceProfile);
+            if (!initialSlotReport.withAllocatedSlot() && matchedPendingTaskManager.isPresent()) {
+                taskManagerTracker.removePendingTaskManager(matchedPendingTaskManager.get());
+                allocateSlotsForRegisteredPendingTaskManager(
+                        matchedPendingTaskManager.get(), taskExecutorConnection.getInstanceID());
+            } else {
+                slotStatusSyncer.reportSlotStatus(
+                        taskExecutorConnection.getInstanceID(), initialSlotReport);
+                checkResourceRequirements();
+            }
+            return true;
+        }
+    }
+
+    private void allocateSlotsForRegisteredPendingTaskManager(
+            PendingTaskManagerId pendingTaskManagerId, InstanceID instanceId) {
+        Map<JobID, Map<InstanceID, ResourceCounter>> allocations = new HashMap<>();
+        taskManagerTracker
+                .getPendingAllocationsOfPendingTaskManager(pendingTaskManagerId)
+                .forEach(
+                        ((jobId, resourceCounter) ->
+                                allocations.put(
+                                        jobId,
+                                        Collections.singletonMap(instanceId, resourceCounter))));
+        allocateSlotsAccordingTo(allocations);
+    }
+
+    private boolean isMaxSlotNumExceededAfterRegistration(
+            SlotReport initialSlotReport,
+            ResourceProfile totalResourceProfile,
+            ResourceProfile defaultSlotResourceProfile) {
+        if (!isMaxSlotNumExceededAfterAdding(1)) {
+            return false;
+        }
+
+        if (initialSlotReport.withAllocatedSlot()) {
+            return isMaxSlotNumExceededAfterAdding(1);
+        }
+
+        return isMaxSlotNumExceededAfterAdding(
+                findMatchingPendingTaskManager(totalResourceProfile, defaultSlotResourceProfile)
+                                .isPresent()
+                        ? 0
+                        : 1);
+    }
+
+    private boolean isMaxSlotNumExceededAfterAdding(int numNewTaskExecutor) {
+        return taskManagerTracker.getPendingTaskManagers().size()
+                        + taskManagerTracker.getRegisteredTaskManagers().size()
+                        + numNewTaskExecutor
+                > maxTaskManagerNum;
+    }
+
+    private Optional<PendingTaskManagerId> findMatchingPendingTaskManager(
+            ResourceProfile totalResourceProfile, ResourceProfile defaultSlotResourceProfile) {
+        return taskManagerTracker.getPendingTaskManagers().stream()
+                .filter(
+                        pendingTaskManager ->
+                                pendingTaskManager
+                                                .getTotalResourceProfile()
+                                                .equals(totalResourceProfile)
+                                        && pendingTaskManager
+                                                .getDefaultSlotResourceProfile()
+                                                .equals(defaultSlotResourceProfile))
+                .max(
+                        Comparator.comparingInt(
+                                pendingTaskManager ->
+                                        taskManagerTracker
+                                                .getPendingAllocationsOfPendingTaskManager(
+                                                        pendingTaskManager
+                                                                .getPendingTaskManagerId())
+                                                .size()))
+                .map(PendingTaskManager::getPendingTaskManagerId);
+    }
+
+    @Override
+    public boolean unregisterTaskManager(InstanceID instanceId, Exception cause) {
+        checkInit();
+
+        LOG.debug("Unregistering task executor {} from the slot manager.", instanceId);
+
+        if (taskManagerTracker.getRegisteredTaskManager(instanceId).isPresent()) {
+            for (AllocationID allocationId :
+                    new HashSet<>(
+                            taskManagerTracker
+                                    .getRegisteredTaskManager(instanceId)
+                                    .get()
+                                    .getAllocatedSlots()
+                                    .keySet())) {
+                slotStatusSyncer.freeSlot(allocationId);
+            }
+            taskManagerTracker.removeTaskManager(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 (taskManagerTracker.getRegisteredTaskManager(instanceId).isPresent()) {
+            slotStatusSyncer.reportSlotStatus(instanceId, 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 freed.
+     *
+     * @param slotId identifying the slot to free, will be ignored
+     * @param allocationId with which the slot is presumably allocated
+     */
+    @Override
+    public void freeSlot(SlotID slotId, AllocationID allocationId) {
+        checkInit();
+        LOG.debug("Freeing slot {}.", allocationId);
+
+        if (taskManagerTracker.getAllocatedOrPendingSlot(allocationId).isPresent()) {
+            slotStatusSyncer.freeSlot(allocationId);
+            checkResourceRequirements();
+        } else {
+            LOG.debug(
+                    "Trying to free a slot {} which has not been allocated. Ignoring this message.",
+                    allocationId);
+        }
+    }
+
+    // ---------------------------------------------------------------------------------------------
+    // Requirement matching
+    // ---------------------------------------------------------------------------------------------
+
+    private void checkResourceRequirements() {
+        final Map<JobID, Collection<ResourceRequirement>> missingResources =
+                resourceTracker.getMissingResources();
+        if (missingResources.isEmpty()) {
+            return;
+        }
+
+        final Map<InstanceID, Tuple2<ResourceProfile, ResourceProfile>> availableResources =
+                taskManagerTracker.getRegisteredTaskManagers().stream()
+                        .collect(
+                                Collectors.toMap(
+                                        TaskManagerInfo::getInstanceId,
+                                        taskManager ->
+                                                Tuple2.of(
+                                                        taskManager.getAvailableResource(),
+                                                        taskManager
+                                                                .getDefaultSlotResourceProfile())));
+        final ResourceAllocationResult result =
+                resourceAllocationStrategy.tryFulfillRequirements(
+                        missingResources,
+                        availableResources,
+                        new ArrayList<>(taskManagerTracker.getPendingTaskManagers()));
+
+        // Allocate slots according to the result
+        allocateSlotsAccordingTo(result.getRegisteredResourceAllocationResult());
+
+        // Allocate task managers according to the result
+        final Set<PendingTaskManagerId> failAllocations =
+                allocateTaskManagersAccordingTo(result.getPendingTaskManagersToBeAllocated());
+
+        // Record slot allocation of pending task managers
+        final Map<PendingTaskManagerId, Map<JobID, ResourceCounter>>
+                pendingResourceAllocationResult =
+                        new HashMap<>(result.getPendingResourceAllocationResult());
+        pendingResourceAllocationResult.keySet().retainAll(failAllocations);
+        taskManagerTracker.recordPendingAllocations(pendingResourceAllocationResult);
+
+        // Notify jobs that can not be fulfilled
+        final Set<JobID> unfulfillableJobs = new HashSet<>(result.getUnfulfillableJobs());
+        for (PendingTaskManagerId pendingTaskManagerId : failAllocations) {
+            unfulfillableJobs.addAll(
+                    result.getPendingResourceAllocationResult().get(pendingTaskManagerId).keySet());
+        }
+        for (JobID jobId : unfulfillableJobs) {
+            if (sendNotEnoughResourceNotifications) {
+                LOG.warn("Could not fulfill resource requirements of job {}.", jobId);
+                resourceActions.notifyNotEnoughResourcesAvailable(
+                        jobId, resourceTracker.getAcquiredResources(jobId));
+            }
+        }
+    }
+
+    private void allocateSlotsAccordingTo(Map<JobID, Map<InstanceID, ResourceCounter>> result) {
+        final List<CompletableFuture<Void>> allocationFutures = new ArrayList<>();
+        for (JobID jobID : result.keySet()) {
+            for (InstanceID instanceID : result.get(jobID).keySet()) {
+                for (Map.Entry<ResourceProfile, Integer> resourceToBeAllocated :
+                        result.get(jobID)
+                                .get(instanceID)
+                                .getResourceProfilesWithCount()
+                                .entrySet()) {
+                    for (int i = 0; i < resourceToBeAllocated.getValue(); ++i) {
+                        allocationFutures.add(
+                                slotStatusSyncer.allocateSlot(
+                                        instanceID,
+                                        jobID,
+                                        jobMasterTargetAddresses.get(jobID),
+                                        resourceToBeAllocated.getKey(),
+                                        resourceManagerId,
+                                        mainThreadExecutor));
+                    }
+                }
+            }
+        }
+        FutureUtils.combineAll(allocationFutures)
+                .whenCompleteAsync(
+                        (s, t) -> {
+                            if (t != null) {
+                                // If there is allocation failure, we need to trigger it again.
+                                checkResourceRequirements();
+                            }
+                        },
+                        mainThreadExecutor);
+    }
+
+    /**
+     * Allocate pending task managers, returns the ids of pending task managers that can not be
+     * allocated.
+     */
+    private Set<PendingTaskManagerId> allocateTaskManagersAccordingTo(
+            List<PendingTaskManager> pendingTaskManagers) {
+        final Set<PendingTaskManagerId> failedAllocations = new HashSet<>();
+        for (PendingTaskManager pendingTaskManager : pendingTaskManagers) {
+            if (!allocateResource(pendingTaskManager)) {
+                failedAllocations.add(pendingTaskManager.getPendingTaskManagerId());
+            }
+        }
+        return failedAllocations;
+    }
+
+    // ---------------------------------------------------------------------------------------------
+    // Legacy APIs
+    // ---------------------------------------------------------------------------------------------
+
+    @Override
+    public int getNumberRegisteredSlots() {
+        return taskManagerTracker.getStatusOverview().getNumberRegisteredSlots();
+    }
+
+    @Override
+    public int getNumberRegisteredSlotsOf(InstanceID instanceId) {
+        return taskManagerTracker.getStatusOverview().getNumberRegisteredSlotsOf(instanceId);
+    }
+
+    @Override
+    public int getNumberFreeSlots() {
+        return taskManagerTracker.getStatusOverview().getNumberFreeSlots();
+    }
+
+    @Override
+    public int getNumberFreeSlotsOf(InstanceID instanceId) {
+        return taskManagerTracker.getStatusOverview().getNumberFreeSlotsOf(instanceId);
+    }
+
+    @Override
+    public Map<WorkerResourceSpec, Integer> getRequiredResources() {
+        return taskManagerTracker.getPendingTaskManagers().stream()
+                .map(PendingTaskManager::getTotalResourceProfile)
+                .map(WorkerResourceSpec::fromTotalResourceProfile)
+                .collect(Collectors.groupingBy(Function.identity(), Collectors.summingInt(e -> 1)));
+    }
+
+    @Override
+    public ResourceProfile getRegisteredResource() {
+        return taskManagerTracker.getStatusOverview().getRegisteredResource();
+    }
+
+    @Override
+    public ResourceProfile getRegisteredResourceOf(InstanceID instanceID) {
+        return taskManagerTracker.getStatusOverview().getRegisteredResourceOf(instanceID);
+    }
+
+    @Override
+    public ResourceProfile getFreeResource() {
+        return taskManagerTracker.getStatusOverview().getFreeResource();
+    }
+
+    @Override
+    public ResourceProfile getFreeResourceOf(InstanceID instanceID) {
+        return taskManagerTracker.getStatusOverview().getRegisteredResourceOf(instanceID);
+    }
+
+    @Override
+    public int getNumberPendingSlotRequests() {
+        // only exists for testing purposes
+        throw new UnsupportedOperationException();
+    }
+
+    // ---------------------------------------------------------------------------------------------
+    // Internal periodic check methods
+    // ---------------------------------------------------------------------------------------------
+
+    private void checkTaskManagerTimeoutsAndRedundancy() {
+        Map<InstanceID, TaskExecutorConnection> timeoutTaskManagers = getTimeOutTaskManagers();
+        if (!timeoutTaskManagers.isEmpty()) {
+            int freeTaskManagersNum =
+                    (int)
+                            taskManagerTracker.getRegisteredTaskManagers().stream()
+                                    .filter(TaskManagerInfo::isIdle)
+                                    .count();
+            int taskManagersDiff = redundantTaskManagerNum - freeTaskManagersNum;

Review comment:
       `taskManagerTimeoutsAndRedundancyCheck` -> `taskManagerTimeoutsCheck`




----------------------------------------------------------------
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 #14647: [FLINK-20835] Implement FineGrainedSlotManager

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "1c605546a2ac429e1c475eea6f80a75d0326c7bf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12054",
       "triggerID" : "1c605546a2ac429e1c475eea6f80a75d0326c7bf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "559cd8d09cca8082e9ca4904d11b11443f4c959f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12226",
       "triggerID" : "559cd8d09cca8082e9ca4904d11b11443f4c959f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "05733d3630a9a72d04545e2e8aeefdf76b2e5cf1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12235",
       "triggerID" : "05733d3630a9a72d04545e2e8aeefdf76b2e5cf1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0a88531c02021ab6bc36c8eec483dac0ee91db97",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12540",
       "triggerID" : "0a88531c02021ab6bc36c8eec483dac0ee91db97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "45b2dc853d01aa726bdf054b3c2afa5b95a1d6fd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12543",
       "triggerID" : "45b2dc853d01aa726bdf054b3c2afa5b95a1d6fd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "80e3121418e741d9bebf3e849817a0c4225a86dd",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12587",
       "triggerID" : "80e3121418e741d9bebf3e849817a0c4225a86dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f5f9f44b437922f5e0edd0b1dd3097a61351dee",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12723",
       "triggerID" : "8f5f9f44b437922f5e0edd0b1dd3097a61351dee",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 80e3121418e741d9bebf3e849817a0c4225a86dd Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12587) 
   * 8f5f9f44b437922f5e0edd0b1dd3097a61351dee Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12723) 
   
   <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 #14647: [FLINK-20835] Implement FineGrainedSlotManager

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "1c605546a2ac429e1c475eea6f80a75d0326c7bf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12054",
       "triggerID" : "1c605546a2ac429e1c475eea6f80a75d0326c7bf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "559cd8d09cca8082e9ca4904d11b11443f4c959f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12226",
       "triggerID" : "559cd8d09cca8082e9ca4904d11b11443f4c959f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "05733d3630a9a72d04545e2e8aeefdf76b2e5cf1",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12235",
       "triggerID" : "05733d3630a9a72d04545e2e8aeefdf76b2e5cf1",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 05733d3630a9a72d04545e2e8aeefdf76b2e5cf1 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12235) 
   
   <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] xintongsong commented on a change in pull request #14647: [FLINK-20835] Implement FineGrainedSlotManager

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/FineGrainedTaskManagerRegistration.java
##########
@@ -159,24 +152,23 @@ public void notifyAllocationComplete(AllocationID allocationId) {
         unusedResource = unusedResource.subtract(slot.getResourceProfile());
     }
 
-    public void notifyAllocationStart(
+    public void notifyAllocation(
             AllocationID allocationId, FineGrainedTaskManagerSlot taskManagerSlot) {
         Preconditions.checkNotNull(allocationId);
         Preconditions.checkNotNull(taskManagerSlot);
-        Preconditions.checkState(taskManagerSlot.getState() == SlotState.PENDING);
+        switch (taskManagerSlot.getState()) {
+            case PENDING:
+                pendingResource = pendingResource.merge(taskManagerSlot.getResourceProfile());
+                Preconditions.checkState(totalResource.allFieldsNoLessThan(pendingResource));

Review comment:
       Changes happen before checking. If the check fails and the exception is caught and handled outside this class, there will be non-recoverable inconsistent states.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/WorkerResourceSpec.java
##########
@@ -70,6 +71,17 @@ public static WorkerResourceSpec fromTaskExecutorProcessSpec(
                 taskExecutorProcessSpec.getManagedMemorySize());
     }
 
+    public static WorkerResourceSpec fromTotalResourceProfile(
+            final ResourceProfile resourceProfile) {
+        Preconditions.checkNotNull(resourceProfile);
+        return new WorkerResourceSpec(
+                (CPUResource) resourceProfile.getCpuCores(),

Review comment:
       It's weird that `ResourceProfile#getCpuCores` returns a `Resource` rather than a `CPUResource` and we have to do the casting here.  Not introduced by this PR though, but we may want to look into this later.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/ResourceAllocationStrategy.java
##########
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+
+/** Strategy how to allocate slots and task managers to fulfill the unfulfilled requirements. */
+public interface ResourceAllocationStrategy {
+    ResourceAllocationStrategy NO_OP_STRATEGY =
+            (missingResources, registeredResources, pendingTaskManagers) ->
+                    ResourceAllocationResult.builder().build();

Review comment:
       No longer need this if we do not introduce a configuration for the strategy.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/FineGrainedSlotManager.java
##########
@@ -0,0 +1,790 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.api.java.tuple.Tuple2;
+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.FutureUtils;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.instance.InstanceID;
+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.util.FlinkException;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+/** Implementation of {@link SlotManager} supporting fine-grained resource management. */
+public class FineGrainedSlotManager implements SlotManager {
+    private static final Logger LOG = LoggerFactory.getLogger(FineGrainedSlotManager.class);
+
+    private final TaskManagerTracker taskManagerTracker;
+    private final ResourceTracker resourceTracker;
+    private final ResourceAllocationStrategy resourceAllocationStrategy;
+
+    private final SlotStatusSyncer slotStatusSyncer;
+
+    /** Scheduled executor for timeouts. */
+    private final ScheduledExecutor scheduledExecutor;
+
+    /** Timeout after which an unused TaskManager is released. */
+    private final Time taskManagerTimeout;
+
+    private final SlotManagerMetricGroup slotManagerMetricGroup;
+
+    private final Map<JobID, String> jobMasterTargetAddresses = new HashMap<>();
+
+    /** Defines the max limitation of the total number of task executors. */
+    private final int maxTaskManagerNum;
+
+    /** Defines the number of redundant task executors. */
+    private final int redundantTaskManagerNum;
+
+    /**
+     * Release task executor only when each produced result partition is either consumed or failed.
+     */
+    private final boolean waitResultConsumedBeforeRelease;
+
+    /** The default resource spec of workers to request. */
+    private final WorkerResourceSpec defaultWorkerResourceSpec;
+
+    /** The resource profile of default slot. */
+    private final ResourceProfile defaultSlotResourceProfile;
+
+    private boolean sendNotEnoughResourceNotifications = true;
+
+    /** ResourceManager's id. */
+    @Nullable private ResourceManagerId resourceManagerId;
+
+    /** Executor for future callbacks which have to be "synchronized". */
+    @Nullable private Executor mainThreadExecutor;
+
+    /** Callbacks for resource (de-)allocations. */
+    @Nullable private ResourceActions resourceActions;
+
+    private ScheduledFuture<?> taskManagerTimeoutsAndRedundancyCheck;
+
+    /** True iff the component has been started. */
+    private boolean started;
+
+    public FineGrainedSlotManager(
+            ScheduledExecutor scheduledExecutor,
+            SlotManagerConfiguration slotManagerConfiguration,
+            SlotManagerMetricGroup slotManagerMetricGroup,
+            ResourceTracker resourceTracker,
+            TaskManagerTracker taskManagerTracker) {
+
+        this.scheduledExecutor = Preconditions.checkNotNull(scheduledExecutor);
+
+        Preconditions.checkNotNull(slotManagerConfiguration);
+        this.taskManagerTimeout = slotManagerConfiguration.getTaskManagerTimeout();
+        this.redundantTaskManagerNum = slotManagerConfiguration.getRedundantTaskManagerNum();
+        this.waitResultConsumedBeforeRelease =
+                slotManagerConfiguration.isWaitResultConsumedBeforeRelease();
+        this.defaultWorkerResourceSpec = slotManagerConfiguration.getDefaultWorkerResourceSpec();
+        this.resourceAllocationStrategy = slotManagerConfiguration.getResourceAllocationStrategy();
+        int numSlotsPerWorker = slotManagerConfiguration.getNumSlotsPerWorker();
+        this.maxTaskManagerNum = slotManagerConfiguration.getMaxSlotNum() / numSlotsPerWorker;
+
+        this.slotManagerMetricGroup = Preconditions.checkNotNull(slotManagerMetricGroup);
+        this.defaultSlotResourceProfile =
+                SlotManagerUtils.generateDefaultSlotResourceProfile(
+                        defaultWorkerResourceSpec, numSlotsPerWorker);
+
+        this.resourceTracker = Preconditions.checkNotNull(resourceTracker);
+        this.taskManagerTracker = Preconditions.checkNotNull(taskManagerTracker);
+        this.slotStatusSyncer =
+                new DefaultSlotStatusSyncer(
+                        taskManagerTracker,
+                        resourceTracker,
+                        slotManagerConfiguration.getTaskManagerRequestTimeout());
+
+        resourceManagerId = null;
+        resourceActions = null;
+        mainThreadExecutor = null;
+        taskManagerTimeoutsAndRedundancyCheck = null;
+
+        started = false;
+    }
+
+    @Override
+    public void setFailUnfulfillableRequest(boolean failUnfulfillableRequest) {
+        // this sets up a grace period, e.g., when the cluster was started, to give task executors
+        // time to connect
+        sendNotEnoughResourceNotifications = failUnfulfillableRequest;
+
+        if (failUnfulfillableRequest) {
+            checkResourceRequirements();
+        }
+    }
+
+    // ---------------------------------------------------------------------------------------------
+    // 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.");
+
+        resourceManagerId = Preconditions.checkNotNull(newResourceManagerId);
+        mainThreadExecutor = Preconditions.checkNotNull(newMainThreadExecutor);
+        resourceActions = Preconditions.checkNotNull(newResourceActions);
+
+        started = true;
+
+        taskManagerTimeoutsAndRedundancyCheck =
+                scheduledExecutor.scheduleWithFixedDelay(
+                        () ->
+                                mainThreadExecutor.execute(
+                                        this::checkTaskManagerTimeoutsAndRedundancy),
+                        0L,
+                        taskManagerTimeout.toMilliseconds(),
+                        TimeUnit.MILLISECONDS);
+
+        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() {
+        if (!started) {
+            return;
+        }
+
+        LOG.info("Suspending the slot manager.");
+
+        resourceTracker.clear();
+
+        // stop the timeout checks for the TaskManagers
+        if (taskManagerTimeoutsAndRedundancyCheck != null) {
+            taskManagerTimeoutsAndRedundancyCheck.cancel(false);
+            taskManagerTimeoutsAndRedundancyCheck = null;
+        }
+
+        for (TaskManagerInfo registeredTaskManager :
+                new ArrayList<>(taskManagerTracker.getRegisteredTaskManagers())) {
+            unregisterTaskManager(
+                    registeredTaskManager.getInstanceId(),
+                    new SlotManagerException("The slot manager is being suspended."));
+        }
+
+        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
+     * @param totalResourceProfile of the new task manager
+     * @param defaultSlotResourceProfile of 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,
+            ResourceProfile totalResourceProfile,
+            ResourceProfile defaultSlotResourceProfile) {
+        checkInit();
+        LOG.debug(
+                "Registering task executor {} under {} at the slot manager.",
+                taskExecutorConnection.getResourceID(),
+                taskExecutorConnection.getInstanceID());
+
+        // we identify task managers by their instance id
+        if (taskManagerTracker
+                .getRegisteredTaskManager(taskExecutorConnection.getInstanceID())
+                .isPresent()) {
+            LOG.debug(
+                    "Task executor {} was already registered.",
+                    taskExecutorConnection.getResourceID());
+            reportSlotStatus(taskExecutorConnection.getInstanceID(), initialSlotReport);
+            return false;
+        } else {
+            if (isMaxSlotNumExceededAfterRegistration(
+                    initialSlotReport, totalResourceProfile, defaultSlotResourceProfile)) {
+                LOG.info(
+                        "The total number of slots exceeds the max limitation {}, releasing the excess task executor.",
+                        maxTaskManagerNum);
+                resourceActions.releaseResource(
+                        taskExecutorConnection.getInstanceID(),
+                        new FlinkException(
+                                "The total number of slots exceeds the max limitation."));
+                return false;
+            }
+            taskManagerTracker.addTaskManager(
+                    taskExecutorConnection, totalResourceProfile, defaultSlotResourceProfile);
+            final Optional<PendingTaskManagerId> matchedPendingTaskManager =
+                    findMatchingPendingTaskManager(
+                            totalResourceProfile, defaultSlotResourceProfile);
+            if (!initialSlotReport.withAllocatedSlot() && matchedPendingTaskManager.isPresent()) {
+                taskManagerTracker.removePendingTaskManager(matchedPendingTaskManager.get());
+                allocateSlotsForRegisteredPendingTaskManager(
+                        matchedPendingTaskManager.get(), taskExecutorConnection.getInstanceID());

Review comment:
       Seems if `initialSlotReport.withAllocatedSlot()` is `true`, we won't need to call `findMatchingPendingTaskManager`.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/FineGrainedSlotManager.java
##########
@@ -0,0 +1,790 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.api.java.tuple.Tuple2;
+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.FutureUtils;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.instance.InstanceID;
+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.util.FlinkException;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+/** Implementation of {@link SlotManager} supporting fine-grained resource management. */
+public class FineGrainedSlotManager implements SlotManager {
+    private static final Logger LOG = LoggerFactory.getLogger(FineGrainedSlotManager.class);
+
+    private final TaskManagerTracker taskManagerTracker;
+    private final ResourceTracker resourceTracker;
+    private final ResourceAllocationStrategy resourceAllocationStrategy;
+
+    private final SlotStatusSyncer slotStatusSyncer;
+
+    /** Scheduled executor for timeouts. */
+    private final ScheduledExecutor scheduledExecutor;
+
+    /** Timeout after which an unused TaskManager is released. */
+    private final Time taskManagerTimeout;
+
+    private final SlotManagerMetricGroup slotManagerMetricGroup;
+
+    private final Map<JobID, String> jobMasterTargetAddresses = new HashMap<>();
+
+    /** Defines the max limitation of the total number of task executors. */
+    private final int maxTaskManagerNum;
+
+    /** Defines the number of redundant task executors. */
+    private final int redundantTaskManagerNum;
+
+    /**
+     * Release task executor only when each produced result partition is either consumed or failed.
+     */
+    private final boolean waitResultConsumedBeforeRelease;
+
+    /** The default resource spec of workers to request. */
+    private final WorkerResourceSpec defaultWorkerResourceSpec;
+
+    /** The resource profile of default slot. */
+    private final ResourceProfile defaultSlotResourceProfile;
+
+    private boolean sendNotEnoughResourceNotifications = true;
+
+    /** ResourceManager's id. */
+    @Nullable private ResourceManagerId resourceManagerId;
+
+    /** Executor for future callbacks which have to be "synchronized". */
+    @Nullable private Executor mainThreadExecutor;
+
+    /** Callbacks for resource (de-)allocations. */
+    @Nullable private ResourceActions resourceActions;
+
+    private ScheduledFuture<?> taskManagerTimeoutsAndRedundancyCheck;
+
+    /** True iff the component has been started. */
+    private boolean started;
+
+    public FineGrainedSlotManager(
+            ScheduledExecutor scheduledExecutor,
+            SlotManagerConfiguration slotManagerConfiguration,
+            SlotManagerMetricGroup slotManagerMetricGroup,
+            ResourceTracker resourceTracker,
+            TaskManagerTracker taskManagerTracker) {
+
+        this.scheduledExecutor = Preconditions.checkNotNull(scheduledExecutor);
+
+        Preconditions.checkNotNull(slotManagerConfiguration);
+        this.taskManagerTimeout = slotManagerConfiguration.getTaskManagerTimeout();
+        this.redundantTaskManagerNum = slotManagerConfiguration.getRedundantTaskManagerNum();
+        this.waitResultConsumedBeforeRelease =
+                slotManagerConfiguration.isWaitResultConsumedBeforeRelease();
+        this.defaultWorkerResourceSpec = slotManagerConfiguration.getDefaultWorkerResourceSpec();
+        this.resourceAllocationStrategy = slotManagerConfiguration.getResourceAllocationStrategy();
+        int numSlotsPerWorker = slotManagerConfiguration.getNumSlotsPerWorker();
+        this.maxTaskManagerNum = slotManagerConfiguration.getMaxSlotNum() / numSlotsPerWorker;
+
+        this.slotManagerMetricGroup = Preconditions.checkNotNull(slotManagerMetricGroup);
+        this.defaultSlotResourceProfile =
+                SlotManagerUtils.generateDefaultSlotResourceProfile(
+                        defaultWorkerResourceSpec, numSlotsPerWorker);
+
+        this.resourceTracker = Preconditions.checkNotNull(resourceTracker);
+        this.taskManagerTracker = Preconditions.checkNotNull(taskManagerTracker);
+        this.slotStatusSyncer =
+                new DefaultSlotStatusSyncer(
+                        taskManagerTracker,
+                        resourceTracker,
+                        slotManagerConfiguration.getTaskManagerRequestTimeout());
+
+        resourceManagerId = null;
+        resourceActions = null;
+        mainThreadExecutor = null;
+        taskManagerTimeoutsAndRedundancyCheck = null;
+
+        started = false;
+    }
+
+    @Override
+    public void setFailUnfulfillableRequest(boolean failUnfulfillableRequest) {
+        // this sets up a grace period, e.g., when the cluster was started, to give task executors
+        // time to connect
+        sendNotEnoughResourceNotifications = failUnfulfillableRequest;
+
+        if (failUnfulfillableRequest) {
+            checkResourceRequirements();
+        }
+    }
+
+    // ---------------------------------------------------------------------------------------------
+    // 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.");
+
+        resourceManagerId = Preconditions.checkNotNull(newResourceManagerId);
+        mainThreadExecutor = Preconditions.checkNotNull(newMainThreadExecutor);
+        resourceActions = Preconditions.checkNotNull(newResourceActions);
+
+        started = true;
+
+        taskManagerTimeoutsAndRedundancyCheck =
+                scheduledExecutor.scheduleWithFixedDelay(
+                        () ->
+                                mainThreadExecutor.execute(
+                                        this::checkTaskManagerTimeoutsAndRedundancy),
+                        0L,
+                        taskManagerTimeout.toMilliseconds(),
+                        TimeUnit.MILLISECONDS);
+
+        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() {
+        if (!started) {
+            return;
+        }
+
+        LOG.info("Suspending the slot manager.");
+
+        resourceTracker.clear();
+
+        // stop the timeout checks for the TaskManagers
+        if (taskManagerTimeoutsAndRedundancyCheck != null) {
+            taskManagerTimeoutsAndRedundancyCheck.cancel(false);
+            taskManagerTimeoutsAndRedundancyCheck = null;
+        }
+
+        for (TaskManagerInfo registeredTaskManager :
+                new ArrayList<>(taskManagerTracker.getRegisteredTaskManagers())) {
+            unregisterTaskManager(
+                    registeredTaskManager.getInstanceId(),
+                    new SlotManagerException("The slot manager is being suspended."));
+        }
+
+        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
+     * @param totalResourceProfile of the new task manager
+     * @param defaultSlotResourceProfile of 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,
+            ResourceProfile totalResourceProfile,
+            ResourceProfile defaultSlotResourceProfile) {
+        checkInit();
+        LOG.debug(
+                "Registering task executor {} under {} at the slot manager.",
+                taskExecutorConnection.getResourceID(),
+                taskExecutorConnection.getInstanceID());
+
+        // we identify task managers by their instance id
+        if (taskManagerTracker
+                .getRegisteredTaskManager(taskExecutorConnection.getInstanceID())
+                .isPresent()) {
+            LOG.debug(
+                    "Task executor {} was already registered.",
+                    taskExecutorConnection.getResourceID());
+            reportSlotStatus(taskExecutorConnection.getInstanceID(), initialSlotReport);
+            return false;
+        } else {
+            if (isMaxSlotNumExceededAfterRegistration(
+                    initialSlotReport, totalResourceProfile, defaultSlotResourceProfile)) {
+                LOG.info(
+                        "The total number of slots exceeds the max limitation {}, releasing the excess task executor.",
+                        maxTaskManagerNum);
+                resourceActions.releaseResource(
+                        taskExecutorConnection.getInstanceID(),
+                        new FlinkException(
+                                "The total number of slots exceeds the max limitation."));
+                return false;
+            }
+            taskManagerTracker.addTaskManager(
+                    taskExecutorConnection, totalResourceProfile, defaultSlotResourceProfile);
+            final Optional<PendingTaskManagerId> matchedPendingTaskManager =
+                    findMatchingPendingTaskManager(
+                            totalResourceProfile, defaultSlotResourceProfile);
+            if (!initialSlotReport.withAllocatedSlot() && matchedPendingTaskManager.isPresent()) {
+                taskManagerTracker.removePendingTaskManager(matchedPendingTaskManager.get());
+                allocateSlotsForRegisteredPendingTaskManager(
+                        matchedPendingTaskManager.get(), taskExecutorConnection.getInstanceID());
+            } else {
+                slotStatusSyncer.reportSlotStatus(
+                        taskExecutorConnection.getInstanceID(), initialSlotReport);
+                checkResourceRequirements();
+            }
+            return true;
+        }
+    }
+
+    private void allocateSlotsForRegisteredPendingTaskManager(
+            PendingTaskManagerId pendingTaskManagerId, InstanceID instanceId) {
+        Map<JobID, Map<InstanceID, ResourceCounter>> allocations = new HashMap<>();
+        taskManagerTracker
+                .getPendingAllocationsOfPendingTaskManager(pendingTaskManagerId)
+                .forEach(
+                        ((jobId, resourceCounter) ->
+                                allocations.put(
+                                        jobId,
+                                        Collections.singletonMap(instanceId, resourceCounter))));
+        allocateSlotsAccordingTo(allocations);
+    }
+
+    private boolean isMaxSlotNumExceededAfterRegistration(
+            SlotReport initialSlotReport,
+            ResourceProfile totalResourceProfile,
+            ResourceProfile defaultSlotResourceProfile) {
+        if (!isMaxSlotNumExceededAfterAdding(1)) {
+            return false;
+        }
+
+        if (initialSlotReport.withAllocatedSlot()) {
+            return isMaxSlotNumExceededAfterAdding(1);
+        }
+
+        return isMaxSlotNumExceededAfterAdding(
+                findMatchingPendingTaskManager(totalResourceProfile, defaultSlotResourceProfile)
+                                .isPresent()
+                        ? 0
+                        : 1);
+    }
+
+    private boolean isMaxSlotNumExceededAfterAdding(int numNewTaskExecutor) {
+        return taskManagerTracker.getPendingTaskManagers().size()
+                        + taskManagerTracker.getRegisteredTaskManagers().size()
+                        + numNewTaskExecutor
+                > maxTaskManagerNum;
+    }
+
+    private Optional<PendingTaskManagerId> findMatchingPendingTaskManager(
+            ResourceProfile totalResourceProfile, ResourceProfile defaultSlotResourceProfile) {
+        return taskManagerTracker.getPendingTaskManagers().stream()
+                .filter(
+                        pendingTaskManager ->
+                                pendingTaskManager
+                                                .getTotalResourceProfile()
+                                                .equals(totalResourceProfile)
+                                        && pendingTaskManager
+                                                .getDefaultSlotResourceProfile()
+                                                .equals(defaultSlotResourceProfile))
+                .max(
+                        Comparator.comparingInt(
+                                pendingTaskManager ->
+                                        taskManagerTracker
+                                                .getPendingAllocationsOfPendingTaskManager(
+                                                        pendingTaskManager
+                                                                .getPendingTaskManagerId())
+                                                .size()))
+                .map(PendingTaskManager::getPendingTaskManagerId);
+    }
+
+    @Override
+    public boolean unregisterTaskManager(InstanceID instanceId, Exception cause) {
+        checkInit();
+
+        LOG.debug("Unregistering task executor {} from the slot manager.", instanceId);
+
+        if (taskManagerTracker.getRegisteredTaskManager(instanceId).isPresent()) {
+            for (AllocationID allocationId :
+                    new HashSet<>(
+                            taskManagerTracker
+                                    .getRegisteredTaskManager(instanceId)
+                                    .get()
+                                    .getAllocatedSlots()
+                                    .keySet())) {
+                slotStatusSyncer.freeSlot(allocationId);
+            }
+            taskManagerTracker.removeTaskManager(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 (taskManagerTracker.getRegisteredTaskManager(instanceId).isPresent()) {
+            slotStatusSyncer.reportSlotStatus(instanceId, 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 freed.
+     *
+     * @param slotId identifying the slot to free, will be ignored
+     * @param allocationId with which the slot is presumably allocated
+     */
+    @Override
+    public void freeSlot(SlotID slotId, AllocationID allocationId) {
+        checkInit();
+        LOG.debug("Freeing slot {}.", allocationId);
+
+        if (taskManagerTracker.getAllocatedOrPendingSlot(allocationId).isPresent()) {
+            slotStatusSyncer.freeSlot(allocationId);
+            checkResourceRequirements();
+        } else {
+            LOG.debug(
+                    "Trying to free a slot {} which has not been allocated. Ignoring this message.",
+                    allocationId);
+        }
+    }
+
+    // ---------------------------------------------------------------------------------------------
+    // Requirement matching
+    // ---------------------------------------------------------------------------------------------
+
+    private void checkResourceRequirements() {
+        final Map<JobID, Collection<ResourceRequirement>> missingResources =
+                resourceTracker.getMissingResources();
+        if (missingResources.isEmpty()) {
+            return;
+        }
+
+        final Map<InstanceID, Tuple2<ResourceProfile, ResourceProfile>> availableResources =
+                taskManagerTracker.getRegisteredTaskManagers().stream()
+                        .collect(
+                                Collectors.toMap(
+                                        TaskManagerInfo::getInstanceId,
+                                        taskManager ->
+                                                Tuple2.of(
+                                                        taskManager.getAvailableResource(),
+                                                        taskManager
+                                                                .getDefaultSlotResourceProfile())));
+        final ResourceAllocationResult result =
+                resourceAllocationStrategy.tryFulfillRequirements(
+                        missingResources,
+                        availableResources,
+                        new ArrayList<>(taskManagerTracker.getPendingTaskManagers()));

Review comment:
       Contract wise, it's unclear whether these arguments would be modified inside the strategy.
   
   It works fine currently, that only `registeredResources` and `pendingTaskManagers` are modified in `DefaultResourceAllocationStrategy`, both of which are generated here and are not reused elsewhere. `missingResources` is not modified in `DefaultResourceAllocationStrategy`. However, this strongly depends on the implementation of `ResourceAllocationStrategy`.
   
   I think we should make it clear in the contract of `ResourceAllocationStrategy` that these arguments can be modified internally, and guard `missingResources` with a deep copy.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/PendingTaskManagerId.java
##########
@@ -22,6 +22,8 @@
 
 /** Id of {@link PendingTaskManager}. */
 public class PendingTaskManagerId extends AbstractID {
+    private static final long serialVersionUID = -6449482902998877861L;

Review comment:
       > Serializable classes must define a Serial Version UID
   > The Serial Version UID for new classes should start at 1
   
   https://flink.apache.org/contributing/code-style-and-quality-java.html#java-serialization

##########
File path: flink-core/src/main/java/org/apache/flink/configuration/ResourceManagerOptions.java
##########
@@ -84,6 +86,14 @@
                                     + "started by Flink, in order to speed up job recovery in case of failures due to task manager lost. "
                                     + "Note that this feature is available only to the active deployments (native K8s, Yarn and Mesos).");
 
+    @Documentation.ExcludeFromDocumentation
+    public static final ConfigOption<String> RESOURCE_ALLOCATION_STRATEGY =
+            ConfigOptions.key("slotmanager.resource-allocation.strategy")
+                    .stringType()
+                    .defaultValue(DEFAULT_RESOURCE_ALLOCATION_STRATEGY)
+                    .withDescription(
+                            "Defines which strategy is used in allocation new slots and task managers.");
+

Review comment:
       I think we don't need this option atm, since there's only the default strategy. Let's add this option later if needed.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/DefaultResourceAllocationStrategy.java
##########
@@ -0,0 +1,233 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.slots.ResourceCounter;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.runtime.resourcemanager.slotmanager.SlotManagerUtils.getEffectiveResourceProfile;
+
+/**
+ * The default implementation of {@link ResourceAllocationStrategy}, which always allocate pending
+ * task managers with the fixed profile.
+ */
+public class DefaultResourceAllocationStrategy implements ResourceAllocationStrategy {
+    private final ResourceProfile defaultSlotResourceProfile;
+    private final ResourceProfile totalResourceProfile;
+
+    public DefaultResourceAllocationStrategy(
+            ResourceProfile defaultSlotResourceProfile, int numSlotsPerWorker) {
+        this.defaultSlotResourceProfile = defaultSlotResourceProfile;
+        this.totalResourceProfile = defaultSlotResourceProfile.multiply(numSlotsPerWorker);
+    }
+
+    /**
+     * Matches resource requirements against available and pending resources. For each job, in a
+     * first round requirements are matched against registered resources. The remaining unfulfilled
+     * requirements are matched against pending resources, allocating more workers if no matching
+     * pending resources could be found. If the requirements for a job could not be fulfilled then
+     * it will be recorded in {@link ResourceAllocationResult#getUnfulfillableJobs()}.
+     *
+     * <p>Performance notes: At it's core this method loops, for each job, over all resources for
+     * each required slot, trying to find a matching registered/pending task manager. One should
+     * generally go in with the assumption that this runs in numberOfJobsRequiringResources *
+     * numberOfRequiredSlots * numberOfFreeOrPendingTaskManagers.
+     *
+     * <p>In the absolute worst case, with J jobs, requiring R slots each with a unique resource
+     * profile such each pair of these profiles is not matching, and T registered/pending task
+     * managers that don't fulfill any requirement, then this method does a total of J*R*T resource
+     * profile comparisons.
+     */
+    @Override
+    public ResourceAllocationResult tryFulfillRequirements(
+            Map<JobID, Collection<ResourceRequirement>> missingResources,
+            Map<InstanceID, Tuple2<ResourceProfile, ResourceProfile>> registeredResources,
+            List<PendingTaskManager> pendingTaskManagers) {
+        final ResourceAllocationResult.Builder resultBuilder = ResourceAllocationResult.builder();
+        final Map<PendingTaskManagerId, ResourceProfile> pendingResources =
+                pendingTaskManagers.stream()
+                        .collect(
+                                Collectors.toMap(
+                                        PendingTaskManager::getPendingTaskManagerId,
+                                        PendingTaskManager::getTotalResourceProfile));
+        for (Map.Entry<JobID, Collection<ResourceRequirement>> resourceRequirements :
+                missingResources.entrySet()) {
+            final JobID jobId = resourceRequirements.getKey();
+
+            final ResourceCounter unfulfilledJobRequirements =
+                    tryFulfillRequirementsForJobWithRegisteredResources(
+                            jobId,
+                            resourceRequirements.getValue(),
+                            registeredResources,
+                            resultBuilder);
+
+            if (!unfulfilledJobRequirements.isEmpty()) {
+                tryFulfillRequirementsForJobWithPendingResources(
+                        jobId, unfulfilledJobRequirements, pendingResources, resultBuilder);
+            }
+        }
+        return resultBuilder.build();
+    }
+
+    private static ResourceCounter tryFulfillRequirementsForJobWithRegisteredResources(
+            JobID jobId,
+            Collection<ResourceRequirement> missingResources,
+            Map<InstanceID, Tuple2<ResourceProfile, ResourceProfile>> registeredResources,
+            ResourceAllocationResult.Builder resultBuilder) {
+        final ResourceCounter outstandingRequirements = new ResourceCounter();
+
+        for (ResourceRequirement resourceRequirement : missingResources) {
+            int numMissingRequirements =
+                    tryFindSlotsForRequirement(
+                            jobId, resourceRequirement, registeredResources, resultBuilder);
+            if (numMissingRequirements > 0) {
+                outstandingRequirements.incrementCount(
+                        resourceRequirement.getResourceProfile(), numMissingRequirements);
+            }
+        }
+        return outstandingRequirements;
+    }
+
+    private static int tryFindSlotsForRequirement(
+            JobID jobId,
+            ResourceRequirement resourceRequirement,
+            Map<InstanceID, Tuple2<ResourceProfile, ResourceProfile>> registeredResources,
+            ResourceAllocationResult.Builder resultBuilder) {
+        final ResourceProfile requiredResource = resourceRequirement.getResourceProfile();
+
+        int numUnfulfilled = 0;
+        for (int x = 0; x < resourceRequirement.getNumberOfRequiredSlots(); x++) {
+            final Optional<InstanceID> matchedTaskManager =
+                    findMatchingTaskManager(requiredResource, registeredResources);
+            if (matchedTaskManager.isPresent()) {
+                final ResourceProfile effectiveProfile =
+                        getEffectiveResourceProfile(
+                                requiredResource,
+                                registeredResources.get(matchedTaskManager.get()).f1);
+                resultBuilder.recordAllocationForRegisteredResource(
+                        jobId, matchedTaskManager.get(), effectiveProfile);
+                deductionRegisteredResource(
+                        registeredResources, matchedTaskManager.get(), effectiveProfile);
+            } else {
+                // exit loop early; we won't find a matching slot for this requirement
+                int numRemaining = resourceRequirement.getNumberOfRequiredSlots() - x;
+                numUnfulfilled += numRemaining;
+                break;
+            }
+        }
+        return numUnfulfilled;
+    }
+
+    private static Optional<InstanceID> findMatchingTaskManager(
+            ResourceProfile requirement,
+            Map<InstanceID, Tuple2<ResourceProfile, ResourceProfile>> registeredResources) {
+        return registeredResources.entrySet().stream()
+                .filter(
+                        taskManager ->
+                                canFulfillRequirement(
+                                        getEffectiveResourceProfile(
+                                                requirement, taskManager.getValue().f1),
+                                        taskManager.getValue().f0))
+                .findFirst()
+                .map(Map.Entry::getKey);
+    }
+
+    private static boolean canFulfillRequirement(
+            ResourceProfile requirement, ResourceProfile resourceProfile) {
+        return resourceProfile.allFieldsNoLessThan(requirement);
+    }
+
+    private static void deductionRegisteredResource(
+            Map<InstanceID, Tuple2<ResourceProfile, ResourceProfile>> registeredResources,
+            InstanceID instanceId,
+            ResourceProfile resourceProfile) {
+        registeredResources.computeIfPresent(

Review comment:
       IIUC, we should never deduct on a non-exist registered resource. Therefore, I would suggest `compute` rather than `computeIfPresent`, and throws an error if the entry is not present.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/PendingTaskManager.java
##########
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.util.Preconditions;
+
+/** Represents a pending task manager in the {@link SlotManager}. */
+public class PendingTaskManager {
+    private final PendingTaskManagerId pendingTaskManagerId;
+    private final ResourceProfile totalResourceProfile;
+    private final ResourceProfile defaultSlotResourceProfile;
+
+    public PendingTaskManager(
+            PendingTaskManagerId pendingTaskManagerId,
+            ResourceProfile totalResourceProfile,
+            ResourceProfile defaultSlotResourceProfile) {
+        this.defaultSlotResourceProfile = Preconditions.checkNotNull(defaultSlotResourceProfile);
+        this.totalResourceProfile = Preconditions.checkNotNull(totalResourceProfile);
+        this.pendingTaskManagerId = Preconditions.checkNotNull(pendingTaskManagerId);
+    }
+
+    public ResourceProfile getTotalResourceProfile() {
+        return totalResourceProfile;
+    }
+
+    public ResourceProfile getDefaultSlotResourceProfile() {
+        return defaultSlotResourceProfile;
+    }
+
+    public PendingTaskManagerId getPendingTaskManagerId() {
+        return pendingTaskManagerId;
+    }
+
+    @Override
+    public String toString() {
+        return String.format(
+                "PendingTaskManager Id=%s, totalResourceProfile=%s, defaultSlotResourceProfile=%s",
+                pendingTaskManagerId, totalResourceProfile, defaultSlotResourceProfile);

Review comment:
       Better to surround with `{}`, so it's easier to understand where it ends when used in a long sentence.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/FineGrainedSlotManager.java
##########
@@ -0,0 +1,790 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.api.java.tuple.Tuple2;
+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.FutureUtils;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.instance.InstanceID;
+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.util.FlinkException;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+/** Implementation of {@link SlotManager} supporting fine-grained resource management. */
+public class FineGrainedSlotManager implements SlotManager {
+    private static final Logger LOG = LoggerFactory.getLogger(FineGrainedSlotManager.class);
+
+    private final TaskManagerTracker taskManagerTracker;
+    private final ResourceTracker resourceTracker;
+    private final ResourceAllocationStrategy resourceAllocationStrategy;
+
+    private final SlotStatusSyncer slotStatusSyncer;
+
+    /** Scheduled executor for timeouts. */
+    private final ScheduledExecutor scheduledExecutor;
+
+    /** Timeout after which an unused TaskManager is released. */
+    private final Time taskManagerTimeout;
+
+    private final SlotManagerMetricGroup slotManagerMetricGroup;
+
+    private final Map<JobID, String> jobMasterTargetAddresses = new HashMap<>();
+
+    /** Defines the max limitation of the total number of task executors. */
+    private final int maxTaskManagerNum;
+
+    /** Defines the number of redundant task executors. */
+    private final int redundantTaskManagerNum;
+
+    /**
+     * Release task executor only when each produced result partition is either consumed or failed.
+     */
+    private final boolean waitResultConsumedBeforeRelease;
+
+    /** The default resource spec of workers to request. */
+    private final WorkerResourceSpec defaultWorkerResourceSpec;
+
+    /** The resource profile of default slot. */
+    private final ResourceProfile defaultSlotResourceProfile;
+
+    private boolean sendNotEnoughResourceNotifications = true;
+
+    /** ResourceManager's id. */
+    @Nullable private ResourceManagerId resourceManagerId;
+
+    /** Executor for future callbacks which have to be "synchronized". */
+    @Nullable private Executor mainThreadExecutor;
+
+    /** Callbacks for resource (de-)allocations. */
+    @Nullable private ResourceActions resourceActions;
+
+    private ScheduledFuture<?> taskManagerTimeoutsAndRedundancyCheck;
+
+    /** True iff the component has been started. */
+    private boolean started;
+
+    public FineGrainedSlotManager(
+            ScheduledExecutor scheduledExecutor,
+            SlotManagerConfiguration slotManagerConfiguration,
+            SlotManagerMetricGroup slotManagerMetricGroup,
+            ResourceTracker resourceTracker,
+            TaskManagerTracker taskManagerTracker) {
+
+        this.scheduledExecutor = Preconditions.checkNotNull(scheduledExecutor);
+
+        Preconditions.checkNotNull(slotManagerConfiguration);
+        this.taskManagerTimeout = slotManagerConfiguration.getTaskManagerTimeout();
+        this.redundantTaskManagerNum = slotManagerConfiguration.getRedundantTaskManagerNum();
+        this.waitResultConsumedBeforeRelease =
+                slotManagerConfiguration.isWaitResultConsumedBeforeRelease();
+        this.defaultWorkerResourceSpec = slotManagerConfiguration.getDefaultWorkerResourceSpec();
+        this.resourceAllocationStrategy = slotManagerConfiguration.getResourceAllocationStrategy();
+        int numSlotsPerWorker = slotManagerConfiguration.getNumSlotsPerWorker();
+        this.maxTaskManagerNum = slotManagerConfiguration.getMaxSlotNum() / numSlotsPerWorker;
+
+        this.slotManagerMetricGroup = Preconditions.checkNotNull(slotManagerMetricGroup);
+        this.defaultSlotResourceProfile =
+                SlotManagerUtils.generateDefaultSlotResourceProfile(
+                        defaultWorkerResourceSpec, numSlotsPerWorker);
+
+        this.resourceTracker = Preconditions.checkNotNull(resourceTracker);
+        this.taskManagerTracker = Preconditions.checkNotNull(taskManagerTracker);
+        this.slotStatusSyncer =
+                new DefaultSlotStatusSyncer(
+                        taskManagerTracker,
+                        resourceTracker,
+                        slotManagerConfiguration.getTaskManagerRequestTimeout());
+
+        resourceManagerId = null;
+        resourceActions = null;
+        mainThreadExecutor = null;
+        taskManagerTimeoutsAndRedundancyCheck = null;
+
+        started = false;
+    }
+
+    @Override
+    public void setFailUnfulfillableRequest(boolean failUnfulfillableRequest) {
+        // this sets up a grace period, e.g., when the cluster was started, to give task executors
+        // time to connect
+        sendNotEnoughResourceNotifications = failUnfulfillableRequest;
+
+        if (failUnfulfillableRequest) {
+            checkResourceRequirements();
+        }
+    }
+
+    // ---------------------------------------------------------------------------------------------
+    // 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.");
+
+        resourceManagerId = Preconditions.checkNotNull(newResourceManagerId);
+        mainThreadExecutor = Preconditions.checkNotNull(newMainThreadExecutor);
+        resourceActions = Preconditions.checkNotNull(newResourceActions);
+
+        started = true;
+
+        taskManagerTimeoutsAndRedundancyCheck =
+                scheduledExecutor.scheduleWithFixedDelay(
+                        () ->
+                                mainThreadExecutor.execute(
+                                        this::checkTaskManagerTimeoutsAndRedundancy),
+                        0L,
+                        taskManagerTimeout.toMilliseconds(),
+                        TimeUnit.MILLISECONDS);
+
+        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() {
+        if (!started) {
+            return;
+        }
+
+        LOG.info("Suspending the slot manager.");
+
+        resourceTracker.clear();
+
+        // stop the timeout checks for the TaskManagers
+        if (taskManagerTimeoutsAndRedundancyCheck != null) {
+            taskManagerTimeoutsAndRedundancyCheck.cancel(false);
+            taskManagerTimeoutsAndRedundancyCheck = null;
+        }
+
+        for (TaskManagerInfo registeredTaskManager :
+                new ArrayList<>(taskManagerTracker.getRegisteredTaskManagers())) {
+            unregisterTaskManager(
+                    registeredTaskManager.getInstanceId(),
+                    new SlotManagerException("The slot manager is being suspended."));
+        }

Review comment:
       `resourceTracker` may still be used during this.
   We should clear `resourceTracker` after this.
   And maybe also clear `taskManagerTracker`, just incase.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/FineGrainedTaskManagerTracker.java
##########
@@ -0,0 +1,237 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.slots.ResourceCounter;
+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.HashSet;
+import java.util.Map;
+import java.util.Optional;
+
+/** Implementation of {@link TaskManagerTracker} supporting fine-grained resource management. */
+public class FineGrainedTaskManagerTracker implements TaskManagerTracker {
+    private static final Logger LOG = LoggerFactory.getLogger(FineGrainedTaskManagerTracker.class);
+
+    /** Map for allocated and pending slots. */
+    private final Map<AllocationID, FineGrainedTaskManagerSlot> slots;
+
+    /** All currently registered task managers. */
+    private final Map<InstanceID, FineGrainedTaskManagerRegistration> taskManagerRegistrations;
+
+    private final Map<PendingTaskManagerId, PendingTaskManager> pendingTaskManagers;
+
+    private final Map<PendingTaskManagerId, Map<JobID, ResourceCounter>>
+            pendingSlotAllocationRecords;
+
+    public FineGrainedTaskManagerTracker() {
+        slots = new HashMap<>(16);
+        taskManagerRegistrations = new HashMap<>(4);
+        pendingTaskManagers = new HashMap<>(16);
+        pendingSlotAllocationRecords = new HashMap<>();
+    }
+
+    @Override
+    public void recordPendingAllocations(
+            Map<PendingTaskManagerId, Map<JobID, ResourceCounter>> pendingSlotAllocations) {
+        LOG.trace("Record the pending allocations {}.", pendingSlotAllocations);

Review comment:
       `ResourceCounter` has not implemented `toString`.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/FineGrainedSlotManager.java
##########
@@ -0,0 +1,790 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.api.java.tuple.Tuple2;
+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.FutureUtils;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.instance.InstanceID;
+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.util.FlinkException;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+/** Implementation of {@link SlotManager} supporting fine-grained resource management. */
+public class FineGrainedSlotManager implements SlotManager {
+    private static final Logger LOG = LoggerFactory.getLogger(FineGrainedSlotManager.class);
+
+    private final TaskManagerTracker taskManagerTracker;
+    private final ResourceTracker resourceTracker;
+    private final ResourceAllocationStrategy resourceAllocationStrategy;
+
+    private final SlotStatusSyncer slotStatusSyncer;
+
+    /** Scheduled executor for timeouts. */
+    private final ScheduledExecutor scheduledExecutor;
+
+    /** Timeout after which an unused TaskManager is released. */
+    private final Time taskManagerTimeout;
+
+    private final SlotManagerMetricGroup slotManagerMetricGroup;
+
+    private final Map<JobID, String> jobMasterTargetAddresses = new HashMap<>();
+
+    /** Defines the max limitation of the total number of task executors. */
+    private final int maxTaskManagerNum;
+
+    /** Defines the number of redundant task executors. */
+    private final int redundantTaskManagerNum;
+
+    /**
+     * Release task executor only when each produced result partition is either consumed or failed.
+     */
+    private final boolean waitResultConsumedBeforeRelease;
+
+    /** The default resource spec of workers to request. */
+    private final WorkerResourceSpec defaultWorkerResourceSpec;
+
+    /** The resource profile of default slot. */
+    private final ResourceProfile defaultSlotResourceProfile;
+
+    private boolean sendNotEnoughResourceNotifications = true;
+
+    /** ResourceManager's id. */
+    @Nullable private ResourceManagerId resourceManagerId;
+
+    /** Executor for future callbacks which have to be "synchronized". */
+    @Nullable private Executor mainThreadExecutor;
+
+    /** Callbacks for resource (de-)allocations. */
+    @Nullable private ResourceActions resourceActions;
+
+    private ScheduledFuture<?> taskManagerTimeoutsAndRedundancyCheck;
+
+    /** True iff the component has been started. */
+    private boolean started;
+
+    public FineGrainedSlotManager(
+            ScheduledExecutor scheduledExecutor,
+            SlotManagerConfiguration slotManagerConfiguration,
+            SlotManagerMetricGroup slotManagerMetricGroup,
+            ResourceTracker resourceTracker,
+            TaskManagerTracker taskManagerTracker) {
+
+        this.scheduledExecutor = Preconditions.checkNotNull(scheduledExecutor);
+
+        Preconditions.checkNotNull(slotManagerConfiguration);
+        this.taskManagerTimeout = slotManagerConfiguration.getTaskManagerTimeout();
+        this.redundantTaskManagerNum = slotManagerConfiguration.getRedundantTaskManagerNum();
+        this.waitResultConsumedBeforeRelease =
+                slotManagerConfiguration.isWaitResultConsumedBeforeRelease();
+        this.defaultWorkerResourceSpec = slotManagerConfiguration.getDefaultWorkerResourceSpec();
+        this.resourceAllocationStrategy = slotManagerConfiguration.getResourceAllocationStrategy();
+        int numSlotsPerWorker = slotManagerConfiguration.getNumSlotsPerWorker();
+        this.maxTaskManagerNum = slotManagerConfiguration.getMaxSlotNum() / numSlotsPerWorker;
+
+        this.slotManagerMetricGroup = Preconditions.checkNotNull(slotManagerMetricGroup);
+        this.defaultSlotResourceProfile =
+                SlotManagerUtils.generateDefaultSlotResourceProfile(
+                        defaultWorkerResourceSpec, numSlotsPerWorker);
+
+        this.resourceTracker = Preconditions.checkNotNull(resourceTracker);
+        this.taskManagerTracker = Preconditions.checkNotNull(taskManagerTracker);
+        this.slotStatusSyncer =
+                new DefaultSlotStatusSyncer(
+                        taskManagerTracker,
+                        resourceTracker,
+                        slotManagerConfiguration.getTaskManagerRequestTimeout());
+
+        resourceManagerId = null;
+        resourceActions = null;
+        mainThreadExecutor = null;
+        taskManagerTimeoutsAndRedundancyCheck = null;
+
+        started = false;
+    }
+
+    @Override
+    public void setFailUnfulfillableRequest(boolean failUnfulfillableRequest) {
+        // this sets up a grace period, e.g., when the cluster was started, to give task executors
+        // time to connect
+        sendNotEnoughResourceNotifications = failUnfulfillableRequest;
+
+        if (failUnfulfillableRequest) {
+            checkResourceRequirements();
+        }
+    }
+
+    // ---------------------------------------------------------------------------------------------
+    // 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.");
+
+        resourceManagerId = Preconditions.checkNotNull(newResourceManagerId);
+        mainThreadExecutor = Preconditions.checkNotNull(newMainThreadExecutor);
+        resourceActions = Preconditions.checkNotNull(newResourceActions);
+
+        started = true;
+
+        taskManagerTimeoutsAndRedundancyCheck =
+                scheduledExecutor.scheduleWithFixedDelay(
+                        () ->
+                                mainThreadExecutor.execute(
+                                        this::checkTaskManagerTimeoutsAndRedundancy),
+                        0L,
+                        taskManagerTimeout.toMilliseconds(),
+                        TimeUnit.MILLISECONDS);
+
+        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() {
+        if (!started) {
+            return;
+        }
+
+        LOG.info("Suspending the slot manager.");
+
+        resourceTracker.clear();
+
+        // stop the timeout checks for the TaskManagers
+        if (taskManagerTimeoutsAndRedundancyCheck != null) {
+            taskManagerTimeoutsAndRedundancyCheck.cancel(false);
+            taskManagerTimeoutsAndRedundancyCheck = null;
+        }
+
+        for (TaskManagerInfo registeredTaskManager :
+                new ArrayList<>(taskManagerTracker.getRegisteredTaskManagers())) {
+            unregisterTaskManager(
+                    registeredTaskManager.getInstanceId(),
+                    new SlotManagerException("The slot manager is being suspended."));
+        }
+
+        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
+     * @param totalResourceProfile of the new task manager
+     * @param defaultSlotResourceProfile of 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,
+            ResourceProfile totalResourceProfile,
+            ResourceProfile defaultSlotResourceProfile) {
+        checkInit();
+        LOG.debug(
+                "Registering task executor {} under {} at the slot manager.",
+                taskExecutorConnection.getResourceID(),
+                taskExecutorConnection.getInstanceID());
+
+        // we identify task managers by their instance id
+        if (taskManagerTracker
+                .getRegisteredTaskManager(taskExecutorConnection.getInstanceID())
+                .isPresent()) {
+            LOG.debug(
+                    "Task executor {} was already registered.",
+                    taskExecutorConnection.getResourceID());
+            reportSlotStatus(taskExecutorConnection.getInstanceID(), initialSlotReport);
+            return false;
+        } else {
+            if (isMaxSlotNumExceededAfterRegistration(
+                    initialSlotReport, totalResourceProfile, defaultSlotResourceProfile)) {
+                LOG.info(
+                        "The total number of slots exceeds the max limitation {}, releasing the excess task executor.",
+                        maxTaskManagerNum);
+                resourceActions.releaseResource(
+                        taskExecutorConnection.getInstanceID(),
+                        new FlinkException(
+                                "The total number of slots exceeds the max limitation."));
+                return false;
+            }
+            taskManagerTracker.addTaskManager(
+                    taskExecutorConnection, totalResourceProfile, defaultSlotResourceProfile);
+            final Optional<PendingTaskManagerId> matchedPendingTaskManager =
+                    findMatchingPendingTaskManager(
+                            totalResourceProfile, defaultSlotResourceProfile);
+            if (!initialSlotReport.withAllocatedSlot() && matchedPendingTaskManager.isPresent()) {
+                taskManagerTracker.removePendingTaskManager(matchedPendingTaskManager.get());
+                allocateSlotsForRegisteredPendingTaskManager(
+                        matchedPendingTaskManager.get(), taskExecutorConnection.getInstanceID());
+            } else {
+                slotStatusSyncer.reportSlotStatus(
+                        taskExecutorConnection.getInstanceID(), initialSlotReport);
+                checkResourceRequirements();
+            }
+            return true;
+        }
+    }
+
+    private void allocateSlotsForRegisteredPendingTaskManager(
+            PendingTaskManagerId pendingTaskManagerId, InstanceID instanceId) {
+        Map<JobID, Map<InstanceID, ResourceCounter>> allocations = new HashMap<>();
+        taskManagerTracker
+                .getPendingAllocationsOfPendingTaskManager(pendingTaskManagerId)
+                .forEach(
+                        ((jobId, resourceCounter) ->
+                                allocations.put(
+                                        jobId,
+                                        Collections.singletonMap(instanceId, resourceCounter))));

Review comment:
       Depending on the map implementation, this might cause concurrent modification to `allocations`.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/FineGrainedSlotManager.java
##########
@@ -0,0 +1,790 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.api.java.tuple.Tuple2;
+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.FutureUtils;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.instance.InstanceID;
+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.util.FlinkException;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+/** Implementation of {@link SlotManager} supporting fine-grained resource management. */
+public class FineGrainedSlotManager implements SlotManager {
+    private static final Logger LOG = LoggerFactory.getLogger(FineGrainedSlotManager.class);
+
+    private final TaskManagerTracker taskManagerTracker;
+    private final ResourceTracker resourceTracker;
+    private final ResourceAllocationStrategy resourceAllocationStrategy;
+
+    private final SlotStatusSyncer slotStatusSyncer;
+
+    /** Scheduled executor for timeouts. */
+    private final ScheduledExecutor scheduledExecutor;
+
+    /** Timeout after which an unused TaskManager is released. */
+    private final Time taskManagerTimeout;
+
+    private final SlotManagerMetricGroup slotManagerMetricGroup;
+
+    private final Map<JobID, String> jobMasterTargetAddresses = new HashMap<>();
+
+    /** Defines the max limitation of the total number of task executors. */
+    private final int maxTaskManagerNum;
+
+    /** Defines the number of redundant task executors. */
+    private final int redundantTaskManagerNum;
+
+    /**
+     * Release task executor only when each produced result partition is either consumed or failed.
+     */
+    private final boolean waitResultConsumedBeforeRelease;
+
+    /** The default resource spec of workers to request. */
+    private final WorkerResourceSpec defaultWorkerResourceSpec;
+
+    /** The resource profile of default slot. */
+    private final ResourceProfile defaultSlotResourceProfile;
+
+    private boolean sendNotEnoughResourceNotifications = true;
+
+    /** ResourceManager's id. */
+    @Nullable private ResourceManagerId resourceManagerId;
+
+    /** Executor for future callbacks which have to be "synchronized". */
+    @Nullable private Executor mainThreadExecutor;
+
+    /** Callbacks for resource (de-)allocations. */
+    @Nullable private ResourceActions resourceActions;
+
+    private ScheduledFuture<?> taskManagerTimeoutsAndRedundancyCheck;
+
+    /** True iff the component has been started. */
+    private boolean started;
+
+    public FineGrainedSlotManager(
+            ScheduledExecutor scheduledExecutor,
+            SlotManagerConfiguration slotManagerConfiguration,
+            SlotManagerMetricGroup slotManagerMetricGroup,
+            ResourceTracker resourceTracker,
+            TaskManagerTracker taskManagerTracker) {
+
+        this.scheduledExecutor = Preconditions.checkNotNull(scheduledExecutor);
+
+        Preconditions.checkNotNull(slotManagerConfiguration);
+        this.taskManagerTimeout = slotManagerConfiguration.getTaskManagerTimeout();
+        this.redundantTaskManagerNum = slotManagerConfiguration.getRedundantTaskManagerNum();
+        this.waitResultConsumedBeforeRelease =
+                slotManagerConfiguration.isWaitResultConsumedBeforeRelease();
+        this.defaultWorkerResourceSpec = slotManagerConfiguration.getDefaultWorkerResourceSpec();
+        this.resourceAllocationStrategy = slotManagerConfiguration.getResourceAllocationStrategy();
+        int numSlotsPerWorker = slotManagerConfiguration.getNumSlotsPerWorker();
+        this.maxTaskManagerNum = slotManagerConfiguration.getMaxSlotNum() / numSlotsPerWorker;
+
+        this.slotManagerMetricGroup = Preconditions.checkNotNull(slotManagerMetricGroup);
+        this.defaultSlotResourceProfile =
+                SlotManagerUtils.generateDefaultSlotResourceProfile(
+                        defaultWorkerResourceSpec, numSlotsPerWorker);
+
+        this.resourceTracker = Preconditions.checkNotNull(resourceTracker);
+        this.taskManagerTracker = Preconditions.checkNotNull(taskManagerTracker);
+        this.slotStatusSyncer =
+                new DefaultSlotStatusSyncer(
+                        taskManagerTracker,
+                        resourceTracker,
+                        slotManagerConfiguration.getTaskManagerRequestTimeout());
+
+        resourceManagerId = null;
+        resourceActions = null;
+        mainThreadExecutor = null;
+        taskManagerTimeoutsAndRedundancyCheck = null;
+
+        started = false;
+    }
+
+    @Override
+    public void setFailUnfulfillableRequest(boolean failUnfulfillableRequest) {
+        // this sets up a grace period, e.g., when the cluster was started, to give task executors
+        // time to connect
+        sendNotEnoughResourceNotifications = failUnfulfillableRequest;
+
+        if (failUnfulfillableRequest) {
+            checkResourceRequirements();
+        }
+    }
+
+    // ---------------------------------------------------------------------------------------------
+    // 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.");
+
+        resourceManagerId = Preconditions.checkNotNull(newResourceManagerId);
+        mainThreadExecutor = Preconditions.checkNotNull(newMainThreadExecutor);
+        resourceActions = Preconditions.checkNotNull(newResourceActions);
+
+        started = true;
+
+        taskManagerTimeoutsAndRedundancyCheck =
+                scheduledExecutor.scheduleWithFixedDelay(
+                        () ->
+                                mainThreadExecutor.execute(
+                                        this::checkTaskManagerTimeoutsAndRedundancy),
+                        0L,
+                        taskManagerTimeout.toMilliseconds(),
+                        TimeUnit.MILLISECONDS);
+
+        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() {
+        if (!started) {
+            return;
+        }
+
+        LOG.info("Suspending the slot manager.");
+
+        resourceTracker.clear();
+
+        // stop the timeout checks for the TaskManagers
+        if (taskManagerTimeoutsAndRedundancyCheck != null) {
+            taskManagerTimeoutsAndRedundancyCheck.cancel(false);
+            taskManagerTimeoutsAndRedundancyCheck = null;
+        }
+
+        for (TaskManagerInfo registeredTaskManager :
+                new ArrayList<>(taskManagerTracker.getRegisteredTaskManagers())) {
+            unregisterTaskManager(
+                    registeredTaskManager.getInstanceId(),
+                    new SlotManagerException("The slot manager is being suspended."));
+        }
+
+        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
+     * @param totalResourceProfile of the new task manager
+     * @param defaultSlotResourceProfile of 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,
+            ResourceProfile totalResourceProfile,
+            ResourceProfile defaultSlotResourceProfile) {
+        checkInit();
+        LOG.debug(
+                "Registering task executor {} under {} at the slot manager.",
+                taskExecutorConnection.getResourceID(),
+                taskExecutorConnection.getInstanceID());
+
+        // we identify task managers by their instance id
+        if (taskManagerTracker
+                .getRegisteredTaskManager(taskExecutorConnection.getInstanceID())
+                .isPresent()) {
+            LOG.debug(
+                    "Task executor {} was already registered.",
+                    taskExecutorConnection.getResourceID());
+            reportSlotStatus(taskExecutorConnection.getInstanceID(), initialSlotReport);
+            return false;
+        } else {
+            if (isMaxSlotNumExceededAfterRegistration(
+                    initialSlotReport, totalResourceProfile, defaultSlotResourceProfile)) {
+                LOG.info(
+                        "The total number of slots exceeds the max limitation {}, releasing the excess task executor.",
+                        maxTaskManagerNum);
+                resourceActions.releaseResource(
+                        taskExecutorConnection.getInstanceID(),
+                        new FlinkException(
+                                "The total number of slots exceeds the max limitation."));
+                return false;
+            }
+            taskManagerTracker.addTaskManager(
+                    taskExecutorConnection, totalResourceProfile, defaultSlotResourceProfile);
+            final Optional<PendingTaskManagerId> matchedPendingTaskManager =
+                    findMatchingPendingTaskManager(
+                            totalResourceProfile, defaultSlotResourceProfile);
+            if (!initialSlotReport.withAllocatedSlot() && matchedPendingTaskManager.isPresent()) {
+                taskManagerTracker.removePendingTaskManager(matchedPendingTaskManager.get());
+                allocateSlotsForRegisteredPendingTaskManager(
+                        matchedPendingTaskManager.get(), taskExecutorConnection.getInstanceID());
+            } else {
+                slotStatusSyncer.reportSlotStatus(
+                        taskExecutorConnection.getInstanceID(), initialSlotReport);
+                checkResourceRequirements();
+            }
+            return true;
+        }
+    }
+
+    private void allocateSlotsForRegisteredPendingTaskManager(
+            PendingTaskManagerId pendingTaskManagerId, InstanceID instanceId) {
+        Map<JobID, Map<InstanceID, ResourceCounter>> allocations = new HashMap<>();
+        taskManagerTracker
+                .getPendingAllocationsOfPendingTaskManager(pendingTaskManagerId)
+                .forEach(
+                        ((jobId, resourceCounter) ->
+                                allocations.put(
+                                        jobId,
+                                        Collections.singletonMap(instanceId, resourceCounter))));
+        allocateSlotsAccordingTo(allocations);
+    }
+
+    private boolean isMaxSlotNumExceededAfterRegistration(
+            SlotReport initialSlotReport,
+            ResourceProfile totalResourceProfile,
+            ResourceProfile defaultSlotResourceProfile) {
+        if (!isMaxSlotNumExceededAfterAdding(1)) {
+            return false;
+        }
+
+        if (initialSlotReport.withAllocatedSlot()) {
+            return isMaxSlotNumExceededAfterAdding(1);
+        }
+
+        return isMaxSlotNumExceededAfterAdding(
+                findMatchingPendingTaskManager(totalResourceProfile, defaultSlotResourceProfile)
+                                .isPresent()
+                        ? 0
+                        : 1);
+    }
+
+    private boolean isMaxSlotNumExceededAfterAdding(int numNewTaskExecutor) {
+        return taskManagerTracker.getPendingTaskManagers().size()
+                        + taskManagerTracker.getRegisteredTaskManagers().size()
+                        + numNewTaskExecutor
+                > maxTaskManagerNum;
+    }
+
+    private Optional<PendingTaskManagerId> findMatchingPendingTaskManager(
+            ResourceProfile totalResourceProfile, ResourceProfile defaultSlotResourceProfile) {
+        return taskManagerTracker.getPendingTaskManagers().stream()
+                .filter(
+                        pendingTaskManager ->
+                                pendingTaskManager
+                                                .getTotalResourceProfile()
+                                                .equals(totalResourceProfile)
+                                        && pendingTaskManager
+                                                .getDefaultSlotResourceProfile()
+                                                .equals(defaultSlotResourceProfile))
+                .max(
+                        Comparator.comparingInt(
+                                pendingTaskManager ->
+                                        taskManagerTracker
+                                                .getPendingAllocationsOfPendingTaskManager(
+                                                        pendingTaskManager
+                                                                .getPendingTaskManagerId())
+                                                .size()))
+                .map(PendingTaskManager::getPendingTaskManagerId);
+    }
+
+    @Override
+    public boolean unregisterTaskManager(InstanceID instanceId, Exception cause) {
+        checkInit();
+
+        LOG.debug("Unregistering task executor {} from the slot manager.", instanceId);
+
+        if (taskManagerTracker.getRegisteredTaskManager(instanceId).isPresent()) {
+            for (AllocationID allocationId :
+                    new HashSet<>(
+                            taskManagerTracker
+                                    .getRegisteredTaskManager(instanceId)
+                                    .get()
+                                    .getAllocatedSlots()
+                                    .keySet())) {
+                slotStatusSyncer.freeSlot(allocationId);
+            }
+            taskManagerTracker.removeTaskManager(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 (taskManagerTracker.getRegisteredTaskManager(instanceId).isPresent()) {
+            slotStatusSyncer.reportSlotStatus(instanceId, 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 freed.
+     *
+     * @param slotId identifying the slot to free, will be ignored
+     * @param allocationId with which the slot is presumably allocated
+     */
+    @Override
+    public void freeSlot(SlotID slotId, AllocationID allocationId) {
+        checkInit();
+        LOG.debug("Freeing slot {}.", allocationId);
+
+        if (taskManagerTracker.getAllocatedOrPendingSlot(allocationId).isPresent()) {
+            slotStatusSyncer.freeSlot(allocationId);
+            checkResourceRequirements();
+        } else {
+            LOG.debug(
+                    "Trying to free a slot {} which has not been allocated. Ignoring this message.",
+                    allocationId);
+        }
+    }
+
+    // ---------------------------------------------------------------------------------------------
+    // Requirement matching
+    // ---------------------------------------------------------------------------------------------
+
+    private void checkResourceRequirements() {
+        final Map<JobID, Collection<ResourceRequirement>> missingResources =
+                resourceTracker.getMissingResources();
+        if (missingResources.isEmpty()) {
+            return;
+        }
+
+        final Map<InstanceID, Tuple2<ResourceProfile, ResourceProfile>> availableResources =
+                taskManagerTracker.getRegisteredTaskManagers().stream()
+                        .collect(
+                                Collectors.toMap(
+                                        TaskManagerInfo::getInstanceId,
+                                        taskManager ->
+                                                Tuple2.of(
+                                                        taskManager.getAvailableResource(),
+                                                        taskManager
+                                                                .getDefaultSlotResourceProfile())));
+        final ResourceAllocationResult result =
+                resourceAllocationStrategy.tryFulfillRequirements(
+                        missingResources,
+                        availableResources,
+                        new ArrayList<>(taskManagerTracker.getPendingTaskManagers()));
+
+        // Allocate slots according to the result
+        allocateSlotsAccordingTo(result.getRegisteredResourceAllocationResult());
+
+        // Allocate task managers according to the result
+        final Set<PendingTaskManagerId> failAllocations =
+                allocateTaskManagersAccordingTo(result.getPendingTaskManagersToBeAllocated());
+
+        // Record slot allocation of pending task managers
+        final Map<PendingTaskManagerId, Map<JobID, ResourceCounter>>
+                pendingResourceAllocationResult =
+                        new HashMap<>(result.getPendingResourceAllocationResult());
+        pendingResourceAllocationResult.keySet().retainAll(failAllocations);

Review comment:
       Shouldn't it be `removeAll`?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/FineGrainedTaskManagerSlot.java
##########
@@ -0,0 +1,113 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.util.Preconditions;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * A FineGrainedTaskManagerSlot represents a slot located in a TaskManager. It contains the
+ * necessary information for the allocation of the slot, and keeps track of the state of the slot.
+ * Note that it should in the state of {@link SlotState#FREE}.
+ *
+ * <p>This class is the fine-grained resource management version of the {@link TaskManagerSlot}.
+ */
+public class FineGrainedTaskManagerSlot implements TaskManagerSlotInformation {
+    /** The resource profile of this slot. */
+    private final ResourceProfile resourceProfile;
+
+    /** Gateway to the TaskExecutor which owns the slot. */
+    private final TaskExecutorConnection taskManagerConnection;
+
+    /** Allocation id for which this slot has been allocated. */
+    private final AllocationID allocationId;
+
+    /** Job id for which this slot has been allocated. */
+    private final JobID jobId;
+
+    /** Current state of this slot. Should be either PENDING or ALLOCATED. */
+    private SlotState state;
+
+    public FineGrainedTaskManagerSlot(
+            AllocationID allocationId,
+            JobID jobId,
+            ResourceProfile resourceProfile,
+            TaskExecutorConnection taskManagerConnection,
+            SlotState slotState) {
+        checkArgument(
+                !slotState.equals(SlotState.FREE),

Review comment:
       NPE if `slotState` is `null`

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/FineGrainedSlotManager.java
##########
@@ -0,0 +1,790 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.api.java.tuple.Tuple2;
+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.FutureUtils;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.instance.InstanceID;
+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.util.FlinkException;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+/** Implementation of {@link SlotManager} supporting fine-grained resource management. */
+public class FineGrainedSlotManager implements SlotManager {
+    private static final Logger LOG = LoggerFactory.getLogger(FineGrainedSlotManager.class);
+
+    private final TaskManagerTracker taskManagerTracker;
+    private final ResourceTracker resourceTracker;
+    private final ResourceAllocationStrategy resourceAllocationStrategy;
+
+    private final SlotStatusSyncer slotStatusSyncer;
+
+    /** Scheduled executor for timeouts. */
+    private final ScheduledExecutor scheduledExecutor;
+
+    /** Timeout after which an unused TaskManager is released. */
+    private final Time taskManagerTimeout;
+
+    private final SlotManagerMetricGroup slotManagerMetricGroup;
+
+    private final Map<JobID, String> jobMasterTargetAddresses = new HashMap<>();
+
+    /** Defines the max limitation of the total number of task executors. */
+    private final int maxTaskManagerNum;
+
+    /** Defines the number of redundant task executors. */
+    private final int redundantTaskManagerNum;
+
+    /**
+     * Release task executor only when each produced result partition is either consumed or failed.
+     */
+    private final boolean waitResultConsumedBeforeRelease;
+
+    /** The default resource spec of workers to request. */
+    private final WorkerResourceSpec defaultWorkerResourceSpec;
+
+    /** The resource profile of default slot. */
+    private final ResourceProfile defaultSlotResourceProfile;
+
+    private boolean sendNotEnoughResourceNotifications = true;
+
+    /** ResourceManager's id. */
+    @Nullable private ResourceManagerId resourceManagerId;
+
+    /** Executor for future callbacks which have to be "synchronized". */
+    @Nullable private Executor mainThreadExecutor;
+
+    /** Callbacks for resource (de-)allocations. */
+    @Nullable private ResourceActions resourceActions;
+
+    private ScheduledFuture<?> taskManagerTimeoutsAndRedundancyCheck;
+
+    /** True iff the component has been started. */
+    private boolean started;
+
+    public FineGrainedSlotManager(
+            ScheduledExecutor scheduledExecutor,
+            SlotManagerConfiguration slotManagerConfiguration,
+            SlotManagerMetricGroup slotManagerMetricGroup,
+            ResourceTracker resourceTracker,
+            TaskManagerTracker taskManagerTracker) {
+
+        this.scheduledExecutor = Preconditions.checkNotNull(scheduledExecutor);
+
+        Preconditions.checkNotNull(slotManagerConfiguration);
+        this.taskManagerTimeout = slotManagerConfiguration.getTaskManagerTimeout();
+        this.redundantTaskManagerNum = slotManagerConfiguration.getRedundantTaskManagerNum();
+        this.waitResultConsumedBeforeRelease =
+                slotManagerConfiguration.isWaitResultConsumedBeforeRelease();
+        this.defaultWorkerResourceSpec = slotManagerConfiguration.getDefaultWorkerResourceSpec();
+        this.resourceAllocationStrategy = slotManagerConfiguration.getResourceAllocationStrategy();
+        int numSlotsPerWorker = slotManagerConfiguration.getNumSlotsPerWorker();
+        this.maxTaskManagerNum = slotManagerConfiguration.getMaxSlotNum() / numSlotsPerWorker;
+
+        this.slotManagerMetricGroup = Preconditions.checkNotNull(slotManagerMetricGroup);
+        this.defaultSlotResourceProfile =
+                SlotManagerUtils.generateDefaultSlotResourceProfile(
+                        defaultWorkerResourceSpec, numSlotsPerWorker);
+
+        this.resourceTracker = Preconditions.checkNotNull(resourceTracker);
+        this.taskManagerTracker = Preconditions.checkNotNull(taskManagerTracker);
+        this.slotStatusSyncer =
+                new DefaultSlotStatusSyncer(
+                        taskManagerTracker,
+                        resourceTracker,
+                        slotManagerConfiguration.getTaskManagerRequestTimeout());
+
+        resourceManagerId = null;
+        resourceActions = null;
+        mainThreadExecutor = null;
+        taskManagerTimeoutsAndRedundancyCheck = null;
+
+        started = false;
+    }
+
+    @Override
+    public void setFailUnfulfillableRequest(boolean failUnfulfillableRequest) {
+        // this sets up a grace period, e.g., when the cluster was started, to give task executors
+        // time to connect
+        sendNotEnoughResourceNotifications = failUnfulfillableRequest;
+
+        if (failUnfulfillableRequest) {
+            checkResourceRequirements();
+        }
+    }
+
+    // ---------------------------------------------------------------------------------------------
+    // 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.");
+
+        resourceManagerId = Preconditions.checkNotNull(newResourceManagerId);
+        mainThreadExecutor = Preconditions.checkNotNull(newMainThreadExecutor);
+        resourceActions = Preconditions.checkNotNull(newResourceActions);
+
+        started = true;
+
+        taskManagerTimeoutsAndRedundancyCheck =
+                scheduledExecutor.scheduleWithFixedDelay(
+                        () ->
+                                mainThreadExecutor.execute(
+                                        this::checkTaskManagerTimeoutsAndRedundancy),
+                        0L,
+                        taskManagerTimeout.toMilliseconds(),
+                        TimeUnit.MILLISECONDS);
+
+        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() {
+        if (!started) {
+            return;
+        }
+
+        LOG.info("Suspending the slot manager.");
+
+        resourceTracker.clear();
+
+        // stop the timeout checks for the TaskManagers
+        if (taskManagerTimeoutsAndRedundancyCheck != null) {
+            taskManagerTimeoutsAndRedundancyCheck.cancel(false);
+            taskManagerTimeoutsAndRedundancyCheck = null;
+        }
+
+        for (TaskManagerInfo registeredTaskManager :
+                new ArrayList<>(taskManagerTracker.getRegisteredTaskManagers())) {
+            unregisterTaskManager(
+                    registeredTaskManager.getInstanceId(),
+                    new SlotManagerException("The slot manager is being suspended."));
+        }
+
+        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
+     * @param totalResourceProfile of the new task manager
+     * @param defaultSlotResourceProfile of 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,
+            ResourceProfile totalResourceProfile,
+            ResourceProfile defaultSlotResourceProfile) {
+        checkInit();
+        LOG.debug(
+                "Registering task executor {} under {} at the slot manager.",
+                taskExecutorConnection.getResourceID(),
+                taskExecutorConnection.getInstanceID());
+
+        // we identify task managers by their instance id
+        if (taskManagerTracker
+                .getRegisteredTaskManager(taskExecutorConnection.getInstanceID())
+                .isPresent()) {
+            LOG.debug(
+                    "Task executor {} was already registered.",
+                    taskExecutorConnection.getResourceID());
+            reportSlotStatus(taskExecutorConnection.getInstanceID(), initialSlotReport);
+            return false;
+        } else {
+            if (isMaxSlotNumExceededAfterRegistration(
+                    initialSlotReport, totalResourceProfile, defaultSlotResourceProfile)) {
+                LOG.info(
+                        "The total number of slots exceeds the max limitation {}, releasing the excess task executor.",
+                        maxTaskManagerNum);
+                resourceActions.releaseResource(
+                        taskExecutorConnection.getInstanceID(),
+                        new FlinkException(
+                                "The total number of slots exceeds the max limitation."));
+                return false;
+            }
+            taskManagerTracker.addTaskManager(
+                    taskExecutorConnection, totalResourceProfile, defaultSlotResourceProfile);
+            final Optional<PendingTaskManagerId> matchedPendingTaskManager =
+                    findMatchingPendingTaskManager(
+                            totalResourceProfile, defaultSlotResourceProfile);
+            if (!initialSlotReport.withAllocatedSlot() && matchedPendingTaskManager.isPresent()) {
+                taskManagerTracker.removePendingTaskManager(matchedPendingTaskManager.get());
+                allocateSlotsForRegisteredPendingTaskManager(
+                        matchedPendingTaskManager.get(), taskExecutorConnection.getInstanceID());
+            } else {
+                slotStatusSyncer.reportSlotStatus(
+                        taskExecutorConnection.getInstanceID(), initialSlotReport);
+                checkResourceRequirements();
+            }
+            return true;
+        }
+    }
+
+    private void allocateSlotsForRegisteredPendingTaskManager(
+            PendingTaskManagerId pendingTaskManagerId, InstanceID instanceId) {
+        Map<JobID, Map<InstanceID, ResourceCounter>> allocations = new HashMap<>();
+        taskManagerTracker
+                .getPendingAllocationsOfPendingTaskManager(pendingTaskManagerId)
+                .forEach(
+                        ((jobId, resourceCounter) ->
+                                allocations.put(
+                                        jobId,
+                                        Collections.singletonMap(instanceId, resourceCounter))));
+        allocateSlotsAccordingTo(allocations);
+    }
+
+    private boolean isMaxSlotNumExceededAfterRegistration(
+            SlotReport initialSlotReport,
+            ResourceProfile totalResourceProfile,
+            ResourceProfile defaultSlotResourceProfile) {
+        if (!isMaxSlotNumExceededAfterAdding(1)) {
+            return false;
+        }
+
+        if (initialSlotReport.withAllocatedSlot()) {
+            return isMaxSlotNumExceededAfterAdding(1);
+        }
+
+        return isMaxSlotNumExceededAfterAdding(
+                findMatchingPendingTaskManager(totalResourceProfile, defaultSlotResourceProfile)
+                                .isPresent()
+                        ? 0
+                        : 1);
+    }
+
+    private boolean isMaxSlotNumExceededAfterAdding(int numNewTaskExecutor) {
+        return taskManagerTracker.getPendingTaskManagers().size()
+                        + taskManagerTracker.getRegisteredTaskManagers().size()
+                        + numNewTaskExecutor
+                > maxTaskManagerNum;
+    }
+
+    private Optional<PendingTaskManagerId> findMatchingPendingTaskManager(
+            ResourceProfile totalResourceProfile, ResourceProfile defaultSlotResourceProfile) {
+        return taskManagerTracker.getPendingTaskManagers().stream()
+                .filter(
+                        pendingTaskManager ->
+                                pendingTaskManager
+                                                .getTotalResourceProfile()
+                                                .equals(totalResourceProfile)
+                                        && pendingTaskManager
+                                                .getDefaultSlotResourceProfile()
+                                                .equals(defaultSlotResourceProfile))
+                .max(
+                        Comparator.comparingInt(
+                                pendingTaskManager ->
+                                        taskManagerTracker
+                                                .getPendingAllocationsOfPendingTaskManager(
+                                                        pendingTaskManager
+                                                                .getPendingTaskManagerId())
+                                                .size()))
+                .map(PendingTaskManager::getPendingTaskManagerId);
+    }
+
+    @Override
+    public boolean unregisterTaskManager(InstanceID instanceId, Exception cause) {
+        checkInit();
+
+        LOG.debug("Unregistering task executor {} from the slot manager.", instanceId);
+
+        if (taskManagerTracker.getRegisteredTaskManager(instanceId).isPresent()) {
+            for (AllocationID allocationId :
+                    new HashSet<>(
+                            taskManagerTracker
+                                    .getRegisteredTaskManager(instanceId)
+                                    .get()
+                                    .getAllocatedSlots()
+                                    .keySet())) {
+                slotStatusSyncer.freeSlot(allocationId);
+            }
+            taskManagerTracker.removeTaskManager(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 (taskManagerTracker.getRegisteredTaskManager(instanceId).isPresent()) {
+            slotStatusSyncer.reportSlotStatus(instanceId, 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 freed.
+     *
+     * @param slotId identifying the slot to free, will be ignored
+     * @param allocationId with which the slot is presumably allocated
+     */
+    @Override
+    public void freeSlot(SlotID slotId, AllocationID allocationId) {
+        checkInit();
+        LOG.debug("Freeing slot {}.", allocationId);
+
+        if (taskManagerTracker.getAllocatedOrPendingSlot(allocationId).isPresent()) {
+            slotStatusSyncer.freeSlot(allocationId);
+            checkResourceRequirements();
+        } else {
+            LOG.debug(
+                    "Trying to free a slot {} which has not been allocated. Ignoring this message.",
+                    allocationId);
+        }
+    }
+
+    // ---------------------------------------------------------------------------------------------
+    // Requirement matching
+    // ---------------------------------------------------------------------------------------------
+
+    private void checkResourceRequirements() {
+        final Map<JobID, Collection<ResourceRequirement>> missingResources =
+                resourceTracker.getMissingResources();
+        if (missingResources.isEmpty()) {
+            return;
+        }
+
+        final Map<InstanceID, Tuple2<ResourceProfile, ResourceProfile>> availableResources =
+                taskManagerTracker.getRegisteredTaskManagers().stream()
+                        .collect(
+                                Collectors.toMap(
+                                        TaskManagerInfo::getInstanceId,
+                                        taskManager ->
+                                                Tuple2.of(
+                                                        taskManager.getAvailableResource(),
+                                                        taskManager
+                                                                .getDefaultSlotResourceProfile())));
+        final ResourceAllocationResult result =
+                resourceAllocationStrategy.tryFulfillRequirements(
+                        missingResources,
+                        availableResources,
+                        new ArrayList<>(taskManagerTracker.getPendingTaskManagers()));
+
+        // Allocate slots according to the result
+        allocateSlotsAccordingTo(result.getRegisteredResourceAllocationResult());
+
+        // Allocate task managers according to the result
+        final Set<PendingTaskManagerId> failAllocations =
+                allocateTaskManagersAccordingTo(result.getPendingTaskManagersToBeAllocated());
+
+        // Record slot allocation of pending task managers
+        final Map<PendingTaskManagerId, Map<JobID, ResourceCounter>>
+                pendingResourceAllocationResult =
+                        new HashMap<>(result.getPendingResourceAllocationResult());
+        pendingResourceAllocationResult.keySet().retainAll(failAllocations);
+        taskManagerTracker.recordPendingAllocations(pendingResourceAllocationResult);
+
+        // Notify jobs that can not be fulfilled
+        final Set<JobID> unfulfillableJobs = new HashSet<>(result.getUnfulfillableJobs());
+        for (PendingTaskManagerId pendingTaskManagerId : failAllocations) {
+            unfulfillableJobs.addAll(
+                    result.getPendingResourceAllocationResult().get(pendingTaskManagerId).keySet());
+        }
+        for (JobID jobId : unfulfillableJobs) {
+            if (sendNotEnoughResourceNotifications) {
+                LOG.warn("Could not fulfill resource requirements of job {}.", jobId);
+                resourceActions.notifyNotEnoughResourcesAvailable(
+                        jobId, resourceTracker.getAcquiredResources(jobId));
+            }
+        }
+    }
+
+    private void allocateSlotsAccordingTo(Map<JobID, Map<InstanceID, ResourceCounter>> result) {
+        final List<CompletableFuture<Void>> allocationFutures = new ArrayList<>();
+        for (JobID jobID : result.keySet()) {
+            for (InstanceID instanceID : result.get(jobID).keySet()) {
+                for (Map.Entry<ResourceProfile, Integer> resourceToBeAllocated :
+                        result.get(jobID)
+                                .get(instanceID)
+                                .getResourceProfilesWithCount()
+                                .entrySet()) {
+                    for (int i = 0; i < resourceToBeAllocated.getValue(); ++i) {
+                        allocationFutures.add(
+                                slotStatusSyncer.allocateSlot(
+                                        instanceID,
+                                        jobID,
+                                        jobMasterTargetAddresses.get(jobID),
+                                        resourceToBeAllocated.getKey(),
+                                        resourceManagerId,
+                                        mainThreadExecutor));
+                    }
+                }
+            }
+        }
+        FutureUtils.combineAll(allocationFutures)
+                .whenCompleteAsync(
+                        (s, t) -> {
+                            if (t != null) {
+                                // If there is allocation failure, we need to trigger it again.
+                                checkResourceRequirements();
+                            }
+                        },
+                        mainThreadExecutor);
+    }
+
+    /**
+     * Allocate pending task managers, returns the ids of pending task managers that can not be
+     * allocated.
+     */
+    private Set<PendingTaskManagerId> allocateTaskManagersAccordingTo(
+            List<PendingTaskManager> pendingTaskManagers) {
+        final Set<PendingTaskManagerId> failedAllocations = new HashSet<>();
+        for (PendingTaskManager pendingTaskManager : pendingTaskManagers) {
+            if (!allocateResource(pendingTaskManager)) {
+                failedAllocations.add(pendingTaskManager.getPendingTaskManagerId());
+            }
+        }
+        return failedAllocations;
+    }
+
+    // ---------------------------------------------------------------------------------------------
+    // Legacy APIs
+    // ---------------------------------------------------------------------------------------------
+
+    @Override
+    public int getNumberRegisteredSlots() {
+        return taskManagerTracker.getStatusOverview().getNumberRegisteredSlots();
+    }
+
+    @Override
+    public int getNumberRegisteredSlotsOf(InstanceID instanceId) {
+        return taskManagerTracker.getStatusOverview().getNumberRegisteredSlotsOf(instanceId);
+    }
+
+    @Override
+    public int getNumberFreeSlots() {
+        return taskManagerTracker.getStatusOverview().getNumberFreeSlots();
+    }
+
+    @Override
+    public int getNumberFreeSlotsOf(InstanceID instanceId) {
+        return taskManagerTracker.getStatusOverview().getNumberFreeSlotsOf(instanceId);
+    }
+
+    @Override
+    public Map<WorkerResourceSpec, Integer> getRequiredResources() {
+        return taskManagerTracker.getPendingTaskManagers().stream()
+                .map(PendingTaskManager::getTotalResourceProfile)
+                .map(WorkerResourceSpec::fromTotalResourceProfile)
+                .collect(Collectors.groupingBy(Function.identity(), Collectors.summingInt(e -> 1)));
+    }
+
+    @Override
+    public ResourceProfile getRegisteredResource() {
+        return taskManagerTracker.getStatusOverview().getRegisteredResource();
+    }
+
+    @Override
+    public ResourceProfile getRegisteredResourceOf(InstanceID instanceID) {
+        return taskManagerTracker.getStatusOverview().getRegisteredResourceOf(instanceID);
+    }
+
+    @Override
+    public ResourceProfile getFreeResource() {
+        return taskManagerTracker.getStatusOverview().getFreeResource();
+    }
+
+    @Override
+    public ResourceProfile getFreeResourceOf(InstanceID instanceID) {
+        return taskManagerTracker.getStatusOverview().getRegisteredResourceOf(instanceID);
+    }
+
+    @Override
+    public int getNumberPendingSlotRequests() {
+        // only exists for testing purposes
+        throw new UnsupportedOperationException();
+    }
+
+    // ---------------------------------------------------------------------------------------------
+    // Internal periodic check methods
+    // ---------------------------------------------------------------------------------------------
+
+    private void checkTaskManagerTimeoutsAndRedundancy() {
+        Map<InstanceID, TaskExecutorConnection> timeoutTaskManagers = getTimeOutTaskManagers();
+        if (!timeoutTaskManagers.isEmpty()) {
+            int freeTaskManagersNum =
+                    (int)
+                            taskManagerTracker.getRegisteredTaskManagers().stream()
+                                    .filter(TaskManagerInfo::isIdle)
+                                    .count();
+            int taskManagersDiff = redundantTaskManagerNum - freeTaskManagersNum;

Review comment:
       These calculations are not equivalent to those in `SlotManagerImpl`. I'm not entirely sure atm about how this feature should be supported for fine grained resource management. Let's exclude this from the scope of this PR and make it a follow-up issue.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/FineGrainedSlotManager.java
##########
@@ -0,0 +1,790 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.api.java.tuple.Tuple2;
+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.FutureUtils;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.instance.InstanceID;
+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.util.FlinkException;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+/** Implementation of {@link SlotManager} supporting fine-grained resource management. */
+public class FineGrainedSlotManager implements SlotManager {
+    private static final Logger LOG = LoggerFactory.getLogger(FineGrainedSlotManager.class);
+
+    private final TaskManagerTracker taskManagerTracker;
+    private final ResourceTracker resourceTracker;
+    private final ResourceAllocationStrategy resourceAllocationStrategy;
+
+    private final SlotStatusSyncer slotStatusSyncer;
+
+    /** Scheduled executor for timeouts. */
+    private final ScheduledExecutor scheduledExecutor;
+
+    /** Timeout after which an unused TaskManager is released. */
+    private final Time taskManagerTimeout;
+
+    private final SlotManagerMetricGroup slotManagerMetricGroup;
+
+    private final Map<JobID, String> jobMasterTargetAddresses = new HashMap<>();
+
+    /** Defines the max limitation of the total number of task executors. */
+    private final int maxTaskManagerNum;
+
+    /** Defines the number of redundant task executors. */
+    private final int redundantTaskManagerNum;
+
+    /**
+     * Release task executor only when each produced result partition is either consumed or failed.
+     */
+    private final boolean waitResultConsumedBeforeRelease;
+
+    /** The default resource spec of workers to request. */
+    private final WorkerResourceSpec defaultWorkerResourceSpec;
+
+    /** The resource profile of default slot. */
+    private final ResourceProfile defaultSlotResourceProfile;
+
+    private boolean sendNotEnoughResourceNotifications = true;
+
+    /** ResourceManager's id. */
+    @Nullable private ResourceManagerId resourceManagerId;
+
+    /** Executor for future callbacks which have to be "synchronized". */
+    @Nullable private Executor mainThreadExecutor;
+
+    /** Callbacks for resource (de-)allocations. */
+    @Nullable private ResourceActions resourceActions;
+
+    private ScheduledFuture<?> taskManagerTimeoutsAndRedundancyCheck;
+
+    /** True iff the component has been started. */
+    private boolean started;
+
+    public FineGrainedSlotManager(
+            ScheduledExecutor scheduledExecutor,
+            SlotManagerConfiguration slotManagerConfiguration,
+            SlotManagerMetricGroup slotManagerMetricGroup,
+            ResourceTracker resourceTracker,
+            TaskManagerTracker taskManagerTracker) {
+
+        this.scheduledExecutor = Preconditions.checkNotNull(scheduledExecutor);
+
+        Preconditions.checkNotNull(slotManagerConfiguration);
+        this.taskManagerTimeout = slotManagerConfiguration.getTaskManagerTimeout();
+        this.redundantTaskManagerNum = slotManagerConfiguration.getRedundantTaskManagerNum();
+        this.waitResultConsumedBeforeRelease =
+                slotManagerConfiguration.isWaitResultConsumedBeforeRelease();
+        this.defaultWorkerResourceSpec = slotManagerConfiguration.getDefaultWorkerResourceSpec();
+        this.resourceAllocationStrategy = slotManagerConfiguration.getResourceAllocationStrategy();
+        int numSlotsPerWorker = slotManagerConfiguration.getNumSlotsPerWorker();
+        this.maxTaskManagerNum = slotManagerConfiguration.getMaxSlotNum() / numSlotsPerWorker;
+
+        this.slotManagerMetricGroup = Preconditions.checkNotNull(slotManagerMetricGroup);
+        this.defaultSlotResourceProfile =
+                SlotManagerUtils.generateDefaultSlotResourceProfile(
+                        defaultWorkerResourceSpec, numSlotsPerWorker);
+
+        this.resourceTracker = Preconditions.checkNotNull(resourceTracker);
+        this.taskManagerTracker = Preconditions.checkNotNull(taskManagerTracker);
+        this.slotStatusSyncer =
+                new DefaultSlotStatusSyncer(
+                        taskManagerTracker,
+                        resourceTracker,
+                        slotManagerConfiguration.getTaskManagerRequestTimeout());
+
+        resourceManagerId = null;
+        resourceActions = null;
+        mainThreadExecutor = null;
+        taskManagerTimeoutsAndRedundancyCheck = null;
+
+        started = false;
+    }
+
+    @Override
+    public void setFailUnfulfillableRequest(boolean failUnfulfillableRequest) {
+        // this sets up a grace period, e.g., when the cluster was started, to give task executors
+        // time to connect
+        sendNotEnoughResourceNotifications = failUnfulfillableRequest;
+
+        if (failUnfulfillableRequest) {
+            checkResourceRequirements();
+        }
+    }
+
+    // ---------------------------------------------------------------------------------------------
+    // 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.");
+
+        resourceManagerId = Preconditions.checkNotNull(newResourceManagerId);
+        mainThreadExecutor = Preconditions.checkNotNull(newMainThreadExecutor);
+        resourceActions = Preconditions.checkNotNull(newResourceActions);
+
+        started = true;
+
+        taskManagerTimeoutsAndRedundancyCheck =
+                scheduledExecutor.scheduleWithFixedDelay(
+                        () ->
+                                mainThreadExecutor.execute(
+                                        this::checkTaskManagerTimeoutsAndRedundancy),
+                        0L,
+                        taskManagerTimeout.toMilliseconds(),
+                        TimeUnit.MILLISECONDS);
+
+        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() {
+        if (!started) {
+            return;
+        }
+
+        LOG.info("Suspending the slot manager.");
+
+        resourceTracker.clear();
+
+        // stop the timeout checks for the TaskManagers
+        if (taskManagerTimeoutsAndRedundancyCheck != null) {
+            taskManagerTimeoutsAndRedundancyCheck.cancel(false);
+            taskManagerTimeoutsAndRedundancyCheck = null;
+        }
+
+        for (TaskManagerInfo registeredTaskManager :
+                new ArrayList<>(taskManagerTracker.getRegisteredTaskManagers())) {
+            unregisterTaskManager(
+                    registeredTaskManager.getInstanceId(),
+                    new SlotManagerException("The slot manager is being suspended."));
+        }
+
+        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
+     * @param totalResourceProfile of the new task manager
+     * @param defaultSlotResourceProfile of 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,
+            ResourceProfile totalResourceProfile,
+            ResourceProfile defaultSlotResourceProfile) {
+        checkInit();
+        LOG.debug(
+                "Registering task executor {} under {} at the slot manager.",
+                taskExecutorConnection.getResourceID(),
+                taskExecutorConnection.getInstanceID());
+
+        // we identify task managers by their instance id
+        if (taskManagerTracker
+                .getRegisteredTaskManager(taskExecutorConnection.getInstanceID())
+                .isPresent()) {
+            LOG.debug(
+                    "Task executor {} was already registered.",
+                    taskExecutorConnection.getResourceID());
+            reportSlotStatus(taskExecutorConnection.getInstanceID(), initialSlotReport);
+            return false;
+        } else {
+            if (isMaxSlotNumExceededAfterRegistration(
+                    initialSlotReport, totalResourceProfile, defaultSlotResourceProfile)) {
+                LOG.info(
+                        "The total number of slots exceeds the max limitation {}, releasing the excess task executor.",
+                        maxTaskManagerNum);
+                resourceActions.releaseResource(
+                        taskExecutorConnection.getInstanceID(),
+                        new FlinkException(
+                                "The total number of slots exceeds the max limitation."));
+                return false;
+            }
+            taskManagerTracker.addTaskManager(
+                    taskExecutorConnection, totalResourceProfile, defaultSlotResourceProfile);
+            final Optional<PendingTaskManagerId> matchedPendingTaskManager =
+                    findMatchingPendingTaskManager(
+                            totalResourceProfile, defaultSlotResourceProfile);
+            if (!initialSlotReport.withAllocatedSlot() && matchedPendingTaskManager.isPresent()) {
+                taskManagerTracker.removePendingTaskManager(matchedPendingTaskManager.get());
+                allocateSlotsForRegisteredPendingTaskManager(
+                        matchedPendingTaskManager.get(), taskExecutorConnection.getInstanceID());
+            } else {
+                slotStatusSyncer.reportSlotStatus(
+                        taskExecutorConnection.getInstanceID(), initialSlotReport);
+                checkResourceRequirements();
+            }
+            return true;
+        }
+    }
+
+    private void allocateSlotsForRegisteredPendingTaskManager(
+            PendingTaskManagerId pendingTaskManagerId, InstanceID instanceId) {
+        Map<JobID, Map<InstanceID, ResourceCounter>> allocations = new HashMap<>();
+        taskManagerTracker
+                .getPendingAllocationsOfPendingTaskManager(pendingTaskManagerId)
+                .forEach(
+                        ((jobId, resourceCounter) ->
+                                allocations.put(
+                                        jobId,
+                                        Collections.singletonMap(instanceId, resourceCounter))));
+        allocateSlotsAccordingTo(allocations);
+    }
+
+    private boolean isMaxSlotNumExceededAfterRegistration(
+            SlotReport initialSlotReport,
+            ResourceProfile totalResourceProfile,
+            ResourceProfile defaultSlotResourceProfile) {
+        if (!isMaxSlotNumExceededAfterAdding(1)) {
+            return false;
+        }
+
+        if (initialSlotReport.withAllocatedSlot()) {
+            return isMaxSlotNumExceededAfterAdding(1);
+        }
+
+        return isMaxSlotNumExceededAfterAdding(
+                findMatchingPendingTaskManager(totalResourceProfile, defaultSlotResourceProfile)
+                                .isPresent()
+                        ? 0
+                        : 1);
+    }
+
+    private boolean isMaxSlotNumExceededAfterAdding(int numNewTaskExecutor) {
+        return taskManagerTracker.getPendingTaskManagers().size()
+                        + taskManagerTracker.getRegisteredTaskManagers().size()
+                        + numNewTaskExecutor
+                > maxTaskManagerNum;
+    }
+
+    private Optional<PendingTaskManagerId> findMatchingPendingTaskManager(
+            ResourceProfile totalResourceProfile, ResourceProfile defaultSlotResourceProfile) {
+        return taskManagerTracker.getPendingTaskManagers().stream()
+                .filter(
+                        pendingTaskManager ->
+                                pendingTaskManager
+                                                .getTotalResourceProfile()
+                                                .equals(totalResourceProfile)
+                                        && pendingTaskManager
+                                                .getDefaultSlotResourceProfile()
+                                                .equals(defaultSlotResourceProfile))
+                .max(
+                        Comparator.comparingInt(
+                                pendingTaskManager ->
+                                        taskManagerTracker
+                                                .getPendingAllocationsOfPendingTaskManager(
+                                                        pendingTaskManager
+                                                                .getPendingTaskManagerId())
+                                                .size()))
+                .map(PendingTaskManager::getPendingTaskManagerId);
+    }
+
+    @Override
+    public boolean unregisterTaskManager(InstanceID instanceId, Exception cause) {
+        checkInit();
+
+        LOG.debug("Unregistering task executor {} from the slot manager.", instanceId);
+
+        if (taskManagerTracker.getRegisteredTaskManager(instanceId).isPresent()) {
+            for (AllocationID allocationId :
+                    new HashSet<>(
+                            taskManagerTracker
+                                    .getRegisteredTaskManager(instanceId)
+                                    .get()
+                                    .getAllocatedSlots()
+                                    .keySet())) {
+                slotStatusSyncer.freeSlot(allocationId);
+            }
+            taskManagerTracker.removeTaskManager(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 (taskManagerTracker.getRegisteredTaskManager(instanceId).isPresent()) {
+            slotStatusSyncer.reportSlotStatus(instanceId, 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 freed.
+     *
+     * @param slotId identifying the slot to free, will be ignored
+     * @param allocationId with which the slot is presumably allocated
+     */
+    @Override
+    public void freeSlot(SlotID slotId, AllocationID allocationId) {
+        checkInit();
+        LOG.debug("Freeing slot {}.", allocationId);
+
+        if (taskManagerTracker.getAllocatedOrPendingSlot(allocationId).isPresent()) {
+            slotStatusSyncer.freeSlot(allocationId);
+            checkResourceRequirements();
+        } else {
+            LOG.debug(
+                    "Trying to free a slot {} which has not been allocated. Ignoring this message.",
+                    allocationId);
+        }
+    }
+
+    // ---------------------------------------------------------------------------------------------
+    // Requirement matching
+    // ---------------------------------------------------------------------------------------------
+
+    private void checkResourceRequirements() {
+        final Map<JobID, Collection<ResourceRequirement>> missingResources =
+                resourceTracker.getMissingResources();
+        if (missingResources.isEmpty()) {
+            return;
+        }
+
+        final Map<InstanceID, Tuple2<ResourceProfile, ResourceProfile>> availableResources =
+                taskManagerTracker.getRegisteredTaskManagers().stream()
+                        .collect(
+                                Collectors.toMap(
+                                        TaskManagerInfo::getInstanceId,
+                                        taskManager ->
+                                                Tuple2.of(
+                                                        taskManager.getAvailableResource(),
+                                                        taskManager
+                                                                .getDefaultSlotResourceProfile())));
+        final ResourceAllocationResult result =
+                resourceAllocationStrategy.tryFulfillRequirements(
+                        missingResources,
+                        availableResources,
+                        new ArrayList<>(taskManagerTracker.getPendingTaskManagers()));
+
+        // Allocate slots according to the result
+        allocateSlotsAccordingTo(result.getRegisteredResourceAllocationResult());
+
+        // Allocate task managers according to the result
+        final Set<PendingTaskManagerId> failAllocations =
+                allocateTaskManagersAccordingTo(result.getPendingTaskManagersToBeAllocated());
+
+        // Record slot allocation of pending task managers
+        final Map<PendingTaskManagerId, Map<JobID, ResourceCounter>>
+                pendingResourceAllocationResult =
+                        new HashMap<>(result.getPendingResourceAllocationResult());
+        pendingResourceAllocationResult.keySet().retainAll(failAllocations);
+        taskManagerTracker.recordPendingAllocations(pendingResourceAllocationResult);
+
+        // Notify jobs that can not be fulfilled
+        final Set<JobID> unfulfillableJobs = new HashSet<>(result.getUnfulfillableJobs());
+        for (PendingTaskManagerId pendingTaskManagerId : failAllocations) {
+            unfulfillableJobs.addAll(
+                    result.getPendingResourceAllocationResult().get(pendingTaskManagerId).keySet());
+        }
+        for (JobID jobId : unfulfillableJobs) {
+            if (sendNotEnoughResourceNotifications) {
+                LOG.warn("Could not fulfill resource requirements of job {}.", jobId);
+                resourceActions.notifyNotEnoughResourcesAvailable(
+                        jobId, resourceTracker.getAcquiredResources(jobId));
+            }
+        }
+    }
+
+    private void allocateSlotsAccordingTo(Map<JobID, Map<InstanceID, ResourceCounter>> result) {
+        final List<CompletableFuture<Void>> allocationFutures = new ArrayList<>();
+        for (JobID jobID : result.keySet()) {
+            for (InstanceID instanceID : result.get(jobID).keySet()) {
+                for (Map.Entry<ResourceProfile, Integer> resourceToBeAllocated :
+                        result.get(jobID)
+                                .get(instanceID)

Review comment:
       Duplicated map lookup.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/FineGrainedSlotManager.java
##########
@@ -0,0 +1,790 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.api.java.tuple.Tuple2;
+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.FutureUtils;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.instance.InstanceID;
+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.util.FlinkException;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+/** Implementation of {@link SlotManager} supporting fine-grained resource management. */
+public class FineGrainedSlotManager implements SlotManager {
+    private static final Logger LOG = LoggerFactory.getLogger(FineGrainedSlotManager.class);
+
+    private final TaskManagerTracker taskManagerTracker;
+    private final ResourceTracker resourceTracker;
+    private final ResourceAllocationStrategy resourceAllocationStrategy;
+
+    private final SlotStatusSyncer slotStatusSyncer;
+
+    /** Scheduled executor for timeouts. */
+    private final ScheduledExecutor scheduledExecutor;
+
+    /** Timeout after which an unused TaskManager is released. */
+    private final Time taskManagerTimeout;
+
+    private final SlotManagerMetricGroup slotManagerMetricGroup;
+
+    private final Map<JobID, String> jobMasterTargetAddresses = new HashMap<>();
+
+    /** Defines the max limitation of the total number of task executors. */
+    private final int maxTaskManagerNum;
+
+    /** Defines the number of redundant task executors. */
+    private final int redundantTaskManagerNum;
+
+    /**
+     * Release task executor only when each produced result partition is either consumed or failed.
+     */
+    private final boolean waitResultConsumedBeforeRelease;
+
+    /** The default resource spec of workers to request. */
+    private final WorkerResourceSpec defaultWorkerResourceSpec;
+
+    /** The resource profile of default slot. */
+    private final ResourceProfile defaultSlotResourceProfile;
+
+    private boolean sendNotEnoughResourceNotifications = true;
+
+    /** ResourceManager's id. */
+    @Nullable private ResourceManagerId resourceManagerId;
+
+    /** Executor for future callbacks which have to be "synchronized". */
+    @Nullable private Executor mainThreadExecutor;
+
+    /** Callbacks for resource (de-)allocations. */
+    @Nullable private ResourceActions resourceActions;
+
+    private ScheduledFuture<?> taskManagerTimeoutsAndRedundancyCheck;
+
+    /** True iff the component has been started. */
+    private boolean started;
+
+    public FineGrainedSlotManager(
+            ScheduledExecutor scheduledExecutor,
+            SlotManagerConfiguration slotManagerConfiguration,
+            SlotManagerMetricGroup slotManagerMetricGroup,
+            ResourceTracker resourceTracker,
+            TaskManagerTracker taskManagerTracker) {
+
+        this.scheduledExecutor = Preconditions.checkNotNull(scheduledExecutor);
+
+        Preconditions.checkNotNull(slotManagerConfiguration);
+        this.taskManagerTimeout = slotManagerConfiguration.getTaskManagerTimeout();
+        this.redundantTaskManagerNum = slotManagerConfiguration.getRedundantTaskManagerNum();
+        this.waitResultConsumedBeforeRelease =
+                slotManagerConfiguration.isWaitResultConsumedBeforeRelease();
+        this.defaultWorkerResourceSpec = slotManagerConfiguration.getDefaultWorkerResourceSpec();
+        this.resourceAllocationStrategy = slotManagerConfiguration.getResourceAllocationStrategy();
+        int numSlotsPerWorker = slotManagerConfiguration.getNumSlotsPerWorker();
+        this.maxTaskManagerNum = slotManagerConfiguration.getMaxSlotNum() / numSlotsPerWorker;
+
+        this.slotManagerMetricGroup = Preconditions.checkNotNull(slotManagerMetricGroup);
+        this.defaultSlotResourceProfile =
+                SlotManagerUtils.generateDefaultSlotResourceProfile(
+                        defaultWorkerResourceSpec, numSlotsPerWorker);
+
+        this.resourceTracker = Preconditions.checkNotNull(resourceTracker);
+        this.taskManagerTracker = Preconditions.checkNotNull(taskManagerTracker);
+        this.slotStatusSyncer =
+                new DefaultSlotStatusSyncer(
+                        taskManagerTracker,
+                        resourceTracker,
+                        slotManagerConfiguration.getTaskManagerRequestTimeout());
+
+        resourceManagerId = null;
+        resourceActions = null;
+        mainThreadExecutor = null;
+        taskManagerTimeoutsAndRedundancyCheck = null;
+
+        started = false;
+    }
+
+    @Override
+    public void setFailUnfulfillableRequest(boolean failUnfulfillableRequest) {
+        // this sets up a grace period, e.g., when the cluster was started, to give task executors
+        // time to connect
+        sendNotEnoughResourceNotifications = failUnfulfillableRequest;
+
+        if (failUnfulfillableRequest) {
+            checkResourceRequirements();
+        }
+    }
+
+    // ---------------------------------------------------------------------------------------------
+    // 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.");
+
+        resourceManagerId = Preconditions.checkNotNull(newResourceManagerId);
+        mainThreadExecutor = Preconditions.checkNotNull(newMainThreadExecutor);
+        resourceActions = Preconditions.checkNotNull(newResourceActions);
+
+        started = true;
+
+        taskManagerTimeoutsAndRedundancyCheck =
+                scheduledExecutor.scheduleWithFixedDelay(
+                        () ->
+                                mainThreadExecutor.execute(
+                                        this::checkTaskManagerTimeoutsAndRedundancy),
+                        0L,
+                        taskManagerTimeout.toMilliseconds(),
+                        TimeUnit.MILLISECONDS);
+
+        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() {
+        if (!started) {
+            return;
+        }
+
+        LOG.info("Suspending the slot manager.");
+
+        resourceTracker.clear();
+
+        // stop the timeout checks for the TaskManagers
+        if (taskManagerTimeoutsAndRedundancyCheck != null) {
+            taskManagerTimeoutsAndRedundancyCheck.cancel(false);
+            taskManagerTimeoutsAndRedundancyCheck = null;
+        }
+
+        for (TaskManagerInfo registeredTaskManager :
+                new ArrayList<>(taskManagerTracker.getRegisteredTaskManagers())) {
+            unregisterTaskManager(
+                    registeredTaskManager.getInstanceId(),
+                    new SlotManagerException("The slot manager is being suspended."));
+        }
+
+        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
+     * @param totalResourceProfile of the new task manager
+     * @param defaultSlotResourceProfile of 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,
+            ResourceProfile totalResourceProfile,
+            ResourceProfile defaultSlotResourceProfile) {
+        checkInit();
+        LOG.debug(
+                "Registering task executor {} under {} at the slot manager.",
+                taskExecutorConnection.getResourceID(),
+                taskExecutorConnection.getInstanceID());
+
+        // we identify task managers by their instance id
+        if (taskManagerTracker
+                .getRegisteredTaskManager(taskExecutorConnection.getInstanceID())
+                .isPresent()) {
+            LOG.debug(
+                    "Task executor {} was already registered.",
+                    taskExecutorConnection.getResourceID());
+            reportSlotStatus(taskExecutorConnection.getInstanceID(), initialSlotReport);
+            return false;
+        } else {
+            if (isMaxSlotNumExceededAfterRegistration(
+                    initialSlotReport, totalResourceProfile, defaultSlotResourceProfile)) {
+                LOG.info(
+                        "The total number of slots exceeds the max limitation {}, releasing the excess task executor.",
+                        maxTaskManagerNum);
+                resourceActions.releaseResource(
+                        taskExecutorConnection.getInstanceID(),
+                        new FlinkException(
+                                "The total number of slots exceeds the max limitation."));
+                return false;
+            }
+            taskManagerTracker.addTaskManager(
+                    taskExecutorConnection, totalResourceProfile, defaultSlotResourceProfile);
+            final Optional<PendingTaskManagerId> matchedPendingTaskManager =
+                    findMatchingPendingTaskManager(
+                            totalResourceProfile, defaultSlotResourceProfile);
+            if (!initialSlotReport.withAllocatedSlot() && matchedPendingTaskManager.isPresent()) {
+                taskManagerTracker.removePendingTaskManager(matchedPendingTaskManager.get());
+                allocateSlotsForRegisteredPendingTaskManager(
+                        matchedPendingTaskManager.get(), taskExecutorConnection.getInstanceID());
+            } else {
+                slotStatusSyncer.reportSlotStatus(
+                        taskExecutorConnection.getInstanceID(), initialSlotReport);
+                checkResourceRequirements();
+            }
+            return true;
+        }
+    }
+
+    private void allocateSlotsForRegisteredPendingTaskManager(
+            PendingTaskManagerId pendingTaskManagerId, InstanceID instanceId) {
+        Map<JobID, Map<InstanceID, ResourceCounter>> allocations = new HashMap<>();
+        taskManagerTracker
+                .getPendingAllocationsOfPendingTaskManager(pendingTaskManagerId)
+                .forEach(
+                        ((jobId, resourceCounter) ->
+                                allocations.put(
+                                        jobId,
+                                        Collections.singletonMap(instanceId, resourceCounter))));
+        allocateSlotsAccordingTo(allocations);
+    }
+
+    private boolean isMaxSlotNumExceededAfterRegistration(
+            SlotReport initialSlotReport,
+            ResourceProfile totalResourceProfile,
+            ResourceProfile defaultSlotResourceProfile) {
+        if (!isMaxSlotNumExceededAfterAdding(1)) {
+            return false;
+        }
+
+        if (initialSlotReport.withAllocatedSlot()) {
+            return isMaxSlotNumExceededAfterAdding(1);
+        }
+
+        return isMaxSlotNumExceededAfterAdding(
+                findMatchingPendingTaskManager(totalResourceProfile, defaultSlotResourceProfile)
+                                .isPresent()
+                        ? 0
+                        : 1);
+    }
+
+    private boolean isMaxSlotNumExceededAfterAdding(int numNewTaskExecutor) {
+        return taskManagerTracker.getPendingTaskManagers().size()
+                        + taskManagerTracker.getRegisteredTaskManagers().size()
+                        + numNewTaskExecutor
+                > maxTaskManagerNum;
+    }
+
+    private Optional<PendingTaskManagerId> findMatchingPendingTaskManager(
+            ResourceProfile totalResourceProfile, ResourceProfile defaultSlotResourceProfile) {
+        return taskManagerTracker.getPendingTaskManagers().stream()
+                .filter(
+                        pendingTaskManager ->
+                                pendingTaskManager
+                                                .getTotalResourceProfile()
+                                                .equals(totalResourceProfile)
+                                        && pendingTaskManager
+                                                .getDefaultSlotResourceProfile()
+                                                .equals(defaultSlotResourceProfile))
+                .max(
+                        Comparator.comparingInt(
+                                pendingTaskManager ->
+                                        taskManagerTracker
+                                                .getPendingAllocationsOfPendingTaskManager(
+                                                        pendingTaskManager
+                                                                .getPendingTaskManagerId())
+                                                .size()))

Review comment:
       This means always going through all pending workers.
   IIUC, this is to prioritize matching pending workers with assigned slots over those without assigned slots. Not sure if it's necessary to follow a strict order that workers with more assigned slots should be prioritized over those with less assigned slots, given the price. 




----------------------------------------------------------------
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] KarmaGYZ commented on a change in pull request #14647: [FLINK-20835] Implement FineGrainedSlotManager

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/ResourceAllocationResult.java
##########
@@ -0,0 +1,121 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.slots.ResourceCounter;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+/** Contains the results of the {@link ResourceAllocationStrategy}. */
+public class ResourceAllocationResult {
+    private final Set<JobID> unfulfillableJobs;
+    private final Map<JobID, Map<InstanceID, ResourceCounter>> registeredResourceAllocationResult;
+    private final List<PendingTaskManager> pendingTaskManagersToBeAllocated;
+    private final Map<PendingTaskManagerId, Map<JobID, ResourceCounter>>
+            pendingResourceAllocationResult;
+
+    private ResourceAllocationResult(
+            Set<JobID> unfulfillableJobs,
+            Map<JobID, Map<InstanceID, ResourceCounter>> registeredResourceAllocationResult,
+            List<PendingTaskManager> pendingTaskManagersToBeAllocated,
+            Map<PendingTaskManagerId, Map<JobID, ResourceCounter>>
+                    pendingResourceAllocationResult) {
+        this.unfulfillableJobs = unfulfillableJobs;
+        this.registeredResourceAllocationResult = registeredResourceAllocationResult;
+        this.pendingTaskManagersToBeAllocated = pendingTaskManagersToBeAllocated;
+        this.pendingResourceAllocationResult = pendingResourceAllocationResult;
+    }
+
+    public List<PendingTaskManager> getPendingTaskManagersToBeAllocated() {
+        return Collections.unmodifiableList(pendingTaskManagersToBeAllocated);
+    }
+
+    public Set<JobID> getUnfulfillableJobs() {
+        return Collections.unmodifiableSet(unfulfillableJobs);
+    }
+
+    public Map<JobID, Map<InstanceID, ResourceCounter>> getRegisteredResourceAllocationResult() {
+        return Collections.unmodifiableMap(registeredResourceAllocationResult);

Review comment:
       That's a good point. I agree that it is not necessary for this PR,  given that it is more than 5000 lines now.




----------------------------------------------------------------
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 #14647: [FLINK-20835] Implement FineGrainedSlotManager

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "1c605546a2ac429e1c475eea6f80a75d0326c7bf",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "1c605546a2ac429e1c475eea6f80a75d0326c7bf",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 1c605546a2ac429e1c475eea6f80a75d0326c7bf 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] KarmaGYZ commented on a change in pull request #14647: [FLINK-20835] Implement FineGrainedSlotManager

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/FineGrainedTaskManagerSlot.java
##########
@@ -0,0 +1,113 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.util.Preconditions;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * A FineGrainedTaskManagerSlot represents a slot located in a TaskManager. It contains the
+ * necessary information for the allocation of the slot, and keeps track of the state of the slot.
+ * Note that it should in the state of {@link SlotState#FREE}.

Review comment:
       nice catch




----------------------------------------------------------------
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] xintongsong commented on a change in pull request #14647: [FLINK-20835] Implement FineGrainedSlotManager

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



##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/FineGrainedTaskManagerTrackerTest.java
##########
@@ -0,0 +1,254 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.slots.ResourceCounter;
+import org.apache.flink.runtime.taskexecutor.TestingTaskExecutorGatewayBuilder;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Test;
+
+import java.util.Collections;
+
+import static org.hamcrest.Matchers.empty;
+import static org.hamcrest.core.Is.is;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+
+/** Tests for the {@link FineGrainedTaskManagerTracker}. */
+public class FineGrainedTaskManagerTrackerTest extends TestLogger {

Review comment:
       I think we might need to cover more cases.
   - When `recordPendingAllocations` is called multiple times, only the last time is recorded.
   - `removePendingTaskManager` called on pending task manager where pending slots are allocated.
   
   And also some failure cases.
   - Trying to remove a (pending) task manager that does not exist.
   - Try to allocate slot from a (pending) task manager that does not have enough resource
   - Try to free a non-exist slot.




----------------------------------------------------------------
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] KarmaGYZ commented on a change in pull request #14647: [FLINK-20835] Implement FineGrainedSlotManager

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/DefaultSlotStatusSyncer.java
##########
@@ -0,0 +1,258 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.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.FutureUtils;
+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.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.HashSet;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+
+/** Default implementation of {@link SlotStatusSyncer} for fine-grained slot management. */
+public class DefaultSlotStatusSyncer implements SlotStatusSyncer {
+    private static final Logger LOG = LoggerFactory.getLogger(DefaultSlotStatusSyncer.class);
+
+    private final TaskManagerTracker taskManagerTracker;
+    private final ResourceTracker resourceTracker;
+    private final Set<AllocationID> pendingSlotAllocations;
+    /** Timeout for slot requests to the task manager. */
+    private final Time taskManagerRequestTimeout;
+
+    public DefaultSlotStatusSyncer(
+            TaskManagerTracker taskManagerTracker,
+            ResourceTracker resourceTracker,
+            Time taskManagerRequestTimeout) {
+        this.taskManagerTracker = Preconditions.checkNotNull(taskManagerTracker);
+        this.resourceTracker = Preconditions.checkNotNull(resourceTracker);
+        this.taskManagerRequestTimeout = Preconditions.checkNotNull(taskManagerRequestTimeout);
+
+        this.pendingSlotAllocations = new HashSet<>(16);
+    }
+
+    @Override
+    public CompletableFuture<Void> allocateSlot(
+            InstanceID instanceId,
+            JobID jobId,
+            String targetAddress,
+            ResourceProfile resourceProfile,
+            ResourceManagerId resourceManagerId,
+            Executor mainThreadExecutor) {
+        final AllocationID allocationId = new AllocationID();
+        final Optional<TaskManagerInfo> taskManager =
+                taskManagerTracker.getRegisteredTaskManager(instanceId);
+        Preconditions.checkState(
+                taskManager.isPresent(),
+                "Could not find a registered task manager for instance id " + instanceId + '.');
+        final TaskExecutorGateway gateway =
+                taskManager.get().getTaskExecutorConnection().getTaskExecutorGateway();
+
+        taskManagerTracker.notifySlotStatus(
+                allocationId, jobId, instanceId, resourceProfile, SlotState.PENDING);
+        resourceTracker.notifyAcquiredResource(jobId, resourceProfile);
+        pendingSlotAllocations.add(allocationId);
+
+        // RPC call to the task manager
+        CompletableFuture<Acknowledge> requestFuture =
+                gateway.requestSlot(
+                        SlotID.getDynamicSlotID(
+                                taskManager.get().getTaskExecutorConnection().getResourceID()),
+                        jobId,
+                        allocationId,
+                        resourceProfile,
+                        targetAddress,
+                        resourceManagerId,
+                        taskManagerRequestTimeout);
+
+        CompletableFuture<Void> returnedFuture = new CompletableFuture<>();
+
+        FutureUtils.assertNoException(
+                requestFuture.handleAsync(

Review comment:
       We use `handleAsync` in case there is an exception during the handling of the `acknowledge` or `throwable` returned by the `requestFuture `.




----------------------------------------------------------------
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] KarmaGYZ commented on a change in pull request #14647: [FLINK-20835] Implement FineGrainedSlotManager

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



##########
File path: flink-core/src/main/java/org/apache/flink/configuration/ClusterOptions.java
##########
@@ -111,6 +111,19 @@
                     .withDescription(
                             "Defines whether the cluster uses declarative resource management.");
 
+    @Documentation.ExcludeFromDocumentation
+    public static final ConfigOption<Boolean> ENABLE_FINE_GRAINED_RESOURCE_MANAGEMENT =
+            ConfigOptions.key("cluster.fine-grained-resource-management.enabled")
+                    .booleanType()
+                    .defaultValue(false)
+                    .withDescription(
+                            "Defines whether the cluster uses fine-grained resource management.");
+
+    public static boolean isFineGrainedResourceManagementEnabled(Configuration configuration) {

Review comment:
       I think it's just about personal taste.




----------------------------------------------------------------
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] xintongsong commented on a change in pull request #14647: [FLINK-20835] Implement FineGrainedSlotManager

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/DefaultSlotStatusSyncer.java
##########
@@ -224,11 +224,12 @@ public void reportSlotStatus(InstanceID instanceId, SlotReport slotReport) {
             LOG.debug(
                     "Received slot report for unknown task manager with instance id {}. Ignoring this report.",
                     instanceId);
-            return;
+            return false;
         }
 
         LOG.debug("Received slot report from instance {}: {}.", instanceId, slotReport);
 
+        boolean triggerCheckRequirement = false;

Review comment:
       The name `canApplyPreviousAllocations ` is not consistent with the value now




----------------------------------------------------------------
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 #14647: [FLINK-20835] Implement FineGrainedSlotManager

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "1c605546a2ac429e1c475eea6f80a75d0326c7bf",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12054",
       "triggerID" : "1c605546a2ac429e1c475eea6f80a75d0326c7bf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "559cd8d09cca8082e9ca4904d11b11443f4c959f",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12226",
       "triggerID" : "559cd8d09cca8082e9ca4904d11b11443f4c959f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 1c605546a2ac429e1c475eea6f80a75d0326c7bf Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12054) 
   * 559cd8d09cca8082e9ca4904d11b11443f4c959f Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12226) 
   
   <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] KarmaGYZ commented on a change in pull request #14647: [FLINK-20835] Implement FineGrainedSlotManager

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/PendingTaskManagerId.java
##########
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.util.AbstractID;
+
+/** Id of {@link PendingTaskManager}. */
+public class PendingTaskManagerId extends AbstractID {
+    private PendingTaskManagerId() {}

Review comment:
       Why we need that as A call to the parent class's empty constructor super() is done automatically?




----------------------------------------------------------------
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 #14647: [FLINK-20835] Implement FineGrainedSlotManager

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "1c605546a2ac429e1c475eea6f80a75d0326c7bf",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12054",
       "triggerID" : "1c605546a2ac429e1c475eea6f80a75d0326c7bf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "559cd8d09cca8082e9ca4904d11b11443f4c959f",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12226",
       "triggerID" : "559cd8d09cca8082e9ca4904d11b11443f4c959f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "05733d3630a9a72d04545e2e8aeefdf76b2e5cf1",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12235",
       "triggerID" : "05733d3630a9a72d04545e2e8aeefdf76b2e5cf1",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 1c605546a2ac429e1c475eea6f80a75d0326c7bf Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12054) 
   * 559cd8d09cca8082e9ca4904d11b11443f4c959f Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12226) 
   * 05733d3630a9a72d04545e2e8aeefdf76b2e5cf1 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12235) 
   
   <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 #14647: [FLINK-20835] Implement FineGrainedSlotManager

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "1c605546a2ac429e1c475eea6f80a75d0326c7bf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12054",
       "triggerID" : "1c605546a2ac429e1c475eea6f80a75d0326c7bf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "559cd8d09cca8082e9ca4904d11b11443f4c959f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12226",
       "triggerID" : "559cd8d09cca8082e9ca4904d11b11443f4c959f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "05733d3630a9a72d04545e2e8aeefdf76b2e5cf1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12235",
       "triggerID" : "05733d3630a9a72d04545e2e8aeefdf76b2e5cf1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0a88531c02021ab6bc36c8eec483dac0ee91db97",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12540",
       "triggerID" : "0a88531c02021ab6bc36c8eec483dac0ee91db97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "45b2dc853d01aa726bdf054b3c2afa5b95a1d6fd",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12543",
       "triggerID" : "45b2dc853d01aa726bdf054b3c2afa5b95a1d6fd",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 0a88531c02021ab6bc36c8eec483dac0ee91db97 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12540) 
   * 45b2dc853d01aa726bdf054b3c2afa5b95a1d6fd Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12543) 
   
   <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 commented on pull request #14647: [FLINK-20835] Implement FineGrainedSlotManager

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


   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 1c605546a2ac429e1c475eea6f80a75d0326c7bf (Thu Jan 14 11:43:26 UTC 2021)
   
   **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] KarmaGYZ commented on a change in pull request #14647: [FLINK-20835] Implement FineGrainedSlotManager

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/ResourceAllocationResult.java
##########
@@ -0,0 +1,121 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.slots.ResourceCounter;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+/** Contains the results of the {@link ResourceAllocationStrategy}. */
+public class ResourceAllocationResult {
+    private final Set<JobID> unfulfillableJobs;
+    private final Map<JobID, Map<InstanceID, ResourceCounter>> registeredResourceAllocationResult;
+    private final List<PendingTaskManager> pendingTaskManagersToBeAllocated;
+    private final Map<PendingTaskManagerId, Map<JobID, ResourceCounter>>
+            pendingResourceAllocationResult;
+
+    private ResourceAllocationResult(
+            Set<JobID> unfulfillableJobs,
+            Map<JobID, Map<InstanceID, ResourceCounter>> registeredResourceAllocationResult,
+            List<PendingTaskManager> pendingTaskManagersToBeAllocated,
+            Map<PendingTaskManagerId, Map<JobID, ResourceCounter>>
+                    pendingResourceAllocationResult) {
+        this.unfulfillableJobs = unfulfillableJobs;
+        this.registeredResourceAllocationResult = registeredResourceAllocationResult;
+        this.pendingTaskManagersToBeAllocated = pendingTaskManagersToBeAllocated;
+        this.pendingResourceAllocationResult = pendingResourceAllocationResult;
+    }
+
+    public List<PendingTaskManager> getPendingTaskManagersToBeAllocated() {
+        return Collections.unmodifiableList(pendingTaskManagersToBeAllocated);
+    }
+
+    public Set<JobID> getUnfulfillableJobs() {
+        return Collections.unmodifiableSet(unfulfillableJobs);
+    }
+
+    public Map<JobID, Map<InstanceID, ResourceCounter>> getRegisteredResourceAllocationResult() {
+        return Collections.unmodifiableMap(registeredResourceAllocationResult);

Review comment:
       That's a good point. I agree that the strict unmodifiability is not necessary for this PR,  given that it is more than 5000 lines now.




----------------------------------------------------------------
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 #14647: [FLINK-20835] Implement FineGrainedSlotManager

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "1c605546a2ac429e1c475eea6f80a75d0326c7bf",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12054",
       "triggerID" : "1c605546a2ac429e1c475eea6f80a75d0326c7bf",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 1c605546a2ac429e1c475eea6f80a75d0326c7bf Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12054) 
   
   <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] KarmaGYZ commented on a change in pull request #14647: [FLINK-20835] Implement FineGrainedSlotManager

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/FineGrainedSlotManager.java
##########
@@ -0,0 +1,790 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.api.java.tuple.Tuple2;
+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.FutureUtils;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.instance.InstanceID;
+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.util.FlinkException;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+/** Implementation of {@link SlotManager} supporting fine-grained resource management. */
+public class FineGrainedSlotManager implements SlotManager {
+    private static final Logger LOG = LoggerFactory.getLogger(FineGrainedSlotManager.class);
+
+    private final TaskManagerTracker taskManagerTracker;
+    private final ResourceTracker resourceTracker;
+    private final ResourceAllocationStrategy resourceAllocationStrategy;
+
+    private final SlotStatusSyncer slotStatusSyncer;
+
+    /** Scheduled executor for timeouts. */
+    private final ScheduledExecutor scheduledExecutor;
+
+    /** Timeout after which an unused TaskManager is released. */
+    private final Time taskManagerTimeout;
+
+    private final SlotManagerMetricGroup slotManagerMetricGroup;
+
+    private final Map<JobID, String> jobMasterTargetAddresses = new HashMap<>();
+
+    /** Defines the max limitation of the total number of task executors. */
+    private final int maxTaskManagerNum;
+
+    /** Defines the number of redundant task executors. */
+    private final int redundantTaskManagerNum;
+
+    /**
+     * Release task executor only when each produced result partition is either consumed or failed.
+     */
+    private final boolean waitResultConsumedBeforeRelease;
+
+    /** The default resource spec of workers to request. */
+    private final WorkerResourceSpec defaultWorkerResourceSpec;
+
+    /** The resource profile of default slot. */
+    private final ResourceProfile defaultSlotResourceProfile;
+
+    private boolean sendNotEnoughResourceNotifications = true;
+
+    /** ResourceManager's id. */
+    @Nullable private ResourceManagerId resourceManagerId;
+
+    /** Executor for future callbacks which have to be "synchronized". */
+    @Nullable private Executor mainThreadExecutor;
+
+    /** Callbacks for resource (de-)allocations. */
+    @Nullable private ResourceActions resourceActions;
+
+    private ScheduledFuture<?> taskManagerTimeoutsAndRedundancyCheck;
+
+    /** True iff the component has been started. */
+    private boolean started;
+
+    public FineGrainedSlotManager(
+            ScheduledExecutor scheduledExecutor,
+            SlotManagerConfiguration slotManagerConfiguration,
+            SlotManagerMetricGroup slotManagerMetricGroup,
+            ResourceTracker resourceTracker,
+            TaskManagerTracker taskManagerTracker) {
+
+        this.scheduledExecutor = Preconditions.checkNotNull(scheduledExecutor);
+
+        Preconditions.checkNotNull(slotManagerConfiguration);
+        this.taskManagerTimeout = slotManagerConfiguration.getTaskManagerTimeout();
+        this.redundantTaskManagerNum = slotManagerConfiguration.getRedundantTaskManagerNum();
+        this.waitResultConsumedBeforeRelease =
+                slotManagerConfiguration.isWaitResultConsumedBeforeRelease();
+        this.defaultWorkerResourceSpec = slotManagerConfiguration.getDefaultWorkerResourceSpec();
+        this.resourceAllocationStrategy = slotManagerConfiguration.getResourceAllocationStrategy();
+        int numSlotsPerWorker = slotManagerConfiguration.getNumSlotsPerWorker();
+        this.maxTaskManagerNum = slotManagerConfiguration.getMaxSlotNum() / numSlotsPerWorker;

Review comment:
       I create https://issues.apache.org/jira/browse/FLINK-21177 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 #14647: [FLINK-20835] Implement FineGrainedSlotManager

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "1c605546a2ac429e1c475eea6f80a75d0326c7bf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12054",
       "triggerID" : "1c605546a2ac429e1c475eea6f80a75d0326c7bf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "559cd8d09cca8082e9ca4904d11b11443f4c959f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12226",
       "triggerID" : "559cd8d09cca8082e9ca4904d11b11443f4c959f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "05733d3630a9a72d04545e2e8aeefdf76b2e5cf1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12235",
       "triggerID" : "05733d3630a9a72d04545e2e8aeefdf76b2e5cf1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0a88531c02021ab6bc36c8eec483dac0ee91db97",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12540",
       "triggerID" : "0a88531c02021ab6bc36c8eec483dac0ee91db97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "45b2dc853d01aa726bdf054b3c2afa5b95a1d6fd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12543",
       "triggerID" : "45b2dc853d01aa726bdf054b3c2afa5b95a1d6fd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "80e3121418e741d9bebf3e849817a0c4225a86dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12587",
       "triggerID" : "80e3121418e741d9bebf3e849817a0c4225a86dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f5f9f44b437922f5e0edd0b1dd3097a61351dee",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12723",
       "triggerID" : "8f5f9f44b437922f5e0edd0b1dd3097a61351dee",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0b72f53fbfe19ef426572f78ac716521bc63ab64",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12769",
       "triggerID" : "0b72f53fbfe19ef426572f78ac716521bc63ab64",
       "triggerType" : "PUSH"
     }, {
       "hash" : "970360401dcd333f1a3b29355392385262953246",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12823",
       "triggerID" : "970360401dcd333f1a3b29355392385262953246",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 970360401dcd333f1a3b29355392385262953246 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12823) 
   
   <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] KarmaGYZ commented on a change in pull request #14647: [FLINK-20835] Implement FineGrainedSlotManager

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/DefaultResourceAllocationStrategy.java
##########
@@ -0,0 +1,233 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.slots.ResourceCounter;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.runtime.resourcemanager.slotmanager.SlotManagerUtils.getEffectiveResourceProfile;
+
+/**
+ * The default implementation of {@link ResourceAllocationStrategy}, which always allocate pending
+ * task managers with the fixed profile.
+ */
+public class DefaultResourceAllocationStrategy implements ResourceAllocationStrategy {
+    private final ResourceProfile defaultSlotResourceProfile;
+    private final ResourceProfile totalResourceProfile;
+
+    public DefaultResourceAllocationStrategy(
+            ResourceProfile defaultSlotResourceProfile, int numSlotsPerWorker) {
+        this.defaultSlotResourceProfile = defaultSlotResourceProfile;
+        this.totalResourceProfile = defaultSlotResourceProfile.multiply(numSlotsPerWorker);
+    }
+
+    /**
+     * Matches resource requirements against available and pending resources. For each job, in a
+     * first round requirements are matched against registered resources. The remaining unfulfilled
+     * requirements are matched against pending resources, allocating more workers if no matching
+     * pending resources could be found. If the requirements for a job could not be fulfilled then
+     * it will be recorded in {@link ResourceAllocationResult#getUnfulfillableJobs()}.
+     *
+     * <p>Performance notes: At it's core this method loops, for each job, over all resources for
+     * each required slot, trying to find a matching registered/pending task manager. One should
+     * generally go in with the assumption that this runs in numberOfJobsRequiringResources *
+     * numberOfRequiredSlots * numberOfFreeOrPendingTaskManagers.
+     *
+     * <p>In the absolute worst case, with J jobs, requiring R slots each with a unique resource
+     * profile such each pair of these profiles is not matching, and T registered/pending task
+     * managers that don't fulfill any requirement, then this method does a total of J*R*T resource
+     * profile comparisons.
+     */
+    @Override
+    public ResourceAllocationResult tryFulfillRequirements(
+            Map<JobID, Collection<ResourceRequirement>> missingResources,
+            Map<InstanceID, Tuple2<ResourceProfile, ResourceProfile>> registeredResources,
+            List<PendingTaskManager> pendingTaskManagers) {
+        final ResourceAllocationResult.Builder resultBuilder = ResourceAllocationResult.builder();
+        final Map<PendingTaskManagerId, ResourceProfile> pendingResources =
+                pendingTaskManagers.stream()
+                        .collect(
+                                Collectors.toMap(
+                                        PendingTaskManager::getPendingTaskManagerId,
+                                        PendingTaskManager::getTotalResourceProfile));
+        for (Map.Entry<JobID, Collection<ResourceRequirement>> resourceRequirements :
+                missingResources.entrySet()) {
+            final JobID jobId = resourceRequirements.getKey();
+
+            final ResourceCounter unfulfilledJobRequirements =
+                    tryFulfillRequirementsForJobWithRegisteredResources(
+                            jobId,
+                            resourceRequirements.getValue(),
+                            registeredResources,
+                            resultBuilder);
+
+            if (!unfulfilledJobRequirements.isEmpty()) {
+                tryFulfillRequirementsForJobWithPendingResources(
+                        jobId, unfulfilledJobRequirements, pendingResources, resultBuilder);
+            }
+        }
+        return resultBuilder.build();
+    }
+
+    private static ResourceCounter tryFulfillRequirementsForJobWithRegisteredResources(
+            JobID jobId,
+            Collection<ResourceRequirement> missingResources,
+            Map<InstanceID, Tuple2<ResourceProfile, ResourceProfile>> registeredResources,
+            ResourceAllocationResult.Builder resultBuilder) {
+        final ResourceCounter outstandingRequirements = new ResourceCounter();
+
+        for (ResourceRequirement resourceRequirement : missingResources) {
+            int numMissingRequirements =
+                    tryFindSlotsForRequirement(
+                            jobId, resourceRequirement, registeredResources, resultBuilder);
+            if (numMissingRequirements > 0) {
+                outstandingRequirements.incrementCount(
+                        resourceRequirement.getResourceProfile(), numMissingRequirements);
+            }
+        }
+        return outstandingRequirements;
+    }
+
+    private static int tryFindSlotsForRequirement(
+            JobID jobId,
+            ResourceRequirement resourceRequirement,
+            Map<InstanceID, Tuple2<ResourceProfile, ResourceProfile>> registeredResources,
+            ResourceAllocationResult.Builder resultBuilder) {
+        final ResourceProfile requiredResource = resourceRequirement.getResourceProfile();
+
+        int numUnfulfilled = 0;
+        for (int x = 0; x < resourceRequirement.getNumberOfRequiredSlots(); x++) {
+            final Optional<InstanceID> matchedTaskManager =
+                    findMatchingTaskManager(requiredResource, registeredResources);
+            if (matchedTaskManager.isPresent()) {
+                final ResourceProfile effectiveProfile =
+                        getEffectiveResourceProfile(
+                                requiredResource,
+                                registeredResources.get(matchedTaskManager.get()).f1);
+                resultBuilder.recordAllocationForRegisteredResource(
+                        jobId, matchedTaskManager.get(), effectiveProfile);
+                deductionRegisteredResource(
+                        registeredResources, matchedTaskManager.get(), effectiveProfile);
+            } else {
+                // exit loop early; we won't find a matching slot for this requirement
+                int numRemaining = resourceRequirement.getNumberOfRequiredSlots() - x;
+                numUnfulfilled += numRemaining;
+                break;
+            }
+        }
+        return numUnfulfilled;
+    }
+
+    private static Optional<InstanceID> findMatchingTaskManager(
+            ResourceProfile requirement,
+            Map<InstanceID, Tuple2<ResourceProfile, ResourceProfile>> registeredResources) {
+        return registeredResources.entrySet().stream()
+                .filter(
+                        taskManager ->
+                                canFulfillRequirement(
+                                        getEffectiveResourceProfile(
+                                                requirement, taskManager.getValue().f1),
+                                        taskManager.getValue().f0))
+                .findFirst()
+                .map(Map.Entry::getKey);
+    }
+
+    private static boolean canFulfillRequirement(
+            ResourceProfile requirement, ResourceProfile resourceProfile) {
+        return resourceProfile.allFieldsNoLessThan(requirement);
+    }
+
+    private static void deductionRegisteredResource(
+            Map<InstanceID, Tuple2<ResourceProfile, ResourceProfile>> registeredResources,
+            InstanceID instanceId,
+            ResourceProfile resourceProfile) {
+        registeredResources.computeIfPresent(
+                instanceId,
+                (id, tuple2) -> {
+                    if (tuple2.f0.subtract(resourceProfile).equals(ResourceProfile.ZERO)) {
+                        return null;
+                    } else {
+                        return Tuple2.of(tuple2.f0.subtract(resourceProfile), tuple2.f1);
+                    }
+                });
+    }
+
+    private static Optional<PendingTaskManagerId> findPendingManagerToFulfill(
+            ResourceProfile resourceProfile,
+            Map<PendingTaskManagerId, ResourceProfile> availableResources) {
+        return availableResources.entrySet().stream()
+                .filter(entry -> entry.getValue().allFieldsNoLessThan(resourceProfile))
+                .findAny()
+                .map(Map.Entry::getKey);
+    }
+
+    private void tryFulfillRequirementsForJobWithPendingResources(
+            JobID jobId,
+            ResourceCounter unfulfilledRequirements,
+            Map<PendingTaskManagerId, ResourceProfile> availableResources,
+            ResourceAllocationResult.Builder resultBuilder) {
+        for (Map.Entry<ResourceProfile, Integer> missingResource :
+                unfulfilledRequirements.getResourceProfilesWithCount().entrySet()) {
+            final ResourceProfile effectiveProfile =
+                    getEffectiveResourceProfile(
+                            missingResource.getKey(), defaultSlotResourceProfile);
+            for (int i = 0; i < missingResource.getValue(); i++) {
+                Optional<PendingTaskManagerId> matchedPendingTaskManager =
+                        findPendingManagerToFulfill(effectiveProfile, availableResources);
+                if (matchedPendingTaskManager.isPresent()) {
+                    availableResources.compute(
+                            matchedPendingTaskManager.get(),
+                            ((pendingTaskManagerId, resourceProfile) ->
+                                    resourceProfile.subtract(effectiveProfile)));
+                    resultBuilder.recordAllocationForPendingResource(
+                            jobId, matchedPendingTaskManager.get(), effectiveProfile);
+                } else {
+                    if (totalResourceProfile.allFieldsNoLessThan(effectiveProfile)) {
+                        // Add new pending task manager
+                        final PendingTaskManagerId pendingTaskManagerId =
+                                PendingTaskManagerId.generate();
+                        resultBuilder.addPendingTaskManagerToBeAllocated(
+                                new PendingTaskManager(
+                                        pendingTaskManagerId,
+                                        totalResourceProfile,
+                                        defaultSlotResourceProfile));
+                        resultBuilder.recordAllocationForPendingResource(
+                                jobId, pendingTaskManagerId, effectiveProfile);
+                        availableResources.put(
+                                pendingTaskManagerId,
+                                totalResourceProfile.subtract(effectiveProfile));
+                    } else {
+                        resultBuilder.addUnfulfillableJob(jobId);

Review comment:
       > An unfulfillable job will probably fail anyway, maybe we should not allocate for this job at all, save the resources for the fulfillable jobs.
   
   I agree it's a valid optimization in some cases. The reason I didn't do this optimization is I am not sure the assumption(unfulfillable job will probably fail anyway) holds forever. Reactive mode is a counterexample. +1 for making it an open follow-up question.




----------------------------------------------------------------
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] KarmaGYZ commented on a change in pull request #14647: [FLINK-20835] Implement FineGrainedSlotManager

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



##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/FineGrainedTaskManagerTrackerTest.java
##########
@@ -0,0 +1,254 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.slots.ResourceCounter;
+import org.apache.flink.runtime.taskexecutor.TestingTaskExecutorGatewayBuilder;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Test;
+
+import java.util.Collections;
+
+import static org.hamcrest.Matchers.empty;
+import static org.hamcrest.core.Is.is;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+
+/** Tests for the {@link FineGrainedTaskManagerTracker}. */
+public class FineGrainedTaskManagerTrackerTest extends TestLogger {

Review comment:
       > Try to allocate slot from a (pending) task manager that does not have enough resource
   I'll put it into the `FineGrainedTaskManagerRegistrationTest`.




----------------------------------------------------------------
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 #14647: [FLINK-20835] Implement FineGrainedSlotManager

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "1c605546a2ac429e1c475eea6f80a75d0326c7bf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12054",
       "triggerID" : "1c605546a2ac429e1c475eea6f80a75d0326c7bf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "559cd8d09cca8082e9ca4904d11b11443f4c959f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12226",
       "triggerID" : "559cd8d09cca8082e9ca4904d11b11443f4c959f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "05733d3630a9a72d04545e2e8aeefdf76b2e5cf1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12235",
       "triggerID" : "05733d3630a9a72d04545e2e8aeefdf76b2e5cf1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0a88531c02021ab6bc36c8eec483dac0ee91db97",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12540",
       "triggerID" : "0a88531c02021ab6bc36c8eec483dac0ee91db97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "45b2dc853d01aa726bdf054b3c2afa5b95a1d6fd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12543",
       "triggerID" : "45b2dc853d01aa726bdf054b3c2afa5b95a1d6fd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "80e3121418e741d9bebf3e849817a0c4225a86dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12587",
       "triggerID" : "80e3121418e741d9bebf3e849817a0c4225a86dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f5f9f44b437922f5e0edd0b1dd3097a61351dee",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12723",
       "triggerID" : "8f5f9f44b437922f5e0edd0b1dd3097a61351dee",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0b72f53fbfe19ef426572f78ac716521bc63ab64",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "0b72f53fbfe19ef426572f78ac716521bc63ab64",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 8f5f9f44b437922f5e0edd0b1dd3097a61351dee Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12723) 
   * 0b72f53fbfe19ef426572f78ac716521bc63ab64 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 edited a comment on pull request #14647: [FLINK-20835] Implement FineGrainedSlotManager

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


   Could someone walk me through the core differences between this slot manager and the declarative slot manager?
   From a cursory glance, they appear to be quite similar. Is it just that we track resources per task executor and based on that determine which TM we ask for a slot with X resources?
   
   Specifically what I'm wondering about is whether we could unify these from the get-go. I'd think we'd just need an abstraction for the available resources of a task executor; everything for already allocated slots should behave the same?


----------------------------------------------------------------
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] KarmaGYZ commented on a change in pull request #14647: [FLINK-20835] Implement FineGrainedSlotManager

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



##########
File path: flink-core/src/main/java/org/apache/flink/configuration/ClusterOptions.java
##########
@@ -111,6 +111,19 @@
                     .withDescription(
                             "Defines whether the cluster uses declarative resource management.");
 
+    @Documentation.ExcludeFromDocumentation
+    public static final ConfigOption<Boolean> ENABLE_FINE_GRAINED_RESOURCE_MANAGEMENT =
+            ConfigOptions.key("cluster.fine-grained-resource-management.enabled")
+                    .booleanType()
+                    .defaultValue(false)
+                    .withDescription(
+                            "Defines whether the cluster uses fine-grained resource management.");
+
+    public static boolean isFineGrainedResourceManagementEnabled(Configuration configuration) {

Review comment:
       I think it's just about personal taste.




----------------------------------------------------------------
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 #14647: [FLINK-20835] Implement FineGrainedSlotManager

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


   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 f0dc516d44616986d6ddf6e03902bbb1ffb2cb03 (Fri May 28 08:17:07 UTC 2021)
   
   **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] xintongsong commented on a change in pull request #14647: [FLINK-20835] Implement FineGrainedSlotManager

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/TaskExecutorInfo.java
##########
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+
+/** Basic resource information of a TaskExecutor. */
+public interface TaskExecutorInfo {

Review comment:
       I think the class name is a bit too general.
   Maybe `TaskManagerResourceView` or `TaskManagerResourceStatus` to indicate 1) it's about resource and 2) it provides a view of the a changeable status.

##########
File path: flink-core/src/main/java/org/apache/flink/configuration/ClusterOptions.java
##########
@@ -111,6 +111,19 @@
                     .withDescription(
                             "Defines whether the cluster uses declarative resource management.");
 
+    @Documentation.ExcludeFromDocumentation
+    public static final ConfigOption<Boolean> ENABLE_FINE_GRAINED_RESOURCE_MANAGEMENT =
+            ConfigOptions.key("cluster.fine-grained-resource-management.enabled")
+                    .booleanType()
+                    .defaultValue(false)
+                    .withDescription(
+                            "Defines whether the cluster uses fine-grained resource management.");
+
+    public static boolean isFineGrainedResourceManagementEnabled(Configuration configuration) {

Review comment:
       nit: call me captious, but maybe keep the same order between fine-grained and declarative for the config options and methods. 

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/TaskExecutorTracker.java
##########
@@ -0,0 +1,274 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.common.time.Time;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.taskexecutor.SlotReport;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+
+/** Tracks TaskExecutor's resource and slot status. */
+interface TaskExecutorTracker {
+
+    /**
+     * Registers the given listener with this tracker.
+     *
+     * @param slotStatusUpdateListener listener to register
+     */
+    void registerSlotStatusUpdateListener(SlotStatusUpdateListener slotStatusUpdateListener);
+
+    // ---------------------------------------------------------------------------------------------
+    // Add / Remove (pending) Resource
+    // ---------------------------------------------------------------------------------------------
+
+    /**
+     * Register a new task executor with its initial slot report.
+     *
+     * @param taskExecutorConnection of the new task executor
+     * @param initialSlotReport of the new task executor
+     * @param totalResourceProfile of the new task executor
+     * @param defaultSlotResourceProfile of the new task executor
+     */
+    void addTaskExecutor(
+            TaskExecutorConnection taskExecutorConnection,
+            SlotReport initialSlotReport,
+            ResourceProfile totalResourceProfile,
+            ResourceProfile defaultSlotResourceProfile);
+
+    /**
+     * Add a new pending task executor with its resource profile.
+     *
+     * @param totalResourceProfile of the pending task executor
+     * @param defaultSlotResourceProfile of the pending task executor
+     */
+    void addPendingTaskExecutor(
+            ResourceProfile totalResourceProfile, ResourceProfile defaultSlotResourceProfile);
+
+    /**
+     * Unregister a task executor with the given instance id.
+     *
+     * @param instanceId of the task executor
+     */
+    void removeTaskExecutor(InstanceID instanceId);
+

Review comment:
       Should there be a method for removing a pending task executor? Say the worker is not registered and is no longer needed.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/TaskExecutorTracker.java
##########
@@ -0,0 +1,274 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.common.time.Time;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.taskexecutor.SlotReport;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+
+/** Tracks TaskExecutor's resource and slot status. */
+interface TaskExecutorTracker {
+
+    /**
+     * Registers the given listener with this tracker.
+     *
+     * @param slotStatusUpdateListener listener to register
+     */
+    void registerSlotStatusUpdateListener(SlotStatusUpdateListener slotStatusUpdateListener);
+
+    // ---------------------------------------------------------------------------------------------
+    // Add / Remove (pending) Resource
+    // ---------------------------------------------------------------------------------------------
+
+    /**
+     * Register a new task executor with its initial slot report.
+     *
+     * @param taskExecutorConnection of the new task executor
+     * @param initialSlotReport of the new task executor
+     * @param totalResourceProfile of the new task executor
+     * @param defaultSlotResourceProfile of the new task executor
+     */
+    void addTaskExecutor(
+            TaskExecutorConnection taskExecutorConnection,
+            SlotReport initialSlotReport,
+            ResourceProfile totalResourceProfile,
+            ResourceProfile defaultSlotResourceProfile);
+
+    /**
+     * Add a new pending task executor with its resource profile.
+     *
+     * @param totalResourceProfile of the pending task executor
+     * @param defaultSlotResourceProfile of the pending task executor
+     */
+    void addPendingTaskExecutor(
+            ResourceProfile totalResourceProfile, ResourceProfile defaultSlotResourceProfile);
+
+    /**
+     * Unregister a task executor with the given instance id.
+     *
+     * @param instanceId of the task executor
+     */
+    void removeTaskExecutor(InstanceID instanceId);
+
+    // ---------------------------------------------------------------------------------------------
+    // Slot status updates
+    // ---------------------------------------------------------------------------------------------
+
+    /**
+     * Find a task executor to fulfill the given resource requirement.
+     *
+     * @param resourceProfile of the given requirement
+     * @param taskExecutorMatchingStrategy to use
+     * @return An Optional of {@link TaskExecutorConnection}, if find, of the matching task
+     *     executor.
+     */
+    Optional<TaskExecutorConnection> findTaskExecutorToFulfill(
+            ResourceProfile resourceProfile,
+            TaskExecutorMatchingStrategy taskExecutorMatchingStrategy);
+
+    /**
+     * Notify the allocation of the given slot is started.
+     *
+     * @param allocationId of the allocated slot
+     * @param jobId of the allocated slot
+     * @param instanceId of the located task executor
+     * @param requiredResource of the allocated slot
+     */
+    void notifyAllocationStart(
+            AllocationID allocationId,
+            JobID jobId,
+            InstanceID instanceId,
+            ResourceProfile requiredResource);
+
+    /**
+     * Notify the allocation of the given slot is completed.
+     *
+     * @param instanceId of the located task executor
+     * @param allocationId of the allocated slot
+     */
+    void notifyAllocationComplete(InstanceID instanceId, AllocationID allocationId);
+
+    /**
+     * Notify the given allocated slot is free.
+     *
+     * @param allocationId of the allocated slot
+     */
+    void notifyFree(AllocationID allocationId);
+
+    /**
+     * Notifies the tracker about the slot statuses.
+     *
+     * @param slotReport slot report
+     * @param instanceId of the task executor
+     */
+    void notifySlotStatus(SlotReport slotReport, InstanceID instanceId);
+
+    // ---------------------------------------------------------------------------------------------
+    // Utility method
+    // ---------------------------------------------------------------------------------------------
+
+    /**
+     * Get the instance ids of all registered task executors.
+     *
+     * @return a set of instance ids of all registered task executors.
+     */
+    Set<InstanceID> getTaskExecutors();
+
+    /**
+     * Check if there is a registered task executor with the given instance id.
+     *
+     * @param instanceId of the task executor
+     * @return whether there is a registered task executor with the given instance id
+     */
+    boolean isTaskExecutorRegistered(InstanceID instanceId);
+
+    /**
+     * Find an exactly matching pending task executor with the given resource profile.
+     *
+     * @param initialSlotReport of the task executor
+     * @param totalResourceProfile of the task executor
+     * @param defaultResourceProfile of the task executor
+     * @return An Optional of {@link PendingTaskManager}, if find, of the matching pending task
+     *     executor.
+     */
+    Optional<PendingTaskManager> findMatchingPendingTaskManager(
+            SlotReport initialSlotReport,
+            ResourceProfile totalResourceProfile,
+            ResourceProfile defaultResourceProfile);
+
+    // ---------------------------------------------------------------------------------------------
+    // TaskExecutor idleness / redundancy
+    // ---------------------------------------------------------------------------------------------
+
+    /**
+     * Get all task executors which idle exceed the given timeout period
+     *
+     * @param taskManagerTimeout timeout period
+     * @return a map of timeout task executors' connection index by the instance id
+     */
+    Map<InstanceID, TaskExecutorConnection> getTimeOutTaskExecutors(Time taskManagerTimeout);

Review comment:
       This method should not belong to this interface.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/TaskExecutorAllocationStrategy.java
##########
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.runtime.slots.ResourceCounter;
+
+import java.util.Collections;
+import java.util.Map;
+
+/** Strategy how to allocate new task executors to fulfill the unfulfilled requirements. */
+public interface TaskExecutorAllocationStrategy {
+    TaskExecutorAllocationStrategy NO_OP_STRATEGY =
+            (requirements, existingPendingResources) -> Collections.emptyMap();
+
+    /**
+     * Calculate {@link PendingTaskManager}s needed to fulfill the given requirements.
+     *
+     * @param requirements requirements indexed by jobId
+     * @param existingPendingResources existing pending resources can be used to fulfill requirement
+     * @return {@link PendingTaskManager}s needed and whether all the requirements can be fulfilled
+     *     after allocation of pending task executors, indexed by jobId
+     */
+    Map<JobID, Tuple2<Map<PendingTaskManager, Integer>, Boolean>> getTaskExecutorsToFulfill(

Review comment:
       How do we guarantee `TaskExecutorMatchingStrategy` matches slot to pending TMs in the same way `TaskExecutorAllocationStrategy` calculates needed TMs?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerUtils.java
##########
@@ -0,0 +1,36 @@
+/*

Review comment:
       The commit message is a bit misleading.
   It sounds like a new util class is implemented, which should not be a hotfix, while actually the commit only moves some existing codes for deduplication.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/AnyMatchingTaskExecutorMatchingStrategy.java
##########
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.instance.InstanceID;
+
+import java.util.Map;
+import java.util.Optional;
+
+/** {@link TaskExecutorMatchingStrategy} which picks the first matching task executor. */
+public enum AnyMatchingTaskExecutorMatchingStrategy implements TaskExecutorMatchingStrategy {
+    INSTANCE;
+
+    @Override
+    public Optional<InstanceID> findMatchingTaskExecutor(
+            ResourceProfile requirement,
+            Map<InstanceID, ? extends TaskExecutorInfo> taskExecutors) {
+        return taskExecutors.entrySet().stream()
+                .filter(taskExecutor -> canFulfillRequirement(requirement, taskExecutor.getValue()))
+                .findFirst()
+                .map(Map.Entry::getKey);
+    }
+
+    private boolean canFulfillRequirement(

Review comment:
       can be static

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/TaskExecutorTracker.java
##########
@@ -0,0 +1,274 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.common.time.Time;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.taskexecutor.SlotReport;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+
+/** Tracks TaskExecutor's resource and slot status. */
+interface TaskExecutorTracker {

Review comment:
       My gut feeling is that, this interface contains too many methods, and not all of them are closely related. They could be further grouped and structured.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/TaskManagerSlotInformation.java
##########
@@ -28,6 +30,10 @@
 
     SlotID getSlotId();
 
+    AllocationID getAllocationId();
+
+    JobID getJobId();

Review comment:
       These two should be marked `@Nullable`

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/AnyMatchingTaskExecutorMatchingStrategyTest.java
##########
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.is;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+/** Tests for the {@link AnyMatchingTaskExecutorMatchingStrategy}. */
+public class AnyMatchingTaskExecutorMatchingStrategyTest extends TestLogger {
+    private final InstanceID instanceIdOfLargeTaskExecutor = new InstanceID();
+    private final InstanceID instanceIdOfSmallTaskExecutor = new InstanceID();
+    private final ResourceProfile largeResourceProfile = ResourceProfile.fromResources(10.2, 42);
+    private final ResourceProfile smallResourceProfile = ResourceProfile.fromResources(1, 1);

Review comment:
       static

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/TaskExecutorTracker.java
##########
@@ -0,0 +1,274 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.common.time.Time;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.taskexecutor.SlotReport;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+
+/** Tracks TaskExecutor's resource and slot status. */
+interface TaskExecutorTracker {
+
+    /**
+     * Registers the given listener with this tracker.
+     *
+     * @param slotStatusUpdateListener listener to register
+     */
+    void registerSlotStatusUpdateListener(SlotStatusUpdateListener slotStatusUpdateListener);
+
+    // ---------------------------------------------------------------------------------------------
+    // Add / Remove (pending) Resource
+    // ---------------------------------------------------------------------------------------------
+
+    /**
+     * Register a new task executor with its initial slot report.
+     *
+     * @param taskExecutorConnection of the new task executor
+     * @param initialSlotReport of the new task executor
+     * @param totalResourceProfile of the new task executor
+     * @param defaultSlotResourceProfile of the new task executor
+     */
+    void addTaskExecutor(
+            TaskExecutorConnection taskExecutorConnection,
+            SlotReport initialSlotReport,
+            ResourceProfile totalResourceProfile,
+            ResourceProfile defaultSlotResourceProfile);
+
+    /**
+     * Add a new pending task executor with its resource profile.
+     *
+     * @param totalResourceProfile of the pending task executor
+     * @param defaultSlotResourceProfile of the pending task executor
+     */
+    void addPendingTaskExecutor(
+            ResourceProfile totalResourceProfile, ResourceProfile defaultSlotResourceProfile);
+
+    /**
+     * Unregister a task executor with the given instance id.
+     *
+     * @param instanceId of the task executor
+     */
+    void removeTaskExecutor(InstanceID instanceId);
+
+    // ---------------------------------------------------------------------------------------------
+    // Slot status updates
+    // ---------------------------------------------------------------------------------------------
+
+    /**
+     * Find a task executor to fulfill the given resource requirement.
+     *
+     * @param resourceProfile of the given requirement
+     * @param taskExecutorMatchingStrategy to use
+     * @return An Optional of {@link TaskExecutorConnection}, if find, of the matching task
+     *     executor.
+     */
+    Optional<TaskExecutorConnection> findTaskExecutorToFulfill(

Review comment:
       This method should not belong to this interface.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorResourceUtils.java
##########
@@ -134,11 +134,13 @@ public static TaskExecutorResourceSpec resourceSpecFromConfigForLocalExecution(
     public static Configuration adjustForLocalExecution(Configuration config) {
         UNUSED_CONFIG_OPTIONS.forEach(option -> warnOptionHasNoEffectIfSet(config, option));
 
-        setConfigOptionToPassedMaxIfNotSet(config, TaskManagerOptions.CPU_CORES, Double.MAX_VALUE);
+        setConfigOptionToPassedMaxIfNotSet(config, TaskManagerOptions.CPU_CORES, 1000000.0);
         setConfigOptionToPassedMaxIfNotSet(
-                config, TaskManagerOptions.TASK_HEAP_MEMORY, MemorySize.MAX_VALUE);
+                config, TaskManagerOptions.TASK_HEAP_MEMORY, MemorySize.ofMebiBytes(1024 * 1024));
         setConfigOptionToPassedMaxIfNotSet(
-                config, TaskManagerOptions.TASK_OFF_HEAP_MEMORY, MemorySize.MAX_VALUE);
+                config,
+                TaskManagerOptions.TASK_OFF_HEAP_MEMORY,
+                MemorySize.ofMebiBytes(1024 * 1024));

Review comment:
       Let's replace the magic numbers with static constants.

##########
File path: flink-core/src/main/java/org/apache/flink/configuration/ClusterOptions.java
##########
@@ -111,6 +111,19 @@
                     .withDescription(
                             "Defines whether the cluster uses declarative resource management.");
 
+    @Documentation.ExcludeFromDocumentation
+    public static final ConfigOption<Boolean> ENABLE_FINE_GRAINED_RESOURCE_MANAGEMENT =
+            ConfigOptions.key("cluster.fine-grained-resource-management.enabled")
+                    .booleanType()
+                    .defaultValue(false)
+                    .withDescription(
+                            "Defines whether the cluster uses fine-grained resource management.");
+
+    public static boolean isFineGrainedResourceManagementEnabled(Configuration configuration) {
+        return isDeclarativeResourceManagementEnabled(configuration)

Review comment:
       IIUC, we need to bind fine-grained with declarative because in the first step we implement the feature base on the declarative protocol, which also requires declarative slot pool being used. And once FLINK-20838 is finished, we would be able to support both protocols and no longer need this binding.
   
   If this is the case, I would suggest to add a `TODO` here explaining why this is temporary needed and when should be removed.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/TaskExecutorTracker.java
##########
@@ -0,0 +1,274 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.common.time.Time;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.taskexecutor.SlotReport;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+
+/** Tracks TaskExecutor's resource and slot status. */
+interface TaskExecutorTracker {
+
+    /**
+     * Registers the given listener with this tracker.
+     *
+     * @param slotStatusUpdateListener listener to register
+     */
+    void registerSlotStatusUpdateListener(SlotStatusUpdateListener slotStatusUpdateListener);
+
+    // ---------------------------------------------------------------------------------------------
+    // Add / Remove (pending) Resource
+    // ---------------------------------------------------------------------------------------------
+
+    /**
+     * Register a new task executor with its initial slot report.
+     *
+     * @param taskExecutorConnection of the new task executor
+     * @param initialSlotReport of the new task executor
+     * @param totalResourceProfile of the new task executor
+     * @param defaultSlotResourceProfile of the new task executor
+     */
+    void addTaskExecutor(
+            TaskExecutorConnection taskExecutorConnection,
+            SlotReport initialSlotReport,
+            ResourceProfile totalResourceProfile,
+            ResourceProfile defaultSlotResourceProfile);
+
+    /**
+     * Add a new pending task executor with its resource profile.
+     *
+     * @param totalResourceProfile of the pending task executor
+     * @param defaultSlotResourceProfile of the pending task executor
+     */
+    void addPendingTaskExecutor(
+            ResourceProfile totalResourceProfile, ResourceProfile defaultSlotResourceProfile);
+
+    /**
+     * Unregister a task executor with the given instance id.
+     *
+     * @param instanceId of the task executor
+     */
+    void removeTaskExecutor(InstanceID instanceId);
+
+    // ---------------------------------------------------------------------------------------------
+    // Slot status updates
+    // ---------------------------------------------------------------------------------------------
+
+    /**
+     * Find a task executor to fulfill the given resource requirement.
+     *
+     * @param resourceProfile of the given requirement
+     * @param taskExecutorMatchingStrategy to use
+     * @return An Optional of {@link TaskExecutorConnection}, if find, of the matching task
+     *     executor.
+     */
+    Optional<TaskExecutorConnection> findTaskExecutorToFulfill(
+            ResourceProfile resourceProfile,
+            TaskExecutorMatchingStrategy taskExecutorMatchingStrategy);
+
+    /**
+     * Notify the allocation of the given slot is started.
+     *
+     * @param allocationId of the allocated slot
+     * @param jobId of the allocated slot
+     * @param instanceId of the located task executor
+     * @param requiredResource of the allocated slot
+     */
+    void notifyAllocationStart(
+            AllocationID allocationId,
+            JobID jobId,
+            InstanceID instanceId,
+            ResourceProfile requiredResource);
+
+    /**
+     * Notify the allocation of the given slot is completed.
+     *
+     * @param instanceId of the located task executor
+     * @param allocationId of the allocated slot
+     */
+    void notifyAllocationComplete(InstanceID instanceId, AllocationID allocationId);
+
+    /**
+     * Notify the given allocated slot is free.
+     *
+     * @param allocationId of the allocated slot
+     */
+    void notifyFree(AllocationID allocationId);
+
+    /**
+     * Notifies the tracker about the slot statuses.
+     *
+     * @param slotReport slot report
+     * @param instanceId of the task executor
+     */
+    void notifySlotStatus(SlotReport slotReport, InstanceID instanceId);
+
+    // ---------------------------------------------------------------------------------------------
+    // Utility method
+    // ---------------------------------------------------------------------------------------------
+
+    /**
+     * Get the instance ids of all registered task executors.
+     *
+     * @return a set of instance ids of all registered task executors.
+     */
+    Set<InstanceID> getTaskExecutors();
+
+    /**
+     * Check if there is a registered task executor with the given instance id.
+     *
+     * @param instanceId of the task executor
+     * @return whether there is a registered task executor with the given instance id
+     */
+    boolean isTaskExecutorRegistered(InstanceID instanceId);
+
+    /**
+     * Find an exactly matching pending task executor with the given resource profile.
+     *
+     * @param initialSlotReport of the task executor
+     * @param totalResourceProfile of the task executor
+     * @param defaultResourceProfile of the task executor
+     * @return An Optional of {@link PendingTaskManager}, if find, of the matching pending task
+     *     executor.
+     */
+    Optional<PendingTaskManager> findMatchingPendingTaskManager(
+            SlotReport initialSlotReport,
+            ResourceProfile totalResourceProfile,
+            ResourceProfile defaultResourceProfile);
+
+    // ---------------------------------------------------------------------------------------------
+    // TaskExecutor idleness / redundancy
+    // ---------------------------------------------------------------------------------------------
+
+    /**
+     * Get all task executors which idle exceed the given timeout period
+     *
+     * @param taskManagerTimeout timeout period
+     * @return a map of timeout task executors' connection index by the instance id
+     */
+    Map<InstanceID, TaskExecutorConnection> getTimeOutTaskExecutors(Time taskManagerTimeout);
+
+    /**
+     * Get the start time of idleness of the task executor with the given instance id.
+     *
+     * @param instanceId of the task executor
+     * @return the start time of idleness
+     */
+    long getTaskExecutorIdleSince(InstanceID instanceId);
+
+    // ---------------------------------------------------------------------------------------------
+    // slot / resource counts
+    // ---------------------------------------------------------------------------------------------
+
+    /**
+     * Get the total registered resources.
+     *
+     * @return the total registered resources
+     */
+    ResourceProfile getTotalRegisteredResources();
+
+    /**
+     * Get the total registered resources of the given task executor
+     *
+     * @param instanceId of the task executor
+     * @return the total registered resources of the given task executor
+     */
+    ResourceProfile getTotalRegisteredResourcesOf(InstanceID instanceId);

Review comment:
       The following methods can be aggregated to something like `TMStatus getTMStatus(InstanceID instanceId)`, where `TMStatus` (maybe another name) is a data structure containing all information needed.
   - ResourceProfile getTotalRegisteredResourcesOf(InstanceID instanceId)
   - ResourceProfile getTotalFreeResourcesOf(InstanceID instanceId)
   - int getNumberRegisteredSlotsOf(InstanceID instanceId)
   - int getNumberFreeSlotsOf(InstanceID instanceId)
   - long getTaskExecutorIdleSince(InstanceID instanceId)

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/TestingTaskExecutorInfo.java
##########
@@ -0,0 +1,100 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+
+public class TestingTaskExecutorInfo implements TaskExecutorInfo {
+    private final ResourceProfile totalResource;
+    private final ResourceProfile availableResource;
+    private final ResourceProfile defaultSlotResourceProfile;
+    private final int numberAllocatedSlots;
+
+    private TestingTaskExecutorInfo(
+            ResourceProfile totalResource,
+            ResourceProfile availableResource,
+            ResourceProfile defaultSlotResourceProfile,
+            int numberAllocatedSlots) {
+        this.totalResource = totalResource;
+        this.availableResource = availableResource;
+        this.defaultSlotResourceProfile = defaultSlotResourceProfile;
+        this.numberAllocatedSlots = numberAllocatedSlots;
+    }
+
+    @Override
+    public ResourceProfile getTotalResource() {
+        return totalResource;
+    }
+
+    @Override
+    public ResourceProfile getDefaultSlotResourceProfile() {
+        return defaultSlotResourceProfile;
+    }
+
+    @Override
+    public ResourceProfile getAvailableResource() {
+        return availableResource;
+    }
+
+    @Override
+    public int getNumberAllocatedSlots() {
+        return numberAllocatedSlots;
+    }
+
+    public static Builder newBuilder() {
+        return new Builder();
+    }
+
+    public static class Builder {
+        private ResourceProfile totalResource = ResourceProfile.ANY;
+        private ResourceProfile availableResource = ResourceProfile.ANY;
+        private ResourceProfile defaultSlotResourceProfile = ResourceProfile.ANY;
+        private int numberAllocatedSlots = 0;
+
+        private Builder() {};
+
+        public Builder withTotalResource(ResourceProfile totalResource) {
+            this.totalResource = totalResource;
+            return this;
+        }
+
+        public Builder withAvailableResource(ResourceProfile availableResource) {
+            this.availableResource = availableResource;
+            return this;
+        }
+
+        public Builder withDefaultSlotResourceProfile(ResourceProfile defaultSlotResourceProfile) {
+            this.defaultSlotResourceProfile = defaultSlotResourceProfile;
+            return this;
+        }
+
+        public Builder withNumberAllocatedSlots(int numberAllocatedSlots) {
+            this.numberAllocatedSlots = numberAllocatedSlots;
+            return this;
+        }

Review comment:
       Better to add sanity checks for the setters:
   - non-null
   - available <= total
   - default <= total

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotStatusUpdateListener.java
##########
@@ -42,4 +45,24 @@
      */
     void notifySlotStatusChange(
             TaskManagerSlotInformation slot, SlotState previous, SlotState current, JobID jobId);
+
+    class MultiSlotStatusUpdateListener implements SlotStatusUpdateListener {

Review comment:
       IIUC, this change is to allow `MultiSlotStatusUpdateListener` to be reused by other classes than `DefaultSlotTracker`.
   Then I would suggest to make `MultiSlotStatusUpdateListener` a separate file. An interface does not normally have an inner class that implements itself. 

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/TaskExecutorMatchingStrategy.java
##########
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.instance.InstanceID;
+
+import java.util.Map;
+import java.util.Optional;
+
+/** Strategy how to find a matching task executor. */
+public interface TaskExecutorMatchingStrategy {
+
+    /**
+     * Finds a matching task executor for the requested {@link ResourceProfile} given the collection
+     * of task executors.
+     *
+     * @param requirement to find a matching task executor for
+     * @param taskExecutors candidates
+     * @return Returns the instance id of matching task executor or {@link Optional#empty()} if
+     *     there is none
+     */
+    Optional<InstanceID> findMatchingTaskExecutor(
+            ResourceProfile requirement, Map<InstanceID, ? extends TaskExecutorInfo> taskExecutors);

Review comment:
       Why do we need a map for this interface? We should not need to lookup a `TaskExecutorInfo` by its `InstanceID` in the strategy.
   It might be better to include `InstanceID` in `TaskExecutorInfo` and pass in a list or collection.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/AnyMatchingTaskExecutorMatchingStrategyTest.java
##########
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.is;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+/** Tests for the {@link AnyMatchingTaskExecutorMatchingStrategy}. */
+public class AnyMatchingTaskExecutorMatchingStrategyTest extends TestLogger {
+    private final InstanceID instanceIdOfLargeTaskExecutor = new InstanceID();
+    private final InstanceID instanceIdOfSmallTaskExecutor = new InstanceID();
+    private final ResourceProfile largeResourceProfile = ResourceProfile.fromResources(10.2, 42);
+    private final ResourceProfile smallResourceProfile = ResourceProfile.fromResources(1, 1);
+    private Map<InstanceID, TaskExecutorInfo> taskExecutors = null;
+    private Set<InstanceID> candidates = null;
+
+    @Before
+    public void setup() {
+        taskExecutors = new HashMap<>();
+        candidates = new HashSet<>();
+        taskExecutors.put(
+                instanceIdOfSmallTaskExecutor,
+                TestingTaskExecutorInfo.newBuilder()
+                        .withAvailableResource(smallResourceProfile)
+                        .build());
+        taskExecutors.put(
+                instanceIdOfLargeTaskExecutor,
+                TestingTaskExecutorInfo.newBuilder()
+                        .withAvailableResource(largeResourceProfile)
+                        .build());

Review comment:
       Why do we need this `setup()`? We can also make `taskExecutors` a constant.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/DefaultSlotTracker.java
##########
@@ -278,30 +277,4 @@ public void executeStateTransition(DeclarativeTaskManagerSlot slot, JobID jobId)
             }
         }
     }
-
-    private static class MultiSlotStatusUpdateListener implements SlotStatusUpdateListener {

Review comment:
       This should also be a hotfix commit.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/AnyMatchingTaskExecutorMatchingStrategy.java
##########
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.instance.InstanceID;
+
+import java.util.Map;
+import java.util.Optional;
+
+/** {@link TaskExecutorMatchingStrategy} which picks the first matching task executor. */
+public enum AnyMatchingTaskExecutorMatchingStrategy implements TaskExecutorMatchingStrategy {
+    INSTANCE;
+
+    @Override
+    public Optional<InstanceID> findMatchingTaskExecutor(
+            ResourceProfile requirement,
+            Map<InstanceID, ? extends TaskExecutorInfo> taskExecutors) {
+        return taskExecutors.entrySet().stream()
+                .filter(taskExecutor -> canFulfillRequirement(requirement, taskExecutor.getValue()))
+                .findFirst()

Review comment:
       `findFirst` -> `findAny`

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/LeastAllocateSlotsTaskExecutorMatchingStrategy.java
##########
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.instance.InstanceID;
+
+import java.util.Comparator;
+import java.util.Map;
+import java.util.Optional;
+
+/**
+ * {@link TaskExecutorMatchingStrategy} which picks a matching TaskExecutor with the least number of
+ * allocated slots.
+ */
+public enum LeastAllocateSlotsTaskExecutorMatchingStrategy implements TaskExecutorMatchingStrategy {

Review comment:
       I believe this is for the configuration option `cluster.evenly-spread-out-slots`.
   
   However, I'm not sure how we want this feature for the fine-grained resource management, where amount of resource plays a more important role than number of slots.
   
   This may need to revisit how this feature should be exposed, to be compatible for both coarse/fine-grained resource management. Given that this PR is already quite big, I would suggest to not introduce this strategy in this PR, but create a follow-up issue instead.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/AnyMatchingTaskExecutorMatchingStrategyTest.java
##########
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.is;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+/** Tests for the {@link AnyMatchingTaskExecutorMatchingStrategy}. */
+public class AnyMatchingTaskExecutorMatchingStrategyTest extends TestLogger {
+    private final InstanceID instanceIdOfLargeTaskExecutor = new InstanceID();
+    private final InstanceID instanceIdOfSmallTaskExecutor = new InstanceID();
+    private final ResourceProfile largeResourceProfile = ResourceProfile.fromResources(10.2, 42);
+    private final ResourceProfile smallResourceProfile = ResourceProfile.fromResources(1, 1);
+    private Map<InstanceID, TaskExecutorInfo> taskExecutors = null;
+    private Set<InstanceID> candidates = null;

Review comment:
       Never used.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorResourceUtils.java
##########
@@ -134,11 +134,13 @@ public static TaskExecutorResourceSpec resourceSpecFromConfigForLocalExecution(
     public static Configuration adjustForLocalExecution(Configuration config) {
         UNUSED_CONFIG_OPTIONS.forEach(option -> warnOptionHasNoEffectIfSet(config, option));
 
-        setConfigOptionToPassedMaxIfNotSet(config, TaskManagerOptions.CPU_CORES, Double.MAX_VALUE);
+        setConfigOptionToPassedMaxIfNotSet(config, TaskManagerOptions.CPU_CORES, 1000000.0);
         setConfigOptionToPassedMaxIfNotSet(
-                config, TaskManagerOptions.TASK_HEAP_MEMORY, MemorySize.MAX_VALUE);
+                config, TaskManagerOptions.TASK_HEAP_MEMORY, MemorySize.ofMebiBytes(1024 * 1024));
         setConfigOptionToPassedMaxIfNotSet(
-                config, TaskManagerOptions.TASK_OFF_HEAP_MEMORY, MemorySize.MAX_VALUE);
+                config,
+                TaskManagerOptions.TASK_OFF_HEAP_MEMORY,
+                MemorySize.ofMebiBytes(1024 * 1024));

Review comment:
       And it would be helpful to explain the purpose of this change in the commit message.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/TaskExecutorTracker.java
##########
@@ -0,0 +1,274 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.common.time.Time;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.taskexecutor.SlotReport;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+
+/** Tracks TaskExecutor's resource and slot status. */
+interface TaskExecutorTracker {
+
+    /**
+     * Registers the given listener with this tracker.
+     *
+     * @param slotStatusUpdateListener listener to register
+     */
+    void registerSlotStatusUpdateListener(SlotStatusUpdateListener slotStatusUpdateListener);
+
+    // ---------------------------------------------------------------------------------------------
+    // Add / Remove (pending) Resource
+    // ---------------------------------------------------------------------------------------------
+
+    /**
+     * Register a new task executor with its initial slot report.
+     *
+     * @param taskExecutorConnection of the new task executor
+     * @param initialSlotReport of the new task executor
+     * @param totalResourceProfile of the new task executor
+     * @param defaultSlotResourceProfile of the new task executor
+     */
+    void addTaskExecutor(
+            TaskExecutorConnection taskExecutorConnection,
+            SlotReport initialSlotReport,
+            ResourceProfile totalResourceProfile,
+            ResourceProfile defaultSlotResourceProfile);
+
+    /**
+     * Add a new pending task executor with its resource profile.
+     *
+     * @param totalResourceProfile of the pending task executor
+     * @param defaultSlotResourceProfile of the pending task executor
+     */
+    void addPendingTaskExecutor(
+            ResourceProfile totalResourceProfile, ResourceProfile defaultSlotResourceProfile);
+
+    /**
+     * Unregister a task executor with the given instance id.
+     *
+     * @param instanceId of the task executor
+     */
+    void removeTaskExecutor(InstanceID instanceId);
+
+    // ---------------------------------------------------------------------------------------------
+    // Slot status updates
+    // ---------------------------------------------------------------------------------------------
+
+    /**
+     * Find a task executor to fulfill the given resource requirement.
+     *
+     * @param resourceProfile of the given requirement
+     * @param taskExecutorMatchingStrategy to use
+     * @return An Optional of {@link TaskExecutorConnection}, if find, of the matching task
+     *     executor.
+     */
+    Optional<TaskExecutorConnection> findTaskExecutorToFulfill(
+            ResourceProfile resourceProfile,
+            TaskExecutorMatchingStrategy taskExecutorMatchingStrategy);
+
+    /**
+     * Notify the allocation of the given slot is started.
+     *
+     * @param allocationId of the allocated slot
+     * @param jobId of the allocated slot
+     * @param instanceId of the located task executor
+     * @param requiredResource of the allocated slot
+     */
+    void notifyAllocationStart(
+            AllocationID allocationId,
+            JobID jobId,
+            InstanceID instanceId,
+            ResourceProfile requiredResource);
+
+    /**
+     * Notify the allocation of the given slot is completed.
+     *
+     * @param instanceId of the located task executor
+     * @param allocationId of the allocated slot
+     */
+    void notifyAllocationComplete(InstanceID instanceId, AllocationID allocationId);
+
+    /**
+     * Notify the given allocated slot is free.
+     *
+     * @param allocationId of the allocated slot
+     */
+    void notifyFree(AllocationID allocationId);
+
+    /**
+     * Notifies the tracker about the slot statuses.
+     *
+     * @param slotReport slot report
+     * @param instanceId of the task executor
+     */
+    void notifySlotStatus(SlotReport slotReport, InstanceID instanceId);
+
+    // ---------------------------------------------------------------------------------------------
+    // Utility method
+    // ---------------------------------------------------------------------------------------------
+
+    /**
+     * Get the instance ids of all registered task executors.
+     *
+     * @return a set of instance ids of all registered task executors.
+     */
+    Set<InstanceID> getTaskExecutors();
+
+    /**
+     * Check if there is a registered task executor with the given instance id.
+     *
+     * @param instanceId of the task executor
+     * @return whether there is a registered task executor with the given instance id
+     */
+    boolean isTaskExecutorRegistered(InstanceID instanceId);
+
+    /**
+     * Find an exactly matching pending task executor with the given resource profile.
+     *
+     * @param initialSlotReport of the task executor
+     * @param totalResourceProfile of the task executor
+     * @param defaultResourceProfile of the task executor
+     * @return An Optional of {@link PendingTaskManager}, if find, of the matching pending task
+     *     executor.
+     */
+    Optional<PendingTaskManager> findMatchingPendingTaskManager(

Review comment:
       This method should not belong to this interface.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/WorkerResourceSpec.java
##########
@@ -70,6 +71,16 @@ public static WorkerResourceSpec fromTaskExecutorProcessSpec(
                 taskExecutorProcessSpec.getManagedMemorySize());
     }
 
+    public static WorkerResourceSpec fromResourceProfile(final ResourceProfile resourceProfile) {

Review comment:
       Again, should explain the purpose of this change in the commit message?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/TaskExecutorMatchingStrategy.java
##########
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.instance.InstanceID;
+
+import java.util.Map;
+import java.util.Optional;
+
+/** Strategy how to find a matching task executor. */
+public interface TaskExecutorMatchingStrategy {

Review comment:
       I would suggest `SlotTaskExecutorMatchingStrategy` to suggest that the strategy is for matching between a slot and a TM rather than two TMs.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/AnyMatchingTaskExecutorMatchingStrategy.java
##########
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.instance.InstanceID;
+
+import java.util.Map;
+import java.util.Optional;
+
+/** {@link TaskExecutorMatchingStrategy} which picks the first matching task executor. */
+public enum AnyMatchingTaskExecutorMatchingStrategy implements TaskExecutorMatchingStrategy {

Review comment:
       JavaDoc: first -> any

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/TaskExecutorTracker.java
##########
@@ -0,0 +1,274 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.common.time.Time;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.taskexecutor.SlotReport;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+
+/** Tracks TaskExecutor's resource and slot status. */
+interface TaskExecutorTracker {
+
+    /**
+     * Registers the given listener with this tracker.
+     *
+     * @param slotStatusUpdateListener listener to register
+     */
+    void registerSlotStatusUpdateListener(SlotStatusUpdateListener slotStatusUpdateListener);
+
+    // ---------------------------------------------------------------------------------------------
+    // Add / Remove (pending) Resource
+    // ---------------------------------------------------------------------------------------------
+
+    /**
+     * Register a new task executor with its initial slot report.
+     *
+     * @param taskExecutorConnection of the new task executor
+     * @param initialSlotReport of the new task executor
+     * @param totalResourceProfile of the new task executor
+     * @param defaultSlotResourceProfile of the new task executor
+     */
+    void addTaskExecutor(
+            TaskExecutorConnection taskExecutorConnection,
+            SlotReport initialSlotReport,
+            ResourceProfile totalResourceProfile,
+            ResourceProfile defaultSlotResourceProfile);
+
+    /**
+     * Add a new pending task executor with its resource profile.
+     *
+     * @param totalResourceProfile of the pending task executor
+     * @param defaultSlotResourceProfile of the pending task executor
+     */
+    void addPendingTaskExecutor(
+            ResourceProfile totalResourceProfile, ResourceProfile defaultSlotResourceProfile);
+
+    /**
+     * Unregister a task executor with the given instance id.
+     *
+     * @param instanceId of the task executor
+     */
+    void removeTaskExecutor(InstanceID instanceId);
+
+    // ---------------------------------------------------------------------------------------------
+    // Slot status updates
+    // ---------------------------------------------------------------------------------------------
+
+    /**
+     * Find a task executor to fulfill the given resource requirement.
+     *
+     * @param resourceProfile of the given requirement
+     * @param taskExecutorMatchingStrategy to use
+     * @return An Optional of {@link TaskExecutorConnection}, if find, of the matching task
+     *     executor.
+     */
+    Optional<TaskExecutorConnection> findTaskExecutorToFulfill(
+            ResourceProfile resourceProfile,
+            TaskExecutorMatchingStrategy taskExecutorMatchingStrategy);
+
+    /**
+     * Notify the allocation of the given slot is started.
+     *
+     * @param allocationId of the allocated slot
+     * @param jobId of the allocated slot
+     * @param instanceId of the located task executor
+     * @param requiredResource of the allocated slot
+     */
+    void notifyAllocationStart(
+            AllocationID allocationId,
+            JobID jobId,
+            InstanceID instanceId,
+            ResourceProfile requiredResource);
+
+    /**
+     * Notify the allocation of the given slot is completed.
+     *
+     * @param instanceId of the located task executor
+     * @param allocationId of the allocated slot
+     */
+    void notifyAllocationComplete(InstanceID instanceId, AllocationID allocationId);
+
+    /**
+     * Notify the given allocated slot is free.
+     *
+     * @param allocationId of the allocated slot
+     */
+    void notifyFree(AllocationID allocationId);
+
+    /**
+     * Notifies the tracker about the slot statuses.
+     *
+     * @param slotReport slot report
+     * @param instanceId of the task executor
+     */
+    void notifySlotStatus(SlotReport slotReport, InstanceID instanceId);

Review comment:
       The slot status synchronization logics are a bit too complicate to be included in this interface. Might be better to have a dedicated slot status synchronizer. The synchronizer can take slot manager actions (allocate, free), received slot reports, and the current status as inputs, and notify the `TaskExecutorTracker` and `SlotStatusUpdateListener` as output. 

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/WorkerResourceSpec.java
##########
@@ -70,6 +71,16 @@ public static WorkerResourceSpec fromTaskExecutorProcessSpec(
                 taskExecutorProcessSpec.getManagedMemorySize());
     }
 
+    public static WorkerResourceSpec fromResourceProfile(final ResourceProfile resourceProfile) {

Review comment:
       To avoid confusion against slot resource profile.
   ```suggestion
       public static WorkerResourceSpec fromTotalResourceProfile(final ResourceProfile totalResourceProfile) {
   ```

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/TaskExecutorTracker.java
##########
@@ -0,0 +1,274 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.common.time.Time;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.taskexecutor.SlotReport;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+
+/** Tracks TaskExecutor's resource and slot status. */
+interface TaskExecutorTracker {
+
+    /**
+     * Registers the given listener with this tracker.
+     *
+     * @param slotStatusUpdateListener listener to register
+     */
+    void registerSlotStatusUpdateListener(SlotStatusUpdateListener slotStatusUpdateListener);
+
+    // ---------------------------------------------------------------------------------------------
+    // Add / Remove (pending) Resource
+    // ---------------------------------------------------------------------------------------------
+
+    /**
+     * Register a new task executor with its initial slot report.
+     *
+     * @param taskExecutorConnection of the new task executor
+     * @param initialSlotReport of the new task executor
+     * @param totalResourceProfile of the new task executor
+     * @param defaultSlotResourceProfile of the new task executor
+     */
+    void addTaskExecutor(
+            TaskExecutorConnection taskExecutorConnection,
+            SlotReport initialSlotReport,
+            ResourceProfile totalResourceProfile,
+            ResourceProfile defaultSlotResourceProfile);
+
+    /**
+     * Add a new pending task executor with its resource profile.
+     *
+     * @param totalResourceProfile of the pending task executor
+     * @param defaultSlotResourceProfile of the pending task executor
+     */
+    void addPendingTaskExecutor(
+            ResourceProfile totalResourceProfile, ResourceProfile defaultSlotResourceProfile);
+
+    /**
+     * Unregister a task executor with the given instance id.
+     *
+     * @param instanceId of the task executor
+     */
+    void removeTaskExecutor(InstanceID instanceId);
+
+    // ---------------------------------------------------------------------------------------------
+    // Slot status updates
+    // ---------------------------------------------------------------------------------------------
+
+    /**
+     * Find a task executor to fulfill the given resource requirement.
+     *
+     * @param resourceProfile of the given requirement
+     * @param taskExecutorMatchingStrategy to use
+     * @return An Optional of {@link TaskExecutorConnection}, if find, of the matching task
+     *     executor.
+     */
+    Optional<TaskExecutorConnection> findTaskExecutorToFulfill(
+            ResourceProfile resourceProfile,
+            TaskExecutorMatchingStrategy taskExecutorMatchingStrategy);
+
+    /**
+     * Notify the allocation of the given slot is started.
+     *
+     * @param allocationId of the allocated slot
+     * @param jobId of the allocated slot
+     * @param instanceId of the located task executor
+     * @param requiredResource of the allocated slot
+     */
+    void notifyAllocationStart(
+            AllocationID allocationId,
+            JobID jobId,
+            InstanceID instanceId,
+            ResourceProfile requiredResource);
+
+    /**
+     * Notify the allocation of the given slot is completed.
+     *
+     * @param instanceId of the located task executor
+     * @param allocationId of the allocated slot
+     */
+    void notifyAllocationComplete(InstanceID instanceId, AllocationID allocationId);
+
+    /**
+     * Notify the given allocated slot is free.
+     *
+     * @param allocationId of the allocated slot
+     */
+    void notifyFree(AllocationID allocationId);
+
+    /**
+     * Notifies the tracker about the slot statuses.
+     *
+     * @param slotReport slot report
+     * @param instanceId of the task executor
+     */
+    void notifySlotStatus(SlotReport slotReport, InstanceID instanceId);
+
+    // ---------------------------------------------------------------------------------------------
+    // Utility method
+    // ---------------------------------------------------------------------------------------------
+
+    /**
+     * Get the instance ids of all registered task executors.
+     *
+     * @return a set of instance ids of all registered task executors.
+     */
+    Set<InstanceID> getTaskExecutors();

Review comment:
       Maybe return a collection of `TMInfo` rather than only `InstanceID`s.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/TaskManagerSlotInformation.java
##########
@@ -28,6 +30,10 @@
 
     SlotID getSlotId();
 
+    AllocationID getAllocationId();
+
+    JobID getJobId();

Review comment:
       And this looks like a pure minor refactor. Shouldn't it be a hotfix commit?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/TaskExecutorTracker.java
##########
@@ -0,0 +1,274 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.common.time.Time;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection;
+import org.apache.flink.runtime.taskexecutor.SlotReport;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+
+/** Tracks TaskExecutor's resource and slot status. */
+interface TaskExecutorTracker {
+
+    /**
+     * Registers the given listener with this tracker.
+     *
+     * @param slotStatusUpdateListener listener to register
+     */
+    void registerSlotStatusUpdateListener(SlotStatusUpdateListener slotStatusUpdateListener);
+
+    // ---------------------------------------------------------------------------------------------
+    // Add / Remove (pending) Resource
+    // ---------------------------------------------------------------------------------------------
+
+    /**
+     * Register a new task executor with its initial slot report.
+     *
+     * @param taskExecutorConnection of the new task executor
+     * @param initialSlotReport of the new task executor
+     * @param totalResourceProfile of the new task executor
+     * @param defaultSlotResourceProfile of the new task executor
+     */
+    void addTaskExecutor(
+            TaskExecutorConnection taskExecutorConnection,
+            SlotReport initialSlotReport,
+            ResourceProfile totalResourceProfile,
+            ResourceProfile defaultSlotResourceProfile);
+
+    /**
+     * Add a new pending task executor with its resource profile.
+     *
+     * @param totalResourceProfile of the pending task executor
+     * @param defaultSlotResourceProfile of the pending task executor
+     */
+    void addPendingTaskExecutor(
+            ResourceProfile totalResourceProfile, ResourceProfile defaultSlotResourceProfile);
+
+    /**
+     * Unregister a task executor with the given instance id.
+     *
+     * @param instanceId of the task executor
+     */
+    void removeTaskExecutor(InstanceID instanceId);
+
+    // ---------------------------------------------------------------------------------------------
+    // Slot status updates
+    // ---------------------------------------------------------------------------------------------
+
+    /**
+     * Find a task executor to fulfill the given resource requirement.
+     *
+     * @param resourceProfile of the given requirement
+     * @param taskExecutorMatchingStrategy to use
+     * @return An Optional of {@link TaskExecutorConnection}, if find, of the matching task
+     *     executor.
+     */
+    Optional<TaskExecutorConnection> findTaskExecutorToFulfill(
+            ResourceProfile resourceProfile,
+            TaskExecutorMatchingStrategy taskExecutorMatchingStrategy);
+
+    /**
+     * Notify the allocation of the given slot is started.
+     *
+     * @param allocationId of the allocated slot
+     * @param jobId of the allocated slot
+     * @param instanceId of the located task executor
+     * @param requiredResource of the allocated slot
+     */
+    void notifyAllocationStart(
+            AllocationID allocationId,
+            JobID jobId,
+            InstanceID instanceId,
+            ResourceProfile requiredResource);
+
+    /**
+     * Notify the allocation of the given slot is completed.
+     *
+     * @param instanceId of the located task executor
+     * @param allocationId of the allocated slot
+     */
+    void notifyAllocationComplete(InstanceID instanceId, AllocationID allocationId);
+
+    /**
+     * Notify the given allocated slot is free.
+     *
+     * @param allocationId of the allocated slot
+     */
+    void notifyFree(AllocationID allocationId);
+
+    /**
+     * Notifies the tracker about the slot statuses.
+     *
+     * @param slotReport slot report
+     * @param instanceId of the task executor
+     */
+    void notifySlotStatus(SlotReport slotReport, InstanceID instanceId);
+
+    // ---------------------------------------------------------------------------------------------
+    // Utility method
+    // ---------------------------------------------------------------------------------------------
+
+    /**
+     * Get the instance ids of all registered task executors.
+     *
+     * @return a set of instance ids of all registered task executors.
+     */
+    Set<InstanceID> getTaskExecutors();
+
+    /**
+     * Check if there is a registered task executor with the given instance id.
+     *
+     * @param instanceId of the task executor
+     * @return whether there is a registered task executor with the given instance id
+     */
+    boolean isTaskExecutorRegistered(InstanceID instanceId);
+
+    /**
+     * Find an exactly matching pending task executor with the given resource profile.
+     *
+     * @param initialSlotReport of the task executor
+     * @param totalResourceProfile of the task executor
+     * @param defaultResourceProfile of the task executor
+     * @return An Optional of {@link PendingTaskManager}, if find, of the matching pending task
+     *     executor.
+     */
+    Optional<PendingTaskManager> findMatchingPendingTaskManager(
+            SlotReport initialSlotReport,
+            ResourceProfile totalResourceProfile,
+            ResourceProfile defaultResourceProfile);
+
+    // ---------------------------------------------------------------------------------------------
+    // TaskExecutor idleness / redundancy
+    // ---------------------------------------------------------------------------------------------
+
+    /**
+     * Get all task executors which idle exceed the given timeout period
+     *
+     * @param taskManagerTimeout timeout period
+     * @return a map of timeout task executors' connection index by the instance id
+     */
+    Map<InstanceID, TaskExecutorConnection> getTimeOutTaskExecutors(Time taskManagerTimeout);
+
+    /**
+     * Get the start time of idleness of the task executor with the given instance id.
+     *
+     * @param instanceId of the task executor
+     * @return the start time of idleness
+     */
+    long getTaskExecutorIdleSince(InstanceID instanceId);
+
+    // ---------------------------------------------------------------------------------------------
+    // slot / resource counts
+    // ---------------------------------------------------------------------------------------------
+
+    /**
+     * Get the total registered resources.
+     *
+     * @return the total registered resources
+     */
+    ResourceProfile getTotalRegisteredResources();

Review comment:
       The following methods can be aggregated to something like `StatusOverview getStatusOverview()`, where `StatusOverview` (maybe another name) is a data structure containing all information needed.
   - ResourceProfile getTotalRegisteredResources()
   - ResourceProfile getTotalFreeResources()
   - int getNumberRegisteredSlots()
   - int getNumberFreeSlots()
   - int getNumberFreeTaskExecutors()
   - int getNumberRegisteredTaskExecutors()
   - int getNumberPendingTaskExecutors();

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/TaskExecutorMatchingStrategy.java
##########
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.instance.InstanceID;
+
+import java.util.Map;
+import java.util.Optional;
+
+/** Strategy how to find a matching task executor. */
+public interface TaskExecutorMatchingStrategy {

Review comment:
       Moreover, the latest glossary no longer use the term `TaskExecutor`.
   It's not necessary to change the existing codes, but introducing new codes I would suggest to replace `TaskExecutor` with `TaskManager`.




----------------------------------------------------------------
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 #14647: [FLINK-20835] Implement FineGrainedSlotManager

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "1c605546a2ac429e1c475eea6f80a75d0326c7bf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12054",
       "triggerID" : "1c605546a2ac429e1c475eea6f80a75d0326c7bf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "559cd8d09cca8082e9ca4904d11b11443f4c959f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12226",
       "triggerID" : "559cd8d09cca8082e9ca4904d11b11443f4c959f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "05733d3630a9a72d04545e2e8aeefdf76b2e5cf1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12235",
       "triggerID" : "05733d3630a9a72d04545e2e8aeefdf76b2e5cf1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0a88531c02021ab6bc36c8eec483dac0ee91db97",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12540",
       "triggerID" : "0a88531c02021ab6bc36c8eec483dac0ee91db97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "45b2dc853d01aa726bdf054b3c2afa5b95a1d6fd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12543",
       "triggerID" : "45b2dc853d01aa726bdf054b3c2afa5b95a1d6fd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "80e3121418e741d9bebf3e849817a0c4225a86dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12587",
       "triggerID" : "80e3121418e741d9bebf3e849817a0c4225a86dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f5f9f44b437922f5e0edd0b1dd3097a61351dee",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12723",
       "triggerID" : "8f5f9f44b437922f5e0edd0b1dd3097a61351dee",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0b72f53fbfe19ef426572f78ac716521bc63ab64",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12769",
       "triggerID" : "0b72f53fbfe19ef426572f78ac716521bc63ab64",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 0b72f53fbfe19ef426572f78ac716521bc63ab64 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12769) 
   
   <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 #14647: [FLINK-20835] Implement FineGrainedSlotManager

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "1c605546a2ac429e1c475eea6f80a75d0326c7bf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12054",
       "triggerID" : "1c605546a2ac429e1c475eea6f80a75d0326c7bf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "559cd8d09cca8082e9ca4904d11b11443f4c959f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12226",
       "triggerID" : "559cd8d09cca8082e9ca4904d11b11443f4c959f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "05733d3630a9a72d04545e2e8aeefdf76b2e5cf1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12235",
       "triggerID" : "05733d3630a9a72d04545e2e8aeefdf76b2e5cf1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0a88531c02021ab6bc36c8eec483dac0ee91db97",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12540",
       "triggerID" : "0a88531c02021ab6bc36c8eec483dac0ee91db97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "45b2dc853d01aa726bdf054b3c2afa5b95a1d6fd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12543",
       "triggerID" : "45b2dc853d01aa726bdf054b3c2afa5b95a1d6fd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "80e3121418e741d9bebf3e849817a0c4225a86dd",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12587",
       "triggerID" : "80e3121418e741d9bebf3e849817a0c4225a86dd",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 80e3121418e741d9bebf3e849817a0c4225a86dd Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12587) 
   
   <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 #14647: [FLINK-20835] Implement FineGrainedSlotManager

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "1c605546a2ac429e1c475eea6f80a75d0326c7bf",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12054",
       "triggerID" : "1c605546a2ac429e1c475eea6f80a75d0326c7bf",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 1c605546a2ac429e1c475eea6f80a75d0326c7bf Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12054) 
   
   <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] KarmaGYZ commented on a change in pull request #14647: [FLINK-20835] Implement FineGrainedSlotManager

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/WorkerResourceSpec.java
##########
@@ -70,6 +71,17 @@ public static WorkerResourceSpec fromTaskExecutorProcessSpec(
                 taskExecutorProcessSpec.getManagedMemorySize());
     }
 
+    public static WorkerResourceSpec fromTotalResourceProfile(
+            final ResourceProfile resourceProfile) {
+        Preconditions.checkNotNull(resourceProfile);
+        return new WorkerResourceSpec(
+                (CPUResource) resourceProfile.getCpuCores(),

Review comment:
       +1, create https://issues.apache.org/jira/browse/FLINK-21220 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 #14647: [FLINK-20835] Implement FineGrainedSlotManager

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "1c605546a2ac429e1c475eea6f80a75d0326c7bf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12054",
       "triggerID" : "1c605546a2ac429e1c475eea6f80a75d0326c7bf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "559cd8d09cca8082e9ca4904d11b11443f4c959f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12226",
       "triggerID" : "559cd8d09cca8082e9ca4904d11b11443f4c959f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "05733d3630a9a72d04545e2e8aeefdf76b2e5cf1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12235",
       "triggerID" : "05733d3630a9a72d04545e2e8aeefdf76b2e5cf1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0a88531c02021ab6bc36c8eec483dac0ee91db97",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12540",
       "triggerID" : "0a88531c02021ab6bc36c8eec483dac0ee91db97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "45b2dc853d01aa726bdf054b3c2afa5b95a1d6fd",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12543",
       "triggerID" : "45b2dc853d01aa726bdf054b3c2afa5b95a1d6fd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "80e3121418e741d9bebf3e849817a0c4225a86dd",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "80e3121418e741d9bebf3e849817a0c4225a86dd",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 45b2dc853d01aa726bdf054b3c2afa5b95a1d6fd Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12543) 
   * 80e3121418e741d9bebf3e849817a0c4225a86dd 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] xintongsong commented on a change in pull request #14647: [FLINK-20835] Implement FineGrainedSlotManager

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotStatusSyncer.java
##########
@@ -79,6 +79,7 @@ void initialize(
      *
      * @param instanceId of the task manager
      * @param slotReport reported
+     * @return whether to trigger the resource requirement check

Review comment:
       "whether the previous allocations can be applied"

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/DefaultSlotStatusSyncer.java
##########
@@ -224,11 +224,12 @@ public void reportSlotStatus(InstanceID instanceId, SlotReport slotReport) {
             LOG.debug(
                     "Received slot report for unknown task manager with instance id {}. Ignoring this report.",
                     instanceId);
-            return;
+            return false;
         }
 
         LOG.debug("Received slot report from instance {}: {}.", instanceId, slotReport);
 
+        boolean triggerCheckRequirement = false;

Review comment:
       I think triggering checking requirements is a decision made by `FineGrainedSlotManager`, which is not something the syncer needs to understand. I would suggest `canApplyPreviousAllocations`.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/FineGrainedSlotManager.java
##########
@@ -88,6 +88,8 @@
 
     private boolean sendNotEnoughResourceNotifications = true;
 
+    private Set<JobID> unfulfillableJobs = new HashSet<>();

Review comment:
       Should be cleared in `suspend()`




----------------------------------------------------------------
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] KarmaGYZ commented on pull request #14647: [FLINK-20835] Implement FineGrainedSlotManager

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


   @zentol I also glad to have a unified SlotManager. As the feature is not stable. I tend to put it out of the scope of this PR.


----------------------------------------------------------------
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] xintongsong commented on a change in pull request #14647: [FLINK-20835] Implement FineGrainedSlotManager

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotStatusSyncer.java
##########
@@ -79,6 +79,7 @@ void initialize(
      *
      * @param instanceId of the task manager
      * @param slotReport reported
+     * @return whether to trigger the resource requirement check

Review comment:
       "whether the previous allocations can be applied"

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/DefaultSlotStatusSyncer.java
##########
@@ -224,11 +224,12 @@ public void reportSlotStatus(InstanceID instanceId, SlotReport slotReport) {
             LOG.debug(
                     "Received slot report for unknown task manager with instance id {}. Ignoring this report.",
                     instanceId);
-            return;
+            return false;
         }
 
         LOG.debug("Received slot report from instance {}: {}.", instanceId, slotReport);
 
+        boolean triggerCheckRequirement = false;

Review comment:
       I think triggering checking requirements is a decision made by `FineGrainedSlotManager`, which is not something the syncer needs to understand. I would suggest `canApplyPreviousAllocations`.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/FineGrainedSlotManager.java
##########
@@ -88,6 +88,8 @@
 
     private boolean sendNotEnoughResourceNotifications = true;
 
+    private Set<JobID> unfulfillableJobs = new HashSet<>();

Review comment:
       Should be cleared in `suspend()`




----------------------------------------------------------------
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] KarmaGYZ commented on a change in pull request #14647: [FLINK-20835] Implement FineGrainedSlotManager

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/DefaultResourceAllocationStrategy.java
##########
@@ -0,0 +1,233 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.slots.ResourceCounter;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.runtime.resourcemanager.slotmanager.SlotManagerUtils.getEffectiveResourceProfile;
+
+/**
+ * The default implementation of {@link ResourceAllocationStrategy}, which always allocate pending
+ * task managers with the fixed profile.
+ */
+public class DefaultResourceAllocationStrategy implements ResourceAllocationStrategy {
+    private final ResourceProfile defaultSlotResourceProfile;
+    private final ResourceProfile totalResourceProfile;
+
+    public DefaultResourceAllocationStrategy(
+            ResourceProfile defaultSlotResourceProfile, int numSlotsPerWorker) {
+        this.defaultSlotResourceProfile = defaultSlotResourceProfile;
+        this.totalResourceProfile = defaultSlotResourceProfile.multiply(numSlotsPerWorker);
+    }
+
+    /**
+     * Matches resource requirements against available and pending resources. For each job, in a
+     * first round requirements are matched against registered resources. The remaining unfulfilled
+     * requirements are matched against pending resources, allocating more workers if no matching
+     * pending resources could be found. If the requirements for a job could not be fulfilled then
+     * it will be recorded in {@link ResourceAllocationResult#getUnfulfillableJobs()}.
+     *
+     * <p>Performance notes: At it's core this method loops, for each job, over all resources for
+     * each required slot, trying to find a matching registered/pending task manager. One should
+     * generally go in with the assumption that this runs in numberOfJobsRequiringResources *
+     * numberOfRequiredSlots * numberOfFreeOrPendingTaskManagers.
+     *
+     * <p>In the absolute worst case, with J jobs, requiring R slots each with a unique resource
+     * profile such each pair of these profiles is not matching, and T registered/pending task
+     * managers that don't fulfill any requirement, then this method does a total of J*R*T resource
+     * profile comparisons.
+     */
+    @Override
+    public ResourceAllocationResult tryFulfillRequirements(
+            Map<JobID, Collection<ResourceRequirement>> missingResources,
+            Map<InstanceID, Tuple2<ResourceProfile, ResourceProfile>> registeredResources,
+            List<PendingTaskManager> pendingTaskManagers) {
+        final ResourceAllocationResult.Builder resultBuilder = ResourceAllocationResult.builder();
+        final Map<PendingTaskManagerId, ResourceProfile> pendingResources =
+                pendingTaskManagers.stream()
+                        .collect(
+                                Collectors.toMap(
+                                        PendingTaskManager::getPendingTaskManagerId,
+                                        PendingTaskManager::getTotalResourceProfile));
+        for (Map.Entry<JobID, Collection<ResourceRequirement>> resourceRequirements :
+                missingResources.entrySet()) {
+            final JobID jobId = resourceRequirements.getKey();
+
+            final ResourceCounter unfulfilledJobRequirements =
+                    tryFulfillRequirementsForJobWithRegisteredResources(
+                            jobId,
+                            resourceRequirements.getValue(),
+                            registeredResources,
+                            resultBuilder);
+
+            if (!unfulfilledJobRequirements.isEmpty()) {
+                tryFulfillRequirementsForJobWithPendingResources(
+                        jobId, unfulfilledJobRequirements, pendingResources, resultBuilder);
+            }
+        }
+        return resultBuilder.build();
+    }
+
+    private static ResourceCounter tryFulfillRequirementsForJobWithRegisteredResources(
+            JobID jobId,
+            Collection<ResourceRequirement> missingResources,
+            Map<InstanceID, Tuple2<ResourceProfile, ResourceProfile>> registeredResources,
+            ResourceAllocationResult.Builder resultBuilder) {
+        final ResourceCounter outstandingRequirements = new ResourceCounter();
+
+        for (ResourceRequirement resourceRequirement : missingResources) {
+            int numMissingRequirements =
+                    tryFindSlotsForRequirement(
+                            jobId, resourceRequirement, registeredResources, resultBuilder);
+            if (numMissingRequirements > 0) {
+                outstandingRequirements.incrementCount(
+                        resourceRequirement.getResourceProfile(), numMissingRequirements);
+            }
+        }
+        return outstandingRequirements;
+    }
+
+    private static int tryFindSlotsForRequirement(
+            JobID jobId,
+            ResourceRequirement resourceRequirement,
+            Map<InstanceID, Tuple2<ResourceProfile, ResourceProfile>> registeredResources,
+            ResourceAllocationResult.Builder resultBuilder) {
+        final ResourceProfile requiredResource = resourceRequirement.getResourceProfile();
+
+        int numUnfulfilled = 0;
+        for (int x = 0; x < resourceRequirement.getNumberOfRequiredSlots(); x++) {
+            final Optional<InstanceID> matchedTaskManager =
+                    findMatchingTaskManager(requiredResource, registeredResources);
+            if (matchedTaskManager.isPresent()) {
+                final ResourceProfile effectiveProfile =
+                        getEffectiveResourceProfile(
+                                requiredResource,
+                                registeredResources.get(matchedTaskManager.get()).f1);
+                resultBuilder.recordAllocationForRegisteredResource(
+                        jobId, matchedTaskManager.get(), effectiveProfile);
+                deductionRegisteredResource(
+                        registeredResources, matchedTaskManager.get(), effectiveProfile);
+            } else {
+                // exit loop early; we won't find a matching slot for this requirement
+                int numRemaining = resourceRequirement.getNumberOfRequiredSlots() - x;
+                numUnfulfilled += numRemaining;
+                break;
+            }
+        }
+        return numUnfulfilled;
+    }
+
+    private static Optional<InstanceID> findMatchingTaskManager(
+            ResourceProfile requirement,
+            Map<InstanceID, Tuple2<ResourceProfile, ResourceProfile>> registeredResources) {
+        return registeredResources.entrySet().stream()
+                .filter(
+                        taskManager ->
+                                canFulfillRequirement(
+                                        getEffectiveResourceProfile(
+                                                requirement, taskManager.getValue().f1),
+                                        taskManager.getValue().f0))
+                .findFirst()
+                .map(Map.Entry::getKey);
+    }
+
+    private static boolean canFulfillRequirement(
+            ResourceProfile requirement, ResourceProfile resourceProfile) {
+        return resourceProfile.allFieldsNoLessThan(requirement);
+    }
+
+    private static void deductionRegisteredResource(
+            Map<InstanceID, Tuple2<ResourceProfile, ResourceProfile>> registeredResources,
+            InstanceID instanceId,
+            ResourceProfile resourceProfile) {
+        registeredResources.computeIfPresent(
+                instanceId,
+                (id, tuple2) -> {
+                    if (tuple2.f0.subtract(resourceProfile).equals(ResourceProfile.ZERO)) {
+                        return null;
+                    } else {
+                        return Tuple2.of(tuple2.f0.subtract(resourceProfile), tuple2.f1);
+                    }
+                });
+    }
+
+    private static Optional<PendingTaskManagerId> findPendingManagerToFulfill(
+            ResourceProfile resourceProfile,
+            Map<PendingTaskManagerId, ResourceProfile> availableResources) {
+        return availableResources.entrySet().stream()
+                .filter(entry -> entry.getValue().allFieldsNoLessThan(resourceProfile))
+                .findAny()
+                .map(Map.Entry::getKey);
+    }
+
+    private void tryFulfillRequirementsForJobWithPendingResources(
+            JobID jobId,
+            ResourceCounter unfulfilledRequirements,
+            Map<PendingTaskManagerId, ResourceProfile> availableResources,
+            ResourceAllocationResult.Builder resultBuilder) {
+        for (Map.Entry<ResourceProfile, Integer> missingResource :
+                unfulfilledRequirements.getResourceProfilesWithCount().entrySet()) {
+            final ResourceProfile effectiveProfile =
+                    getEffectiveResourceProfile(
+                            missingResource.getKey(), defaultSlotResourceProfile);
+            for (int i = 0; i < missingResource.getValue(); i++) {
+                Optional<PendingTaskManagerId> matchedPendingTaskManager =
+                        findPendingManagerToFulfill(effectiveProfile, availableResources);
+                if (matchedPendingTaskManager.isPresent()) {
+                    availableResources.compute(
+                            matchedPendingTaskManager.get(),
+                            ((pendingTaskManagerId, resourceProfile) ->
+                                    resourceProfile.subtract(effectiveProfile)));
+                    resultBuilder.recordAllocationForPendingResource(
+                            jobId, matchedPendingTaskManager.get(), effectiveProfile);
+                } else {
+                    if (totalResourceProfile.allFieldsNoLessThan(effectiveProfile)) {
+                        // Add new pending task manager
+                        final PendingTaskManagerId pendingTaskManagerId =
+                                PendingTaskManagerId.generate();
+                        resultBuilder.addPendingTaskManagerToBeAllocated(
+                                new PendingTaskManager(
+                                        pendingTaskManagerId,
+                                        totalResourceProfile,
+                                        defaultSlotResourceProfile));
+                        resultBuilder.recordAllocationForPendingResource(
+                                jobId, pendingTaskManagerId, effectiveProfile);
+                        availableResources.put(
+                                pendingTaskManagerId,
+                                totalResourceProfile.subtract(effectiveProfile));
+                    } else {
+                        resultBuilder.addUnfulfillableJob(jobId);

Review comment:
       > An unfulfillable job will probably fail anyway, maybe we should not allocate for this job at all, save the resources for the fulfillable jobs.
   I agree it's a valid optimization in some cases. The reason I didn't do this optimization is I am not sure the assumption(unfulfillable job will probably fail anyway) holds forever. Reactive mode is a counterexample.




----------------------------------------------------------------
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] KarmaGYZ commented on a change in pull request #14647: [FLINK-20835] Implement FineGrainedSlotManager

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/PendingTaskManagerId.java
##########
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.util.AbstractID;
+
+/** Id of {@link PendingTaskManager}. */
+public class PendingTaskManagerId extends AbstractID {

Review comment:
       Not sure whether we need it. This ID should only be used inside the slotmanager. It will not be serialized in any case, and thus no need to check the version.




----------------------------------------------------------------
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 #14647: [FLINK-20835] Implement FineGrainedSlotManager

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "1c605546a2ac429e1c475eea6f80a75d0326c7bf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12054",
       "triggerID" : "1c605546a2ac429e1c475eea6f80a75d0326c7bf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "559cd8d09cca8082e9ca4904d11b11443f4c959f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12226",
       "triggerID" : "559cd8d09cca8082e9ca4904d11b11443f4c959f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "05733d3630a9a72d04545e2e8aeefdf76b2e5cf1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12235",
       "triggerID" : "05733d3630a9a72d04545e2e8aeefdf76b2e5cf1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0a88531c02021ab6bc36c8eec483dac0ee91db97",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12540",
       "triggerID" : "0a88531c02021ab6bc36c8eec483dac0ee91db97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "45b2dc853d01aa726bdf054b3c2afa5b95a1d6fd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12543",
       "triggerID" : "45b2dc853d01aa726bdf054b3c2afa5b95a1d6fd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "80e3121418e741d9bebf3e849817a0c4225a86dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12587",
       "triggerID" : "80e3121418e741d9bebf3e849817a0c4225a86dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f5f9f44b437922f5e0edd0b1dd3097a61351dee",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12723",
       "triggerID" : "8f5f9f44b437922f5e0edd0b1dd3097a61351dee",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0b72f53fbfe19ef426572f78ac716521bc63ab64",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12769",
       "triggerID" : "0b72f53fbfe19ef426572f78ac716521bc63ab64",
       "triggerType" : "PUSH"
     }, {
       "hash" : "970360401dcd333f1a3b29355392385262953246",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12823",
       "triggerID" : "970360401dcd333f1a3b29355392385262953246",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1b395270541b226b08653dabc1e6a6bcc3f7ed5c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12835",
       "triggerID" : "1b395270541b226b08653dabc1e6a6bcc3f7ed5c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f0dc516d44616986d6ddf6e03902bbb1ffb2cb03",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12839",
       "triggerID" : "f0dc516d44616986d6ddf6e03902bbb1ffb2cb03",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * f0dc516d44616986d6ddf6e03902bbb1ffb2cb03 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12839) 
   
   <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 #14647: [FLINK-20835] Implement FineGrainedSlotManager

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "1c605546a2ac429e1c475eea6f80a75d0326c7bf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12054",
       "triggerID" : "1c605546a2ac429e1c475eea6f80a75d0326c7bf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "559cd8d09cca8082e9ca4904d11b11443f4c959f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12226",
       "triggerID" : "559cd8d09cca8082e9ca4904d11b11443f4c959f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "05733d3630a9a72d04545e2e8aeefdf76b2e5cf1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12235",
       "triggerID" : "05733d3630a9a72d04545e2e8aeefdf76b2e5cf1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0a88531c02021ab6bc36c8eec483dac0ee91db97",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12540",
       "triggerID" : "0a88531c02021ab6bc36c8eec483dac0ee91db97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "45b2dc853d01aa726bdf054b3c2afa5b95a1d6fd",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12543",
       "triggerID" : "45b2dc853d01aa726bdf054b3c2afa5b95a1d6fd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "80e3121418e741d9bebf3e849817a0c4225a86dd",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12587",
       "triggerID" : "80e3121418e741d9bebf3e849817a0c4225a86dd",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 45b2dc853d01aa726bdf054b3c2afa5b95a1d6fd Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12543) 
   * 80e3121418e741d9bebf3e849817a0c4225a86dd Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12587) 
   
   <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 #14647: [FLINK-20835] Implement FineGrainedSlotManager

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "1c605546a2ac429e1c475eea6f80a75d0326c7bf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12054",
       "triggerID" : "1c605546a2ac429e1c475eea6f80a75d0326c7bf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "559cd8d09cca8082e9ca4904d11b11443f4c959f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12226",
       "triggerID" : "559cd8d09cca8082e9ca4904d11b11443f4c959f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "05733d3630a9a72d04545e2e8aeefdf76b2e5cf1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12235",
       "triggerID" : "05733d3630a9a72d04545e2e8aeefdf76b2e5cf1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0a88531c02021ab6bc36c8eec483dac0ee91db97",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12540",
       "triggerID" : "0a88531c02021ab6bc36c8eec483dac0ee91db97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "45b2dc853d01aa726bdf054b3c2afa5b95a1d6fd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12543",
       "triggerID" : "45b2dc853d01aa726bdf054b3c2afa5b95a1d6fd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "80e3121418e741d9bebf3e849817a0c4225a86dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12587",
       "triggerID" : "80e3121418e741d9bebf3e849817a0c4225a86dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f5f9f44b437922f5e0edd0b1dd3097a61351dee",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12723",
       "triggerID" : "8f5f9f44b437922f5e0edd0b1dd3097a61351dee",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0b72f53fbfe19ef426572f78ac716521bc63ab64",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12769",
       "triggerID" : "0b72f53fbfe19ef426572f78ac716521bc63ab64",
       "triggerType" : "PUSH"
     }, {
       "hash" : "970360401dcd333f1a3b29355392385262953246",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12823",
       "triggerID" : "970360401dcd333f1a3b29355392385262953246",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1b395270541b226b08653dabc1e6a6bcc3f7ed5c",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12835",
       "triggerID" : "1b395270541b226b08653dabc1e6a6bcc3f7ed5c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f0dc516d44616986d6ddf6e03902bbb1ffb2cb03",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "f0dc516d44616986d6ddf6e03902bbb1ffb2cb03",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 970360401dcd333f1a3b29355392385262953246 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12823) 
   * 1b395270541b226b08653dabc1e6a6bcc3f7ed5c Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12835) 
   * f0dc516d44616986d6ddf6e03902bbb1ffb2cb03 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 #14647: [FLINK-20835] Implement FineGrainedSlotManager

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "1c605546a2ac429e1c475eea6f80a75d0326c7bf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12054",
       "triggerID" : "1c605546a2ac429e1c475eea6f80a75d0326c7bf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "559cd8d09cca8082e9ca4904d11b11443f4c959f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12226",
       "triggerID" : "559cd8d09cca8082e9ca4904d11b11443f4c959f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "05733d3630a9a72d04545e2e8aeefdf76b2e5cf1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12235",
       "triggerID" : "05733d3630a9a72d04545e2e8aeefdf76b2e5cf1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0a88531c02021ab6bc36c8eec483dac0ee91db97",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12540",
       "triggerID" : "0a88531c02021ab6bc36c8eec483dac0ee91db97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "45b2dc853d01aa726bdf054b3c2afa5b95a1d6fd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12543",
       "triggerID" : "45b2dc853d01aa726bdf054b3c2afa5b95a1d6fd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "80e3121418e741d9bebf3e849817a0c4225a86dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12587",
       "triggerID" : "80e3121418e741d9bebf3e849817a0c4225a86dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f5f9f44b437922f5e0edd0b1dd3097a61351dee",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12723",
       "triggerID" : "8f5f9f44b437922f5e0edd0b1dd3097a61351dee",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0b72f53fbfe19ef426572f78ac716521bc63ab64",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12769",
       "triggerID" : "0b72f53fbfe19ef426572f78ac716521bc63ab64",
       "triggerType" : "PUSH"
     }, {
       "hash" : "970360401dcd333f1a3b29355392385262953246",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12823",
       "triggerID" : "970360401dcd333f1a3b29355392385262953246",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1b395270541b226b08653dabc1e6a6bcc3f7ed5c",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12835",
       "triggerID" : "1b395270541b226b08653dabc1e6a6bcc3f7ed5c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f0dc516d44616986d6ddf6e03902bbb1ffb2cb03",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12839",
       "triggerID" : "f0dc516d44616986d6ddf6e03902bbb1ffb2cb03",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 1b395270541b226b08653dabc1e6a6bcc3f7ed5c Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12835) 
   * f0dc516d44616986d6ddf6e03902bbb1ffb2cb03 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12839) 
   
   <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 #14647: [FLINK-20835] Implement FineGrainedSlotManager

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "1c605546a2ac429e1c475eea6f80a75d0326c7bf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12054",
       "triggerID" : "1c605546a2ac429e1c475eea6f80a75d0326c7bf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "559cd8d09cca8082e9ca4904d11b11443f4c959f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12226",
       "triggerID" : "559cd8d09cca8082e9ca4904d11b11443f4c959f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "05733d3630a9a72d04545e2e8aeefdf76b2e5cf1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12235",
       "triggerID" : "05733d3630a9a72d04545e2e8aeefdf76b2e5cf1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0a88531c02021ab6bc36c8eec483dac0ee91db97",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12540",
       "triggerID" : "0a88531c02021ab6bc36c8eec483dac0ee91db97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "45b2dc853d01aa726bdf054b3c2afa5b95a1d6fd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12543",
       "triggerID" : "45b2dc853d01aa726bdf054b3c2afa5b95a1d6fd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "80e3121418e741d9bebf3e849817a0c4225a86dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12587",
       "triggerID" : "80e3121418e741d9bebf3e849817a0c4225a86dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f5f9f44b437922f5e0edd0b1dd3097a61351dee",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12723",
       "triggerID" : "8f5f9f44b437922f5e0edd0b1dd3097a61351dee",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 8f5f9f44b437922f5e0edd0b1dd3097a61351dee Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12723) 
   
   <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] xintongsong closed pull request #14647: [FLINK-20835] Implement FineGrainedSlotManager

Posted by GitBox <gi...@apache.org>.
xintongsong closed pull request #14647:
URL: https://github.com/apache/flink/pull/14647


   


----------------------------------------------------------------
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] xintongsong commented on a change in pull request #14647: [FLINK-20835] Implement FineGrainedSlotManager

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



##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/FineGrainedSlotManagerTestBase.java
##########
@@ -0,0 +1,157 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.common.time.Time;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.concurrent.ScheduledExecutorServiceAdapter;
+import org.apache.flink.runtime.metrics.groups.SlotManagerMetricGroup;
+import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerId;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+import org.apache.flink.runtime.slots.ResourceRequirements;
+import org.apache.flink.runtime.taskexecutor.SlotStatus;
+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.RunnableWithException;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+/** Base class for the tests of {@link FineGrainedSlotManager}. */
+public abstract class FineGrainedSlotManagerTestBase extends TestLogger {
+    private static final String MAIN_THREAD_NAME = "testing-rpc-main-thread";
+    private static final ScheduledExecutor MAIN_THREAD_EXECUTOR =
+            new ScheduledExecutorServiceAdapter(
+                    Executors.newSingleThreadScheduledExecutor(
+                            runnable -> new Thread(runnable, MAIN_THREAD_NAME)));
+    static final FlinkException TEST_EXCEPTION = new FlinkException("Test exception");
+
+    protected abstract ResourceProfile getDefaultTaskManagerResourceProfile();
+
+    protected abstract ResourceProfile getDefaultSlotResourceProfile();
+
+    protected abstract int getDefaultNumberSlotsPerWorker();
+
+    protected abstract ResourceProfile getLargeTaskManagerResourceProfile();
+
+    protected abstract ResourceProfile getLargeSlotResourceProfile();

Review comment:
       Better to explain the requirements on the resource profiles.
   IIUC, DefaultTM can fulfill DefaultSlot but not LargeSlot, LargeTM can fulfill DefaultSlot and LargeSlot.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/FineGrainedSlotManagerTestBase.java
##########
@@ -0,0 +1,157 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.common.time.Time;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.concurrent.ScheduledExecutorServiceAdapter;
+import org.apache.flink.runtime.metrics.groups.SlotManagerMetricGroup;
+import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerId;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+import org.apache.flink.runtime.slots.ResourceRequirements;
+import org.apache.flink.runtime.taskexecutor.SlotStatus;
+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.RunnableWithException;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+/** Base class for the tests of {@link FineGrainedSlotManager}. */
+public abstract class FineGrainedSlotManagerTestBase extends TestLogger {
+    private static final String MAIN_THREAD_NAME = "testing-rpc-main-thread";
+    private static final ScheduledExecutor MAIN_THREAD_EXECUTOR =
+            new ScheduledExecutorServiceAdapter(
+                    Executors.newSingleThreadScheduledExecutor(
+                            runnable -> new Thread(runnable, MAIN_THREAD_NAME)));
+    static final FlinkException TEST_EXCEPTION = new FlinkException("Test exception");
+
+    protected abstract ResourceProfile getDefaultTaskManagerResourceProfile();
+
+    protected abstract ResourceProfile getDefaultSlotResourceProfile();
+
+    protected abstract int getDefaultNumberSlotsPerWorker();
+
+    protected abstract ResourceProfile getLargeTaskManagerResourceProfile();
+
+    protected abstract ResourceProfile getLargeSlotResourceProfile();
+
+    protected abstract ResourceAllocationStrategy getResourceAllocationStrategy();
+
+    static SlotStatus createAllocatedSlotStatus(
+            AllocationID allocationID, ResourceProfile resourceProfile) {
+        return new SlotStatus(
+                new SlotID(ResourceID.generate(), 0), resourceProfile, new JobID(), allocationID);
+    }
+
+    static int getTotalResourceCount(Collection<ResourceRequirement> resources) {
+        if (resources == null) {
+            return 0;
+        }
+        return resources.stream()
+                .map(ResourceRequirement::getNumberOfRequiredSlots)
+                .reduce(0, Integer::sum);
+    }
+
+    static ResourceRequirements createResourceRequirementsForSingleSlot() {
+        return createResourceRequirementsForSingleSlot(new JobID());
+    }
+
+    static ResourceRequirements createResourceRequirementsForSingleSlot(JobID jobId) {
+        return createResourceRequirements(jobId, 1);
+    }
+
+    static ResourceRequirements createResourceRequirements(JobID jobId, int numRequiredSlots) {
+        return createResourceRequirements(jobId, numRequiredSlots, ResourceProfile.UNKNOWN);
+    }
+
+    static ResourceRequirements createResourceRequirements(
+            JobID jobId, int numRequiredSlots, ResourceProfile resourceProfile) {
+        return ResourceRequirements.create(
+                jobId,
+                "foobar",
+                Collections.singleton(
+                        ResourceRequirement.create(resourceProfile, numRequiredSlots)));
+    }
+
+    /** This class provides a self-contained context for each test case. */
+    protected class Context {
+        private final ResourceManagerId resourceManagerId = ResourceManagerId.generate();
+        private final ResourceTracker resourceTracker = new DefaultResourceTracker();
+        private final TaskManagerTracker taskManagerTracker = new FineGrainedTaskManagerTracker();
+        private final SlotStatusSyncer slotStatusSyncer =
+                new DefaultSlotStatusSyncer(Time.seconds(10L));
+        private final ScheduledExecutor scheduledExecutor = TestingUtils.defaultScheduledExecutor();
+        private final SlotManagerMetricGroup slotManagerMetricGroup =
+                UnregisteredMetricGroups.createUnregisteredSlotManagerMetricGroup();
+        private FineGrainedSlotManager slotManager;
+
+        protected TestingResourceAllocationStrategy.Builder resourceAllocationStrategyBuilder;
+        protected Executor mainThreadExecutor = MAIN_THREAD_EXECUTOR;
+
+        final TestingResourceActionsBuilder resourceActionsBuilder =
+                new TestingResourceActionsBuilder();
+        final SlotManagerConfigurationBuilder slotManagerConfigurationBuilder =
+                SlotManagerConfigurationBuilder.newBuilder();
+
+        FineGrainedSlotManager getSlotManager() {
+            return slotManager;
+        }
+
+        ResourceTracker getResourceTracker() {
+            return resourceTracker;
+        }
+
+        TaskManagerTracker getTaskManagerTracker() {
+            return taskManagerTracker;
+        }
+
+        ResourceManagerId getResourceManagerId() {
+            return resourceManagerId;
+        }
+
+        protected final void runTest(RunnableWithException testMethod) throws Exception {
+            slotManager =
+                    new FineGrainedSlotManager(
+                            scheduledExecutor,
+                            slotManagerConfigurationBuilder.build(),
+                            slotManagerMetricGroup,
+                            resourceTracker,
+                            taskManagerTracker,
+                            slotStatusSyncer,
+                            resourceAllocationStrategyBuilder == null
+                                    ? getResourceAllocationStrategy()
+                                    : resourceAllocationStrategyBuilder.build());

Review comment:
       I think we can make `getResourceAllocationStrategy()` return `null` to indicate that a testing strategy should be built. In this way, we won't need to create the `TestingResourceAllocationStrategy.Builder` in test cases.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/FineGrainedSlotManagerTest.java
##########
@@ -35,158 +33,152 @@
 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.testingUtils.TestingUtils;
-import org.apache.flink.runtime.testutils.SystemExitTrackingSecurityManager;
-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.Test;
 
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.Collection;
-import java.util.Collections;
-import java.util.Iterator;
 import java.util.List;
 import java.util.Optional;
-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.atomic.AtomicInteger;
-import java.util.function.Supplier;
 
-import static org.hamcrest.CoreMatchers.hasItem;
 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.hamcrest.Matchers.not;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertSame;
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 
-/** Tests for the {@link FineGrainedSlotManager}. */
-public class FineGrainedSlotManagerTest extends TestLogger {
-    private static final FlinkException TEST_EXCEPTION = new FlinkException("Test exception");
+/** Tests of {@link FineGrainedSlotManager}. */
+public class FineGrainedSlotManagerTest extends FineGrainedSlotManagerTestBase {
+
     private static final WorkerResourceSpec DEFAULT_WORKER_RESOURCE_SPEC =
+            new WorkerResourceSpec.Builder()
+                    .setCpuCores(10.0)
+                    .setTaskHeapMemoryMB(1000)
+                    .setTaskOffHeapMemoryMB(1000)
+                    .setNetworkMemoryMB(1000)
+                    .setManagedMemoryMB(1000)
+                    .build();
+    private static final WorkerResourceSpec LARGE_WORKER_RESOURCE_SPEC =
             new WorkerResourceSpec.Builder()
                     .setCpuCores(100.0)
                     .setTaskHeapMemoryMB(10000)
                     .setTaskOffHeapMemoryMB(10000)
                     .setNetworkMemoryMB(10000)
                     .setManagedMemoryMB(10000)
                     .build();
-    private static final int DEFAULT_NUM_SLOTS_PER_WORKER = 1;
+    private static final int DEFAULT_NUM_SLOTS_PER_WORKER = 2;
     private static final ResourceProfile DEFAULT_TOTAL_RESOURCE_PROFILE =
             SlotManagerUtils.generateTaskManagerTotalResourceProfile(DEFAULT_WORKER_RESOURCE_SPEC);
     private static final ResourceProfile DEFAULT_SLOT_RESOURCE_PROFILE =
             SlotManagerUtils.generateDefaultSlotResourceProfile(
                     DEFAULT_WORKER_RESOURCE_SPEC, DEFAULT_NUM_SLOTS_PER_WORKER);
+    private static final ResourceProfile LARGE_TOTAL_RESOURCE_PROFILE =
+            SlotManagerUtils.generateTaskManagerTotalResourceProfile(LARGE_WORKER_RESOURCE_SPEC);
+    private static final ResourceProfile LARGE_SLOT_RESOURCE_PROFILE =
+            SlotManagerUtils.generateDefaultSlotResourceProfile(
+                    LARGE_WORKER_RESOURCE_SPEC, DEFAULT_NUM_SLOTS_PER_WORKER);
 
-    @Test
-    public void testCloseAfterSuspendDoesNotThrowException() throws Exception {
-        try (FineGrainedSlotManager slotManager =
-                createFineGrainedSlotManagerBuilder().buildAndStartWithDirectExec()) {
-            slotManager.suspend();
-        }
+    @Override
+    protected ResourceProfile getDefaultTaskManagerResourceProfile() {
+        return DEFAULT_TOTAL_RESOURCE_PROFILE;
     }
 
-    /** Tests that we can register task manager at the slot manager. */
-    @Test
-    public void testTaskManagerRegistration() throws Exception {
-        final TaskExecutorGateway taskExecutorGateway =
-                new TestingTaskExecutorGatewayBuilder().createTestingTaskExecutorGateway();
-        final TaskExecutorConnection taskManagerConnection =
-                new TaskExecutorConnection(ResourceID.generate(), taskExecutorGateway);
+    @Override
+    protected ResourceProfile getDefaultSlotResourceProfile() {
+        return DEFAULT_SLOT_RESOURCE_PROFILE;
+    }
+
+    @Override
+    protected int getDefaultNumberSlotsPerWorker() {
+        return DEFAULT_NUM_SLOTS_PER_WORKER;
+    }
 
-        final TaskManagerTracker tracker = new FineGrainedTaskManagerTracker();
+    @Override
+    protected ResourceProfile getLargeTaskManagerResourceProfile() {
+        return LARGE_TOTAL_RESOURCE_PROFILE;
+    }
 
-        try (FineGrainedSlotManager slotManager =
-                createFineGrainedSlotManagerBuilder()
-                        .setTaskManagerTracker(tracker)
-                        .buildAndStartWithDirectExec()) {
-            slotManager.registerTaskManager(
-                    taskManagerConnection,
-                    new SlotReport(),
-                    DEFAULT_TOTAL_RESOURCE_PROFILE,
-                    DEFAULT_SLOT_RESOURCE_PROFILE);
+    @Override
+    protected ResourceProfile getLargeSlotResourceProfile() {
+        return LARGE_SLOT_RESOURCE_PROFILE;
+    }
 
-            assertThat(slotManager.getNumberRegisteredSlots(), equalTo(1));
-            assertThat(tracker.getRegisteredTaskManagers().size(), equalTo(1));
-            assertTrue(
-                    tracker.getRegisteredTaskManager(taskManagerConnection.getInstanceID())
-                            .isPresent());
-            assertThat(
-                    tracker.getRegisteredTaskManager(taskManagerConnection.getInstanceID())
-                            .get()
-                            .getAvailableResource(),
-                    equalTo(DEFAULT_TOTAL_RESOURCE_PROFILE));
-            assertThat(
-                    tracker.getRegisteredTaskManager(taskManagerConnection.getInstanceID())
-                            .get()
-                            .getTotalResource(),
-                    equalTo(DEFAULT_TOTAL_RESOURCE_PROFILE));
-        }
+    @Override
+    protected ResourceAllocationStrategy getResourceAllocationStrategy() {
+        return TestingResourceAllocationStrategy.newBuilder().build();
     }
 
-    /** Tests that we can matched task manager will deduct pending task manager. */
+    // ---------------------------------------------------------------------------------------------
+    // Initialize and close
+    // ---------------------------------------------------------------------------------------------
+
     @Test
-    public void testTaskManagerRegistrationDeductPendingTaskManager() throws Exception {
-        final TaskManagerTracker tracker = new FineGrainedTaskManagerTracker();
+    public void testInitializeAndClose() throws Exception {
+        new Context() {
+            {
+                runTest(() -> getSlotManager().close());
+            }
+        };
+    }
 
-        try (FineGrainedSlotManager slotManager =
-                createFineGrainedSlotManagerBuilder()
-                        .setTaskManagerTracker(tracker)
-                        .buildAndStartWithDirectExec()) {
+    // ---------------------------------------------------------------------------------------------
+    // Register / unregister TaskManager and and slot status reconciliation
+    // ---------------------------------------------------------------------------------------------
 
-            final TaskExecutorGateway taskExecutorGateway =
-                    new TestingTaskExecutorGatewayBuilder().createTestingTaskExecutorGateway();
-            final TaskExecutorConnection taskExecutionConnection1 =
-                    new TaskExecutorConnection(ResourceID.generate(), taskExecutorGateway);
-            final TaskExecutorConnection taskExecutionConnection2 =
-                    new TaskExecutorConnection(ResourceID.generate(), taskExecutorGateway);
-            final TaskExecutorConnection taskExecutionConnection3 =
-                    new TaskExecutorConnection(ResourceID.generate(), taskExecutorGateway);
-            final SlotReport slotReportWithAllocatedSlot =
-                    new SlotReport(
-                            createAllocatedSlotStatus(new AllocationID(), ResourceProfile.ANY));
-            tracker.addPendingTaskManager(
-                    new PendingTaskManager(
-                            PendingTaskManagerId.generate(),
-                            ResourceProfile.ANY,
-                            ResourceProfile.ANY));
-            // task manager with allocated slot cannot deduct pending task manager
-            slotManager.registerTaskManager(
-                    taskExecutionConnection1,
-                    slotReportWithAllocatedSlot,
-                    ResourceProfile.ANY,
-                    ResourceProfile.ANY);
-            assertThat(tracker.getPendingTaskManagers().size(), is(1));
-            // task manager with mismatched resource cannot deduct pending task manager
-            slotManager.registerTaskManager(
-                    taskExecutionConnection2,
-                    new SlotReport(),
-                    ResourceProfile.fromResources(10, 100),
-                    ResourceProfile.fromResources(10, 100));
-            assertThat(tracker.getPendingTaskManagers().size(), is(1));
-            slotManager.registerTaskManager(
-                    taskExecutionConnection3,
-                    new SlotReport(),
-                    ResourceProfile.ANY,
-                    ResourceProfile.ANY);
-            assertThat(tracker.getPendingTaskManagers().size(), is(0));
-        }
+    /** Tests that we can register task manager at the slot manager. */
+    @Test
+    public void testTaskManagerRegistration() throws Exception {
+        final TaskExecutorGateway taskExecutorGateway =
+                new TestingTaskExecutorGatewayBuilder().createTestingTaskExecutorGateway();
+        final TaskExecutorConnection taskManagerConnection =
+                new TaskExecutorConnection(ResourceID.generate(), taskExecutorGateway);

Review comment:
       There's many test cases that need creating the `TaskExecutorConnection`. I think it worth introducing a static util method.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/FineGrainedSlotManagerITCase.java
##########
@@ -0,0 +1,83 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec;
+
+/** IT Cases of {@link FineGrainedSlotManager}. */
+public class FineGrainedSlotManagerITCase extends AbstractFineGrainedSlotManagerITCase {

Review comment:
       I would name it `FineGrainedSlotManager_DefaultResourceAllocationStrategyITCase`

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/AbstractFineGrainedSlotManagerITCase.java
##########
@@ -0,0 +1,672 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.Tuple6;
+import org.apache.flink.core.testutils.ManuallyTriggeredScheduledExecutorService;
+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.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.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.testutils.SystemExitTrackingSecurityManager;
+import org.apache.flink.util.function.FunctionUtils;
+
+import akka.pattern.AskTimeoutException;
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Supplier;
+
+import static org.hamcrest.Matchers.empty;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.is;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertThat;
+
+/** IT Cases of {@link FineGrainedSlotManager}. */
+public abstract class AbstractFineGrainedSlotManagerITCase extends FineGrainedSlotManagerTestBase {
+
+    // ---------------------------------------------------------------------------------------------
+    // Requirement declaration
+    // ---------------------------------------------------------------------------------------------
+
+    /**
+     * Tests that a requirement declaration with no free slots will trigger the resource allocation.
+     */
+    @Test
+    public void testRequirementDeclarationWithoutFreeSlotsTriggersWorkerAllocation()
+            throws Exception {
+        final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot();
+
+        final CompletableFuture<WorkerResourceSpec> allocateResourceFuture =
+                new CompletableFuture<>();
+        new Context() {
+            {
+                resourceActionsBuilder.setAllocateResourceConsumer(
+                        allocateResourceFuture::complete);
+                runTest(
+                        () -> {
+                            getSlotManager().processResourceRequirements(resourceRequirements);
+
+                            allocateResourceFuture.get();
+                        });
+            }
+        };
+    }
+
+    /**
+     * Tests that resources continue to be considered missing if we cannot allocate more resources.
+     */
+    @Test
+    public void testRequirementDeclarationWithResourceAllocationFailure() throws Exception {
+        final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot();
+        new Context() {
+            {
+                resourceActionsBuilder.setAllocateResourceFunction(value -> false);
+                runTest(
+                        () -> {
+                            getSlotManager().processResourceRequirements(resourceRequirements);
+
+                            final JobID jobId = resourceRequirements.getJobId();
+                            assertThat(
+                                    getTotalResourceCount(
+                                            getResourceTracker().getMissingResources().get(jobId)),
+                                    is(1));
+                        });
+            }
+        };
+    }
+
+    /** Tests that resource requirements can be fulfilled with resource that are currently free. */
+    @Test
+    public void testRequirementDeclarationWithFreeResource() throws Exception {
+        testRequirementDeclaration(
+                RequirementDeclarationScenario
+                        .TASK_EXECUTOR_REGISTRATION_BEFORE_REQUIREMENT_DECLARATION);
+    }
+
+    /**
+     * Tests that resource requirements can be fulfilled with resource that are registered after the
+     * requirement declaration.
+     */
+    @Test
+    public void testRequirementDeclarationWithPendingResource() throws Exception {
+        testRequirementDeclaration(
+                RequirementDeclarationScenario
+                        .TASK_EXECUTOR_REGISTRATION_AFTER_REQUIREMENT_DECLARATION);
+    }
+
+    private enum RequirementDeclarationScenario {
+        // Tests that a slot request which can be fulfilled will trigger a slot allocation
+        TASK_EXECUTOR_REGISTRATION_BEFORE_REQUIREMENT_DECLARATION,
+        // Tests that pending slot requests are tried to be fulfilled upon new slot registrations
+        TASK_EXECUTOR_REGISTRATION_AFTER_REQUIREMENT_DECLARATION
+    }
+
+    private void testRequirementDeclaration(RequirementDeclarationScenario scenario)
+            throws Exception {
+        final ResourceID resourceID = ResourceID.generate();
+        final JobID jobId = new JobID();
+        final SlotID slotId = SlotID.getDynamicSlotID(resourceID);
+        final String targetAddress = "localhost";
+        final ResourceRequirements requirements =
+                ResourceRequirements.create(
+                        jobId,
+                        targetAddress,
+                        Collections.singleton(
+                                ResourceRequirement.create(getDefaultSlotResourceProfile(), 1)));
+
+        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);
+                                    return CompletableFuture.completedFuture(Acknowledge.get());
+                                })
+                        .createTestingTaskExecutorGateway();
+
+        final TaskExecutorConnection taskExecutorConnection =
+                new TaskExecutorConnection(resourceID, taskExecutorGateway);
+
+        new Context() {
+            {
+                runTest(
+                        () -> {
+                            if (scenario
+                                    == RequirementDeclarationScenario
+                                            .TASK_EXECUTOR_REGISTRATION_BEFORE_REQUIREMENT_DECLARATION) {
+                                getSlotManager()
+                                        .registerTaskManager(
+                                                taskExecutorConnection,
+                                                new SlotReport(),
+                                                getDefaultTaskManagerResourceProfile(),
+                                                getDefaultSlotResourceProfile());
+                            }
+
+                            getSlotManager().processResourceRequirements(requirements);
+
+                            if (scenario
+                                    == RequirementDeclarationScenario
+                                            .TASK_EXECUTOR_REGISTRATION_AFTER_REQUIREMENT_DECLARATION) {
+                                getSlotManager()
+                                        .registerTaskManager(
+                                                taskExecutorConnection,
+                                                new SlotReport(),
+                                                getDefaultTaskManagerResourceProfile(),
+                                                getDefaultSlotResourceProfile());
+                            }
+
+                            assertThat(
+                                    requestFuture.get(),
+                                    is(
+                                            equalTo(
+                                                    Tuple6.of(
+                                                            slotId,
+                                                            jobId,
+                                                            requestFuture.get().f2,
+                                                            getDefaultSlotResourceProfile(),
+                                                            targetAddress,
+                                                            getResourceManagerId()))));
+
+                            final TaskManagerSlotInformation slot =
+                                    getTaskManagerTracker()
+                                            .getAllocatedOrPendingSlot(requestFuture.get().f2)
+                                            .get();
+
+                            assertEquals(
+                                    "The slot has not been allocated to the expected allocation id.",
+                                    requestFuture.get().f2,
+                                    slot.getAllocationId());
+                        });
+            }
+        };
+    }
+
+    /**
+     * 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 AtomicInteger allocateResourceCalls = new AtomicInteger(0);
+        final ResourceRequirements requirements = createResourceRequirementsForSingleSlot();
+
+        new Context() {
+            {
+                resourceActionsBuilder.setAllocateResourceConsumer(
+                        tuple6 -> allocateResourceCalls.incrementAndGet());
+                runTest(
+                        () -> {
+                            getSlotManager().processResourceRequirements(requirements);
+
+                            getSlotManager().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(1));
+                        });
+            }
+        };
+    }
+
+    /**
+     * Tests that a resource allocated for one job can be allocated for another job after being
+     * freed.
+     */
+    @Test
+    public void testResourceCanBeAllocatedForDifferentJobAfterFree() throws Exception {
+        testResourceCanBeAllocatedForDifferentJobAfterFree(
+                SecondRequirementDeclarationTime.BEFORE_FREE);
+        testResourceCanBeAllocatedForDifferentJobAfterFree(
+                SecondRequirementDeclarationTime.AFTER_FREE);
+    }
+
+    private enum SecondRequirementDeclarationTime {
+        BEFORE_FREE,
+        AFTER_FREE
+    }
+
+    private void testResourceCanBeAllocatedForDifferentJobAfterFree(
+            SecondRequirementDeclarationTime secondRequirementDeclarationTime) throws Exception {
+        final CompletableFuture<AllocationID> allocationId1 = new CompletableFuture<>();
+        final CompletableFuture<AllocationID> allocationId2 = new CompletableFuture<>();
+        final ResourceRequirements resourceRequirements1 =
+                createResourceRequirementsForSingleSlot();
+        final ResourceRequirements resourceRequirements2 =
+                createResourceRequirementsForSingleSlot();
+
+        final TaskExecutorGateway taskExecutorGateway =
+                new TestingTaskExecutorGatewayBuilder()
+                        .setRequestSlotFunction(
+                                tuple6 -> {
+                                    if (!allocationId1.isDone()) {
+                                        allocationId1.complete(tuple6.f2);
+                                    } else {
+                                        allocationId2.complete(tuple6.f2);
+                                    }
+                                    return CompletableFuture.completedFuture(Acknowledge.get());
+                                })
+                        .createTestingTaskExecutorGateway();
+        final ResourceID resourceID = ResourceID.generate();
+        final TaskExecutorConnection taskManagerConnection =
+                new TaskExecutorConnection(resourceID, taskExecutorGateway);
+        final SlotReport slotReport = new SlotReport();
+
+        new Context() {
+            {
+                runTest(
+                        () -> {
+                            getSlotManager()
+                                    .registerTaskManager(
+                                            taskManagerConnection,
+                                            slotReport,
+                                            getDefaultTaskManagerResourceProfile(),
+                                            getDefaultSlotResourceProfile());
+
+                            getSlotManager().processResourceRequirements(resourceRequirements1);
+
+                            TaskManagerSlotInformation slot =
+                                    getTaskManagerTracker()
+                                            .getAllocatedOrPendingSlot(allocationId1.get())
+                                            .get();
+
+                            assertEquals(
+                                    "The slot has not been allocated to the expected job id.",
+                                    resourceRequirements1.getJobId(),
+                                    slot.getJobId());
+
+                            if (secondRequirementDeclarationTime
+                                    == SecondRequirementDeclarationTime.BEFORE_FREE) {
+                                getSlotManager().processResourceRequirements(resourceRequirements2);
+                            }
+
+                            // clear resource requirements first so that the freed slot isn't
+                            // immediately re-assigned to the job
+                            getSlotManager()
+                                    .processResourceRequirements(
+                                            ResourceRequirements.create(
+                                                    resourceRequirements1.getJobId(),
+                                                    resourceRequirements1.getTargetAddress(),
+                                                    Collections.emptyList()));
+                            getSlotManager()
+                                    .freeSlot(
+                                            SlotID.getDynamicSlotID(resourceID),
+                                            allocationId1.get());
+
+                            if (secondRequirementDeclarationTime
+                                    == SecondRequirementDeclarationTime.AFTER_FREE) {
+                                getSlotManager().processResourceRequirements(resourceRequirements2);
+                            }
+
+                            slot =
+                                    getTaskManagerTracker()
+                                            .getAllocatedOrPendingSlot(allocationId2.get())
+                                            .get();
+                            assertEquals(
+                                    "The slot has not been allocated to the expected job id.",
+                                    resourceRequirements2.getJobId(),
+                                    slot.getJobId());
+                        });
+            }
+        };
+    }
+
+    /**
+     * Tests that we only request new resources/containers once we have assigned all pending task
+     * managers.
+     */
+    @Test
+    public void testRequestNewResources() throws Exception {
+        final AtomicInteger resourceRequests = new AtomicInteger(0);
+        final JobID jobId = new JobID();
+
+        new Context() {
+            {
+                resourceActionsBuilder.setAllocateResourceFunction(
+                        ignored -> {
+                            resourceRequests.incrementAndGet();
+                            return true;
+                        });
+                runTest(
+                        () -> {
+                            // the first requirements should be fulfillable with the pending task
+                            // managers of the first allocation
+                            getSlotManager()
+                                    .processResourceRequirements(
+                                            createResourceRequirements(
+                                                    jobId, getDefaultNumberSlotsPerWorker()));
+                            assertThat(resourceRequests.get(), is(1));
+
+                            getSlotManager()
+                                    .processResourceRequirements(
+                                            createResourceRequirements(
+                                                    jobId, getDefaultNumberSlotsPerWorker() + 1));
+                            assertThat(resourceRequests.get(), is(2));
+                        });
+            }
+        };
+    }
+
+    /**
+     * Test that the slot manager only allocates new workers if their worker spec can fulfill the
+     * requested resource profile.
+     */
+    @Test
+    public void testWorkerOnlyAllocatedIfRequestedSlotCouldBeFulfilled() throws Exception {
+        final AtomicInteger resourceRequests = new AtomicInteger(0);
+
+        new Context() {
+            {
+                resourceActionsBuilder.setAllocateResourceFunction(
+                        ignored -> {
+                            resourceRequests.incrementAndGet();
+                            return true;
+                        });
+                runTest(
+                        () -> {
+                            getSlotManager()
+                                    .processResourceRequirements(
+                                            createResourceRequirements(
+                                                    new JobID(), 1, getLargeSlotResourceProfile()));

Review comment:
       Whether large slot profile can be fulfilled depends on the strategy. I think this test case should be moved to `FineGrainedSlotManagerITCase`.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/AbstractFineGrainedSlotManagerITCase.java
##########
@@ -0,0 +1,672 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.Tuple6;
+import org.apache.flink.core.testutils.ManuallyTriggeredScheduledExecutorService;
+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.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.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.testutils.SystemExitTrackingSecurityManager;
+import org.apache.flink.util.function.FunctionUtils;
+
+import akka.pattern.AskTimeoutException;
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Supplier;
+
+import static org.hamcrest.Matchers.empty;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.is;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertThat;
+
+/** IT Cases of {@link FineGrainedSlotManager}. */
+public abstract class AbstractFineGrainedSlotManagerITCase extends FineGrainedSlotManagerTestBase {
+
+    // ---------------------------------------------------------------------------------------------
+    // Requirement declaration
+    // ---------------------------------------------------------------------------------------------
+
+    /**
+     * Tests that a requirement declaration with no free slots will trigger the resource allocation.
+     */
+    @Test
+    public void testRequirementDeclarationWithoutFreeSlotsTriggersWorkerAllocation()
+            throws Exception {
+        final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot();
+
+        final CompletableFuture<WorkerResourceSpec> allocateResourceFuture =
+                new CompletableFuture<>();
+        new Context() {
+            {
+                resourceActionsBuilder.setAllocateResourceConsumer(
+                        allocateResourceFuture::complete);
+                runTest(
+                        () -> {
+                            getSlotManager().processResourceRequirements(resourceRequirements);
+
+                            allocateResourceFuture.get();
+                        });
+            }
+        };
+    }
+
+    /**
+     * Tests that resources continue to be considered missing if we cannot allocate more resources.
+     */
+    @Test
+    public void testRequirementDeclarationWithResourceAllocationFailure() throws Exception {
+        final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot();
+        new Context() {
+            {
+                resourceActionsBuilder.setAllocateResourceFunction(value -> false);
+                runTest(
+                        () -> {
+                            getSlotManager().processResourceRequirements(resourceRequirements);
+
+                            final JobID jobId = resourceRequirements.getJobId();
+                            assertThat(
+                                    getTotalResourceCount(
+                                            getResourceTracker().getMissingResources().get(jobId)),
+                                    is(1));
+                        });
+            }
+        };
+    }
+
+    /** Tests that resource requirements can be fulfilled with resource that are currently free. */
+    @Test
+    public void testRequirementDeclarationWithFreeResource() throws Exception {
+        testRequirementDeclaration(
+                RequirementDeclarationScenario
+                        .TASK_EXECUTOR_REGISTRATION_BEFORE_REQUIREMENT_DECLARATION);
+    }
+
+    /**
+     * Tests that resource requirements can be fulfilled with resource that are registered after the
+     * requirement declaration.
+     */
+    @Test
+    public void testRequirementDeclarationWithPendingResource() throws Exception {
+        testRequirementDeclaration(
+                RequirementDeclarationScenario
+                        .TASK_EXECUTOR_REGISTRATION_AFTER_REQUIREMENT_DECLARATION);
+    }
+
+    private enum RequirementDeclarationScenario {
+        // Tests that a slot request which can be fulfilled will trigger a slot allocation
+        TASK_EXECUTOR_REGISTRATION_BEFORE_REQUIREMENT_DECLARATION,
+        // Tests that pending slot requests are tried to be fulfilled upon new slot registrations
+        TASK_EXECUTOR_REGISTRATION_AFTER_REQUIREMENT_DECLARATION
+    }
+
+    private void testRequirementDeclaration(RequirementDeclarationScenario scenario)
+            throws Exception {
+        final ResourceID resourceID = ResourceID.generate();
+        final JobID jobId = new JobID();
+        final SlotID slotId = SlotID.getDynamicSlotID(resourceID);
+        final String targetAddress = "localhost";
+        final ResourceRequirements requirements =
+                ResourceRequirements.create(
+                        jobId,
+                        targetAddress,
+                        Collections.singleton(
+                                ResourceRequirement.create(getDefaultSlotResourceProfile(), 1)));
+
+        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);
+                                    return CompletableFuture.completedFuture(Acknowledge.get());
+                                })
+                        .createTestingTaskExecutorGateway();
+
+        final TaskExecutorConnection taskExecutorConnection =
+                new TaskExecutorConnection(resourceID, taskExecutorGateway);
+
+        new Context() {
+            {
+                runTest(
+                        () -> {
+                            if (scenario
+                                    == RequirementDeclarationScenario
+                                            .TASK_EXECUTOR_REGISTRATION_BEFORE_REQUIREMENT_DECLARATION) {
+                                getSlotManager()
+                                        .registerTaskManager(
+                                                taskExecutorConnection,
+                                                new SlotReport(),
+                                                getDefaultTaskManagerResourceProfile(),
+                                                getDefaultSlotResourceProfile());
+                            }
+
+                            getSlotManager().processResourceRequirements(requirements);
+
+                            if (scenario
+                                    == RequirementDeclarationScenario
+                                            .TASK_EXECUTOR_REGISTRATION_AFTER_REQUIREMENT_DECLARATION) {
+                                getSlotManager()
+                                        .registerTaskManager(
+                                                taskExecutorConnection,
+                                                new SlotReport(),
+                                                getDefaultTaskManagerResourceProfile(),
+                                                getDefaultSlotResourceProfile());
+                            }
+
+                            assertThat(
+                                    requestFuture.get(),
+                                    is(
+                                            equalTo(
+                                                    Tuple6.of(
+                                                            slotId,
+                                                            jobId,
+                                                            requestFuture.get().f2,
+                                                            getDefaultSlotResourceProfile(),
+                                                            targetAddress,
+                                                            getResourceManagerId()))));
+
+                            final TaskManagerSlotInformation slot =
+                                    getTaskManagerTracker()
+                                            .getAllocatedOrPendingSlot(requestFuture.get().f2)
+                                            .get();
+
+                            assertEquals(
+                                    "The slot has not been allocated to the expected allocation id.",
+                                    requestFuture.get().f2,
+                                    slot.getAllocationId());
+                        });
+            }
+        };
+    }
+
+    /**
+     * 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 AtomicInteger allocateResourceCalls = new AtomicInteger(0);
+        final ResourceRequirements requirements = createResourceRequirementsForSingleSlot();
+
+        new Context() {
+            {
+                resourceActionsBuilder.setAllocateResourceConsumer(
+                        tuple6 -> allocateResourceCalls.incrementAndGet());
+                runTest(
+                        () -> {
+                            getSlotManager().processResourceRequirements(requirements);
+
+                            getSlotManager().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(1));
+                        });
+            }
+        };
+    }
+
+    /**
+     * Tests that a resource allocated for one job can be allocated for another job after being
+     * freed.
+     */
+    @Test
+    public void testResourceCanBeAllocatedForDifferentJobAfterFree() throws Exception {
+        testResourceCanBeAllocatedForDifferentJobAfterFree(
+                SecondRequirementDeclarationTime.BEFORE_FREE);
+        testResourceCanBeAllocatedForDifferentJobAfterFree(
+                SecondRequirementDeclarationTime.AFTER_FREE);
+    }
+
+    private enum SecondRequirementDeclarationTime {
+        BEFORE_FREE,
+        AFTER_FREE
+    }
+
+    private void testResourceCanBeAllocatedForDifferentJobAfterFree(
+            SecondRequirementDeclarationTime secondRequirementDeclarationTime) throws Exception {
+        final CompletableFuture<AllocationID> allocationId1 = new CompletableFuture<>();
+        final CompletableFuture<AllocationID> allocationId2 = new CompletableFuture<>();
+        final ResourceRequirements resourceRequirements1 =
+                createResourceRequirementsForSingleSlot();
+        final ResourceRequirements resourceRequirements2 =
+                createResourceRequirementsForSingleSlot();
+
+        final TaskExecutorGateway taskExecutorGateway =
+                new TestingTaskExecutorGatewayBuilder()
+                        .setRequestSlotFunction(
+                                tuple6 -> {
+                                    if (!allocationId1.isDone()) {
+                                        allocationId1.complete(tuple6.f2);
+                                    } else {
+                                        allocationId2.complete(tuple6.f2);
+                                    }
+                                    return CompletableFuture.completedFuture(Acknowledge.get());
+                                })
+                        .createTestingTaskExecutorGateway();
+        final ResourceID resourceID = ResourceID.generate();
+        final TaskExecutorConnection taskManagerConnection =
+                new TaskExecutorConnection(resourceID, taskExecutorGateway);
+        final SlotReport slotReport = new SlotReport();
+
+        new Context() {
+            {
+                runTest(
+                        () -> {
+                            getSlotManager()
+                                    .registerTaskManager(
+                                            taskManagerConnection,
+                                            slotReport,
+                                            getDefaultTaskManagerResourceProfile(),
+                                            getDefaultSlotResourceProfile());
+
+                            getSlotManager().processResourceRequirements(resourceRequirements1);
+
+                            TaskManagerSlotInformation slot =
+                                    getTaskManagerTracker()
+                                            .getAllocatedOrPendingSlot(allocationId1.get())
+                                            .get();
+
+                            assertEquals(
+                                    "The slot has not been allocated to the expected job id.",
+                                    resourceRequirements1.getJobId(),
+                                    slot.getJobId());
+
+                            if (secondRequirementDeclarationTime
+                                    == SecondRequirementDeclarationTime.BEFORE_FREE) {
+                                getSlotManager().processResourceRequirements(resourceRequirements2);
+                            }
+
+                            // clear resource requirements first so that the freed slot isn't
+                            // immediately re-assigned to the job
+                            getSlotManager()
+                                    .processResourceRequirements(
+                                            ResourceRequirements.create(
+                                                    resourceRequirements1.getJobId(),
+                                                    resourceRequirements1.getTargetAddress(),
+                                                    Collections.emptyList()));
+                            getSlotManager()
+                                    .freeSlot(
+                                            SlotID.getDynamicSlotID(resourceID),
+                                            allocationId1.get());
+
+                            if (secondRequirementDeclarationTime
+                                    == SecondRequirementDeclarationTime.AFTER_FREE) {
+                                getSlotManager().processResourceRequirements(resourceRequirements2);
+                            }
+
+                            slot =
+                                    getTaskManagerTracker()
+                                            .getAllocatedOrPendingSlot(allocationId2.get())
+                                            .get();
+                            assertEquals(
+                                    "The slot has not been allocated to the expected job id.",
+                                    resourceRequirements2.getJobId(),
+                                    slot.getJobId());
+                        });
+            }
+        };
+    }
+
+    /**
+     * Tests that we only request new resources/containers once we have assigned all pending task
+     * managers.
+     */
+    @Test
+    public void testRequestNewResources() throws Exception {
+        final AtomicInteger resourceRequests = new AtomicInteger(0);
+        final JobID jobId = new JobID();
+
+        new Context() {
+            {
+                resourceActionsBuilder.setAllocateResourceFunction(
+                        ignored -> {
+                            resourceRequests.incrementAndGet();
+                            return true;
+                        });
+                runTest(
+                        () -> {
+                            // the first requirements should be fulfillable with the pending task
+                            // managers of the first allocation
+                            getSlotManager()
+                                    .processResourceRequirements(
+                                            createResourceRequirements(
+                                                    jobId, getDefaultNumberSlotsPerWorker()));
+                            assertThat(resourceRequests.get(), is(1));
+
+                            getSlotManager()
+                                    .processResourceRequirements(
+                                            createResourceRequirements(
+                                                    jobId, getDefaultNumberSlotsPerWorker() + 1));
+                            assertThat(resourceRequests.get(), is(2));
+                        });
+            }
+        };
+    }
+
+    /**
+     * Test that the slot manager only allocates new workers if their worker spec can fulfill the
+     * requested resource profile.
+     */
+    @Test
+    public void testWorkerOnlyAllocatedIfRequestedSlotCouldBeFulfilled() throws Exception {
+        final AtomicInteger resourceRequests = new AtomicInteger(0);
+
+        new Context() {
+            {
+                resourceActionsBuilder.setAllocateResourceFunction(
+                        ignored -> {
+                            resourceRequests.incrementAndGet();
+                            return true;
+                        });
+                runTest(
+                        () -> {
+                            getSlotManager()
+                                    .processResourceRequirements(
+                                            createResourceRequirements(
+                                                    new JobID(), 1, getLargeSlotResourceProfile()));
+                            assertThat(resourceRequests.get(), is(0));
+                        });
+            }
+        };
+    }
+
+    // ---------------------------------------------------------------------------------------------
+    // Slot allocation failure handling
+    // ---------------------------------------------------------------------------------------------
+
+    /** 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 BlockingQueue<Supplier<CompletableFuture<Acknowledge>>> responseQueue =
+                new ArrayBlockingQueue<>(2);
+        responseQueue.add(
+                () -> FutureUtils.completedExceptionally(new AskTimeoutException("timeout")));
+        responseQueue.add(
+                () -> {
+                    secondSlotRequestFuture.complete(null);
+                    return new CompletableFuture<>();
+                });
+
+        final TaskExecutorGateway taskExecutorGateway =
+                new TestingTaskExecutorGatewayBuilder()
+                        .setRequestSlotFunction(ignored -> responseQueue.remove().get())
+                        .createTestingTaskExecutorGateway();
+        final ResourceID resourceID = ResourceID.generate();
+        final TaskExecutorConnection taskManagerConnection =
+                new TaskExecutorConnection(resourceID, taskExecutorGateway);
+        final SlotReport slotReport = new SlotReport();
+
+        new Context() {
+            {
+                runTest(
+                        () -> {
+                            getSlotManager()
+                                    .registerTaskManager(
+                                            taskManagerConnection,
+                                            slotReport,
+                                            getDefaultTaskManagerResourceProfile(),
+                                            getDefaultSlotResourceProfile());
+                            getSlotManager()
+                                    .processResourceRequirements(
+                                            createResourceRequirementsForSingleSlot());
+
+                            // a second request is only sent if the first request timed out
+                            secondSlotRequestFuture.get();
+                        });
+            }
+        };
+    }
+
+    /**
+     * Tests that the SlotManager retries allocating a slot if the TaskExecutor#requestSlot call
+     * fails.
+     */
+    @Test
+    public void testSlotRequestFailure() throws Exception {
+        final ManuallyTriggeredScheduledExecutorService executor =
+                new ManuallyTriggeredScheduledExecutorService();
+
+        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<AllocationID> allocationIds = new ArrayBlockingQueue<>(2);
+
+        final TestingTaskExecutorGateway taskExecutorGateway =
+                new TestingTaskExecutorGatewayBuilder()
+                        .setRequestSlotFunction(
+                                FunctionUtils.uncheckedFunction(
+                                        requestSlotParameters -> {
+                                            allocationIds.put(requestSlotParameters.f2);
+                                            return slotRequestFutureIterator.next();
+                                        }))
+                        .createTestingTaskExecutorGateway();
+
+        final ResourceID resourceId = ResourceID.generate();
+        final TaskExecutorConnection taskManagerConnection =
+                new TaskExecutorConnection(resourceId, taskExecutorGateway);
+        final SlotReport slotReport = new SlotReport();
+
+        new Context() {
+            {
+                mainThreadExecutor = executor;
+                runTest(
+                        () -> {
+                            getSlotManager()
+                                    .registerTaskManager(
+                                            taskManagerConnection,
+                                            slotReport,
+                                            getDefaultTaskManagerResourceProfile(),
+                                            getDefaultSlotResourceProfile());
+
+                            getSlotManager().processResourceRequirements(resourceRequirements);
+
+                            final AllocationID firstAllocationId = allocationIds.take();
+                            assertThat(allocationIds, is(empty()));
+
+                            // let the first attempt fail --> this should trigger a second attempt
+                            slotRequestFuture1.completeExceptionally(
+                                    new SlotAllocationException("Test exception."));
+
+                            executor.triggerAll();
+                            assertThat(
+                                    getTotalResourceCount(
+                                            getResourceTracker().getAcquiredResources(jobId)),
+                                    is(1));
+
+                            slotRequestFuture2.complete(Acknowledge.get());
+                            final AllocationID secondAllocationId = allocationIds.take();
+                            assertThat(allocationIds, is(empty()));
+
+                            executor.triggerAll();
+                            final TaskManagerSlotInformation slot =
+                                    getTaskManagerTracker()
+                                            .getAllocatedOrPendingSlot(secondAllocationId)
+                                            .get();
+
+                            assertThat(slot.getState(), is(SlotState.ALLOCATED));
+                            assertEquals(jobId, slot.getJobId());
+
+                            assertFalse(
+                                    getTaskManagerTracker()
+                                            .getAllocatedOrPendingSlot(firstAllocationId)
+                                            .isPresent());
+                        });
+            }
+        };
+    }
+
+    // ---------------------------------------------------------------------------------------------
+    // Allocation update
+    // ---------------------------------------------------------------------------------------------
+
+    @Test
+    public void testAllocationUpdatesIgnoredIfTaskExecutorUnregistered() throws Exception {

Review comment:
       It's a bit unclear what this test case verifies.
   
   IIUC, this is to verify that the system does not breakdown when the task manager is unregistered before the ack of request slot is received? I think this needs to be explained in javadoc.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/AbstractFineGrainedSlotManagerITCase.java
##########
@@ -0,0 +1,672 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.Tuple6;
+import org.apache.flink.core.testutils.ManuallyTriggeredScheduledExecutorService;
+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.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.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.testutils.SystemExitTrackingSecurityManager;
+import org.apache.flink.util.function.FunctionUtils;
+
+import akka.pattern.AskTimeoutException;
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Supplier;
+
+import static org.hamcrest.Matchers.empty;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.is;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertThat;
+
+/** IT Cases of {@link FineGrainedSlotManager}. */
+public abstract class AbstractFineGrainedSlotManagerITCase extends FineGrainedSlotManagerTestBase {
+
+    // ---------------------------------------------------------------------------------------------
+    // Requirement declaration
+    // ---------------------------------------------------------------------------------------------
+
+    /**
+     * Tests that a requirement declaration with no free slots will trigger the resource allocation.
+     */
+    @Test
+    public void testRequirementDeclarationWithoutFreeSlotsTriggersWorkerAllocation()
+            throws Exception {
+        final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot();
+
+        final CompletableFuture<WorkerResourceSpec> allocateResourceFuture =
+                new CompletableFuture<>();
+        new Context() {
+            {
+                resourceActionsBuilder.setAllocateResourceConsumer(
+                        allocateResourceFuture::complete);
+                runTest(
+                        () -> {
+                            getSlotManager().processResourceRequirements(resourceRequirements);
+
+                            allocateResourceFuture.get();
+                        });
+            }
+        };
+    }
+
+    /**
+     * Tests that resources continue to be considered missing if we cannot allocate more resources.
+     */
+    @Test
+    public void testRequirementDeclarationWithResourceAllocationFailure() throws Exception {
+        final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot();
+        new Context() {
+            {
+                resourceActionsBuilder.setAllocateResourceFunction(value -> false);
+                runTest(
+                        () -> {
+                            getSlotManager().processResourceRequirements(resourceRequirements);
+
+                            final JobID jobId = resourceRequirements.getJobId();
+                            assertThat(
+                                    getTotalResourceCount(
+                                            getResourceTracker().getMissingResources().get(jobId)),
+                                    is(1));
+                        });
+            }
+        };
+    }
+
+    /** Tests that resource requirements can be fulfilled with resource that are currently free. */
+    @Test
+    public void testRequirementDeclarationWithFreeResource() throws Exception {
+        testRequirementDeclaration(
+                RequirementDeclarationScenario
+                        .TASK_EXECUTOR_REGISTRATION_BEFORE_REQUIREMENT_DECLARATION);
+    }
+
+    /**
+     * Tests that resource requirements can be fulfilled with resource that are registered after the
+     * requirement declaration.
+     */
+    @Test
+    public void testRequirementDeclarationWithPendingResource() throws Exception {
+        testRequirementDeclaration(
+                RequirementDeclarationScenario
+                        .TASK_EXECUTOR_REGISTRATION_AFTER_REQUIREMENT_DECLARATION);
+    }
+
+    private enum RequirementDeclarationScenario {
+        // Tests that a slot request which can be fulfilled will trigger a slot allocation
+        TASK_EXECUTOR_REGISTRATION_BEFORE_REQUIREMENT_DECLARATION,
+        // Tests that pending slot requests are tried to be fulfilled upon new slot registrations
+        TASK_EXECUTOR_REGISTRATION_AFTER_REQUIREMENT_DECLARATION
+    }
+
+    private void testRequirementDeclaration(RequirementDeclarationScenario scenario)
+            throws Exception {
+        final ResourceID resourceID = ResourceID.generate();
+        final JobID jobId = new JobID();
+        final SlotID slotId = SlotID.getDynamicSlotID(resourceID);
+        final String targetAddress = "localhost";
+        final ResourceRequirements requirements =
+                ResourceRequirements.create(
+                        jobId,
+                        targetAddress,
+                        Collections.singleton(
+                                ResourceRequirement.create(getDefaultSlotResourceProfile(), 1)));
+
+        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);
+                                    return CompletableFuture.completedFuture(Acknowledge.get());
+                                })
+                        .createTestingTaskExecutorGateway();
+
+        final TaskExecutorConnection taskExecutorConnection =
+                new TaskExecutorConnection(resourceID, taskExecutorGateway);
+
+        new Context() {
+            {
+                runTest(
+                        () -> {
+                            if (scenario
+                                    == RequirementDeclarationScenario
+                                            .TASK_EXECUTOR_REGISTRATION_BEFORE_REQUIREMENT_DECLARATION) {
+                                getSlotManager()
+                                        .registerTaskManager(
+                                                taskExecutorConnection,
+                                                new SlotReport(),
+                                                getDefaultTaskManagerResourceProfile(),
+                                                getDefaultSlotResourceProfile());
+                            }
+
+                            getSlotManager().processResourceRequirements(requirements);
+
+                            if (scenario
+                                    == RequirementDeclarationScenario
+                                            .TASK_EXECUTOR_REGISTRATION_AFTER_REQUIREMENT_DECLARATION) {
+                                getSlotManager()
+                                        .registerTaskManager(
+                                                taskExecutorConnection,
+                                                new SlotReport(),
+                                                getDefaultTaskManagerResourceProfile(),
+                                                getDefaultSlotResourceProfile());
+                            }
+
+                            assertThat(
+                                    requestFuture.get(),
+                                    is(
+                                            equalTo(
+                                                    Tuple6.of(
+                                                            slotId,
+                                                            jobId,
+                                                            requestFuture.get().f2,
+                                                            getDefaultSlotResourceProfile(),
+                                                            targetAddress,
+                                                            getResourceManagerId()))));
+
+                            final TaskManagerSlotInformation slot =
+                                    getTaskManagerTracker()
+                                            .getAllocatedOrPendingSlot(requestFuture.get().f2)
+                                            .get();
+
+                            assertEquals(
+                                    "The slot has not been allocated to the expected allocation id.",
+                                    requestFuture.get().f2,
+                                    slot.getAllocationId());
+                        });
+            }
+        };
+    }
+
+    /**
+     * 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 AtomicInteger allocateResourceCalls = new AtomicInteger(0);
+        final ResourceRequirements requirements = createResourceRequirementsForSingleSlot();
+
+        new Context() {
+            {
+                resourceActionsBuilder.setAllocateResourceConsumer(
+                        tuple6 -> allocateResourceCalls.incrementAndGet());
+                runTest(
+                        () -> {
+                            getSlotManager().processResourceRequirements(requirements);
+
+                            getSlotManager().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(1));
+                        });
+            }
+        };
+    }
+
+    /**
+     * Tests that a resource allocated for one job can be allocated for another job after being
+     * freed.
+     */
+    @Test
+    public void testResourceCanBeAllocatedForDifferentJobAfterFree() throws Exception {
+        testResourceCanBeAllocatedForDifferentJobAfterFree(
+                SecondRequirementDeclarationTime.BEFORE_FREE);
+        testResourceCanBeAllocatedForDifferentJobAfterFree(
+                SecondRequirementDeclarationTime.AFTER_FREE);
+    }

Review comment:
       Lets split this into two test cases.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/FineGrainedSlotManagerTest.java
##########
@@ -328,667 +327,162 @@ private void testRequirementDeclaration(RequirementDeclarationScenario scenario)
                                 ResourceProfile,
                                 String,
                                 ResourceManagerId>>
-                requestFuture = new CompletableFuture<>();
-        // accept an incoming slot request
-        final TaskExecutorGateway taskExecutorGateway =
-                new TestingTaskExecutorGatewayBuilder()
-                        .setRequestSlotFunction(
-                                tuple6 -> {
-                                    requestFuture.complete(tuple6);
-                                    return CompletableFuture.completedFuture(Acknowledge.get());
-                                })
-                        .createTestingTaskExecutorGateway();
-
-        final TaskExecutorConnection taskExecutorConnection =
-                new TaskExecutorConnection(resourceID, taskExecutorGateway);
-
-        final TaskManagerTracker tracker = new FineGrainedTaskManagerTracker();
-        try (FineGrainedSlotManager slotManager =
-                createFineGrainedSlotManagerBuilder()
-                        .setTaskManagerTracker(tracker)
-                        .buildAndStartWithDirectExec(
-                                resourceManagerId, new TestingResourceActionsBuilder().build())) {
-
-            if (scenario
-                    == RequirementDeclarationScenario
-                            .TASK_EXECUTOR_REGISTRATION_BEFORE_REQUIREMENT_DECLARATION) {
-                slotManager.registerTaskManager(
-                        taskExecutorConnection,
-                        new SlotReport(),
-                        DEFAULT_TOTAL_RESOURCE_PROFILE,
-                        DEFAULT_SLOT_RESOURCE_PROFILE);
-            }
-
-            final ResourceRequirements requirements =
-                    ResourceRequirements.create(
-                            jobId,
-                            targetAddress,
-                            Collections.singleton(ResourceRequirement.create(resourceProfile, 1)));
-            slotManager.processResourceRequirements(requirements);
-
-            if (scenario
-                    == RequirementDeclarationScenario
-                            .TASK_EXECUTOR_REGISTRATION_AFTER_REQUIREMENT_DECLARATION) {
-                slotManager.registerTaskManager(
-                        taskExecutorConnection,
-                        new SlotReport(),
-                        DEFAULT_TOTAL_RESOURCE_PROFILE,
-                        DEFAULT_SLOT_RESOURCE_PROFILE);
-            }
-
-            assertThat(
-                    requestFuture.get(),
-                    is(
-                            equalTo(
-                                    Tuple6.of(
-                                            slotId,
-                                            jobId,
-                                            requestFuture.get().f2,
-                                            resourceProfile,
-                                            targetAddress,
-                                            resourceManagerId))));
-
-            final TaskManagerSlotInformation slot =
-                    tracker.getAllocatedOrPendingSlot(requestFuture.get().f2).get();
-
-            assertEquals(
-                    "The slot has not been allocated to the expected allocation id.",
-                    requestFuture.get().f2,
-                    slot.getAllocationId());
-        }
-    }
-
-    /**
-     * Tests that freeing a slot will correctly reset the slot and give back the resource of it to
-     * task manager.
-     */
-    @Test
-    public void testFreeSlot() throws Exception {
-        final ResourceID resourceId = ResourceID.generate();
-        final TaskExecutorGateway taskExecutorGateway =
-                new TestingTaskExecutorGatewayBuilder().createTestingTaskExecutorGateway();
-        final TaskExecutorConnection taskExecutorConnection =
-                new TaskExecutorConnection(resourceId, taskExecutorGateway);
-        final AllocationID allocationId = new AllocationID();
-        final ResourceProfile resourceProfile = ResourceProfile.fromResources(42.0, 1337);
-
-        final SlotReport slotReport =
-                new SlotReport(createAllocatedSlotStatus(allocationId, resourceProfile));
-
-        final TaskManagerTracker tracker = new FineGrainedTaskManagerTracker();
-        try (FineGrainedSlotManager slotManager =
-                createFineGrainedSlotManagerBuilder()
-                        .setTaskManagerTracker(tracker)
-                        .buildAndStartWithDirectExec()) {
-
-            slotManager.registerTaskManager(
-                    taskExecutorConnection, slotReport, resourceProfile, resourceProfile);
-
-            final TaskManagerSlotInformation slot =
-                    tracker.getAllocatedOrPendingSlot(allocationId).get();
-
-            assertSame(SlotState.ALLOCATED, slot.getState());
-            assertEquals(
-                    ResourceProfile.ZERO,
-                    tracker.getRegisteredTaskManager(taskExecutorConnection.getInstanceID())
-                            .get()
-                            .getAvailableResource());
-
-            slotManager.freeSlot(new SlotID(resourceId, 0), allocationId);
-
-            assertFalse(tracker.getAllocatedOrPendingSlot(allocationId).isPresent());
-
-            assertEquals(
-                    resourceProfile,
-                    tracker.getRegisteredTaskManager(taskExecutorConnection.getInstanceID())
-                            .get()
-                            .getAvailableResource());
-        }
-    }
-
-    /**
-     * 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 CompletableFuture<AllocationID> allocationId = new CompletableFuture<>();
-        final AtomicInteger allocateResourceCalls = new AtomicInteger(0);
-        final ResourceActions resourceManagerActions =
-                new TestingResourceActionsBuilder()
-                        .setAllocateResourceConsumer(
-                                tuple6 -> allocateResourceCalls.incrementAndGet())
-                        .build();
-        final ResourceRequirements requirements = createResourceRequirementsForSingleSlot();
-
+                requestSlotFuture = new CompletableFuture<>();
         final TaskExecutorGateway taskExecutorGateway =
                 new TestingTaskExecutorGatewayBuilder()
                         .setRequestSlotFunction(
                                 tuple6 -> {
-                                    allocationId.complete(tuple6.f2);
+                                    requestSlotFuture.complete(tuple6);
                                     return CompletableFuture.completedFuture(Acknowledge.get());
                                 })
                         .createTestingTaskExecutorGateway();
-
-        final ResourceID resourceId = ResourceID.generate();
-
         final TaskExecutorConnection taskManagerConnection =
-                new TaskExecutorConnection(resourceId, taskExecutorGateway);
+                new TaskExecutorConnection(ResourceID.generate(), taskExecutorGateway);
+        final JobID jobId = new JobID();
         final SlotReport slotReport = new SlotReport();
-
-        final TaskManagerTracker tracker = new FineGrainedTaskManagerTracker();
-        try (FineGrainedSlotManager slotManager =
-                createFineGrainedSlotManagerBuilder()
-                        .setTaskManagerTracker(tracker)
-                        .buildAndStartWithDirectExec(
-                                ResourceManagerId.generate(), resourceManagerActions)) {
-
-            slotManager.registerTaskManager(
-                    taskManagerConnection,
-                    slotReport,
-                    DEFAULT_TOTAL_RESOURCE_PROFILE,
-                    DEFAULT_SLOT_RESOURCE_PROFILE);
-
-            slotManager.processResourceRequirements(requirements);
-
-            final TaskManagerSlotInformation slot =
-                    tracker.getAllocatedOrPendingSlot(allocationId.get()).get();
-
-            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));
+        new Context() {
+            {
+                resourceAllocationStrategyBuilder =
+                        TestingResourceAllocationStrategy.newBuilder()
+                                .setTryFulfillRequirementsFunction(
+                                        ((jobIDCollectionMap,
+                                                instanceIDTuple2Map,
+                                                pendingTaskManagers) ->
+                                                ResourceAllocationResult.builder()
+                                                        .addAllocationOnRegisteredResource(
+                                                                jobId,
+                                                                taskManagerConnection
+                                                                        .getInstanceID(),
+                                                                getDefaultSlotResourceProfile())
+                                                        .build()));
+                runTest(
+                        () -> {
+                            getSlotManager()
+                                    .registerTaskManager(
+                                            taskManagerConnection,
+                                            slotReport,
+                                            getDefaultTaskManagerResourceProfile(),
+                                            getDefaultSlotResourceProfile());
+                            getSlotManager()
+                                    .processResourceRequirements(
+                                            createResourceRequirements(jobId, 1));
+                            final Tuple6<
+                                            SlotID,
+                                            JobID,
+                                            AllocationID,
+                                            ResourceProfile,
+                                            String,
+                                            ResourceManagerId>
+                                    requestSlot = requestSlotFuture.get();

Review comment:
       Please also check this for other test cases.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/AbstractFineGrainedSlotManagerITCase.java
##########
@@ -0,0 +1,672 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.Tuple6;
+import org.apache.flink.core.testutils.ManuallyTriggeredScheduledExecutorService;
+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.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.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.testutils.SystemExitTrackingSecurityManager;
+import org.apache.flink.util.function.FunctionUtils;
+
+import akka.pattern.AskTimeoutException;
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Supplier;
+
+import static org.hamcrest.Matchers.empty;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.is;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertThat;
+
+/** IT Cases of {@link FineGrainedSlotManager}. */
+public abstract class AbstractFineGrainedSlotManagerITCase extends FineGrainedSlotManagerTestBase {
+
+    // ---------------------------------------------------------------------------------------------
+    // Requirement declaration
+    // ---------------------------------------------------------------------------------------------
+
+    /**
+     * Tests that a requirement declaration with no free slots will trigger the resource allocation.
+     */
+    @Test
+    public void testRequirementDeclarationWithoutFreeSlotsTriggersWorkerAllocation()
+            throws Exception {
+        final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot();
+
+        final CompletableFuture<WorkerResourceSpec> allocateResourceFuture =
+                new CompletableFuture<>();
+        new Context() {
+            {
+                resourceActionsBuilder.setAllocateResourceConsumer(
+                        allocateResourceFuture::complete);
+                runTest(
+                        () -> {
+                            getSlotManager().processResourceRequirements(resourceRequirements);
+
+                            allocateResourceFuture.get();
+                        });
+            }
+        };
+    }
+
+    /**
+     * Tests that resources continue to be considered missing if we cannot allocate more resources.
+     */
+    @Test
+    public void testRequirementDeclarationWithResourceAllocationFailure() throws Exception {

Review comment:
       Missing resources are internal states of the slot manager. An IT case should not verify against it.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/FineGrainedSlotManagerTest.java
##########
@@ -328,667 +327,162 @@ private void testRequirementDeclaration(RequirementDeclarationScenario scenario)
                                 ResourceProfile,
                                 String,
                                 ResourceManagerId>>
-                requestFuture = new CompletableFuture<>();
-        // accept an incoming slot request
-        final TaskExecutorGateway taskExecutorGateway =
-                new TestingTaskExecutorGatewayBuilder()
-                        .setRequestSlotFunction(
-                                tuple6 -> {
-                                    requestFuture.complete(tuple6);
-                                    return CompletableFuture.completedFuture(Acknowledge.get());
-                                })
-                        .createTestingTaskExecutorGateway();
-
-        final TaskExecutorConnection taskExecutorConnection =
-                new TaskExecutorConnection(resourceID, taskExecutorGateway);
-
-        final TaskManagerTracker tracker = new FineGrainedTaskManagerTracker();
-        try (FineGrainedSlotManager slotManager =
-                createFineGrainedSlotManagerBuilder()
-                        .setTaskManagerTracker(tracker)
-                        .buildAndStartWithDirectExec(
-                                resourceManagerId, new TestingResourceActionsBuilder().build())) {
-
-            if (scenario
-                    == RequirementDeclarationScenario
-                            .TASK_EXECUTOR_REGISTRATION_BEFORE_REQUIREMENT_DECLARATION) {
-                slotManager.registerTaskManager(
-                        taskExecutorConnection,
-                        new SlotReport(),
-                        DEFAULT_TOTAL_RESOURCE_PROFILE,
-                        DEFAULT_SLOT_RESOURCE_PROFILE);
-            }
-
-            final ResourceRequirements requirements =
-                    ResourceRequirements.create(
-                            jobId,
-                            targetAddress,
-                            Collections.singleton(ResourceRequirement.create(resourceProfile, 1)));
-            slotManager.processResourceRequirements(requirements);
-
-            if (scenario
-                    == RequirementDeclarationScenario
-                            .TASK_EXECUTOR_REGISTRATION_AFTER_REQUIREMENT_DECLARATION) {
-                slotManager.registerTaskManager(
-                        taskExecutorConnection,
-                        new SlotReport(),
-                        DEFAULT_TOTAL_RESOURCE_PROFILE,
-                        DEFAULT_SLOT_RESOURCE_PROFILE);
-            }
-
-            assertThat(
-                    requestFuture.get(),
-                    is(
-                            equalTo(
-                                    Tuple6.of(
-                                            slotId,
-                                            jobId,
-                                            requestFuture.get().f2,
-                                            resourceProfile,
-                                            targetAddress,
-                                            resourceManagerId))));
-
-            final TaskManagerSlotInformation slot =
-                    tracker.getAllocatedOrPendingSlot(requestFuture.get().f2).get();
-
-            assertEquals(
-                    "The slot has not been allocated to the expected allocation id.",
-                    requestFuture.get().f2,
-                    slot.getAllocationId());
-        }
-    }
-
-    /**
-     * Tests that freeing a slot will correctly reset the slot and give back the resource of it to
-     * task manager.
-     */
-    @Test
-    public void testFreeSlot() throws Exception {
-        final ResourceID resourceId = ResourceID.generate();
-        final TaskExecutorGateway taskExecutorGateway =
-                new TestingTaskExecutorGatewayBuilder().createTestingTaskExecutorGateway();
-        final TaskExecutorConnection taskExecutorConnection =
-                new TaskExecutorConnection(resourceId, taskExecutorGateway);
-        final AllocationID allocationId = new AllocationID();
-        final ResourceProfile resourceProfile = ResourceProfile.fromResources(42.0, 1337);
-
-        final SlotReport slotReport =
-                new SlotReport(createAllocatedSlotStatus(allocationId, resourceProfile));
-
-        final TaskManagerTracker tracker = new FineGrainedTaskManagerTracker();
-        try (FineGrainedSlotManager slotManager =
-                createFineGrainedSlotManagerBuilder()
-                        .setTaskManagerTracker(tracker)
-                        .buildAndStartWithDirectExec()) {
-
-            slotManager.registerTaskManager(
-                    taskExecutorConnection, slotReport, resourceProfile, resourceProfile);
-
-            final TaskManagerSlotInformation slot =
-                    tracker.getAllocatedOrPendingSlot(allocationId).get();
-
-            assertSame(SlotState.ALLOCATED, slot.getState());
-            assertEquals(
-                    ResourceProfile.ZERO,
-                    tracker.getRegisteredTaskManager(taskExecutorConnection.getInstanceID())
-                            .get()
-                            .getAvailableResource());
-
-            slotManager.freeSlot(new SlotID(resourceId, 0), allocationId);
-
-            assertFalse(tracker.getAllocatedOrPendingSlot(allocationId).isPresent());
-
-            assertEquals(
-                    resourceProfile,
-                    tracker.getRegisteredTaskManager(taskExecutorConnection.getInstanceID())
-                            .get()
-                            .getAvailableResource());
-        }
-    }
-
-    /**
-     * 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 CompletableFuture<AllocationID> allocationId = new CompletableFuture<>();
-        final AtomicInteger allocateResourceCalls = new AtomicInteger(0);
-        final ResourceActions resourceManagerActions =
-                new TestingResourceActionsBuilder()
-                        .setAllocateResourceConsumer(
-                                tuple6 -> allocateResourceCalls.incrementAndGet())
-                        .build();
-        final ResourceRequirements requirements = createResourceRequirementsForSingleSlot();
-
+                requestSlotFuture = new CompletableFuture<>();
         final TaskExecutorGateway taskExecutorGateway =
                 new TestingTaskExecutorGatewayBuilder()
                         .setRequestSlotFunction(
                                 tuple6 -> {
-                                    allocationId.complete(tuple6.f2);
+                                    requestSlotFuture.complete(tuple6);
                                     return CompletableFuture.completedFuture(Acknowledge.get());
                                 })
                         .createTestingTaskExecutorGateway();
-
-        final ResourceID resourceId = ResourceID.generate();
-
         final TaskExecutorConnection taskManagerConnection =
-                new TaskExecutorConnection(resourceId, taskExecutorGateway);
+                new TaskExecutorConnection(ResourceID.generate(), taskExecutorGateway);
+        final JobID jobId = new JobID();
         final SlotReport slotReport = new SlotReport();
-
-        final TaskManagerTracker tracker = new FineGrainedTaskManagerTracker();
-        try (FineGrainedSlotManager slotManager =
-                createFineGrainedSlotManagerBuilder()
-                        .setTaskManagerTracker(tracker)
-                        .buildAndStartWithDirectExec(
-                                ResourceManagerId.generate(), resourceManagerActions)) {
-
-            slotManager.registerTaskManager(
-                    taskManagerConnection,
-                    slotReport,
-                    DEFAULT_TOTAL_RESOURCE_PROFILE,
-                    DEFAULT_SLOT_RESOURCE_PROFILE);
-
-            slotManager.processResourceRequirements(requirements);
-
-            final TaskManagerSlotInformation slot =
-                    tracker.getAllocatedOrPendingSlot(allocationId.get()).get();
-
-            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));
+        new Context() {
+            {
+                resourceAllocationStrategyBuilder =
+                        TestingResourceAllocationStrategy.newBuilder()
+                                .setTryFulfillRequirementsFunction(
+                                        ((jobIDCollectionMap,
+                                                instanceIDTuple2Map,
+                                                pendingTaskManagers) ->
+                                                ResourceAllocationResult.builder()
+                                                        .addAllocationOnRegisteredResource(
+                                                                jobId,
+                                                                taskManagerConnection
+                                                                        .getInstanceID(),
+                                                                getDefaultSlotResourceProfile())
+                                                        .build()));
+                runTest(
+                        () -> {
+                            getSlotManager()
+                                    .registerTaskManager(
+                                            taskManagerConnection,
+                                            slotReport,
+                                            getDefaultTaskManagerResourceProfile(),
+                                            getDefaultSlotResourceProfile());
+                            getSlotManager()
+                                    .processResourceRequirements(
+                                            createResourceRequirements(jobId, 1));
+                            final Tuple6<
+                                            SlotID,
+                                            JobID,
+                                            AllocationID,
+                                            ResourceProfile,
+                                            String,
+                                            ResourceManagerId>
+                                    requestSlot = requestSlotFuture.get();

Review comment:
       I would suggest to always get a future with a timeout, for test cases, if possible.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/AbstractFineGrainedSlotManagerITCase.java
##########
@@ -0,0 +1,672 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.Tuple6;
+import org.apache.flink.core.testutils.ManuallyTriggeredScheduledExecutorService;
+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.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.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.testutils.SystemExitTrackingSecurityManager;
+import org.apache.flink.util.function.FunctionUtils;
+
+import akka.pattern.AskTimeoutException;
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Supplier;
+
+import static org.hamcrest.Matchers.empty;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.is;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertThat;
+
+/** IT Cases of {@link FineGrainedSlotManager}. */
+public abstract class AbstractFineGrainedSlotManagerITCase extends FineGrainedSlotManagerTestBase {
+
+    // ---------------------------------------------------------------------------------------------
+    // Requirement declaration
+    // ---------------------------------------------------------------------------------------------
+
+    /**
+     * Tests that a requirement declaration with no free slots will trigger the resource allocation.
+     */
+    @Test
+    public void testRequirementDeclarationWithoutFreeSlotsTriggersWorkerAllocation()
+            throws Exception {
+        final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot();
+
+        final CompletableFuture<WorkerResourceSpec> allocateResourceFuture =
+                new CompletableFuture<>();
+        new Context() {
+            {
+                resourceActionsBuilder.setAllocateResourceConsumer(
+                        allocateResourceFuture::complete);
+                runTest(
+                        () -> {
+                            getSlotManager().processResourceRequirements(resourceRequirements);
+
+                            allocateResourceFuture.get();
+                        });
+            }
+        };
+    }
+
+    /**
+     * Tests that resources continue to be considered missing if we cannot allocate more resources.
+     */
+    @Test
+    public void testRequirementDeclarationWithResourceAllocationFailure() throws Exception {
+        final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot();
+        new Context() {
+            {
+                resourceActionsBuilder.setAllocateResourceFunction(value -> false);
+                runTest(
+                        () -> {
+                            getSlotManager().processResourceRequirements(resourceRequirements);
+
+                            final JobID jobId = resourceRequirements.getJobId();
+                            assertThat(
+                                    getTotalResourceCount(
+                                            getResourceTracker().getMissingResources().get(jobId)),
+                                    is(1));
+                        });
+            }
+        };
+    }
+
+    /** Tests that resource requirements can be fulfilled with resource that are currently free. */
+    @Test
+    public void testRequirementDeclarationWithFreeResource() throws Exception {
+        testRequirementDeclaration(
+                RequirementDeclarationScenario
+                        .TASK_EXECUTOR_REGISTRATION_BEFORE_REQUIREMENT_DECLARATION);
+    }
+
+    /**
+     * Tests that resource requirements can be fulfilled with resource that are registered after the
+     * requirement declaration.
+     */
+    @Test
+    public void testRequirementDeclarationWithPendingResource() throws Exception {
+        testRequirementDeclaration(
+                RequirementDeclarationScenario
+                        .TASK_EXECUTOR_REGISTRATION_AFTER_REQUIREMENT_DECLARATION);
+    }
+
+    private enum RequirementDeclarationScenario {
+        // Tests that a slot request which can be fulfilled will trigger a slot allocation
+        TASK_EXECUTOR_REGISTRATION_BEFORE_REQUIREMENT_DECLARATION,
+        // Tests that pending slot requests are tried to be fulfilled upon new slot registrations
+        TASK_EXECUTOR_REGISTRATION_AFTER_REQUIREMENT_DECLARATION
+    }
+
+    private void testRequirementDeclaration(RequirementDeclarationScenario scenario)
+            throws Exception {
+        final ResourceID resourceID = ResourceID.generate();
+        final JobID jobId = new JobID();
+        final SlotID slotId = SlotID.getDynamicSlotID(resourceID);
+        final String targetAddress = "localhost";
+        final ResourceRequirements requirements =
+                ResourceRequirements.create(
+                        jobId,
+                        targetAddress,
+                        Collections.singleton(
+                                ResourceRequirement.create(getDefaultSlotResourceProfile(), 1)));
+
+        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);
+                                    return CompletableFuture.completedFuture(Acknowledge.get());
+                                })
+                        .createTestingTaskExecutorGateway();
+
+        final TaskExecutorConnection taskExecutorConnection =
+                new TaskExecutorConnection(resourceID, taskExecutorGateway);
+
+        new Context() {
+            {
+                runTest(
+                        () -> {
+                            if (scenario
+                                    == RequirementDeclarationScenario
+                                            .TASK_EXECUTOR_REGISTRATION_BEFORE_REQUIREMENT_DECLARATION) {
+                                getSlotManager()
+                                        .registerTaskManager(
+                                                taskExecutorConnection,
+                                                new SlotReport(),
+                                                getDefaultTaskManagerResourceProfile(),
+                                                getDefaultSlotResourceProfile());
+                            }
+
+                            getSlotManager().processResourceRequirements(requirements);
+
+                            if (scenario
+                                    == RequirementDeclarationScenario
+                                            .TASK_EXECUTOR_REGISTRATION_AFTER_REQUIREMENT_DECLARATION) {
+                                getSlotManager()
+                                        .registerTaskManager(
+                                                taskExecutorConnection,
+                                                new SlotReport(),
+                                                getDefaultTaskManagerResourceProfile(),
+                                                getDefaultSlotResourceProfile());
+                            }
+
+                            assertThat(
+                                    requestFuture.get(),
+                                    is(
+                                            equalTo(
+                                                    Tuple6.of(
+                                                            slotId,
+                                                            jobId,
+                                                            requestFuture.get().f2,
+                                                            getDefaultSlotResourceProfile(),
+                                                            targetAddress,
+                                                            getResourceManagerId()))));
+
+                            final TaskManagerSlotInformation slot =
+                                    getTaskManagerTracker()
+                                            .getAllocatedOrPendingSlot(requestFuture.get().f2)
+                                            .get();
+
+                            assertEquals(
+                                    "The slot has not been allocated to the expected allocation id.",
+                                    requestFuture.get().f2,
+                                    slot.getAllocationId());
+                        });
+            }
+        };
+    }
+
+    /**
+     * 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 AtomicInteger allocateResourceCalls = new AtomicInteger(0);
+        final ResourceRequirements requirements = createResourceRequirementsForSingleSlot();
+
+        new Context() {
+            {
+                resourceActionsBuilder.setAllocateResourceConsumer(
+                        tuple6 -> allocateResourceCalls.incrementAndGet());
+                runTest(
+                        () -> {
+                            getSlotManager().processResourceRequirements(requirements);
+
+                            getSlotManager().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(1));
+                        });
+            }
+        };
+    }
+
+    /**
+     * Tests that a resource allocated for one job can be allocated for another job after being
+     * freed.
+     */
+    @Test
+    public void testResourceCanBeAllocatedForDifferentJobAfterFree() throws Exception {
+        testResourceCanBeAllocatedForDifferentJobAfterFree(
+                SecondRequirementDeclarationTime.BEFORE_FREE);
+        testResourceCanBeAllocatedForDifferentJobAfterFree(
+                SecondRequirementDeclarationTime.AFTER_FREE);
+    }
+
+    private enum SecondRequirementDeclarationTime {
+        BEFORE_FREE,
+        AFTER_FREE
+    }
+
+    private void testResourceCanBeAllocatedForDifferentJobAfterFree(
+            SecondRequirementDeclarationTime secondRequirementDeclarationTime) throws Exception {
+        final CompletableFuture<AllocationID> allocationId1 = new CompletableFuture<>();
+        final CompletableFuture<AllocationID> allocationId2 = new CompletableFuture<>();
+        final ResourceRequirements resourceRequirements1 =
+                createResourceRequirementsForSingleSlot();
+        final ResourceRequirements resourceRequirements2 =
+                createResourceRequirementsForSingleSlot();
+
+        final TaskExecutorGateway taskExecutorGateway =
+                new TestingTaskExecutorGatewayBuilder()
+                        .setRequestSlotFunction(
+                                tuple6 -> {
+                                    if (!allocationId1.isDone()) {
+                                        allocationId1.complete(tuple6.f2);
+                                    } else {
+                                        allocationId2.complete(tuple6.f2);
+                                    }
+                                    return CompletableFuture.completedFuture(Acknowledge.get());
+                                })
+                        .createTestingTaskExecutorGateway();
+        final ResourceID resourceID = ResourceID.generate();
+        final TaskExecutorConnection taskManagerConnection =
+                new TaskExecutorConnection(resourceID, taskExecutorGateway);
+        final SlotReport slotReport = new SlotReport();
+
+        new Context() {
+            {
+                runTest(
+                        () -> {
+                            getSlotManager()
+                                    .registerTaskManager(
+                                            taskManagerConnection,
+                                            slotReport,
+                                            getDefaultTaskManagerResourceProfile(),
+                                            getDefaultSlotResourceProfile());
+
+                            getSlotManager().processResourceRequirements(resourceRequirements1);
+
+                            TaskManagerSlotInformation slot =
+                                    getTaskManagerTracker()
+                                            .getAllocatedOrPendingSlot(allocationId1.get())
+                                            .get();
+
+                            assertEquals(
+                                    "The slot has not been allocated to the expected job id.",
+                                    resourceRequirements1.getJobId(),
+                                    slot.getJobId());
+
+                            if (secondRequirementDeclarationTime
+                                    == SecondRequirementDeclarationTime.BEFORE_FREE) {
+                                getSlotManager().processResourceRequirements(resourceRequirements2);
+                            }
+
+                            // clear resource requirements first so that the freed slot isn't
+                            // immediately re-assigned to the job
+                            getSlotManager()
+                                    .processResourceRequirements(
+                                            ResourceRequirements.create(
+                                                    resourceRequirements1.getJobId(),
+                                                    resourceRequirements1.getTargetAddress(),
+                                                    Collections.emptyList()));
+                            getSlotManager()
+                                    .freeSlot(
+                                            SlotID.getDynamicSlotID(resourceID),
+                                            allocationId1.get());
+
+                            if (secondRequirementDeclarationTime
+                                    == SecondRequirementDeclarationTime.AFTER_FREE) {
+                                getSlotManager().processResourceRequirements(resourceRequirements2);
+                            }
+
+                            slot =
+                                    getTaskManagerTracker()
+                                            .getAllocatedOrPendingSlot(allocationId2.get())
+                                            .get();
+                            assertEquals(
+                                    "The slot has not been allocated to the expected job id.",
+                                    resourceRequirements2.getJobId(),
+                                    slot.getJobId());
+                        });
+            }
+        };
+    }
+
+    /**
+     * Tests that we only request new resources/containers once we have assigned all pending task
+     * managers.
+     */
+    @Test
+    public void testRequestNewResources() throws Exception {
+        final AtomicInteger resourceRequests = new AtomicInteger(0);
+        final JobID jobId = new JobID();
+
+        new Context() {
+            {
+                resourceActionsBuilder.setAllocateResourceFunction(
+                        ignored -> {
+                            resourceRequests.incrementAndGet();
+                            return true;
+                        });
+                runTest(
+                        () -> {
+                            // the first requirements should be fulfillable with the pending task
+                            // managers of the first allocation
+                            getSlotManager()
+                                    .processResourceRequirements(
+                                            createResourceRequirements(
+                                                    jobId, getDefaultNumberSlotsPerWorker()));
+                            assertThat(resourceRequests.get(), is(1));
+
+                            getSlotManager()
+                                    .processResourceRequirements(
+                                            createResourceRequirements(
+                                                    jobId, getDefaultNumberSlotsPerWorker() + 1));
+                            assertThat(resourceRequests.get(), is(2));
+                        });
+            }
+        };
+    }
+
+    /**
+     * Test that the slot manager only allocates new workers if their worker spec can fulfill the
+     * requested resource profile.
+     */
+    @Test
+    public void testWorkerOnlyAllocatedIfRequestedSlotCouldBeFulfilled() throws Exception {
+        final AtomicInteger resourceRequests = new AtomicInteger(0);
+
+        new Context() {
+            {
+                resourceActionsBuilder.setAllocateResourceFunction(
+                        ignored -> {
+                            resourceRequests.incrementAndGet();
+                            return true;
+                        });
+                runTest(
+                        () -> {
+                            getSlotManager()
+                                    .processResourceRequirements(
+                                            createResourceRequirements(
+                                                    new JobID(), 1, getLargeSlotResourceProfile()));
+                            assertThat(resourceRequests.get(), is(0));
+                        });
+            }
+        };
+    }
+
+    // ---------------------------------------------------------------------------------------------
+    // Slot allocation failure handling
+    // ---------------------------------------------------------------------------------------------
+
+    /** Tests that if a slot allocation times out we try to allocate another slot. */
+    @Test
+    public void testSlotAllocationTimeout() throws Exception {

Review comment:
       Isn't this case already covered by `testSlotRequestFailure`?

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/AbstractFineGrainedSlotManagerITCase.java
##########
@@ -0,0 +1,672 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.Tuple6;
+import org.apache.flink.core.testutils.ManuallyTriggeredScheduledExecutorService;
+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.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.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.testutils.SystemExitTrackingSecurityManager;
+import org.apache.flink.util.function.FunctionUtils;
+
+import akka.pattern.AskTimeoutException;
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Supplier;
+
+import static org.hamcrest.Matchers.empty;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.is;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertThat;
+
+/** IT Cases of {@link FineGrainedSlotManager}. */
+public abstract class AbstractFineGrainedSlotManagerITCase extends FineGrainedSlotManagerTestBase {
+
+    // ---------------------------------------------------------------------------------------------
+    // Requirement declaration
+    // ---------------------------------------------------------------------------------------------
+
+    /**
+     * Tests that a requirement declaration with no free slots will trigger the resource allocation.
+     */
+    @Test
+    public void testRequirementDeclarationWithoutFreeSlotsTriggersWorkerAllocation()
+            throws Exception {
+        final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot();
+
+        final CompletableFuture<WorkerResourceSpec> allocateResourceFuture =
+                new CompletableFuture<>();
+        new Context() {
+            {
+                resourceActionsBuilder.setAllocateResourceConsumer(
+                        allocateResourceFuture::complete);
+                runTest(
+                        () -> {
+                            getSlotManager().processResourceRequirements(resourceRequirements);
+
+                            allocateResourceFuture.get();
+                        });
+            }
+        };
+    }
+
+    /**
+     * Tests that resources continue to be considered missing if we cannot allocate more resources.
+     */
+    @Test
+    public void testRequirementDeclarationWithResourceAllocationFailure() throws Exception {
+        final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot();
+        new Context() {
+            {
+                resourceActionsBuilder.setAllocateResourceFunction(value -> false);
+                runTest(
+                        () -> {
+                            getSlotManager().processResourceRequirements(resourceRequirements);
+
+                            final JobID jobId = resourceRequirements.getJobId();
+                            assertThat(
+                                    getTotalResourceCount(
+                                            getResourceTracker().getMissingResources().get(jobId)),
+                                    is(1));
+                        });
+            }
+        };
+    }
+
+    /** Tests that resource requirements can be fulfilled with resource that are currently free. */
+    @Test
+    public void testRequirementDeclarationWithFreeResource() throws Exception {
+        testRequirementDeclaration(
+                RequirementDeclarationScenario
+                        .TASK_EXECUTOR_REGISTRATION_BEFORE_REQUIREMENT_DECLARATION);
+    }
+
+    /**
+     * Tests that resource requirements can be fulfilled with resource that are registered after the
+     * requirement declaration.
+     */
+    @Test
+    public void testRequirementDeclarationWithPendingResource() throws Exception {
+        testRequirementDeclaration(
+                RequirementDeclarationScenario
+                        .TASK_EXECUTOR_REGISTRATION_AFTER_REQUIREMENT_DECLARATION);
+    }
+
+    private enum RequirementDeclarationScenario {
+        // Tests that a slot request which can be fulfilled will trigger a slot allocation
+        TASK_EXECUTOR_REGISTRATION_BEFORE_REQUIREMENT_DECLARATION,
+        // Tests that pending slot requests are tried to be fulfilled upon new slot registrations
+        TASK_EXECUTOR_REGISTRATION_AFTER_REQUIREMENT_DECLARATION
+    }
+
+    private void testRequirementDeclaration(RequirementDeclarationScenario scenario)
+            throws Exception {
+        final ResourceID resourceID = ResourceID.generate();
+        final JobID jobId = new JobID();
+        final SlotID slotId = SlotID.getDynamicSlotID(resourceID);
+        final String targetAddress = "localhost";
+        final ResourceRequirements requirements =
+                ResourceRequirements.create(
+                        jobId,
+                        targetAddress,
+                        Collections.singleton(
+                                ResourceRequirement.create(getDefaultSlotResourceProfile(), 1)));
+
+        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);
+                                    return CompletableFuture.completedFuture(Acknowledge.get());
+                                })
+                        .createTestingTaskExecutorGateway();
+
+        final TaskExecutorConnection taskExecutorConnection =
+                new TaskExecutorConnection(resourceID, taskExecutorGateway);
+
+        new Context() {
+            {
+                runTest(
+                        () -> {
+                            if (scenario
+                                    == RequirementDeclarationScenario
+                                            .TASK_EXECUTOR_REGISTRATION_BEFORE_REQUIREMENT_DECLARATION) {
+                                getSlotManager()
+                                        .registerTaskManager(
+                                                taskExecutorConnection,
+                                                new SlotReport(),
+                                                getDefaultTaskManagerResourceProfile(),
+                                                getDefaultSlotResourceProfile());
+                            }
+
+                            getSlotManager().processResourceRequirements(requirements);
+
+                            if (scenario
+                                    == RequirementDeclarationScenario
+                                            .TASK_EXECUTOR_REGISTRATION_AFTER_REQUIREMENT_DECLARATION) {
+                                getSlotManager()
+                                        .registerTaskManager(
+                                                taskExecutorConnection,
+                                                new SlotReport(),
+                                                getDefaultTaskManagerResourceProfile(),
+                                                getDefaultSlotResourceProfile());
+                            }
+
+                            assertThat(
+                                    requestFuture.get(),
+                                    is(
+                                            equalTo(
+                                                    Tuple6.of(
+                                                            slotId,
+                                                            jobId,
+                                                            requestFuture.get().f2,
+                                                            getDefaultSlotResourceProfile(),
+                                                            targetAddress,
+                                                            getResourceManagerId()))));
+
+                            final TaskManagerSlotInformation slot =
+                                    getTaskManagerTracker()
+                                            .getAllocatedOrPendingSlot(requestFuture.get().f2)
+                                            .get();
+
+                            assertEquals(
+                                    "The slot has not been allocated to the expected allocation id.",
+                                    requestFuture.get().f2,
+                                    slot.getAllocationId());
+                        });
+            }
+        };
+    }
+
+    /**
+     * 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 AtomicInteger allocateResourceCalls = new AtomicInteger(0);
+        final ResourceRequirements requirements = createResourceRequirementsForSingleSlot();
+
+        new Context() {
+            {
+                resourceActionsBuilder.setAllocateResourceConsumer(
+                        tuple6 -> allocateResourceCalls.incrementAndGet());
+                runTest(
+                        () -> {
+                            getSlotManager().processResourceRequirements(requirements);
+
+                            getSlotManager().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(1));
+                        });
+            }
+        };
+    }
+
+    /**
+     * Tests that a resource allocated for one job can be allocated for another job after being
+     * freed.
+     */
+    @Test
+    public void testResourceCanBeAllocatedForDifferentJobAfterFree() throws Exception {
+        testResourceCanBeAllocatedForDifferentJobAfterFree(
+                SecondRequirementDeclarationTime.BEFORE_FREE);
+        testResourceCanBeAllocatedForDifferentJobAfterFree(
+                SecondRequirementDeclarationTime.AFTER_FREE);
+    }
+
+    private enum SecondRequirementDeclarationTime {
+        BEFORE_FREE,
+        AFTER_FREE
+    }
+
+    private void testResourceCanBeAllocatedForDifferentJobAfterFree(
+            SecondRequirementDeclarationTime secondRequirementDeclarationTime) throws Exception {
+        final CompletableFuture<AllocationID> allocationId1 = new CompletableFuture<>();
+        final CompletableFuture<AllocationID> allocationId2 = new CompletableFuture<>();
+        final ResourceRequirements resourceRequirements1 =
+                createResourceRequirementsForSingleSlot();
+        final ResourceRequirements resourceRequirements2 =
+                createResourceRequirementsForSingleSlot();
+
+        final TaskExecutorGateway taskExecutorGateway =
+                new TestingTaskExecutorGatewayBuilder()
+                        .setRequestSlotFunction(
+                                tuple6 -> {
+                                    if (!allocationId1.isDone()) {
+                                        allocationId1.complete(tuple6.f2);
+                                    } else {
+                                        allocationId2.complete(tuple6.f2);
+                                    }
+                                    return CompletableFuture.completedFuture(Acknowledge.get());
+                                })
+                        .createTestingTaskExecutorGateway();
+        final ResourceID resourceID = ResourceID.generate();
+        final TaskExecutorConnection taskManagerConnection =
+                new TaskExecutorConnection(resourceID, taskExecutorGateway);
+        final SlotReport slotReport = new SlotReport();
+
+        new Context() {
+            {
+                runTest(
+                        () -> {
+                            getSlotManager()
+                                    .registerTaskManager(
+                                            taskManagerConnection,
+                                            slotReport,
+                                            getDefaultTaskManagerResourceProfile(),
+                                            getDefaultSlotResourceProfile());
+
+                            getSlotManager().processResourceRequirements(resourceRequirements1);
+
+                            TaskManagerSlotInformation slot =
+                                    getTaskManagerTracker()
+                                            .getAllocatedOrPendingSlot(allocationId1.get())
+                                            .get();
+
+                            assertEquals(
+                                    "The slot has not been allocated to the expected job id.",
+                                    resourceRequirements1.getJobId(),
+                                    slot.getJobId());
+
+                            if (secondRequirementDeclarationTime
+                                    == SecondRequirementDeclarationTime.BEFORE_FREE) {
+                                getSlotManager().processResourceRequirements(resourceRequirements2);
+                            }
+
+                            // clear resource requirements first so that the freed slot isn't
+                            // immediately re-assigned to the job
+                            getSlotManager()
+                                    .processResourceRequirements(
+                                            ResourceRequirements.create(
+                                                    resourceRequirements1.getJobId(),
+                                                    resourceRequirements1.getTargetAddress(),
+                                                    Collections.emptyList()));
+                            getSlotManager()
+                                    .freeSlot(
+                                            SlotID.getDynamicSlotID(resourceID),
+                                            allocationId1.get());
+
+                            if (secondRequirementDeclarationTime
+                                    == SecondRequirementDeclarationTime.AFTER_FREE) {
+                                getSlotManager().processResourceRequirements(resourceRequirements2);
+                            }
+
+                            slot =
+                                    getTaskManagerTracker()
+                                            .getAllocatedOrPendingSlot(allocationId2.get())
+                                            .get();
+                            assertEquals(
+                                    "The slot has not been allocated to the expected job id.",
+                                    resourceRequirements2.getJobId(),
+                                    slot.getJobId());
+                        });
+            }
+        };
+    }
+
+    /**
+     * Tests that we only request new resources/containers once we have assigned all pending task
+     * managers.
+     */
+    @Test
+    public void testRequestNewResources() throws Exception {
+        final AtomicInteger resourceRequests = new AtomicInteger(0);
+        final JobID jobId = new JobID();
+
+        new Context() {
+            {
+                resourceActionsBuilder.setAllocateResourceFunction(
+                        ignored -> {
+                            resourceRequests.incrementAndGet();
+                            return true;
+                        });
+                runTest(
+                        () -> {
+                            // the first requirements should be fulfillable with the pending task
+                            // managers of the first allocation
+                            getSlotManager()
+                                    .processResourceRequirements(
+                                            createResourceRequirements(
+                                                    jobId, getDefaultNumberSlotsPerWorker()));
+                            assertThat(resourceRequests.get(), is(1));
+
+                            getSlotManager()
+                                    .processResourceRequirements(
+                                            createResourceRequirements(
+                                                    jobId, getDefaultNumberSlotsPerWorker() + 1));
+                            assertThat(resourceRequests.get(), is(2));
+                        });
+            }
+        };
+    }
+
+    /**
+     * Test that the slot manager only allocates new workers if their worker spec can fulfill the
+     * requested resource profile.
+     */
+    @Test
+    public void testWorkerOnlyAllocatedIfRequestedSlotCouldBeFulfilled() throws Exception {
+        final AtomicInteger resourceRequests = new AtomicInteger(0);
+
+        new Context() {
+            {
+                resourceActionsBuilder.setAllocateResourceFunction(
+                        ignored -> {
+                            resourceRequests.incrementAndGet();
+                            return true;
+                        });
+                runTest(
+                        () -> {
+                            getSlotManager()
+                                    .processResourceRequirements(
+                                            createResourceRequirements(
+                                                    new JobID(), 1, getLargeSlotResourceProfile()));
+                            assertThat(resourceRequests.get(), is(0));
+                        });
+            }
+        };
+    }
+
+    // ---------------------------------------------------------------------------------------------
+    // Slot allocation failure handling
+    // ---------------------------------------------------------------------------------------------
+
+    /** 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 BlockingQueue<Supplier<CompletableFuture<Acknowledge>>> responseQueue =
+                new ArrayBlockingQueue<>(2);
+        responseQueue.add(
+                () -> FutureUtils.completedExceptionally(new AskTimeoutException("timeout")));
+        responseQueue.add(
+                () -> {
+                    secondSlotRequestFuture.complete(null);
+                    return new CompletableFuture<>();
+                });
+
+        final TaskExecutorGateway taskExecutorGateway =
+                new TestingTaskExecutorGatewayBuilder()
+                        .setRequestSlotFunction(ignored -> responseQueue.remove().get())
+                        .createTestingTaskExecutorGateway();
+        final ResourceID resourceID = ResourceID.generate();
+        final TaskExecutorConnection taskManagerConnection =
+                new TaskExecutorConnection(resourceID, taskExecutorGateway);
+        final SlotReport slotReport = new SlotReport();
+
+        new Context() {
+            {
+                runTest(
+                        () -> {
+                            getSlotManager()
+                                    .registerTaskManager(
+                                            taskManagerConnection,
+                                            slotReport,
+                                            getDefaultTaskManagerResourceProfile(),
+                                            getDefaultSlotResourceProfile());
+                            getSlotManager()
+                                    .processResourceRequirements(
+                                            createResourceRequirementsForSingleSlot());
+
+                            // a second request is only sent if the first request timed out
+                            secondSlotRequestFuture.get();
+                        });
+            }
+        };
+    }
+
+    /**
+     * Tests that the SlotManager retries allocating a slot if the TaskExecutor#requestSlot call
+     * fails.
+     */
+    @Test
+    public void testSlotRequestFailure() throws Exception {
+        final ManuallyTriggeredScheduledExecutorService executor =
+                new ManuallyTriggeredScheduledExecutorService();
+
+        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<AllocationID> allocationIds = new ArrayBlockingQueue<>(2);
+
+        final TestingTaskExecutorGateway taskExecutorGateway =
+                new TestingTaskExecutorGatewayBuilder()
+                        .setRequestSlotFunction(
+                                FunctionUtils.uncheckedFunction(
+                                        requestSlotParameters -> {
+                                            allocationIds.put(requestSlotParameters.f2);
+                                            return slotRequestFutureIterator.next();
+                                        }))
+                        .createTestingTaskExecutorGateway();
+
+        final ResourceID resourceId = ResourceID.generate();
+        final TaskExecutorConnection taskManagerConnection =
+                new TaskExecutorConnection(resourceId, taskExecutorGateway);
+        final SlotReport slotReport = new SlotReport();
+
+        new Context() {
+            {
+                mainThreadExecutor = executor;
+                runTest(
+                        () -> {
+                            getSlotManager()
+                                    .registerTaskManager(
+                                            taskManagerConnection,
+                                            slotReport,
+                                            getDefaultTaskManagerResourceProfile(),
+                                            getDefaultSlotResourceProfile());
+
+                            getSlotManager().processResourceRequirements(resourceRequirements);
+
+                            final AllocationID firstAllocationId = allocationIds.take();
+                            assertThat(allocationIds, is(empty()));
+
+                            // let the first attempt fail --> this should trigger a second attempt
+                            slotRequestFuture1.completeExceptionally(
+                                    new SlotAllocationException("Test exception."));
+
+                            executor.triggerAll();
+                            assertThat(
+                                    getTotalResourceCount(
+                                            getResourceTracker().getAcquiredResources(jobId)),
+                                    is(1));
+
+                            slotRequestFuture2.complete(Acknowledge.get());
+                            final AllocationID secondAllocationId = allocationIds.take();
+                            assertThat(allocationIds, is(empty()));
+
+                            executor.triggerAll();
+                            final TaskManagerSlotInformation slot =
+                                    getTaskManagerTracker()
+                                            .getAllocatedOrPendingSlot(secondAllocationId)
+                                            .get();
+
+                            assertThat(slot.getState(), is(SlotState.ALLOCATED));
+                            assertEquals(jobId, slot.getJobId());
+
+                            assertFalse(
+                                    getTaskManagerTracker()
+                                            .getAllocatedOrPendingSlot(firstAllocationId)
+                                            .isPresent());
+                        });
+            }
+        };
+    }
+
+    // ---------------------------------------------------------------------------------------------
+    // Allocation update
+    // ---------------------------------------------------------------------------------------------
+
+    @Test
+    public void testAllocationUpdatesIgnoredIfTaskExecutorUnregistered() throws Exception {
+        final ManuallyTriggeredScheduledExecutorService executor =
+                new ManuallyTriggeredScheduledExecutorService();
+
+        final TestingTaskExecutorGateway taskExecutorGateway =
+                new TestingTaskExecutorGatewayBuilder()
+                        // it is important that the returned future is already completed
+                        // otherwise it will be cancelled when the task executor is unregistered
+                        .setRequestSlotFunction(
+                                ignored -> CompletableFuture.completedFuture(Acknowledge.get()))
+                        .createTestingTaskExecutorGateway();
+
+        final SystemExitTrackingSecurityManager trackingSecurityManager =
+                new SystemExitTrackingSecurityManager();
+        System.setSecurityManager(trackingSecurityManager);

Review comment:
       Not sure if we need the security manager.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/AbstractFineGrainedSlotManagerITCase.java
##########
@@ -0,0 +1,672 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.Tuple6;
+import org.apache.flink.core.testutils.ManuallyTriggeredScheduledExecutorService;
+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.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.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.testutils.SystemExitTrackingSecurityManager;
+import org.apache.flink.util.function.FunctionUtils;
+
+import akka.pattern.AskTimeoutException;
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Supplier;
+
+import static org.hamcrest.Matchers.empty;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.is;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertThat;
+
+/** IT Cases of {@link FineGrainedSlotManager}. */
+public abstract class AbstractFineGrainedSlotManagerITCase extends FineGrainedSlotManagerTestBase {
+
+    // ---------------------------------------------------------------------------------------------
+    // Requirement declaration
+    // ---------------------------------------------------------------------------------------------
+
+    /**
+     * Tests that a requirement declaration with no free slots will trigger the resource allocation.
+     */
+    @Test
+    public void testRequirementDeclarationWithoutFreeSlotsTriggersWorkerAllocation()
+            throws Exception {
+        final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot();
+
+        final CompletableFuture<WorkerResourceSpec> allocateResourceFuture =
+                new CompletableFuture<>();
+        new Context() {
+            {
+                resourceActionsBuilder.setAllocateResourceConsumer(
+                        allocateResourceFuture::complete);
+                runTest(
+                        () -> {
+                            getSlotManager().processResourceRequirements(resourceRequirements);
+
+                            allocateResourceFuture.get();
+                        });
+            }
+        };
+    }
+
+    /**
+     * Tests that resources continue to be considered missing if we cannot allocate more resources.
+     */
+    @Test
+    public void testRequirementDeclarationWithResourceAllocationFailure() throws Exception {
+        final ResourceRequirements resourceRequirements = createResourceRequirementsForSingleSlot();
+        new Context() {
+            {
+                resourceActionsBuilder.setAllocateResourceFunction(value -> false);
+                runTest(
+                        () -> {
+                            getSlotManager().processResourceRequirements(resourceRequirements);
+
+                            final JobID jobId = resourceRequirements.getJobId();
+                            assertThat(
+                                    getTotalResourceCount(
+                                            getResourceTracker().getMissingResources().get(jobId)),
+                                    is(1));
+                        });
+            }
+        };
+    }
+
+    /** Tests that resource requirements can be fulfilled with resource that are currently free. */
+    @Test
+    public void testRequirementDeclarationWithFreeResource() throws Exception {
+        testRequirementDeclaration(
+                RequirementDeclarationScenario
+                        .TASK_EXECUTOR_REGISTRATION_BEFORE_REQUIREMENT_DECLARATION);
+    }
+
+    /**
+     * Tests that resource requirements can be fulfilled with resource that are registered after the
+     * requirement declaration.
+     */
+    @Test
+    public void testRequirementDeclarationWithPendingResource() throws Exception {
+        testRequirementDeclaration(
+                RequirementDeclarationScenario
+                        .TASK_EXECUTOR_REGISTRATION_AFTER_REQUIREMENT_DECLARATION);
+    }
+
+    private enum RequirementDeclarationScenario {
+        // Tests that a slot request which can be fulfilled will trigger a slot allocation
+        TASK_EXECUTOR_REGISTRATION_BEFORE_REQUIREMENT_DECLARATION,
+        // Tests that pending slot requests are tried to be fulfilled upon new slot registrations
+        TASK_EXECUTOR_REGISTRATION_AFTER_REQUIREMENT_DECLARATION
+    }
+
+    private void testRequirementDeclaration(RequirementDeclarationScenario scenario)
+            throws Exception {
+        final ResourceID resourceID = ResourceID.generate();
+        final JobID jobId = new JobID();
+        final SlotID slotId = SlotID.getDynamicSlotID(resourceID);
+        final String targetAddress = "localhost";
+        final ResourceRequirements requirements =
+                ResourceRequirements.create(
+                        jobId,
+                        targetAddress,
+                        Collections.singleton(
+                                ResourceRequirement.create(getDefaultSlotResourceProfile(), 1)));
+
+        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);
+                                    return CompletableFuture.completedFuture(Acknowledge.get());
+                                })
+                        .createTestingTaskExecutorGateway();
+
+        final TaskExecutorConnection taskExecutorConnection =
+                new TaskExecutorConnection(resourceID, taskExecutorGateway);
+
+        new Context() {
+            {
+                runTest(
+                        () -> {
+                            if (scenario
+                                    == RequirementDeclarationScenario
+                                            .TASK_EXECUTOR_REGISTRATION_BEFORE_REQUIREMENT_DECLARATION) {
+                                getSlotManager()
+                                        .registerTaskManager(
+                                                taskExecutorConnection,
+                                                new SlotReport(),
+                                                getDefaultTaskManagerResourceProfile(),
+                                                getDefaultSlotResourceProfile());
+                            }
+
+                            getSlotManager().processResourceRequirements(requirements);
+
+                            if (scenario
+                                    == RequirementDeclarationScenario
+                                            .TASK_EXECUTOR_REGISTRATION_AFTER_REQUIREMENT_DECLARATION) {
+                                getSlotManager()
+                                        .registerTaskManager(
+                                                taskExecutorConnection,
+                                                new SlotReport(),
+                                                getDefaultTaskManagerResourceProfile(),
+                                                getDefaultSlotResourceProfile());
+                            }
+
+                            assertThat(
+                                    requestFuture.get(),
+                                    is(
+                                            equalTo(
+                                                    Tuple6.of(
+                                                            slotId,
+                                                            jobId,
+                                                            requestFuture.get().f2,
+                                                            getDefaultSlotResourceProfile(),
+                                                            targetAddress,
+                                                            getResourceManagerId()))));
+
+                            final TaskManagerSlotInformation slot =
+                                    getTaskManagerTracker()
+                                            .getAllocatedOrPendingSlot(requestFuture.get().f2)
+                                            .get();
+
+                            assertEquals(
+                                    "The slot has not been allocated to the expected allocation id.",
+                                    requestFuture.get().f2,
+                                    slot.getAllocationId());
+                        });
+            }
+        };
+    }
+
+    /**
+     * 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 AtomicInteger allocateResourceCalls = new AtomicInteger(0);
+        final ResourceRequirements requirements = createResourceRequirementsForSingleSlot();
+
+        new Context() {
+            {
+                resourceActionsBuilder.setAllocateResourceConsumer(
+                        tuple6 -> allocateResourceCalls.incrementAndGet());
+                runTest(
+                        () -> {
+                            getSlotManager().processResourceRequirements(requirements);
+
+                            getSlotManager().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(1));
+                        });
+            }
+        };
+    }
+
+    /**
+     * Tests that a resource allocated for one job can be allocated for another job after being
+     * freed.
+     */
+    @Test
+    public void testResourceCanBeAllocatedForDifferentJobAfterFree() throws Exception {
+        testResourceCanBeAllocatedForDifferentJobAfterFree(
+                SecondRequirementDeclarationTime.BEFORE_FREE);
+        testResourceCanBeAllocatedForDifferentJobAfterFree(
+                SecondRequirementDeclarationTime.AFTER_FREE);
+    }
+
+    private enum SecondRequirementDeclarationTime {
+        BEFORE_FREE,
+        AFTER_FREE
+    }
+
+    private void testResourceCanBeAllocatedForDifferentJobAfterFree(
+            SecondRequirementDeclarationTime secondRequirementDeclarationTime) throws Exception {
+        final CompletableFuture<AllocationID> allocationId1 = new CompletableFuture<>();
+        final CompletableFuture<AllocationID> allocationId2 = new CompletableFuture<>();
+        final ResourceRequirements resourceRequirements1 =
+                createResourceRequirementsForSingleSlot();
+        final ResourceRequirements resourceRequirements2 =
+                createResourceRequirementsForSingleSlot();
+
+        final TaskExecutorGateway taskExecutorGateway =
+                new TestingTaskExecutorGatewayBuilder()
+                        .setRequestSlotFunction(
+                                tuple6 -> {
+                                    if (!allocationId1.isDone()) {
+                                        allocationId1.complete(tuple6.f2);
+                                    } else {
+                                        allocationId2.complete(tuple6.f2);
+                                    }
+                                    return CompletableFuture.completedFuture(Acknowledge.get());
+                                })
+                        .createTestingTaskExecutorGateway();
+        final ResourceID resourceID = ResourceID.generate();
+        final TaskExecutorConnection taskManagerConnection =
+                new TaskExecutorConnection(resourceID, taskExecutorGateway);
+        final SlotReport slotReport = new SlotReport();
+
+        new Context() {
+            {
+                runTest(
+                        () -> {
+                            getSlotManager()
+                                    .registerTaskManager(
+                                            taskManagerConnection,
+                                            slotReport,
+                                            getDefaultTaskManagerResourceProfile(),
+                                            getDefaultSlotResourceProfile());
+
+                            getSlotManager().processResourceRequirements(resourceRequirements1);
+
+                            TaskManagerSlotInformation slot =
+                                    getTaskManagerTracker()
+                                            .getAllocatedOrPendingSlot(allocationId1.get())
+                                            .get();
+
+                            assertEquals(
+                                    "The slot has not been allocated to the expected job id.",
+                                    resourceRequirements1.getJobId(),
+                                    slot.getJobId());
+
+                            if (secondRequirementDeclarationTime
+                                    == SecondRequirementDeclarationTime.BEFORE_FREE) {
+                                getSlotManager().processResourceRequirements(resourceRequirements2);
+                            }
+
+                            // clear resource requirements first so that the freed slot isn't
+                            // immediately re-assigned to the job
+                            getSlotManager()
+                                    .processResourceRequirements(
+                                            ResourceRequirements.create(
+                                                    resourceRequirements1.getJobId(),
+                                                    resourceRequirements1.getTargetAddress(),
+                                                    Collections.emptyList()));
+                            getSlotManager()
+                                    .freeSlot(
+                                            SlotID.getDynamicSlotID(resourceID),
+                                            allocationId1.get());
+
+                            if (secondRequirementDeclarationTime
+                                    == SecondRequirementDeclarationTime.AFTER_FREE) {
+                                getSlotManager().processResourceRequirements(resourceRequirements2);
+                            }
+
+                            slot =
+                                    getTaskManagerTracker()
+                                            .getAllocatedOrPendingSlot(allocationId2.get())
+                                            .get();
+                            assertEquals(
+                                    "The slot has not been allocated to the expected job id.",
+                                    resourceRequirements2.getJobId(),
+                                    slot.getJobId());
+                        });
+            }
+        };
+    }
+
+    /**
+     * Tests that we only request new resources/containers once we have assigned all pending task
+     * managers.
+     */
+    @Test
+    public void testRequestNewResources() throws Exception {
+        final AtomicInteger resourceRequests = new AtomicInteger(0);
+        final JobID jobId = new JobID();
+
+        new Context() {
+            {
+                resourceActionsBuilder.setAllocateResourceFunction(
+                        ignored -> {
+                            resourceRequests.incrementAndGet();
+                            return true;
+                        });
+                runTest(
+                        () -> {
+                            // the first requirements should be fulfillable with the pending task
+                            // managers of the first allocation
+                            getSlotManager()
+                                    .processResourceRequirements(
+                                            createResourceRequirements(
+                                                    jobId, getDefaultNumberSlotsPerWorker()));
+                            assertThat(resourceRequests.get(), is(1));
+
+                            getSlotManager()
+                                    .processResourceRequirements(
+                                            createResourceRequirements(
+                                                    jobId, getDefaultNumberSlotsPerWorker() + 1));
+                            assertThat(resourceRequests.get(), is(2));
+                        });
+            }
+        };
+    }
+
+    /**
+     * Test that the slot manager only allocates new workers if their worker spec can fulfill the
+     * requested resource profile.
+     */
+    @Test
+    public void testWorkerOnlyAllocatedIfRequestedSlotCouldBeFulfilled() throws Exception {
+        final AtomicInteger resourceRequests = new AtomicInteger(0);
+
+        new Context() {
+            {
+                resourceActionsBuilder.setAllocateResourceFunction(
+                        ignored -> {
+                            resourceRequests.incrementAndGet();
+                            return true;
+                        });
+                runTest(
+                        () -> {
+                            getSlotManager()
+                                    .processResourceRequirements(
+                                            createResourceRequirements(
+                                                    new JobID(), 1, getLargeSlotResourceProfile()));
+                            assertThat(resourceRequests.get(), is(0));
+                        });
+            }
+        };
+    }
+
+    // ---------------------------------------------------------------------------------------------
+    // Slot allocation failure handling
+    // ---------------------------------------------------------------------------------------------
+
+    /** 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 BlockingQueue<Supplier<CompletableFuture<Acknowledge>>> responseQueue =
+                new ArrayBlockingQueue<>(2);
+        responseQueue.add(
+                () -> FutureUtils.completedExceptionally(new AskTimeoutException("timeout")));
+        responseQueue.add(
+                () -> {
+                    secondSlotRequestFuture.complete(null);
+                    return new CompletableFuture<>();
+                });
+
+        final TaskExecutorGateway taskExecutorGateway =
+                new TestingTaskExecutorGatewayBuilder()
+                        .setRequestSlotFunction(ignored -> responseQueue.remove().get())
+                        .createTestingTaskExecutorGateway();
+        final ResourceID resourceID = ResourceID.generate();
+        final TaskExecutorConnection taskManagerConnection =
+                new TaskExecutorConnection(resourceID, taskExecutorGateway);
+        final SlotReport slotReport = new SlotReport();
+
+        new Context() {
+            {
+                runTest(
+                        () -> {
+                            getSlotManager()
+                                    .registerTaskManager(
+                                            taskManagerConnection,
+                                            slotReport,
+                                            getDefaultTaskManagerResourceProfile(),
+                                            getDefaultSlotResourceProfile());
+                            getSlotManager()
+                                    .processResourceRequirements(
+                                            createResourceRequirementsForSingleSlot());
+
+                            // a second request is only sent if the first request timed out
+                            secondSlotRequestFuture.get();
+                        });
+            }
+        };
+    }
+
+    /**
+     * Tests that the SlotManager retries allocating a slot if the TaskExecutor#requestSlot call
+     * fails.
+     */
+    @Test
+    public void testSlotRequestFailure() throws Exception {
+        final ManuallyTriggeredScheduledExecutorService executor =
+                new ManuallyTriggeredScheduledExecutorService();
+
+        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<AllocationID> allocationIds = new ArrayBlockingQueue<>(2);
+
+        final TestingTaskExecutorGateway taskExecutorGateway =
+                new TestingTaskExecutorGatewayBuilder()
+                        .setRequestSlotFunction(
+                                FunctionUtils.uncheckedFunction(
+                                        requestSlotParameters -> {
+                                            allocationIds.put(requestSlotParameters.f2);
+                                            return slotRequestFutureIterator.next();
+                                        }))
+                        .createTestingTaskExecutorGateway();
+
+        final ResourceID resourceId = ResourceID.generate();
+        final TaskExecutorConnection taskManagerConnection =
+                new TaskExecutorConnection(resourceId, taskExecutorGateway);
+        final SlotReport slotReport = new SlotReport();
+
+        new Context() {
+            {
+                mainThreadExecutor = executor;
+                runTest(
+                        () -> {
+                            getSlotManager()
+                                    .registerTaskManager(
+                                            taskManagerConnection,
+                                            slotReport,
+                                            getDefaultTaskManagerResourceProfile(),
+                                            getDefaultSlotResourceProfile());
+
+                            getSlotManager().processResourceRequirements(resourceRequirements);
+
+                            final AllocationID firstAllocationId = allocationIds.take();
+                            assertThat(allocationIds, is(empty()));
+
+                            // let the first attempt fail --> this should trigger a second attempt
+                            slotRequestFuture1.completeExceptionally(
+                                    new SlotAllocationException("Test exception."));
+
+                            executor.triggerAll();
+                            assertThat(
+                                    getTotalResourceCount(
+                                            getResourceTracker().getAcquiredResources(jobId)),
+                                    is(1));
+
+                            slotRequestFuture2.complete(Acknowledge.get());
+                            final AllocationID secondAllocationId = allocationIds.take();
+                            assertThat(allocationIds, is(empty()));
+
+                            executor.triggerAll();
+                            final TaskManagerSlotInformation slot =
+                                    getTaskManagerTracker()
+                                            .getAllocatedOrPendingSlot(secondAllocationId)
+                                            .get();
+
+                            assertThat(slot.getState(), is(SlotState.ALLOCATED));
+                            assertEquals(jobId, slot.getJobId());
+
+                            assertFalse(
+                                    getTaskManagerTracker()
+                                            .getAllocatedOrPendingSlot(firstAllocationId)
+                                            .isPresent());
+                        });
+            }
+        };
+    }
+
+    // ---------------------------------------------------------------------------------------------
+    // Allocation update
+    // ---------------------------------------------------------------------------------------------
+
+    @Test
+    public void testAllocationUpdatesIgnoredIfTaskExecutorUnregistered() throws Exception {
+        final ManuallyTriggeredScheduledExecutorService executor =
+                new ManuallyTriggeredScheduledExecutorService();
+
+        final TestingTaskExecutorGateway taskExecutorGateway =
+                new TestingTaskExecutorGatewayBuilder()
+                        // it is important that the returned future is already completed
+                        // otherwise it will be cancelled when the task executor is unregistered
+                        .setRequestSlotFunction(
+                                ignored -> CompletableFuture.completedFuture(Acknowledge.get()))

Review comment:
       Shouldn't the future be completed after unregistering the task manager?




----------------------------------------------------------------
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