zhuzhurk commented on a change in pull request #13181:
URL: https://github.com/apache/flink/pull/13181#discussion_r484093835
##########
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 just found a case that this branch can happen and the latest
`checkNotNull(slot)` will fail.
The case is that a task finishes and returns its logical slot which is the
only logical slot of a `SharedSlot`. `SharedSlot#returnLogicalSlot()` will be
invoked first to release the shared slot and underlying physical slot. The
releasing of physical slot, however, could trigger `SharedSlot#release()` which
will try to release this shared slot once more.
I think we need either to add RELEASED/ACTIVE states for `SharedSlot` so
that it can skip duplicated releasing. Or we need to keep the `if (slot !=
null)` condition block here. Anyway, the `else` block does not make much sense
so I think we can drop it.
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]