[
https://issues.apache.org/jira/browse/FLINK-12536?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16873993#comment-16873993
]
Stephan Ewen commented on FLINK-12536:
--------------------------------------
Can you guys share why you believe this is important to change?
- As far as I understand, this affects only the non-credit-based model, which
is supported but meant to be used only in rare cases and to be removed at some
point.
- This adds quite a bit of extra complexity, like adding dependencies to the
I/O manager in the alignment stack
- In any healthy setup, we have extremely few buffers to read after
alignment, which should not be so performance critical that it would warrant a
bigger addition in code complexity.
- The new implementation means that I/O is now queues behind other
potentially large I/O ops (like from hash table reading or sort spill reading).
That means we can introduce delays in those I/O and stall the streaming
pipeline.
- While the mailbox thread should generally not block, I think that these few
occasional I/O ops of a legacy mode are fine. They happen only once in a while
(after checkpoint) in a legacy mode. Do we ever expect these ops to block for
significant enough time? If yes, would we expect that everything else is still
low-latency in that case when the I/O ops block long, or could we actually
tolerate this?
tl:dr - this change seems a lot like premature optimization to me. We introduce
complexity for an assumed problem where I am skeptical that it is a problem in
practice. I am happy to be convinced of the opposite, but checking whether this
is PMO would be good before merging this.
> Make BufferOrEventSequence#getNext() non-blocking
> -------------------------------------------------
>
> Key: FLINK-12536
> URL: https://issues.apache.org/jira/browse/FLINK-12536
> Project: Flink
> Issue Type: Sub-task
> Components: Runtime / Network
> Affects Versions: 1.9.0
> Reporter: Piotr Nowojski
> Assignee: Congxian Qiu(klion26)
> Priority: Major
> Labels: pull-request-available
> Time Spent: 10m
> Remaining Estimate: 0h
>
> Currently it is non-blocking in case of credit-based flow control (default),
> however for \{{SpilledBufferOrEventSequence}} it is blocking on reading from
> file. We might want to consider reimplementing it to be non blocking with
> {{CompletableFuture<?> isAvailable()}} method.
>
> Otherwise we will block mailbox processing for the duration of reading from
> file - for example we will block processing time timers and potentially in
> the future network flushes.
>
> This is not a high priority change, since it affects non-default
> configuration option AND at the moment only processing time timers are
> planned to be moved to the mailbox for 1.9.
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)