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



##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/JobScopedResourceTracker.java
##########
@@ -0,0 +1,224 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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()) {
+                       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.getFulfilledRequirementsBy(resourceProfile).getResourceProfiles();
+
+               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, otherwise use any
+                               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() {
+               final Collection<ResourceRequirement> requiredResources = new 
ArrayList<>();
+               for (Map.Entry<ResourceProfile, Integer> requirement : 
resourceRequirements.getResourceProfilesWithCount()) {
+                       ResourceProfile requirementProfile = 
requirement.getKey();
+
+                       int numRequiredResources = requirement.getValue();
+                       int numAcquiredResources = 
resourceToRequirementMapping.getNumFulfillingResources(requirementProfile);
+
+                       if (numAcquiredResources < numRequiredResources) {
+                               
requiredResources.add(ResourceRequirement.create(requirementProfile, 
numRequiredResources - numAcquiredResources));
+                       }
+
+               }
+               return requiredResources;
+       }
+
+       public Collection<ResourceRequirement> getAcquiredResources() {
+               final Set<ResourceProfile> knownResourceProfiles = new 
HashSet<>();
+               
knownResourceProfiles.addAll(resourceToRequirementMapping.getAllResourceProfiles());
+               
knownResourceProfiles.addAll(excessResources.getResourceProfiles());
+
+               final List<ResourceRequirement> acquiredResources = new 
ArrayList<>();
+               for (ResourceProfile knownResourceProfile : 
knownResourceProfiles) {
+                       int numTotalAcquiredResources = 
resourceToRequirementMapping.getNumFulfilledRequirements(knownResourceProfile) 
+ excessResources.getResourceCount(knownResourceProfile);
+                       ResourceRequirement resourceRequirement = 
ResourceRequirement.create(knownResourceProfile, numTotalAcquiredResources);
+                       acquiredResources.add(resourceRequirement);
+               }
+
+               return acquiredResources;
+       }
+
+       public boolean isEmpty() {
+               return resourceRequirements.isEmpty() && 
excessResources.isEmpty();
+       }
+
+       private void findExcessSlots() {
+               final Collection<ExcessResource> excessResources = new 
ArrayList<>();
+
+               for (ResourceProfile requirementProfile : 
resourceToRequirementMapping.getAllRequirementProfiles()) {
+                       int numTotalRequiredResources = 
resourceRequirements.getResourceCount(requirementProfile);
+                       int numTotalAcquiredResources = 
resourceToRequirementMapping.getNumFulfillingResources(requirementProfile);
+
+                       if (numTotalAcquiredResources > 
numTotalRequiredResources) {
+                               int numExcessResources = 
numTotalAcquiredResources - numTotalRequiredResources;
+
+                               for (Map.Entry<ResourceProfile, Integer> 
acquiredResource : 
resourceToRequirementMapping.getFulfillingResourcesFor(requirementProfile).getResourceProfilesWithCount())
 {
+                                       ResourceProfile acquiredResourceProfile 
= acquiredResource.getKey();
+                                       int numAcquiredResources = 
acquiredResource.getValue();
+
+                                       if (numAcquiredResources <= 
numExcessResources) {
+                                               excessResources.add(new 
ExcessResource(requirementProfile, acquiredResourceProfile, 
numAcquiredResources));
+
+                                               numExcessResources -= 
numAcquiredResources;
+                                       } else {
+                                               excessResources.add(new 
ExcessResource(requirementProfile, acquiredResourceProfile, 
numExcessResources));
+                                               break;
+                                       }
+                               }
+                       }
+               }
+
+               LOG.debug("Detected excess resources for job {}: {}", jobId, 
excessResources);
+               for (ExcessResource excessResource : excessResources) {
+                       
resourceToRequirementMapping.decrementCount(excessResource.requirementProfile, 
excessResource.resourceProfile, excessResource.numExcessResources);
+                       
this.excessResources.incrementCount(excessResource.resourceProfile, 
excessResource.numExcessResources);
+               }
+       }
+
+       private void tryAssigningExcessSlots() {
+               if (LOG.isTraceEnabled()) {
+                       LOG.trace("There are {} excess resources for job {} 
before re-assignment.", jobId, excessResources.getResourceCount());
+               }
+               // this is a quick-and-dirty solution; in the worse case we 
copy the excessResources map twice

Review comment:
       yes, this is only temporary ;)

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/JobScopedResourceTracker.java
##########
@@ -0,0 +1,224 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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()) {
+                       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.getFulfilledRequirementsBy(resourceProfile).getResourceProfiles();
+
+               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, otherwise use any

Review comment:
       If no requirement has too many resources, we deduct one resource from 
the last encountered requirement.

##########
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;
+
+/**
+ * A tracker for required/acquired resources of a job.

Review comment:
       The phrasing isn't necessarily incorrect but I can see how it can be 
misleading.

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/BiDirectionalResourceToRequirementMapping.java
##########
@@ -0,0 +1,103 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.slots.ResourceCounter;
+import org.apache.flink.util.Preconditions;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * A bi-directional mapping between required and acquired resources.
+ */
+class BiDirectionalResourceToRequirementMapping {
+       private final Map<ResourceProfile, ResourceCounter> 
requirementToFulfillingResources = new HashMap<>();
+       private final Map<ResourceProfile, ResourceCounter> 
resourceToFulfilledRequirement = new HashMap<>();
+
+       public void incrementCount(ResourceProfile requirement, ResourceProfile 
resource, int increment) {
+               Preconditions.checkNotNull(requirement);
+               Preconditions.checkNotNull(resource);
+               Preconditions.checkArgument(increment > 0);
+               internalIncrementCount(requirementToFulfillingResources, 
requirement, resource, increment);
+               internalIncrementCount(resourceToFulfilledRequirement, 
resource, requirement, increment);
+       }
+
+       public void decrementCount(ResourceProfile requirement, ResourceProfile 
resource, int decrement) {
+               Preconditions.checkNotNull(requirement);
+               Preconditions.checkNotNull(resource);
+               Preconditions.checkArgument(decrement > 0);
+               internalDecrementCount(requirementToFulfillingResources, 
requirement, resource, decrement);
+               internalDecrementCount(resourceToFulfilledRequirement, 
resource, requirement, decrement);
+       }
+
+       private static void internalIncrementCount(Map<ResourceProfile, 
ResourceCounter> primaryMap, ResourceProfile primaryKey, ResourceProfile 
secondaryKey, int increment) {
+               primaryMap
+                       .computeIfAbsent(primaryKey, ignored -> new 
ResourceCounter())
+                       .incrementCount(secondaryKey, increment);
+       }
+
+       private static void internalDecrementCount(Map<ResourceProfile, 
ResourceCounter> primaryMap, ResourceProfile primaryKey, ResourceProfile 
secondaryKey, int decrement) {
+               primaryMap.computeIfPresent(
+                       primaryKey,
+                       (resourceProfile, resourceCounter) -> {
+                               resourceCounter.decrementCount(secondaryKey, 
decrement);
+                               return resourceCounter.isEmpty() ? null : 
resourceCounter;
+                       });
+       }
+
+       public ResourceCounter getFulfillingResourcesFor(ResourceProfile 
requirement) {
+               Preconditions.checkNotNull(requirement);
+               return 
requirementToFulfillingResources.getOrDefault(requirement, 
ResourceCounter.EMPTY);
+       }
+
+       public ResourceCounter getFulfilledRequirementsBy(ResourceProfile 
resource) {
+               Preconditions.checkNotNull(resource);
+               return resourceToFulfilledRequirement.getOrDefault(resource, 
ResourceCounter.EMPTY);
+       }

Review comment:
       yes, the ResourceCounter should not be returned since it is mutable.

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/slots/ResourceCounter.java
##########
@@ -0,0 +1,94 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.slots;
+
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.util.Preconditions;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * A counter for resources.
+ *
+ * <p>ResourceCounter contains a set of {@link ResourceProfile 
ResourceProfiles} and their
+ * associated counts. The counts are always positive (> 0).
+ */
+public class ResourceCounter {
+       public static final ResourceCounter EMPTY = new 
ResourceCounter(Collections.emptyMap());
+
+       private final Map<ResourceProfile, Integer> resources;
+
+       public ResourceCounter() {
+               this(new HashMap<>());
+       }
+
+       private ResourceCounter(Map<ResourceProfile, Integer> resources) {
+               this.resources = resources;
+       }
+
+       public void incrementCount(ResourceProfile profile, int increment) {
+               Preconditions.checkArgument(increment > 0);
+               resources.compute(profile, (ignored, currentCount) -> 
currentCount == null ? increment : currentCount + increment);
+       }
+
+       public void decrementCount(ResourceProfile profile, int decrement) {
+               Preconditions.checkArgument(decrement > 0);
+               resources.computeIfPresent(profile, (ignored, currentCount) -> 
currentCount == decrement ? null : guardAgainstNegativeCount(currentCount - 
decrement));

Review comment:
       true, will adjust it.

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/BiDirectionalResourceToRequirementMapping.java
##########
@@ -0,0 +1,103 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.resourcemanager.slotmanager;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.slots.ResourceCounter;
+import org.apache.flink.util.Preconditions;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * A bi-directional mapping between required and acquired resources.
+ */
+class BiDirectionalResourceToRequirementMapping {
+       private final Map<ResourceProfile, ResourceCounter> 
requirementToFulfillingResources = new HashMap<>();
+       private final Map<ResourceProfile, ResourceCounter> 
resourceToFulfilledRequirement = new HashMap<>();
+
+       public void incrementCount(ResourceProfile requirement, ResourceProfile 
resource, int increment) {
+               Preconditions.checkNotNull(requirement);
+               Preconditions.checkNotNull(resource);
+               Preconditions.checkArgument(increment > 0);
+               internalIncrementCount(requirementToFulfillingResources, 
requirement, resource, increment);
+               internalIncrementCount(resourceToFulfilledRequirement, 
resource, requirement, increment);
+       }
+
+       public void decrementCount(ResourceProfile requirement, ResourceProfile 
resource, int decrement) {
+               Preconditions.checkNotNull(requirement);
+               Preconditions.checkNotNull(resource);
+               Preconditions.checkArgument(decrement > 0);
+               internalDecrementCount(requirementToFulfillingResources, 
requirement, resource, decrement);
+               internalDecrementCount(resourceToFulfilledRequirement, 
resource, requirement, decrement);
+       }
+
+       private static void internalIncrementCount(Map<ResourceProfile, 
ResourceCounter> primaryMap, ResourceProfile primaryKey, ResourceProfile 
secondaryKey, int increment) {
+               primaryMap
+                       .computeIfAbsent(primaryKey, ignored -> new 
ResourceCounter())
+                       .incrementCount(secondaryKey, increment);
+       }
+
+       private static void internalDecrementCount(Map<ResourceProfile, 
ResourceCounter> primaryMap, ResourceProfile primaryKey, ResourceProfile 
secondaryKey, int decrement) {
+               primaryMap.computeIfPresent(
+                       primaryKey,
+                       (resourceProfile, resourceCounter) -> {
+                               resourceCounter.decrementCount(secondaryKey, 
decrement);
+                               return resourceCounter.isEmpty() ? null : 
resourceCounter;
+                       });

Review comment:
       yes we should fail.




----------------------------------------------------------------
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:
us...@infra.apache.org


Reply via email to