[
https://issues.apache.org/jira/browse/FLINK-13100?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
zhijiang reassigned FLINK-13100:
--------------------------------
Assignee: zhijiang
> Fix the unexpected IOException during FileBufferReader#nextBuffer
> -----------------------------------------------------------------
>
> Key: FLINK-13100
> URL: https://issues.apache.org/jira/browse/FLINK-13100
> Project: Flink
> Issue Type: Sub-task
> Components: Runtime / Network
> Reporter: zhijiang
> Assignee: zhijiang
> Priority: Blocker
>
> In the implementation of FileBufferReader#nextBuffer, we expect the next
> memory segment always available based on the assumption that the nextBuffer
> call could only happen when the previous buffer was recycled before.
> Otherwise it would throw an IOException in current implementation.
> In fact, the above assumption is not making sense based on the credit-based
> and zero-copy features in network. The detail processes are as follows:
> * The netty thread finishes calling the channel.writeAndFlush() in
> PartitionRequestQueue and adds a listener to handle the ChannelFuture later.
> Before future done, the corresponding buffer is not recycled because of
> zero-copy improvement.
> * Before the previous future done, the netty thread could trigger next
> writeAndFlush via processing addCredit message, then
> FileBufferReader#nextBuffer would throw exception because of previous buffer
> not recycled.
> We thought of several ways for solving this potential bug:
> * It does not trigger the next writeAndFlush before the previous future
> done. To do so it has to maintain the future state and check it in relevant
> actions. I wonder it might bring performance regression in network throughput
> and bring extra state management.
> * Adjust the implementation of current FileBufferReader. We ever regarded
> the blocking partition view as always available based on the next buffer read
> ahead, so it would be always added into available queue in
> PartitionRequestQueue. Actually this next buffer ahead only simplifies the
> process of BoundedBlockingSubpartitionReader#notifyDataAvailable. The view
> availability could be judged based on available buffers in FileBufferReader
> instead of next buffer ahead. When the buffer is recycled into
> FileBufferReader after writeAndFlush done, it could call notifyDataAvailable
> to add this view into available queue in PartitionRequestQueue.
> I prefer the second way because it would not bring any bad impacts.
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)