You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@flink.apache.org by Seth Wiesman <sj...@gmail.com> on 2020/09/08 17:05:23 UTC

[DISCUSS] FLIP-142: Disentangle StateBackends from Checkpointing

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

Re: [DISCUSS] FLIP-142: Disentangle StateBackends from Checkpointing

Posted by Yu Li <ca...@gmail.com>.
Yes let's move on, already cast my vote in the voting thread.

Thanks all for the patience answering / addressing my belated questions!

Best Regards,
Yu


On Sun, 27 Sep 2020 at 20:00, Stephan Ewen <se...@apache.org> wrote:

> Good to see this FLIP moving.
>
> From what I understand, the remaining questions are mainly about how to
> express the roles of the CheckpointStorage and specifically the behavior of
> JMCheckpointStorage and FsCheckpointStorage in the docs.
> This sounds like details we should discuss over the concrete text proposals
> in the PR.
>
> On Sun, Sep 27, 2020 at 5:38 AM Yu Li <ca...@gmail.com> wrote:
>
> > Thanks Seth, the updated FLIP overall LGTM, and I've left some inline
> > comments in the doc.
> >
> > Best Regards,
> > Yu
> >
> >
> > On Fri, 25 Sep 2020 at 20:58, Seth Wiesman <sj...@gmail.com> wrote:
> >
> > > Done
> > >
> > > Seth
> > >
> > > On Fri, Sep 25, 2020 at 2:47 AM Yu Li <ca...@gmail.com> wrote:
> > >
> > > > *bq. I think it might help to highlight specific stumbling blocks
> users
> > > > have today and why I believe this change addresses those issues.*
> > > > Thanks for adding more details, I believe adding these blocks to the
> > FLIP
> > > > doc could make the motivation more vivid and convincing.
> > > >
> > > > *bq. To be concrete I think the JavaDoc for setCheckpointStorage
> would
> > be
> > > > something like...*
> > > > I could see this definition extracts the existing description from
> the
> > > > current `StateBackend` interface, it's a valid option, and let me
> quote
> > > it
> > > > again:
> > > > - CheckpointStorage defines how checkpoint snapshots are persisted
> for
> > > > fault tolerance. Various implementations store their checkpoints in
> > > > different fashions and have different requirements and availability
> > > > guarantees.
> > > > - JobManagerCheckpointStorage stores checkpoints in the memory of the
> > > > JobManager. It is lightweight and without additional dependencies but
> > is
> > > > not highly available.
> > > > - FileSystemCheckpointStorage stores checkpoints in a file system.
> For
> > > > systems like HDFS, NFS Drives, S3, and GCS, this storage policy
> > supports
> > > > large state size, in the magnitude of many terabytes while providing
> a
> > > > highly available foundation for stateful applications. This
> checkpoint
> > > > storage policy is recommended for most production deployments.
> > > >
> > > > Sticking to this definition, I think we should have the below
> migration
> > > > plans for existing backends:
> > > > - `MemoryStateBackend(null, String savepointPath)` to
> > > > `HashMapStateBackend() + JobManagerCheckpointStorage()`
> > > > - `MemoryStateBackend(<non-null-checkpoint-path>, String
> > savepointPath)`
> > > to
> > > > `HashMapStateBackend() + FileSystemCheckpointStorage()`
> > > > in addition of the existing:
> > > > - `MemoryStateBackend()` to `HashMapStateBackend() +
> > > > JobManagerCheckpointStorage()`
> > > > and could be summarized as:
> > > > - MemoryStateBackend with checkpoint path: `HashMapStateBackend() +
> > > > FileSystemCheckpointStorage()`
> > > > - MemoryStateBackend w/o checkpoint path: `HashMapStateBackend() +
> > > > JobManagerCheckpointStorage()`
> > > >
> > > > And I believe adding the above highlighted blocks to the FLIP doc
> (the
> > > "New
> > > > StateBackend User API" and "Migration Plan" sections, separately)
> could
> > > > make it more complete.
> > > >
> > > > PS. Please note that although the current javadoc of `StateBackend`
> > > states
> > > > "MemoryStateBackend is not highly available", it actually supports
> > > > persisting the checkpoint data to DFS when checkpoint path is given,
> so
> > > the
> > > > mapping between old and new APIs are not that straight-forward and
> need
> > > > some clear clarifications, from my point of view.
> > > >
> > > > Best Regards,
> > > > Yu
> > > >
> > > >
> > > > On Fri, 25 Sep 2020 at 08:33, Seth Wiesman <sj...@gmail.com>
> > wrote:
> > > >
> > > > > Hi Yu,
> > > > >
> > > > > bq* I thought the FLIP aims at resolving some *existing* confusion,
> > > i.e.
> > > > > the durability mystery to users.
> > > > >
> > > > > I think it might help to highlight specific stumbling blocks users
> > have
> > > > > today and why I believe this change addresses those issues. Some
> > > frequent
> > > > > things I've heard over the past several years include:
> > > > >
> > > > > 1) "We use RocksDB because we don't need fault tolerance."
> > > > > 2) "We don't use RocksDB because we don't want to manage an
> external
> > > > > database."
> > > > > 3) Believing RocksDB is reading and writing directly with S3 or
> HDFS
> > > (vs.
> > > > > local disk)
> > > > > 4) Believing FsStateBackend spills to disk or has anything to do
> with
> > > the
> > > > > local filesystem
> > > > > 5) Pointing RocksDB at network-attached storage, believing that the
> > > state
> > > > > backend needs to be fault-tolerant
> > > > >
> > > > > This question from the ml is very representative of where users are
> > > > > struggling[1]. Many of these questions were not from new users but
> > from
> > > > > organizations that were in production! Just yesterday I was on the
> > > phone
> > > > > with a company that didn't realize they were in production without
> > > > > checkpointing; honestly, you would be shocked how often this
> happens.
> > > The
> > > > > current state backend abstraction is to complex for many of our
> > users.
> > > > What
> > > > > all these questions have in common is misunderstanding the
> > relationship
> > > > > between how data is stored locally on TMs vs how checkpoints make
> > that
> > > > > state durable.
> > > > >
> > > > > The FLIP aims actively help users by allowing them to reason about
> > > state
> > > > > backends separately from checkpoint durability. In the future, a
> > state
> > > > > backend only defines where and how state is stored locally on the
> TM
> > > > while
> > > > > checkpoint storage defines where and how checkpoints are stored for
> > > > > recovery. To be concrete I think the JavaDoc for
> setCheckpointStorage
> > > > would
> > > > > be something like:
> > > > >
> > > > > ```java
> > > > > /**
> > > > >  * CheckpointStorage defines how checkpoint snapshots are persisted
> > for
> > > > > fault tolerance
> > > > > *. Various implementations  store their checkpoints in different
> > > fashions
> > > > > and have different requirements and
> > > > >  * availability guarantees.
> > > > >  *
> > > > >  *<p>For example, JobManagerCheckpointStorage stores checkpoints in
> > the
> > > > > memory of the JobManager.
> > > > >  * It is lightweight and without additional dependencies but is not
> > > > highly
> > > > > available
> > > > >  * and only supports small state sizes. This checkpoint storage
> > policy
> > > is
> > > > > convenient for
> > > > >  * local testing and development.
> > > > >  *
> > > > >  *<p>FileSystemCheckpointStorage stores checkpoints in a
> filesystem.
> > > For
> > > > > systems like
> > > > >  * HDFS, NFS Drives, S3, and GCS, this storage policy supports
> large
> > > > state
> > > > > size,
> > > > >  * in the magnitude of many terabytes while providing a highly
> > > available
> > > > > foundation
> > > > >  * for stateful applications. This checkpoint storage policy is
> > > > recommended
> > > > > for most
> > > > >  * production deployments.
> > > > >  */
> > > > > void setCheckpointStorage(CheckpointStorage storage) {}
> > > > > ```
> > > > >
> > > > > Seth
> > > > >
> > > > > [1]
> > > > >
> > > > >
> > > >
> > >
> >
> http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/State-Storage-Questions-td37919.html
> > > > > [2] Also naming, but we're aligned here
> > > > >
> > > > > On Thu, Sep 24, 2020 at 10:24 AM Yu Li <ca...@gmail.com> wrote:
> > > > >
> > > > > > And to make it clear, I'm +1 on the idea of decoupling state
> > backends
> > > > > with
> > > > > > checkpointing. I don't have any question about making it clear
> that
> > > > > > heap/RocksDB is where we serve the routine state read/write and
> > where
> > > > to
> > > > > > put the checkpoint data is another story. My only concern lies in
> > the
> > > > > newly
> > > > > > introduced setCheckpointStorage API and how we define its
> > semantics,
> > > > and
> > > > > > not sure whether it's due to my ignorance.
> > > > > >
> > > > > > Best Regards,
> > > > > > Yu
> > > > > >
> > > > > >
> > > > > > On Thu, 24 Sep 2020 at 23:11, Yu Li <ca...@gmail.com> wrote:
> > > > > >
> > > > > > > *bq. What new confusion would be introduced here?*
> > > > > > > No *new* confusion introduced, but as mentioned at the very
> > > beginning
> > > > > of
> > > > > > > the motivation ("Apache Flink's durability story is a mystery
> to
> > > many
> > > > > > > users"), I thought the FLIP aims at resolving some *existing*
> > > > > > > confusions, i.e. the durability mystery to users.
> > > > > > >
> > > > > > > For me, I'm not 100% clear about how to write the javadoc of
> the
> > > > > > > setCheckpointStorage API. Would it be like "specify where the
> > > > > checkpoint
> > > > > > > data is stored"? If so, do we need to explain the fact that
> when
> > a
> > > > > > > checkpoint path is given, JM will also persist the checkpoint
> > data
> > > to
> > > > > > DFS?
> > > > > > > It's true that such confusion also exists today, but would the
> > > > > > introduction
> > > > > > > of the new API expose it further?
> > > > > > >
> > > > > > > IMHO we need to document the newly introduced API / classes and
> > > their
> > > > > > > semantics clearly in the FLIP to make sure everyone is on the
> > same
> > > > > page,
> > > > > > > but if we feel such work / discussions are all details and only
> > > need
> > > > to
> > > > > > > happen at the documenting and release note phase, it's also
> fine
> > to
> > > > me.
> > > > > > >
> > > > > > > And if I'm the only one who has such questions / concerns on
> the
> > > new
> > > > > > > `setCheckpointStorage` API and most of others feel its semantic
> > is
> > > > > sound
> > > > > > > and clear, then please just ignore me and move on.
> > > > > > >
> > > > > > > Thanks.
> > > > > > >
> > > > > > > Best Regards,
> > > > > > > Yu
> > > > > > >
> > > > > > >
> > > > > > > On Wed, 23 Sep 2020 at 17:08, Stephan Ewen <se...@apache.org>
> > > wrote:
> > > > > > >
> > > > > > >> I am confused now with the concerns here. This is very much
> from
> > > the
> > > > > > user
> > > > > > >> perspective (which is partially also the developer perspective
> > > which
> > > > > is
> > > > > > >> the
> > > > > > >> sign of an intuitive abstraction).
> > > > > > >>
> > > > > > >> Of course, there will be docs describing what
> > JMCheckpointStorage
> > > > and
> > > > > > >> FsCheckpointStorage are.
> > > > > > >> And having release notes that describe that
> > > > > > >> RocksDBStateBackend("s3://...")
> > > > > > >> now corresponds to a combination of "RocksDBBackend" and
> > > > > > >> "FsCheckpointStorage" is also straightforward.
> > > > > > >>
> > > > > > >> We said to keep the old RocksDBStateBackend class and let it
> > > > implement
> > > > > > >> both
> > > > > > >> interfaces such that the old code still works exactly as
> before.
> > > > > > >>
> > > > > > >> What new confusion would be introduced here?
> > > > > > >> Understanding the difference between JMCheckpointStorage and
> > > > > > >> FsCheckpointStorage was always necessary when one needed to
> > > > understand
> > > > > > the
> > > > > > >> difference between MemoryStateBackend and FsStateBackend. It
> > > should
> > > > be
> > > > > > >> easier to define this after this change, because it is the
> only
> > > > thing
> > > > > > that
> > > > > > >> we describe when explaining what checkpoint storage to use
> > (rather
> > > > > than
> > > > > > >> also having the choice of index structure coupled to that).
> > > > > > >>
> > > > > > >>
> > > > > > >> On Wed, Sep 23, 2020 at 10:39 AM Aljoscha Krettek <
> > > > > aljoscha@apache.org>
> > > > > > >> wrote:
> > > > > > >>
> > > > > > >> > On 23.09.20 04:40, Yu Li wrote:
> > > > > > >> > > To be specific, with the old API users don't need to set
> > > > > checkpoint
> > > > > > >> > > storage, instead they only need to pass the checkpoint
> path
> > > w/o
> > > > > > caring
> > > > > > >> > > about the storage. The new APIs are forcing users to set
> the
> > > > > storage
> > > > > > >> so
> > > > > > >> > > they have to know the difference between different
> storages.
> > > > It's
> > > > > > not
> > > > > > >> an
> > > > > > >> > > implementation change, but an API change that users have
> to
> > > > > > understand
> > > > > > >> > and
> > > > > > >> > > follow-up.
> > > > > > >> >
> > > > > > >> > I think the main point of the FLIP is to make it more
> obvious
> > to
> > > > > users
> > > > > > >> > what is happening.
> > > > > > >> >
> > > > > > >> > With current Flink, they would do a `setStateBackend(new
> > > > > > >> > FsStateBackend(<path>))`. What the user is actually "saying"
> > > with
> > > > > this
> > > > > > >> > is: I want to keep state on heap but store checkpoints in
> DFS.
> > > > They
> > > > > > are
> > > > > > >> > not actually changing the "State Backend", the thing that
> > keeps
> > > > > state
> > > > > > in
> > > > > > >> > operators, but only where state is checkpointed. The thing
> > that
> > > is
> > > > > > used
> > > > > > >> > for local state storage in operators is still the "Heap
> > > Backend".
> > > > > > >> >
> > > > > > >> > With the proposed FLIP, a user would do a
> > > > `setCheckpointStorage(new
> > > > > > >> > FsStorage(<path>))`. Which makes it obvious that they're
> > > changing
> > > > > > where
> > > > > > >> > checkpoints are stored but not the actual "State Backend",
> > which
> > > > is
> > > > > > >> > still "Heap Backend" (the default).
> > > > > > >> >
> > > > > > >> > I do understand Yu's point, though, that this will be
> > confusing
> > > > for
> > > > > > >> > current Flink users. They are used to setting a "State
> > Backend"
> > > > > > if/when
> > > > > > >> > they want to change the storage location. To fit the new
> model
> > > > they
> > > > > > >> > would have to change the call from `setStateBackend()` to
> > > > > > >> > `setCheckpointStorage()`.
> > > > > > >> >
> > > > > > >> > I think we need to life with this short-term confusion
> because
> > > in
> > > > > the
> > > > > > >> > long run the proposed split between checkpoint location and
> > > state
> > > > > > >> > backend makes sense and will be more straightforward for
> users
> > > to
> > > > > > >> > understand.
> > > > > > >> >
> > > > > > >> > Best,
> > > > > > >> > Aljoscha
> > > > > > >> >
> > > > > > >> >
> > > > > > >>
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] FLIP-142: Disentangle StateBackends from Checkpointing

Posted by Stephan Ewen <se...@apache.org>.
Good to see this FLIP moving.

