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/01/08 08:03:00 UTC
[jira] [Commented] (FLINK-7456) Implement Netty sender incoming
pipeline for credit-based
[ https://issues.apache.org/jira/browse/FLINK-7456?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16315831#comment-16315831 ]
ASF GitHub Bot commented on FLINK-7456:
---------------------------------------
Github user NicoK commented on a diff in the pull request:
https://github.com/apache/flink/pull/4552#discussion_r157760366
--- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientHandler.java ---
@@ -41,30 +42,33 @@
import java.io.IOException;
import java.net.SocketAddress;
import java.util.ArrayDeque;
-import java.util.Queue;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.atomic.AtomicReference;
+/**
+ * Channel handler to read the messages of buffer response or error response from the
+ * producer, to write and flush the unannounced credits for the producer.
+ */
class PartitionRequestClientHandler extends ChannelInboundHandlerAdapter {
private static final Logger LOG = LoggerFactory.getLogger(PartitionRequestClientHandler.class);
- private final ConcurrentMap<InputChannelID, RemoteInputChannel> inputChannels = new ConcurrentHashMap<InputChannelID, RemoteInputChannel>();
-
- private final AtomicReference<Throwable> channelError = new AtomicReference<Throwable>();
+ /** Channels, which already requested partitions from the producers. */
+ private final ConcurrentMap<InputChannelID, RemoteInputChannel> inputChannels = new ConcurrentHashMap<>();
- private final BufferListenerTask bufferListener = new BufferListenerTask();
+ /** Channels, which will notify the producers about unannounced credit. */
+ private final ArrayDeque<RemoteInputChannel> inputChannelsWithCredit = new ArrayDeque<>();
- private final Queue<Object> stagedMessages = new ArrayDeque<Object>();
+ private final AtomicReference<Throwable> channelError = new AtomicReference<>();
- private final StagedMessagesHandlerTask stagedMessagesHandler = new StagedMessagesHandlerTask();
+ private final ChannelFutureListener writeListener = new WriteAndFlushNextMessageIfPossibleListener();
/**
* Set of cancelled partition requests. A request is cancelled iff an input channel is cleared
* while data is still coming in for this channel.
*/
- private final ConcurrentMap<InputChannelID, InputChannelID> cancelled = Maps.newConcurrentMap();
+ private final ConcurrentMap<InputChannelID, InputChannelID> cancelled = new ConcurrentHashMap<>();
private volatile ChannelHandlerContext ctx;
--- End diff --
it looks like you missed to migrate some of the comments that were present in `CreditBasedClientHandler` but are not present here, e.g. for `ctx`
> Implement Netty sender incoming pipeline for credit-based
> ---------------------------------------------------------
>
> Key: FLINK-7456
> URL: https://issues.apache.org/jira/browse/FLINK-7456
> Project: Flink
> Issue Type: Sub-task
> Components: Network
> Reporter: zhijiang
> Assignee: zhijiang
> Fix For: 1.5.0
>
>
> This is a part of work for credit-based network flow control.
> On sender side, each subpartition view maintains an atomic integer {{currentCredit}} from receiver. Once receiving the messages of {{PartitionRequest}} and {{AddCredit}}, the {{currentCredit}} is added by deltas.
> Each view also maintains an atomic boolean field to mark it as registered available for transfer to make sure it is enqueued in handler only once. If the {{currentCredit}} increases from zero and there are available buffers in the subpartition, the corresponding view will be enqueued for transferring data.
--
This message was sent by Atlassian JIRA
(v6.4.14#64029)