[ https://issues.apache.org/jira/browse/FLINK-10431?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16660839#comment-16660839 ]
ASF GitHub Bot commented on FLINK-10431: ---------------------------------------- tillrohrmann 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_r227447242 ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java ########## @@ -1676,6 +1684,35 @@ public void updateAccumulators(AccumulatorSnapshot accumulatorSnapshot) { } } + /** + * Computes and returns a set with the prior allocation ids from all execution vertices in the graph. + */ + private Set<AllocationID> computeAllPriorAllocationIds() { + HashSet<AllocationID> allPreviousAllocationIds = new HashSet<>(); + Iterable<ExecutionJobVertex> ejvIterable = getVerticesTopologically(); + for (ExecutionJobVertex executionJobVertex : ejvIterable) { + for (ExecutionVertex executionVertex : executionJobVertex.getTaskVertices()) { Review comment: `getAllExecutionVertices` should be easier to use here. ---------------------------------------------------------------- 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)