Thanks for the FLIP, Arvid. This is a good improvement for checkpoint under backpressure. Currently, if a job under backpressure, it almost can't complete the checkpoint. so +1 from my side.
Best, Congxian zhijiang <wangzhijiang...@aliyun.com.invalid> 于2019年10月10日周四 上午11:02写道: > Thanks for writing up this FLIP, Arvid! > > Many users would expect this feature and also +1 from my side. > > Best, > Zhijiang > ------------------------------------------------------------------ > From:Piotr Nowojski <pi...@ververica.com> > Send Time:2019年10月7日(星期一) 10:13 > To:dev <dev@flink.apache.org> > Subject:Re: [DISCUSS] FLIP-76: Unaligned checkpoints > > Hi Arvid, > > Thanks for coming up with this FLIP. I think it addresses the issues > raised in the previous mailing list discussion [2]. > > For the record: +1 from my side to implement this. > > Piotrek > > > On 30 Sep 2019, at 14:31, Arvid Heise <ar...@ververica.com> wrote: > > > > Hi Devs, > > > > I would like to start the formal discussion about FLIP-76 [1], which > > improves the checkpoint latency in systems under backpressure, where a > > checkpoint can take hours to complete in the worst case. I recommend the > > thread "checkpointing under backpressure" [2] to get a good idea why > users > > are not satisfied with the current behavior. The key points: > > > > - Since the checkpoint barrier flows much slower through the > > back-pressured channels, the other channels and their upstream > operators > > are effectively blocked during checkpointing. > > - The checkpoint barrier takes a long time to reach the sinks causing > > long checkpointing times. A longer checkpointing time in turn means > that > > the checkpoint will be fairly outdated once done. Since a heavily > utilized > > pipeline is inherently more fragile, we may run into a vicious cycle of > > late checkpoints, crash, recovery to a rather outdated checkpoint, more > > back pressure, and even later checkpoints, which would result in > little to > > no progress in the application. > > > > The FLIP proposes "unaligned checkpoints" which improves the current > state, > > such that > > > > - Upstream processes can continue to produce data, even if some > operator > > still waits on a checkpoint barrier on a specific input channel. > > - Checkpointing times are heavily reduced across the execution graph, > > even for operators with a single input channel. > > - End-users will see more progress even in unstable environments as > more > > up-to-date checkpoints will avoid too many recomputations. > > - Facilitate faster rescaling. > > > > The key idea is to allow checkpoint barriers to be forwarded to > downstream > > tasks before the synchronous part of the checkpointing has been conducted > > (see Fig. 1). To that end, we need to store in-flight data as part of the > > checkpoint as described in greater details in this FLIP. > > > > Although the basic idea was already sketched in [2], we would like get > > broader feedback in this dedicated mail thread. > > > > Best, > > > > Arvid > > > > [1] > > > https://cwiki.apache.org/confluence/display/FLINK/FLIP-76%3A+Unaligned+Checkpoints > > [2] > > > http://apache-flink-mailing-list-archive.1008284.n3.nabble.com/Checkpointing-under-backpressure-td31616.html > >