From what I understand, the remaining questions are mainly about how to
express the roles of the CheckpointStorage and specifically the behavior of
JMCheckpointStorage and FsCheckpointStorage in the docs.
This sounds like details we should discuss over the concrete text proposals
in the PR.

On Sun, Sep 27, 2020 at 5:38 AM Yu Li <ca...@gmail.com> wrote:

> Thanks Seth, the updated FLIP overall LGTM, and I've left some inline
> comments in the doc.
>
> Best Regards,
> Yu
>
>
> On Fri, 25 Sep 2020 at 20:58, Seth Wiesman <sj...@gmail.com> wrote:
>
> > Done
> >
> > Seth
> >
> > On Fri, Sep 25, 2020 at 2:47 AM Yu Li <ca...@gmail.com> wrote:
> >
> > > *bq. I think it might help to highlight specific stumbling blocks users
> > > have today and why I believe this change addresses those issues.*
> > > Thanks for adding more details, I believe adding these blocks to the
> FLIP
> > > doc could make the motivation more vivid and convincing.
> > >
> > > *bq. To be concrete I think the JavaDoc for setCheckpointStorage would
> be
> > > something like...*
> > > I could see this definition extracts the existing description from the
> > > current `StateBackend` interface, it's a valid option, and let me quote
> > it
> > > again:
> > > - CheckpointStorage defines how checkpoint snapshots are persisted for
> > > fault tolerance. Various implementations store their checkpoints in
> > > different fashions and have different requirements and availability
> > > guarantees.
> > > - JobManagerCheckpointStorage stores checkpoints in the memory of the
> > > JobManager. It is lightweight and without additional dependencies but
> is
> > > not highly available.
> > > - FileSystemCheckpointStorage stores checkpoints in a file system. For
> > > systems like HDFS, NFS Drives, S3, and GCS, this storage policy
> supports
> > > large state size, in the magnitude of many terabytes while providing a
> > > highly available foundation for stateful applications. This checkpoint
> > > storage policy is recommended for most production deployments.
> > >
> > > Sticking to this definition, I think we should have the below migration
> > > plans for existing backends:
> > > - `MemoryStateBackend(null, String savepointPath)` to
> > > `HashMapStateBackend() + JobManagerCheckpointStorage()`
> > > - `MemoryStateBackend(<non-null-checkpoint-path>, String
> savepointPath)`
> > to
> > > `HashMapStateBackend() + FileSystemCheckpointStorage()`
> > > in addition of the existing:
> > > - `MemoryStateBackend()` to `HashMapStateBackend() +
> > > JobManagerCheckpointStorage()`
> > > and could be summarized as:
> > > - MemoryStateBackend with checkpoint path: `HashMapStateBackend() +
> > > FileSystemCheckpointStorage()`
> > > - MemoryStateBackend w/o checkpoint path: `HashMapStateBackend() +
> > > JobManagerCheckpointStorage()`
> > >
> > > And I believe adding the above highlighted blocks to the FLIP doc (the
> > "New
> > > StateBackend User API" and "Migration Plan" sections, separately) could
> > > make it more complete.
> > >
> > > PS. Please note that although the current javadoc of `StateBackend`
> > states
> > > "MemoryStateBackend is not highly available", it actually supports
> > > persisting the checkpoint data to DFS when checkpoint path is given, so
> > the
> > > mapping between old and new APIs are not that straight-forward and need
> > > some clear clarifications, from my point of view.
> > >
> > > Best Regards,
> > > Yu
> > >
> > >
> > > On Fri, 25 Sep 2020 at 08:33, Seth Wiesman <sj...@gmail.com>
> wrote:
> > >
> > > > Hi Yu,
> > > >
> > > > bq* I thought the FLIP aims at resolving some *existing* confusion,
> > i.e.
> > > > the durability mystery to users.
> > > >
> > > > I think it might help to highlight specific stumbling blocks users
> have
> > > > today and why I believe this change addresses those issues. Some
> > frequent
> > > > things I've heard over the past several years include:
> > > >
> > > > 1) "We use RocksDB because we don't need fault tolerance."
> > > > 2) "We don't use RocksDB because we don't want to manage an external
> > > > database."
> > > > 3) Believing RocksDB is reading and writing directly with S3 or HDFS
> > (vs.
> > > > local disk)
> > > > 4) Believing FsStateBackend spills to disk or has anything to do with
> > the
> > > > local filesystem
> > > > 5) Pointing RocksDB at network-attached storage, believing that the
> > state
> > > > backend needs to be fault-tolerant
> > > >
> > > > This question from the ml is very representative of where users are
> > > > struggling[1]. Many of these questions were not from new users but
> from
> > > > organizations that were in production! Just yesterday I was on the
> > phone
> > > > with a company that didn't realize they were in production without
> > > > checkpointing; honestly, you would be shocked how often this happens.
> > The
> > > > current state backend abstraction is to complex for many of our
> users.
> > > What
> > > > all these questions have in common is misunderstanding the
> relationship
> > > > between how data is stored locally on TMs vs how checkpoints make
> that
> > > > state durable.
> > > >
> > > > The FLIP aims actively help users by allowing them to reason about
> > state
> > > > backends separately from checkpoint durability. In the future, a
> state
> > > > backend only defines where and how state is stored locally on the TM
> > > while
> > > > checkpoint storage defines where and how checkpoints are stored for
> > > > recovery. To be concrete I think the JavaDoc for setCheckpointStorage
> > > would
> > > > be something like:
> > > >
> > > > ```java
> > > > /**
> > > >  * CheckpointStorage defines how checkpoint snapshots are persisted
> for
> > > > fault tolerance
> > > > *. Various implementations  store their checkpoints in different
> > fashions
> > > > and have different requirements and
> > > >  * availability guarantees.
> > > >  *
> > > >  *<p>For example, JobManagerCheckpointStorage stores checkpoints in
> the
> > > > memory of the JobManager.
> > > >  * It is lightweight and without additional dependencies but is not
> > > highly
> > > > available
> > > >  * and only supports small state sizes. This checkpoint storage
> policy
> > is
> > > > convenient for
> > > >  * local testing and development.
> > > >  *
> > > >  *<p>FileSystemCheckpointStorage stores checkpoints in a filesystem.
> > For
> > > > systems like
> > > >  * HDFS, NFS Drives, S3, and GCS, this storage policy supports large
> > > state
> > > > size,
> > > >  * in the magnitude of many terabytes while providing a highly
> > available
> > > > foundation
> > > >  * for stateful applications. This checkpoint storage policy is
> > > recommended
> > > > for most
> > > >  * production deployments.
> > > >  */
> > > > void setCheckpointStorage(CheckpointStorage storage) {}
> > > > ```
> > > >
> > > > Seth
> > > >
> > > > [1]
> > > >
> > > >
> > >
> >
> http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/State-Storage-Questions-td37919.html
> > > > [2] Also naming, but we're aligned here
> > > >
> > > > On Thu, Sep 24, 2020 at 10:24 AM Yu Li <ca...@gmail.com> wrote:
> > > >
> > > > > And to make it clear, I'm +1 on the idea of decoupling state
> backends
> > > > with
> > > > > checkpointing. I don't have any question about making it clear that
> > > > > heap/RocksDB is where we serve the routine state read/write and
> where
> > > to
> > > > > put the checkpoint data is another story. My only concern lies in
> the
> > > > newly
> > > > > introduced setCheckpointStorage API and how we define its
> semantics,
> > > and
> > > > > not sure whether it's due to my ignorance.
> > > > >
> > > > > Best Regards,
> > > > > Yu
> > > > >
> > > > >
> > > > > On Thu, 24 Sep 2020 at 23:11, Yu Li <ca...@gmail.com> wrote:
> > > > >
> > > > > > *bq. What new confusion would be introduced here?*
> > > > > > No *new* confusion introduced, but as mentioned at the very
> > beginning
> > > > of
> > > > > > the motivation ("Apache Flink's durability story is a mystery to
> > many
> > > > > > users"), I thought the FLIP aims at resolving some *existing*
> > > > > > confusions, i.e. the durability mystery to users.
> > > > > >
> > > > > > For me, I'm not 100% clear about how to write the javadoc of the
> > > > > > setCheckpointStorage API. Would it be like "specify where the
> > > > checkpoint
> > > > > > data is stored"? If so, do we need to explain the fact that when
> a
> > > > > > checkpoint path is given, JM will also persist the checkpoint
> data
> > to
> > > > > DFS?
> > > > > > It's true that such confusion also exists today, but would the
> > > > > introduction
> > > > > > of the new API expose it further?
> > > > > >
> > > > > > IMHO we need to document the newly introduced API / classes and
> > their
> > > > > > semantics clearly in the FLIP to make sure everyone is on the
> same
> > > > page,
> > > > > > but if we feel such work / discussions are all details and only
> > need
> > > to
> > > > > > happen at the documenting and release note phase, it's also fine
> to
> > > me.
> > > > > >
> > > > > > And if I'm the only one who has such questions / concerns on the
> > new
> > > > > > `setCheckpointStorage` API and most of others feel its semantic
> is
> > > > sound
> > > > > > and clear, then please just ignore me and move on.
> > > > > >
> > > > > > Thanks.
> > > > > >
> > > > > > Best Regards,
> > > > > > Yu
> > > > > >
> > > > > >
> > > > > > On Wed, 23 Sep 2020 at 17:08, Stephan Ewen <se...@apache.org>
> > wrote:
> > > > > >
> > > > > >> I am confused now with the concerns here. This is very much from
> > the
> > > > > user
> > > > > >> perspective (which is partially also the developer perspective
> > which
> > > > is
> > > > > >> the
> > > > > >> sign of an intuitive abstraction).
> > > > > >>
> > > > > >> Of course, there will be docs describing what
> JMCheckpointStorage
> > > and
> > > > > >> FsCheckpointStorage are.
> > > > > >> And having release notes that describe that
> > > > > >> RocksDBStateBackend("s3://...")
> > > > > >> now corresponds to a combination of "RocksDBBackend" and
> > > > > >> "FsCheckpointStorage" is also straightforward.
> > > > > >>
> > > > > >> We said to keep the old RocksDBStateBackend class and let it
> > > implement
> > > > > >> both
> > > > > >> interfaces such that the old code still works exactly as before.
> > > > > >>
> > > > > >> What new confusion would be introduced here?
> > > > > >> Understanding the difference between JMCheckpointStorage and
> > > > > >> FsCheckpointStorage was always necessary when one needed to
> > > understand
> > > > > the
> > > > > >> difference between MemoryStateBackend and FsStateBackend. It
> > should
> > > be
> > > > > >> easier to define this after this change, because it is the only
> > > thing
> > > > > that
> > > > > >> we describe when explaining what checkpoint storage to use
> (rather
> > > > than
> > > > > >> also having the choice of index structure coupled to that).
> > > > > >>
> > > > > >>
> > > > > >> On Wed, Sep 23, 2020 at 10:39 AM Aljoscha Krettek <
> > > > aljoscha@apache.org>
> > > > > >> wrote:
> > > > > >>
> > > > > >> > On 23.09.20 04:40, Yu Li wrote:
> > > > > >> > > To be specific, with the old API users don't need to set
> > > > checkpoint
> > > > > >> > > storage, instead they only need to pass the checkpoint path
> > w/o
> > > > > caring
> > > > > >> > > about the storage. The new APIs are forcing users to set the
> > > > storage
> > > > > >> so
> > > > > >> > > they have to know the difference between different storages.
> > > It's
> > > > > not
> > > > > >> an
> > > > > >> > > implementation change, but an API change that users have to
> > > > > understand
> > > > > >> > and
> > > > > >> > > follow-up.
> > > > > >> >
> > > > > >> > I think the main point of the FLIP is to make it more obvious
> to
> > > > users
> > > > > >> > what is happening.
> > > > > >> >
> > > > > >> > With current Flink, they would do a `setStateBackend(new
> > > > > >> > FsStateBackend(<path>))`. What the user is actually "saying"
> > with
> > > > this
> > > > > >> > is: I want to keep state on heap but store checkpoints in DFS.
> > > They
> > > > > are
> > > > > >> > not actually changing the "State Backend", the thing that
> keeps
> > > > state
> > > > > in
> > > > > >> > operators, but only where state is checkpointed. The thing
> that
> > is
> > > > > used
> > > > > >> > for local state storage in operators is still the "Heap
> > Backend".
> > > > > >> >
> > > > > >> > With the proposed FLIP, a user would do a
> > > `setCheckpointStorage(new
> > > > > >> > FsStorage(<path>))`. Which makes it obvious that they're
> > changing
> > > > > where
> > > > > >> > checkpoints are stored but not the actual "State Backend",
> which
> > > is
> > > > > >> > still "Heap Backend" (the default).
> > > > > >> >
> > > > > >> > I do understand Yu's point, though, that this will be
> confusing
> > > for
> > > > > >> > current Flink users. They are used to setting a "State
> Backend"
> > > > > if/when
> > > > > >> > they want to change the storage location. To fit the new model
> > > they
> > > > > >> > would have to change the call from `setStateBackend()` to
> > > > > >> > `setCheckpointStorage()`.
> > > > > >> >
> > > > > >> > I think we need to life with this short-term confusion because
> > in
> > > > the
> > > > > >> > long run the proposed split between checkpoint location and
> > state
> > > > > >> > backend makes sense and will be more straightforward for users
> > to
> > > > > >> > understand.
> > > > > >> >
> > > > > >> > Best,
> > > > > >> > Aljoscha
> > > > > >> >
> > > > > >> >
> > > > > >>
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] FLIP-142: Disentangle StateBackends from Checkpointing

Posted by Yu Li <ca...@gmail.com>.
Thanks Seth, the updated FLIP overall LGTM, and I've left some inline
comments in the doc.

Best Regards,
Yu


On Fri, 25 Sep 2020 at 20:58, Seth Wiesman <sj...@gmail.com> wrote:

