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

ASF GitHub Bot commented on FLINK-8581:
---------------------------------------

zhijiangW commented on a change in pull request #6698: [FLINK-8581][network] 
Move flushing remote subpartitions from OutputFlusher to netty
URL: https://github.com/apache/flink/pull/6698#discussion_r223574052
 
 

 ##########
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestQueue.java
 ##########
 @@ -119,6 +141,22 @@ private void enqueueAvailableReader(final 
NetworkSequenceViewReader reader) thro
                }
        }
 
+       private void flushReaders(long flushTimeout) throws Exception {
+               List<NetworkSequenceViewReader> readersToFlush = 
periodicFlushes.getReaders(flushTimeout);
+
+               boolean wasEmpty = availableReaders.isEmpty();
+
+               for (NetworkSequenceViewReader reader : readersToFlush) {
+                       if (!reader.isRegisteredAsAvailable() && 
!reader.isBlocked()) {
 
 Review comment:
   I think the flush operation indicates if this reader has both unfinished 
`BufferConsumer` and credits, we still want to transport this buffer to reduce 
latency. So the conditions of available reader should cover both available 
buffers and available credits.  But the current conditions only confirm the 
reader has available credits to insert into queue. When we poll this reader 
from the queue and get next buffer to find null, it seems not make sense to 
register available reader here.
   
   So is it reasonable to adjust the conditions here to confirm this reader has 
both credits and unfinished buffers?

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


> Improve performance for low latency network
> -------------------------------------------
>
>                 Key: FLINK-8581
>                 URL: https://issues.apache.org/jira/browse/FLINK-8581
>             Project: Flink
>          Issue Type: Improvement
>          Components: Network
>    Affects Versions: 1.5.0
>            Reporter: Piotr Nowojski
>            Assignee: Piotr Nowojski
>            Priority: Major
>              Labels: pull-request-available
>




--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to