[
https://issues.apache.org/jira/browse/FLINK-12536?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16874735#comment-16874735
]
Piotr Nowojski commented on FLINK-12536:
----------------------------------------
> 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.
I agree it's not a very big issue because it affects only non credit based
model, still if we can improve it and some contributor is willing to work, I
think that's fine.
> 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
I would say that "In any healthy setup _usually_ we have extremely few buffer".
Load spikes, hick ups and catching ups after restoring from state can and will
happen even in the most healthy setup. Secondly writing/reading even a couple
of buffers can block network stack (and whole mailbox) for tens of seconds -
not usually, but on a daily basis.
> The new implementation means that I/O is now queues behind other potentially
>large I/O ops
That one indeed might cause some regressions in other scenarios. I haven't
though this through, but even in the current setup, those IOs are interfering
with one another, especially with non SSDs setups (but that's still true to
some extent for SSDs). I would still prefer to have a degree of control about
the IO operations, by setting the number of writer/reader threads and enqueue
operations, while keeping the network stack non blocking.
As I wrote at the beginning, I think this is a rather low priority issue, so if
we do not want to take risks here since this is non-default obsoleted code
path, I'm fine with not fixing 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)