Thanks for the update Seth, and let me further clarify my comments /
concerns around the new `CheckpointStorage`.

1. In the existing `MemoryStateBackend`, there's a `maxStateSize` field
which limits the maximal state size sent to JM from one single memory
backend, with the default size of 5MB. Please make sure to extract this
limitation out and keep it when implementing the new
`JobManagerCheckpointStorage` (as well as writing this down in our FLIP
document).

2. We need to confirm the semantic for `JobManagerCheckpointStorage`.
    - Currently in `MemoryBackendCheckpointStorage` we will
      a) send the checkpoint data to JM and persist it to the remote FS
(included in metadata) if checkpoint path is given, or
      b) send the checkpoint data to JM w/o persistency if no checkpoint
path given
    Does `JobManagerCheckpointStorage` mean checkpoint data will be sent to
JM first and JM handles everything afterwards? Literally it seems to be "JM
is the checkpoint storage and no external system required", which matches
only to case #b. We need to confirm this and make it clear in our FLIP
document (and explain to our users later).

3. Since now we expose the checkpoint storage concept to our users through
the `setCheckpointStorage` API, I suggest to add below notes in our
documents:
    a) `JobManagerCheckpointStorage` should be used iff the state size is
small enough, and users should take special care not to burst the JM memory
when using it. And JM will also persist the data to remote FS after the
checkpoint is globally completed (depending on our decision of the
semantic).
    b) When setting the checkpoint storage to
`FileSystemCheckpointStorage`, there's still chance that the checkpoint
data is sent to JM memory first, decided by the
`state.backend.fs.memory-threshold` configuration (to be honest, I find
this part ambiguous between JM and FS checkpoint storage)

Please let me know your thoughts. Thanks.

Best Regards,
Yu


On Fri, 18 Sep 2020 at 23:02, Seth Wiesman <sjwies...@gmail.com> wrote:

