You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "ASF GitHub Bot (JIRA)" <ji...@apache.org> on 2018/10/08 10:02:00 UTC

[jira] [Commented] (FLINK-8581) Improve performance for low latency network

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

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_r223309406
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartition.java
 ##########
 @@ -268,6 +291,25 @@ public void flushAll() {
 		}
 	}
 
+	@Override
+	public void flushAllLocal() {
+		for (ResultSubpartition localSubpartition : localSubpartitions) {
+			localSubpartition.flush();
+		}
+	}
+
+	@Override
+	public void setFlushTimeout(long flushTimeout) {
+		checkState(!this.flushTimeout.isPresent(), "Flush timeout can not be set twice");
+		for (ResultSubpartition subpartition: remoteSubpartitionsMissingPeriodicFlushes) {
+			checkState(subpartition.isLocal().isPresent());
+			checkState(!subpartition.isLocal().get());
+			subpartition.registerPeriodicFlush(flushTimeout);
+		}
+		remoteSubpartitionsMissingPeriodicFlushes.clear();
 
 Review comment:
   There may exit race condition between `setFlushTimeout()` and `createSubpartitionView()`?
   If the task thread invokes `setFlushTimeout` and clears the `remoteSubpartitionsMissingPeriodicFlushes`, at the same time, the netty thread invokes creating subpartition view and adding this subpartition to the `remoteSubpartitionsMissingPeriodicFlushes` before setting `flushTimeout`.

----------------------------------------------------------------
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:
users@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)