[
https://issues.apache.org/jira/browse/FLINK-7456?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16326432#comment-16326432
]
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_r161546199
--- Diff:
flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientHandlerTest.java
---
@@ -372,16 +379,18 @@ public void testNotifyCreditAvailableAfterReleased()
throws Exception {
assertEquals(2, inputChannel.getUnannouncedCredit());
- // The PartitionRequestClient is tied to
PartitionRequestClientHandler currently, so we
- // have to notify credit available in
CreditBasedClientHandler explicitly
- handler.notifyCreditAvailable(inputChannel);
-
// Release the input channel
inputGate.releaseAllResources();
channel.runPendingTasks();
- // It will not notify credits for released input channel
+ // It should send partition request first, and send
close request after releasing input channel,
+ // but will not notify credits for released input
channel.
+ Object readFromOutbound = channel.readOutbound();
+ assertThat(readFromOutbound,
instanceOf(PartitionRequest.class));
+ assertEquals(2, ((PartitionRequest)
readFromOutbound).credit);
+ readFromOutbound = channel.readOutbound();
+ assertThat(readFromOutbound,
instanceOf(CloseRequest.class));
--- End diff --
put these after `inputGate.releaseAllResources()`?
> 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
> Priority: Major
> 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
(v7.6.3#76005)