> 1. With `FSStateBackend`, we used to decide where to store the checkpoint
> by the `state.backend.fs.memory-threshold` configuration, and we need to
> decide how to align with this behavior with the new implementation.
>
> I see this configuration available on the FileSystemStorage class. I've
> added that to the doc.
>
> 2. With the new implementation, since users could set checkpoint storage
> through API, do we also support the combination of
> `EmbeddedRocksDBStateBackend` with `JobManagerCheckpointStorage`?
>
> This is actually doable today and I don't see any reason to remove this
> functionality: new RocksDBStateBackend(new MemoryStateBackend())
>
> 1. There are still some `SnapshotStorage` / `JobManagerSnapshot` left in
> the code samples, please clean them up
>
> Apologies, fixed
>
> 2. Personally I'm in favor of `JobManagerCheckpointStorage` /
> `FileSystemCheckpointStorage` than `JobManagerStorage` / `FileSystemStorage
>
> That's fine by me
>
> Seth
>
> On Fri, Sep 18, 2020 at 9:36 AM Yu Li <car...@gmail.com> wrote:
>
> > *bq. I agree with your assessment of the CheckpointStorage interface but
> I
> > want to push back at including those changes as a part of this FLIP.*
> > Makes sense, will start a separate discussion around this topic when
> > prepared.
> >
> > *bq. One option could be to rename "CheckpointStorage" to
> > "CheckpointStorageAccess" and then use the name "CheckpointStorage"
> instead
> > of "SnapshotStorage". *
> > +1
> >
> > And thanks for updating the document, some comments for the new version:
> >
> > Questions around migration:
> > 1. With `FSStateBackend`, we used to decide where to store the checkpoint
> > by the `state.backend.fs.memory-threshold` configuration, and we need to
> > decide how to align with this behavior with the new implementation.
> > 2. With the new implementation, since users could set checkpoint storage
> > through API, do we also support the combination of
> > `EmbeddedRocksDBStateBackend` with `JobManagerCheckpointStorage`?
> >
> > One answer to the above questions is making
> > `JobManagerCheckpointStorage` internal implementation and use it as the
> > default checkpoint storage. And when user sets to
> > use `FileSystemCheckpointStorage`, we will still switch to
> > `JobManagerCheckpointStorage` when the task checkpoint size is smaller
> than
> > `state.backend.fs.memory-threshold`, even with RocksDB state backend.
> This
> > will align with most of the current behavior except for RocksDB backend
> > with really small checkpoint size.
> >
> > Minor issues:
> > 1. There are still some `SnapshotStorage` / `JobManagerSnapshot` left in
> > the code samples, please clean them up
> > 2. Personally I'm in favor of `JobManagerCheckpointStorage` /
> > `FileSystemCheckpointStorage` than `JobManagerStorage` /
> > `FileSystemStorage`
> >
> > Thanks.
> >
> > Best Regards,
> > Yu
> >
> >
> > On Fri, 18 Sep 2020 at 01:58, Seth Wiesman <sjwies...@gmail.com> wrote:
> >
> > > That makes sense to me, I've updated the FLIP and also took this chance
> > to
> > > make it clearer what the goals and non-goals of this proposal are.
> > >
> > > Seth
> > >
> > > On Thu, Sep 17, 2020 at 9:17 AM Stephan Ewen <se...@apache.org> wrote:
> > >
> > > > Just a quick note that it should be possible to rename
> > > "CheckpointStorage"
> > > > because it is a purely internal interface.
> > > >
> > > > Looks like the "SnapshotStorage" takes some limited amount of
> > > functionality
> > > > from the "CheckpointStorage", like location pointer resolution.
> > > > One option could be to rename "CheckpointStorage" to
> > > > "CheckpointStorageAccess" and then use the name "CheckpointStorage"
> > > instead
> > > > of "SnapshotStorage".
> > > >
> > > >
> > > >
> > > > On Thu, Sep 17, 2020 at 3:47 PM Seth Wiesman <sjwies...@gmail.com>
> > > wrote:
> > > >
> > > > > Hi Yu,
> > > > >
> > > > > I've updated the Deprecation / Compatibility / Migration section to
> > > more
> > > > > explicitly lay out the steps that we would take as part of this
> FLIP.
> > > It
> > > > > includes your above concerns.
> > > > >
> > > > > Regarding SnapshotStorage vs CheckpointStorage. I'm not sure users
> > are
> > > > > going to have a problem with this. I doubt many people outside this
> > > > thread
> > > > > are familiar with the CheckpointStorage interface today. Even with
> > > these
> > > > > changes implemented, most users will not interact with the
> > > > SnapshotStorage
> > > > > interface. They will only ever see JobManagerStorage and
> > > > FileSystemStorage.
> > > > >
> > > > > I agree with your assessment of the CheckpointStorage interface
> but I
> > > > want
> > > > > to push back at including those changes as a part of this FLIP. The
> > > goal
> > > > is
> > > > > to simplify users' understanding of state backends and
> > checkpointing. I
> > > > > would like to keep anything related to the runtime or internal as a
> > > > > non-goal.
> > > > >
> > > > > Seth
> > > > >
> > > > > On Thu, Sep 17, 2020 at 3:03 AM Yu Li <car...@gmail.com> wrote:
> > > > >
> > > > > > Thanks for the suggestion and discussion, and sorry for being
> late
> > to
> > > > the
> > > > > > party.
> > > > > >
> > > > > > For me, +1 for the idea, but +0 for the current FLIP document.
> > > > > >
> > > > > > First of all, I suggest we explicitly mention the deprecation of
> > > > existing
> > > > > > backends in the document. From the description, we plan to mark
> all
> > > > > > existing backend implementations (i.e.
> > > > > > RocksDBStateBackend/MemoryStateBackend/FSStateBackend) as
> > deprecated,
> > > > and
> > > > > > in their javadoc we should give the suggestion of migration to
> new
> > > > > > implementations (i.e.
> > > HashMapStateBackend/EmbeddedRocksDBStateBackend).
> > > > > >
> > > > > > Secondly, I suggest we explicitly mention the user-facing changes
> > for
> > > > > > customized state backends.
> > > > > >
> > > > > > To be more specific, the above two should be included in the
> > > > > > "Compatibility, Deprecation, and Migration Plan" section. The
> > > existing
> > > > > > document already mentioned these two aspects, but IMO not
> explicit
> > > > > enough.
> > > > > >
> > > > > > Thirdly, we already have a `CheckpointStorage` interface and now
> > > > > > introducing a new `SnapshotStoage`, and I share the same concern
> > with
> > > > > > Stephan that these two interfaces might cause confusion, and
> > suggest
> > > we
> > > > > > discuss more about this part.
> > > > > >
> > > > > > This might sound to be a little bit off-track, but I think it's
> > > > necessary
> > > > > > to review the necessity of the existence of current
> > > > `CheckpointStorage`.
> > > > > It
> > > > > > seems to me that only JM-side logic will use interfaces in
> > > > > > `CheckpointStorageCoordinatorView` and only TM-side logic use
> > > > > > `CheckpointStorageWorkerView`, but we combine these two together.
> > > > What's
> > > > > > more, if we check it carefully, we could find the signature of
> > > > > > `resolveCheckpoint` interface in current `StateBackend` and
> > > > > > `CheckpointStorageCoordinatorView` are exactly the same (even the
> > > > > javadoc),
> > > > > > which means if we simply extract `resolveCheckpoint` out into
> > > > > > `SnapshotStorage`, there will be two interfaces with the same
> > > signature
> > > > > in
> > > > > > `SnapshotStorage` and `CheckpointStorage`, which will be really
> > > > > confusing.
> > > > > > Sorry but I don't have a proposal of solution yet, but I suggest
> we
> > > > > figure
> > > > > > this out clearly.
> > > > > >
> > > > > > Thanks.
> > > > > >
> > > > > > Best Regards,
> > > > > > Yu
> > > > > >
> > > > > >
> > > > > > On Thu, 17 Sep 2020 at 13:19, Congxian Qiu <
> qcx978132...@gmail.com
> > >
> > > > > wrote:
> > > > > >
> > > > > > > Thanks for the detailed replay, +1 from my side.
> > > > > > > Best,
> > > > > > > Congxian
> > > > > > >
> > > > > > >
> > > > > > > Seth Wiesman <sjwies...@gmail.com> 于2020年9月17日周四 上午1:33写道:
> > > > > > >
> > > > > > > > Hi Stephan,
> > > > > > > >
> > > > > > > > Regarding backward compatibility, I agree and the intention
> is
> > > that
> > > > > all
> > > > > > > > existing code will continue to function with the same
> > semantics.
> > > My
> > > > > > > working
> > > > > > > > idea is to remove the two checkpoint-storage related methods
> > from
> > > > > > > > StateBackend into a new SnapshotStorage interface but then
> have
> > > > > > > > AbstractFileStateBackend and RocksDBStateBackend implement
> > > snapshot
> > > > > > > > storage. If a state backend implements SnapshotStorage it
> will
> > be
> > > > > used
> > > > > > > > unconditionally, even if a different snapshot storage
> > > > implementation
> > > > > is
> > > > > > > > configured. This way we don't change any of the concrete
> > classes
> > > > that
> > > > > > > users
> > > > > > > > interact with. The only people who would see breaking changes
> > are
> > > > > state
> > > > > > > > backend implementors and they only need to add `implements
> > > > > > > SnapshotStorage`
> > > > > > > > to their class.
> > > > > > > >
> > > > > > > > The reason I went with SnapshotStorage is there is already an
> > > > > interface
> > > > > > > > `org.apache.flink.runtime.state.CheckpointStorage` in
> > > > flink-runtime.
> > > > > If
> > > > > > > we
> > > > > > > > can rename this interface to something else I'm happy to take
> > the
> > > > > name,
> > > > > > > but
> > > > > > > > if not I think it will lead to import confusion.
> > > > > > > >
> > > > > > > > Seth
> > > > > > > >
> > > > > > > > On Wed, Sep 16, 2020 at 11:54 AM Stephan Ewen <
> > se...@apache.org>
> > > > > > wrote:
> > > > > > > >
> > > > > > > > > @Yun and @Congxian:
> > > > > > > > >
> > > > > > > > > I think "async", "incremental", and similar flags belong
> very
> > > > much
> > > > > > with
> > > > > > > > the
> > > > > > > > > state backend (the index structure).
> > > > > > > > > They define how the snapshotting procedure behaves.
> > > > > > > > >
> > > > > > > > > The SnapshotStorage is really just about storage of
> > checkpoint
> > > > > > streams
> > > > > > > > > (bytes) and handles and pointers.
> > > > > > > > >
> > > > > > > > > Best,
> > > > > > > > > Stephan
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > On Wed, Sep 16, 2020 at 6:48 PM Stephan Ewen <
> > se...@apache.org
> > > >
> > > > > > wrote:
> > > > > > > > >
> > > > > > > > > > Thanks for the great suggestion and the great discussion.
> > > > > Generally
> > > > > > > big
> > > > > > > > > +1
> > > > > > > > > > to this effort.
> > > > > > > > > > Some thoughts from my side:
> > > > > > > > > >
> > > > > > > > > > *## Backwards Compatibility*
> > > > > > > > > >
> > > > > > > > > > I think we should really strive to make this non
> breaking.
> > > > Maybe
> > > > > we
> > > > > > > > have
> > > > > > > > > > new classes / interfaces for StateBackends and
> > > > CheckpointStorage
> > > > > > and
> > > > > > > > let
> > > > > > > > > > the existing State Backend classes implement both (and
> > > > deprecate
> > > > > > > them)?
> > > > > > > > > >
> > > > > > > > > > In the past, I have gotten some harsh comments from users
> > > about
> > > > > > > > breaking
> > > > > > > > > > long-time effectively stable APIs, so let's try hard to
> > avoid
> > > > > this
> > > > > > > > > (unless
> > > > > > > > > > it makes things impossible).
> > > > > > > > > >
> > > > > > > > > > *## Naming*
> > > > > > > > > >
> > > > > > > > > > HashMapStateBackend sounds good to me
> > > > > > > > > >
> > > > > > > > > > Could we rename the SnapshotStorage to CheckpointStorage?
> > Or
> > > > > > converge
> > > > > > > > all
> > > > > > > > > > methods around "Snapshot"?
> > > > > > > > > > I think we already have some confusion from mixing the
> > terms
> > > > > > > checkpoint
> > > > > > > > > > and snapshot and should converge in either direction.
> > > > > > > > > > I am slightly leaning towards converging around
> > checkpoints,
> > > > > > because
> > > > > > > > > > that's the most commonly known term among users as far
> as I
> > > can
> > > > > > tell.
> > > > > > > > > > Checkpoints are Snapshots. But one could also just call
> > them
> > > > > > > > Checkpoints
> > > > > > > > > > and let Savepoints be special Checkpoints.
> > > > > > > > > >
> > > > > > > > > > *## Integrated State / Storage Backends*
> > > > > > > > > >
> > > > > > > > > > There is an idea floating around now and then about a
> > > Cassandra
> > > > > > > backend
> > > > > > > > > > (or other K/V store) where the state index and durable
> > > location
> > > > > are
> > > > > > > > > tightly
> > > > > > > > > > intertwined.
> > > > > > > > > > However, I think this would not contradict, because it
> > might
> > > > just
> > > > > > > mean
> > > > > > > > > > that the checkpoint storage is used less (maybe only for
> > > > > > savepoints,
> > > > > > > or
> > > > > > > > > for
> > > > > > > > > > WALs).
> > > > > > > > > >
> > > > > > > > > > *## Future Fault Tolerance Ideas*
> > > > > > > > > >
> > > > > > > > > > I think this conflicts with none of the future fault
> > > tolerance
> > > > > > ideas
> > > > > > > I
> > > > > > > > am
> > > > > > > > > > involved with.
> > > > > > > > > > Similar to the above, there is always some checkpoint
> > storage
> > > > > > > involved,
> > > > > > > > > > for example for savepoints or for backup/consolidation,
> so
> > no
> > > > > > > problem.
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > Best,
> > > > > > > > > > Stephan
> > > > > > > > > >
> > > > > > > > > > On Wed, Sep 16, 2020 at 5:11 PM Aljoscha Krettek <
> > > > > > > aljos...@apache.org>
> > > > > > > > > > wrote:
> > > > > > > > > >
> > > > > > > > > >> I think the mentioned settings should be in the state
> > > backend.
> > > > > > They
> > > > > > > > > >> configure how a certain backend writes to a snapshot
> > > storage,
> > > > > but
> > > > > > > it’s
> > > > > > > > > >> still the backend that has the logic and decides.
> > > > > > > > > >>
> > > > > > > > > >> I think it's a good point, though, to be conscious about
> > > those
> > > > > > > > settings.
> > > > > > > > > >> I'm sure we can figure out the details during
> > > implementation,
> > > > > > > though.
> > > > > > > > > >>
> > > > > > > > > >> Best,
> > > > > > > > > >> Aljoscha
> > > > > > > > > >>
> > > > > > > > > >> On 16.09.20 16:54, Seth Wiesman wrote:
> > > > > > > > > >> > Hi Congxian,
> > > > > > > > > >> >
> > > > > > > > > >> > There is an allusion to those configs in the wiki but
> > let
> > > me
> > > > > > > better
> > > > > > > > > >> spell
> > > > > > > > > >> > out my thinking. The flink-conf configurations will
> not
> > > > change
> > > > > > > and I
> > > > > > > > > >> > believe the java code switches should remain on the
> > state
> > > > > > backend
> > > > > > > > > >> objects.
> > > > > > > > > >> >
> > > > > > > > > >> > We are of course not fully disentangling state
> backends
> > > from
> > > > > > > > snapshots
> > > > > > > > > >> and
> > > > > > > > > >> > these configurations affect how your state backend
> runs
> > in
> > > > > > > > > production. I
> > > > > > > > > >> > believe users would find it strange to have
> > configurations
> > > > > like
> > > > > > > > > >> > `state.backend.rocksdb.checkpoint.transfer.thred.num`
> > not
> > > be
> > > > > > part
> > > > > > > of
> > > > > > > > > the
> > > > > > > > > >> > EmbeddedRocksdbStateBackend but somewhere else. This
> > then
> > > > > leads
> > > > > > to
> > > > > > > > the
> > > > > > > > > >> > question, is it better to split configurations between
> > > > > multiple
> > > > > > > > places
> > > > > > > > > >> or
> > > > > > > > > >> > not. Users appreciate consistency, and so having all
> the
> > > > > > > > > configurations
> > > > > > > > > >> on
> > > > > > > > > >> > the state backend objects makes them more discoverable
> > and
> > > > > your
> > > > > > > > > >> application
> > > > > > > > > >> > easier to reason about.
> > > > > > > > > >> >
> > > > > > > > > >> > Additionally, I view these as advanced configurations.
> > My
> > > > hope
> > > > > > is
> > > > > > > > most
> > > > > > > > > >> > users can simply use the no-arg constructor for a
> state
> > > > > backend
> > > > > > in
> > > > > > > > > >> > production. If a user is changing the number of
> rocksdb
> > > > > transfer
> > > > > > > > > >> threads or
> > > > > > > > > >> > disabling async checkpoints, they likely know what
> they
> > > are
> > > > > > doing.
> > > > > > > > > >> >
> > > > > > > > > >> > Please let me know if you have any concerns or would
> > like
> > > to
> > > > > > > cancel
> > > > > > > > > the
> > > > > > > > > >> > vote.
> > > > > > > > > >> >
> > > > > > > > > >> > Seth
> > > > > > > > > >> >
> > > > > > > > > >> > On Wed, Sep 16, 2020 at 12:37 AM Congxian Qiu <
> > > > > > > > qcx978132...@gmail.com
> > > > > > > > > >
> > > > > > > > > >> > wrote:
> > > > > > > > > >> >
> > > > > > > > > >> >> 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
> > > > > > > > > >> >>>>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>
> > > > > > > > > >> >>>>>>
> > > > > > > > > >> >>>>
> > > > > > > > > >> >>>>
> > > > > > > > > >> >>>
> > > > > > > > > >> >>
> > > > > > > > > >> >
> > > > > > > > > >>
> > > > > > > > > >>
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Reply via email to