flink-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "ASF GitHub Bot (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (FLINK-8581) Improve performance for low latency network
Date Mon, 08 Oct 2018 12:41:02 GMT

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

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

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

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestQueue.java
 ##########
 @@ -354,4 +404,64 @@ public void operationComplete(ChannelFuture future) throws Exception
{
 			}
 		}
 	}
+
+	private static class RegisterPeriodicFlushEvent {
+		private final NetworkSequenceViewReader reader;
+		private final long flushTimeout;
+
+		public RegisterPeriodicFlushEvent(NetworkSequenceViewReader reader, long flushTimeout)
{
+			this.reader = checkNotNull(reader);
+			this.flushTimeout = flushTimeout;
+		}
+
+		public NetworkSequenceViewReader getReader() {
+			return reader;
+		}
+
+		public long getFlushTimeout() {
+			return flushTimeout;
+		}
+	}
+
+	private static class PeriodicFlushes {
+		private final Map<Long, List<NetworkSequenceViewReader>> periodicFlushes =
new HashMap<>();
+		private final Map<NetworkSequenceViewReader, Long> flushTimeouts = new HashMap<>();
 
 Review comment:
   Is it worth having `flushTimeouts` in memory only to optimise cancellation (and one debug
check in `register()`)?
   
   Currently, this is used in two places:
   - in `register()` to check whether we have already added the reader - we could instead
go through the list in `periodicFlushes` if we really wanted to
   - in `cancel()` to retrieve the right flush timeout so that we can easily remove the reader
from `periodicFlushes` - we could iterate over it instead
   -> both alternative are a bit slower, but only happen during registration (if even)
and cancellation and are therefore not performance critical

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

Mime
View raw message