[ 
https://issues.apache.org/jira/browse/FLINK-10431?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16663662#comment-16663662
 ] 

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_r228147167
 
 

 ##########
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolGateway.java
 ##########
 @@ -145,17 +149,61 @@
         * <p>If the returned future must not be completed right away (a.k.a. 
the slot request
         * can be queued), allowQueuedScheduling must be set to true.
         *
+        * @deprecated this method will be removed once the handling of slot 
sharing is completely extracted from the slot
+        * pool into a dedicated {@link Scheduler} component. The call is then 
replaced by calls to
+        * {@link #getAvailableSlotsInformation()}, {@link 
#allocateAvailableSlot(SlotRequestId, AllocationID)}, and
+        * {@link #requestNewAllocatedSlot(SlotRequestId, ResourceProfile, 
Time)}.
+        *
         * @param slotRequestId identifying the requested slot
         * @param scheduledUnit for which to allocate slot
         * @param slotProfile profile that specifies the requirements for the 
requested slot
         * @param allowQueuedScheduling true if the slot request can be queued 
(e.g. the returned future must not be completed)
         * @param timeout for the operation
         * @return Future which is completed with the allocated {@link 
LogicalSlot}
         */
+       @Deprecated
        CompletableFuture<LogicalSlot> allocateSlot(
                        SlotRequestId slotRequestId,
                        ScheduledUnit scheduledUnit,
                        SlotProfile slotProfile,
                        boolean allowQueuedScheduling,
                        @RpcTimeout Time timeout);
+
+       /**
+        * Returns a list of {@link SlotInfo} objects about all slots that are 
currently available in the slot
+        * pool.
+        *
+        * @return a list of {@link SlotInfo} objects about all slots that are 
currently available in the slot pool.
+        */
+       @Nonnull
+       List<SlotInfo> getAvailableSlotsInformation();
+
+       /**
+        * Allocates the available slot with the given allocation id under the 
given request id. This method returns
+        * {@code null} if no slot with the given allocation id is available.
+        *
+        * @param slotRequestId identifying the requested slot
+        * @param allocationID the allocation id of the requested available slot
+        * @return the previously available slot with the given allocation id 
or {@code null} if no such slot existed.
+        */
+       @Nullable
+       AllocatedSlot allocateAvailableSlot(
 
 Review comment:
   In general, I like the idea, but I wonder how this is easily possible? For 
this methods, I agree but the interface also has `requestNewAllocatedSlot` 
which also exposes `AllocatedSlot` and typically is followed by attempts to 
assign payload.

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

Reply via email to