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



##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/declarative/allocator/SlotSharingSlotAllocator.java
##########
@@ -0,0 +1,202 @@
+/*
+ * 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.scheduler.declarative.allocator;
+
+import org.apache.flink.api.common.operators.ResourceSpec;
+import org.apache.flink.configuration.MemorySize;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.instance.SlotSharingGroupId;
+import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup;
+import org.apache.flink.runtime.jobmaster.LogicalSlot;
+import org.apache.flink.runtime.jobmaster.SlotInfo;
+import org.apache.flink.runtime.jobmaster.SlotRequestId;
+import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlot;
+import org.apache.flink.runtime.jobmaster.slotpool.ResourceCounter;
+import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/** {@link SlotAllocator} implementation that supports slot sharing. */
+public class SlotSharingSlotAllocator implements 
SlotAllocator<SlotSharingAssignments> {
+
+    private final ReserveSlotFunction reserveSlot;
+    private final FreeSlotFunction freeSlot;
+
+    public SlotSharingSlotAllocator(ReserveSlotFunction reserveSlot, 
FreeSlotFunction freeSlot) {
+        this.reserveSlot = reserveSlot;
+        this.freeSlot = freeSlot;
+    }
+
+    @Override
+    public ResourceCounter calculateRequiredSlots(
+            Iterable<JobInformation.VertexInformation> vertices) {
+        int numTotalRequiredSlots = 0;
+        for (Integer requiredSlots : 
getMaxParallelismForSlotSharingGroups(vertices).values()) {
+            numTotalRequiredSlots += requiredSlots;
+        }
+        return ResourceCounter.withResource(ResourceProfile.UNKNOWN, 
numTotalRequiredSlots);
+    }
+
+    private static Map<SlotSharingGroupId, Integer> 
getMaxParallelismForSlotSharingGroups(
+            Iterable<JobInformation.VertexInformation> vertices) {
+        final Map<SlotSharingGroupId, Integer> 
maxParallelismForSlotSharingGroups = new HashMap<>();
+        for (JobInformation.VertexInformation vertex : vertices) {
+            maxParallelismForSlotSharingGroups.compute(
+                    vertex.getSlotSharingGroup().getSlotSharingGroupId(),
+                    (slotSharingGroupId, currentMaxParallelism) ->
+                            currentMaxParallelism == null
+                                    ? vertex.getParallelism()
+                                    : Math.max(currentMaxParallelism, 
vertex.getParallelism()));
+        }
+        return maxParallelismForSlotSharingGroups;
+    }
+
+    @Override
+    public Optional<SlotSharingAssignments> determineParallelism(
+            JobInformation jobInformation, Collection<? extends SlotInfo> 
freeSlots) {
+        // TODO: This can waste slots if the max parallelism for slot sharing 
groups is not equal
+        final int slotsPerSlotSharingGroup =
+                freeSlots.size() / 
jobInformation.getSlotSharingGroups().size();
+
+        if (slotsPerSlotSharingGroup == 0) {
+            // => less slots than slot-sharing groups
+            return Optional.empty();
+        }
+
+        final Iterator<? extends SlotInfo> slotIterator = freeSlots.iterator();
+        final Collection<ExecutionSlotSharingGroupAndSlot> assignments = new 
ArrayList<>();
+
+        for (SlotSharingGroup slotSharingGroup : 
jobInformation.getSlotSharingGroups()) {
+            final List<JobInformation.VertexInformation> containedJobVertices =
+                    slotSharingGroup.getJobVertexIds().stream()
+                            .map(jobInformation::getVertexInformation)
+                            .collect(Collectors.toList());
+
+            final Iterable<ExecutionSlotSharingGroup> 
sharedSlotToVertexAssignment =
+                    determineParallelismAndAssignToFutureSlotIndex(
+                            containedJobVertices, slotsPerSlotSharingGroup);
+
+            for (ExecutionSlotSharingGroup executionSlotSharingGroup :
+                    sharedSlotToVertexAssignment) {
+                final SlotInfo slotInfo = slotIterator.next();
+
+                assignments.add(
+                        new 
ExecutionSlotSharingGroupAndSlot(executionSlotSharingGroup, slotInfo));
+            }
+        }
+
+        return Optional.of(new SlotSharingAssignments(assignments));
+    }
+
+    private static Iterable<ExecutionSlotSharingGroup>
+            determineParallelismAndAssignToFutureSlotIndex(
+                    Collection<JobInformation.VertexInformation> 
containedJobVertices,
+                    int availableSlots) {
+        final Map<Integer, Set<ExecutionVertexID>> 
sharedSlotToVertexAssignment = new HashMap<>();
+        for (JobInformation.VertexInformation jobVertex : 
containedJobVertices) {
+            final int parallelism = Math.min(jobVertex.getParallelism(), 
availableSlots);
+
+            for (int i = 0; i < parallelism; i++) {
+                sharedSlotToVertexAssignment
+                        .computeIfAbsent(i, ignored -> new HashSet<>())
+                        .add(new ExecutionVertexID(jobVertex.getJobVertexID(), 
i));
+            }
+        }
+
+        return sharedSlotToVertexAssignment.values().stream()
+                .map(ExecutionSlotSharingGroup::new)
+                .collect(Collectors.toList());
+    }
+
+    @Override
+    public Map<ExecutionVertexID, LogicalSlot> reserveResources(
+            SlotSharingAssignments parallelism) {
+        final Map<ExecutionVertexID, LogicalSlot> assignedSlots = new 
HashMap<>();
+
+        for (ExecutionSlotSharingGroupAndSlot executionSlotSharingGroup :
+                parallelism.getAssignments()) {
+            final SharedSlot sharedSlot =
+                    reserveSharedSlot(executionSlotSharingGroup.getSlotInfo());
+
+            for (ExecutionVertexID executionVertexId :
+                    executionSlotSharingGroup
+                            .getExecutionSlotSharingGroup()
+                            .getContainedExecutionVertices()) {
+                final LogicalSlot logicalSlot = 
sharedSlot.allocateLogicalSlot();
+                assignedSlots.put(executionVertexId, logicalSlot);
+            }
+        }
+
+        return assignedSlots;
+    }
+
+    private SharedSlot reserveSharedSlot(SlotInfo slotInfo) {
+        final PhysicalSlot physicalSlot =
+                reserveSlot.reserveSlot(
+                        slotInfo.getAllocationId(),
+                        ResourceProfile.fromResourceSpec(ResourceSpec.DEFAULT, 
MemorySize.ZERO));

Review comment:
       that might make things simpler, yes...




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

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to