azagrebin commented on a change in pull request #13181:
URL: https://github.com/apache/flink/pull/13181#discussion_r484436699
##########
File path:
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SlotSharingExecutionSlotAllocator.java
##########
@@ -161,158 +173,66 @@ private SharedSlot getOrAllocateSharedSlot(
.thenCompose(slotProfile ->
slotProvider.allocatePhysicalSlot(
new
PhysicalSlotRequest(physicalSlotRequestId, slotProfile,
slotWillBeOccupiedIndefinitely)))
.thenApply(PhysicalSlotRequest.Result::getPhysicalSlot);
- return new SharedSlot(physicalSlotRequestId,
physicalSlotResourceProfile, group, physicalSlotFuture);
+ return new SharedSlot(
+ physicalSlotRequestId,
+ physicalSlotResourceProfile,
+ group,
+ physicalSlotFuture,
+ slotWillBeOccupiedIndefinitely,
+ this::releaseSharedSlot);
});
}
+ private void releaseSharedSlot(ExecutionSlotSharingGroup
executionSlotSharingGroup) {
+ SharedSlot slot = sharedSlots.remove(executionSlotSharingGroup);
+ if (slot != null) {
+ slotProvider.cancelSlotRequest(
+ slot.getPhysicalSlotRequestId(),
+ new FlinkException("Slot is being returned from
SlotSharingExecutionSlotAllocator."));
+ } else {
+ LOG.debug("There is no slot for
ExecutionSlotSharingGroup {} to release", executionSlotSharingGroup);
Review comment:
I think we could try to eliminate the duplicated
`SlotSharingExecutionSlotAllocator#releaseSharedSlot` call by adding the
`!isEmpty` check at the end of `SharedSlot#release`:
```
if (!isEmpty()) {
requestedLogicalSlots.clear();
releaseCallback.accept(executionSlotSharingGroup);
}
```
and by hardening `SharedSlot#removeLogicalSlotRequest` with:
```
Preconditions.checkState(
requestedLogicalSlots.removeKeyB(logicalSlotRequestId) != null,
"Trying to remove a logical slot request which has been either already
removed or never created.");
```
wdyt?
----------------------------------------------------------------
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.
For queries about this service, please contact Infrastructure at:
[email protected]