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 <myas...@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 <u...@apache.org>
> Sent: Wednesday, May 27, 2020 16:42
> To: dev@flink.apache.org <dev@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 <stevenz...@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 <kna...@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
> > >
> > >
> >
>

Reply via email to