Hi Xuannan,

Thanks for starting the discussion. This would certainly help a lot on both
efficiency and reproducibility in machine learning cases :)

I have a few questions as follows:

1. Can we support caching both the output and sideoutputs of a
SingleOutputStreamOperator (which I believe is a reasonable use case),
given that  `cache()` is defined on `SingleOutputStreamOperator`?

If not, shall we introduce another class, say
"CachedSingleOutputStreamOperator", which extends
SingleOutputStreamOperator and overrides the getSideOutput method and
return CachedDataStream?

2. Is there any chance that we also support cache in Stream Mode if the one
SingleOutputStreamOperator is bounded? We may also want to run batch jobs
in a Stream Mode. Could you add some discussions in the FLIP?

3. What are we going to do if users change the parallelism of
CachedDataStream? Shall we throw an exception or add a new operator when
translating the job graph?

Two typos:
1.  ...a stream node with the sample parallelism as its input is added to
the StreamGraph....
---> "the same parallelism"
2. In figure of Job1, one-input transformation
---> MapTransformation

Best,
Zhipeng


David Morávek <d...@apache.org> 于2021年12月29日周三 18:08写道:

> Hi Xuannan,
>
> thanks for drafting this FLIP.
>
> One immediate thought, from what I've seen for interactive data exploration
> with Spark, most people tend to use the higher level APIs, that allow for
> faster prototyping (Table API in Flink's case). Should the Table API also
> be covered by this FLIP?
>
> Best,
> D.
>
> On Wed, Dec 29, 2021 at 10:36 AM Xuannan Su <suxuanna...@gmail.com> wrote:
>
> > Hi devs,
> >
> > I’d like to start a discussion about adding support to cache the
> > intermediate result at DataStream API for batch processing.
> >
> > As the DataStream API now supports batch execution mode, we see users
> > using the DataStream API to run batch jobs. Interactive programming is
> > an important use case of Flink batch processing. And the ability to
> > cache intermediate results of a DataStream is crucial to the
> > interactive programming experience.
> >
> > Therefore, we propose to support caching a DataStream in Batch
> > execution. We believe that users can benefit a lot from the change and
> > encourage them to use DataStream API for their interactive batch
> > processing work.
> >
> > Please check out the FLIP-205 [1] and feel free to reply to this email
> > thread. Looking forward to your feedback!
> >
> > [1]
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-205%3A+Support+Cache+in+DataStream+for+Batch+Processing
> >
> > Best,
> > Xuannan
> >
>


-- 
best,
Zhipeng

Reply via email to