> Done
>
> Seth
>
> On Fri, Sep 25, 2020 at 2:47 AM Yu Li <ca...@gmail.com> wrote:
>
> > *bq. I think it might help to highlight specific stumbling blocks users
> > have today and why I believe this change addresses those issues.*
> > Thanks for adding more details, I believe adding these blocks to the FLIP
> > doc could make the motivation more vivid and convincing.
> >
> > *bq. To be concrete I think the JavaDoc for setCheckpointStorage would be
> > something like...*
> > I could see this definition extracts the existing description from the
> > current `StateBackend` interface, it's a valid option, and let me quote
> it
> > again:
> > - CheckpointStorage defines how checkpoint snapshots are persisted for
> > fault tolerance. Various implementations store their checkpoints in
> > different fashions and have different requirements and availability
> > guarantees.
> > - JobManagerCheckpointStorage stores checkpoints in the memory of the
> > JobManager. It is lightweight and without additional dependencies but is
> > not highly available.
> > - FileSystemCheckpointStorage stores checkpoints in a file system. For
> > systems like HDFS, NFS Drives, S3, and GCS, this storage policy supports
> > large state size, in the magnitude of many terabytes while providing a
> > highly available foundation for stateful applications. This checkpoint
> > storage policy is recommended for most production deployments.
> >
> > Sticking to this definition, I think we should have the below migration
> > plans for existing backends:
> > - `MemoryStateBackend(null, String savepointPath)` to
> > `HashMapStateBackend() + JobManagerCheckpointStorage()`
> > - `MemoryStateBackend(<non-null-checkpoint-path>, String savepointPath)`
> to
> > `HashMapStateBackend() + FileSystemCheckpointStorage()`
> > in addition of the existing:
> > - `MemoryStateBackend()` to `HashMapStateBackend() +
> > JobManagerCheckpointStorage()`
> > and could be summarized as:
> > - MemoryStateBackend with checkpoint path: `HashMapStateBackend() +
> > FileSystemCheckpointStorage()`
> > - MemoryStateBackend w/o checkpoint path: `HashMapStateBackend() +
> > JobManagerCheckpointStorage()`
> >
> > And I believe adding the above highlighted blocks to the FLIP doc (the
> "New
> > StateBackend User API" and "Migration Plan" sections, separately) could
> > make it more complete.
> >
> > PS. Please note that although the current javadoc of `StateBackend`
> states
> > "MemoryStateBackend is not highly available", it actually supports
> > persisting the checkpoint data to DFS when checkpoint path is given, so
> the
> > mapping between old and new APIs are not that straight-forward and need
> > some clear clarifications, from my point of view.
> >
> > Best Regards,
> > Yu
> >
> >
> > On Fri, 25 Sep 2020 at 08:33, Seth Wiesman <sj...@gmail.com> wrote:
> >
> > > Hi Yu,
> > >
> > > bq* I thought the FLIP aims at resolving some *existing* confusion,
> i.e.
> > > the durability mystery to users.
> > >
> > > I think it might help to highlight specific stumbling blocks users have
> > > today and why I believe this change addresses those issues. Some
> frequent
> > > things I've heard over the past several years include:
> > >
> > > 1) "We use RocksDB because we don't need fault tolerance."
> > > 2) "We don't use RocksDB because we don't want to manage an external
> > > database."
> > > 3) Believing RocksDB is reading and writing directly with S3 or HDFS
> (vs.
> > > local disk)
> > > 4) Believing FsStateBackend spills to disk or has anything to do with
> the
> > > local filesystem
> > > 5) Pointing RocksDB at network-attached storage, believing that the
> state
> > > backend needs to be fault-tolerant
> > >
> > > This question from the ml is very representative of where users are
> > > struggling[1]. Many of these questions were not from new users but from
> > > organizations that were in production! Just yesterday I was on the
> phone
> > > with a company that didn't realize they were in production without
> > > checkpointing; honestly, you would be shocked how often this happens.
> The
> > > current state backend abstraction is to complex for many of our users.
> > What
> > > all these questions have in common is misunderstanding the relationship
> > > between how data is stored locally on TMs vs how checkpoints make that
> > > state durable.
> > >
> > > The FLIP aims actively help users by allowing them to reason about
> state
> > > backends separately from checkpoint durability. In the future, a state
> > > backend only defines where and how state is stored locally on the TM
> > while
> > > checkpoint storage defines where and how checkpoints are stored for
> > > recovery. To be concrete I think the JavaDoc for setCheckpointStorage
> > would
> > > be something like:
> > >
> > > ```java
> > > /**
> > >  * CheckpointStorage defines how checkpoint snapshots are persisted for
> > > fault tolerance
> > > *. Various implementations  store their checkpoints in different
> fashions
> > > and have different requirements and
> > >  * availability guarantees.
> > >  *
> > >  *<p>For example, JobManagerCheckpointStorage stores checkpoints in the
> > > memory of the JobManager.
> > >  * It is lightweight and without additional dependencies but is not
> > highly
> > > available
> > >  * and only supports small state sizes. This checkpoint storage policy
> is
> > > convenient for
> > >  * local testing and development.
> > >  *
> > >  *<p>FileSystemCheckpointStorage stores checkpoints in a filesystem.
> For
> > > systems like
> > >  * HDFS, NFS Drives, S3, and GCS, this storage policy supports large
> > state
> > > size,
> > >  * in the magnitude of many terabytes while providing a highly
> available
> > > foundation
> > >  * for stateful applications. This checkpoint storage policy is
> > recommended
> > > for most
> > >  * production deployments.
> > >  */
> > > void setCheckpointStorage(CheckpointStorage storage) {}
> > > ```
> > >
> > > Seth
> > >
> > > [1]
> > >
> > >
> >
> http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/State-Storage-Questions-td37919.html
> > > [2] Also naming, but we're aligned here
> > >
> > > On Thu, Sep 24, 2020 at 10:24 AM Yu Li <ca...@gmail.com> wrote:
> > >
> > > > And to make it clear, I'm +1 on the idea of decoupling state backends
> > > with
> > > > checkpointing. I don't have any question about making it clear that
> > > > heap/RocksDB is where we serve the routine state read/write and where
> > to
> > > > put the checkpoint data is another story. My only concern lies in the
> > > newly
> > > > introduced setCheckpointStorage API and how we define its semantics,
> > and
> > > > not sure whether it's due to my ignorance.
> > > >
> > > > Best Regards,
> > > > Yu
> > > >
> > > >
> > > > On Thu, 24 Sep 2020 at 23:11, Yu Li <ca...@gmail.com> wrote:
> > > >
> > > > > *bq. What new confusion would be introduced here?*
> > > > > No *new* confusion introduced, but as mentioned at the very
> beginning
> > > of
> > > > > the motivation ("Apache Flink's durability story is a mystery to
> many
> > > > > users"), I thought the FLIP aims at resolving some *existing*
> > > > > confusions, i.e. the durability mystery to users.
> > > > >
> > > > > For me, I'm not 100% clear about how to write the javadoc of the
> > > > > setCheckpointStorage API. Would it be like "specify where the
> > > checkpoint
> > > > > data is stored"? If so, do we need to explain the fact that when a
> > > > > checkpoint path is given, JM will also persist the checkpoint data
> to
> > > > DFS?
> > > > > It's true that such confusion also exists today, but would the
> > > > introduction
> > > > > of the new API expose it further?
> > > > >
> > > > > IMHO we need to document the newly introduced API / classes and
> their
> > > > > semantics clearly in the FLIP to make sure everyone is on the same
> > > page,
> > > > > but if we feel such work / discussions are all details and only
> need
> > to
> > > > > happen at the documenting and release note phase, it's also fine to
> > me.
> > > > >
> > > > > And if I'm the only one who has such questions / concerns on the
> new
> > > > > `setCheckpointStorage` API and most of others feel its semantic is
> > > sound
> > > > > and clear, then please just ignore me and move on.
> > > > >
> > > > > Thanks.
> > > > >
> > > > > Best Regards,
> > > > > Yu
> > > > >
> > > > >
> > > > > On Wed, 23 Sep 2020 at 17:08, Stephan Ewen <se...@apache.org>
> wrote:
> > > > >
> > > > >> I am confused now with the concerns here. This is very much from
> the
> > > > user
> > > > >> perspective (which is partially also the developer perspective
> which
> > > is
> > > > >> the
> > > > >> sign of an intuitive abstraction).
> > > > >>
> > > > >> Of course, there will be docs describing what JMCheckpointStorage
> > and
> > > > >> FsCheckpointStorage are.
> > > > >> And having release notes that describe that
> > > > >> RocksDBStateBackend("s3://...")
> > > > >> now corresponds to a combination of "RocksDBBackend" and
> > > > >> "FsCheckpointStorage" is also straightforward.
> > > > >>
> > > > >> We said to keep the old RocksDBStateBackend class and let it
> > implement
> > > > >> both
> > > > >> interfaces such that the old code still works exactly as before.
> > > > >>
> > > > >> What new confusion would be introduced here?
> > > > >> Understanding the difference between JMCheckpointStorage and
> > > > >> FsCheckpointStorage was always necessary when one needed to
> > understand
> > > > the
> > > > >> difference between MemoryStateBackend and FsStateBackend. It
> should
> > be
> > > > >> easier to define this after this change, because it is the only
> > thing
> > > > that
> > > > >> we describe when explaining what checkpoint storage to use (rather
> > > than
> > > > >> also having the choice of index structure coupled to that).
> > > > >>
> > > > >>
> > > > >> On Wed, Sep 23, 2020 at 10:39 AM Aljoscha Krettek <
> > > aljoscha@apache.org>
> > > > >> wrote:
> > > > >>
> > > > >> > On 23.09.20 04:40, Yu Li wrote:
> > > > >> > > To be specific, with the old API users don't need to set
> > > checkpoint
> > > > >> > > storage, instead they only need to pass the checkpoint path
> w/o
> > > > caring
> > > > >> > > about the storage. The new APIs are forcing users to set the
> > > storage
> > > > >> so
> > > > >> > > they have to know the difference between different storages.
> > It's
> > > > not
> > > > >> an
> > > > >> > > implementation change, but an API change that users have to
> > > > understand
> > > > >> > and
> > > > >> > > follow-up.
> > > > >> >
> > > > >> > I think the main point of the FLIP is to make it more obvious to
> > > users
> > > > >> > what is happening.
> > > > >> >
> > > > >> > With current Flink, they would do a `setStateBackend(new
> > > > >> > FsStateBackend(<path>))`. What the user is actually "saying"
> with
> > > this
> > > > >> > is: I want to keep state on heap but store checkpoints in DFS.
> > They
> > > > are
> > > > >> > not actually changing the "State Backend", the thing that keeps
> > > state
> > > > in
> > > > >> > operators, but only where state is checkpointed. The thing that
> is
> > > > used
> > > > >> > for local state storage in operators is still the "Heap
> Backend".
> > > > >> >
> > > > >> > With the proposed FLIP, a user would do a
> > `setCheckpointStorage(new
> > > > >> > FsStorage(<path>))`. Which makes it obvious that they're
> changing
> > > > where
> > > > >> > checkpoints are stored but not the actual "State Backend", which
> > is
> > > > >> > still "Heap Backend" (the default).
> > > > >> >
> > > > >> > I do understand Yu's point, though, that this will be confusing
> > for
> > > > >> > current Flink users. They are used to setting a "State Backend"
> > > > if/when
> > > > >> > they want to change the storage location. To fit the new model
> > they
> > > > >> > would have to change the call from `setStateBackend()` to
> > > > >> > `setCheckpointStorage()`.
> > > > >> >
> > > > >> > I think we need to life with this short-term confusion because
> in
> > > the
> > > > >> > long run the proposed split between checkpoint location and
> state
> > > > >> > backend makes sense and will be more straightforward for users
> to
> > > > >> > understand.
> > > > >> >
> > > > >> > Best,
> > > > >> > Aljoscha
> > > > >> >
> > > > >> >
> > > > >>
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] FLIP-142: Disentangle StateBackends from Checkpointing

Posted by Seth Wiesman <sj...@gmail.com>.
Done

Seth

On Fri, Sep 25, 2020 at 2:47 AM Yu Li <ca...@gmail.com> wrote:

