[
https://issues.apache.org/jira/browse/FLINK-7456?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16324464#comment-16324464
]
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_r161241148
--- Diff:
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/SequenceNumberingViewReader.java
---
@@ -49,10 +50,24 @@
private volatile ResultSubpartitionView subpartitionView;
+ /**
+ * The status indicating whether this reader is already enqueued in the
pipeline for transferring
+ * data or not. It is mainly used for avoid registering this reader to
the pipeline repeatedly.
+ */
+ private boolean isRegisteredAvailable;
+
+ /** The number of available buffers for holding data on the consumer
side. */
+ private int numCreditsAvailable;
--- End diff --
Just a note since I was wondering whether we need synchronization here (not
needed after verifying the things below):
1) `numCreditsAvailable` is increased via
`PartitionRequestServerHandler#channelRead0` which is a separate channel
handler than `PartitionRequestQueue` (see
`NettyProtocol#getServerChannelHandlers`). According to [Netty's thread
model](https://netty.io/wiki/new-and-noteworthy-in-4.0.html#wiki-h2-34), we
should be safe though:
> A user can specify an EventExecutor when he or she adds a handler to a
ChannelPipeline.
> - If specified, the handler methods of the ChannelHandler are always
invoked by the specified EventExecutor.
> - If unspecified, the handler methods are always invoked by the EventLoop
that its associated Channel is registered to.
2) `numCreditsAvailable` is read from
`PartitionRequestQueue#enqueueAvailableReader()` and
`SequenceNumberingViewReader#getNextBuffer()` which are both accessed by the
channel's IO thread only.
> 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)