KarmaGYZ commented on code in PR #22913: URL: https://github.com/apache/flink/pull/22913#discussion_r1251614787
########## flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/FreeSlotCandidatesManagerTest.java: ########## @@ -0,0 +1,96 @@ +/* + * 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.jobmaster.slotpool; + +import org.apache.flink.runtime.clusterframework.types.AllocationID; +import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.clusterframework.types.ResourceProfile; +import org.apache.flink.runtime.jobmaster.JobMasterId; +import org.apache.flink.runtime.jobmaster.RpcTaskManagerGateway; +import org.apache.flink.runtime.jobmaster.SlotInfo; +import org.apache.flink.runtime.taskexecutor.TestingTaskExecutorGatewayBuilder; +import org.apache.flink.runtime.taskmanager.TaskManagerLocation; + +import org.junit.jupiter.api.Test; + +import java.net.InetAddress; +import java.util.Arrays; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; + +import static org.assertj.core.api.Assertions.assertThat; + +class FreeSlotCandidatesManagerTest { Review Comment: Missing javadoc. ########## flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/FreeSlotCandidatesManagerTest.java: ########## @@ -0,0 +1,96 @@ +/* + * 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.jobmaster.slotpool; + +import org.apache.flink.runtime.clusterframework.types.AllocationID; +import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.clusterframework.types.ResourceProfile; +import org.apache.flink.runtime.jobmaster.JobMasterId; +import org.apache.flink.runtime.jobmaster.RpcTaskManagerGateway; +import org.apache.flink.runtime.jobmaster.SlotInfo; +import org.apache.flink.runtime.taskexecutor.TestingTaskExecutorGatewayBuilder; +import org.apache.flink.runtime.taskmanager.TaskManagerLocation; + +import org.junit.jupiter.api.Test; + +import java.net.InetAddress; +import java.util.Arrays; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; + +import static org.assertj.core.api.Assertions.assertThat; + +class FreeSlotCandidatesManagerTest { + + @Test + void testReserveSlot() { + ResourceID resourceId = ResourceID.generate(); + SlotInfo slotInfo1 = createAllocatedSlot(resourceId); + SlotInfo slotInfo2 = createAllocatedSlot(resourceId); + Map<AllocationID, SlotInfo> slots = new HashMap<>(); Review Comment: ```suggestion final ResourceID resourceId = ResourceID.generate(); final SlotInfo slotInfo1 = createAllocatedSlot(resourceId); final SlotInfo slotInfo2 = createAllocatedSlot(resourceId); final Map<AllocationID, SlotInfo> slots = new HashMap<>(); ``` ########## flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/FreeSlotCandidatesManagerTest.java: ########## @@ -0,0 +1,96 @@ +/* + * 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.jobmaster.slotpool; + +import org.apache.flink.runtime.clusterframework.types.AllocationID; +import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.clusterframework.types.ResourceProfile; +import org.apache.flink.runtime.jobmaster.JobMasterId; +import org.apache.flink.runtime.jobmaster.RpcTaskManagerGateway; +import org.apache.flink.runtime.jobmaster.SlotInfo; +import org.apache.flink.runtime.taskexecutor.TestingTaskExecutorGatewayBuilder; +import org.apache.flink.runtime.taskmanager.TaskManagerLocation; + +import org.junit.jupiter.api.Test; + +import java.net.InetAddress; +import java.util.Arrays; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; + +import static org.assertj.core.api.Assertions.assertThat; + +class FreeSlotCandidatesManagerTest { + + @Test + void testReserveSlot() { + ResourceID resourceId = ResourceID.generate(); + SlotInfo slotInfo1 = createAllocatedSlot(resourceId); + SlotInfo slotInfo2 = createAllocatedSlot(resourceId); + Map<AllocationID, SlotInfo> slots = new HashMap<>(); + slots.put(slotInfo1.getAllocationId(), slotInfo1); + slots.put(slotInfo2.getAllocationId(), slotInfo2); + + FreeSlotCandidatesManager freeSlotCandidatesManager = + new FreeSlotCandidatesManager(slots.keySet(), slots::get, ignore -> 0d); + for (AllocationID candidate : freeSlotCandidatesManager.getAvailableSlots()) { + SlotInfo selectSlot = freeSlotCandidatesManager.getSlotInfo(candidate); + assertThat(slots.remove(selectSlot.getAllocationId())).isEqualTo(selectSlot); + freeSlotCandidatesManager.reserveSlot(selectSlot); + break; + } + + assertThat(freeSlotCandidatesManager.getAvailableSlots()) + .hasSize(1) + .containsAnyOf(slotInfo1.getAllocationId(), slotInfo2.getAllocationId()); + } + + @Test + void testCreateNewCandidatesWithoutBlockedSlots() { + ResourceID resourceId = ResourceID.generate(); + SlotInfo slotInfo1 = createAllocatedSlot(resourceId); + SlotInfo slotInfo2 = createAllocatedSlot(resourceId); + Map<AllocationID, SlotInfo> slots = new HashMap<>(); Review Comment: ditto ########## flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/FreeSlotCandidatesManagerTest.java: ########## @@ -0,0 +1,96 @@ +/* + * 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.jobmaster.slotpool; + +import org.apache.flink.runtime.clusterframework.types.AllocationID; +import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.clusterframework.types.ResourceProfile; +import org.apache.flink.runtime.jobmaster.JobMasterId; +import org.apache.flink.runtime.jobmaster.RpcTaskManagerGateway; +import org.apache.flink.runtime.jobmaster.SlotInfo; +import org.apache.flink.runtime.taskexecutor.TestingTaskExecutorGatewayBuilder; +import org.apache.flink.runtime.taskmanager.TaskManagerLocation; + +import org.junit.jupiter.api.Test; + +import java.net.InetAddress; +import java.util.Arrays; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; + +import static org.assertj.core.api.Assertions.assertThat; + +class FreeSlotCandidatesManagerTest { + + @Test + void testReserveSlot() { + ResourceID resourceId = ResourceID.generate(); + SlotInfo slotInfo1 = createAllocatedSlot(resourceId); + SlotInfo slotInfo2 = createAllocatedSlot(resourceId); + Map<AllocationID, SlotInfo> slots = new HashMap<>(); + slots.put(slotInfo1.getAllocationId(), slotInfo1); + slots.put(slotInfo2.getAllocationId(), slotInfo2); + + FreeSlotCandidatesManager freeSlotCandidatesManager = + new FreeSlotCandidatesManager(slots.keySet(), slots::get, ignore -> 0d); + for (AllocationID candidate : freeSlotCandidatesManager.getAvailableSlots()) { + SlotInfo selectSlot = freeSlotCandidatesManager.getSlotInfo(candidate); + assertThat(slots.remove(selectSlot.getAllocationId())).isEqualTo(selectSlot); + freeSlotCandidatesManager.reserveSlot(selectSlot); + break; + } + + assertThat(freeSlotCandidatesManager.getAvailableSlots()) + .hasSize(1) + .containsAnyOf(slotInfo1.getAllocationId(), slotInfo2.getAllocationId()); + } + + @Test + void testCreateNewCandidatesWithoutBlockedSlots() { + ResourceID resourceId = ResourceID.generate(); + SlotInfo slotInfo1 = createAllocatedSlot(resourceId); + SlotInfo slotInfo2 = createAllocatedSlot(resourceId); + Map<AllocationID, SlotInfo> slots = new HashMap<>(); + slots.put(slotInfo1.getAllocationId(), slotInfo1); + slots.put(slotInfo2.getAllocationId(), slotInfo2); + + FreeSlotCandidatesManager freeSlotCandidatesManager = Review Comment: ```suggestion final FreeSlotCandidatesManager freeSlotCandidatesManager = ``` ########## flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/FreeSlotCandidatesManagerTest.java: ########## @@ -0,0 +1,96 @@ +/* + * 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.jobmaster.slotpool; + +import org.apache.flink.runtime.clusterframework.types.AllocationID; +import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.clusterframework.types.ResourceProfile; +import org.apache.flink.runtime.jobmaster.JobMasterId; +import org.apache.flink.runtime.jobmaster.RpcTaskManagerGateway; +import org.apache.flink.runtime.jobmaster.SlotInfo; +import org.apache.flink.runtime.taskexecutor.TestingTaskExecutorGatewayBuilder; +import org.apache.flink.runtime.taskmanager.TaskManagerLocation; + +import org.junit.jupiter.api.Test; + +import java.net.InetAddress; +import java.util.Arrays; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; + +import static org.assertj.core.api.Assertions.assertThat; + +class FreeSlotCandidatesManagerTest { + + @Test + void testReserveSlot() { + ResourceID resourceId = ResourceID.generate(); + SlotInfo slotInfo1 = createAllocatedSlot(resourceId); + SlotInfo slotInfo2 = createAllocatedSlot(resourceId); + Map<AllocationID, SlotInfo> slots = new HashMap<>(); + slots.put(slotInfo1.getAllocationId(), slotInfo1); + slots.put(slotInfo2.getAllocationId(), slotInfo2); + + FreeSlotCandidatesManager freeSlotCandidatesManager = + new FreeSlotCandidatesManager(slots.keySet(), slots::get, ignore -> 0d); + for (AllocationID candidate : freeSlotCandidatesManager.getAvailableSlots()) { + SlotInfo selectSlot = freeSlotCandidatesManager.getSlotInfo(candidate); + assertThat(slots.remove(selectSlot.getAllocationId())).isEqualTo(selectSlot); + freeSlotCandidatesManager.reserveSlot(selectSlot); + break; + } + + assertThat(freeSlotCandidatesManager.getAvailableSlots()) + .hasSize(1) + .containsAnyOf(slotInfo1.getAllocationId(), slotInfo2.getAllocationId()); + } + + @Test + void testCreateNewCandidatesWithoutBlockedSlots() { + ResourceID resourceId = ResourceID.generate(); + SlotInfo slotInfo1 = createAllocatedSlot(resourceId); + SlotInfo slotInfo2 = createAllocatedSlot(resourceId); + Map<AllocationID, SlotInfo> slots = new HashMap<>(); + slots.put(slotInfo1.getAllocationId(), slotInfo1); + slots.put(slotInfo2.getAllocationId(), slotInfo2); + + FreeSlotCandidatesManager freeSlotCandidatesManager = + new FreeSlotCandidatesManager(slots.keySet(), slots::get, ignore -> 0d); + assertThat(freeSlotCandidatesManager.getAvailableSlots()).hasSize(2); + + FreeSlotCandidatesManager freeSlotCandidatesManagerWithoutBlockedSlots = Review Comment: ```suggestion final FreeSlotCandidatesManager freeSlotCandidatesManagerWithoutBlockedSlots = ``` ########## flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/FreeSlotCandidatesManager.java: ########## @@ -0,0 +1,105 @@ +/* + * 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.jobmaster.slotpool; + +import org.apache.flink.runtime.clusterframework.types.AllocationID; +import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.jobmaster.SlotInfo; +import org.apache.flink.util.Preconditions; + +import java.util.HashSet; +import java.util.Set; +import java.util.function.Function; +import java.util.stream.Collectors; + +/** Manage free slots as candidates, used for {@link SlotSelectionStrategy}. */ +public class FreeSlotCandidatesManager { + private final Set<AllocationID> candidates; + private final Function<AllocationID, SlotInfo> slotInfoLookup; + private final Function<ResourceID, Double> taskExecutorUtilizationLookup; + + public FreeSlotCandidatesManager( + Set<AllocationID> candidates, + Function<AllocationID, SlotInfo> slotInfoLookup, + Function<ResourceID, Double> taskExecutorUtilizationLookup) { + this.candidates = new HashSet<>(candidates); + this.slotInfoLookup = slotInfoLookup; + this.taskExecutorUtilizationLookup = taskExecutorUtilizationLookup; + } + + /** + * Get allocation id of all available slots. + * + * @return allocation id of available slots + */ + public Set<AllocationID> getAvailableSlots() { Review Comment: Do we intend to make `candidate` modifiable outside the manager? ########## flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/EvenlySpreadOutLocationPreferenceSlotSelectionStrategy.java: ########## @@ -33,15 +32,12 @@ class EvenlySpreadOutLocationPreferenceSlotSelectionStrategy @Nonnull @Override protected Optional<SlotInfoAndLocality> selectWithoutLocationPreference( - @Nonnull Collection<SlotInfoWithUtilization> availableSlots, + @Nonnull FreeSlotCandidatesManager freeSlotCandidates, @Nonnull ResourceProfile resourceProfile) { - return availableSlots.stream() - .filter( - slotInfoWithUtilization -> - slotInfoWithUtilization - .getResourceProfile() - .isMatching(resourceProfile)) - .min(Comparator.comparing(SlotInfoWithUtilization::getTaskExecutorUtilization)) + return freeSlotCandidates.getAvailableSlots().stream() + .map(freeSlotCandidates::getSlotInfo) + .filter(slotInfo -> slotInfo.getResourceProfile().isMatching(resourceProfile)) + .min(Comparator.comparing(freeSlotCandidates::getTaskExecutorUtilization)) .map( slotInfoWithUtilization -> Review Comment: ```suggestion slotInfo -> ``` ########## flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SlotSharingExecutionSlotAllocator.java: ########## @@ -225,35 +229,66 @@ private void cancelLogicalSlotRequest(ExecutionVertexID executionVertexId, Throw return assignments; } - private SharedSlot getOrAllocateSharedSlot( - ExecutionSlotSharingGroup executionSlotSharingGroup, + private void assignSharedSlots( Review Comment: I lean to return the SSG set without exiting shared slot. It's not a good practice for me to modify the input argument. Same as other methods. ########## flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SlotSharingExecutionSlotAllocator.java: ########## @@ -225,35 +229,66 @@ private void cancelLogicalSlotRequest(ExecutionVertexID executionVertexId, Throw return assignments; } - private SharedSlot getOrAllocateSharedSlot( - ExecutionSlotSharingGroup executionSlotSharingGroup, + private void assignSharedSlots( Review Comment: ```suggestion private void tryAssignExistingSharedSlots( ``` ########## flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/AllocatedSlotPool.java: ########## @@ -117,6 +117,13 @@ public interface AllocatedSlotPool { */ Collection<FreeSlotInfo> getFreeSlotsInformation(); + /** + * Returns information about all currently free slots. + * + * @return free slot information + */ + FreeSlotCandidatesManager getFreeSlotsCandidates(); Review Comment: Why do we still need the `getFreeSlotsInformation`? ########## flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/FreeSlotCandidatesManagerTest.java: ########## @@ -0,0 +1,96 @@ +/* + * 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.jobmaster.slotpool; + +import org.apache.flink.runtime.clusterframework.types.AllocationID; +import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.clusterframework.types.ResourceProfile; +import org.apache.flink.runtime.jobmaster.JobMasterId; +import org.apache.flink.runtime.jobmaster.RpcTaskManagerGateway; +import org.apache.flink.runtime.jobmaster.SlotInfo; +import org.apache.flink.runtime.taskexecutor.TestingTaskExecutorGatewayBuilder; +import org.apache.flink.runtime.taskmanager.TaskManagerLocation; + +import org.junit.jupiter.api.Test; + +import java.net.InetAddress; +import java.util.Arrays; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; + +import static org.assertj.core.api.Assertions.assertThat; + +class FreeSlotCandidatesManagerTest { + + @Test + void testReserveSlot() { + ResourceID resourceId = ResourceID.generate(); + SlotInfo slotInfo1 = createAllocatedSlot(resourceId); + SlotInfo slotInfo2 = createAllocatedSlot(resourceId); + Map<AllocationID, SlotInfo> slots = new HashMap<>(); + slots.put(slotInfo1.getAllocationId(), slotInfo1); + slots.put(slotInfo2.getAllocationId(), slotInfo2); + + FreeSlotCandidatesManager freeSlotCandidatesManager = + new FreeSlotCandidatesManager(slots.keySet(), slots::get, ignore -> 0d); + for (AllocationID candidate : freeSlotCandidatesManager.getAvailableSlots()) { + SlotInfo selectSlot = freeSlotCandidatesManager.getSlotInfo(candidate); + assertThat(slots.remove(selectSlot.getAllocationId())).isEqualTo(selectSlot); Review Comment: Why we need to remove the selected slot from `slots`? ########## flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/FreeSlotCandidatesManagerTest.java: ########## @@ -0,0 +1,96 @@ +/* + * 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.jobmaster.slotpool; + +import org.apache.flink.runtime.clusterframework.types.AllocationID; +import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.clusterframework.types.ResourceProfile; +import org.apache.flink.runtime.jobmaster.JobMasterId; +import org.apache.flink.runtime.jobmaster.RpcTaskManagerGateway; +import org.apache.flink.runtime.jobmaster.SlotInfo; +import org.apache.flink.runtime.taskexecutor.TestingTaskExecutorGatewayBuilder; +import org.apache.flink.runtime.taskmanager.TaskManagerLocation; + +import org.junit.jupiter.api.Test; + +import java.net.InetAddress; +import java.util.Arrays; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; + +import static org.assertj.core.api.Assertions.assertThat; + +class FreeSlotCandidatesManagerTest { + + @Test + void testReserveSlot() { + ResourceID resourceId = ResourceID.generate(); + SlotInfo slotInfo1 = createAllocatedSlot(resourceId); + SlotInfo slotInfo2 = createAllocatedSlot(resourceId); + Map<AllocationID, SlotInfo> slots = new HashMap<>(); + slots.put(slotInfo1.getAllocationId(), slotInfo1); + slots.put(slotInfo2.getAllocationId(), slotInfo2); + + FreeSlotCandidatesManager freeSlotCandidatesManager = Review Comment: ditto ########## flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/FreeSlotCandidatesManager.java: ########## @@ -0,0 +1,105 @@ +/* + * 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.jobmaster.slotpool; + +import org.apache.flink.runtime.clusterframework.types.AllocationID; +import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.jobmaster.SlotInfo; +import org.apache.flink.util.Preconditions; + +import java.util.HashSet; +import java.util.Set; +import java.util.function.Function; +import java.util.stream.Collectors; + +/** Manage free slots as candidates, used for {@link SlotSelectionStrategy}. */ +public class FreeSlotCandidatesManager { Review Comment: I just wonder if it makes more sense to request all the info of candidates by AllocationID? From my understanding, AllocationID is the indexing key in this class. -- 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: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
