[
https://issues.apache.org/jira/browse/FLINK-8523?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16558813#comment-16558813
]
ASF GitHub Bot commented on FLINK-8523:
---------------------------------------
NicoK commented on a change in pull request #5381: [FLINK-8523][network] Stop
assigning floating buffers for blocked input channels in exactly-once mode
URL: https://github.com/apache/flink/pull/5381#discussion_r205576747
##########
File path:
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java
##########
@@ -485,6 +494,23 @@ public void requestPartitions() throws IOException,
InterruptedException {
}
}
+ @Override
+ public void blockInputChannel(int channelIndex) {
+ InputChannel inputChannel =
indexToInputChannelMap.get(channelIndex);
+ if (inputChannel == null) {
+ throw new IllegalStateException("Could not find input
channel from the channel index " + channelIndex);
Review comment:
both argumentations sound right:
how about `checkArgument(0 <= channelIndex && channelIndex <
numberOfInputChannels)` and `checkState(inputChannel != null)`
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]
> Stop assigning floating buffers for blocked input channels in exactly-once
> mode
> -------------------------------------------------------------------------------
>
> Key: FLINK-8523
> URL: https://issues.apache.org/jira/browse/FLINK-8523
> Project: Flink
> Issue Type: Sub-task
> Components: Network
> Affects Versions: 1.5.0, 1.6.0
> Reporter: zhijiang
> Assignee: zhijiang
> Priority: Major
> Labels: pull-request-available
>
> In exactly-once mode, the input channel is set blocked state when reading
> barrier from it. And the blocked state will be released after barrier
> alignment or cancelled.
>
> In credit-based network flow control, we should avoid assigning floating
> buffers for blocked input channels because the buffers after barrier will not
> be processed by operator until alignment.
> To do so, we can fully make use of floating buffers and speed up barrier
> alignment in some extent.
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)