public interface SlotPoolGateway extends AllocatedSlotActions, RpcGateway
SlotPool
.Modifier and Type | Method and Description |
---|---|
CompletableFuture<LogicalSlot> |
allocateSlot(SlotRequestId slotRequestId,
ScheduledUnit scheduledUnit,
SlotProfile slotProfile,
boolean allowQueuedScheduling,
Time timeout)
Requests to allocate a slot for the given
ScheduledUnit . |
void |
connectToResourceManager(ResourceManagerGateway resourceManagerGateway)
Connects the SlotPool to the given ResourceManager.
|
void |
disconnectResourceManager()
Disconnects the slot pool from its current Resource Manager.
|
void |
failAllocation(AllocationID allocationID,
Exception cause)
Fails the slot with the given allocation id.
|
CompletableFuture<Boolean> |
offerSlot(TaskManagerLocation taskManagerLocation,
TaskManagerGateway taskManagerGateway,
SlotOffer slotOffer)
Offers a slot to the
SlotPool . |
CompletableFuture<Collection<SlotOffer>> |
offerSlots(TaskManagerLocation taskManagerLocation,
TaskManagerGateway taskManagerGateway,
Collection<SlotOffer> offers)
Offers multiple slots to the
SlotPool . |
CompletableFuture<Acknowledge> |
registerTaskManager(ResourceID resourceID)
Registers a TaskExecutor with the given
ResourceID at SlotPool . |
CompletableFuture<Acknowledge> |
releaseTaskManager(ResourceID resourceId,
Exception cause)
Releases a TaskExecutor with the given
ResourceID from the SlotPool . |
void |
suspend() |
releaseSlot
getAddress, getHostname
void suspend()
void connectToResourceManager(ResourceManagerGateway resourceManagerGateway)
resourceManagerGateway
- The RPC gateway for the resource manager.void disconnectResourceManager()
The slot pool will still be able to serve slots from its internal pool.
CompletableFuture<Acknowledge> registerTaskManager(ResourceID resourceID)
ResourceID
at SlotPool
.resourceID
- identifying the TaskExecutor to registerCompletableFuture<Acknowledge> releaseTaskManager(ResourceID resourceId, Exception cause)
ResourceID
from the SlotPool
.resourceId
- identifying the TaskExecutor which shall be released from the SlotPoolcause
- for the releasing of the TaskManagerCompletableFuture<Boolean> offerSlot(TaskManagerLocation taskManagerLocation, TaskManagerGateway taskManagerGateway, SlotOffer slotOffer)
SlotPool
. The slot offer can be accepted or
rejected.taskManagerLocation
- from which the slot offer originatestaskManagerGateway
- to talk to the slot offererslotOffer
- slot which is offered to the SlotPool
CompletableFuture<Collection<SlotOffer>> offerSlots(TaskManagerLocation taskManagerLocation, TaskManagerGateway taskManagerGateway, Collection<SlotOffer> offers)
SlotPool
. The slot offerings can be
individually accepted or rejected by returning the collection of accepted
slot offers.taskManagerLocation
- from which the slot offers originatetaskManagerGateway
- to talk to the slot offereroffers
- slot offers which are offered to the SlotPool
void failAllocation(AllocationID allocationID, Exception cause)
allocationID
- identifying the slot which is being failedcause
- of the failureCompletableFuture<LogicalSlot> allocateSlot(SlotRequestId slotRequestId, ScheduledUnit scheduledUnit, SlotProfile slotProfile, boolean allowQueuedScheduling, Time timeout)
ScheduledUnit
. The request
is uniquely identified by the provided SlotRequestId
which can also
be used to release the slot via AllocatedSlotActions.releaseSlot(SlotRequestId, SlotSharingGroupId, Throwable)
.
The allocated slot will fulfill the requested ResourceProfile
and it
is tried to place it on one of the location preferences.
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.
slotRequestId
- identifying the requested slotscheduledUnit
- for which to allocate slotslotProfile
- profile that specifies the requirements for the requested slotallowQueuedScheduling
- true if the slot request can be queued (e.g. the returned future must not be completed)timeout
- for the operationLogicalSlot
Copyright © 2014–2019 The Apache Software Foundation. All rights reserved.