tillrohrmann commented on a change in pull request #13464:
URL: https://github.com/apache/flink/pull/13464#discussion_r498221342



##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/JobScopedResourceTracker.java
##########
@@ -0,0 +1,225 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.slots.ResourceCounter;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+
+/**
+ * Tracks resource for a single job.
+ */
+class JobScopedResourceTracker {
+
+       private static final Logger LOG = 
LoggerFactory.getLogger(JobScopedResourceTracker.class);
+
+       // only for logging purposes
+       private final JobID jobId;
+
+       private final ResourceCounter resourceRequirements = new 
ResourceCounter();
+       private final BiDirectionalResourceToRequirementMapping 
resourceToRequirementMapping = new BiDirectionalResourceToRequirementMapping();
+       private final ResourceCounter excessResources = new ResourceCounter();
+
+       JobScopedResourceTracker(JobID jobId) {
+               this.jobId = Preconditions.checkNotNull(jobId);
+       }
+
+       public void notifyResourceRequirements(Collection<ResourceRequirement> 
newResourceRequirements) {
+               Preconditions.checkNotNull(newResourceRequirements);
+
+               resourceRequirements.clear();
+               for (ResourceRequirement newResourceRequirement : 
newResourceRequirements) {
+                       
resourceRequirements.incrementCount(newResourceRequirement.getResourceProfile(),
 newResourceRequirement.getNumberOfRequiredSlots());
+               }
+               findExcessSlots();
+               tryAssigningExcessSlots();
+       }
+
+       public void notifyAcquiredResource(ResourceProfile resourceProfile) {
+               Preconditions.checkNotNull(resourceProfile);
+               final Optional<ResourceProfile> matchingRequirement = 
findMatchingRequirement(resourceProfile);
+               if (matchingRequirement.isPresent()) {
+                       
resourceToRequirementMapping.incrementCount(matchingRequirement.get(), 
resourceProfile, 1);
+               } else {
+                       LOG.debug("Job {} acquired excess resource {}.", 
resourceProfile, jobId);
+                       excessResources.incrementCount(resourceProfile, 1);
+               }
+       }
+
+       private Optional<ResourceProfile> 
findMatchingRequirement(ResourceProfile resourceProfile) {
+               for (Map.Entry<ResourceProfile, Integer> requirementCandidate : 
resourceRequirements.getResourceProfilesWithCount().entrySet()) {
+                       ResourceProfile requirementProfile = 
requirementCandidate.getKey();
+
+                       // beware the order when matching resources to 
requirements, because ResourceProfile.UNKNOWN (which only
+                       // occurs as a requirement) does not match any resource!
+                       if (resourceProfile.isMatching(requirementProfile) && 
requirementCandidate.getValue() > 
resourceToRequirementMapping.getNumFulfillingResources(requirementProfile)) {
+                               return Optional.of(requirementProfile);
+                       }
+               }
+               return Optional.empty();
+       }
+
+       public void notifyLostResource(ResourceProfile resourceProfile) {
+               Preconditions.checkNotNull(resourceProfile);
+               if (excessResources.getResourceCount(resourceProfile) > 0) {
+                       LOG.trace("Job {} lost excess resource {}.", jobId, 
resourceProfile);
+                       excessResources.decrementCount(resourceProfile, 1);
+                       return;
+               }
+
+               Set<ResourceProfile> fulfilledRequirements = 
resourceToRequirementMapping.getRequirementsFulfilledBy(resourceProfile).keySet();
+
+               if (!fulfilledRequirements.isEmpty()) {
+                       // determine for which of the requirements, that the 
resource could be used for, the resource count should be reduced for
+                       ResourceProfile assignedRequirement = null;
+
+                       for (ResourceProfile requirementProfile : 
fulfilledRequirements) {
+                               assignedRequirement = requirementProfile;
+
+                               // try finding a requirement that has too many 
resources; if non are exceeding the requirements we deduct
+                               // the resource from any requirement having 
such a resource
+                               if 
(resourceRequirements.getResourceCount(requirementProfile) < 
resourceToRequirementMapping.getNumFulfillingResources(requirementProfile)) {
+                                       break;
+                               }
+                       }
+
+                       if (assignedRequirement == null) {
+                               // safeguard against programming errors
+                               throw new 
IllegalStateException(String.format("Job %s lost a (non-excess) resource %s but 
no requirement was assigned to it.", jobId, resourceProfile));
+                       }
+
+                       
resourceToRequirementMapping.decrementCount(assignedRequirement, 
resourceProfile, 1);
+
+                       tryAssigningExcessSlots();
+               } else {
+                       LOG.warn("Job {} lost a resource {} but no such 
resource was tracked.", jobId, resourceProfile);
+               }
+       }
+
+       public Collection<ResourceRequirement> getRequiredResources() {

Review comment:
       Should we give this method a clearer name: `getMissingResources`, 
`getOutstandingResources`, or so?

##########
File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/DefaultResourceTrackerTest.java
##########
@@ -0,0 +1,144 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.slots.ResourceRequirement;
+import org.apache.flink.util.TestLogger;
+
+import org.hamcrest.collection.IsMapContaining;
+import org.junit.Test;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Map;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.Matchers.contains;
+import static org.hamcrest.Matchers.empty;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Tests for the {@link DefaultResourceTracker}.
+ *
+ * <p>Note: The majority is of the tracking logic is covered by the {@link 
JobScopedResourceTrackerTest}.
+ */
+public class DefaultResourceTrackerTest extends TestLogger {
+
+       @Test
+       public void testInitialBehavior() {
+               DefaultResourceTracker tracker = new DefaultResourceTracker();
+
+               assertThat(tracker.isEmpty(), is(true));
+               tracker.notifyLostResource(JobID.generate(), 
ResourceProfile.ANY);
+               tracker.clear();

Review comment:
       Why are these calls necessary?

##########
File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/DefaultResourceTrackerTest.java
##########
@@ -0,0 +1,144 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.slots.ResourceRequirement;
+import org.apache.flink.util.TestLogger;
+
+import org.hamcrest.collection.IsMapContaining;
+import org.junit.Test;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Map;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.Matchers.contains;
+import static org.hamcrest.Matchers.empty;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Tests for the {@link DefaultResourceTracker}.
+ *
+ * <p>Note: The majority is of the tracking logic is covered by the {@link 
JobScopedResourceTrackerTest}.
+ */
+public class DefaultResourceTrackerTest extends TestLogger {
+
+       @Test
+       public void testInitialBehavior() {
+               DefaultResourceTracker tracker = new DefaultResourceTracker();
+
+               assertThat(tracker.isEmpty(), is(true));
+               tracker.notifyLostResource(JobID.generate(), 
ResourceProfile.ANY);
+               tracker.clear();
+       }
+
+       @Test
+       public void testGetRequiredResources() {
+               DefaultResourceTracker tracker = new DefaultResourceTracker();
+               JobID jobId1 = JobID.generate();
+               JobID jobId2 = JobID.generate();

Review comment:
       Does it make sense to make this a fields of 
`DefaultResourceTrackerTest`? That way we would not have to create these values 
multiple times.

##########
File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/JobScopedResourceTrackerTest.java
##########
@@ -0,0 +1,257 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.Collections;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.Matchers.contains;
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.collection.IsEmptyCollection.empty;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Tests for the {@link JobScopedResourceTracker}.
+ */
+public class JobScopedResourceTrackerTest extends TestLogger {
+
+       private static final ResourceProfile PROFILE_1 = 
ResourceProfile.newBuilder().setCpuCores(1).build();
+       private static final ResourceProfile PROFILE_2 = 
ResourceProfile.newBuilder().setCpuCores(2).build();
+
+       @Test
+       public void testInitialBehavior() {
+               JobScopedResourceTracker tracker = new 
JobScopedResourceTracker(JobID.generate());
+
+               assertThat(tracker.isEmpty(), is(true));
+               assertThat(tracker.getAcquiredResources(), empty());
+               assertThat(tracker.getRequiredResources(), empty());
+
+               // should not throw an exception
+               tracker.notifyLostResource(ResourceProfile.UNKNOWN);
+       }
+
+       @Test
+       public void testIsEmpty() {
+               JobScopedResourceTracker tracker = new 
JobScopedResourceTracker(JobID.generate());
+
+               
tracker.notifyResourceRequirements(Collections.singleton(ResourceRequirement.create(ResourceProfile.UNKNOWN,
 1)));
+               assertThat(tracker.isEmpty(), is(false));
+               tracker.notifyResourceRequirements(Collections.emptyList());
+               assertThat(tracker.isEmpty(), is(true));
+
+               tracker.notifyAcquiredResource(ResourceProfile.ANY);
+               assertThat(tracker.isEmpty(), is(false));
+               tracker.notifyLostResource(ResourceProfile.ANY);
+               assertThat(tracker.isEmpty(), is(true));

Review comment:
       Same here. I would split this into two tests.

##########
File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/JobScopedResourceTrackerTest.java
##########
@@ -0,0 +1,257 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.Collections;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.Matchers.contains;
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.collection.IsEmptyCollection.empty;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Tests for the {@link JobScopedResourceTracker}.
+ */
+public class JobScopedResourceTrackerTest extends TestLogger {
+
+       private static final ResourceProfile PROFILE_1 = 
ResourceProfile.newBuilder().setCpuCores(1).build();
+       private static final ResourceProfile PROFILE_2 = 
ResourceProfile.newBuilder().setCpuCores(2).build();
+
+       @Test
+       public void testInitialBehavior() {
+               JobScopedResourceTracker tracker = new 
JobScopedResourceTracker(JobID.generate());
+
+               assertThat(tracker.isEmpty(), is(true));
+               assertThat(tracker.getAcquiredResources(), empty());
+               assertThat(tracker.getRequiredResources(), empty());
+
+               // should not throw an exception
+               tracker.notifyLostResource(ResourceProfile.UNKNOWN);

Review comment:
       I would make this a separate test with a descriptive test name.

##########
File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/JobScopedResourceTrackerTest.java
##########
@@ -0,0 +1,257 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.Collections;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.Matchers.contains;
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.collection.IsEmptyCollection.empty;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Tests for the {@link JobScopedResourceTracker}.
+ */
+public class JobScopedResourceTrackerTest extends TestLogger {
+
+       private static final ResourceProfile PROFILE_1 = 
ResourceProfile.newBuilder().setCpuCores(1).build();
+       private static final ResourceProfile PROFILE_2 = 
ResourceProfile.newBuilder().setCpuCores(2).build();
+
+       @Test
+       public void testInitialBehavior() {
+               JobScopedResourceTracker tracker = new 
JobScopedResourceTracker(JobID.generate());
+
+               assertThat(tracker.isEmpty(), is(true));
+               assertThat(tracker.getAcquiredResources(), empty());
+               assertThat(tracker.getRequiredResources(), empty());
+
+               // should not throw an exception
+               tracker.notifyLostResource(ResourceProfile.UNKNOWN);
+       }
+
+       @Test
+       public void testIsEmpty() {
+               JobScopedResourceTracker tracker = new 
JobScopedResourceTracker(JobID.generate());
+
+               
tracker.notifyResourceRequirements(Collections.singleton(ResourceRequirement.create(ResourceProfile.UNKNOWN,
 1)));
+               assertThat(tracker.isEmpty(), is(false));
+               tracker.notifyResourceRequirements(Collections.emptyList());
+               assertThat(tracker.isEmpty(), is(true));
+
+               tracker.notifyAcquiredResource(ResourceProfile.ANY);
+               assertThat(tracker.isEmpty(), is(false));
+               tracker.notifyLostResource(ResourceProfile.ANY);
+               assertThat(tracker.isEmpty(), is(true));
+       }
+
+       @Test
+       public void testRequirementsNotificationWithoutResources() {
+               JobScopedResourceTracker tracker = new 
JobScopedResourceTracker(JobID.generate());
+
+               ResourceRequirement[][] resourceRequirements = new 
ResourceRequirement[][]{
+                       new ResourceRequirement[]{
+                               ResourceRequirement.create(PROFILE_1, 4),
+                               ResourceRequirement.create(PROFILE_2, 2)},
+                       new ResourceRequirement[]{
+                               ResourceRequirement.create(PROFILE_1, 8),
+                               ResourceRequirement.create(PROFILE_2, 2)},
+                       new ResourceRequirement[]{
+                               ResourceRequirement.create(PROFILE_1, 4),
+                               ResourceRequirement.create(PROFILE_2, 2)}};
+
+               for (ResourceRequirement[] resourceRequirement : 
resourceRequirements) {
+                       
tracker.notifyResourceRequirements(Arrays.asList(resourceRequirement));
+
+                       assertThat(tracker.isEmpty(), is(false));
+                       assertThat(tracker.getAcquiredResources(), empty());
+                       assertThat(tracker.getRequiredResources(), 
containsInAnyOrder(resourceRequirement));
+               }
+
+               tracker.notifyResourceRequirements(Collections.emptyList());
+
+               assertThat(tracker.getAcquiredResources(), empty());
+               assertThat(tracker.getRequiredResources(), empty());
+       }
+
+       @Test
+       public void testRequirementsNotificationWithResources() {
+               JobScopedResourceTracker tracker = new 
JobScopedResourceTracker(JobID.generate());
+
+               ResourceRequirement[][] resourceRequirements = new 
ResourceRequirement[][]{
+                       new ResourceRequirement[]{
+                               ResourceRequirement.create(PROFILE_1, 4),
+                               ResourceRequirement.create(PROFILE_2, 2)},
+                       new ResourceRequirement[]{
+                               ResourceRequirement.create(PROFILE_1, 8),
+                               ResourceRequirement.create(PROFILE_2, 2)},
+                       new ResourceRequirement[]{
+                               ResourceRequirement.create(PROFILE_1, 4),
+                               ResourceRequirement.create(PROFILE_2, 2)}};
+
+               int numAcquiredSlotsP1 = 
resourceRequirements[0][0].getNumberOfRequiredSlots() - 1;
+               int numAcquiredSlotsP2 = 
resourceRequirements[0][1].getNumberOfRequiredSlots();
+
+               for (int x = 0; x < numAcquiredSlotsP1; x++) {
+                       tracker.notifyAcquiredResource(PROFILE_1);
+               }
+               for (int x = 0; x < numAcquiredSlotsP2; x++) {
+                       tracker.notifyAcquiredResource(PROFILE_2);
+               }
+
+               for (ResourceRequirement[] resourceRequirement : 
resourceRequirements) {
+                       
tracker.notifyResourceRequirements(Arrays.asList(resourceRequirement));
+
+                       assertThat(tracker.getAcquiredResources(), 
containsInAnyOrder(ResourceRequirement.create(PROFILE_1, numAcquiredSlotsP1), 
ResourceRequirement.create(PROFILE_2, numAcquiredSlotsP2)));
+                       assertThat(tracker.getRequiredResources(), 
contains(ResourceRequirement.create(PROFILE_1, 
resourceRequirement[0].getNumberOfRequiredSlots() - numAcquiredSlotsP1)));
+               }
+
+               tracker.notifyResourceRequirements(Collections.emptyList());
+
+               assertThat(tracker.getAcquiredResources(), 
containsInAnyOrder(ResourceRequirement.create(PROFILE_1, numAcquiredSlotsP1), 
ResourceRequirement.create(PROFILE_2, numAcquiredSlotsP2)));
+               assertThat(tracker.getRequiredResources(), empty());
+       }
+
+       @Test
+       public void testMatchingWithResourceExceedingRequirement() {
+               JobScopedResourceTracker tracker = new 
JobScopedResourceTracker(JobID.generate());
+
+               
tracker.notifyResourceRequirements(Arrays.asList(ResourceRequirement.create(PROFILE_1,
 1)));
+
+               tracker.notifyAcquiredResource(PROFILE_2);
+
+               assertThat(tracker.getAcquiredResources(), 
contains(ResourceRequirement.create(PROFILE_2, 1)));
+       }
+
+       @Test
+       public void testMatchingWithResourceLessThanRequirement() {
+               JobScopedResourceTracker tracker = new 
JobScopedResourceTracker(JobID.generate());
+
+               
tracker.notifyResourceRequirements(Arrays.asList(ResourceRequirement.create(PROFILE_2,
 1)));
+
+               tracker.notifyAcquiredResource(PROFILE_1);
+
+               assertThat(tracker.getAcquiredResources(), 
contains(ResourceRequirement.create(PROFILE_1, 1)));
+               assertThat(tracker.getRequiredResources(), 
contains(ResourceRequirement.create(PROFILE_2, 1)));
+       }
+
+       @Test
+       public void testResourceNotificationsWithoutRequirements() {
+               JobScopedResourceTracker tracker = new 
JobScopedResourceTracker(JobID.generate());
+
+               tracker.notifyAcquiredResource(ResourceProfile.ANY);
+
+               assertThat(tracker.isEmpty(), is(false));
+               assertThat(tracker.getAcquiredResources(), 
contains(ResourceRequirement.create(ResourceProfile.ANY, 1)));
+               assertThat(tracker.getRequiredResources(), empty());
+
+               tracker.notifyAcquiredResource(ResourceProfile.ANY);
+
+               assertThat(tracker.isEmpty(), is(false));
+               assertThat(tracker.getAcquiredResources(), 
contains(ResourceRequirement.create(ResourceProfile.ANY, 2)));
+               assertThat(tracker.getRequiredResources(), empty());
+
+               tracker.notifyLostResource(ResourceProfile.ANY);
+
+               assertThat(tracker.isEmpty(), is(false));
+               assertThat(tracker.getAcquiredResources(), 
contains(ResourceRequirement.create(ResourceProfile.ANY, 1)));
+               assertThat(tracker.getRequiredResources(), empty());
+
+               tracker.notifyLostResource(ResourceProfile.ANY);
+
+               assertThat(tracker.isEmpty(), is(true));
+               assertThat(tracker.getAcquiredResources(), empty());
+               assertThat(tracker.getRequiredResources(), empty());
+       }
+
+       @Test
+       public void testResourceNotificationsWithRequirements() {
+               JobScopedResourceTracker tracker = new 
JobScopedResourceTracker(JobID.generate());
+
+               ResourceRequirement[] resourceRequirementsArray = new 
ResourceRequirement[]{
+                       ResourceRequirement.create(PROFILE_1, 2),
+                       ResourceRequirement.create(PROFILE_2, 1)
+               };
+
+               
tracker.notifyResourceRequirements(Arrays.asList(resourceRequirementsArray));
+
+               for (int x = 0; x < 2; x++) {
+                       tracker.notifyAcquiredResource(PROFILE_1);
+               }
+
+               assertThat(tracker.getAcquiredResources(), 
contains(ResourceRequirement.create(PROFILE_1, 2)));
+               assertThat(tracker.getRequiredResources(), 
contains(ResourceRequirement.create(PROFILE_2, 1)));
+
+               tracker.notifyLostResource(PROFILE_1);
+
+               assertThat(tracker.getAcquiredResources(), 
contains(ResourceRequirement.create(PROFILE_1, 1)));
+               assertThat(tracker.getRequiredResources(), 
containsInAnyOrder(ResourceRequirement.create(PROFILE_1, 1), 
ResourceRequirement.create(PROFILE_2, 1)));
+       }
+
+       @Test
+       public void testRequirementReductionRetainsExceedingResources() {
+               JobScopedResourceTracker tracker = new 
JobScopedResourceTracker(JobID.generate());
+
+               
tracker.notifyResourceRequirements(Collections.singleton(ResourceRequirement.create(ResourceProfile.UNKNOWN,
 1)));
+
+               tracker.notifyAcquiredResource(ResourceProfile.ANY);
+
+               tracker.notifyResourceRequirements(Collections.emptyList());
+
+               assertThat(tracker.getAcquiredResources(), 
contains(ResourceRequirement.create(ResourceProfile.ANY, 1)));
+               assertThat(tracker.getRequiredResources(), empty());
+
+               
tracker.notifyResourceRequirements(Collections.singleton(ResourceRequirement.create(ResourceProfile.UNKNOWN,
 1)));
+
+               assertThat(tracker.getAcquiredResources(), 
contains(ResourceRequirement.create(ResourceProfile.ANY, 1)));
+               assertThat(tracker.getRequiredResources(), empty());
+       }
+
+       @Test
+       public void testExcessResourcesAreAssignedOnRequirementIncrease() {
+               JobScopedResourceTracker tracker = new 
JobScopedResourceTracker(JobID.generate());
+
+               tracker.notifyAcquiredResource(ResourceProfile.ANY);
+
+               
tracker.notifyResourceRequirements(Collections.singleton(ResourceRequirement.create(ResourceProfile.UNKNOWN,
 1)));
+
+               assertThat(tracker.getAcquiredResources(), 
contains(ResourceRequirement.create(ResourceProfile.ANY, 1)));
+       }
+
+       @Test
+       public void testExcessResourcesAreAssignedOnResourceLoss() {
+               JobScopedResourceTracker tracker = new 
JobScopedResourceTracker(JobID.generate());
+
+               tracker.notifyAcquiredResource(ResourceProfile.ANY);
+               tracker.notifyAcquiredResource(ResourceProfile.ANY);
+
+               
tracker.notifyResourceRequirements(Collections.singleton(ResourceRequirement.create(ResourceProfile.UNKNOWN,
 1)));
+
+               tracker.notifyLostResource(ResourceProfile.ANY);
+
+               assertThat(tracker.getAcquiredResources(), 
contains(ResourceRequirement.create(ResourceProfile.ANY, 1)));

Review comment:
       Same here?

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/DefaultResourceTracker.java
##########
@@ -0,0 +1,126 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+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.LinkedHashMap;
+import java.util.Map;
+
+/**
+ * Default {@link ResourceTracker} implementation.
+ */
+public class DefaultResourceTracker implements ResourceTracker {
+
+       private static final Logger LOG = 
LoggerFactory.getLogger(DefaultResourceTracker.class);
+
+       private final Map<JobID, JobScopedResourceTracker> trackers = new 
LinkedHashMap<>();
+
+       @Override
+       public void notifyResourceRequirements(JobID jobId, 
Collection<ResourceRequirement> resourceRequirements) {
+               Preconditions.checkNotNull(jobId);
+               Preconditions.checkNotNull(resourceRequirements);
+               LOG.trace("Received notification for job {} having new resource 
requirements {}.", jobId, resourceRequirements);
+               
getOrCreateTracker(jobId).notifyResourceRequirements(resourceRequirements);
+
+               if (resourceRequirements.isEmpty()) {
+                       checkWhetherTrackerCanBeRemoved(jobId, 
trackers.get(jobId));
+               }
+       }
+
+       private void checkWhetherTrackerCanBeRemoved(JobID jobId, 
JobScopedResourceTracker tracker) {
+               if (tracker.isEmpty()) {
+                       LOG.debug("Stopping tracking of resources for job {}.", 
jobId);
+                       trackers.remove(jobId);
+               }
+       }
+
+       @Override
+       public void notifyAcquiredResource(JobID jobId, ResourceProfile 
resourceProfile) {
+               Preconditions.checkNotNull(jobId);
+               Preconditions.checkNotNull(resourceProfile);
+               LOG.trace("Received notification for job {} having acquired 
resource {}.", jobId, resourceProfile);
+               
getOrCreateTracker(jobId).notifyAcquiredResource(resourceProfile);
+       }
+
+       private JobScopedResourceTracker getOrCreateTracker(JobID jobId) {
+               return trackers.computeIfAbsent(jobId, ignored -> {
+                       LOG.debug("Initiating tracking of resources for job 
{}.", jobId);
+                       return new JobScopedResourceTracker(jobId);
+               });
+       }
+
+       @Override
+       public void notifyLostResource(JobID jobId, ResourceProfile 
resourceProfile) {
+               Preconditions.checkNotNull(jobId);
+               Preconditions.checkNotNull(resourceProfile);
+               JobScopedResourceTracker tracker = trackers.get(jobId);
+
+               // during shutdown the tracker is cleared before task executors 
are unregistered,
+               // to prevent the loss of resources triggering new allocations
+               if (tracker != null) {
+                       LOG.trace("Received notification for job {} having lost 
resource {}.", jobId, resourceProfile);
+                       tracker.notifyLostResource(resourceProfile);
+
+                       checkWhetherTrackerCanBeRemoved(jobId, tracker);
+               } else {
+                       LOG.trace("Received notification for job {} having lost 
resource {}, but no such job was tracked.", jobId, resourceProfile);
+               }
+       }
+
+       @Override
+       public void clear() {
+               trackers.clear();
+       }
+
+       @Override
+       public Map<JobID, Collection<ResourceRequirement>> 
getRequiredResources() {
+               Map<JobID, Collection<ResourceRequirement>> requiredResources = 
new LinkedHashMap<>();
+               for (Map.Entry<JobID, JobScopedResourceTracker> 
jobIDJobScopedRequirementsTrackerEntry : trackers.entrySet()) {
+                       Collection<ResourceRequirement> 
exceedingOrRequiredResources = 
jobIDJobScopedRequirementsTrackerEntry.getValue().getRequiredResources();

Review comment:
       Why is this variable called `exceedingOrRequiredResources` and not 
`requiredResources`?

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/DefaultResourceTracker.java
##########
@@ -0,0 +1,126 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+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.LinkedHashMap;
+import java.util.Map;
+
+/**
+ * Default {@link ResourceTracker} implementation.
+ */
+public class DefaultResourceTracker implements ResourceTracker {
+
+       private static final Logger LOG = 
LoggerFactory.getLogger(DefaultResourceTracker.class);
+
+       private final Map<JobID, JobScopedResourceTracker> trackers = new 
LinkedHashMap<>();

Review comment:
       is it important that we use a `LinkedHashMap` here?

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/JobScopedResourceTracker.java
##########
@@ -0,0 +1,225 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.slots.ResourceCounter;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+
+/**
+ * Tracks resource for a single job.
+ */
+class JobScopedResourceTracker {
+
+       private static final Logger LOG = 
LoggerFactory.getLogger(JobScopedResourceTracker.class);
+
+       // only for logging purposes
+       private final JobID jobId;
+
+       private final ResourceCounter resourceRequirements = new 
ResourceCounter();
+       private final BiDirectionalResourceToRequirementMapping 
resourceToRequirementMapping = new BiDirectionalResourceToRequirementMapping();
+       private final ResourceCounter excessResources = new ResourceCounter();
+
+       JobScopedResourceTracker(JobID jobId) {
+               this.jobId = Preconditions.checkNotNull(jobId);
+       }
+
+       public void notifyResourceRequirements(Collection<ResourceRequirement> 
newResourceRequirements) {
+               Preconditions.checkNotNull(newResourceRequirements);
+
+               resourceRequirements.clear();
+               for (ResourceRequirement newResourceRequirement : 
newResourceRequirements) {
+                       
resourceRequirements.incrementCount(newResourceRequirement.getResourceProfile(),
 newResourceRequirement.getNumberOfRequiredSlots());
+               }
+               findExcessSlots();
+               tryAssigningExcessSlots();
+       }
+
+       public void notifyAcquiredResource(ResourceProfile resourceProfile) {
+               Preconditions.checkNotNull(resourceProfile);
+               final Optional<ResourceProfile> matchingRequirement = 
findMatchingRequirement(resourceProfile);
+               if (matchingRequirement.isPresent()) {
+                       
resourceToRequirementMapping.incrementCount(matchingRequirement.get(), 
resourceProfile, 1);
+               } else {
+                       LOG.debug("Job {} acquired excess resource {}.", 
resourceProfile, jobId);
+                       excessResources.incrementCount(resourceProfile, 1);
+               }
+       }
+
+       private Optional<ResourceProfile> 
findMatchingRequirement(ResourceProfile resourceProfile) {
+               for (Map.Entry<ResourceProfile, Integer> requirementCandidate : 
resourceRequirements.getResourceProfilesWithCount().entrySet()) {
+                       ResourceProfile requirementProfile = 
requirementCandidate.getKey();
+
+                       // beware the order when matching resources to 
requirements, because ResourceProfile.UNKNOWN (which only
+                       // occurs as a requirement) does not match any resource!
+                       if (resourceProfile.isMatching(requirementProfile) && 
requirementCandidate.getValue() > 
resourceToRequirementMapping.getNumFulfillingResources(requirementProfile)) {
+                               return Optional.of(requirementProfile);
+                       }
+               }
+               return Optional.empty();
+       }
+
+       public void notifyLostResource(ResourceProfile resourceProfile) {
+               Preconditions.checkNotNull(resourceProfile);
+               if (excessResources.getResourceCount(resourceProfile) > 0) {
+                       LOG.trace("Job {} lost excess resource {}.", jobId, 
resourceProfile);
+                       excessResources.decrementCount(resourceProfile, 1);
+                       return;
+               }
+
+               Set<ResourceProfile> fulfilledRequirements = 
resourceToRequirementMapping.getRequirementsFulfilledBy(resourceProfile).keySet();
+
+               if (!fulfilledRequirements.isEmpty()) {
+                       // determine for which of the requirements, that the 
resource could be used for, the resource count should be reduced for
+                       ResourceProfile assignedRequirement = null;
+
+                       for (ResourceProfile requirementProfile : 
fulfilledRequirements) {
+                               assignedRequirement = requirementProfile;
+
+                               // try finding a requirement that has too many 
resources; if non are exceeding the requirements we deduct
+                               // the resource from any requirement having 
such a resource
+                               if 
(resourceRequirements.getResourceCount(requirementProfile) < 
resourceToRequirementMapping.getNumFulfillingResources(requirementProfile)) {
+                                       break;
+                               }
+                       }
+
+                       if (assignedRequirement == null) {
+                               // safeguard against programming errors
+                               throw new 
IllegalStateException(String.format("Job %s lost a (non-excess) resource %s but 
no requirement was assigned to it.", jobId, resourceProfile));
+                       }
+
+                       
resourceToRequirementMapping.decrementCount(assignedRequirement, 
resourceProfile, 1);
+
+                       tryAssigningExcessSlots();
+               } else {
+                       LOG.warn("Job {} lost a resource {} but no such 
resource was tracked.", jobId, resourceProfile);

Review comment:
       Should this ever happen or is this an invalid state?

##########
File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/JobScopedResourceTrackerTest.java
##########
@@ -0,0 +1,257 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.Collections;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.Matchers.contains;
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.collection.IsEmptyCollection.empty;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Tests for the {@link JobScopedResourceTracker}.
+ */
+public class JobScopedResourceTrackerTest extends TestLogger {
+
+       private static final ResourceProfile PROFILE_1 = 
ResourceProfile.newBuilder().setCpuCores(1).build();
+       private static final ResourceProfile PROFILE_2 = 
ResourceProfile.newBuilder().setCpuCores(2).build();
+
+       @Test
+       public void testInitialBehavior() {
+               JobScopedResourceTracker tracker = new 
JobScopedResourceTracker(JobID.generate());
+
+               assertThat(tracker.isEmpty(), is(true));
+               assertThat(tracker.getAcquiredResources(), empty());
+               assertThat(tracker.getRequiredResources(), empty());
+
+               // should not throw an exception
+               tracker.notifyLostResource(ResourceProfile.UNKNOWN);
+       }
+
+       @Test
+       public void testIsEmpty() {
+               JobScopedResourceTracker tracker = new 
JobScopedResourceTracker(JobID.generate());
+
+               
tracker.notifyResourceRequirements(Collections.singleton(ResourceRequirement.create(ResourceProfile.UNKNOWN,
 1)));
+               assertThat(tracker.isEmpty(), is(false));
+               tracker.notifyResourceRequirements(Collections.emptyList());
+               assertThat(tracker.isEmpty(), is(true));
+
+               tracker.notifyAcquiredResource(ResourceProfile.ANY);
+               assertThat(tracker.isEmpty(), is(false));
+               tracker.notifyLostResource(ResourceProfile.ANY);
+               assertThat(tracker.isEmpty(), is(true));
+       }
+
+       @Test
+       public void testRequirementsNotificationWithoutResources() {
+               JobScopedResourceTracker tracker = new 
JobScopedResourceTracker(JobID.generate());
+
+               ResourceRequirement[][] resourceRequirements = new 
ResourceRequirement[][]{
+                       new ResourceRequirement[]{
+                               ResourceRequirement.create(PROFILE_1, 4),
+                               ResourceRequirement.create(PROFILE_2, 2)},
+                       new ResourceRequirement[]{
+                               ResourceRequirement.create(PROFILE_1, 8),
+                               ResourceRequirement.create(PROFILE_2, 2)},
+                       new ResourceRequirement[]{
+                               ResourceRequirement.create(PROFILE_1, 4),
+                               ResourceRequirement.create(PROFILE_2, 2)}};
+
+               for (ResourceRequirement[] resourceRequirement : 
resourceRequirements) {
+                       
tracker.notifyResourceRequirements(Arrays.asList(resourceRequirement));
+
+                       assertThat(tracker.isEmpty(), is(false));
+                       assertThat(tracker.getAcquiredResources(), empty());
+                       assertThat(tracker.getRequiredResources(), 
containsInAnyOrder(resourceRequirement));
+               }
+
+               tracker.notifyResourceRequirements(Collections.emptyList());
+
+               assertThat(tracker.getAcquiredResources(), empty());
+               assertThat(tracker.getRequiredResources(), empty());
+       }
+
+       @Test
+       public void testRequirementsNotificationWithResources() {
+               JobScopedResourceTracker tracker = new 
JobScopedResourceTracker(JobID.generate());
+
+               ResourceRequirement[][] resourceRequirements = new 
ResourceRequirement[][]{
+                       new ResourceRequirement[]{
+                               ResourceRequirement.create(PROFILE_1, 4),
+                               ResourceRequirement.create(PROFILE_2, 2)},
+                       new ResourceRequirement[]{
+                               ResourceRequirement.create(PROFILE_1, 8),
+                               ResourceRequirement.create(PROFILE_2, 2)},
+                       new ResourceRequirement[]{
+                               ResourceRequirement.create(PROFILE_1, 4),
+                               ResourceRequirement.create(PROFILE_2, 2)}};
+
+               int numAcquiredSlotsP1 = 
resourceRequirements[0][0].getNumberOfRequiredSlots() - 1;
+               int numAcquiredSlotsP2 = 
resourceRequirements[0][1].getNumberOfRequiredSlots();
+
+               for (int x = 0; x < numAcquiredSlotsP1; x++) {
+                       tracker.notifyAcquiredResource(PROFILE_1);
+               }
+               for (int x = 0; x < numAcquiredSlotsP2; x++) {
+                       tracker.notifyAcquiredResource(PROFILE_2);
+               }
+
+               for (ResourceRequirement[] resourceRequirement : 
resourceRequirements) {
+                       
tracker.notifyResourceRequirements(Arrays.asList(resourceRequirement));
+
+                       assertThat(tracker.getAcquiredResources(), 
containsInAnyOrder(ResourceRequirement.create(PROFILE_1, numAcquiredSlotsP1), 
ResourceRequirement.create(PROFILE_2, numAcquiredSlotsP2)));
+                       assertThat(tracker.getRequiredResources(), 
contains(ResourceRequirement.create(PROFILE_1, 
resourceRequirement[0].getNumberOfRequiredSlots() - numAcquiredSlotsP1)));
+               }
+
+               tracker.notifyResourceRequirements(Collections.emptyList());
+
+               assertThat(tracker.getAcquiredResources(), 
containsInAnyOrder(ResourceRequirement.create(PROFILE_1, numAcquiredSlotsP1), 
ResourceRequirement.create(PROFILE_2, numAcquiredSlotsP2)));
+               assertThat(tracker.getRequiredResources(), empty());
+       }
+
+       @Test
+       public void testMatchingWithResourceExceedingRequirement() {
+               JobScopedResourceTracker tracker = new 
JobScopedResourceTracker(JobID.generate());
+
+               
tracker.notifyResourceRequirements(Arrays.asList(ResourceRequirement.create(PROFILE_1,
 1)));
+
+               tracker.notifyAcquiredResource(PROFILE_2);
+
+               assertThat(tracker.getAcquiredResources(), 
contains(ResourceRequirement.create(PROFILE_2, 1)));
+       }
+
+       @Test
+       public void testMatchingWithResourceLessThanRequirement() {
+               JobScopedResourceTracker tracker = new 
JobScopedResourceTracker(JobID.generate());
+
+               
tracker.notifyResourceRequirements(Arrays.asList(ResourceRequirement.create(PROFILE_2,
 1)));
+
+               tracker.notifyAcquiredResource(PROFILE_1);
+
+               assertThat(tracker.getAcquiredResources(), 
contains(ResourceRequirement.create(PROFILE_1, 1)));
+               assertThat(tracker.getRequiredResources(), 
contains(ResourceRequirement.create(PROFILE_2, 1)));
+       }
+
+       @Test
+       public void testResourceNotificationsWithoutRequirements() {
+               JobScopedResourceTracker tracker = new 
JobScopedResourceTracker(JobID.generate());
+
+               tracker.notifyAcquiredResource(ResourceProfile.ANY);
+
+               assertThat(tracker.isEmpty(), is(false));
+               assertThat(tracker.getAcquiredResources(), 
contains(ResourceRequirement.create(ResourceProfile.ANY, 1)));
+               assertThat(tracker.getRequiredResources(), empty());
+
+               tracker.notifyAcquiredResource(ResourceProfile.ANY);
+
+               assertThat(tracker.isEmpty(), is(false));
+               assertThat(tracker.getAcquiredResources(), 
contains(ResourceRequirement.create(ResourceProfile.ANY, 2)));
+               assertThat(tracker.getRequiredResources(), empty());
+
+               tracker.notifyLostResource(ResourceProfile.ANY);
+
+               assertThat(tracker.isEmpty(), is(false));
+               assertThat(tracker.getAcquiredResources(), 
contains(ResourceRequirement.create(ResourceProfile.ANY, 1)));
+               assertThat(tracker.getRequiredResources(), empty());
+
+               tracker.notifyLostResource(ResourceProfile.ANY);
+
+               assertThat(tracker.isEmpty(), is(true));
+               assertThat(tracker.getAcquiredResources(), empty());
+               assertThat(tracker.getRequiredResources(), empty());
+       }
+
+       @Test
+       public void testResourceNotificationsWithRequirements() {
+               JobScopedResourceTracker tracker = new 
JobScopedResourceTracker(JobID.generate());
+
+               ResourceRequirement[] resourceRequirementsArray = new 
ResourceRequirement[]{
+                       ResourceRequirement.create(PROFILE_1, 2),
+                       ResourceRequirement.create(PROFILE_2, 1)
+               };
+
+               
tracker.notifyResourceRequirements(Arrays.asList(resourceRequirementsArray));
+
+               for (int x = 0; x < 2; x++) {
+                       tracker.notifyAcquiredResource(PROFILE_1);
+               }
+
+               assertThat(tracker.getAcquiredResources(), 
contains(ResourceRequirement.create(PROFILE_1, 2)));
+               assertThat(tracker.getRequiredResources(), 
contains(ResourceRequirement.create(PROFILE_2, 1)));
+
+               tracker.notifyLostResource(PROFILE_1);
+
+               assertThat(tracker.getAcquiredResources(), 
contains(ResourceRequirement.create(PROFILE_1, 1)));
+               assertThat(tracker.getRequiredResources(), 
containsInAnyOrder(ResourceRequirement.create(PROFILE_1, 1), 
ResourceRequirement.create(PROFILE_2, 1)));
+       }
+
+       @Test
+       public void testRequirementReductionRetainsExceedingResources() {
+               JobScopedResourceTracker tracker = new 
JobScopedResourceTracker(JobID.generate());
+
+               
tracker.notifyResourceRequirements(Collections.singleton(ResourceRequirement.create(ResourceProfile.UNKNOWN,
 1)));
+
+               tracker.notifyAcquiredResource(ResourceProfile.ANY);
+
+               tracker.notifyResourceRequirements(Collections.emptyList());
+
+               assertThat(tracker.getAcquiredResources(), 
contains(ResourceRequirement.create(ResourceProfile.ANY, 1)));
+               assertThat(tracker.getRequiredResources(), empty());
+
+               
tracker.notifyResourceRequirements(Collections.singleton(ResourceRequirement.create(ResourceProfile.UNKNOWN,
 1)));
+
+               assertThat(tracker.getAcquiredResources(), 
contains(ResourceRequirement.create(ResourceProfile.ANY, 1)));
+               assertThat(tracker.getRequiredResources(), empty());
+       }
+
+       @Test
+       public void testExcessResourcesAreAssignedOnRequirementIncrease() {
+               JobScopedResourceTracker tracker = new 
JobScopedResourceTracker(JobID.generate());
+
+               tracker.notifyAcquiredResource(ResourceProfile.ANY);
+
+               
tracker.notifyResourceRequirements(Collections.singleton(ResourceRequirement.create(ResourceProfile.UNKNOWN,
 1)));
+
+               assertThat(tracker.getAcquiredResources(), 
contains(ResourceRequirement.create(ResourceProfile.ANY, 1)));

Review comment:
       How can we tell that the excess resources are used to fulfill the 
requirement? Aren't we missing a `assertThat(tracker.getRequiredResources(), 
empty());`?

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/JobScopedResourceTracker.java
##########
@@ -0,0 +1,225 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.slots.ResourceCounter;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+
+/**
+ * Tracks resource for a single job.
+ */
+class JobScopedResourceTracker {
+
+       private static final Logger LOG = 
LoggerFactory.getLogger(JobScopedResourceTracker.class);
+
+       // only for logging purposes
+       private final JobID jobId;
+
+       private final ResourceCounter resourceRequirements = new 
ResourceCounter();
+       private final BiDirectionalResourceToRequirementMapping 
resourceToRequirementMapping = new BiDirectionalResourceToRequirementMapping();
+       private final ResourceCounter excessResources = new ResourceCounter();
+
+       JobScopedResourceTracker(JobID jobId) {
+               this.jobId = Preconditions.checkNotNull(jobId);
+       }
+
+       public void notifyResourceRequirements(Collection<ResourceRequirement> 
newResourceRequirements) {
+               Preconditions.checkNotNull(newResourceRequirements);
+
+               resourceRequirements.clear();
+               for (ResourceRequirement newResourceRequirement : 
newResourceRequirements) {
+                       
resourceRequirements.incrementCount(newResourceRequirement.getResourceProfile(),
 newResourceRequirement.getNumberOfRequiredSlots());
+               }
+               findExcessSlots();
+               tryAssigningExcessSlots();
+       }
+
+       public void notifyAcquiredResource(ResourceProfile resourceProfile) {
+               Preconditions.checkNotNull(resourceProfile);
+               final Optional<ResourceProfile> matchingRequirement = 
findMatchingRequirement(resourceProfile);
+               if (matchingRequirement.isPresent()) {
+                       
resourceToRequirementMapping.incrementCount(matchingRequirement.get(), 
resourceProfile, 1);
+               } else {
+                       LOG.debug("Job {} acquired excess resource {}.", 
resourceProfile, jobId);
+                       excessResources.incrementCount(resourceProfile, 1);
+               }
+       }
+
+       private Optional<ResourceProfile> 
findMatchingRequirement(ResourceProfile resourceProfile) {
+               for (Map.Entry<ResourceProfile, Integer> requirementCandidate : 
resourceRequirements.getResourceProfilesWithCount().entrySet()) {
+                       ResourceProfile requirementProfile = 
requirementCandidate.getKey();
+
+                       // beware the order when matching resources to 
requirements, because ResourceProfile.UNKNOWN (which only
+                       // occurs as a requirement) does not match any resource!
+                       if (resourceProfile.isMatching(requirementProfile) && 
requirementCandidate.getValue() > 
resourceToRequirementMapping.getNumFulfillingResources(requirementProfile)) {
+                               return Optional.of(requirementProfile);
+                       }
+               }
+               return Optional.empty();
+       }
+
+       public void notifyLostResource(ResourceProfile resourceProfile) {
+               Preconditions.checkNotNull(resourceProfile);
+               if (excessResources.getResourceCount(resourceProfile) > 0) {
+                       LOG.trace("Job {} lost excess resource {}.", jobId, 
resourceProfile);
+                       excessResources.decrementCount(resourceProfile, 1);
+                       return;
+               }
+
+               Set<ResourceProfile> fulfilledRequirements = 
resourceToRequirementMapping.getRequirementsFulfilledBy(resourceProfile).keySet();
+
+               if (!fulfilledRequirements.isEmpty()) {
+                       // determine for which of the requirements, that the 
resource could be used for, the resource count should be reduced for
+                       ResourceProfile assignedRequirement = null;
+
+                       for (ResourceProfile requirementProfile : 
fulfilledRequirements) {
+                               assignedRequirement = requirementProfile;
+
+                               // try finding a requirement that has too many 
resources; if non are exceeding the requirements we deduct
+                               // the resource from any requirement having 
such a resource
+                               if 
(resourceRequirements.getResourceCount(requirementProfile) < 
resourceToRequirementMapping.getNumFulfillingResources(requirementProfile)) {
+                                       break;
+                               }

Review comment:
       Should this ever happen? I thought that all slots which over fulfill the 
requirements will be added to the `excessResources` field.

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/ResourceTracker.java
##########
@@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+import org.apache.flink.runtime.slots.ResourceRequirements;
+
+import java.util.Collection;
+import java.util.Map;
+
+/**
+ * Tracks for each job how many resource are required/acquired.
+ */
+public interface ResourceTracker {
+
+       /**
+        * Notifies the tracker about a new or updated {@link 
ResourceRequirements}.
+        *
+        * @param jobId the job that that the resource requirements belongs to
+        * @param resourceRequirements new resource requirements
+        */
+       void notifyResourceRequirements(JobID jobId, 
Collection<ResourceRequirement> resourceRequirements);
+
+       /**
+        * Notifies the tracker about the acquisition of a resource with the 
given resource profile, for the given job.
+        *
+        * @param jobId the job that acquired the resource
+        * @param resourceProfile profile of the resource
+        */
+       void notifyAcquiredResource(JobID jobId, ResourceProfile 
resourceProfile);
+
+       /**
+        * Notifies the tracker about the loss of a resource with the given 
resource profile, for the given job.
+        *
+        * @param jobId the job that lost the resource
+        * @param resourceProfile profile of the resource
+        */
+       void notifyLostResource(JobID jobId, ResourceProfile resourceProfile);
+
+       /**
+        * Returns a collection of {@link ResourceRequirements} that describe 
which resources the corresponding job is
+        * in need/excess of.
+        *
+        * @return required/exceeding resources for each jobs
+        */
+       Map<JobID, Collection<ResourceRequirement>> getRequiredResources();

Review comment:
       I would suggest to give this method a different name because here it 
means the required resources to fulfill all requirements whereas 
`notifyResourceRequirements` specifies the absolute set of required resources. 
I can see that this will cause confusion (at least it happened to me).

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/ResourceTracker.java
##########
@@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.slots.ResourceRequirement;
+import org.apache.flink.runtime.slots.ResourceRequirements;
+
+import java.util.Collection;
+import java.util.Map;
+
+/**
+ * Tracks for each job how many resource are required/acquired.
+ */
+public interface ResourceTracker {
+
+       /**
+        * Notifies the tracker about a new or updated {@link 
ResourceRequirements}.
+        *
+        * @param jobId the job that that the resource requirements belongs to
+        * @param resourceRequirements new resource requirements
+        */
+       void notifyResourceRequirements(JobID jobId, 
Collection<ResourceRequirement> resourceRequirements);
+
+       /**
+        * Notifies the tracker about the acquisition of a resource with the 
given resource profile, for the given job.
+        *
+        * @param jobId the job that acquired the resource
+        * @param resourceProfile profile of the resource
+        */
+       void notifyAcquiredResource(JobID jobId, ResourceProfile 
resourceProfile);
+
+       /**
+        * Notifies the tracker about the loss of a resource with the given 
resource profile, for the given job.
+        *
+        * @param jobId the job that lost the resource
+        * @param resourceProfile profile of the resource
+        */
+       void notifyLostResource(JobID jobId, ResourceProfile resourceProfile);
+
+       /**
+        * Returns a collection of {@link ResourceRequirements} that describe 
which resources the corresponding job is
+        * in need/excess of.

Review comment:
       Is it correct that this method returns the excess resources? Need and 
excess seems to be quite different to me.




----------------------------------------------------------------
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:
[email protected]


Reply via email to