> *bq. I think it might help to highlight specific stumbling blocks users
> have today and why I believe this change addresses those issues.*
> Thanks for adding more details, I believe adding these blocks to the FLIP
> doc could make the motivation more vivid and convincing.
>
> *bq. To be concrete I think the JavaDoc for setCheckpointStorage would be
> something like...*
> I could see this definition extracts the existing description from the
> current `StateBackend` interface, it's a valid option, and let me quote it
> again:
> - CheckpointStorage defines how checkpoint snapshots are persisted for
> fault tolerance. Various implementations store their checkpoints in
> different fashions and have different requirements and availability
> guarantees.
> - JobManagerCheckpointStorage stores checkpoints in the memory of the
> JobManager. It is lightweight and without additional dependencies but is
> not highly available.
> - FileSystemCheckpointStorage stores checkpoints in a file system. For
> systems like HDFS, NFS Drives, S3, and GCS, this storage policy supports
> large state size, in the magnitude of many terabytes while providing a
> highly available foundation for stateful applications. This checkpoint
> storage policy is recommended for most production deployments.
>
> Sticking to this definition, I think we should have the below migration
> plans for existing backends:
> - `MemoryStateBackend(null, String savepointPath)` to
> `HashMapStateBackend() + JobManagerCheckpointStorage()`
> - `MemoryStateBackend(<non-null-checkpoint-path>, String savepointPath)` to
> `HashMapStateBackend() + FileSystemCheckpointStorage()`
> in addition of the existing:
> - `MemoryStateBackend()` to `HashMapStateBackend() +
> JobManagerCheckpointStorage()`
> and could be summarized as:
> - MemoryStateBackend with checkpoint path: `HashMapStateBackend() +
> FileSystemCheckpointStorage()`
> - MemoryStateBackend w/o checkpoint path: `HashMapStateBackend() +
> JobManagerCheckpointStorage()`
>
> And I believe adding the above highlighted blocks to the FLIP doc (the "New
> StateBackend User API" and "Migration Plan" sections, separately) could
> make it more complete.
>
> PS. Please note that although the current javadoc of `StateBackend` states
> "MemoryStateBackend is not highly available", it actually supports
> persisting the checkpoint data to DFS when checkpoint path is given, so the
> mapping between old and new APIs are not that straight-forward and need
> some clear clarifications, from my point of view.
>
> Best Regards,
> Yu
>
>
> On Fri, 25 Sep 2020 at 08:33, Seth Wiesman <sj...@gmail.com> wrote:
>
> > Hi Yu,
> >
> > bq* I thought the FLIP aims at resolving some *existing* confusion, i.e.
> > the durability mystery to users.
> >
> > I think it might help to highlight specific stumbling blocks users have
> > today and why I believe this change addresses those issues. Some frequent
> > things I've heard over the past several years include:
> >
> > 1) "We use RocksDB because we don't need fault tolerance."
> > 2) "We don't use RocksDB because we don't want to manage an external
> > database."
> > 3) Believing RocksDB is reading and writing directly with S3 or HDFS (vs.
> > local disk)
> > 4) Believing FsStateBackend spills to disk or has anything to do with the
> > local filesystem
> > 5) Pointing RocksDB at network-attached storage, believing that the state
> > backend needs to be fault-tolerant
> >
> > This question from the ml is very representative of where users are
> > struggling[1]. Many of these questions were not from new users but from
> > organizations that were in production! Just yesterday I was on the phone
> > with a company that didn't realize they were in production without
> > checkpointing; honestly, you would be shocked how often this happens. The
> > current state backend abstraction is to complex for many of our users.
> What
> > all these questions have in common is misunderstanding the relationship
> > between how data is stored locally on TMs vs how checkpoints make that
> > state durable.
> >
> > The FLIP aims actively help users by allowing them to reason about state
> > backends separately from checkpoint durability. In the future, a state
> > backend only defines where and how state is stored locally on the TM
> while
> > checkpoint storage defines where and how checkpoints are stored for
> > recovery. To be concrete I think the JavaDoc for setCheckpointStorage
> would
> > be something like:
> >
> > ```java
> > /**
> >  * CheckpointStorage defines how checkpoint snapshots are persisted for
> > fault tolerance
> > *. Various implementations  store their checkpoints in different fashions
> > and have different requirements and
> >  * availability guarantees.
> >  *
> >  *<p>For example, JobManagerCheckpointStorage stores checkpoints in the
> > memory of the JobManager.
> >  * It is lightweight and without additional dependencies but is not
> highly
> > available
> >  * and only supports small state sizes. This checkpoint storage policy is
> > convenient for
> >  * local testing and development.
> >  *
> >  *<p>FileSystemCheckpointStorage stores checkpoints in a filesystem. For
> > systems like
> >  * HDFS, NFS Drives, S3, and GCS, this storage policy supports large
> state
> > size,
> >  * in the magnitude of many terabytes while providing a highly available
> > foundation
> >  * for stateful applications. This checkpoint storage policy is
> recommended
> > for most
> >  * production deployments.
> >  */
> > void setCheckpointStorage(CheckpointStorage storage) {}
> > ```
> >
> > Seth
> >
> > [1]
> >
> >
> http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/State-Storage-Questions-td37919.html
> > [2] Also naming, but we're aligned here
> >
> > On Thu, Sep 24, 2020 at 10:24 AM Yu Li <ca...@gmail.com> wrote:
> >
> > > And to make it clear, I'm +1 on the idea of decoupling state backends
> > with
> > > checkpointing. I don't have any question about making it clear that
> > > heap/RocksDB is where we serve the routine state read/write and where
> to
> > > put the checkpoint data is another story. My only concern lies in the
> > newly
> > > introduced setCheckpointStorage API and how we define its semantics,
> and
> > > not sure whether it's due to my ignorance.
> > >
> > > Best Regards,
> > > Yu
> > >
> > >
> > > On Thu, 24 Sep 2020 at 23:11, Yu Li <ca...@gmail.com> wrote:
> > >
> > > > *bq. What new confusion would be introduced here?*
> > > > No *new* confusion introduced, but as mentioned at the very beginning
> > of
> > > > the motivation ("Apache Flink's durability story is a mystery to many
> > > > users"), I thought the FLIP aims at resolving some *existing*
> > > > confusions, i.e. the durability mystery to users.
> > > >
> > > > For me, I'm not 100% clear about how to write the javadoc of the
> > > > setCheckpointStorage API. Would it be like "specify where the
> > checkpoint
> > > > data is stored"? If so, do we need to explain the fact that when a
> > > > checkpoint path is given, JM will also persist the checkpoint data to
> > > DFS?
> > > > It's true that such confusion also exists today, but would the
> > > introduction
> > > > of the new API expose it further?
> > > >
> > > > IMHO we need to document the newly introduced API / classes and their
> > > > semantics clearly in the FLIP to make sure everyone is on the same
> > page,
> > > > but if we feel such work / discussions are all details and only need
> to
> > > > happen at the documenting and release note phase, it's also fine to
> me.
> > > >
> > > > And if I'm the only one who has such questions / concerns on the new
> > > > `setCheckpointStorage` API and most of others feel its semantic is
> > sound
> > > > and clear, then please just ignore me and move on.
> > > >
> > > > Thanks.
> > > >
> > > > Best Regards,
> > > > Yu
> > > >
> > > >
> > > > On Wed, 23 Sep 2020 at 17:08, Stephan Ewen <se...@apache.org> wrote:
> > > >
> > > >> I am confused now with the concerns here. This is very much from the
> > > user
> > > >> perspective (which is partially also the developer perspective which
> > is
> > > >> the
> > > >> sign of an intuitive abstraction).
> > > >>
> > > >> Of course, there will be docs describing what JMCheckpointStorage
> and
> > > >> FsCheckpointStorage are.
> > > >> And having release notes that describe that
> > > >> RocksDBStateBackend("s3://...")
> > > >> now corresponds to a combination of "RocksDBBackend" and
> > > >> "FsCheckpointStorage" is also straightforward.
> > > >>
> > > >> We said to keep the old RocksDBStateBackend class and let it
> implement
> > > >> both
> > > >> interfaces such that the old code still works exactly as before.
> > > >>
> > > >> What new confusion would be introduced here?
> > > >> Understanding the difference between JMCheckpointStorage and
> > > >> FsCheckpointStorage was always necessary when one needed to
> understand
> > > the
> > > >> difference between MemoryStateBackend and FsStateBackend. It should
> be
> > > >> easier to define this after this change, because it is the only
> thing
> > > that
> > > >> we describe when explaining what checkpoint storage to use (rather
> > than
> > > >> also having the choice of index structure coupled to that).
> > > >>
> > > >>
> > > >> On Wed, Sep 23, 2020 at 10:39 AM Aljoscha Krettek <
> > aljoscha@apache.org>
> > > >> wrote:
> > > >>
> > > >> > On 23.09.20 04:40, Yu Li wrote:
> > > >> > > To be specific, with the old API users don't need to set
> > checkpoint
> > > >> > > storage, instead they only need to pass the checkpoint path w/o
> > > caring
> > > >> > > about the storage. The new APIs are forcing users to set the
> > storage
> > > >> so
> > > >> > > they have to know the difference between different storages.
> It's
> > > not
> > > >> an
> > > >> > > implementation change, but an API change that users have to
> > > understand
> > > >> > and
> > > >> > > follow-up.
> > > >> >
> > > >> > I think the main point of the FLIP is to make it more obvious to
> > users
> > > >> > what is happening.
> > > >> >
> > > >> > With current Flink, they would do a `setStateBackend(new
> > > >> > FsStateBackend(<path>))`. What the user is actually "saying" with
> > this
> > > >> > is: I want to keep state on heap but store checkpoints in DFS.
> They
> > > are
> > > >> > not actually changing the "State Backend", the thing that keeps
> > state
> > > in
> > > >> > operators, but only where state is checkpointed. The thing that is
> > > used
> > > >> > for local state storage in operators is still the "Heap Backend".
> > > >> >
> > > >> > With the proposed FLIP, a user would do a
> `setCheckpointStorage(new
> > > >> > FsStorage(<path>))`. Which makes it obvious that they're changing
> > > where
> > > >> > checkpoints are stored but not the actual "State Backend", which
> is
> > > >> > still "Heap Backend" (the default).
> > > >> >
> > > >> > I do understand Yu's point, though, that this will be confusing
> for
> > > >> > current Flink users. They are used to setting a "State Backend"
> > > if/when
> > > >> > they want to change the storage location. To fit the new model
> they
> > > >> > would have to change the call from `setStateBackend()` to
> > > >> > `setCheckpointStorage()`.
> > > >> >
> > > >> > I think we need to life with this short-term confusion because in
> > the
> > > >> > long run the proposed split between checkpoint location and state
> > > >> > backend makes sense and will be more straightforward for users to
> > > >> > understand.
> > > >> >
> > > >> > Best,
> > > >> > Aljoscha
> > > >> >
> > > >> >
> > > >>
> > > >
> > >
> >
>

Re: [DISCUSS] FLIP-142: Disentangle StateBackends from Checkpointing

Posted by Yu Li <ca...@gmail.com>.
*bq. I think it might help to highlight specific stumbling blocks users
have today and why I believe this change addresses those issues.*
Thanks for adding more details, I believe adding these blocks to the FLIP
doc could make the motivation more vivid and convincing.

*bq. To be concrete I think the JavaDoc for setCheckpointStorage would be
something like...*
I could see this definition extracts the existing description from the
current `StateBackend` interface, it's a valid option, and let me quote it
again:
- CheckpointStorage defines how checkpoint snapshots are persisted for
fault tolerance. Various implementations store their checkpoints in
different fashions and have different requirements and availability
guarantees.
- JobManagerCheckpointStorage stores checkpoints in the memory of the
JobManager. It is lightweight and without additional dependencies but is
not highly available.
- FileSystemCheckpointStorage stores checkpoints in a file system. For
systems like HDFS, NFS Drives, S3, and GCS, this storage policy supports
large state size, in the magnitude of many terabytes while providing a
highly available foundation for stateful applications. This checkpoint
storage policy is recommended for most production deployments.

Sticking to this definition, I think we should have the below migration
plans for existing backends:
- `MemoryStateBackend(null, String savepointPath)` to
`HashMapStateBackend() + JobManagerCheckpointStorage()`
- `MemoryStateBackend(<non-null-checkpoint-path>, String savepointPath)` to
`HashMapStateBackend() + FileSystemCheckpointStorage()`
in addition of the existing:
- `MemoryStateBackend()` to `HashMapStateBackend() +
JobManagerCheckpointStorage()`
and could be summarized as:
- MemoryStateBackend with checkpoint path: `HashMapStateBackend() +
FileSystemCheckpointStorage()`
- MemoryStateBackend w/o checkpoint path: `HashMapStateBackend() +
JobManagerCheckpointStorage()`

And I believe adding the above highlighted blocks to the FLIP doc (the "New
StateBackend User API" and "Migration Plan" sections, separately) could
make it more complete.

PS. Please note that although the current javadoc of `StateBackend` states
"MemoryStateBackend is not highly available", it actually supports
persisting the checkpoint data to DFS when checkpoint path is given, so the
mapping between old and new APIs are not that straight-forward and need
some clear clarifications, from my point of view.

Best Regards,
Yu


On Fri, 25 Sep 2020 at 08:33, Seth Wiesman <sj...@gmail.com> wrote:

> Hi Yu,
>
> bq* I thought the FLIP aims at resolving some *existing* confusion, i.e.
> the durability mystery to users.
>
> I think it might help to highlight specific stumbling blocks users have
> today and why I believe this change addresses those issues. Some frequent
> things I've heard over the past several years include:
>
> 1) "We use RocksDB because we don't need fault tolerance."
> 2) "We don't use RocksDB because we don't want to manage an external
> database."
> 3) Believing RocksDB is reading and writing directly with S3 or HDFS (vs.
> local disk)
> 4) Believing FsStateBackend spills to disk or has anything to do with the
> local filesystem
> 5) Pointing RocksDB at network-attached storage, believing that the state
> backend needs to be fault-tolerant
>
> This question from the ml is very representative of where users are
> struggling[1]. Many of these questions were not from new users but from
> organizations that were in production! Just yesterday I was on the phone
> with a company that didn't realize they were in production without
> checkpointing; honestly, you would be shocked how often this happens. The
> current state backend abstraction is to complex for many of our users. What
> all these questions have in common is misunderstanding the relationship
> between how data is stored locally on TMs vs how checkpoints make that
> state durable.
>
> The FLIP aims actively help users by allowing them to reason about state
> backends separately from checkpoint durability. In the future, a state
> backend only defines where and how state is stored locally on the TM while
> checkpoint storage defines where and how checkpoints are stored for
> recovery. To be concrete I think the JavaDoc for setCheckpointStorage would
> be something like:
>
> ```java
> /**
>  * CheckpointStorage defines how checkpoint snapshots are persisted for
> fault tolerance
> *. Various implementations  store their checkpoints in different fashions
> and have different requirements and
>  * availability guarantees.
>  *
>  *<p>For example, JobManagerCheckpointStorage stores checkpoints in the
> memory of the JobManager.
>  * It is lightweight and without additional dependencies but is not highly
> available
>  * and only supports small state sizes. This checkpoint storage policy is
> convenient for
>  * local testing and development.
>  *
>  *<p>FileSystemCheckpointStorage stores checkpoints in a filesystem. For
> systems like
>  * HDFS, NFS Drives, S3, and GCS, this storage policy supports large state
> size,
>  * in the magnitude of many terabytes while providing a highly available
> foundation
>  * for stateful applications. This checkpoint storage policy is recommended
> for most
>  * production deployments.
>  */
> void setCheckpointStorage(CheckpointStorage storage) {}
> ```
>
> Seth
>
> [1]
>
> http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/State-Storage-Questions-td37919.html
> [2] Also naming, but we're aligned here
>
> On Thu, Sep 24, 2020 at 10:24 AM Yu Li <ca...@gmail.com> wrote:
>
> > And to make it clear, I'm +1 on the idea of decoupling state backends
> with
> > checkpointing. I don't have any question about making it clear that
> > heap/RocksDB is where we serve the routine state read/write and where to
> > put the checkpoint data is another story. My only concern lies in the
> newly
> > introduced setCheckpointStorage API and how we define its semantics, and
> > not sure whether it's due to my ignorance.
> >
> > Best Regards,
> > Yu
> >
> >
> > On Thu, 24 Sep 2020 at 23:11, Yu Li <ca...@gmail.com> wrote:
> >
> > > *bq. What new confusion would be introduced here?*
> > > No *new* confusion introduced, but as mentioned at the very beginning
> of
> > > the motivation ("Apache Flink's durability story is a mystery to many
> > > users"), I thought the FLIP aims at resolving some *existing*
> > > confusions, i.e. the durability mystery to users.
> > >
> > > For me, I'm not 100% clear about how to write the javadoc of the
> > > setCheckpointStorage API. Would it be like "specify where the
> checkpoint
> > > data is stored"? If so, do we need to explain the fact that when a
> > > checkpoint path is given, JM will also persist the checkpoint data to
> > DFS?
> > > It's true that such confusion also exists today, but would the
> > introduction
> > > of the new API expose it further?
> > >
> > > IMHO we need to document the newly introduced API / classes and their
> > > semantics clearly in the FLIP to make sure everyone is on the same
> page,
> > > but if we feel such work / discussions are all details and only need to
> > > happen at the documenting and release note phase, it's also fine to me.
> > >
> > > And if I'm the only one who has such questions / concerns on the new
> > > `setCheckpointStorage` API and most of others feel its semantic is
> sound
> > > and clear, then please just ignore me and move on.
> > >
> > > Thanks.
> > >
> > > Best Regards,
> > > Yu
> > >
> > >
> > > On Wed, 23 Sep 2020 at 17:08, Stephan Ewen <se...@apache.org> wrote:
> > >
> > >> I am confused now with the concerns here. This is very much from the
> > user
> > >> perspective (which is partially also the developer perspective which
> is
> > >> the
> > >> sign of an intuitive abstraction).
> > >>
> > >> Of course, there will be docs describing what JMCheckpointStorage and
> > >> FsCheckpointStorage are.
> > >> And having release notes that describe that
> > >> RocksDBStateBackend("s3://...")
> > >> now corresponds to a combination of "RocksDBBackend" and
> > >> "FsCheckpointStorage" is also straightforward.
> > >>
> > >> We said to keep the old RocksDBStateBackend class and let it implement
> > >> both
> > >> interfaces such that the old code still works exactly as before.
> > >>
> > >> What new confusion would be introduced here?
> > >> Understanding the difference between JMCheckpointStorage and
> > >> FsCheckpointStorage was always necessary when one needed to understand
> > the
> > >> difference between MemoryStateBackend and FsStateBackend. It should be
> > >> easier to define this after this change, because it is the only thing
> > that
> > >> we describe when explaining what checkpoint storage to use (rather
> than
> > >> also having the choice of index structure coupled to that).
> > >>
> > >>
> > >> On Wed, Sep 23, 2020 at 10:39 AM Aljoscha Krettek <
> aljoscha@apache.org>
> > >> wrote:
> > >>
> > >> > On 23.09.20 04:40, Yu Li wrote:
> > >> > > To be specific, with the old API users don't need to set
> checkpoint
> > >> > > storage, instead they only need to pass the checkpoint path w/o
> > caring
> > >> > > about the storage. The new APIs are forcing users to set the
> storage
> > >> so
> > >> > > they have to know the difference between different storages. It's
> > not
> > >> an
> > >> > > implementation change, but an API change that users have to
> > understand
> > >> > and
> > >> > > follow-up.
> > >> >
> > >> > I think the main point of the FLIP is to make it more obvious to
> users
> > >> > what is happening.
> > >> >
> > >> > With current Flink, they would do a `setStateBackend(new
> > >> > FsStateBackend(<path>))`. What the user is actually "saying" with
> this
> > >> > is: I want to keep state on heap but store checkpoints in DFS. They
> > are
> > >> > not actually changing the "State Backend", the thing that keeps
> state
> > in
> > >> > operators, but only where state is checkpointed. The thing that is
> > used
> > >> > for local state storage in operators is still the "Heap Backend".
> > >> >
> > >> > With the proposed FLIP, a user would do a `setCheckpointStorage(new
> > >> > FsStorage(<path>))`. Which makes it obvious that they're changing
> > where
> > >> > checkpoints are stored but not the actual "State Backend", which is
> > >> > still "Heap Backend" (the default).
> > >> >
> > >> > I do understand Yu's point, though, that this will be confusing for
> > >> > current Flink users. They are used to setting a "State Backend"
> > if/when
> > >> > they want to change the storage location. To fit the new model they
> > >> > would have to change the call from `setStateBackend()` to
> > >> > `setCheckpointStorage()`.
> > >> >
> > >> > I think we need to life with this short-term confusion because in
> the
> > >> > long run the proposed split between checkpoint location and state
> > >> > backend makes sense and will be more straightforward for users to
> > >> > understand.
> > >> >
> > >> > Best,
> > >> > Aljoscha
> > >> >
> > >> >
> > >>
> > >
> >
>

Re: [DISCUSS] FLIP-142: Disentangle StateBackends from Checkpointing

Posted by Seth Wiesman <sj...@gmail.com>.
Hi Yu,

bq* I thought the FLIP aims at resolving some *existing* confusion, i.e.
the durability mystery to users.

I think it might help to highlight specific stumbling blocks users have
today and why I believe this change addresses those issues. Some frequent
things I've heard over the past several years include:

1) "We use RocksDB because we don't need fault tolerance."
2) "We don't use RocksDB because we don't want to manage an external
database."
3) Believing RocksDB is reading and writing directly with S3 or HDFS (vs.
local disk)
4) Believing FsStateBackend spills to disk or has anything to do with the
local filesystem
5) Pointing RocksDB at network-attached storage, believing that the state
backend needs to be fault-tolerant

