wsry commented on a change in pull request #11877:
URL: https://github.com/apache/flink/pull/11877#discussion_r665814336



##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java
##########
@@ -357,11 +359,26 @@ public void resumeConsumption() throws IOException {
         checkState(!isReleased.get(), "Channel released.");
         checkPartitionRequestQueueInitialized();
 
+        if (initialCredit == 0) {
+            unannouncedCredit.set(0);

Review comment:
       > Something sounds wrong here. The race condition that you described 
above, does it mean that unannouncedCredit can be out of sync? That we in 
reality have released all floating buffers, channel is blocked, but actually 
unannouncedCredit > 0? And it's only fixed after calling resumeConsumption()?
   
   Yes, exactly.
   
   > And as I understand it, without your change, this problem doesn't exist, 
as floating buffers are kept assigned to the blocked channel and the 
unannouncedCredit (or maybe even assigned AddCredit that might have been sent 
to the upstream node) are consistent with the reality. Also those assigned 
floating buffers are not used because channel is blocked, but that is not a big 
issue, because thanks to the exclusive buffers, other channels can make a 
progress?
   
   That only happens when the exclusive credit is 0. If the exclusive credit is 
not 0, the allocated floating buffers will not be released and if the exclusive 
credit is 0, we release the floating buffers allocated to let other channel use 
them to avoid deadlock, an extreme case is that we only have 1 floating buffer 
and no exclusive buffer. At downstream, the unannounced credit will be reset, 
at the upstream, the available credit is also reset to 0 when resume 
consumption. 




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

To unsubscribe, e-mail: issues-unsubscr...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Reply via email to