Sorry for jump late in. I like the separation here, this separation makes more user friendly now.
I just wonder how the configuration such as 'state.backend.incremental', 'state.backend.async' and `state.backend.rocksdb.checkpoint.transfer.thred.num` will be configured after the separation, I think these configurations are more related to snapshots (maybe a little strange to configure these on statebackend side). did not see this on the FLIP wiki currently. Best, Congxian Seth Wiesman <sjwies...@gmail.com> 于2020年9月15日周二 下午9:51写道: > Sounds good to me. I'll update the FLIP. > > On Tue, Sep 15, 2020 at 8:35 AM Dawid Wysakowicz <dwysakow...@apache.org> > wrote: > > > There is a good number of precedents that introduced backwards > > incompatible changes to that interface (which is PublicEvolving btw). We > > introduced a couple of additional arguments to the > > createKeyedStateBackend method and later on removed the methods with > > default implementation for backwards compatibility. I want to introduce > > a backward incompatible change in FLIP-140 (replace the > > AbstractKeyedStateBackend with an interface). From my perspective we > > should just do these changes. The impact should be minimal. > > > > Best, > > > > Dawid > > > > > > On 15/09/2020 15:20, Seth Wiesman wrote: > > > Hey Dawid, > > > > > > I didn't want to break compatibility but if there is precedent and > > everyone > > > is ok with it then I'm +1. > > > > > > Seth > > > > > > On Tue, Sep 15, 2020 at 2:22 AM Dawid Wysakowicz < > dwysakow...@apache.org > > > > > > wrote: > > > > > >> Sorry for joining so late. > > >> > > >> Generally speaking I like this idea very much! > > >> > > >> I have one idea about the StateBackend interface. Could we instead of > > >> adding a flag method boolean isLegacyStateBackend remove the > > >> checkpointstorage related methods from StateBackend right away? The > > >> old/legacy implementations could then implement both StateBackend and > > >> SnapshotStorage. In turn in the method env.setStateBackend we could > do: > > >> > > >> setStateBackend(StateBackend backend) { > > >> > > >> this.stateBackend = backend; > > >> > > >> if (backend instanceof SnapshotStorage) { > > >> > > >> this.setSnapshotStorage(backend); > > >> > > >> } > > >> > > >> } > > >> > > >> This has the benefit that we could already get rid off the methods > from > > >> StateBackend which would be problematic in the new implementations > (such > > >> as e.g. HashMapStateBackend - what would you return there? null?). I > > >> know this would break the interface, but StateBackend is actually > quite > > >> internal, we did it quite freely in the past, and I don't think there > > >> are many custom state implementation in the wild. And even if there > are > > >> some the workaround is as easy as simply adding implements > > SnapshotStorage. > > >> > > >> Best, > > >> > > >> Dawid > > >> > > >> On 11/09/2020 16:48, Aljoscha Krettek wrote: > > >>> I could try and come up with a longer name if you need it ... ;-) > > >>> > > >>> Aljoscha > > >>> > > >>> On 11.09.20 16:25, Seth Wiesman wrote: > > >>>> Having thought about it more, HashMapStateBackend has won me over. > > I'll > > >>>> update the FLIP. If there aren't any more comments I'll open it up > for > > >>>> voting on monday. > > >>>> > > >>>> Seth > > >>>> > > >>>> On Wed, Sep 9, 2020 at 9:09 AM Seth Wiesman <sjwies...@gmail.com> > > >> wrote: > > >>>>> @Yun yes, this is really about making CheckpointStorage an > orthogonal > > >>>>> concept. I think we can remain pragmatic and keep state-backend > > >>>>> specific > > >>>>> configurations (async, incremental, etc) in the state backend > > >>>>> themselves. I > > >>>>> view these as more advanced configurations and by the time someone > is > > >>>>> changing the defaults they likely understand what is going on. My > > >>>>> goal is > > >>>>> to help on-board users and so long as each state backend has a > no-arg > > >>>>> default constructor that works for many users I think we've > achieved > > >>>>> that > > >>>>> goal. > > >>>>> > > >>>>> Regarding the checkpoint coordinator, that makes sense but I will > > >>>>> consider > > >>>>> out of the scope of this FLIP. I want to focus on simplifying APIs. > > >>>>> > > >>>>> @Aljoscha Krettek <aljos...@apache.org> > > >>>>> > > >>>>> My feeling is that state backends and checkpointing are going to be > > >>>>> integral to Flink for many years, regardless or other enhancements > > >>>>> so this > > >>>>> change is still valuable. > > >>>>> > > >>>>> Since this is a FLIP about improving the user api I'm happy to > > bikeshed > > >>>>> the names a little more than normal. HashMap makes sense, my other > > >>>>> thought > > >>>>> was InMemory. > > >>>>> > > >>>>> Seth > > >>>>> > > >>>>> > > >>>>> > > >>>>> On Wed, Sep 9, 2020 at 8:04 AM Aljoscha Krettek < > aljos...@apache.org > > > > > >>>>> wrote: > > >>>>> > > >>>>>> I like it a lot! > > >>>>>> > > >>>>>> I think it makes sense to clean this up despite the planned new > > >>>>>> fault-tolerance mechanisms. In the future, users will decide which > > >>>>>> mechanism to use and I can imagine that a lot of them will keep > > using > > >>>>>> the current mechanism for quite a while to come. But I'm happy to > > >>>>>> yield > > >>>>>> to Stephan's opinion here, he knows more about the progress of > that > > >>>>>> work. > > >>>>>> > > >>>>>> The one nitpick I have is about naming: will users understand > > >>>>>> OnHeapStateBackend? I mean, do they know what on-heap/off-heap > > >>>>>> memory is > > >>>>>> and the tradeoffs? An alternative could be HashMapStateBackend, > > >>>>>> because > > >>>>>> that's essentially what it is. I wouldn't block anything on this, > > >>>>>> though. > > >>>>>> > > >>>>>> Aljoscha > > >>>>>> > > >>>>>> On 09.09.20 10:05, Konstantin Knauf wrote: > > >>>>>>> Thanks for the initiative. Big +1. Would be interested to hear if > > the > > >>>>>>> proposed interfaces still make sense in the face of the new > > >>>>>> fault-tolerance > > >>>>>>> work that is planned. Stephan/Piotr will know. > > >>>>>>> > > >>>>>>> On Tue, Sep 8, 2020 at 7:05 PM Seth Wiesman <sjwies...@gmail.com > > > > >>>>>> wrote: > > >>>>>>>> Hi Devs, > > >>>>>>>> > > >>>>>>>> I'd like to propose an update to how state backends and > checkpoint > > >>>>>> storage > > >>>>>>>> are configured to help users better understand Flink. > > >>>>>>>> > > >>>>>>>> Apache Flink's durability story is a mystery to many users. One > > >>>>>>>> of the > > >>>>>> most > > >>>>>>>> common recurring questions from users comes from not > > >>>>>>>> understanding the > > >>>>>>>> relationship between state, state backends, and snapshots. Some > > >>>>>>>> of this > > >>>>>>>> confusion can be abated with learning material but the question > > >>>>>>>> is so > > >>>>>>>> pervasive that we believe Flink’s user APIs should be better > > >>>>>> communicate > > >>>>>>>> what different components are responsible for. > > >>>>>>>> > > >>>>>>>> > > >>>>>>>> > > >>>>>>>> > > >> > > > https://cwiki.apache.org/confluence/display/FLINK/FLIP-142%3A+Disentangle+StateBackends+from+Checkpointing > > >>>>>>>> > > >>>>>>>> I look forward to a healthy discussion. > > >>>>>>>> > > >>>>>>>> > > >>>>>>>> Seth > > >>>>>>>> > > >>>>>>> > > >>>>>> > > >> > > > > >