DataStream API ------------------------------------------------------------------ Sender:Guowei Ma <guowei....@gmail.com> Sent At:2022 Jan. 26 (Wed.) 21:51 Recipient:Shawn Du <shawn...@neulion.com.cn> Cc:user <user@flink.apache.org> Subject:Re: create savepoint on bounded source in streaming mode
Hi, Shawn Thank you for your sharing. Unfortunately I do not think there is an easy way to achieve this now. Actually we have a customer who has the same requirement but the scenario is a little different. The bounded and unbounded pipeline have some differences but the customer wants reuse some state of the bounded pipeline. Another question is what the api does the pipelined use? DataStream or SQL Best, Guowei On Wed, Jan 26, 2022 at 8:58 PM Shawn Du <shawn...@neulion.com.cn> wrote: right! ------------------------------------------------------------------ Sender:Guowei Ma <guowei....@gmail.com> Sent At:2022 Jan. 26 (Wed.) 19:50 Recipient:Shawn Du <shawn...@neulion.com.cn> Cc:user <user@flink.apache.org> Subject:Re: create savepoint on bounded source in streaming mode Hi,Shawn You want to use the correct state(n-1) for day n-1 and the full amount of data for day n to produce the correct state(n) for day n. Then use state(n) to initialize a job to process the data for day n+1. Am I understanding this correctly? Best, Guowei Shawn Du <shawn...@neulion.com.cn>于2022年1月26日 周三下午7:15写道: Hi Gaowei, think the case: we have one streaming application built by flink, but kinds of reason, the event may be disordered or delayed terribly. we want to replay the data day by day(the data was processed like reordered.). it looks like a batching job but with state. we want to use the same code for replaying. thus we need persist the state for next job. any ideas? Thanks Shawn ------------------------------------------------------------------ Sender:Guowei Ma <guowei....@gmail.com> Sent At:2022 Jan. 26 (Wed.) 15:39 Recipient:Shawn Du <shawn...@neulion.com.cn> Cc:user <user@flink.apache.org> Subject:Re: create savepoint on bounded source in streaming mode Hi Shawn Currently Flink can not trigger the sp at the end of the input. An alternative way might be that you need to develop a customized source, which triggers a savepoint when it notices that all the input split has been handled. Or you could see the state process api[1], which might be helpful. Thanks for your sharing but I have another little question: I think you need to process all the historical events to rebuild the correct state. So there might be no gain even if you periodically create a savepoint. So why did you need to "rebuild" the state periodically? Do I miss something? [1] https://nightlies.apache.org/flink/flink-docs-release-1.14/docs/libs/state_processor_api/ Best, Guowei On Wed, Jan 26, 2022 at 2:17 PM Shawn Du <shawn...@neulion.com.cn> wrote: our application is stateful. processing live events depends on the state. but for kinds of reason, we need rebuild the state. it will be very costly to replay all data. our historical events data are stored in s3. so we want to create states/savepoints periodically so that we can rebuild the state from a point. we call this as a bootstrap process. any ideas? Thanks. ------------------------------------------------------------------ Sender:Guowei Ma <guowei....@gmail.com> Sent At:2022 Jan. 26 (Wed.) 14:04 Recipient:Shawn Du <shawn...@neulion.com.cn> Cc:user <user@flink.apache.org> Subject:Re: create savepoint on bounded source in streaming mode Hi, Shawn I think Flink does not support this mechanism yet. Would you like to share the scenario in which you need this savepoint at the end of the bounded input? Best, Guowei On Wed, Jan 26, 2022 at 1:50 PM Shawn Du <shawn...@neulion.com.cn> wrote: Hi experts, assume I have several files and I want replay these files in order in streaming mode and create a savepoint when files play at the end. it is possible? I wrote a simple test app, and job are finished when source is at the end. I have no chance to creat a savepoint. please help. Thanks Shawn -- Best, Guowei