This question from the ml is very representative of where users are
struggling[1]. Many of these questions were not from new users but from
organizations that were in production! Just yesterday I was on the phone
with a company that didn't realize they were in production without
checkpointing; honestly, you would be shocked how often this happens. The
current state backend abstraction is to complex for many of our users. What
all these questions have in common is misunderstanding the relationship
between how data is stored locally on TMs vs how checkpoints make that
state durable.

The FLIP aims actively help users by allowing them to reason about state
backends separately from checkpoint durability. In the future, a state
backend only defines where and how state is stored locally on the TM while
checkpoint storage defines where and how checkpoints are stored for
recovery. To be concrete I think the JavaDoc for setCheckpointStorage would
be something like:

```java
/**
 * CheckpointStorage defines how checkpoint snapshots are persisted for
fault tolerance
*. Various implementations  store their checkpoints in different fashions
and have different requirements and
 * availability guarantees.
 *
 *<p>For example, JobManagerCheckpointStorage stores checkpoints in the
memory of the JobManager.
 * It is lightweight and without additional dependencies but is not highly
available
 * and only supports small state sizes. This checkpoint storage policy is
convenient for
 * local testing and development.
 *
 *<p>FileSystemCheckpointStorage stores checkpoints in a filesystem. For
systems like
 * HDFS, NFS Drives, S3, and GCS, this storage policy supports large state
size,
 * in the magnitude of many terabytes while providing a highly available
foundation
 * for stateful applications. This checkpoint storage policy is recommended
for most
 * production deployments.
 */
void setCheckpointStorage(CheckpointStorage storage) {}
```

Seth

[1]
http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/State-Storage-Questions-td37919.html
[2] Also naming, but we're aligned here

On Thu, Sep 24, 2020 at 10:24 AM Yu Li <ca...@gmail.com> wrote:

