KarmaGYZ commented on a change in pull request #14647:
URL: https://github.com/apache/flink/pull/14647#discussion_r565845168



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

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




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

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


Reply via email to