> 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