[ 
https://issues.apache.org/jira/browse/FLINK-18662?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17164495#comment-17164495
 ] 

Piotr Nowojski edited comment on FLINK-18662 at 7/24/20, 3:34 PM:
------------------------------------------------------------------

I'm not sure which option is better:

# Add new metric {{checkpointUnalignmentTime}}, which would be mutually 
exclusive with {{checkpointAlignmentTime}} - if one is non zero, the other must 
be zero and vice versa. This might be a bit annoying to for someone setting up 
some metric reporting system - he would have to setup and monitor both things.
# Overload the existing {{checkpointAlignmentTime}} metric, and adapt it to 
repot unalignment time for unaligned checkpoints. Note that in the webUI user 
would need to understand with what type of checkpoint he is dealing with and 
what's the difference. Also it might be a bit confusing if web UI presents an 
aligned savepoint and unaligned checkpoint at the same time. On the other hand 
the delay between the first and last checkpoint barrier for unaligned 
checkpoints could arguably be called "alignment time".

Regarding the size of unaligned data, we need to add a new metric, as 
{{lastCheckpointUnalignmentSize}}/"Alignment Buffered" was removed in 
FLINK-16404.

[~zjwang], [~AHeise], [~roman_khachatryan] what do you think?


was (Author: pnowojski):
I'm not sure which option is better:

# Add new metric {{checkpointUnalignmentTime}}, which would be mutually 
exclusive with {{checkpointAlignmentTime}} - if one is non zero, the other must 
be zero and vice versa. This might be a bit annoying to for someone setting up 
some metric reporting system - he would have to setup both things.
# Overload the existing {{checkpointAlignmentTime}} metric, and adapt it to 
repot unalignment time for unaligned checkpoints. Note that in the webUI user 
would need to understand with what type of checkpoint he is dealing with and 
what's the difference. Also it might be a bit confusing if web UI presents an 
aligned savepoint and unaligned checkpoint at the same time. On the other hand 
the delay between the first and last checkpoint barrier for unaligned 
checkpoints could arguably be called "alignment time".

Regarding the size of unaligned data, we need to add a new metric, as 
{{lastCheckpointUnalignmentSize}}/"Alignment Buffered" was removed in 
FLINK-16404.

[~zjwang], [~AHeise], [~roman_khachatryan] what do you think?

> Provide more detailed metrics why unaligned checkpoint is taking long time
> --------------------------------------------------------------------------
>
>                 Key: FLINK-18662
>                 URL: https://issues.apache.org/jira/browse/FLINK-18662
>             Project: Flink
>          Issue Type: Improvement
>          Components: Runtime / Metrics, Runtime / Network
>    Affects Versions: 1.11.1
>            Reporter: Piotr Nowojski
>            Priority: Critical
>             Fix For: 1.12.0
>
>         Attachments: Screenshot 2020-07-21 at 11.50.02.png
>
>
> With unaligned checkpoint there can happen situation as in the attached 
> screenshot.
>  Task reports long end to end checkpoint time (~2h50min), ~0s sync time, 
> ~2h50min async time, ~0s start delay. It means that task received first 
> checkpoint barrier from one of the channels very quickly (~0s), sync part was 
> quick, but we do not know why async part was taking so long. It could be 
> because of three things:
> # long operator state IO writes
> # long spilling of in-flight data
> # long time to receive the final checkpoint barrier from the last lagging 
> channel
> First and second are probably indistinguishable and the difference between 
> them doesn't matter much for analyzing. However the last one is quite 
> different. It might be independent of the IO, and we are missing this 
> information. 
> Maybe we could report it as "alignment duration" and while we are at it, we 
> could also report amount of spilled in-flight data for unaligned checkpoints 
> as "alignment buffered"? 
> Ideally we should report it as new metrics, but that leaves a question how to 
> display it in the UI, with limited space available. Maybe it could be 
> reported as:
> ||Alignment Buffered||Alignment Duration||
> |0 B (632 MB)|0ms (2h 49m 32s)|
> Where the values in the parenthesis would come from unaligned checkpoints. 



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to