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 OutputBufferPoolUsage of task A and InputBufferPoolUsage of task B is 100%, but the OutputBufferPoolUsage of task B is less than 100%, then it should be the task B that causes the backpressure.
However, currently we think that the InputBufferPoolUsage and OutputBufferPoolUsage requires some modification to be more accurate: 1. When there are multiple inputs or outputs, the InputBufferPoolUsage and OutputBufferPoolUsage should show the maximum usage instead of the average usage [1]. 2. Currently the sender side will report backlog right before fulfilling the output Buffer. Together with the pre-allocate logic in the receiver side, the InputBufferPoolUsage may be 100% even if the data have not been received yet [2]. We may need to address these problems before adopting the InputBufferPoolUsage and OutputBufferPoolUsage as the backpressure indicator. Besides, another similar thought is that we may also add new InputBufferUsage and OutputBufferUsage metrics to show (number of queued buffers / number of all buffers) instead. Best, Yun Gao [1] https://issues.apache.org/jira/browse/FLINK-10981 [2] https://issues.apache.org/jira/browse/FLINK-11082 ------------------------------------------------------------------ From:裴立平 <peiliping...@gmail.com> Send Time:2019 Jan. 3 (Thu.) 13:39 To:dev <dev@flink.apache.org> Subject:[DISCUSS] Detection Flink Backpressure Recently I want to optimize the way to find the positions where the backpressures occured . I read some blogs about flink-backpressure and have a rough idea of it . The method which Flink adopted is thread-stack-sample , it's heavy and no-lasting . The positions where backpressures occured are very important to the developers . They should be treated as monitor-metrics . Any other choice that we can take to detection the flink backpressures ?