> And to make it clear, I'm +1 on the idea of decoupling state backends with
> checkpointing. I don't have any question about making it clear that
> heap/RocksDB is where we serve the routine state read/write and where to
> put the checkpoint data is another story. My only concern lies in the newly
> introduced setCheckpointStorage API and how we define its semantics, and
> not sure whether it's due to my ignorance.
>
> Best Regards,
> Yu
>
>
> On Thu, 24 Sep 2020 at 23:11, Yu Li <ca...@gmail.com> wrote:
>
> > *bq. What new confusion would be introduced here?*
> > No *new* confusion introduced, but as mentioned at the very beginning of
> > the motivation ("Apache Flink's durability story is a mystery to many
> > users"), I thought the FLIP aims at resolving some *existing*
> > confusions, i.e. the durability mystery to users.
> >
> > For me, I'm not 100% clear about how to write the javadoc of the
> > setCheckpointStorage API. Would it be like "specify where the checkpoint
> > data is stored"? If so, do we need to explain the fact that when a
> > checkpoint path is given, JM will also persist the checkpoint data to
> DFS?
> > It's true that such confusion also exists today, but would the
> introduction
> > of the new API expose it further?
> >
> > IMHO we need to document the newly introduced API / classes and their
> > semantics clearly in the FLIP to make sure everyone is on the same page,
> > but if we feel such work / discussions are all details and only need to
> > happen at the documenting and release note phase, it's also fine to me.
> >
> > And if I'm the only one who has such questions / concerns on the new
> > `setCheckpointStorage` API and most of others feel its semantic is sound
> > and clear, then please just ignore me and move on.
> >
> > Thanks.
> >
> > Best Regards,
> > Yu
> >
> >
> > On Wed, 23 Sep 2020 at 17:08, Stephan Ewen <se...@apache.org> wrote:
> >
> >> I am confused now with the concerns here. This is very much from the
> user
> >> perspective (which is partially also the developer perspective which is
> >> the
> >> sign of an intuitive abstraction).
> >>
> >> Of course, there will be docs describing what JMCheckpointStorage and
> >> FsCheckpointStorage are.
> >> And having release notes that describe that
> >> RocksDBStateBackend("s3://...")
> >> now corresponds to a combination of "RocksDBBackend" and
> >> "FsCheckpointStorage" is also straightforward.
> >>
> >> We said to keep the old RocksDBStateBackend class and let it implement
> >> both
> >> interfaces such that the old code still works exactly as before.
> >>
> >> What new confusion would be introduced here?
> >> Understanding the difference between JMCheckpointStorage and
> >> FsCheckpointStorage was always necessary when one needed to understand
> the
> >> difference between MemoryStateBackend and FsStateBackend. It should be
> >> easier to define this after this change, because it is the only thing
> that
> >> we describe when explaining what checkpoint storage to use (rather than
> >> also having the choice of index structure coupled to that).
> >>
> >>
> >> On Wed, Sep 23, 2020 at 10:39 AM Aljoscha Krettek <al...@apache.org>
> >> wrote:
> >>
> >> > On 23.09.20 04:40, Yu Li wrote:
> >> > > To be specific, with the old API users don't need to set checkpoint
> >> > > storage, instead they only need to pass the checkpoint path w/o
> caring
> >> > > about the storage. The new APIs are forcing users to set the storage
> >> so
> >> > > they have to know the difference between different storages. It's
> not
> >> an
> >> > > implementation change, but an API change that users have to
> understand
> >> > and
> >> > > follow-up.
> >> >
> >> > I think the main point of the FLIP is to make it more obvious to users
> >> > what is happening.
> >> >
> >> > With current Flink, they would do a `setStateBackend(new
> >> > FsStateBackend(<path>))`. What the user is actually "saying" with this
> >> > is: I want to keep state on heap but store checkpoints in DFS. They
> are
> >> > not actually changing the "State Backend", the thing that keeps state
> in
> >> > operators, but only where state is checkpointed. The thing that is
> used
> >> > for local state storage in operators is still the "Heap Backend".
> >> >
> >> > With the proposed FLIP, a user would do a `setCheckpointStorage(new
> >> > FsStorage(<path>))`. Which makes it obvious that they're changing
> where
> >> > checkpoints are stored but not the actual "State Backend", which is
> >> > still "Heap Backend" (the default).
> >> >
> >> > I do understand Yu's point, though, that this will be confusing for
> >> > current Flink users. They are used to setting a "State Backend"
> if/when
> >> > they want to change the storage location. To fit the new model they
> >> > would have to change the call from `setStateBackend()` to
> >> > `setCheckpointStorage()`.
> >> >
> >> > I think we need to life with this short-term confusion because in the
> >> > long run the proposed split between checkpoint location and state
> >> > backend makes sense and will be more straightforward for users to
> >> > understand.
> >> >
> >> > Best,
> >> > Aljoscha
> >> >
> >> >
> >>
> >
>

Re: [DISCUSS] FLIP-142: Disentangle StateBackends from Checkpointing

Posted by Yu Li <ca...@gmail.com>.
And to make it clear, I'm +1 on the idea of decoupling state backends with
checkpointing. I don't have any question about making it clear that
heap/RocksDB is where we serve the routine state read/write and where to
put the checkpoint data is another story. My only concern lies in the newly
introduced setCheckpointStorage API and how we define its semantics, and
not sure whether it's due to my ignorance.

Best Regards,
Yu


On Thu, 24 Sep 2020 at 23:11, Yu Li <ca...@gmail.com> wrote:

> *bq. What new confusion would be introduced here?*
> No *new* confusion introduced, but as mentioned at the very beginning of
> the motivation ("Apache Flink's durability story is a mystery to many
> users"), I thought the FLIP aims at resolving some *existing*
> confusions, i.e. the durability mystery to users.
>
> For me, I'm not 100% clear about how to write the javadoc of the
> setCheckpointStorage API. Would it be like "specify where the checkpoint
> data is stored"? If so, do we need to explain the fact that when a
> checkpoint path is given, JM will also persist the checkpoint data to DFS?
> It's true that such confusion also exists today, but would the introduction
> of the new API expose it further?
>
> IMHO we need to document the newly introduced API / classes and their
> semantics clearly in the FLIP to make sure everyone is on the same page,
> but if we feel such work / discussions are all details and only need to
> happen at the documenting and release note phase, it's also fine to me.
>
> And if I'm the only one who has such questions / concerns on the new
> `setCheckpointStorage` API and most of others feel its semantic is sound
> and clear, then please just ignore me and move on.
>
> Thanks.
>
> Best Regards,
> Yu
>
>
> On Wed, 23 Sep 2020 at 17:08, Stephan Ewen <se...@apache.org> wrote:
>
>> I am confused now with the concerns here. This is very much from the user
>> perspective (which is partially also the developer perspective which is
>> the
>> sign of an intuitive abstraction).
>>
>> Of course, there will be docs describing what JMCheckpointStorage and
>> FsCheckpointStorage are.
>> And having release notes that describe that
>> RocksDBStateBackend("s3://...")
>> now corresponds to a combination of "RocksDBBackend" and
>> "FsCheckpointStorage" is also straightforward.
>>
>> We said to keep the old RocksDBStateBackend class and let it implement
>> both
>> interfaces such that the old code still works exactly as before.
>>
>> What new confusion would be introduced here?
>> Understanding the difference between JMCheckpointStorage and
>> FsCheckpointStorage was always necessary when one needed to understand the
>> difference between MemoryStateBackend and FsStateBackend. It should be
>> easier to define this after this change, because it is the only thing that
>> we describe when explaining what checkpoint storage to use (rather than
>> also having the choice of index structure coupled to that).
>>
>>
>> On Wed, Sep 23, 2020 at 10:39 AM Aljoscha Krettek <al...@apache.org>
>> wrote:
>>
>> > On 23.09.20 04:40, Yu Li wrote:
>> > > To be specific, with the old API users don't need to set checkpoint
>> > > storage, instead they only need to pass the checkpoint path w/o caring
>> > > about the storage. The new APIs are forcing users to set the storage
>> so
>> > > they have to know the difference between different storages. It's not
>> an
>> > > implementation change, but an API change that users have to understand
>> > and
>> > > follow-up.
>> >
>> > I think the main point of the FLIP is to make it more obvious to users
>> > what is happening.
>> >
>> > With current Flink, they would do a `setStateBackend(new
>> > FsStateBackend(<path>))`. What the user is actually "saying" with this
>> > is: I want to keep state on heap but store checkpoints in DFS. They are
>> > not actually changing the "State Backend", the thing that keeps state in
>> > operators, but only where state is checkpointed. The thing that is used
>> > for local state storage in operators is still the "Heap Backend".
>> >
>> > With the proposed FLIP, a user would do a `setCheckpointStorage(new
>> > FsStorage(<path>))`. Which makes it obvious that they're changing where
>> > checkpoints are stored but not the actual "State Backend", which is
>> > still "Heap Backend" (the default).
>> >
>> > I do understand Yu's point, though, that this will be confusing for
>> > current Flink users. They are used to setting a "State Backend" if/when
>> > they want to change the storage location. To fit the new model they
>> > would have to change the call from `setStateBackend()` to
>> > `setCheckpointStorage()`.
>> >
>> > I think we need to life with this short-term confusion because in the
>> > long run the proposed split between checkpoint location and state
>> > backend makes sense and will be more straightforward for users to
>> > understand.
>> >
>> > Best,
>> > Aljoscha
>> >
>> >
>>
>

Re: [DISCUSS] FLIP-142: Disentangle StateBackends from Checkpointing

Posted by Yu Li <ca...@gmail.com>.
*bq. What new confusion would be introduced here?*
No *new* confusion introduced, but as mentioned at the very beginning of
the motivation ("Apache Flink's durability story is a mystery to many
users"), I thought the FLIP aims at resolving some *existing*
confusions, i.e. the durability mystery to users.

For me, I'm not 100% clear about how to write the javadoc of the
setCheckpointStorage API. Would it be like "specify where the checkpoint
data is stored"? If so, do we need to explain the fact that when a
checkpoint path is given, JM will also persist the checkpoint data to DFS?
It's true that such confusion also exists today, but would the introduction
of the new API expose it further?

IMHO we need to document the newly introduced API / classes and their
semantics clearly in the FLIP to make sure everyone is on the same page,
but if we feel such work / discussions are all details and only need to
happen at the documenting and release note phase, it's also fine to me.

And if I'm the only one who has such questions / concerns on the new
`setCheckpointStorage` API and most of others feel its semantic is sound
and clear, then please just ignore me and move on.

Thanks.

Best Regards,
Yu


On Wed, 23 Sep 2020 at 17:08, Stephan Ewen <se...@apache.org> wrote:

> I am confused now with the concerns here. This is very much from the user
> perspective (which is partially also the developer perspective which is the
> sign of an intuitive abstraction).
>
> Of course, there will be docs describing what JMCheckpointStorage and
> FsCheckpointStorage are.
> And having release notes that describe that RocksDBStateBackend("s3://...")
> now corresponds to a combination of "RocksDBBackend" and
> "FsCheckpointStorage" is also straightforward.
>
> We said to keep the old RocksDBStateBackend class and let it implement both
> interfaces such that the old code still works exactly as before.
>
> What new confusion would be introduced here?
> Understanding the difference between JMCheckpointStorage and
> FsCheckpointStorage was always necessary when one needed to understand the
> difference between MemoryStateBackend and FsStateBackend. It should be
> easier to define this after this change, because it is the only thing that
> we describe when explaining what checkpoint storage to use (rather than
> also having the choice of index structure coupled to that).
>
>
> On Wed, Sep 23, 2020 at 10:39 AM Aljoscha Krettek <al...@apache.org>
> wrote:
>
> > On 23.09.20 04:40, Yu Li wrote:
> > > To be specific, with the old API users don't need to set checkpoint
> > > storage, instead they only need to pass the checkpoint path w/o caring
> > > about the storage. The new APIs are forcing users to set the storage so
> > > they have to know the difference between different storages. It's not
> an
> > > implementation change, but an API change that users have to understand
> > and
> > > follow-up.
> >
> > I think the main point of the FLIP is to make it more obvious to users
> > what is happening.
> >
> > With current Flink, they would do a `setStateBackend(new
> > FsStateBackend(<path>))`. What the user is actually "saying" with this
> > is: I want to keep state on heap but store checkpoints in DFS. They are
> > not actually changing the "State Backend", the thing that keeps state in
> > operators, but only where state is checkpointed. The thing that is used
> > for local state storage in operators is still the "Heap Backend".
> >
> > With the proposed FLIP, a user would do a `setCheckpointStorage(new
> > FsStorage(<path>))`. Which makes it obvious that they're changing where
> > checkpoints are stored but not the actual "State Backend", which is
> > still "Heap Backend" (the default).
> >
> > I do understand Yu's point, though, that this will be confusing for
> > current Flink users. They are used to setting a "State Backend" if/when
> > they want to change the storage location. To fit the new model they
> > would have to change the call from `setStateBackend()` to
> > `setCheckpointStorage()`.
> >
> > I think we need to life with this short-term confusion because in the
> > long run the proposed split between checkpoint location and state
> > backend makes sense and will be more straightforward for users to
> > understand.
> >
> > Best,
> > Aljoscha
> >
> >
>

Re: [DISCUSS] FLIP-142: Disentangle StateBackends from Checkpointing

Posted by Stephan Ewen <se...@apache.org>.
I am confused now with the concerns here. This is very much from the user
perspective (which is partially also the developer perspective which is the
sign of an intuitive abstraction).

Of course, there will be docs describing what JMCheckpointStorage and
FsCheckpointStorage are.
And having release notes that describe that RocksDBStateBackend("s3://...")
now corresponds to a combination of "RocksDBBackend" and
"FsCheckpointStorage" is also straightforward.

We said to keep the old RocksDBStateBackend class and let it implement both
interfaces such that the old code still works exactly as before.

What new confusion would be introduced here?
Understanding the difference between JMCheckpointStorage and
FsCheckpointStorage was always necessary when one needed to understand the
difference between MemoryStateBackend and FsStateBackend. It should be
easier to define this after this change, because it is the only thing that
we describe when explaining what checkpoint storage to use (rather than
also having the choice of index structure coupled to that).


On Wed, Sep 23, 2020 at 10:39 AM Aljoscha Krettek <al...@apache.org>
wrote:

> On 23.09.20 04:40, Yu Li wrote:
> > To be specific, with the old API users don't need to set checkpoint
> > storage, instead they only need to pass the checkpoint path w/o caring
> > about the storage. The new APIs are forcing users to set the storage so
> > they have to know the difference between different storages. It's not an
> > implementation change, but an API change that users have to understand
> and
> > follow-up.
>
> I think the main point of the FLIP is to make it more obvious to users
> what is happening.
>
> With current Flink, they would do a `setStateBackend(new
> FsStateBackend(<path>))`. What the user is actually "saying" with this
> is: I want to keep state on heap but store checkpoints in DFS. They are
> not actually changing the "State Backend", the thing that keeps state in
> operators, but only where state is checkpointed. The thing that is used
> for local state storage in operators is still the "Heap Backend".
>
> With the proposed FLIP, a user would do a `setCheckpointStorage(new
> FsStorage(<path>))`. Which makes it obvious that they're changing where
> checkpoints are stored but not the actual "State Backend", which is
> still "Heap Backend" (the default).
>
> I do understand Yu's point, though, that this will be confusing for
> current Flink users. They are used to setting a "State Backend" if/when
> they want to change the storage location. To fit the new model they
> would have to change the call from `setStateBackend()` to
> `setCheckpointStorage()`.
>
> I think we need to life with this short-term confusion because in the
> long run the proposed split between checkpoint location and state
> backend makes sense and will be more straightforward for users to
> understand.
>
> Best,
> Aljoscha
>
>

Re: [DISCUSS] FLIP-142: Disentangle StateBackends from Checkpointing

Posted by Aljoscha Krettek <al...@apache.org>.
On 23.09.20 04:40, Yu Li wrote:
> To be specific, with the old API users don't need to set checkpoint
> storage, instead they only need to pass the checkpoint path w/o caring
> about the storage. The new APIs are forcing users to set the storage so
> they have to know the difference between different storages. It's not an
> implementation change, but an API change that users have to understand and
> follow-up.

I think the main point of the FLIP is to make it more obvious to users 
what is happening.

With current Flink, they would do a `setStateBackend(new 
FsStateBackend(<path>))`. What the user is actually "saying" with this 
is: I want to keep state on heap but store checkpoints in DFS. They are 
not actually changing the "State Backend", the thing that keeps state in 
operators, but only where state is checkpointed. The thing that is used 
for local state storage in operators is still the "Heap Backend".

With the proposed FLIP, a user would do a `setCheckpointStorage(new 
FsStorage(<path>))`. Which makes it obvious that they're changing where 
checkpoints are stored but not the actual "State Backend", which is 
still "Heap Backend" (the default).

I do understand Yu's point, though, that this will be confusing for 
current Flink users. They are used to setting a "State Backend" if/when 
they want to change the storage location. To fit the new model they 
would have to change the call from `setStateBackend()` to 
`setCheckpointStorage()`.

I think we need to life with this short-term confusion because in the 
long run the proposed split between checkpoint location and state 
backend makes sense and will be more straightforward for users to 
understand.

Best,
Aljoscha


Re: [DISCUSS] FLIP-142: Disentangle StateBackends from Checkpointing

Posted by Yu Li <ca...@gmail.com>.
*bq. To restate the motivation of this FLIP, the issue we are trying to
solve is that users do not understand how to choose between the different
state backends today.*
Yes, but my point is that now users may not understand how to choose
between different checkpoint storages. We cannot resolve one problem by
introducing another one, right? And it's true that users probably also
don't understand such differences today, but isn't this something we are
aiming to resolve?

*bq. This design is based on having spent a lot of time working directly
with many different Flink users over the past several years.*
I'm not saying we shouldn't have this FLIP, but suggesting to state clearly
about the difference between different checkpoint storages and how to
choose them.

*bq. This document does not address each checkpoint storage option's pros
and cons because it does not introduce any new runtime implementations. The
pros and cons are the same as those when choosing between
MemoryStateBackendCheckpointStorage and FsCheckpointStorage today.*
TBH, I think this statement is purely from the developer's view and lack of
thoughts for users. The major difference is that in the FLIP we are
*exposing* the checkpoint storage concept to our end users through a public
API, it's a new API and I don't think it's a good idea to introduce any new
API w/o explicit semantics.

To be specific, with the old API users don't need to set checkpoint
storage, instead they only need to pass the checkpoint path w/o caring
about the storage. The new APIs are forcing users to set the storage so
they have to know the difference between different storages. It's not an
implementation change, but an API change that users have to understand and
follow-up.

Best Regards,
Yu


On Tue, 22 Sep 2020 at 23:18, Seth Wiesman <sj...@gmail.com> wrote:

> Hi Yu,
>
> To restate the motivation of this FLIP, the issue we are trying to solve is
> that users do not understand how to choose between the different state
> backends today. The goal is to decouple checkpoint storage from local state
> storage so users can reason about these configurations separately. This
> design is based on having spent a lot of time working directly with many
> different Flink users over the past several years.
>
> This document does not address each checkpoint storage option's pros and
> cons because it does not introduce any new runtime implementations. The
> pros and cons are the same as those when choosing between
> MemoryStateBackendCheckpointStorage and FsCheckpointStorage today. Of
> course, the documentation and JavaDoc will be updated with the new user API
> to address these questions, but that is an implementation detail, separate
> from discussing how interfaces should look.
>
> Similarly, the migration section states we will guide users on updating
> their code (both in release notes and deprecation notes in code). The
> migration section of the FLIP lists the updates a user will need to make
> for each configuration.
>
> Seth
>
> On Tue, Sep 22, 2020 at 12:29 AM Yu Li <ca...@gmail.com> wrote:
>
> > It seems my questions are misunderstood to be about details on
> > implementations, but actually my concerns are from the users' view,
> > especially how to understand the new APIs and how to choose/use them.
> >
> > To better express my concern, let's list the new APIs this FLIP proposes
> to
> > introduce:
> >
> >
> > *env.getCheckpointConfig().setCheckpointStorage(new
> >
> >
> JobManagerCheckpointStorage());env.getCheckpointConfig().setCheckpointStorage(new
> >
> >
> FileSystemCheckpointStorage("s3://checkpoints"));env.getCheckpointConfig().setCheckpointStorage("s3://checkpoints");*
> >
> > From the end user's perspective, I will probably ask the below questions:
> >
> > - As a new Flink user:
> >   a) What does it mean when setting the checkpoint storage to
> > `JobManagerCheckpointStorage`? And what's the difference between
> > `JobManagerCheckpointStorage` and `FileSystemCheckpointStorage`?
> >   b) How should I choose between these two?
> >   c) Where the checkpoint data will be stored when using different
> > checkpoint storage? (As mentioned in the motivation of this FLIP:
> "Flink's
> > durability story is a mystery to many users", how the proposal resolves
> > it?)
> >
> > - As an old user:
> >   a) How should I migrate my old codes to the new API? For example, if
> now
> > I'm using `*new RocksDBStateBackend("s3://checkpoints")*`, which
> checkpoint
> > storage should I choose with the new API? Could I live w/o setting the
> > checkpoint storage? If so, which one will it use by default?
> >
> > The old API is hiding something from the end user, and although I totally
> > agree it's better to improve it, I believe we need to figure out answers
> to
> > the above questions. Or if I have misunderstood anything, please let me
> > know. Thanks.
> >
> > Best Regards,
> > Yu
> >
> >
> > On Tue, 22 Sep 2020 at 02:42, Stephan Ewen <se...@apache.org> wrote:
> >
> > > To me, the simplifications made by Seth sound good and do make a lot of
> > > sense.
> > > We should really break this down to a few orthogonal guides, then it is
> > > easy for users:
> > >
> > >   - Metadata always goes through the JobManager, no matter what
> > > CheckpointStorage.
> > >   - The JobManagerCheckpointStorage has the option to put the metadata
> > on a
> > > file system, to make it externally accessible/addressable, and for
> master
> > > failover (HA).
> > >   - The file state size threshold is the threshold where data is stored
> > > inline with the metadata, rather than in a separate file. Whatever the
> > > JobManager does with metadata, it is orthogonal.
> > >
> > >
> > >
> > >
> > > On Mon, Sep 21, 2020 at 3:59 PM Seth Wiesman <sj...@gmail.com>
> > wrote:
> > >
> > > > Hi Yu,
> > > >
> > > > Let me address your comments one at a time.
> > > >
> > > > I think I can address comments one and two with a single answer. This
> > > FLIP
> > > > does not change any runtime data structures or implementations. As
> > such,
> > > it
> > > > only provides new user-facing factory classes for those components.
> > > > StateBackend (the interface) is effectively a factory for both
> > checkpoint
> > > > storage and state backends today and I want to move the checkpoint
> > > storage
> > > > methods to a new factory interface CheckpointStorage.
> > > >
> > > > What this means is JobManagerCheckpointStorage is a factory for
> > > > MemoryBackendCheckpointStorage and FileSystemCheckpointStorage is a
> > > factory
> > > > for FsCheckpointStorage[1]. All configurations from these classes
> will
> > be
> > > > made available via the new public APIs. I will make that clear in the
> > > > document. The semantics of how and where they checkpoint will be
> > dictated
> > > > by these runtime classes whose implementations are not to be changed.
> > > >
> > > > Regarding three, yes, a part of any FLIP is to update the
> documentation
> > > and
> > > > as such, I am not going to explicitly outline it in the document.
> > > >
> > > > [1] We might want to rename MemoryBackendCheckpointStorage and
> > > > FsCheckpointStorage but these are internal classes and as such that
> > > > discussion does not need to be a part of the FLIP process.
> > > >
> > > > On Mon, Sep 21, 2020 at 2:40 AM Yu Li <ca...@gmail.com> wrote:
> > > >
> > > > > 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 <sj...@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 <ca...@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 <
> sjwiesman@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 <
> sewen@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 <
> > > > sjwiesman@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 <ca...@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 <
> > > > > > qcx978132955@gmail.com
> > > > > > > >
> > > > > > > > > > wrote:
> > > > > > > > > > >
> > > > > > > > > > > > Thanks for the detailed replay, +1 from my side.
> > > > > > > > > > > > Best,
> > > > > > > > > > > > Congxian
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > > Seth Wiesman <sj...@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 <
> > > > > > > sewen@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 <
> > > > > > > sewen@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
> > <
> > > > > > > > > > > > aljoscha@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
> > <
> > > > > > > > > > > > > qcx978132955@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 <sj...@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 <
> > > > > > > > > > > > > > >> dwysakowicz@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 <
> > > > > > > > > > > > > > >> >>> dwysakowicz@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 <
> > > > > > > > > > > > > > sjwiesman@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 <
> > aljoscha@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
> > > > > > > <
> > > > > > > > > > > > > > >> >>> aljoscha@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
> > > > > > <
> > > > > > > > > > > > > > >> >> sjwiesman@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
> > > > > > > > > > > > > > >> >>>>>>>>>>>>
> > > > > > > > > > > > > > >> >>>>>>>>>>>
> > > > > > > > > > > > > > >> >>>>>>>>>>
> > > > > > > > > > > > > > >> >>>>>>
> > > > > > > > > > > > > > >> >>>>
> > > > > > > > > > > > > > >> >>>>
> > > > > > > > > > > > > > >> >>>
> > > > > > > > > > > > > > >> >>
> > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > >>
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] FLIP-142: Disentangle StateBackends from Checkpointing

Posted by Seth Wiesman <sj...@gmail.com>.
Hi Yu,

To restate the motivation of this FLIP, the issue we are trying to solve is
that users do not understand how to choose between the different state
backends today. The goal is to decouple checkpoint storage from local state
storage so users can reason about these configurations separately. This
design is based on having spent a lot of time working directly with many
different Flink users over the past several years.

This document does not address each checkpoint storage option's pros and
cons because it does not introduce any new runtime implementations. The
pros and cons are the same as those when choosing between
MemoryStateBackendCheckpointStorage and FsCheckpointStorage today. Of
course, the documentation and JavaDoc will be updated with the new user API
to address these questions, but that is an implementation detail, separate
from discussing how interfaces should look.

Similarly, the migration section states we will guide users on updating
their code (both in release notes and deprecation notes in code). The
migration section of the FLIP lists the updates a user will need to make
for each configuration.

Seth

On Tue, Sep 22, 2020 at 12:29 AM Yu Li <ca...@gmail.com> wrote:

