[
https://issues.apache.org/jira/browse/FLINK-7456?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16326447#comment-16326447
]
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_r161576253
--- Diff:
flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskCancelAsyncProducerConsumerITCase.java
---
@@ -84,7 +84,7 @@ public void testCancelAsyncProducerAndConsumer() throws
Exception {
config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, 1);
config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 1);
config.setInteger(TaskManagerOptions.MEMORY_SEGMENT_SIZE, 4096);
-
config.setInteger(TaskManagerOptions.NETWORK_NUM_BUFFERS, 8);
+
config.setInteger(TaskManagerOptions.NETWORK_NUM_BUFFERS, 16);
--- End diff --
just a note for the curious: this test can cope with higher number of
network buffers and is waiting for all of them to be blocked - increasing this
to `9` would have been enough here though (we require 2 exclusive buffers now
per default, while 1 was the minimum per incoming channel)
> 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)