Re: [DISCUSS] Allow at-most-once delivery in case of failures

2019-07-24 Thread Zhu Zhu
Hi All, I think using plugins, as Stephan suggested, would be the best way to serve different requirements for difference scenarios, even after they are merged into Flink core. As far as I know, the pluggable shuffle service is ready for use. Failover strategy does not support plugin yet, but

Re: [DISCUSS] Allow at-most-once delivery in case of failures

2019-07-23 Thread Biao Liu
Hi Stephan & Xiaogang, It's great to see this discussion active again! It makes sense to me that doing some private optimization and trial through plugin. I understand that the community could not satisfy every one and every requirement due to limited resources. The pluggable strategy is a good

Re: [DISCUSS] Allow at-most-once delivery in case of failures

2019-07-23 Thread SHI Xiaogang
Hi Stephan, I agree with you that the implementation of "at-most-once" or "best-effort" recovery will benefit from pluggable shuffle service and pluggable scheduler. Actually we made some attempts in our private repository and it turns out that it requires quite a lot of work to implement this

Re: [DISCUSS] Allow at-most-once delivery in case of failures

2019-07-23 Thread Stephan Ewen
Hi all! This is an interesting discussion for sure. Concerning user requests for changes modes, I also hear the following quite often: - reduce the expensiveness of checkpoint alignment (unaligned checkpoints) to make checkpoints fast/stable under high backpressure - more fine-grained

Re: [DISCUSS] Allow at-most-once delivery in case of failures

2019-06-11 Thread SHI Xiaogang
Hi All, It definitely requires a massive effort to allow at-most-once delivery in Flink. But as the feature is urgently demanded by many Flink users, i think every effort we made is worthy. Actually, the inability to support at-most-once delivery has become a major obstacle for Storm users to

Re: [DISCUSS] Allow at-most-once delivery in case of failures

2019-06-11 Thread zhijiang
Thanks for launching this topic xiaogang! I also heard of this requirement from users before and I agree it could bring benefits for some scenarios. As we know, the fault tolerance is one of the biggest challenges in stream architecuture, because it is difficult to change if the initial system

Re: [DISCUSS] Allow at-most-once delivery in case of failures

2019-06-11 Thread vino yang
+1 from my side to support this feature in Flink. Best, Vino Biao Liu 于2019年6月11日周二 下午6:14写道: > Hi Piotrek, > I agree with you that there are strained resources of community to support > such a feature. I was planing to start a similar discussion after 1.9 > released. Anyway we don't have

Re: [DISCUSS] Allow at-most-once delivery in case of failures

2019-06-11 Thread Biao Liu
Hi Piotrek, I agree with you that there are strained resources of community to support such a feature. I was planing to start a similar discussion after 1.9 released. Anyway we don't have enough time to support this feature now, but I think a discussion is fine. It's very interesting of your

Re: [DISCUSS] Allow at-most-once delivery in case of failures

2019-06-11 Thread Zhu Zhu
Thanks Xiaogang for initiating the discussion. I think it is a very good proposal. We also received this requirements for Flink from Alibaba internal and external customers. In these cases, users are less concerned of the data consistency, but have higher demands for low latency. Here are a

Re: [DISCUSS] Allow at-most-once delivery in case of failures

2019-06-11 Thread Piotr Nowojski
Hi Xiaogang, It sounds interesting and definitely a useful feature, however the questions for me would be how useful, how much effort would it require and is it worth it? We simply can not do all things at once, and currently people that could review/drive/mentor this effort are pretty much

Re: [DISCUSS] Allow at-most-once delivery in case of failures

2019-06-11 Thread Zili Chen
Hi Xiaogang, It is an interesting topic. Notice that there is some effort to build a mature mllib of flink these days, it could be also possible for some ml cases trade off correctness for timeliness or throughput. Excatly-once delivery excatly makes flink stand out but an at-most-once option