[ https://issues.apache.org/jira/browse/FLINK-7416?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16268589#comment-16268589 ]
ASF GitHub Bot commented on FLINK-7416: --------------------------------------- Github user NicoK commented on a diff in the pull request: https://github.com/apache/flink/pull/4533#discussion_r153455238 --- Diff: flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientHandlerTest.java --- @@ -208,8 +244,79 @@ public void testCancelBeforeActive() throws Exception { client.cancelRequestFor(inputChannel.getInputChannelId()); } + /** + * Verifies that {@link RemoteInputChannel} is enqueued in the pipeline, and + * {@link AddCredit} message is sent to the producer. + */ + @Test + public void testNotifyCreditAvailable() throws Exception { + final CreditBasedClientHandler handler = new CreditBasedClientHandler(); + final EmbeddedChannel channel = new EmbeddedChannel(handler); + + final RemoteInputChannel inputChannel = createRemoteInputChannel(mock(SingleInputGate.class)); + + // Enqueue the input channel + handler.notifyCreditAvailable(inputChannel); + + channel.runPendingTasks(); + + // Read the enqueued msg + Object msg1 = channel.readOutbound(); + + // Should notify credit + assertEquals(msg1.getClass(), AddCredit.class); + } + + /** + * Verifies that {@link RemoteInputChannel} is enqueued in the pipeline, but {@link AddCredit} + * message is not sent actually after this input channel is released. + */ + @Test + public void testNotifyCreditAvailableAfterReleased() throws Exception { + final CreditBasedClientHandler handler = new CreditBasedClientHandler(); + final EmbeddedChannel channel = new EmbeddedChannel(handler); + + final RemoteInputChannel inputChannel = createRemoteInputChannel(mock(SingleInputGate.class)); + + // Enqueue the input channel then release it + handler.notifyCreditAvailable(inputChannel); + inputChannel.releaseAllResources(); + + channel.runPendingTasks(); + + // Read the enqueued msg + Object msg2 = channel.readOutbound(); + + // No need to notify credit for released input channel + assertEquals(msg2, null); + } + // --------------------------------------------------------------------------------------------- + private SingleInputGate createSingleInputGate() { --- End diff -- please add a javadoc about the properties of the created input gate > Implement Netty receiver outgoing pipeline for credit-based > ----------------------------------------------------------- > > Key: FLINK-7416 > URL: https://issues.apache.org/jira/browse/FLINK-7416 > 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. > The related works are : > * We define a new message called {{AddCredit}} to notify the incremental > credit during data shuffle. > * Whenever an {{InputChannel}}’s unannounced credit goes up from zero, the > channel is enqueued in the pipeline. > * Whenever the channel becomes writable, it takes the next {{InputChannel}} > and sends its unannounced credit. The credit is reset to zero after each sent. > * That way, messages are sent as often as the network has capacity and > contain as much credit as available for the channel at that point in time. > Otherwise, it would only add latency to the announcements and not increase > throughput. -- This message was sent by Atlassian JIRA (v6.4.14#64029)