[ https://issues.apache.org/jira/browse/FLINK-10431?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16663400#comment-16663400 ]
ASF GitHub Bot commented on FLINK-10431: ---------------------------------------- StefanRRichter commented on a change in pull request #6898: [FLINK-10431] Extraction of scheduling-related code from SlotPool into preliminary Scheduler URL: https://github.com/apache/flink/pull/6898#discussion_r228066391 ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPool.java ########## @@ -324,78 +297,98 @@ public void disconnectResourceManager() { boolean allowQueuedScheduling, Time allocationTimeout) { - log.debug("Received slot request [{}] for task: {}", slotRequestId, task.getTaskToExecute()); - - final SlotSharingGroupId slotSharingGroupId = task.getSlotSharingGroupId(); - - if (slotSharingGroupId != null) { - // allocate slot with slot sharing - final SlotSharingManager multiTaskSlotManager = slotSharingManagers.computeIfAbsent( - slotSharingGroupId, - id -> new SlotSharingManager( - id, - this, - providerAndOwner)); - - final SlotSharingManager.MultiTaskSlotLocality multiTaskSlotLocality; - - try { - if (task.getCoLocationConstraint() != null) { - multiTaskSlotLocality = allocateCoLocatedMultiTaskSlot( - task.getCoLocationConstraint(), - multiTaskSlotManager, - slotProfile, - allowQueuedScheduling, - allocationTimeout); + return CompletableFuture.completedFuture(null).thenComposeAsync((i) -> { Review comment: In the end yes, absolutely right. But this is currently on purpose together with some of your other comments that there is still slot pool code around that could be removed. The purpose is you can switch back and forth now between the new and the old code path by changing a single line in the job master and to make the change as incremental as possible, as we previously discussed. ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on 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 > Extract scheduling-related code from SlotPool > --------------------------------------------- > > Key: FLINK-10431 > URL: https://issues.apache.org/jira/browse/FLINK-10431 > Project: Flink > Issue Type: Sub-task > Components: Distributed Coordination > Affects Versions: 1.7.0 > Reporter: Stefan Richter > Assignee: Stefan Richter > Priority: Major > Labels: pull-request-available > > The other half of the current scheduling logic is the management of slot > sharing and is located in the SlotPool. We need to extract this logic into > our new Scheduler component from the previous step. This leaves us with a > simpler SlotPool that mainly cares about obtaining, holding, and releasing > slots in interaction with a ResourceManager. The new Scheduler can now > identify slot sharing groups and interacts with the SlotPool. -- This message was sent by Atlassian JIRA (v7.6.3#76005)