> It seems my questions are misunderstood to be about details on
> implementations, but actually my concerns are from the users' view,
> especially how to understand the new APIs and how to choose/use them.
>
> To better express my concern, let's list the new APIs this FLIP proposes to
> introduce:
>
>
> *env.getCheckpointConfig().setCheckpointStorage(new
>
> JobManagerCheckpointStorage());env.getCheckpointConfig().setCheckpointStorage(new
>
> FileSystemCheckpointStorage("s3://checkpoints"));env.getCheckpointConfig().setCheckpointStorage("s3://checkpoints");*
>
> From the end user's perspective, I will probably ask the below questions:
>
> - As a new Flink user:
>   a) What does it mean when setting the checkpoint storage to
> `JobManagerCheckpointStorage`? And what's the difference between
> `JobManagerCheckpointStorage` and `FileSystemCheckpointStorage`?
>   b) How should I choose between these two?
>   c) Where the checkpoint data will be stored when using different
> checkpoint storage? (As mentioned in the motivation of this FLIP: "Flink's
> durability story is a mystery to many users", how the proposal resolves
> it?)
>
> - As an old user:
>   a) How should I migrate my old codes to the new API? For example, if now
> I'm using `*new RocksDBStateBackend("s3://checkpoints")*`, which checkpoint
> storage should I choose with the new API? Could I live w/o setting the
> checkpoint storage? If so, which one will it use by default?
>
> The old API is hiding something from the end user, and although I totally
> agree it's better to improve it, I believe we need to figure out answers to
> the above questions. Or if I have misunderstood anything, please let me
> know. Thanks.
>
> Best Regards,
> Yu
>
>
> On Tue, 22 Sep 2020 at 02:42, Stephan Ewen <se...@apache.org> wrote:
>
> > To me, the simplifications made by Seth sound good and do make a lot of
> > sense.
> > We should really break this down to a few orthogonal guides, then it is
> > easy for users:
> >
> >   - Metadata always goes through the JobManager, no matter what
> > CheckpointStorage.
> >   - The JobManagerCheckpointStorage has the option to put the metadata
> on a
> > file system, to make it externally accessible/addressable, and for master
> > failover (HA).
> >   - The file state size threshold is the threshold where data is stored
> > inline with the metadata, rather than in a separate file. Whatever the
> > JobManager does with metadata, it is orthogonal.
> >
> >
> >
> >
> > On Mon, Sep 21, 2020 at 3:59 PM Seth Wiesman <sj...@gmail.com>
> wrote:
> >
> > > Hi Yu,
> > >
> > > Let me address your comments one at a time.
> > >
> > > I think I can address comments one and two with a single answer. This
> > FLIP
> > > does not change any runtime data structures or implementations. As
> such,
> > it
> > > only provides new user-facing factory classes for those components.
> > > StateBackend (the interface) is effectively a factory for both
> checkpoint
> > > storage and state backends today and I want to move the checkpoint
> > storage
> > > methods to a new factory interface CheckpointStorage.
> > >
> > > What this means is JobManagerCheckpointStorage is a factory for
> > > MemoryBackendCheckpointStorage and FileSystemCheckpointStorage is a
> > factory
> > > for FsCheckpointStorage[1]. All configurations from these classes will
> be
> > > made available via the new public APIs. I will make that clear in the
> > > document. The semantics of how and where they checkpoint will be
> dictated
> > > by these runtime classes whose implementations are not to be changed.
> > >
> > > Regarding three, yes, a part of any FLIP is to update the documentation
> > and
> > > as such, I am not going to explicitly outline it in the document.
> > >
> > > [1] We might want to rename MemoryBackendCheckpointStorage and
> > > FsCheckpointStorage but these are internal classes and as such that
> > > discussion does not need to be a part of the FLIP process.
> > >
> > > On Mon, Sep 21, 2020 at 2:40 AM Yu Li <ca...@gmail.com> wrote:
> > >
> > > > 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 <sj...@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 <ca...@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 <sj...@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 <sewen@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 <
> > > sjwiesman@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 <ca...@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 <
> > > > > qcx978132955@gmail.com
> > > > > > >
> > > > > > > > > wrote:
> > > > > > > > > >
> > > > > > > > > > > Thanks for the detailed replay, +1 from my side.
> > > > > > > > > > > Best,
> > > > > > > > > > > Congxian
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > Seth Wiesman <sj...@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 <
> > > > > > sewen@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 <
> > > > > > sewen@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
> <
> > > > > > > > > > > aljoscha@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
> <
> > > > > > > > > > > > qcx978132955@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 <sj...@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 <
> > > > > > > > > > > > > >> dwysakowicz@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 <
> > > > > > > > > > > > > >> >>> dwysakowicz@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 <
> > > > > > > > > > > > > sjwiesman@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 <
> aljoscha@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
> > > > > > <
> > > > > > > > > > > > > >> >>> aljoscha@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
> > > > > <
> > > > > > > > > > > > > >> >> sjwiesman@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
> > > > > > > > > > > > > >> >>>>>>>>>>>>
> > > > > > > > > > > > > >> >>>>>>>>>>>
> > > > > > > > > > > > > >> >>>>>>>>>>
> > > > > > > > > > > > > >> >>>>>>
> > > > > > > > > > > > > >> >>>>
> > > > > > > > > > > > > >> >>>>
> > > > > > > > > > > > > >> >>>
> > > > > > > > > > > > > >> >>
> > > > > > > > > > > > > >> >
> > > > > > > > > > > > > >>
> > > > > > > > > > > > > >>
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] FLIP-142: Disentangle StateBackends from Checkpointing

Posted by Yu Li <ca...@gmail.com>.
It seems my questions are misunderstood to be about details on
implementations, but actually my concerns are from the users' view,
especially how to understand the new APIs and how to choose/use them.

To better express my concern, let's list the new APIs this FLIP proposes to
introduce:


*env.getCheckpointConfig().setCheckpointStorage(new
JobManagerCheckpointStorage());env.getCheckpointConfig().setCheckpointStorage(new
FileSystemCheckpointStorage("s3://checkpoints"));env.getCheckpointConfig().setCheckpointStorage("s3://checkpoints");*

From the end user's perspective, I will probably ask the below questions:

- As a new Flink user:
  a) What does it mean when setting the checkpoint storage to
`JobManagerCheckpointStorage`? And what's the difference between
`JobManagerCheckpointStorage` and `FileSystemCheckpointStorage`?
  b) How should I choose between these two?
  c) Where the checkpoint data will be stored when using different
checkpoint storage? (As mentioned in the motivation of this FLIP: "Flink's
durability story is a mystery to many users", how the proposal resolves it?)

- As an old user:
  a) How should I migrate my old codes to the new API? For example, if now
I'm using `*new RocksDBStateBackend("s3://checkpoints")*`, which checkpoint
storage should I choose with the new API? Could I live w/o setting the
checkpoint storage? If so, which one will it use by default?

The old API is hiding something from the end user, and although I totally
agree it's better to improve it, I believe we need to figure out answers to
the above questions. Or if I have misunderstood anything, please let me
know. Thanks.

Best Regards,
Yu


On Tue, 22 Sep 2020 at 02:42, Stephan Ewen <se...@apache.org> wrote:

> To me, the simplifications made by Seth sound good and do make a lot of
> sense.
> We should really break this down to a few orthogonal guides, then it is
> easy for users:
>
>   - Metadata always goes through the JobManager, no matter what
> CheckpointStorage.
>   - The JobManagerCheckpointStorage has the option to put the metadata on a
> file system, to make it externally accessible/addressable, and for master
> failover (HA).
>   - The file state size threshold is the threshold where data is stored
> inline with the metadata, rather than in a separate file. Whatever the
> JobManager does with metadata, it is orthogonal.
>
>
>
>
> On Mon, Sep 21, 2020 at 3:59 PM Seth Wiesman <sj...@gmail.com> wrote:
>
> > Hi Yu,
> >
> > Let me address your comments one at a time.
> >
> > I think I can address comments one and two with a single answer. This
> FLIP
> > does not change any runtime data structures or implementations. As such,
> it
> > only provides new user-facing factory classes for those components.
> > StateBackend (the interface) is effectively a factory for both checkpoint
> > storage and state backends today and I want to move the checkpoint
> storage
> > methods to a new factory interface CheckpointStorage.
> >
> > What this means is JobManagerCheckpointStorage is a factory for
> > MemoryBackendCheckpointStorage and FileSystemCheckpointStorage is a
> factory
> > for FsCheckpointStorage[1]. All configurations from these classes will be
> > made available via the new public APIs. I will make that clear in the
> > document. The semantics of how and where they checkpoint will be dictated
> > by these runtime classes whose implementations are not to be changed.
> >
> > Regarding three, yes, a part of any FLIP is to update the documentation
> and
> > as such, I am not going to explicitly outline it in the document.
> >
> > [1] We might want to rename MemoryBackendCheckpointStorage and
> > FsCheckpointStorage but these are internal classes and as such that
> > discussion does not need to be a part of the FLIP process.
> >
> > On Mon, Sep 21, 2020 at 2:40 AM Yu Li <ca...@gmail.com> wrote:
> >
> > > 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 <sj...@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 <ca...@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 <sj...@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 <
> > sjwiesman@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 <ca...@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 <
> > > > qcx978132955@gmail.com
> > > > > >
> > > > > > > > wrote:
> > > > > > > > >
> > > > > > > > > > Thanks for the detailed replay, +1 from my side.
> > > > > > > > > > Best,
> > > > > > > > > > Congxian
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > Seth Wiesman <sj...@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 <
> > > > > sewen@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 <
> > > > > sewen@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 <
> > > > > > > > > > aljoscha@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 <
> > > > > > > > > > > qcx978132955@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 <sj...@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 <
> > > > > > > > > > > > >> dwysakowicz@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 <
> > > > > > > > > > > > >> >>> dwysakowicz@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 <
> > > > > > > > > > > > sjwiesman@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 <al...@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
> > > > > <
> > > > > > > > > > > > >> >>> aljoscha@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
> > > > <
> > > > > > > > > > > > >> >> sjwiesman@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
> > > > > > > > > > > > >> >>>>>>>>>>>>
> > > > > > > > > > > > >> >>>>>>>>>>>
> > > > > > > > > > > > >> >>>>>>>>>>
> > > > > > > > > > > > >> >>>>>>
> > > > > > > > > > > > >> >>>>
> > > > > > > > > > > > >> >>>>
> > > > > > > > > > > > >> >>>
> > > > > > > > > > > > >> >>
> > > > > > > > > > > > >> >
> > > > > > > > > > > > >>
> > > > > > > > > > > > >>
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] FLIP-142: Disentangle StateBackends from Checkpointing

Posted by Stephan Ewen <se...@apache.org>.
To me, the simplifications made by Seth sound good and do make a lot of
sense.
We should really break this down to a few orthogonal guides, then it is
easy for users:

  - Metadata always goes through the JobManager, no matter what
CheckpointStorage.
  - The JobManagerCheckpointStorage has the option to put the metadata on a
file system, to make it externally accessible/addressable, and for master
failover (HA).
  - The file state size threshold is the threshold where data is stored
inline with the metadata, rather than in a separate file. Whatever the
JobManager does with metadata, it is orthogonal.




On Mon, Sep 21, 2020 at 3:59 PM Seth Wiesman <sj...@gmail.com> wrote:

> Hi Yu,
>
> Let me address your comments one at a time.
>
> I think I can address comments one and two with a single answer. This FLIP
> does not change any runtime data structures or implementations. As such, it
> only provides new user-facing factory classes for those components.
> StateBackend (the interface) is effectively a factory for both checkpoint
> storage and state backends today and I want to move the checkpoint storage
> methods to a new factory interface CheckpointStorage.
>
> What this means is JobManagerCheckpointStorage is a factory for
> MemoryBackendCheckpointStorage and FileSystemCheckpointStorage is a factory
> for FsCheckpointStorage[1]. All configurations from these classes will be
> made available via the new public APIs. I will make that clear in the
> document. The semantics of how and where they checkpoint will be dictated
> by these runtime classes whose implementations are not to be changed.
>
> Regarding three, yes, a part of any FLIP is to update the documentation and
> as such, I am not going to explicitly outline it in the document.
>
> [1] We might want to rename MemoryBackendCheckpointStorage and
> FsCheckpointStorage but these are internal classes and as such that
> discussion does not need to be a part of the FLIP process.
>
> On Mon, Sep 21, 2020 at 2:40 AM Yu Li <ca...@gmail.com> wrote:
>
> > 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 <sj...@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 <ca...@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 <sj...@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 <
> sjwiesman@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 <ca...@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 <
> > > qcx978132955@gmail.com
> > > > >
> > > > > > > wrote:
> > > > > > > >
> > > > > > > > > Thanks for the detailed replay, +1 from my side.
> > > > > > > > > Best,
> > > > > > > > > Congxian
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > Seth Wiesman <sj...@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 <
> > > > sewen@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 <
> > > > sewen@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 <
> > > > > > > > > aljoscha@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 <
> > > > > > > > > > qcx978132955@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 <sj...@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 <
> > > > > > > > > > > >> dwysakowicz@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 <
> > > > > > > > > > > >> >>> dwysakowicz@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 <
> > > > > > > > > > > sjwiesman@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 <al...@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
> > > > <
> > > > > > > > > > > >> >>> aljoscha@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
> > > <
> > > > > > > > > > > >> >> sjwiesman@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
> > > > > > > > > > > >> >>>>>>>>>>>>
> > > > > > > > > > > >> >>>>>>>>>>>
> > > > > > > > > > > >> >>>>>>>>>>
> > > > > > > > > > > >> >>>>>>
> > > > > > > > > > > >> >>>>
> > > > > > > > > > > >> >>>>
> > > > > > > > > > > >> >>>
> > > > > > > > > > > >> >>
> > > > > > > > > > > >> >
> > > > > > > > > > > >>
> > > > > > > > > > > >>
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] FLIP-142: Disentangle StateBackends from Checkpointing

Posted by Seth Wiesman <sj...@gmail.com>.
Hi Yu,

Let me address your comments one at a time.

I think I can address comments one and two with a single answer. This FLIP
does not change any runtime data structures or implementations. As such, it
only provides new user-facing factory classes for those components.
StateBackend (the interface) is effectively a factory for both checkpoint
storage and state backends today and I want to move the checkpoint storage
methods to a new factory interface CheckpointStorage.

What this means is JobManagerCheckpointStorage is a factory for
MemoryBackendCheckpointStorage and FileSystemCheckpointStorage is a factory
for FsCheckpointStorage[1]. All configurations from these classes will be
made available via the new public APIs. I will make that clear in the
document. The semantics of how and where they checkpoint will be dictated
by these runtime classes whose implementations are not to be changed.

Regarding three, yes, a part of any FLIP is to update the documentation and
as such, I am not going to explicitly outline it in the document.

[1] We might want to rename MemoryBackendCheckpointStorage and
FsCheckpointStorage but these are internal classes and as such that
discussion does not need to be a part of the FLIP process.

On Mon, Sep 21, 2020 at 2:40 AM Yu Li <ca...@gmail.com> wrote:

> 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 <sj...@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 <ca...@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 <sj...@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 <sj...@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 <ca...@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 <
> > qcx978132955@gmail.com
> > > >
> > > > > > wrote:
> > > > > > >
> > > > > > > > Thanks for the detailed replay, +1 from my side.
> > > > > > > > Best,
> > > > > > > > Congxian
> > > > > > > >
> > > > > > > >
> > > > > > > > Seth Wiesman <sj...@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 <
> > > sewen@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 <
> > > sewen@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 <
> > > > > > > > aljoscha@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 <
> > > > > > > > > qcx978132955@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 <sj...@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 <
> > > > > > > > > > >> dwysakowicz@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 <
> > > > > > > > > > >> >>> dwysakowicz@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 <
> > > > > > > > > > sjwiesman@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 <al...@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
> > > <
> > > > > > > > > > >> >>> aljoscha@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
> > <
> > > > > > > > > > >> >> sjwiesman@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
> > > > > > > > > > >> >>>>>>>>>>>>
> > > > > > > > > > >> >>>>>>>>>>>
> > > > > > > > > > >> >>>>>>>>>>
> > > > > > > > > > >> >>>>>>
> > > > > > > > > > >> >>>>
> > > > > > > > > > >> >>>>
> > > > > > > > > > >> >>>
> > > > > > > > > > >> >>
> > > > > > > > > > >> >
> > > > > > > > > > >>
> > > > > > > > > > >>
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] FLIP-142: Disentangle StateBackends from Checkpointing

