You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@flink.apache.org by Yun Tang <my...@live.com> on 2020/06/02 10:20:25 UTC

Re: [DISCUSS] (Document) Backwards Compatibility of Savepoints

Since Flink lacks of such kind of experiments to ensure the backwards compatibility of savepoints before, especially those built-in operators with their own operator state.
I am afraid we need huge energy to cover all cases to give the most correct result.

I prefer to just point out this in documentation to say explicitly Flink does not guarantee such kind of backwards compatibility.

Best
Yun Tang
________________________________
From: Ufuk Celebi <uc...@apache.org>
Sent: Wednesday, May 27, 2020 16:42
To: dev@flink.apache.org <de...@flink.apache.org>
Subject: Re: [DISCUSS] (Document) Backwards Compatibility of Savepoints

I agree with Konstantin and Steven that it makes sense to point this out
explicitly.

I think that the following would be helpful:

1/ Mention breaking compatibility in release notes

2/ Update the linked table to reflect compatibilities while pointing out
what the community commits to maintain going forward (e.g. "happens to
work" vs. "guaranteed to work")

In general, the table is quite large. Would it make sense to order the
releases in reverse order (assuming that the table is more relevant for
recent releases)?

– Ufuk

On Tue, May 26, 2020 at 8:36 PM Steven Wu <st...@gmail.com> wrote:

> > A use case for this might be when you want to rollback a framework
> upgrade (after some time) due to e.g. a performance
> or stability issue.
>
> Downgrade (that Konstantin called out) is an important and realistic
> scenario. It will be great to support backward compatibility for savepoint
> or at least document any breaking change.
>
> On Tue, May 26, 2020 at 4:39 AM Piotr Nowojski <pi...@ververica.com>
> wrote:
>
> > Hi,
> >
> > It might have been implicit choice, but so far we were not supporting the
> > scenario that you are asking for. It has never been tested and we have
> > lot’s of state migration code sprinkled among our code base (for example
> > upgrading state fields of the operators like [1]), that only supports
> > upgrades, not downgrades.
> >
> > Also we do not have testing infrastructure for checking the downgrades.
> We
> > would need to check if save points taken from master branch, are readable
> > by previous releases (not release branch!).
> >
> > So all in all, I don’t think it can be easily done. It would require some
> > effort to start maintaining backward compatibility.
> >
> > Piotrek
> >
> > [1]
> >
> org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer011#migrateNextTransactionalIdHindState
> >
> > > On 26 May 2020, at 13:18, Konstantin Knauf <kn...@apache.org> wrote:
> > >
> > > Hi everyone,
> > >
> > > I recently stumbled across the fact that Savepoints created with Flink
> > 1.11
> > > can not be read by Flink 1.10. A use case for this might be when you
> want
> > > to rollback a framework upgrade (after some time) due to e.g. a
> > performance
> > > or stability issue.
> > >
> > > From the documentation [1] it seems as if the Savepoint format is
> > generally
> > > only forward-compatible although in many cases it is actually also
> > > backwards compatible (e.g. Savepoint taken in Flink 1.10, restored with
> > > Flink 1.9).
> > >
> > > Was it a deliberate choice not to document any backwards compatibility?
> > If
> > > not, should we add the missing entries in the compatibility table?
> > >
> > > Thanks,
> > >
> > > Konstantin
> > >
> > > [1]
> > >
> >
> https://ci.apache.org/projects/flink/flink-docs-master/ops/upgrading.html#compatibility-table
> > >
> > > --
> > >
> > > Konstantin Knauf
> > >
> > > https://twitter.com/snntrable
> > >
> > > https://github.com/knaufk
> >
> >
>

Re: [DISCUSS] (Document) Backwards Compatibility of Savepoints

Posted by Konstantin Knauf <kn...@apache.org>.
Hi everyone,

it seems that guaranteeing any backwards compatibility for Savepoints
stretches the resources of the community too much right now. As Steven
says, the documentation is actually pretty clear on that.

I still think it would be great if we would only break compatibility
knowingly, so that we can document whenever we maintain compatibility. For
many jobs and versions this will actually work, so it is a bit of a pity we
can not give more guidance to users. Having said that, even that seems
complex given what Piotr & Yun have written.

Best,

Konstantin





On Sun, Jun 7, 2020 at 6:57 AM Steven Wu <st...@gmail.com> wrote:

> > Why do we want to restore from the savepoint taken the new Flink version
> instead of the previous savepoint, is that we want to minimize the source
> rewind?
>
> You are exactly right. E.g. A user upgraded to the new version for a few
> days and decided to roll back to the old version due to some stability
> issue. Previous savepoint for the old version was taken a few days ago,
> which is a long time to rewind and reprocess. It can even be out of Kafka
> retention.
>
> On Fri, Jun 5, 2020 at 8:13 PM Congxian Qiu <qc...@gmail.com>
> wrote:
>
> > Sorry for jumping in late.
> >
> > Currently, we only have a forward-compatible guarantee and do not have
> the
> > backward-compatible guarantee. And as this may take a large effort to
> > support the backward-compatible guarantee. so I agree that we should
> write
> > this down explicitly.
> >
> > For the given scenario, I have a little question: Why do we want to
> restore
> > from the savepoint taken the new Flink version instead of the previous
> > savepoint, is that we want to minimize the source rewind?
> >
> > Best,
> > Congxian
> >
> >
> > Steven Wu <st...@gmail.com> 于2020年6月3日周三 上午9:08写道:
> >
> > > Current Flink documentation is actually pretty clear about no
> guarantees
> > > for backward compatibility.
> > >
> > >
> >
> https://ci.apache.org/projects/flink/flink-docs-stable/ops/upgrading.html#compatibility-table
> > >
> > > On Tue, Jun 2, 2020 at 3:20 AM Yun Tang <my...@live.com> wrote:
> > >
> > > > Since Flink lacks of such kind of experiments to ensure the backwards
> > > > compatibility of savepoints before, especially those built-in
> operators
> > > > with their own operator state.
> > > > I am afraid we need huge energy to cover all cases to give the most
> > > > correct result.
> > > >
> > > > I prefer to just point out this in documentation to say explicitly
> > Flink
> > > > does not guarantee such kind of backwards compatibility.
> > > >
> > > > Best
> > > > Yun Tang
> > > > ________________________________
> > > > From: Ufuk Celebi <uc...@apache.org>
> > > > Sent: Wednesday, May 27, 2020 16:42
> > > > To: dev@flink.apache.org <de...@flink.apache.org>
> > > > Subject: Re: [DISCUSS] (Document) Backwards Compatibility of
> Savepoints
> > > >
> > > > I agree with Konstantin and Steven that it makes sense to point this
> > out
> > > > explicitly.
> > > >
> > > > I think that the following would be helpful:
> > > >
> > > > 1/ Mention breaking compatibility in release notes
> > > >
> > > > 2/ Update the linked table to reflect compatibilities while pointing
> > out
> > > > what the community commits to maintain going forward (e.g. "happens
> to
> > > > work" vs. "guaranteed to work")
> > > >
> > > > In general, the table is quite large. Would it make sense to order
> the
> > > > releases in reverse order (assuming that the table is more relevant
> for
> > > > recent releases)?
> > > >
> > > > – Ufuk
> > > >
> > > > On Tue, May 26, 2020 at 8:36 PM Steven Wu <st...@gmail.com>
> > wrote:
> > > >
> > > > > > A use case for this might be when you want to rollback a
> framework
> > > > > upgrade (after some time) due to e.g. a performance
> > > > > or stability issue.
> > > > >
> > > > > Downgrade (that Konstantin called out) is an important and
> realistic
> > > > > scenario. It will be great to support backward compatibility for
> > > > savepoint
> > > > > or at least document any breaking change.
> > > > >
> > > > > On Tue, May 26, 2020 at 4:39 AM Piotr Nowojski <
> piotr@ververica.com>
> > > > > wrote:
> > > > >
> > > > > > Hi,
> > > > > >
> > > > > > It might have been implicit choice, but so far we were not
> > supporting
> > > > the
> > > > > > scenario that you are asking for. It has never been tested and we
> > > have
> > > > > > lot’s of state migration code sprinkled among our code base (for
> > > > example
> > > > > > upgrading state fields of the operators like [1]), that only
> > supports
> > > > > > upgrades, not downgrades.
> > > > > >
> > > > > > Also we do not have testing infrastructure for checking the
> > > downgrades.
> > > > > We
> > > > > > would need to check if save points taken from master branch, are
> > > > readable
> > > > > > by previous releases (not release branch!).
> > > > > >
> > > > > > So all in all, I don’t think it can be easily done. It would
> > require
> > > > some
> > > > > > effort to start maintaining backward compatibility.
> > > > > >
> > > > > > Piotrek
> > > > > >
> > > > > > [1]
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer011#migrateNextTransactionalIdHindState
> > > > > >
> > > > > > > On 26 May 2020, at 13:18, Konstantin Knauf <kn...@apache.org>
> > > > wrote:
> > > > > > >
> > > > > > > Hi everyone,
> > > > > > >
> > > > > > > I recently stumbled across the fact that Savepoints created
> with
> > > > Flink
> > > > > > 1.11
> > > > > > > can not be read by Flink 1.10. A use case for this might be
> when
> > > you
> > > > > want
> > > > > > > to rollback a framework upgrade (after some time) due to e.g. a
> > > > > > performance
> > > > > > > or stability issue.
> > > > > > >
> > > > > > > From the documentation [1] it seems as if the Savepoint format
> is
> > > > > > generally
> > > > > > > only forward-compatible although in many cases it is actually
> > also
> > > > > > > backwards compatible (e.g. Savepoint taken in Flink 1.10,
> > restored
> > > > with
> > > > > > > Flink 1.9).
> > > > > > >
> > > > > > > Was it a deliberate choice not to document any backwards
> > > > compatibility?
> > > > > > If
> > > > > > > not, should we add the missing entries in the compatibility
> > table?
> > > > > > >
> > > > > > > Thanks,
> > > > > > >
> > > > > > > Konstantin
> > > > > > >
> > > > > > > [1]
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> https://ci.apache.org/projects/flink/flink-docs-master/ops/upgrading.html#compatibility-table
> > > > > > >
> > > > > > > --
> > > > > > >
> > > > > > > Konstantin Knauf
> > > > > > >
> > > > > > > https://twitter.com/snntrable
> > > > > > >
> > > > > > > https://github.com/knaufk
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>


-- 

Konstantin Knauf

https://twitter.com/snntrable

https://github.com/knaufk

Re: [DISCUSS] (Document) Backwards Compatibility of Savepoints

Posted by Steven Wu <st...@gmail.com>.
> Why do we want to restore from the savepoint taken the new Flink version
instead of the previous savepoint, is that we want to minimize the source
rewind?

You are exactly right. E.g. A user upgraded to the new version for a few
days and decided to roll back to the old version due to some stability
issue. Previous savepoint for the old version was taken a few days ago,
which is a long time to rewind and reprocess. It can even be out of Kafka
retention.

On Fri, Jun 5, 2020 at 8:13 PM Congxian Qiu <qc...@gmail.com> wrote:

> Sorry for jumping in late.
>
> Currently, we only have a forward-compatible guarantee and do not have the
> backward-compatible guarantee. And as this may take a large effort to
> support the backward-compatible guarantee. so I agree that we should write
> this down explicitly.
>
> For the given scenario, I have a little question: Why do we want to restore
> from the savepoint taken the new Flink version instead of the previous
> savepoint, is that we want to minimize the source rewind?
>
> Best,
> Congxian
>
>
> Steven Wu <st...@gmail.com> 于2020年6月3日周三 上午9:08写道:
>
> > Current Flink documentation is actually pretty clear about no guarantees
> > for backward compatibility.
> >
> >
> https://ci.apache.org/projects/flink/flink-docs-stable/ops/upgrading.html#compatibility-table
> >
> > On Tue, Jun 2, 2020 at 3:20 AM Yun Tang <my...@live.com> wrote:
> >
> > > Since Flink lacks of such kind of experiments to ensure the backwards
> > > compatibility of savepoints before, especially those built-in operators
> > > with their own operator state.
> > > I am afraid we need huge energy to cover all cases to give the most
> > > correct result.
> > >
> > > I prefer to just point out this in documentation to say explicitly
> Flink
> > > does not guarantee such kind of backwards compatibility.
> > >
> > > Best
> > > Yun Tang
> > > ________________________________
> > > From: Ufuk Celebi <uc...@apache.org>
> > > Sent: Wednesday, May 27, 2020 16:42
> > > To: dev@flink.apache.org <de...@flink.apache.org>
> > > Subject: Re: [DISCUSS] (Document) Backwards Compatibility of Savepoints
> > >
> > > I agree with Konstantin and Steven that it makes sense to point this
> out
> > > explicitly.
> > >
> > > I think that the following would be helpful:
> > >
> > > 1/ Mention breaking compatibility in release notes
> > >
> > > 2/ Update the linked table to reflect compatibilities while pointing
> out
> > > what the community commits to maintain going forward (e.g. "happens to
> > > work" vs. "guaranteed to work")
> > >
> > > In general, the table is quite large. Would it make sense to order the
> > > releases in reverse order (assuming that the table is more relevant for
> > > recent releases)?
> > >
> > > – Ufuk
> > >
> > > On Tue, May 26, 2020 at 8:36 PM Steven Wu <st...@gmail.com>
> wrote:
> > >
> > > > > A use case for this might be when you want to rollback a framework
> > > > upgrade (after some time) due to e.g. a performance
> > > > or stability issue.
> > > >
> > > > Downgrade (that Konstantin called out) is an important and realistic
> > > > scenario. It will be great to support backward compatibility for
> > > savepoint
> > > > or at least document any breaking change.
> > > >
> > > > On Tue, May 26, 2020 at 4:39 AM Piotr Nowojski <pi...@ververica.com>
> > > > wrote:
> > > >
> > > > > Hi,
> > > > >
> > > > > It might have been implicit choice, but so far we were not
> supporting
> > > the
> > > > > scenario that you are asking for. It has never been tested and we
> > have
> > > > > lot’s of state migration code sprinkled among our code base (for
> > > example
> > > > > upgrading state fields of the operators like [1]), that only
> supports
> > > > > upgrades, not downgrades.
> > > > >
> > > > > Also we do not have testing infrastructure for checking the
> > downgrades.
> > > > We
> > > > > would need to check if save points taken from master branch, are
> > > readable
> > > > > by previous releases (not release branch!).
> > > > >
> > > > > So all in all, I don’t think it can be easily done. It would
> require
> > > some
> > > > > effort to start maintaining backward compatibility.
> > > > >
> > > > > Piotrek
> > > > >
> > > > > [1]
> > > > >
> > > >
> > >
> >
> org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer011#migrateNextTransactionalIdHindState
> > > > >
> > > > > > On 26 May 2020, at 13:18, Konstantin Knauf <kn...@apache.org>
> > > wrote:
> > > > > >
> > > > > > Hi everyone,
> > > > > >
> > > > > > I recently stumbled across the fact that Savepoints created with
> > > Flink
> > > > > 1.11
> > > > > > can not be read by Flink 1.10. A use case for this might be when
> > you
> > > > want
> > > > > > to rollback a framework upgrade (after some time) due to e.g. a
> > > > > performance
> > > > > > or stability issue.
> > > > > >
> > > > > > From the documentation [1] it seems as if the Savepoint format is
> > > > > generally
> > > > > > only forward-compatible although in many cases it is actually
> also
> > > > > > backwards compatible (e.g. Savepoint taken in Flink 1.10,
> restored
> > > with
> > > > > > Flink 1.9).
> > > > > >
> > > > > > Was it a deliberate choice not to document any backwards
> > > compatibility?
> > > > > If
> > > > > > not, should we add the missing entries in the compatibility
> table?
> > > > > >
> > > > > > Thanks,
> > > > > >
> > > > > > Konstantin
> > > > > >
> > > > > > [1]
> > > > > >
> > > > >
> > > >
> > >
> >
> https://ci.apache.org/projects/flink/flink-docs-master/ops/upgrading.html#compatibility-table
> > > > > >
> > > > > > --
> > > > > >
> > > > > > Konstantin Knauf
> > > > > >
> > > > > > https://twitter.com/snntrable
> > > > > >
> > > > > > https://github.com/knaufk
> > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] (Document) Backwards Compatibility of Savepoints

Posted by Congxian Qiu <qc...@gmail.com>.
Sorry for jumping in late.

Currently, we only have a forward-compatible guarantee and do not have the
backward-compatible guarantee. And as this may take a large effort to
support the backward-compatible guarantee. so I agree that we should write
this down explicitly.

For the given scenario, I have a little question: Why do we want to restore
from the savepoint taken the new Flink version instead of the previous
savepoint, is that we want to minimize the source rewind?

Best,
Congxian


Steven Wu <st...@gmail.com> 于2020年6月3日周三 上午9:08写道:

> Current Flink documentation is actually pretty clear about no guarantees
> for backward compatibility.
>
> https://ci.apache.org/projects/flink/flink-docs-stable/ops/upgrading.html#compatibility-table
>
> On Tue, Jun 2, 2020 at 3:20 AM Yun Tang <my...@live.com> wrote:
>
> > Since Flink lacks of such kind of experiments to ensure the backwards
> > compatibility of savepoints before, especially those built-in operators
> > with their own operator state.
> > I am afraid we need huge energy to cover all cases to give the most
> > correct result.
> >
> > I prefer to just point out this in documentation to say explicitly Flink
> > does not guarantee such kind of backwards compatibility.
> >
> > Best
> > Yun Tang
> > ________________________________
> > From: Ufuk Celebi <uc...@apache.org>
> > Sent: Wednesday, May 27, 2020 16:42
> > To: dev@flink.apache.org <de...@flink.apache.org>
> > Subject: Re: [DISCUSS] (Document) Backwards Compatibility of Savepoints
> >
> > I agree with Konstantin and Steven that it makes sense to point this out
> > explicitly.
> >
> > I think that the following would be helpful:
> >
> > 1/ Mention breaking compatibility in release notes
> >
> > 2/ Update the linked table to reflect compatibilities while pointing out
> > what the community commits to maintain going forward (e.g. "happens to
> > work" vs. "guaranteed to work")
> >
> > In general, the table is quite large. Would it make sense to order the
> > releases in reverse order (assuming that the table is more relevant for
> > recent releases)?
> >
> > – Ufuk
> >
> > On Tue, May 26, 2020 at 8:36 PM Steven Wu <st...@gmail.com> wrote:
> >
> > > > A use case for this might be when you want to rollback a framework
> > > upgrade (after some time) due to e.g. a performance
> > > or stability issue.
> > >
> > > Downgrade (that Konstantin called out) is an important and realistic
> > > scenario. It will be great to support backward compatibility for
> > savepoint
> > > or at least document any breaking change.
> > >
> > > On Tue, May 26, 2020 at 4:39 AM Piotr Nowojski <pi...@ververica.com>
> > > wrote:
> > >
> > > > Hi,
> > > >
> > > > It might have been implicit choice, but so far we were not supporting
> > the
> > > > scenario that you are asking for. It has never been tested and we
> have
> > > > lot’s of state migration code sprinkled among our code base (for
> > example
> > > > upgrading state fields of the operators like [1]), that only supports
> > > > upgrades, not downgrades.
> > > >
> > > > Also we do not have testing infrastructure for checking the
> downgrades.
> > > We
> > > > would need to check if save points taken from master branch, are
> > readable
> > > > by previous releases (not release branch!).
> > > >
> > > > So all in all, I don’t think it can be easily done. It would require
> > some
> > > > effort to start maintaining backward compatibility.
> > > >
> > > > Piotrek
> > > >
> > > > [1]
> > > >
> > >
> >
> org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer011#migrateNextTransactionalIdHindState
> > > >
> > > > > On 26 May 2020, at 13:18, Konstantin Knauf <kn...@apache.org>
> > wrote:
> > > > >
> > > > > Hi everyone,
> > > > >
> > > > > I recently stumbled across the fact that Savepoints created with
> > Flink
> > > > 1.11
> > > > > can not be read by Flink 1.10. A use case for this might be when
> you
> > > want
> > > > > to rollback a framework upgrade (after some time) due to e.g. a
> > > > performance
> > > > > or stability issue.
> > > > >
> > > > > From the documentation [1] it seems as if the Savepoint format is
> > > > generally
> > > > > only forward-compatible although in many cases it is actually also
> > > > > backwards compatible (e.g. Savepoint taken in Flink 1.10, restored
> > with
> > > > > Flink 1.9).
> > > > >
> > > > > Was it a deliberate choice not to document any backwards
> > compatibility?
> > > > If
> > > > > not, should we add the missing entries in the compatibility table?
> > > > >
> > > > > Thanks,
> > > > >
> > > > > Konstantin
> > > > >
> > > > > [1]
> > > > >
> > > >
> > >
> >
> https://ci.apache.org/projects/flink/flink-docs-master/ops/upgrading.html#compatibility-table
> > > > >
> > > > > --
> > > > >
> > > > > Konstantin Knauf
> > > > >
> > > > > https://twitter.com/snntrable
> > > > >
> > > > > https://github.com/knaufk
> > > >
> > > >
> > >
> >
>

Re: [DISCUSS] (Document) Backwards Compatibility of Savepoints

Posted by Steven Wu <st...@gmail.com>.
Current Flink documentation is actually pretty clear about no guarantees
for backward compatibility.
https://ci.apache.org/projects/flink/flink-docs-stable/ops/upgrading.html#compatibility-table

On Tue, Jun 2, 2020 at 3:20 AM Yun Tang <my...@live.com> wrote:

> Since Flink lacks of such kind of experiments to ensure the backwards
> compatibility of savepoints before, especially those built-in operators
> with their own operator state.
> I am afraid we need huge energy to cover all cases to give the most
> correct result.
>
> I prefer to just point out this in documentation to say explicitly Flink
> does not guarantee such kind of backwards compatibility.
>
> Best
> Yun Tang
> ________________________________
> From: Ufuk Celebi <uc...@apache.org>
> Sent: Wednesday, May 27, 2020 16:42
> To: dev@flink.apache.org <de...@flink.apache.org>
> Subject: Re: [DISCUSS] (Document) Backwards Compatibility of Savepoints
>
> I agree with Konstantin and Steven that it makes sense to point this out
> explicitly.
>
> I think that the following would be helpful:
>
> 1/ Mention breaking compatibility in release notes
>
> 2/ Update the linked table to reflect compatibilities while pointing out
> what the community commits to maintain going forward (e.g. "happens to
> work" vs. "guaranteed to work")
>
> In general, the table is quite large. Would it make sense to order the
> releases in reverse order (assuming that the table is more relevant for
> recent releases)?
>
> – Ufuk
>
> On Tue, May 26, 2020 at 8:36 PM Steven Wu <st...@gmail.com> wrote:
>
> > > A use case for this might be when you want to rollback a framework
> > upgrade (after some time) due to e.g. a performance
> > or stability issue.
> >
> > Downgrade (that Konstantin called out) is an important and realistic
> > scenario. It will be great to support backward compatibility for
> savepoint
> > or at least document any breaking change.
> >
> > On Tue, May 26, 2020 at 4:39 AM Piotr Nowojski <pi...@ververica.com>
> > wrote:
> >
> > > Hi,
> > >
> > > It might have been implicit choice, but so far we were not supporting
> the
> > > scenario that you are asking for. It has never been tested and we have
> > > lot’s of state migration code sprinkled among our code base (for
> example
> > > upgrading state fields of the operators like [1]), that only supports
> > > upgrades, not downgrades.
> > >
> > > Also we do not have testing infrastructure for checking the downgrades.
> > We
> > > would need to check if save points taken from master branch, are
> readable
> > > by previous releases (not release branch!).
> > >
> > > So all in all, I don’t think it can be easily done. It would require
> some
> > > effort to start maintaining backward compatibility.
> > >
> > > Piotrek
> > >
> > > [1]
> > >
> >
> org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer011#migrateNextTransactionalIdHindState
> > >
> > > > On 26 May 2020, at 13:18, Konstantin Knauf <kn...@apache.org>
> wrote:
> > > >
> > > > Hi everyone,
> > > >
> > > > I recently stumbled across the fact that Savepoints created with
> Flink
> > > 1.11
> > > > can not be read by Flink 1.10. A use case for this might be when you
> > want
> > > > to rollback a framework upgrade (after some time) due to e.g. a
> > > performance
> > > > or stability issue.
> > > >
> > > > From the documentation [1] it seems as if the Savepoint format is
> > > generally
> > > > only forward-compatible although in many cases it is actually also
> > > > backwards compatible (e.g. Savepoint taken in Flink 1.10, restored
> with
> > > > Flink 1.9).
> > > >
> > > > Was it a deliberate choice not to document any backwards
> compatibility?
> > > If
> > > > not, should we add the missing entries in the compatibility table?
> > > >
> > > > Thanks,
> > > >
> > > > Konstantin
> > > >
> > > > [1]
> > > >
> > >
> >
> https://ci.apache.org/projects/flink/flink-docs-master/ops/upgrading.html#compatibility-table
> > > >
> > > > --
> > > >
> > > > Konstantin Knauf
> > > >
> > > > https://twitter.com/snntrable
> > > >
> > > > https://github.com/knaufk
> > >
> > >
> >
>