dmvk commented on code in PR #21981:
URL: https://github.com/apache/flink/pull/21981#discussion_r1117189073


##########
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/allocator/StateLocalitySlotAssigner.java:
##########
@@ -0,0 +1,199 @@
+/*
+ * 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.adaptive.allocator;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup;
+import org.apache.flink.runtime.jobmaster.SlotInfo;
+import 
org.apache.flink.runtime.scheduler.adaptive.JobSchedulingPlan.SlotAssignment;
+import 
org.apache.flink.runtime.scheduler.adaptive.allocator.SlotSharingSlotAllocator.ExecutionSlotSharingGroup;
+import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
+import org.apache.flink.runtime.state.KeyGroupRange;
+import org.apache.flink.runtime.state.KeyGroupRangeAssignment;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.PriorityQueue;
+
+import static java.util.function.Function.identity;
+import static java.util.stream.Collectors.toMap;
+import static 
org.apache.flink.runtime.scheduler.adaptive.allocator.DefaultSlotAssigner.createExecutionSlotSharingGroups;
+
+/** A {@link SlotAssigner} that assigns slots based on the number of local key 
groups. */
+@Internal
+public class StateLocalitySlotAssigner implements SlotAssigner {
+
+    private static class AllocationScore implements 
Comparable<AllocationScore> {
+
+        private final String group;
+        private final AllocationID allocationId;
+
+        public AllocationScore(String group, AllocationID allocationId, long 
score) {
+            this.group = group;
+            this.allocationId = allocationId;
+            this.score = score;
+        }
+
+        private final long score;
+
+        public String getGroup() {
+            return group;
+        }
+
+        public AllocationID getAllocationId() {
+            return allocationId;
+        }
+
+        public long getScore() {
+            return score;
+        }
+
+        @Override
+        public int compareTo(StateLocalitySlotAssigner.AllocationScore other) {
+            int result = Long.compare(score, other.score);
+            if (result != 0) {
+                return result;
+            }
+            result = other.allocationId.compareTo(allocationId);
+            if (result != 0) {
+                return result;
+            }
+            return other.group.compareTo(group);
+        }
+    }
+
+    @Override
+    public Collection<SlotAssignment> assignSlots(
+            JobInformation jobInformation,
+            Collection<? extends SlotInfo> freeSlots,
+            VertexParallelism vertexParallelism,
+            AllocationsInfo previousAllocations,
+            StateSizeEstimates stateSizeEstimates) {
+        final List<ExecutionSlotSharingGroup> allGroups = new ArrayList<>();
+        for (SlotSharingGroup slotSharingGroup : 
jobInformation.getSlotSharingGroups()) {
+            
allGroups.addAll(createExecutionSlotSharingGroups(vertexParallelism, 
slotSharingGroup));
+        }
+        final Map<JobVertexID, Integer> parallelism = 
getParallelism(allGroups);
+
+        // PQ orders the pairs (allocationID, groupID) by score, decreasing
+        // the score is computed as the potential amount of state that would 
reside locally
+        final PriorityQueue<AllocationScore> scores =
+                new PriorityQueue<>(Comparator.reverseOrder());
+        for (ExecutionSlotSharingGroup group : allGroups) {
+            calculateScore(
+                            group,
+                            parallelism,
+                            jobInformation,
+                            previousAllocations,
+                            stateSizeEstimates)
+                    .entrySet().stream()
+                    .map(e -> new AllocationScore(group.getId(), e.getKey(), 
e.getValue()))
+                    .forEach(scores::add);
+        }
+
+        Map<String, ExecutionSlotSharingGroup> groupsById =
+                
allGroups.stream().collect(toMap(ExecutionSlotSharingGroup::getId, identity()));
+        Map<AllocationID, SlotInfo> slotsById =
+                freeSlots.stream().collect(toMap(SlotInfo::getAllocationId, 
identity()));
+        AllocationScore score;
+        final Collection<SlotAssignment> assignments = new ArrayList<>();
+        while ((score = scores.poll()) != null) {
+            SlotInfo slot = slotsById.remove(score.getAllocationId());
+            if (slot != null) {
+                ExecutionSlotSharingGroup group = 
groupsById.remove(score.getGroup());
+                if (group != null) {

Review Comment:
   👍 Awesome.
   
   nit: we can make this slightly more readable
   
   ```
               if (slotsById.containsKey(score.getAllocationId())
                       && groupsById.containsKey(score.getGroupId())) {
                   assignments.add(
                           new SlotAssignment(
                                   slotsById.remove(score.getAllocationId()),
                                   groupsById.remove(score.getGroupId())));
               }
   ```



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

To unsubscribe, e-mail: issues-unsubscr...@flink.apache.org

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

Reply via email to