Posted by Yu Li <ca...@gmail.com>.
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 <sj...@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 <ca...@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 <sj...@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 <sj...@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 <ca...@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 <
> qcx978132955@gmail.com
> > >
> > > > > wrote:
> > > > > >
> > > > > > > Thanks for the detailed replay, +1 from my side.
> > > > > > > Best,
> > > > > > > Congxian
> > > > > > >
> > > > > > >
> > > > > > > Seth Wiesman <sj...@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 <
> > sewen@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 <
> > sewen@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 <
> > > > > > > aljoscha@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 <
> > > > > > > > qcx978132955@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 <sj...@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 <
> > > > > > > > > >> dwysakowicz@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 <
> > > > > > > > > >> >>> dwysakowicz@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 <
> > > > > > > > > sjwiesman@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 <al...@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
> > <
> > > > > > > > > >> >>> aljoscha@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
> <
> > > > > > > > > >> >> sjwiesman@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
> > > > > > > > > >> >>>>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>>
> > > > > > > > > >> >>>>>>>>>>
> > > > > > > > > >> >>>>>>
> > > > > > > > > >> >>>>
> > > > > > > > > >> >>>>
> > > > > > > > > >> >>>
> > > > > > > > > >> >>
> > > > > > > > > >> >
> > > > > > > > > >>
> > > > > > > > > >>
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] FLIP-142: Disentangle StateBackends from Checkpointing

Posted by Seth Wiesman <sj...@gmail.com>.
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 <ca...@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 <sj...@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 <sj...@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 <ca...@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 <qcx978132955@gmail.com
> >
> > > > wrote:
> > > > >
> > > > > > Thanks for the detailed replay, +1 from my side.
> > > > > > Best,
> > > > > > Congxian
> > > > > >
> > > > > >
> > > > > > Seth Wiesman <sj...@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 <
> sewen@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 <
> sewen@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 <
> > > > > > aljoscha@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 <
> > > > > > > qcx978132955@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 <sj...@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 <
> > > > > > > > >> dwysakowicz@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 <
> > > > > > > > >> >>> dwysakowicz@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 <
> > > > > > > > sjwiesman@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 <al...@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
> <
> > > > > > > > >> >>> aljoscha@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 <
> > > > > > > > >> >> sjwiesman@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
> > > > > > > > >> >>>>>>>>>>>>
> > > > > > > > >> >>>>>>>>>>>
> > > > > > > > >> >>>>>>>>>>
> > > > > > > > >> >>>>>>
> > > > > > > > >> >>>>
> > > > > > > > >> >>>>
> > > > > > > > >> >>>
> > > > > > > > >> >>
> > > > > > > > >> >
> > > > > > > > >>
> > > > > > > > >>
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] FLIP-142: Disentangle StateBackends from Checkpointing

Posted by Yu Li <ca...@gmail.com>.
*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 <sj...@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 <sj...@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 <ca...@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 <qc...@gmail.com>
> > > wrote:
> > > >
> > > > > Thanks for the detailed replay, +1 from my side.
> > > > > Best,
> > > > > Congxian
> > > > >
> > > > >
> > > > > Seth Wiesman <sj...@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 <sewen@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 <
> > > > > aljoscha@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 <
> > > > > > qcx978132955@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 <sj...@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 <
> > > > > > > >> dwysakowicz@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 <
> > > > > > > >> >>> dwysakowicz@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 <
> > > > > > > sjwiesman@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 <al...@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 <
> > > > > > > >> >>> aljoscha@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 <
> > > > > > > >> >> sjwiesman@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
> > > > > > > >> >>>>>>>>>>>>
> > > > > > > >> >>>>>>>>>>>
> > > > > > > >> >>>>>>>>>>
> > > > > > > >> >>>>>>
> > > > > > > >> >>>>
> > > > > > > >> >>>>
> > > > > > > >> >>>
> > > > > > > >> >>
> > > > > > > >> >
> > > > > > > >>
> > > > > > > >>
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] FLIP-142: Disentangle StateBackends from Checkpointing

Posted by Seth Wiesman <sj...@gmail.com>.
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 <sj...@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 <ca...@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 <qc...@gmail.com>
> > wrote:
> > >
> > > > Thanks for the detailed replay, +1 from my side.
> > > > Best,
> > > > Congxian
> > > >
> > > >
> > > > Seth Wiesman <sj...@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 <
> > > > aljoscha@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 <
> > > > > qcx978132955@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 <sj...@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 <
> > > > > > >> dwysakowicz@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 <
> > > > > > >> >>> dwysakowicz@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 <
> > > > > > sjwiesman@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 <al...@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 <
> > > > > > >> >>> aljoscha@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 <
> > > > > > >> >> sjwiesman@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
> > > > > > >> >>>>>>>>>>>>
> > > > > > >> >>>>>>>>>>>
> > > > > > >> >>>>>>>>>>
> > > > > > >> >>>>>>
> > > > > > >> >>>>
> > > > > > >> >>>>
> > > > > > >> >>>
> > > > > > >> >>
> > > > > > >> >
> > > > > > >>
> > > > > > >>
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] FLIP-142: Disentangle StateBackends from Checkpointing

Posted by Stephan Ewen <se...@apache.org>.
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 <sj...@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 <ca...@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 <qc...@gmail.com>
> wrote:
> >
> > > Thanks for the detailed replay, +1 from my side.
> > > Best,
> > > Congxian
> > >
> > >
> > > Seth Wiesman <sj...@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 <
> > > aljoscha@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 <
> > > > qcx978132955@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 <sj...@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 <
> > > > > >> dwysakowicz@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 <
> > > > > >> >>> dwysakowicz@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 <
> > > > > sjwiesman@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 <al...@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 <
> > > > > >> >>> aljoscha@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 <
> > > > > >> >> sjwiesman@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
> > > > > >> >>>>>>>>>>>>
> > > > > >> >>>>>>>>>>>
> > > > > >> >>>>>>>>>>
> > > > > >> >>>>>>
> > > > > >> >>>>
> > > > > >> >>>>
> > > > > >> >>>
> > > > > >> >>
> > > > > >> >
> > > > > >>
> > > > > >>
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] FLIP-142: Disentangle StateBackends from Checkpointing

Posted by Seth Wiesman <sj...@gmail.com>.
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 <ca...@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 <qc...@gmail.com> wrote:
>
> > Thanks for the detailed replay, +1 from my side.
> > Best,
> > Congxian
> >
> >
> > Seth Wiesman <sj...@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 <
> > aljoscha@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 <
> > > qcx978132955@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 <sj...@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 <
> > > > >> dwysakowicz@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 <
> > > > >> >>> dwysakowicz@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 <
> > > > sjwiesman@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 <al...@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 <
> > > > >> >>> aljoscha@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 <
> > > > >> >> sjwiesman@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
> > > > >> >>>>>>>>>>>>
> > > > >> >>>>>>>>>>>
> > > > >> >>>>>>>>>>
> > > > >> >>>>>>
> > > > >> >>>>
> > > > >> >>>>
> > > > >> >>>
> > > > >> >>
> > > > >> >
> > > > >>
> > > > >>
> > > >
> > >
> >
>

Re: [DISCUSS] FLIP-142: Disentangle StateBackends from Checkpointing

Posted by Yu Li <ca...@gmail.com>.
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 <qc...@gmail.com> wrote:

> Thanks for the detailed replay, +1 from my side.
> Best,
> Congxian
>
>
> Seth Wiesman <sj...@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 <
> aljoscha@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 <
> > qcx978132955@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 <sj...@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 <
> > > >> dwysakowicz@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 <
> > > >> >>> dwysakowicz@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 <
> > > sjwiesman@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 <al...@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 <
> > > >> >>> aljoscha@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 <
> > > >> >> sjwiesman@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
> > > >> >>>>>>>>>>>>
> > > >> >>>>>>>>>>>
> > > >> >>>>>>>>>>
> > > >> >>>>>>
> > > >> >>>>
> > > >> >>>>
> > > >> >>>
> > > >> >>
> > > >> >
> > > >>
> > > >>
> > >
> >
>

Re: [DISCUSS] FLIP-142: Disentangle StateBackends from Checkpointing

Posted by Congxian Qiu <qc...@gmail.com>.
Thanks for the detailed replay, +1 from my side.
Best,
Congxian


Seth Wiesman <sj...@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 <al...@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 <
> qcx978132955@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 <sj...@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 <
> > >> dwysakowicz@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 <
> > >> >>> dwysakowicz@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 <
> > sjwiesman@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 <al...@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 <
> > >> >>> aljoscha@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 <
> > >> >> sjwiesman@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
> > >> >>>>>>>>>>>>
> > >> >>>>>>>>>>>
> > >> >>>>>>>>>>
> > >> >>>>>>
> > >> >>>>
> > >> >>>>
> > >> >>>
> > >> >>
> > >> >
> > >>
> > >>
> >
>

Re: [DISCUSS] FLIP-142: Disentangle StateBackends from Checkpointing

Posted by Seth Wiesman <sj...@gmail.com>.
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 <al...@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 <qcx978132955@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 <sj...@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 <
> >> dwysakowicz@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 <
> >> >>> dwysakowicz@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 <
> sjwiesman@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 <al...@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 <
> >> >>> aljoscha@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 <
> >> >> sjwiesman@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
> >> >>>>>>>>>>>>
> >> >>>>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>
> >> >>>>
> >> >>>>
> >> >>>
> >> >>
> >> >
> >>
> >>
>

Re: [DISCUSS] FLIP-142: Disentangle StateBackends from Checkpointing

Posted by Stephan Ewen <se...@apache.org>.
@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 <al...@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 <qc...@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 <sj...@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 <
>> dwysakowicz@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 <
>> >>> dwysakowicz@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 <sjwiesman@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 <al...@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 <
>> >>> aljoscha@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 <
>> >> sjwiesman@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
>> >>>>>>>>>>>>
>> >>>>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>
>> >>>>
>> >>>>
>> >>>
>> >>
>> >
>>
>>

Re: [DISCUSS] FLIP-142: Disentangle StateBackends from Checkpointing

Posted by Stephan Ewen <se...@apache.org>.
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 <al...@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 <qc...@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 <sj...@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 <
> dwysakowicz@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 <
> >>> dwysakowicz@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 <sj...@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 <al...@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 <
> >>> aljoscha@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 <
> >> sjwiesman@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
> >>>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>
> >>>>
> >>>>
> >>>
> >>
> >
>
>

Re: [DISCUSS] FLIP-142: Disentangle StateBackends from Checkpointing

Posted by Aljoscha Krettek <al...@apache.org>.
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 <qc...@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 <sj...@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 <dwysakowicz@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 <
>>> dwysakowicz@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 <sj...@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 <al...@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 <
>>> aljoscha@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 <
>> sjwiesman@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
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>
>>>>
>>>>
>>>
>>
> 


Re: [DISCUSS] FLIP-142: Disentangle StateBackends from Checkpointing

Posted by Seth Wiesman <sj...@gmail.com>.
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 <qc...@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 <sj...@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 <dwysakowicz@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 <
> > dwysakowicz@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 <sj...@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 <al...@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 <
> > aljoscha@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 <
> sjwiesman@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
> > > >>>>>>>>
> > > >>>>>>>
> > > >>>>>>
> > > >>
> > >
> > >
> >
>

Re: [DISCUSS] FLIP-142: Disentangle StateBackends from Checkpointing

Posted by Congxian Qiu <qc...@gmail.com>.
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 <sj...@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 <dw...@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 <
> dwysakowicz@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 <sj...@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 <al...@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 <
> aljoscha@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 <sjwiesman@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
> > >>>>>>>>
> > >>>>>>>
> > >>>>>>
> > >>
> >
> >
>

Re: [DISCUSS] FLIP-142: Disentangle StateBackends from Checkpointing

Posted by Seth Wiesman <sj...@gmail.com>.
Sounds good to me. I'll update the FLIP.

On Tue, Sep 15, 2020 at 8:35 AM Dawid Wysakowicz <dw...@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 <dwysakowicz@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 <sj...@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 <al...@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 <aljoscha@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 <sj...@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
> >>>>>>>>
> >>>>>>>
> >>>>>>
> >>
>
>

Re: [DISCUSS] FLIP-142: Disentangle StateBackends from Checkpointing

Posted by Dawid Wysakowicz <dw...@apache.org>.
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 <dw...@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 <sj...@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 <al...@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 <al...@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 <sj...@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
>>>>>>>>
>>>>>>>
>>>>>>
>>


Re: [DISCUSS] FLIP-142: Disentangle StateBackends from Checkpointing

Posted by Seth Wiesman <sj...@gmail.com>.
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 <dw...@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 <sj...@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 <al...@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 <al...@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 <sj...@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
> >>>>>>
> >>>>>
> >>>>>
> >>>>
> >>>>
> >>
> >
>
>

Re: [DISCUSS] FLIP-142: Disentangle StateBackends from Checkpointing

Posted by Dawid Wysakowicz <dw...@apache.org>.
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 <sj...@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 <al...@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 <al...@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 <sj...@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
>>>>>>
>>>>>
>>>>>
>>>>
>>>>
>>
>


Re: [DISCUSS] FLIP-142: Disentangle StateBackends from Checkpointing

Posted by Aljoscha Krettek <al...@apache.org>.
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 <sj...@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 <al...@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 <al...@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 <sj...@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
>>>>>
>>>>
>>>>
>>>
>>>
> 


Re: [DISCUSS] FLIP-142: Disentangle StateBackends from Checkpointing

Posted by Seth Wiesman <sj...@gmail.com>.
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 <sj...@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 <al...@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 <al...@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 <sj...@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
>> >>
>> >
>> >
>>
>>

Re: [DISCUSS] FLIP-142: Disentangle StateBackends from Checkpointing

Posted by Seth Wiesman <sj...@gmail.com>.
@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 <al...@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 <al...@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 <sj...@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
> >>
> >
> >
>
>

Re: [DISCUSS] FLIP-142: Disentangle StateBackends from Checkpointing

Posted by Aljoscha Krettek <al...@apache.org>.
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 <sj...@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
>>
> 
> 


Re: [DISCUSS] FLIP-142: Disentangle StateBackends from Checkpointing

Posted by Yun Tang <my...@live.com>.
Hi Seth

Thanks for bringing this discussion, and I really like this refactor to give more cleaner concepts!

When we talk about the relationship between state, state backends, and snapshots. The 'CheckpointStorage'
only focus on how to persist the checkpointed state (to JM or to DFS), there still exist some concepts related to the
implementation of state backend, e.g. the configuration of 'state.backend.incremental' and 'state.backend.async'.

What do you think of these configurations? They're still related with checkpointing but limits to the feature of state backend.

Moreover, when talk about separating state backend from checkpointing, I also want to give another two cents here:
state backend which holds the state is must-to-have when we use state in streaming job, however, checkpointing is not a must-to-have
if we do not enable the checkpointing. And 'ExecutionGraph' could live without checkpointCoordinator on JM side while
'StreamTask' always initialize the 'checkpointStorage' on task side. I think JM knows the inner relationship between state backend and checkpoint
while TM seems mix them together.

Best
Yun Tang
________________________________
From: Konstantin Knauf <kn...@apache.org>
Sent: Wednesday, September 9, 2020 16:05
To: dev <de...@flink.apache.org>
Subject: Re: [DISCUSS] FLIP-142: Disentangle StateBackends from Checkpointing

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 <sj...@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
>


--

Konstantin Knauf

https://twitter.com/snntrable

https://github.com/knaufk

Re: [DISCUSS] FLIP-142: Disentangle StateBackends from Checkpointing

Posted by Konstantin Knauf <kn...@apache.org>.
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 <sj...@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
>


-- 

Konstantin Knauf

https://twitter.com/snntrable

https://github.com/knaufk