Re: [DISCUSS] Detection Flink Backpressure

2019-01-08 Thread Piotr Nowojski
Hi, > I think the direct and accurate way to monitor backpressure is stating how > many times "availableMemorySegments.wait(2000)" is triggered during > "LocalBufferPool#requestMemorySegment", and reflecting the ratio as > backpressure if not affect theperformance. This could be an interesting

Re: [DISCUSS] Detection Flink Backpressure

2019-01-07 Thread zhijiang
Hi all, The current thread strace sample can not cover all the cases. And the output queue length might not be very accurate sometimes, because the last BufferConsumer in each subpartition might not be fulfilled and still be writable although the outqueu length is already equal or more (conside

Re: [DISCUSS] Detection Flink Backpressure

2019-01-04 Thread Piotr Nowojski
Hi, In that case I think instead of fixing the current back pressure monitoring mechanism, it would be better to replace it with a new one based on output queues length. But I haven’t thought it through, especially with respect to performance implications, however my gut feeling is that it shou

Re: [DISCUSS] Detection Flink Backpressure

2019-01-03 Thread Ken Krugler
There’s the related issue of Async I/O not showing up in back pressure reporting, also due to the same issue of threads not being sampled. — Ken > On Jan 3, 2019, at 10:25 AM, Jamie Grier wrote: > > One unfortunate problem with the current back-pressure detection mechanism > is that it doesn't

Re: [DISCUSS] Detection Flink Backpressure

2019-01-03 Thread Jamie Grier
.. Maybe we should add a way to register those threads such that they are also sampled. Thoughts? On Thu, Jan 3, 2019 at 10:25 AM Jamie Grier wrote: > One unfortunate problem with the current back-pressure detection mechanism > is that it doesn't work well with all of our sources. The problem

Re: [DISCUSS] Detection Flink Backpressure

2019-01-03 Thread Jamie Grier
One unfortunate problem with the current back-pressure detection mechanism is that it doesn't work well with all of our sources. The problem is that some sources (Kinesis for sure) emit elements from threads Flink knows nothing about and therefore those stack traces aren't sampled. The result is

Re: [DISCUSS] Detection Flink Backpressure

2019-01-03 Thread Piotr Nowojski
Hi all, peiliping: I think your idea could be problematic for couple of reasons. Probably minor concern is that checkpoint time could be affected not only because of the back pressure, but also because how long does it take to actually perform the checkpoint. Bigger issues are that this bottlen

Re: [DISCUSS] Detection Flink Backpressure

2019-01-03 Thread peiliping
I have some ideas about detecting the backpressure (the blocking operators)  by checkpoint barrier . I have some flink-jobs with checkpoint , but their checkpoints will take a long time to be completed . I need to find out the blocking operators  , the same as the backpressure detection .

Re: [DISCUSS] Detection Flink Backpressure

2019-01-02 Thread Yun Gao
Hello liping, Thank you for proposing to optimize the backpressure detection! From our previous experience, we think the InputBufferPoolUsageGauge and OutputBufferPoolUsageGauge may be useful for detecting backpressure: for a list of tasks A ---> B > C, if we found that the OutputBuf