dmvk commented on a change in pull request #18238:
URL: https://github.com/apache/flink/pull/18238#discussion_r776614745



##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/allocator/StateLocalitySlotAssigner.java
##########
@@ -0,0 +1,301 @@
+/*
+ * 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.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph;
+import org.apache.flink.runtime.executiongraph.ArchivedExecutionJobVertex;
+import org.apache.flink.runtime.executiongraph.ArchivedExecutionVertex;
+import org.apache.flink.runtime.instance.SlotSharingGroupId;
+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.strategy.ExecutionVertexID;
+import org.apache.flink.runtime.state.KeyGroupRange;
+import org.apache.flink.runtime.state.KeyGroupRangeAssignment;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.flink.shaded.guava30.com.google.common.base.MoreObjects;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.PriorityQueue;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+
+/** A {@link SlotAssigner} that assigns slots based on the number of local key 
groups. */
+public class StateLocalitySlotAssigner implements SlotAssigner {
+
+    private static class AllocationScore<T extends Comparable<? super T>>
+            implements Comparable<AllocationScore<T>> {
+
+        private final T id;
+        private final AllocationID allocationId;
+
+        public AllocationScore(T id, AllocationID allocationId, int score) {
+            this.id = id;
+            this.allocationId = allocationId;
+            this.score = score;
+        }
+
+        private final int score;
+
+        public T getId() {
+            return id;
+        }
+
+        public AllocationID getAllocationId() {
+            return allocationId;
+        }
+
+        public int getScore() {
+            return score;
+        }
+
+        @Override
+        public int compareTo(StateLocalitySlotAssigner.AllocationScore<T> 
other) {
+            int result = Integer.compare(score, other.score);
+            if (result != 0) {
+                return result;
+            }
+            result = other.allocationId.compareTo(allocationId);
+            if (result != 0) {
+                return result;
+            }
+            return other.id.compareTo(id);
+        }
+
+        @Override
+        public String toString() {
+            return MoreObjects.toStringHelper(this)
+                    .add("id", id)
+                    .add("allocationId", allocationId)
+                    .add("score", score)
+                    .toString();
+        }
+    }
+
+    private final Map<AllocationID, Map<JobVertexID, KeyGroupRange>> locality;
+    private final Map<JobVertexID, Integer> maxParallelism;
+
+    public StateLocalitySlotAssigner(ArchivedExecutionGraph 
archivedExecutionGraph) {
+        this.locality = calculateLocalKeyGroups(archivedExecutionGraph);
+        this.maxParallelism =
+                StreamSupport.stream(
+                                
archivedExecutionGraph.getVerticesTopologically().spliterator(),
+                                false)
+                        .collect(
+                                Collectors.toMap(
+                                        
ArchivedExecutionJobVertex::getJobVertexId,
+                                        
ArchivedExecutionJobVertex::getMaxParallelism));
+    }
+
+    @Override
+    public List<SlotSharingSlotAllocator.ExecutionSlotSharingGroupAndSlot> 
assignSlots(
+            Collection<? extends SlotInfo> slots,
+            Collection<SlotSharingSlotAllocator.ExecutionSlotSharingGroup> 
groups) {
+        final PriorityQueue<AllocationScore<String>> groupScore =
+                new PriorityQueue<>(Comparator.reverseOrder());
+
+        final Map<JobVertexID, Integer> parallelism = new HashMap<>();
+        groups.forEach(
+                group ->
+                        group.getContainedExecutionVertices()
+                                .forEach(
+                                        evi ->
+                                                parallelism.merge(
+                                                        evi.getJobVertexId(), 
1, Integer::sum)));
+
+        for (SlotSharingSlotAllocator.ExecutionSlotSharingGroup group : 
groups) {
+            final Map<AllocationID, Integer> allocationScore = 
calculateScore(group, parallelism);
+            allocationScore.forEach(
+                    (allocationId, score) ->
+                            groupScore.add(
+                                    new AllocationScore<>(group.getId(), 
allocationId, score)));
+        }
+        final List<SlotSharingSlotAllocator.ExecutionSlotSharingGroupAndSlot> 
result =
+                new ArrayList<>();
+        final Map<String, SlotSharingSlotAllocator.ExecutionSlotSharingGroup> 
groupsById =
+                groups.stream()
+                        .collect(
+                                Collectors.toMap(
+                                        
SlotSharingSlotAllocator.ExecutionSlotSharingGroup::getId,
+                                        Function.identity()));
+        final Map<AllocationID, ? extends SlotInfo> slotsById =
+                slots.stream()
+                        .collect(Collectors.toMap(SlotInfo::getAllocationId, 
Function.identity()));
+        AllocationScore<String> item;
+        while ((item = groupScore.poll()) != null) {
+            System.out.println("======= " + item);
+            @Nullable
+            final SlotSharingSlotAllocator.ExecutionSlotSharingGroup group =
+                    groupsById.get(item.getId());
+            if (group != null) {
+                @Nullable final SlotInfo slot = 
slotsById.remove(item.getAllocationId());
+                if (slot != null) {
+                    result.add(
+                            new 
SlotSharingSlotAllocator.ExecutionSlotSharingGroupAndSlot(
+                                    group, slot));
+                    Objects.requireNonNull(groupsById.remove(item.getId()));
+                }
+            }
+        }
+
+        // Let's distribute remaining slots with no score...
+        final Iterator<? extends SlotInfo> remainingSlots = 
slotsById.values().iterator();
+        for (SlotSharingSlotAllocator.ExecutionSlotSharingGroup group : 
groupsById.values()) {
+            result.add(
+                    new 
SlotSharingSlotAllocator.ExecutionSlotSharingGroupAndSlot(
+                            group, remainingSlots.next()));
+            remainingSlots.remove();
+        }
+
+        return result;
+    }
+
+    public Map<AllocationID, Integer> calculateScore(
+            SlotSharingSlotAllocator.ExecutionSlotSharingGroup group,
+            Map<JobVertexID, Integer> parallelism) {
+        final Map<AllocationID, Integer> score = new HashMap<>();
+        for (ExecutionVertexID evi : group.getContainedExecutionVertices()) {
+            if (maxParallelism.containsKey(evi.getJobVertexId())) {
+                final KeyGroupRange kgr =
+                        
KeyGroupRangeAssignment.computeKeyGroupRangeForOperatorIndex(
+                                maxParallelism.get(evi.getJobVertexId()),
+                                parallelism.get(evi.getJobVertexId()),
+                                evi.getSubtaskIndex());
+                locality.forEach(
+                        (allocationId, potentials) -> {
+                            @Nullable
+                            final KeyGroupRange prev = 
potentials.get(evi.getJobVertexId());
+                            if (prev != null) {
+                                final int intersection =
+                                        
prev.getIntersection(kgr).getNumberOfKeyGroups();
+                                if (intersection > 0) {
+                                    score.merge(
+                                            allocationId,
+                                            
prev.getIntersection(kgr).getNumberOfKeyGroups(),

Review comment:
       👍 




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