You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@flink.apache.org by Kostas Kloudas <k....@data-artisans.com> on 2017/05/20 17:09:56 UTC

[DISCUSS] Backwards compatibility policy.

Hi all,

As we are getting closer to releasing Flink-1.3, I would like to open a discussion
on how far back we provide backwards compatibility for.

The reason for opening the discussion is that i) for the users and for the 
adoption of the project, it is good to have an explicitely stated policy that implies
certain guarantees, and ii) keeping code and tests for backwards compatibility with 
Flink-1.1 does not offer much. On the contrary, I think that it leads to:

1) dead or ugly code in the codebase, e.g. deprecated class fields that could go away and 
ugly if() loops (see aligned window operators that were deprecated in 1.2 and are now 
normal windows), etc
2) expensive tests (as, normally, they read from a savepoint)
3) binary files in the codebase for holding the aforementioned savepoints

My proposal for such a policy would be to offer backwards compatibility for one previous version. 

This means that 1.3 will be compatible with 1.2 (not 1.1). This still allows a clear 
"backwards compatibility" path when jumping versions (a user that goes 
from 1.1 to 1.3 can go initially 1.1 -> 1.2, take a savepoint, and then 1.2 -> 1.3),
while also allowing us to clean up the codebase a bit. 

What do you think?

Kostas

Re: [DISCUSS] Backwards compatibility policy.

Posted by Greg Hogan <co...@greghogan.com>.
I can’t find when the time-based maintenance schedule switched from “6 months” to “2 concurrent versions” (which has not yet made it into the website [0]). Is it correct to assume that most users are waiting until the first bug fix release or later to upgrade? That only leaves a narrow window of stability.

Greg

[0] https://github.com/apache/flink-web/pull/50



> On May 22, 2017, at 1:39 AM, Tzu-Li (Gordon) Tai <tz...@apache.org> wrote:
> 
> Hi Kostas,
> 
> Thanks for bringing this up!
> I think it is reasonable to keep this coherent with our timely-based release model guarantees.
> 
> With the timely-based release model, there is a guarantee that the current latest major version and the previous one is supported.
> For example, upon releasing 1.3, only 1.3 and 1.2 will still be supported by the community for any required bug fixes.
> I think this was initially decided not only to ease old version maintenance efforts for the community, but also as a means to let users upgrade their Flink versions in a reasonable pace (at least every other major release.)
> 
> Therefore, I think its also reasonable to also clearly state that savepoints compatibility will only be guaranteed for the previous release.
> Although I think at the moment almost, if not all, of the current code still maintains compatibility for 1.1, in the long run these migration codes would definitely start to pile up and pollute the actual codebase if we try to always be compatible with all previous versions.
> 
> Cheers,
> Gordon
> 
> 
> On 21 May 2017 at 2:24:53 AM, Kostas Kloudas (k.kloudas@data-artisans.com) wrote:
> 
> Hi Chesnay, 
> 
> I believe that for APIs we already have a pretty clear policy with the annotations. 
> I was referring to savepoints and state related backwards compatibility. 
> 
> 
>> On May 20, 2017, at 7:20 PM, Chesnay Schepler <ch...@apache.org> wrote: 
>> 
>> I think it would be a good to clarify what kind of backwards-compatibilitiy we're talking about here. As in are we talking about APIs or savepoints? 
>> 
>> On 20.05.2017 19:09, Kostas Kloudas wrote: 
>>> Hi all, 
>>> 
>>> As we are getting closer to releasing Flink-1.3, I would like to open a discussion 
>>> on how far back we provide backwards compatibility for. 
>>> 
>>> The reason for opening the discussion is that i) for the users and for the 
>>> adoption of the project, it is good to have an explicitely stated policy that implies 
>>> certain guarantees, and ii) keeping code and tests for backwards compatibility with 
>>> Flink-1.1 does not offer much. On the contrary, I think that it leads to: 
>>> 
>>> 1) dead or ugly code in the codebase, e.g. deprecated class fields that could go away and 
>>> ugly if() loops (see aligned window operators that were deprecated in 1.2 and are now 
>>> normal windows), etc 
>>> 2) expensive tests (as, normally, they read from a savepoint) 
>>> 3) binary files in the codebase for holding the aforementioned savepoints 
>>> 
>>> My proposal for such a policy would be to offer backwards compatibility for one previous version. 
>>> 
>>> This means that 1.3 will be compatible with 1.2 (not 1.1). This still allows a clear 
>>> "backwards compatibility" path when jumping versions (a user that goes 
>>> from 1.1 to 1.3 can go initially 1.1 -> 1.2, take a savepoint, and then 1.2 -> 1.3), 
>>> while also allowing us to clean up the codebase a bit. 
>>> 
>>> What do you think? 
>>> 
>>> Kostas


Re: [DISCUSS] Backwards compatibility policy.

Posted by Stephan Ewen <se...@apache.org>.
Bumping this thread again.

I think it would help Flink development a lot to drop 1.1.x savepoint
format compatibility in 1.4.0. That means NOT dropping API compatibility,
only not further supporting the old savepoint format version.

I'll start a poll on the users list.

On Fri, Jun 30, 2017 at 7:58 PM, Stephan Ewen <se...@apache.org> wrote:

> @Sebastian: I am not sure Apache has really guidelines there. So far, I
> thought projects establish their own policies.
>
> The compatibility questions here is also one of APIs (code), but of
> savepoint forwarding, which is a but different, I think. For example 1.0
> and 1.1 were not compatible there, the introduction of backwards
> compatibility from 1.2 to 1.1 was a big new thing for 1.2
>
>
> On Wed, Jun 28, 2017 at 11:53 AM, Sebastian Schelter <
> ssc.open@googlemail.com> wrote:
>
>> I haven't closely followed the discussion so far, but isn't it Apache
>> policy that major versions should stay backwards compatible to all
>> previous
>> releases with the same major version?
>>
>> -s
>>
>> 2017-06-28 12:26 GMT+02:00 Kostas Kloudas <k....@data-artisans.com>:
>>
>> > I agree that 1.1 compatibility is the most important “pain point", as
>> > compatibility with the rest of the versions follows a more “systematic”
>> > approach.
>> >
>> > I think that discarding compatibility with 1.1 will clear some parts
>> > of the codebase significantly.
>> >
>> > Kostas
>> >
>> > > On Jun 27, 2017, at 6:03 PM, Stephan Ewen <se...@apache.org> wrote:
>> > >
>> > > I think that this discussion is probably motivated especially by the
>> > > "legacy state" handling of Flink 1.1.
>> > > The biggest gain in codebase and productivity would be won only by
>> > dropping
>> > > 1.1 compatibility in Flink 1.4.
>> > >
>> > > My gut feeling is that this is reasonable. We support two versions
>> back,
>> > > which means that users can skip one upgrade, but not two.
>> > >
>> > > From what I can tell, users are usually eager to upgrade. They don't
>> do
>> > it
>> > > immediately, but as soon as the new release is a bit battle tested.
>> > >
>> > > I would expect skipping two entire versions to be rare enough to be
>> okay
>> > > with a solution which is a bit more effort for the user:
>> > > You can upgrade from Flink 1.1. to 1.4 by loading the 1.1 savepoint
>> into
>> > > Flink 1.2, take a savepoint (1.2 format), and resume that in Flink
>> 1.4.
>> > >
>> > > Greetings,
>> > > Stephan
>> > >
>> > >
>> > > On Tue, Jun 27, 2017 at 12:01 PM, Stefan Richter <
>> > > s.richter@data-artisans.com> wrote:
>> > >
>> > >> For many parts of the code, I would agree with Aljoscha. However, I
>> can
>> > >> also see notable exceptions, such as maintaining support for the
>> legacy
>> > >> state from Flink <=1.1. For example, I think dropping support for
>> this
>> > can
>> > >> simplify new developments such as fast local recovery or state
>> > replication
>> > >> quiet a bit because this is a special case that runs through a lot of
>> > code
>> > >> from backend to JM. So besides this general discussion about a
>> backwards
>> > >> compatible policy, do you think it could make sense to start another
>> > >> concrete discussion about if we still must or want backwards
>> > compatibility
>> > >> to Flink 1.1 in Flink 1.4?
>> > >>
>> > >>> Am 29.05.2017 um 12:08 schrieb Aljoscha Krettek <
>> aljoscha@apache.org>:
>> > >>>
>> > >>> Normally, I’m the first one to suggest removing everything that is
>> not
>> > >> absolutely necessary in order to have a clean code base. On this
>> issue,
>> > >> though, I think we should support restoring from old Savepoints as
>> far
>> > back
>> > >> as possible if it does not make the code completely unmaintainable.
>> Some
>> > >> users might jump versions and always forcing them to go though every
>> > >> version from their old version to the current version doesn’t seem
>> > feasible
>> > >> and might put off some users.
>> > >>>
>> > >>> So far, I think the burden of supporting restore from 1.1 is still
>> > small
>> > >> enough and with each new version the changes between versions become
>> > less
>> > >> and less. The changes from 1.2 to the upcoming 1.3 are quite
>> minimal, I
>> > >> think.
>> > >>>
>> > >>> Best,
>> > >>> Aljoscha
>> > >>>> On 24. May 2017, at 17:58, Ted Yu <yu...@gmail.com> wrote:
>> > >>>>
>> > >>>> bq. about having LTS versions once a year
>> > >>>>
>> > >>>> +1 to the above.
>> > >>>>
>> > >>>> There may be various reasons users don't want to upgrade (after new
>> > >>>> releases come out). We should give such users enough flexibility on
>> > the
>> > >>>> upgrade path.
>> > >>>>
>> > >>>> Cheers
>> > >>>>
>> > >>>> On Wed, May 24, 2017 at 8:39 AM, Kostas Kloudas <
>> > >> k.kloudas@data-artisans.com
>> > >>>>> wrote:
>> > >>>>
>> > >>>>> Hi all,
>> > >>>>>
>> > >>>>> For the proposal of having a third party tool, I agree with Ted.
>> > >>>>> Maintaining
>> > >>>>> it is a big and far from trivial effort.
>> > >>>>>
>> > >>>>> Now for the window of backwards compatibility, I would argue that
>> > even
>> > >> if
>> > >>>>> for some users 4 months (1 release) is not enough to bump their
>> Flink
>> > >>>>> version,
>> > >>>>> the proposed policy guarantees that there will always be a path
>> from
>> > >> any
>> > >>>>> old
>> > >>>>> version to any subsequent one.
>> > >>>>>
>> > >>>>> Finally, for the proposal about having LTS versions once a year,
>> I am
>> > >> not
>> > >>>>> sure if this will reduce or create more overhead. If I understand
>> the
>> > >> plan
>> > >>>>> correctly, this would mean that the community will have to
>> maintain
>> > >>>>> 2 or 3 LTS versions and the last two major ones, right?
>> > >>>>>
>> > >>>>>> On May 22, 2017, at 7:31 PM, Ted Yu <yu...@gmail.com> wrote:
>> > >>>>>>
>> > >>>>>> For #2, it is difficult to achieve:
>> > >>>>>>
>> > >>>>>> a. maintaining savepoint migration is non-trivial and should be
>> > >> reviewed
>> > >>>>> by
>> > >>>>>> domain experts
>> > >>>>>> b. how to certify such third-party tool
>> > >>>>>>
>> > >>>>>> Cheers
>> > >>>>>>
>> > >>>>>> On Mon, May 22, 2017 at 3:04 AM, 施晓罡 <sh...@gmail.com>
>> > wrote:
>> > >>>>>>
>> > >>>>>>> Hi all,
>> > >>>>>>>
>> > >>>>>>> Currently, we work a lot in the maintenance of compatibility.
>> > >>>>>>> There exist much code in runtime to support the migration of
>> > >> savepoints
>> > >>>>>>> (most of which are deprecated), making it hard to focus on the
>> > >> current
>> > >>>>>>> implementation.
>> > >>>>>>> When more versions are released, much more efforts will be
>> needed
>> > if
>> > >> we
>> > >>>>>>> try to make these released versions compatible.
>> > >>>>>>>
>> > >>>>>>> I agree with Tzu-Li that we should provide a method to let users
>> > >> upgrade
>> > >>>>>>> Flink in a reasonable pace.
>> > >>>>>>> But i am against the proposal that we only offer backwards
>> > >> compatibility
>> > >>>>>>> for one previous version.
>> > >>>>>>> According our time-based release model, a major version is
>> released
>> > >>>>> every
>> > >>>>>>> four month.
>> > >>>>>>> That means, users have to upgrade their versions every 8 months.
>> > >>>>> Otherwise
>> > >>>>>>> they will have difficulties in the migration of existing
>> > savepoints.
>> > >>>>>>>
>> > >>>>>>> My suggestions include
>> > >>>>>>>
>> > >>>>>>> (1) We can release Long-Term Support (LTS) versions which are
>> > widely
>> > >>>>>>> adopted in other open-source projects.
>> > >>>>>>> LTS versions should be stable and are free of found bugs.
>> > Savepoints
>> > >> in
>> > >>>>>>> LTS versions are guaranteed to be back-compatible so that users
>> can
>> > >>>>> easily
>> > >>>>>>> upgrade to newer LTS versions.
>> > >>>>>>>
>> > >>>>>>> The releasing of LTS versions is slower than that of major
>> versions
>> > >>>>> (maybe
>> > >>>>>>> once a year, determined by users’ upgrade frequency).
>> > >>>>>>> Each LTS version will be supported a period of time and
>> typically
>> > >> there
>> > >>>>>>> are no more than three active LTS versions.
>> > >>>>>>> By encouraging users to use LTS versions, we can ease the
>> > >> maintenance of
>> > >>>>>>> released versions (bug fixes, back compatibility, and critical
>> > >>>>> performance
>> > >>>>>>> improvement).
>> > >>>>>>>
>> > >>>>>>> (2) We can provide a third-party tool to do the migration of
>> > >>>>> old-versioned
>> > >>>>>>> savepoints.
>> > >>>>>>> When users upgrade their versions, they can use the provided
>> tool
>> > to
>> > >>>>>>> migrate existing savepoints.
>> > >>>>>>> This can help move the code for savepoint migration out of the
>> > actual
>> > >>>>>>> codebase,  making code focuses on current implementation.
>> > >>>>>>>
>> > >>>>>>> What do you think?
>> > >>>>>>>
>> > >>>>>>> Regards,
>> > >>>>>>> Xiaogang
>> > >>>>>>>
>> > >>>>>>>
>> > >>>>>>>> 在 2017年5月22日,下午1:39,Tzu-Li (Gordon) Tai <tz...@apache.org>
>> 写道:
>> > >>>>>>>>
>> > >>>>>>>> Hi Kostas,
>> > >>>>>>>>
>> > >>>>>>>> Thanks for bringing this up!
>> > >>>>>>>> I think it is reasonable to keep this coherent with our
>> > timely-based
>> > >>>>>>> release model guarantees.
>> > >>>>>>>>
>> > >>>>>>>> With the timely-based release model, there is a guarantee that
>> the
>> > >>>>>>> current latest major version and the previous one is supported.
>> > >>>>>>>> For example, upon releasing 1.3, only 1.3 and 1.2 will still be
>> > >>>>>>> supported by the community for any required bug fixes.
>> > >>>>>>>> I think this was initially decided not only to ease old version
>> > >>>>>>> maintenance efforts for the community, but also as a means to
>> let
>> > >> users
>> > >>>>>>> upgrade their Flink versions in a reasonable pace (at least
>> every
>> > >> other
>> > >>>>>>> major release.)
>> > >>>>>>>>
>> > >>>>>>>> Therefore, I think its also reasonable to also clearly state
>> that
>> > >>>>>>> savepoints compatibility will only be guaranteed for the
>> previous
>> > >>>>> release.
>> > >>>>>>>> Although I think at the moment almost, if not all, of the
>> current
>> > >> code
>> > >>>>>>> still maintains compatibility for 1.1, in the long run these
>> > >> migration
>> > >>>>>>> codes would definitely start to pile up and pollute the actual
>> > >> codebase
>> > >>>>> if
>> > >>>>>>> we try to always be compatible with all previous versions.
>> > >>>>>>>>
>> > >>>>>>>> Cheers,
>> > >>>>>>>> Gordon
>> > >>>>>>>>
>> > >>>>>>>>
>> > >>>>>>>> On 21 May 2017 at 2:24:53 AM, Kostas Kloudas (
>> > >>>>>>> k.kloudas@data-artisans.com) wrote:
>> > >>>>>>>>
>> > >>>>>>>> Hi Chesnay,
>> > >>>>>>>>
>> > >>>>>>>> I believe that for APIs we already have a pretty clear policy
>> with
>> > >> the
>> > >>>>>>> annotations.
>> > >>>>>>>> I was referring to savepoints and state related backwards
>> > >>>>> compatibility.
>> > >>>>>>>>
>> > >>>>>>>>
>> > >>>>>>>>> On May 20, 2017, at 7:20 PM, Chesnay Schepler <
>> > chesnay@apache.org>
>> > >>>>>>> wrote:
>> > >>>>>>>>>
>> > >>>>>>>>> I think it would be a good to clarify what kind of
>> > >>>>>>> backwards-compatibilitiy we're talking about here. As in are we
>> > >> talking
>> > >>>>>>> about APIs or savepoints?
>> > >>>>>>>>>
>> > >>>>>>>>> On 20.05.2017 19:09, Kostas Kloudas wrote:
>> > >>>>>>>>>> Hi all,
>> > >>>>>>>>>>
>> > >>>>>>>>>> As we are getting closer to releasing Flink-1.3, I would
>> like to
>> > >> open
>> > >>>>>>> a discussion
>> > >>>>>>>>>> on how far back we provide backwards compatibility for.
>> > >>>>>>>>>>
>> > >>>>>>>>>> The reason for opening the discussion is that i) for the
>> users
>> > and
>> > >>>>> for
>> > >>>>>>> the
>> > >>>>>>>>>> adoption of the project, it is good to have an explicitely
>> > stated
>> > >>>>>>> policy that implies
>> > >>>>>>>>>> certain guarantees, and ii) keeping code and tests for
>> backwards
>> > >>>>>>> compatibility with
>> > >>>>>>>>>> Flink-1.1 does not offer much. On the contrary, I think that
>> it
>> > >> leads
>> > >>>>>>> to:
>> > >>>>>>>>>>
>> > >>>>>>>>>> 1) dead or ugly code in the codebase, e.g. deprecated class
>> > fields
>> > >>>>>>> that could go away and
>> > >>>>>>>>>> ugly if() loops (see aligned window operators that were
>> > >> deprecated in
>> > >>>>>>> 1.2 and are now
>> > >>>>>>>>>> normal windows), etc
>> > >>>>>>>>>> 2) expensive tests (as, normally, they read from a savepoint)
>> > >>>>>>>>>> 3) binary files in the codebase for holding the
>> aforementioned
>> > >>>>>>> savepoints
>> > >>>>>>>>>>
>> > >>>>>>>>>> My proposal for such a policy would be to offer backwards
>> > >>>>>>> compatibility for one previous version.
>> > >>>>>>>>>>
>> > >>>>>>>>>> This means that 1.3 will be compatible with 1.2 (not 1.1).
>> This
>> > >> still
>> > >>>>>>> allows a clear
>> > >>>>>>>>>> "backwards compatibility" path when jumping versions (a user
>> > that
>> > >>>>> goes
>> > >>>>>>>>>> from 1.1 to 1.3 can go initially 1.1 -> 1.2, take a
>> savepoint,
>> > and
>> > >>>>>>> then 1.2 -> 1.3),
>> > >>>>>>>>>> while also allowing us to clean up the codebase a bit.
>> > >>>>>>>>>>
>> > >>>>>>>>>> What do you think?
>> > >>>>>>>>>>
>> > >>>>>>>>>> Kostas
>> > >>>>>>>>>
>> > >>>>>>>>>
>> > >>>>>>>>
>> > >>>>>>>
>> > >>>>>>>
>> > >>>>>
>> > >>>>>
>> > >>>
>> > >>
>> > >>
>> >
>> >
>>
>
>

Re: [DISCUSS] Backwards compatibility policy.

Posted by Stephan Ewen <se...@apache.org>.
@Sebastian: I am not sure Apache has really guidelines there. So far, I
thought projects establish their own policies.

The compatibility questions here is also one of APIs (code), but of
savepoint forwarding, which is a but different, I think. For example 1.0
and 1.1 were not compatible there, the introduction of backwards
compatibility from 1.2 to 1.1 was a big new thing for 1.2


On Wed, Jun 28, 2017 at 11:53 AM, Sebastian Schelter <
ssc.open@googlemail.com> wrote:

> I haven't closely followed the discussion so far, but isn't it Apache
> policy that major versions should stay backwards compatible to all previous
> releases with the same major version?
>
> -s
>
> 2017-06-28 12:26 GMT+02:00 Kostas Kloudas <k....@data-artisans.com>:
>
> > I agree that 1.1 compatibility is the most important “pain point", as
> > compatibility with the rest of the versions follows a more “systematic”
> > approach.
> >
> > I think that discarding compatibility with 1.1 will clear some parts
> > of the codebase significantly.
> >
> > Kostas
> >
> > > On Jun 27, 2017, at 6:03 PM, Stephan Ewen <se...@apache.org> wrote:
> > >
> > > I think that this discussion is probably motivated especially by the
> > > "legacy state" handling of Flink 1.1.
> > > The biggest gain in codebase and productivity would be won only by
> > dropping
> > > 1.1 compatibility in Flink 1.4.
> > >
> > > My gut feeling is that this is reasonable. We support two versions
> back,
> > > which means that users can skip one upgrade, but not two.
> > >
> > > From what I can tell, users are usually eager to upgrade. They don't do
> > it
> > > immediately, but as soon as the new release is a bit battle tested.
> > >
> > > I would expect skipping two entire versions to be rare enough to be
> okay
> > > with a solution which is a bit more effort for the user:
> > > You can upgrade from Flink 1.1. to 1.4 by loading the 1.1 savepoint
> into
> > > Flink 1.2, take a savepoint (1.2 format), and resume that in Flink 1.4.
> > >
> > > Greetings,
> > > Stephan
> > >
> > >
> > > On Tue, Jun 27, 2017 at 12:01 PM, Stefan Richter <
> > > s.richter@data-artisans.com> wrote:
> > >
> > >> For many parts of the code, I would agree with Aljoscha. However, I
> can
> > >> also see notable exceptions, such as maintaining support for the
> legacy
> > >> state from Flink <=1.1. For example, I think dropping support for this
> > can
> > >> simplify new developments such as fast local recovery or state
> > replication
> > >> quiet a bit because this is a special case that runs through a lot of
> > code
> > >> from backend to JM. So besides this general discussion about a
> backwards
> > >> compatible policy, do you think it could make sense to start another
> > >> concrete discussion about if we still must or want backwards
> > compatibility
> > >> to Flink 1.1 in Flink 1.4?
> > >>
> > >>> Am 29.05.2017 um 12:08 schrieb Aljoscha Krettek <aljoscha@apache.org
> >:
> > >>>
> > >>> Normally, I’m the first one to suggest removing everything that is
> not
> > >> absolutely necessary in order to have a clean code base. On this
> issue,
> > >> though, I think we should support restoring from old Savepoints as far
> > back
> > >> as possible if it does not make the code completely unmaintainable.
> Some
> > >> users might jump versions and always forcing them to go though every
> > >> version from their old version to the current version doesn’t seem
> > feasible
> > >> and might put off some users.
> > >>>
> > >>> So far, I think the burden of supporting restore from 1.1 is still
> > small
> > >> enough and with each new version the changes between versions become
> > less
> > >> and less. The changes from 1.2 to the upcoming 1.3 are quite minimal,
> I
> > >> think.
> > >>>
> > >>> Best,
> > >>> Aljoscha
> > >>>> On 24. May 2017, at 17:58, Ted Yu <yu...@gmail.com> wrote:
> > >>>>
> > >>>> bq. about having LTS versions once a year
> > >>>>
> > >>>> +1 to the above.
> > >>>>
> > >>>> There may be various reasons users don't want to upgrade (after new
> > >>>> releases come out). We should give such users enough flexibility on
> > the
> > >>>> upgrade path.
> > >>>>
> > >>>> Cheers
> > >>>>
> > >>>> On Wed, May 24, 2017 at 8:39 AM, Kostas Kloudas <
> > >> k.kloudas@data-artisans.com
> > >>>>> wrote:
> > >>>>
> > >>>>> Hi all,
> > >>>>>
> > >>>>> For the proposal of having a third party tool, I agree with Ted.
> > >>>>> Maintaining
> > >>>>> it is a big and far from trivial effort.
> > >>>>>
> > >>>>> Now for the window of backwards compatibility, I would argue that
> > even
> > >> if
> > >>>>> for some users 4 months (1 release) is not enough to bump their
> Flink
> > >>>>> version,
> > >>>>> the proposed policy guarantees that there will always be a path
> from
> > >> any
> > >>>>> old
> > >>>>> version to any subsequent one.
> > >>>>>
> > >>>>> Finally, for the proposal about having LTS versions once a year, I
> am
> > >> not
> > >>>>> sure if this will reduce or create more overhead. If I understand
> the
> > >> plan
> > >>>>> correctly, this would mean that the community will have to maintain
> > >>>>> 2 or 3 LTS versions and the last two major ones, right?
> > >>>>>
> > >>>>>> On May 22, 2017, at 7:31 PM, Ted Yu <yu...@gmail.com> wrote:
> > >>>>>>
> > >>>>>> For #2, it is difficult to achieve:
> > >>>>>>
> > >>>>>> a. maintaining savepoint migration is non-trivial and should be
> > >> reviewed
> > >>>>> by
> > >>>>>> domain experts
> > >>>>>> b. how to certify such third-party tool
> > >>>>>>
> > >>>>>> Cheers
> > >>>>>>
> > >>>>>> On Mon, May 22, 2017 at 3:04 AM, 施晓罡 <sh...@gmail.com>
> > wrote:
> > >>>>>>
> > >>>>>>> Hi all,
> > >>>>>>>
> > >>>>>>> Currently, we work a lot in the maintenance of compatibility.
> > >>>>>>> There exist much code in runtime to support the migration of
> > >> savepoints
> > >>>>>>> (most of which are deprecated), making it hard to focus on the
> > >> current
> > >>>>>>> implementation.
> > >>>>>>> When more versions are released, much more efforts will be needed
> > if
> > >> we
> > >>>>>>> try to make these released versions compatible.
> > >>>>>>>
> > >>>>>>> I agree with Tzu-Li that we should provide a method to let users
> > >> upgrade
> > >>>>>>> Flink in a reasonable pace.
> > >>>>>>> But i am against the proposal that we only offer backwards
> > >> compatibility
> > >>>>>>> for one previous version.
> > >>>>>>> According our time-based release model, a major version is
> released
> > >>>>> every
> > >>>>>>> four month.
> > >>>>>>> That means, users have to upgrade their versions every 8 months.
> > >>>>> Otherwise
> > >>>>>>> they will have difficulties in the migration of existing
> > savepoints.
> > >>>>>>>
> > >>>>>>> My suggestions include
> > >>>>>>>
> > >>>>>>> (1) We can release Long-Term Support (LTS) versions which are
> > widely
> > >>>>>>> adopted in other open-source projects.
> > >>>>>>> LTS versions should be stable and are free of found bugs.
> > Savepoints
> > >> in
> > >>>>>>> LTS versions are guaranteed to be back-compatible so that users
> can
> > >>>>> easily
> > >>>>>>> upgrade to newer LTS versions.
> > >>>>>>>
> > >>>>>>> The releasing of LTS versions is slower than that of major
> versions
> > >>>>> (maybe
> > >>>>>>> once a year, determined by users’ upgrade frequency).
> > >>>>>>> Each LTS version will be supported a period of time and typically
> > >> there
> > >>>>>>> are no more than three active LTS versions.
> > >>>>>>> By encouraging users to use LTS versions, we can ease the
> > >> maintenance of
> > >>>>>>> released versions (bug fixes, back compatibility, and critical
> > >>>>> performance
> > >>>>>>> improvement).
> > >>>>>>>
> > >>>>>>> (2) We can provide a third-party tool to do the migration of
> > >>>>> old-versioned
> > >>>>>>> savepoints.
> > >>>>>>> When users upgrade their versions, they can use the provided tool
> > to
> > >>>>>>> migrate existing savepoints.
> > >>>>>>> This can help move the code for savepoint migration out of the
> > actual
> > >>>>>>> codebase,  making code focuses on current implementation.
> > >>>>>>>
> > >>>>>>> What do you think?
> > >>>>>>>
> > >>>>>>> Regards,
> > >>>>>>> Xiaogang
> > >>>>>>>
> > >>>>>>>
> > >>>>>>>> 在 2017年5月22日,下午1:39,Tzu-Li (Gordon) Tai <tz...@apache.org>
> 写道:
> > >>>>>>>>
> > >>>>>>>> Hi Kostas,
> > >>>>>>>>
> > >>>>>>>> Thanks for bringing this up!
> > >>>>>>>> I think it is reasonable to keep this coherent with our
> > timely-based
> > >>>>>>> release model guarantees.
> > >>>>>>>>
> > >>>>>>>> With the timely-based release model, there is a guarantee that
> the
> > >>>>>>> current latest major version and the previous one is supported.
> > >>>>>>>> For example, upon releasing 1.3, only 1.3 and 1.2 will still be
> > >>>>>>> supported by the community for any required bug fixes.
> > >>>>>>>> I think this was initially decided not only to ease old version
> > >>>>>>> maintenance efforts for the community, but also as a means to let
> > >> users
> > >>>>>>> upgrade their Flink versions in a reasonable pace (at least every
> > >> other
> > >>>>>>> major release.)
> > >>>>>>>>
> > >>>>>>>> Therefore, I think its also reasonable to also clearly state
> that
> > >>>>>>> savepoints compatibility will only be guaranteed for the previous
> > >>>>> release.
> > >>>>>>>> Although I think at the moment almost, if not all, of the
> current
> > >> code
> > >>>>>>> still maintains compatibility for 1.1, in the long run these
> > >> migration
> > >>>>>>> codes would definitely start to pile up and pollute the actual
> > >> codebase
> > >>>>> if
> > >>>>>>> we try to always be compatible with all previous versions.
> > >>>>>>>>
> > >>>>>>>> Cheers,
> > >>>>>>>> Gordon
> > >>>>>>>>
> > >>>>>>>>
> > >>>>>>>> On 21 May 2017 at 2:24:53 AM, Kostas Kloudas (
> > >>>>>>> k.kloudas@data-artisans.com) wrote:
> > >>>>>>>>
> > >>>>>>>> Hi Chesnay,
> > >>>>>>>>
> > >>>>>>>> I believe that for APIs we already have a pretty clear policy
> with
> > >> the
> > >>>>>>> annotations.
> > >>>>>>>> I was referring to savepoints and state related backwards
> > >>>>> compatibility.
> > >>>>>>>>
> > >>>>>>>>
> > >>>>>>>>> On May 20, 2017, at 7:20 PM, Chesnay Schepler <
> > chesnay@apache.org>
> > >>>>>>> wrote:
> > >>>>>>>>>
> > >>>>>>>>> I think it would be a good to clarify what kind of
> > >>>>>>> backwards-compatibilitiy we're talking about here. As in are we
> > >> talking
> > >>>>>>> about APIs or savepoints?
> > >>>>>>>>>
> > >>>>>>>>> On 20.05.2017 19:09, Kostas Kloudas wrote:
> > >>>>>>>>>> Hi all,
> > >>>>>>>>>>
> > >>>>>>>>>> As we are getting closer to releasing Flink-1.3, I would like
> to
> > >> open
> > >>>>>>> a discussion
> > >>>>>>>>>> on how far back we provide backwards compatibility for.
> > >>>>>>>>>>
> > >>>>>>>>>> The reason for opening the discussion is that i) for the users
> > and
> > >>>>> for
> > >>>>>>> the
> > >>>>>>>>>> adoption of the project, it is good to have an explicitely
> > stated
> > >>>>>>> policy that implies
> > >>>>>>>>>> certain guarantees, and ii) keeping code and tests for
> backwards
> > >>>>>>> compatibility with
> > >>>>>>>>>> Flink-1.1 does not offer much. On the contrary, I think that
> it
> > >> leads
> > >>>>>>> to:
> > >>>>>>>>>>
> > >>>>>>>>>> 1) dead or ugly code in the codebase, e.g. deprecated class
> > fields
> > >>>>>>> that could go away and
> > >>>>>>>>>> ugly if() loops (see aligned window operators that were
> > >> deprecated in
> > >>>>>>> 1.2 and are now
> > >>>>>>>>>> normal windows), etc
> > >>>>>>>>>> 2) expensive tests (as, normally, they read from a savepoint)
> > >>>>>>>>>> 3) binary files in the codebase for holding the aforementioned
> > >>>>>>> savepoints
> > >>>>>>>>>>
> > >>>>>>>>>> My proposal for such a policy would be to offer backwards
> > >>>>>>> compatibility for one previous version.
> > >>>>>>>>>>
> > >>>>>>>>>> This means that 1.3 will be compatible with 1.2 (not 1.1).
> This
> > >> still
> > >>>>>>> allows a clear
> > >>>>>>>>>> "backwards compatibility" path when jumping versions (a user
> > that
> > >>>>> goes
> > >>>>>>>>>> from 1.1 to 1.3 can go initially 1.1 -> 1.2, take a savepoint,
> > and
> > >>>>>>> then 1.2 -> 1.3),
> > >>>>>>>>>> while also allowing us to clean up the codebase a bit.
> > >>>>>>>>>>
> > >>>>>>>>>> What do you think?
> > >>>>>>>>>>
> > >>>>>>>>>> Kostas
> > >>>>>>>>>
> > >>>>>>>>>
> > >>>>>>>>
> > >>>>>>>
> > >>>>>>>
> > >>>>>
> > >>>>>
> > >>>
> > >>
> > >>
> >
> >
>

Re: [DISCUSS] Backwards compatibility policy.

Posted by Sebastian Schelter <ss...@googlemail.com>.
I haven't closely followed the discussion so far, but isn't it Apache
policy that major versions should stay backwards compatible to all previous
releases with the same major version?

-s

2017-06-28 12:26 GMT+02:00 Kostas Kloudas <k....@data-artisans.com>:

> I agree that 1.1 compatibility is the most important “pain point", as
> compatibility with the rest of the versions follows a more “systematic”
> approach.
>
> I think that discarding compatibility with 1.1 will clear some parts
> of the codebase significantly.
>
> Kostas
>
> > On Jun 27, 2017, at 6:03 PM, Stephan Ewen <se...@apache.org> wrote:
> >
> > I think that this discussion is probably motivated especially by the
> > "legacy state" handling of Flink 1.1.
> > The biggest gain in codebase and productivity would be won only by
> dropping
> > 1.1 compatibility in Flink 1.4.
> >
> > My gut feeling is that this is reasonable. We support two versions back,
> > which means that users can skip one upgrade, but not two.
> >
> > From what I can tell, users are usually eager to upgrade. They don't do
> it
> > immediately, but as soon as the new release is a bit battle tested.
> >
> > I would expect skipping two entire versions to be rare enough to be okay
> > with a solution which is a bit more effort for the user:
> > You can upgrade from Flink 1.1. to 1.4 by loading the 1.1 savepoint into
> > Flink 1.2, take a savepoint (1.2 format), and resume that in Flink 1.4.
> >
> > Greetings,
> > Stephan
> >
> >
> > On Tue, Jun 27, 2017 at 12:01 PM, Stefan Richter <
> > s.richter@data-artisans.com> wrote:
> >
> >> For many parts of the code, I would agree with Aljoscha. However, I can
> >> also see notable exceptions, such as maintaining support for the legacy
> >> state from Flink <=1.1. For example, I think dropping support for this
> can
> >> simplify new developments such as fast local recovery or state
> replication
> >> quiet a bit because this is a special case that runs through a lot of
> code
> >> from backend to JM. So besides this general discussion about a backwards
> >> compatible policy, do you think it could make sense to start another
> >> concrete discussion about if we still must or want backwards
> compatibility
> >> to Flink 1.1 in Flink 1.4?
> >>
> >>> Am 29.05.2017 um 12:08 schrieb Aljoscha Krettek <al...@apache.org>:
> >>>
> >>> Normally, I’m the first one to suggest removing everything that is not
> >> absolutely necessary in order to have a clean code base. On this issue,
> >> though, I think we should support restoring from old Savepoints as far
> back
> >> as possible if it does not make the code completely unmaintainable. Some
> >> users might jump versions and always forcing them to go though every
> >> version from their old version to the current version doesn’t seem
> feasible
> >> and might put off some users.
> >>>
> >>> So far, I think the burden of supporting restore from 1.1 is still
> small
> >> enough and with each new version the changes between versions become
> less
> >> and less. The changes from 1.2 to the upcoming 1.3 are quite minimal, I
> >> think.
> >>>
> >>> Best,
> >>> Aljoscha
> >>>> On 24. May 2017, at 17:58, Ted Yu <yu...@gmail.com> wrote:
> >>>>
> >>>> bq. about having LTS versions once a year
> >>>>
> >>>> +1 to the above.
> >>>>
> >>>> There may be various reasons users don't want to upgrade (after new
> >>>> releases come out). We should give such users enough flexibility on
> the
> >>>> upgrade path.
> >>>>
> >>>> Cheers
> >>>>
> >>>> On Wed, May 24, 2017 at 8:39 AM, Kostas Kloudas <
> >> k.kloudas@data-artisans.com
> >>>>> wrote:
> >>>>
> >>>>> Hi all,
> >>>>>
> >>>>> For the proposal of having a third party tool, I agree with Ted.
> >>>>> Maintaining
> >>>>> it is a big and far from trivial effort.
> >>>>>
> >>>>> Now for the window of backwards compatibility, I would argue that
> even
> >> if
> >>>>> for some users 4 months (1 release) is not enough to bump their Flink
> >>>>> version,
> >>>>> the proposed policy guarantees that there will always be a path from
> >> any
> >>>>> old
> >>>>> version to any subsequent one.
> >>>>>
> >>>>> Finally, for the proposal about having LTS versions once a year, I am
> >> not
> >>>>> sure if this will reduce or create more overhead. If I understand the
> >> plan
> >>>>> correctly, this would mean that the community will have to maintain
> >>>>> 2 or 3 LTS versions and the last two major ones, right?
> >>>>>
> >>>>>> On May 22, 2017, at 7:31 PM, Ted Yu <yu...@gmail.com> wrote:
> >>>>>>
> >>>>>> For #2, it is difficult to achieve:
> >>>>>>
> >>>>>> a. maintaining savepoint migration is non-trivial and should be
> >> reviewed
> >>>>> by
> >>>>>> domain experts
> >>>>>> b. how to certify such third-party tool
> >>>>>>
> >>>>>> Cheers
> >>>>>>
> >>>>>> On Mon, May 22, 2017 at 3:04 AM, 施晓罡 <sh...@gmail.com>
> wrote:
> >>>>>>
> >>>>>>> Hi all,
> >>>>>>>
> >>>>>>> Currently, we work a lot in the maintenance of compatibility.
> >>>>>>> There exist much code in runtime to support the migration of
> >> savepoints
> >>>>>>> (most of which are deprecated), making it hard to focus on the
> >> current
> >>>>>>> implementation.
> >>>>>>> When more versions are released, much more efforts will be needed
> if
> >> we
> >>>>>>> try to make these released versions compatible.
> >>>>>>>
> >>>>>>> I agree with Tzu-Li that we should provide a method to let users
> >> upgrade
> >>>>>>> Flink in a reasonable pace.
> >>>>>>> But i am against the proposal that we only offer backwards
> >> compatibility
> >>>>>>> for one previous version.
> >>>>>>> According our time-based release model, a major version is released
> >>>>> every
> >>>>>>> four month.
> >>>>>>> That means, users have to upgrade their versions every 8 months.
> >>>>> Otherwise
> >>>>>>> they will have difficulties in the migration of existing
> savepoints.
> >>>>>>>
> >>>>>>> My suggestions include
> >>>>>>>
> >>>>>>> (1) We can release Long-Term Support (LTS) versions which are
> widely
> >>>>>>> adopted in other open-source projects.
> >>>>>>> LTS versions should be stable and are free of found bugs.
> Savepoints
> >> in
> >>>>>>> LTS versions are guaranteed to be back-compatible so that users can
> >>>>> easily
> >>>>>>> upgrade to newer LTS versions.
> >>>>>>>
> >>>>>>> The releasing of LTS versions is slower than that of major versions
> >>>>> (maybe
> >>>>>>> once a year, determined by users’ upgrade frequency).
> >>>>>>> Each LTS version will be supported a period of time and typically
> >> there
> >>>>>>> are no more than three active LTS versions.
> >>>>>>> By encouraging users to use LTS versions, we can ease the
> >> maintenance of
> >>>>>>> released versions (bug fixes, back compatibility, and critical
> >>>>> performance
> >>>>>>> improvement).
> >>>>>>>
> >>>>>>> (2) We can provide a third-party tool to do the migration of
> >>>>> old-versioned
> >>>>>>> savepoints.
> >>>>>>> When users upgrade their versions, they can use the provided tool
> to
> >>>>>>> migrate existing savepoints.
> >>>>>>> This can help move the code for savepoint migration out of the
> actual
> >>>>>>> codebase,  making code focuses on current implementation.
> >>>>>>>
> >>>>>>> What do you think?
> >>>>>>>
> >>>>>>> Regards,
> >>>>>>> Xiaogang
> >>>>>>>
> >>>>>>>
> >>>>>>>> 在 2017年5月22日,下午1:39,Tzu-Li (Gordon) Tai <tz...@apache.org> 写道:
> >>>>>>>>
> >>>>>>>> Hi Kostas,
> >>>>>>>>
> >>>>>>>> Thanks for bringing this up!
> >>>>>>>> I think it is reasonable to keep this coherent with our
> timely-based
> >>>>>>> release model guarantees.
> >>>>>>>>
> >>>>>>>> With the timely-based release model, there is a guarantee that the
> >>>>>>> current latest major version and the previous one is supported.
> >>>>>>>> For example, upon releasing 1.3, only 1.3 and 1.2 will still be
> >>>>>>> supported by the community for any required bug fixes.
> >>>>>>>> I think this was initially decided not only to ease old version
> >>>>>>> maintenance efforts for the community, but also as a means to let
> >> users
> >>>>>>> upgrade their Flink versions in a reasonable pace (at least every
> >> other
> >>>>>>> major release.)
> >>>>>>>>
> >>>>>>>> Therefore, I think its also reasonable to also clearly state that
> >>>>>>> savepoints compatibility will only be guaranteed for the previous
> >>>>> release.
> >>>>>>>> Although I think at the moment almost, if not all, of the current
> >> code
> >>>>>>> still maintains compatibility for 1.1, in the long run these
> >> migration
> >>>>>>> codes would definitely start to pile up and pollute the actual
> >> codebase
> >>>>> if
> >>>>>>> we try to always be compatible with all previous versions.
> >>>>>>>>
> >>>>>>>> Cheers,
> >>>>>>>> Gordon
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> On 21 May 2017 at 2:24:53 AM, Kostas Kloudas (
> >>>>>>> k.kloudas@data-artisans.com) wrote:
> >>>>>>>>
> >>>>>>>> Hi Chesnay,
> >>>>>>>>
> >>>>>>>> I believe that for APIs we already have a pretty clear policy with
> >> the
> >>>>>>> annotations.
> >>>>>>>> I was referring to savepoints and state related backwards
> >>>>> compatibility.
> >>>>>>>>
> >>>>>>>>
> >>>>>>>>> On May 20, 2017, at 7:20 PM, Chesnay Schepler <
> chesnay@apache.org>
> >>>>>>> wrote:
> >>>>>>>>>
> >>>>>>>>> I think it would be a good to clarify what kind of
> >>>>>>> backwards-compatibilitiy we're talking about here. As in are we
> >> talking
> >>>>>>> about APIs or savepoints?
> >>>>>>>>>
> >>>>>>>>> On 20.05.2017 19:09, Kostas Kloudas wrote:
> >>>>>>>>>> Hi all,
> >>>>>>>>>>
> >>>>>>>>>> As we are getting closer to releasing Flink-1.3, I would like to
> >> open
> >>>>>>> a discussion
> >>>>>>>>>> on how far back we provide backwards compatibility for.
> >>>>>>>>>>
> >>>>>>>>>> The reason for opening the discussion is that i) for the users
> and
> >>>>> for
> >>>>>>> the
> >>>>>>>>>> adoption of the project, it is good to have an explicitely
> stated
> >>>>>>> policy that implies
> >>>>>>>>>> certain guarantees, and ii) keeping code and tests for backwards
> >>>>>>> compatibility with
> >>>>>>>>>> Flink-1.1 does not offer much. On the contrary, I think that it
> >> leads
> >>>>>>> to:
> >>>>>>>>>>
> >>>>>>>>>> 1) dead or ugly code in the codebase, e.g. deprecated class
> fields
> >>>>>>> that could go away and
> >>>>>>>>>> ugly if() loops (see aligned window operators that were
> >> deprecated in
> >>>>>>> 1.2 and are now
> >>>>>>>>>> normal windows), etc
> >>>>>>>>>> 2) expensive tests (as, normally, they read from a savepoint)
> >>>>>>>>>> 3) binary files in the codebase for holding the aforementioned
> >>>>>>> savepoints
> >>>>>>>>>>
> >>>>>>>>>> My proposal for such a policy would be to offer backwards
> >>>>>>> compatibility for one previous version.
> >>>>>>>>>>
> >>>>>>>>>> This means that 1.3 will be compatible with 1.2 (not 1.1). This
> >> still
> >>>>>>> allows a clear
> >>>>>>>>>> "backwards compatibility" path when jumping versions (a user
> that
> >>>>> goes
> >>>>>>>>>> from 1.1 to 1.3 can go initially 1.1 -> 1.2, take a savepoint,
> and
> >>>>>>> then 1.2 -> 1.3),
> >>>>>>>>>> while also allowing us to clean up the codebase a bit.
> >>>>>>>>>>
> >>>>>>>>>> What do you think?
> >>>>>>>>>>
> >>>>>>>>>> Kostas
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>>>
> >>>>>
> >>>>>
> >>>
> >>
> >>
>
>

Re: [DISCUSS] Backwards compatibility policy.

Posted by Kostas Kloudas <k....@data-artisans.com>.
I agree that 1.1 compatibility is the most important “pain point", as 
compatibility with the rest of the versions follows a more “systematic” approach.

I think that discarding compatibility with 1.1 will clear some parts 
of the codebase significantly.

Kostas

> On Jun 27, 2017, at 6:03 PM, Stephan Ewen <se...@apache.org> wrote:
> 
> I think that this discussion is probably motivated especially by the
> "legacy state" handling of Flink 1.1.
> The biggest gain in codebase and productivity would be won only by dropping
> 1.1 compatibility in Flink 1.4.
> 
> My gut feeling is that this is reasonable. We support two versions back,
> which means that users can skip one upgrade, but not two.
> 
> From what I can tell, users are usually eager to upgrade. They don't do it
> immediately, but as soon as the new release is a bit battle tested.
> 
> I would expect skipping two entire versions to be rare enough to be okay
> with a solution which is a bit more effort for the user:
> You can upgrade from Flink 1.1. to 1.4 by loading the 1.1 savepoint into
> Flink 1.2, take a savepoint (1.2 format), and resume that in Flink 1.4.
> 
> Greetings,
> Stephan
> 
> 
> On Tue, Jun 27, 2017 at 12:01 PM, Stefan Richter <
> s.richter@data-artisans.com> wrote:
> 
>> For many parts of the code, I would agree with Aljoscha. However, I can
>> also see notable exceptions, such as maintaining support for the legacy
>> state from Flink <=1.1. For example, I think dropping support for this can
>> simplify new developments such as fast local recovery or state replication
>> quiet a bit because this is a special case that runs through a lot of code
>> from backend to JM. So besides this general discussion about a backwards
>> compatible policy, do you think it could make sense to start another
>> concrete discussion about if we still must or want backwards compatibility
>> to Flink 1.1 in Flink 1.4?
>> 
>>> Am 29.05.2017 um 12:08 schrieb Aljoscha Krettek <al...@apache.org>:
>>> 
>>> Normally, I’m the first one to suggest removing everything that is not
>> absolutely necessary in order to have a clean code base. On this issue,
>> though, I think we should support restoring from old Savepoints as far back
>> as possible if it does not make the code completely unmaintainable. Some
>> users might jump versions and always forcing them to go though every
>> version from their old version to the current version doesn’t seem feasible
>> and might put off some users.
>>> 
>>> So far, I think the burden of supporting restore from 1.1 is still small
>> enough and with each new version the changes between versions become less
>> and less. The changes from 1.2 to the upcoming 1.3 are quite minimal, I
>> think.
>>> 
>>> Best,
>>> Aljoscha
>>>> On 24. May 2017, at 17:58, Ted Yu <yu...@gmail.com> wrote:
>>>> 
>>>> bq. about having LTS versions once a year
>>>> 
>>>> +1 to the above.
>>>> 
>>>> There may be various reasons users don't want to upgrade (after new
>>>> releases come out). We should give such users enough flexibility on the
>>>> upgrade path.
>>>> 
>>>> Cheers
>>>> 
>>>> On Wed, May 24, 2017 at 8:39 AM, Kostas Kloudas <
>> k.kloudas@data-artisans.com
>>>>> wrote:
>>>> 
>>>>> Hi all,
>>>>> 
>>>>> For the proposal of having a third party tool, I agree with Ted.
>>>>> Maintaining
>>>>> it is a big and far from trivial effort.
>>>>> 
>>>>> Now for the window of backwards compatibility, I would argue that even
>> if
>>>>> for some users 4 months (1 release) is not enough to bump their Flink
>>>>> version,
>>>>> the proposed policy guarantees that there will always be a path from
>> any
>>>>> old
>>>>> version to any subsequent one.
>>>>> 
>>>>> Finally, for the proposal about having LTS versions once a year, I am
>> not
>>>>> sure if this will reduce or create more overhead. If I understand the
>> plan
>>>>> correctly, this would mean that the community will have to maintain
>>>>> 2 or 3 LTS versions and the last two major ones, right?
>>>>> 
>>>>>> On May 22, 2017, at 7:31 PM, Ted Yu <yu...@gmail.com> wrote:
>>>>>> 
>>>>>> For #2, it is difficult to achieve:
>>>>>> 
>>>>>> a. maintaining savepoint migration is non-trivial and should be
>> reviewed
>>>>> by
>>>>>> domain experts
>>>>>> b. how to certify such third-party tool
>>>>>> 
>>>>>> Cheers
>>>>>> 
>>>>>> On Mon, May 22, 2017 at 3:04 AM, 施晓罡 <sh...@gmail.com> wrote:
>>>>>> 
>>>>>>> Hi all,
>>>>>>> 
>>>>>>> Currently, we work a lot in the maintenance of compatibility.
>>>>>>> There exist much code in runtime to support the migration of
>> savepoints
>>>>>>> (most of which are deprecated), making it hard to focus on the
>> current
>>>>>>> implementation.
>>>>>>> When more versions are released, much more efforts will be needed if
>> we
>>>>>>> try to make these released versions compatible.
>>>>>>> 
>>>>>>> I agree with Tzu-Li that we should provide a method to let users
>> upgrade
>>>>>>> Flink in a reasonable pace.
>>>>>>> But i am against the proposal that we only offer backwards
>> compatibility
>>>>>>> for one previous version.
>>>>>>> According our time-based release model, a major version is released
>>>>> every
>>>>>>> four month.
>>>>>>> That means, users have to upgrade their versions every 8 months.
>>>>> Otherwise
>>>>>>> they will have difficulties in the migration of existing savepoints.
>>>>>>> 
>>>>>>> My suggestions include
>>>>>>> 
>>>>>>> (1) We can release Long-Term Support (LTS) versions which are widely
>>>>>>> adopted in other open-source projects.
>>>>>>> LTS versions should be stable and are free of found bugs. Savepoints
>> in
>>>>>>> LTS versions are guaranteed to be back-compatible so that users can
>>>>> easily
>>>>>>> upgrade to newer LTS versions.
>>>>>>> 
>>>>>>> The releasing of LTS versions is slower than that of major versions
>>>>> (maybe
>>>>>>> once a year, determined by users’ upgrade frequency).
>>>>>>> Each LTS version will be supported a period of time and typically
>> there
>>>>>>> are no more than three active LTS versions.
>>>>>>> By encouraging users to use LTS versions, we can ease the
>> maintenance of
>>>>>>> released versions (bug fixes, back compatibility, and critical
>>>>> performance
>>>>>>> improvement).
>>>>>>> 
>>>>>>> (2) We can provide a third-party tool to do the migration of
>>>>> old-versioned
>>>>>>> savepoints.
>>>>>>> When users upgrade their versions, they can use the provided tool to
>>>>>>> migrate existing savepoints.
>>>>>>> This can help move the code for savepoint migration out of the actual
>>>>>>> codebase,  making code focuses on current implementation.
>>>>>>> 
>>>>>>> What do you think?
>>>>>>> 
>>>>>>> Regards,
>>>>>>> Xiaogang
>>>>>>> 
>>>>>>> 
>>>>>>>> 在 2017年5月22日,下午1:39,Tzu-Li (Gordon) Tai <tz...@apache.org> 写道:
>>>>>>>> 
>>>>>>>> Hi Kostas,
>>>>>>>> 
>>>>>>>> Thanks for bringing this up!
>>>>>>>> I think it is reasonable to keep this coherent with our timely-based
>>>>>>> release model guarantees.
>>>>>>>> 
>>>>>>>> With the timely-based release model, there is a guarantee that the
>>>>>>> current latest major version and the previous one is supported.
>>>>>>>> For example, upon releasing 1.3, only 1.3 and 1.2 will still be
>>>>>>> supported by the community for any required bug fixes.
>>>>>>>> I think this was initially decided not only to ease old version
>>>>>>> maintenance efforts for the community, but also as a means to let
>> users
>>>>>>> upgrade their Flink versions in a reasonable pace (at least every
>> other
>>>>>>> major release.)
>>>>>>>> 
>>>>>>>> Therefore, I think its also reasonable to also clearly state that
>>>>>>> savepoints compatibility will only be guaranteed for the previous
>>>>> release.
>>>>>>>> Although I think at the moment almost, if not all, of the current
>> code
>>>>>>> still maintains compatibility for 1.1, in the long run these
>> migration
>>>>>>> codes would definitely start to pile up and pollute the actual
>> codebase
>>>>> if
>>>>>>> we try to always be compatible with all previous versions.
>>>>>>>> 
>>>>>>>> Cheers,
>>>>>>>> Gordon
>>>>>>>> 
>>>>>>>> 
>>>>>>>> On 21 May 2017 at 2:24:53 AM, Kostas Kloudas (
>>>>>>> k.kloudas@data-artisans.com) wrote:
>>>>>>>> 
>>>>>>>> Hi Chesnay,
>>>>>>>> 
>>>>>>>> I believe that for APIs we already have a pretty clear policy with
>> the
>>>>>>> annotations.
>>>>>>>> I was referring to savepoints and state related backwards
>>>>> compatibility.
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> On May 20, 2017, at 7:20 PM, Chesnay Schepler <ch...@apache.org>
>>>>>>> wrote:
>>>>>>>>> 
>>>>>>>>> I think it would be a good to clarify what kind of
>>>>>>> backwards-compatibilitiy we're talking about here. As in are we
>> talking
>>>>>>> about APIs or savepoints?
>>>>>>>>> 
>>>>>>>>> On 20.05.2017 19:09, Kostas Kloudas wrote:
>>>>>>>>>> Hi all,
>>>>>>>>>> 
>>>>>>>>>> As we are getting closer to releasing Flink-1.3, I would like to
>> open
>>>>>>> a discussion
>>>>>>>>>> on how far back we provide backwards compatibility for.
>>>>>>>>>> 
>>>>>>>>>> The reason for opening the discussion is that i) for the users and
>>>>> for
>>>>>>> the
>>>>>>>>>> adoption of the project, it is good to have an explicitely stated
>>>>>>> policy that implies
>>>>>>>>>> certain guarantees, and ii) keeping code and tests for backwards
>>>>>>> compatibility with
>>>>>>>>>> Flink-1.1 does not offer much. On the contrary, I think that it
>> leads
>>>>>>> to:
>>>>>>>>>> 
>>>>>>>>>> 1) dead or ugly code in the codebase, e.g. deprecated class fields
>>>>>>> that could go away and
>>>>>>>>>> ugly if() loops (see aligned window operators that were
>> deprecated in
>>>>>>> 1.2 and are now
>>>>>>>>>> normal windows), etc
>>>>>>>>>> 2) expensive tests (as, normally, they read from a savepoint)
>>>>>>>>>> 3) binary files in the codebase for holding the aforementioned
>>>>>>> savepoints
>>>>>>>>>> 
>>>>>>>>>> My proposal for such a policy would be to offer backwards
>>>>>>> compatibility for one previous version.
>>>>>>>>>> 
>>>>>>>>>> This means that 1.3 will be compatible with 1.2 (not 1.1). This
>> still
>>>>>>> allows a clear
>>>>>>>>>> "backwards compatibility" path when jumping versions (a user that
>>>>> goes
>>>>>>>>>> from 1.1 to 1.3 can go initially 1.1 -> 1.2, take a savepoint, and
>>>>>>> then 1.2 -> 1.3),
>>>>>>>>>> while also allowing us to clean up the codebase a bit.
>>>>>>>>>> 
>>>>>>>>>> What do you think?
>>>>>>>>>> 
>>>>>>>>>> Kostas
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>> 
>>>>> 
>>> 
>> 
>> 


Re: [DISCUSS] Backwards compatibility policy.

Posted by Stephan Ewen <se...@apache.org>.
I think that this discussion is probably motivated especially by the
"legacy state" handling of Flink 1.1.
The biggest gain in codebase and productivity would be won only by dropping
1.1 compatibility in Flink 1.4.

My gut feeling is that this is reasonable. We support two versions back,
which means that users can skip one upgrade, but not two.

From what I can tell, users are usually eager to upgrade. They don't do it
immediately, but as soon as the new release is a bit battle tested.

I would expect skipping two entire versions to be rare enough to be okay
with a solution which is a bit more effort for the user:
You can upgrade from Flink 1.1. to 1.4 by loading the 1.1 savepoint into
Flink 1.2, take a savepoint (1.2 format), and resume that in Flink 1.4.

Greetings,
Stephan


On Tue, Jun 27, 2017 at 12:01 PM, Stefan Richter <
s.richter@data-artisans.com> wrote:

> For many parts of the code, I would agree with Aljoscha. However, I can
> also see notable exceptions, such as maintaining support for the legacy
> state from Flink <=1.1. For example, I think dropping support for this can
> simplify new developments such as fast local recovery or state replication
> quiet a bit because this is a special case that runs through a lot of code
> from backend to JM. So besides this general discussion about a backwards
> compatible policy, do you think it could make sense to start another
> concrete discussion about if we still must or want backwards compatibility
> to Flink 1.1 in Flink 1.4?
>
> > Am 29.05.2017 um 12:08 schrieb Aljoscha Krettek <al...@apache.org>:
> >
> > Normally, I’m the first one to suggest removing everything that is not
> absolutely necessary in order to have a clean code base. On this issue,
> though, I think we should support restoring from old Savepoints as far back
> as possible if it does not make the code completely unmaintainable. Some
> users might jump versions and always forcing them to go though every
> version from their old version to the current version doesn’t seem feasible
> and might put off some users.
> >
> > So far, I think the burden of supporting restore from 1.1 is still small
> enough and with each new version the changes between versions become less
> and less. The changes from 1.2 to the upcoming 1.3 are quite minimal, I
> think.
> >
> > Best,
> > Aljoscha
> >> On 24. May 2017, at 17:58, Ted Yu <yu...@gmail.com> wrote:
> >>
> >> bq. about having LTS versions once a year
> >>
> >> +1 to the above.
> >>
> >> There may be various reasons users don't want to upgrade (after new
> >> releases come out). We should give such users enough flexibility on the
> >> upgrade path.
> >>
> >> Cheers
> >>
> >> On Wed, May 24, 2017 at 8:39 AM, Kostas Kloudas <
> k.kloudas@data-artisans.com
> >>> wrote:
> >>
> >>> Hi all,
> >>>
> >>> For the proposal of having a third party tool, I agree with Ted.
> >>> Maintaining
> >>> it is a big and far from trivial effort.
> >>>
> >>> Now for the window of backwards compatibility, I would argue that even
> if
> >>> for some users 4 months (1 release) is not enough to bump their Flink
> >>> version,
> >>> the proposed policy guarantees that there will always be a path from
> any
> >>> old
> >>> version to any subsequent one.
> >>>
> >>> Finally, for the proposal about having LTS versions once a year, I am
> not
> >>> sure if this will reduce or create more overhead. If I understand the
> plan
> >>> correctly, this would mean that the community will have to maintain
> >>> 2 or 3 LTS versions and the last two major ones, right?
> >>>
> >>>> On May 22, 2017, at 7:31 PM, Ted Yu <yu...@gmail.com> wrote:
> >>>>
> >>>> For #2, it is difficult to achieve:
> >>>>
> >>>> a. maintaining savepoint migration is non-trivial and should be
> reviewed
> >>> by
> >>>> domain experts
> >>>> b. how to certify such third-party tool
> >>>>
> >>>> Cheers
> >>>>
> >>>> On Mon, May 22, 2017 at 3:04 AM, 施晓罡 <sh...@gmail.com> wrote:
> >>>>
> >>>>> Hi all,
> >>>>>
> >>>>> Currently, we work a lot in the maintenance of compatibility.
> >>>>> There exist much code in runtime to support the migration of
> savepoints
> >>>>> (most of which are deprecated), making it hard to focus on the
> current
> >>>>> implementation.
> >>>>> When more versions are released, much more efforts will be needed if
> we
> >>>>> try to make these released versions compatible.
> >>>>>
> >>>>> I agree with Tzu-Li that we should provide a method to let users
> upgrade
> >>>>> Flink in a reasonable pace.
> >>>>> But i am against the proposal that we only offer backwards
> compatibility
> >>>>> for one previous version.
> >>>>> According our time-based release model, a major version is released
> >>> every
> >>>>> four month.
> >>>>> That means, users have to upgrade their versions every 8 months.
> >>> Otherwise
> >>>>> they will have difficulties in the migration of existing savepoints.
> >>>>>
> >>>>> My suggestions include
> >>>>>
> >>>>> (1) We can release Long-Term Support (LTS) versions which are widely
> >>>>> adopted in other open-source projects.
> >>>>> LTS versions should be stable and are free of found bugs. Savepoints
> in
> >>>>> LTS versions are guaranteed to be back-compatible so that users can
> >>> easily
> >>>>> upgrade to newer LTS versions.
> >>>>>
> >>>>> The releasing of LTS versions is slower than that of major versions
> >>> (maybe
> >>>>> once a year, determined by users’ upgrade frequency).
> >>>>> Each LTS version will be supported a period of time and typically
> there
> >>>>> are no more than three active LTS versions.
> >>>>> By encouraging users to use LTS versions, we can ease the
> maintenance of
> >>>>> released versions (bug fixes, back compatibility, and critical
> >>> performance
> >>>>> improvement).
> >>>>>
> >>>>> (2) We can provide a third-party tool to do the migration of
> >>> old-versioned
> >>>>> savepoints.
> >>>>> When users upgrade their versions, they can use the provided tool to
> >>>>> migrate existing savepoints.
> >>>>> This can help move the code for savepoint migration out of the actual
> >>>>> codebase,  making code focuses on current implementation.
> >>>>>
> >>>>> What do you think?
> >>>>>
> >>>>> Regards,
> >>>>> Xiaogang
> >>>>>
> >>>>>
> >>>>>> 在 2017年5月22日,下午1:39,Tzu-Li (Gordon) Tai <tz...@apache.org> 写道:
> >>>>>>
> >>>>>> Hi Kostas,
> >>>>>>
> >>>>>> Thanks for bringing this up!
> >>>>>> I think it is reasonable to keep this coherent with our timely-based
> >>>>> release model guarantees.
> >>>>>>
> >>>>>> With the timely-based release model, there is a guarantee that the
> >>>>> current latest major version and the previous one is supported.
> >>>>>> For example, upon releasing 1.3, only 1.3 and 1.2 will still be
> >>>>> supported by the community for any required bug fixes.
> >>>>>> I think this was initially decided not only to ease old version
> >>>>> maintenance efforts for the community, but also as a means to let
> users
> >>>>> upgrade their Flink versions in a reasonable pace (at least every
> other
> >>>>> major release.)
> >>>>>>
> >>>>>> Therefore, I think its also reasonable to also clearly state that
> >>>>> savepoints compatibility will only be guaranteed for the previous
> >>> release.
> >>>>>> Although I think at the moment almost, if not all, of the current
> code
> >>>>> still maintains compatibility for 1.1, in the long run these
> migration
> >>>>> codes would definitely start to pile up and pollute the actual
> codebase
> >>> if
> >>>>> we try to always be compatible with all previous versions.
> >>>>>>
> >>>>>> Cheers,
> >>>>>> Gordon
> >>>>>>
> >>>>>>
> >>>>>> On 21 May 2017 at 2:24:53 AM, Kostas Kloudas (
> >>>>> k.kloudas@data-artisans.com) wrote:
> >>>>>>
> >>>>>> Hi Chesnay,
> >>>>>>
> >>>>>> I believe that for APIs we already have a pretty clear policy with
> the
> >>>>> annotations.
> >>>>>> I was referring to savepoints and state related backwards
> >>> compatibility.
> >>>>>>
> >>>>>>
> >>>>>>> On May 20, 2017, at 7:20 PM, Chesnay Schepler <ch...@apache.org>
> >>>>> wrote:
> >>>>>>>
> >>>>>>> I think it would be a good to clarify what kind of
> >>>>> backwards-compatibilitiy we're talking about here. As in are we
> talking
> >>>>> about APIs or savepoints?
> >>>>>>>
> >>>>>>> On 20.05.2017 19:09, Kostas Kloudas wrote:
> >>>>>>>> Hi all,
> >>>>>>>>
> >>>>>>>> As we are getting closer to releasing Flink-1.3, I would like to
> open
> >>>>> a discussion
> >>>>>>>> on how far back we provide backwards compatibility for.
> >>>>>>>>
> >>>>>>>> The reason for opening the discussion is that i) for the users and
> >>> for
> >>>>> the
> >>>>>>>> adoption of the project, it is good to have an explicitely stated
> >>>>> policy that implies
> >>>>>>>> certain guarantees, and ii) keeping code and tests for backwards
> >>>>> compatibility with
> >>>>>>>> Flink-1.1 does not offer much. On the contrary, I think that it
> leads
> >>>>> to:
> >>>>>>>>
> >>>>>>>> 1) dead or ugly code in the codebase, e.g. deprecated class fields
> >>>>> that could go away and
> >>>>>>>> ugly if() loops (see aligned window operators that were
> deprecated in
> >>>>> 1.2 and are now
> >>>>>>>> normal windows), etc
> >>>>>>>> 2) expensive tests (as, normally, they read from a savepoint)
> >>>>>>>> 3) binary files in the codebase for holding the aforementioned
> >>>>> savepoints
> >>>>>>>>
> >>>>>>>> My proposal for such a policy would be to offer backwards
> >>>>> compatibility for one previous version.
> >>>>>>>>
> >>>>>>>> This means that 1.3 will be compatible with 1.2 (not 1.1). This
> still
> >>>>> allows a clear
> >>>>>>>> "backwards compatibility" path when jumping versions (a user that
> >>> goes
> >>>>>>>> from 1.1 to 1.3 can go initially 1.1 -> 1.2, take a savepoint, and
> >>>>> then 1.2 -> 1.3),
> >>>>>>>> while also allowing us to clean up the codebase a bit.
> >>>>>>>>
> >>>>>>>> What do you think?
> >>>>>>>>
> >>>>>>>> Kostas
> >>>>>>>
> >>>>>>>
> >>>>>>
> >>>>>
> >>>>>
> >>>
> >>>
> >
>
>

Re: [DISCUSS] Backwards compatibility policy.

Posted by Stefan Richter <s....@data-artisans.com>.
For many parts of the code, I would agree with Aljoscha. However, I can also see notable exceptions, such as maintaining support for the legacy state from Flink <=1.1. For example, I think dropping support for this can simplify new developments such as fast local recovery or state replication quiet a bit because this is a special case that runs through a lot of code from backend to JM. So besides this general discussion about a backwards compatible policy, do you think it could make sense to start another concrete discussion about if we still must or want backwards compatibility to Flink 1.1 in Flink 1.4?
 
> Am 29.05.2017 um 12:08 schrieb Aljoscha Krettek <al...@apache.org>:
> 
> Normally, I’m the first one to suggest removing everything that is not absolutely necessary in order to have a clean code base. On this issue, though, I think we should support restoring from old Savepoints as far back as possible if it does not make the code completely unmaintainable. Some users might jump versions and always forcing them to go though every version from their old version to the current version doesn’t seem feasible and might put off some users.
> 
> So far, I think the burden of supporting restore from 1.1 is still small enough and with each new version the changes between versions become less and less. The changes from 1.2 to the upcoming 1.3 are quite minimal, I think.
> 
> Best,
> Aljoscha
>> On 24. May 2017, at 17:58, Ted Yu <yu...@gmail.com> wrote:
>> 
>> bq. about having LTS versions once a year
>> 
>> +1 to the above.
>> 
>> There may be various reasons users don't want to upgrade (after new
>> releases come out). We should give such users enough flexibility on the
>> upgrade path.
>> 
>> Cheers
>> 
>> On Wed, May 24, 2017 at 8:39 AM, Kostas Kloudas <k.kloudas@data-artisans.com
>>> wrote:
>> 
>>> Hi all,
>>> 
>>> For the proposal of having a third party tool, I agree with Ted.
>>> Maintaining
>>> it is a big and far from trivial effort.
>>> 
>>> Now for the window of backwards compatibility, I would argue that even if
>>> for some users 4 months (1 release) is not enough to bump their Flink
>>> version,
>>> the proposed policy guarantees that there will always be a path from any
>>> old
>>> version to any subsequent one.
>>> 
>>> Finally, for the proposal about having LTS versions once a year, I am not
>>> sure if this will reduce or create more overhead. If I understand the plan
>>> correctly, this would mean that the community will have to maintain
>>> 2 or 3 LTS versions and the last two major ones, right?
>>> 
>>>> On May 22, 2017, at 7:31 PM, Ted Yu <yu...@gmail.com> wrote:
>>>> 
>>>> For #2, it is difficult to achieve:
>>>> 
>>>> a. maintaining savepoint migration is non-trivial and should be reviewed
>>> by
>>>> domain experts
>>>> b. how to certify such third-party tool
>>>> 
>>>> Cheers
>>>> 
>>>> On Mon, May 22, 2017 at 3:04 AM, 施晓罡 <sh...@gmail.com> wrote:
>>>> 
>>>>> Hi all,
>>>>> 
>>>>> Currently, we work a lot in the maintenance of compatibility.
>>>>> There exist much code in runtime to support the migration of savepoints
>>>>> (most of which are deprecated), making it hard to focus on the current
>>>>> implementation.
>>>>> When more versions are released, much more efforts will be needed if we
>>>>> try to make these released versions compatible.
>>>>> 
>>>>> I agree with Tzu-Li that we should provide a method to let users upgrade
>>>>> Flink in a reasonable pace.
>>>>> But i am against the proposal that we only offer backwards compatibility
>>>>> for one previous version.
>>>>> According our time-based release model, a major version is released
>>> every
>>>>> four month.
>>>>> That means, users have to upgrade their versions every 8 months.
>>> Otherwise
>>>>> they will have difficulties in the migration of existing savepoints.
>>>>> 
>>>>> My suggestions include
>>>>> 
>>>>> (1) We can release Long-Term Support (LTS) versions which are widely
>>>>> adopted in other open-source projects.
>>>>> LTS versions should be stable and are free of found bugs. Savepoints in
>>>>> LTS versions are guaranteed to be back-compatible so that users can
>>> easily
>>>>> upgrade to newer LTS versions.
>>>>> 
>>>>> The releasing of LTS versions is slower than that of major versions
>>> (maybe
>>>>> once a year, determined by users’ upgrade frequency).
>>>>> Each LTS version will be supported a period of time and typically there
>>>>> are no more than three active LTS versions.
>>>>> By encouraging users to use LTS versions, we can ease the maintenance of
>>>>> released versions (bug fixes, back compatibility, and critical
>>> performance
>>>>> improvement).
>>>>> 
>>>>> (2) We can provide a third-party tool to do the migration of
>>> old-versioned
>>>>> savepoints.
>>>>> When users upgrade their versions, they can use the provided tool to
>>>>> migrate existing savepoints.
>>>>> This can help move the code for savepoint migration out of the actual
>>>>> codebase,  making code focuses on current implementation.
>>>>> 
>>>>> What do you think?
>>>>> 
>>>>> Regards,
>>>>> Xiaogang
>>>>> 
>>>>> 
>>>>>> 在 2017年5月22日,下午1:39,Tzu-Li (Gordon) Tai <tz...@apache.org> 写道:
>>>>>> 
>>>>>> Hi Kostas,
>>>>>> 
>>>>>> Thanks for bringing this up!
>>>>>> I think it is reasonable to keep this coherent with our timely-based
>>>>> release model guarantees.
>>>>>> 
>>>>>> With the timely-based release model, there is a guarantee that the
>>>>> current latest major version and the previous one is supported.
>>>>>> For example, upon releasing 1.3, only 1.3 and 1.2 will still be
>>>>> supported by the community for any required bug fixes.
>>>>>> I think this was initially decided not only to ease old version
>>>>> maintenance efforts for the community, but also as a means to let users
>>>>> upgrade their Flink versions in a reasonable pace (at least every other
>>>>> major release.)
>>>>>> 
>>>>>> Therefore, I think its also reasonable to also clearly state that
>>>>> savepoints compatibility will only be guaranteed for the previous
>>> release.
>>>>>> Although I think at the moment almost, if not all, of the current code
>>>>> still maintains compatibility for 1.1, in the long run these migration
>>>>> codes would definitely start to pile up and pollute the actual codebase
>>> if
>>>>> we try to always be compatible with all previous versions.
>>>>>> 
>>>>>> Cheers,
>>>>>> Gordon
>>>>>> 
>>>>>> 
>>>>>> On 21 May 2017 at 2:24:53 AM, Kostas Kloudas (
>>>>> k.kloudas@data-artisans.com) wrote:
>>>>>> 
>>>>>> Hi Chesnay,
>>>>>> 
>>>>>> I believe that for APIs we already have a pretty clear policy with the
>>>>> annotations.
>>>>>> I was referring to savepoints and state related backwards
>>> compatibility.
>>>>>> 
>>>>>> 
>>>>>>> On May 20, 2017, at 7:20 PM, Chesnay Schepler <ch...@apache.org>
>>>>> wrote:
>>>>>>> 
>>>>>>> I think it would be a good to clarify what kind of
>>>>> backwards-compatibilitiy we're talking about here. As in are we talking
>>>>> about APIs or savepoints?
>>>>>>> 
>>>>>>> On 20.05.2017 19:09, Kostas Kloudas wrote:
>>>>>>>> Hi all,
>>>>>>>> 
>>>>>>>> As we are getting closer to releasing Flink-1.3, I would like to open
>>>>> a discussion
>>>>>>>> on how far back we provide backwards compatibility for.
>>>>>>>> 
>>>>>>>> The reason for opening the discussion is that i) for the users and
>>> for
>>>>> the
>>>>>>>> adoption of the project, it is good to have an explicitely stated
>>>>> policy that implies
>>>>>>>> certain guarantees, and ii) keeping code and tests for backwards
>>>>> compatibility with
>>>>>>>> Flink-1.1 does not offer much. On the contrary, I think that it leads
>>>>> to:
>>>>>>>> 
>>>>>>>> 1) dead or ugly code in the codebase, e.g. deprecated class fields
>>>>> that could go away and
>>>>>>>> ugly if() loops (see aligned window operators that were deprecated in
>>>>> 1.2 and are now
>>>>>>>> normal windows), etc
>>>>>>>> 2) expensive tests (as, normally, they read from a savepoint)
>>>>>>>> 3) binary files in the codebase for holding the aforementioned
>>>>> savepoints
>>>>>>>> 
>>>>>>>> My proposal for such a policy would be to offer backwards
>>>>> compatibility for one previous version.
>>>>>>>> 
>>>>>>>> This means that 1.3 will be compatible with 1.2 (not 1.1). This still
>>>>> allows a clear
>>>>>>>> "backwards compatibility" path when jumping versions (a user that
>>> goes
>>>>>>>> from 1.1 to 1.3 can go initially 1.1 -> 1.2, take a savepoint, and
>>>>> then 1.2 -> 1.3),
>>>>>>>> while also allowing us to clean up the codebase a bit.
>>>>>>>> 
>>>>>>>> What do you think?
>>>>>>>> 
>>>>>>>> Kostas
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>> 
>>> 
> 


Re: [DISCUSS] Backwards compatibility policy.

Posted by Aljoscha Krettek <al...@apache.org>.
Normally, I’m the first one to suggest removing everything that is not absolutely necessary in order to have a clean code base. On this issue, though, I think we should support restoring from old Savepoints as far back as possible if it does not make the code completely unmaintainable. Some users might jump versions and always forcing them to go though every version from their old version to the current version doesn’t seem feasible and might put off some users.

So far, I think the burden of supporting restore from 1.1 is still small enough and with each new version the changes between versions become less and less. The changes from 1.2 to the upcoming 1.3 are quite minimal, I think.

Best,
Aljoscha
> On 24. May 2017, at 17:58, Ted Yu <yu...@gmail.com> wrote:
> 
> bq. about having LTS versions once a year
> 
> +1 to the above.
> 
> There may be various reasons users don't want to upgrade (after new
> releases come out). We should give such users enough flexibility on the
> upgrade path.
> 
> Cheers
> 
> On Wed, May 24, 2017 at 8:39 AM, Kostas Kloudas <k.kloudas@data-artisans.com
>> wrote:
> 
>> Hi all,
>> 
>> For the proposal of having a third party tool, I agree with Ted.
>> Maintaining
>> it is a big and far from trivial effort.
>> 
>> Now for the window of backwards compatibility, I would argue that even if
>> for some users 4 months (1 release) is not enough to bump their Flink
>> version,
>> the proposed policy guarantees that there will always be a path from any
>> old
>> version to any subsequent one.
>> 
>> Finally, for the proposal about having LTS versions once a year, I am not
>> sure if this will reduce or create more overhead. If I understand the plan
>> correctly, this would mean that the community will have to maintain
>> 2 or 3 LTS versions and the last two major ones, right?
>> 
>>> On May 22, 2017, at 7:31 PM, Ted Yu <yu...@gmail.com> wrote:
>>> 
>>> For #2, it is difficult to achieve:
>>> 
>>> a. maintaining savepoint migration is non-trivial and should be reviewed
>> by
>>> domain experts
>>> b. how to certify such third-party tool
>>> 
>>> Cheers
>>> 
>>> On Mon, May 22, 2017 at 3:04 AM, 施晓罡 <sh...@gmail.com> wrote:
>>> 
>>>> Hi all,
>>>> 
>>>> Currently, we work a lot in the maintenance of compatibility.
>>>> There exist much code in runtime to support the migration of savepoints
>>>> (most of which are deprecated), making it hard to focus on the current
>>>> implementation.
>>>> When more versions are released, much more efforts will be needed if we
>>>> try to make these released versions compatible.
>>>> 
>>>> I agree with Tzu-Li that we should provide a method to let users upgrade
>>>> Flink in a reasonable pace.
>>>> But i am against the proposal that we only offer backwards compatibility
>>>> for one previous version.
>>>> According our time-based release model, a major version is released
>> every
>>>> four month.
>>>> That means, users have to upgrade their versions every 8 months.
>> Otherwise
>>>> they will have difficulties in the migration of existing savepoints.
>>>> 
>>>> My suggestions include
>>>> 
>>>> (1) We can release Long-Term Support (LTS) versions which are widely
>>>> adopted in other open-source projects.
>>>> LTS versions should be stable and are free of found bugs. Savepoints in
>>>> LTS versions are guaranteed to be back-compatible so that users can
>> easily
>>>> upgrade to newer LTS versions.
>>>> 
>>>> The releasing of LTS versions is slower than that of major versions
>> (maybe
>>>> once a year, determined by users’ upgrade frequency).
>>>> Each LTS version will be supported a period of time and typically there
>>>> are no more than three active LTS versions.
>>>> By encouraging users to use LTS versions, we can ease the maintenance of
>>>> released versions (bug fixes, back compatibility, and critical
>> performance
>>>> improvement).
>>>> 
>>>> (2) We can provide a third-party tool to do the migration of
>> old-versioned
>>>> savepoints.
>>>> When users upgrade their versions, they can use the provided tool to
>>>> migrate existing savepoints.
>>>> This can help move the code for savepoint migration out of the actual
>>>> codebase,  making code focuses on current implementation.
>>>> 
>>>> What do you think?
>>>> 
>>>> Regards,
>>>> Xiaogang
>>>> 
>>>> 
>>>>> 在 2017年5月22日,下午1:39,Tzu-Li (Gordon) Tai <tz...@apache.org> 写道:
>>>>> 
>>>>> Hi Kostas,
>>>>> 
>>>>> Thanks for bringing this up!
>>>>> I think it is reasonable to keep this coherent with our timely-based
>>>> release model guarantees.
>>>>> 
>>>>> With the timely-based release model, there is a guarantee that the
>>>> current latest major version and the previous one is supported.
>>>>> For example, upon releasing 1.3, only 1.3 and 1.2 will still be
>>>> supported by the community for any required bug fixes.
>>>>> I think this was initially decided not only to ease old version
>>>> maintenance efforts for the community, but also as a means to let users
>>>> upgrade their Flink versions in a reasonable pace (at least every other
>>>> major release.)
>>>>> 
>>>>> Therefore, I think its also reasonable to also clearly state that
>>>> savepoints compatibility will only be guaranteed for the previous
>> release.
>>>>> Although I think at the moment almost, if not all, of the current code
>>>> still maintains compatibility for 1.1, in the long run these migration
>>>> codes would definitely start to pile up and pollute the actual codebase
>> if
>>>> we try to always be compatible with all previous versions.
>>>>> 
>>>>> Cheers,
>>>>> Gordon
>>>>> 
>>>>> 
>>>>> On 21 May 2017 at 2:24:53 AM, Kostas Kloudas (
>>>> k.kloudas@data-artisans.com) wrote:
>>>>> 
>>>>> Hi Chesnay,
>>>>> 
>>>>> I believe that for APIs we already have a pretty clear policy with the
>>>> annotations.
>>>>> I was referring to savepoints and state related backwards
>> compatibility.
>>>>> 
>>>>> 
>>>>>> On May 20, 2017, at 7:20 PM, Chesnay Schepler <ch...@apache.org>
>>>> wrote:
>>>>>> 
>>>>>> I think it would be a good to clarify what kind of
>>>> backwards-compatibilitiy we're talking about here. As in are we talking
>>>> about APIs or savepoints?
>>>>>> 
>>>>>> On 20.05.2017 19:09, Kostas Kloudas wrote:
>>>>>>> Hi all,
>>>>>>> 
>>>>>>> As we are getting closer to releasing Flink-1.3, I would like to open
>>>> a discussion
>>>>>>> on how far back we provide backwards compatibility for.
>>>>>>> 
>>>>>>> The reason for opening the discussion is that i) for the users and
>> for
>>>> the
>>>>>>> adoption of the project, it is good to have an explicitely stated
>>>> policy that implies
>>>>>>> certain guarantees, and ii) keeping code and tests for backwards
>>>> compatibility with
>>>>>>> Flink-1.1 does not offer much. On the contrary, I think that it leads
>>>> to:
>>>>>>> 
>>>>>>> 1) dead or ugly code in the codebase, e.g. deprecated class fields
>>>> that could go away and
>>>>>>> ugly if() loops (see aligned window operators that were deprecated in
>>>> 1.2 and are now
>>>>>>> normal windows), etc
>>>>>>> 2) expensive tests (as, normally, they read from a savepoint)
>>>>>>> 3) binary files in the codebase for holding the aforementioned
>>>> savepoints
>>>>>>> 
>>>>>>> My proposal for such a policy would be to offer backwards
>>>> compatibility for one previous version.
>>>>>>> 
>>>>>>> This means that 1.3 will be compatible with 1.2 (not 1.1). This still
>>>> allows a clear
>>>>>>> "backwards compatibility" path when jumping versions (a user that
>> goes
>>>>>>> from 1.1 to 1.3 can go initially 1.1 -> 1.2, take a savepoint, and
>>>> then 1.2 -> 1.3),
>>>>>>> while also allowing us to clean up the codebase a bit.
>>>>>>> 
>>>>>>> What do you think?
>>>>>>> 
>>>>>>> Kostas
>>>>>> 
>>>>>> 
>>>>> 
>>>> 
>>>> 
>> 
>> 


Re: [DISCUSS] Backwards compatibility policy.

Posted by Ted Yu <yu...@gmail.com>.
bq. about having LTS versions once a year

+1 to the above.

There may be various reasons users don't want to upgrade (after new
releases come out). We should give such users enough flexibility on the
upgrade path.

Cheers

On Wed, May 24, 2017 at 8:39 AM, Kostas Kloudas <k.kloudas@data-artisans.com
> wrote:

> Hi all,
>
> For the proposal of having a third party tool, I agree with Ted.
> Maintaining
> it is a big and far from trivial effort.
>
> Now for the window of backwards compatibility, I would argue that even if
> for some users 4 months (1 release) is not enough to bump their Flink
> version,
> the proposed policy guarantees that there will always be a path from any
> old
> version to any subsequent one.
>
> Finally, for the proposal about having LTS versions once a year, I am not
> sure if this will reduce or create more overhead. If I understand the plan
> correctly, this would mean that the community will have to maintain
> 2 or 3 LTS versions and the last two major ones, right?
>
> > On May 22, 2017, at 7:31 PM, Ted Yu <yu...@gmail.com> wrote:
> >
> > For #2, it is difficult to achieve:
> >
> > a. maintaining savepoint migration is non-trivial and should be reviewed
> by
> > domain experts
> > b. how to certify such third-party tool
> >
> > Cheers
> >
> > On Mon, May 22, 2017 at 3:04 AM, 施晓罡 <sh...@gmail.com> wrote:
> >
> >> Hi all,
> >>
> >> Currently, we work a lot in the maintenance of compatibility.
> >> There exist much code in runtime to support the migration of savepoints
> >> (most of which are deprecated), making it hard to focus on the current
> >> implementation.
> >> When more versions are released, much more efforts will be needed if we
> >> try to make these released versions compatible.
> >>
> >> I agree with Tzu-Li that we should provide a method to let users upgrade
> >> Flink in a reasonable pace.
> >> But i am against the proposal that we only offer backwards compatibility
> >> for one previous version.
> >> According our time-based release model, a major version is released
> every
> >> four month.
> >> That means, users have to upgrade their versions every 8 months.
> Otherwise
> >> they will have difficulties in the migration of existing savepoints.
> >>
> >> My suggestions include
> >>
> >> (1) We can release Long-Term Support (LTS) versions which are widely
> >> adopted in other open-source projects.
> >> LTS versions should be stable and are free of found bugs. Savepoints in
> >> LTS versions are guaranteed to be back-compatible so that users can
> easily
> >> upgrade to newer LTS versions.
> >>
> >> The releasing of LTS versions is slower than that of major versions
> (maybe
> >> once a year, determined by users’ upgrade frequency).
> >> Each LTS version will be supported a period of time and typically there
> >> are no more than three active LTS versions.
> >> By encouraging users to use LTS versions, we can ease the maintenance of
> >> released versions (bug fixes, back compatibility, and critical
> performance
> >> improvement).
> >>
> >> (2) We can provide a third-party tool to do the migration of
> old-versioned
> >> savepoints.
> >> When users upgrade their versions, they can use the provided tool to
> >> migrate existing savepoints.
> >> This can help move the code for savepoint migration out of the actual
> >> codebase,  making code focuses on current implementation.
> >>
> >> What do you think?
> >>
> >> Regards,
> >> Xiaogang
> >>
> >>
> >>> 在 2017年5月22日,下午1:39,Tzu-Li (Gordon) Tai <tz...@apache.org> 写道:
> >>>
> >>> Hi Kostas,
> >>>
> >>> Thanks for bringing this up!
> >>> I think it is reasonable to keep this coherent with our timely-based
> >> release model guarantees.
> >>>
> >>> With the timely-based release model, there is a guarantee that the
> >> current latest major version and the previous one is supported.
> >>> For example, upon releasing 1.3, only 1.3 and 1.2 will still be
> >> supported by the community for any required bug fixes.
> >>> I think this was initially decided not only to ease old version
> >> maintenance efforts for the community, but also as a means to let users
> >> upgrade their Flink versions in a reasonable pace (at least every other
> >> major release.)
> >>>
> >>> Therefore, I think its also reasonable to also clearly state that
> >> savepoints compatibility will only be guaranteed for the previous
> release.
> >>> Although I think at the moment almost, if not all, of the current code
> >> still maintains compatibility for 1.1, in the long run these migration
> >> codes would definitely start to pile up and pollute the actual codebase
> if
> >> we try to always be compatible with all previous versions.
> >>>
> >>> Cheers,
> >>> Gordon
> >>>
> >>>
> >>> On 21 May 2017 at 2:24:53 AM, Kostas Kloudas (
> >> k.kloudas@data-artisans.com) wrote:
> >>>
> >>> Hi Chesnay,
> >>>
> >>> I believe that for APIs we already have a pretty clear policy with the
> >> annotations.
> >>> I was referring to savepoints and state related backwards
> compatibility.
> >>>
> >>>
> >>>> On May 20, 2017, at 7:20 PM, Chesnay Schepler <ch...@apache.org>
> >> wrote:
> >>>>
> >>>> I think it would be a good to clarify what kind of
> >> backwards-compatibilitiy we're talking about here. As in are we talking
> >> about APIs or savepoints?
> >>>>
> >>>> On 20.05.2017 19:09, Kostas Kloudas wrote:
> >>>>> Hi all,
> >>>>>
> >>>>> As we are getting closer to releasing Flink-1.3, I would like to open
> >> a discussion
> >>>>> on how far back we provide backwards compatibility for.
> >>>>>
> >>>>> The reason for opening the discussion is that i) for the users and
> for
> >> the
> >>>>> adoption of the project, it is good to have an explicitely stated
> >> policy that implies
> >>>>> certain guarantees, and ii) keeping code and tests for backwards
> >> compatibility with
> >>>>> Flink-1.1 does not offer much. On the contrary, I think that it leads
> >> to:
> >>>>>
> >>>>> 1) dead or ugly code in the codebase, e.g. deprecated class fields
> >> that could go away and
> >>>>> ugly if() loops (see aligned window operators that were deprecated in
> >> 1.2 and are now
> >>>>> normal windows), etc
> >>>>> 2) expensive tests (as, normally, they read from a savepoint)
> >>>>> 3) binary files in the codebase for holding the aforementioned
> >> savepoints
> >>>>>
> >>>>> My proposal for such a policy would be to offer backwards
> >> compatibility for one previous version.
> >>>>>
> >>>>> This means that 1.3 will be compatible with 1.2 (not 1.1). This still
> >> allows a clear
> >>>>> "backwards compatibility" path when jumping versions (a user that
> goes
> >>>>> from 1.1 to 1.3 can go initially 1.1 -> 1.2, take a savepoint, and
> >> then 1.2 -> 1.3),
> >>>>> while also allowing us to clean up the codebase a bit.
> >>>>>
> >>>>> What do you think?
> >>>>>
> >>>>> Kostas
> >>>>
> >>>>
> >>>
> >>
> >>
>
>

Re: [DISCUSS] Backwards compatibility policy.

Posted by Kostas Kloudas <k....@data-artisans.com>.
Hi all,

For the proposal of having a third party tool, I agree with Ted. Maintaining 
it is a big and far from trivial effort.

Now for the window of backwards compatibility, I would argue that even if 
for some users 4 months (1 release) is not enough to bump their Flink version, 
the proposed policy guarantees that there will always be a path from any old 
version to any subsequent one.

Finally, for the proposal about having LTS versions once a year, I am not
sure if this will reduce or create more overhead. If I understand the plan
correctly, this would mean that the community will have to maintain
2 or 3 LTS versions and the last two major ones, right?

> On May 22, 2017, at 7:31 PM, Ted Yu <yu...@gmail.com> wrote:
> 
> For #2, it is difficult to achieve:
> 
> a. maintaining savepoint migration is non-trivial and should be reviewed by
> domain experts
> b. how to certify such third-party tool
> 
> Cheers
> 
> On Mon, May 22, 2017 at 3:04 AM, 施晓罡 <sh...@gmail.com> wrote:
> 
>> Hi all,
>> 
>> Currently, we work a lot in the maintenance of compatibility.
>> There exist much code in runtime to support the migration of savepoints
>> (most of which are deprecated), making it hard to focus on the current
>> implementation.
>> When more versions are released, much more efforts will be needed if we
>> try to make these released versions compatible.
>> 
>> I agree with Tzu-Li that we should provide a method to let users upgrade
>> Flink in a reasonable pace.
>> But i am against the proposal that we only offer backwards compatibility
>> for one previous version.
>> According our time-based release model, a major version is released every
>> four month.
>> That means, users have to upgrade their versions every 8 months. Otherwise
>> they will have difficulties in the migration of existing savepoints.
>> 
>> My suggestions include
>> 
>> (1) We can release Long-Term Support (LTS) versions which are widely
>> adopted in other open-source projects.
>> LTS versions should be stable and are free of found bugs. Savepoints in
>> LTS versions are guaranteed to be back-compatible so that users can easily
>> upgrade to newer LTS versions.
>> 
>> The releasing of LTS versions is slower than that of major versions (maybe
>> once a year, determined by users’ upgrade frequency).
>> Each LTS version will be supported a period of time and typically there
>> are no more than three active LTS versions.
>> By encouraging users to use LTS versions, we can ease the maintenance of
>> released versions (bug fixes, back compatibility, and critical performance
>> improvement).
>> 
>> (2) We can provide a third-party tool to do the migration of old-versioned
>> savepoints.
>> When users upgrade their versions, they can use the provided tool to
>> migrate existing savepoints.
>> This can help move the code for savepoint migration out of the actual
>> codebase,  making code focuses on current implementation.
>> 
>> What do you think?
>> 
>> Regards,
>> Xiaogang
>> 
>> 
>>> 在 2017年5月22日,下午1:39,Tzu-Li (Gordon) Tai <tz...@apache.org> 写道:
>>> 
>>> Hi Kostas,
>>> 
>>> Thanks for bringing this up!
>>> I think it is reasonable to keep this coherent with our timely-based
>> release model guarantees.
>>> 
>>> With the timely-based release model, there is a guarantee that the
>> current latest major version and the previous one is supported.
>>> For example, upon releasing 1.3, only 1.3 and 1.2 will still be
>> supported by the community for any required bug fixes.
>>> I think this was initially decided not only to ease old version
>> maintenance efforts for the community, but also as a means to let users
>> upgrade their Flink versions in a reasonable pace (at least every other
>> major release.)
>>> 
>>> Therefore, I think its also reasonable to also clearly state that
>> savepoints compatibility will only be guaranteed for the previous release.
>>> Although I think at the moment almost, if not all, of the current code
>> still maintains compatibility for 1.1, in the long run these migration
>> codes would definitely start to pile up and pollute the actual codebase if
>> we try to always be compatible with all previous versions.
>>> 
>>> Cheers,
>>> Gordon
>>> 
>>> 
>>> On 21 May 2017 at 2:24:53 AM, Kostas Kloudas (
>> k.kloudas@data-artisans.com) wrote:
>>> 
>>> Hi Chesnay,
>>> 
>>> I believe that for APIs we already have a pretty clear policy with the
>> annotations.
>>> I was referring to savepoints and state related backwards compatibility.
>>> 
>>> 
>>>> On May 20, 2017, at 7:20 PM, Chesnay Schepler <ch...@apache.org>
>> wrote:
>>>> 
>>>> I think it would be a good to clarify what kind of
>> backwards-compatibilitiy we're talking about here. As in are we talking
>> about APIs or savepoints?
>>>> 
>>>> On 20.05.2017 19:09, Kostas Kloudas wrote:
>>>>> Hi all,
>>>>> 
>>>>> As we are getting closer to releasing Flink-1.3, I would like to open
>> a discussion
>>>>> on how far back we provide backwards compatibility for.
>>>>> 
>>>>> The reason for opening the discussion is that i) for the users and for
>> the
>>>>> adoption of the project, it is good to have an explicitely stated
>> policy that implies
>>>>> certain guarantees, and ii) keeping code and tests for backwards
>> compatibility with
>>>>> Flink-1.1 does not offer much. On the contrary, I think that it leads
>> to:
>>>>> 
>>>>> 1) dead or ugly code in the codebase, e.g. deprecated class fields
>> that could go away and
>>>>> ugly if() loops (see aligned window operators that were deprecated in
>> 1.2 and are now
>>>>> normal windows), etc
>>>>> 2) expensive tests (as, normally, they read from a savepoint)
>>>>> 3) binary files in the codebase for holding the aforementioned
>> savepoints
>>>>> 
>>>>> My proposal for such a policy would be to offer backwards
>> compatibility for one previous version.
>>>>> 
>>>>> This means that 1.3 will be compatible with 1.2 (not 1.1). This still
>> allows a clear
>>>>> "backwards compatibility" path when jumping versions (a user that goes
>>>>> from 1.1 to 1.3 can go initially 1.1 -> 1.2, take a savepoint, and
>> then 1.2 -> 1.3),
>>>>> while also allowing us to clean up the codebase a bit.
>>>>> 
>>>>> What do you think?
>>>>> 
>>>>> Kostas
>>>> 
>>>> 
>>> 
>> 
>> 


Re: [DISCUSS] Backwards compatibility policy.

Posted by Ted Yu <yu...@gmail.com>.
For #2, it is difficult to achieve:

a. maintaining savepoint migration is non-trivial and should be reviewed by
domain experts
b. how to certify such third-party tool

Cheers

On Mon, May 22, 2017 at 3:04 AM, 施晓罡 <sh...@gmail.com> wrote:

> Hi all,
>
> Currently, we work a lot in the maintenance of compatibility.
> There exist much code in runtime to support the migration of savepoints
> (most of which are deprecated), making it hard to focus on the current
> implementation.
> When more versions are released, much more efforts will be needed if we
> try to make these released versions compatible.
>
> I agree with Tzu-Li that we should provide a method to let users upgrade
> Flink in a reasonable pace.
> But i am against the proposal that we only offer backwards compatibility
> for one previous version.
> According our time-based release model, a major version is released every
> four month.
> That means, users have to upgrade their versions every 8 months. Otherwise
> they will have difficulties in the migration of existing savepoints.
>
> My suggestions include
>
> (1) We can release Long-Term Support (LTS) versions which are widely
> adopted in other open-source projects.
> LTS versions should be stable and are free of found bugs. Savepoints in
> LTS versions are guaranteed to be back-compatible so that users can easily
> upgrade to newer LTS versions.
>
> The releasing of LTS versions is slower than that of major versions (maybe
> once a year, determined by users’ upgrade frequency).
> Each LTS version will be supported a period of time and typically there
> are no more than three active LTS versions.
> By encouraging users to use LTS versions, we can ease the maintenance of
> released versions (bug fixes, back compatibility, and critical performance
> improvement).
>
> (2) We can provide a third-party tool to do the migration of old-versioned
> savepoints.
> When users upgrade their versions, they can use the provided tool to
> migrate existing savepoints.
> This can help move the code for savepoint migration out of the actual
> codebase,  making code focuses on current implementation.
>
> What do you think?
>
> Regards,
> Xiaogang
>
>
> > 在 2017年5月22日,下午1:39,Tzu-Li (Gordon) Tai <tz...@apache.org> 写道:
> >
> > Hi Kostas,
> >
> > Thanks for bringing this up!
> > I think it is reasonable to keep this coherent with our timely-based
> release model guarantees.
> >
> > With the timely-based release model, there is a guarantee that the
> current latest major version and the previous one is supported.
> > For example, upon releasing 1.3, only 1.3 and 1.2 will still be
> supported by the community for any required bug fixes.
> > I think this was initially decided not only to ease old version
> maintenance efforts for the community, but also as a means to let users
> upgrade their Flink versions in a reasonable pace (at least every other
> major release.)
> >
> > Therefore, I think its also reasonable to also clearly state that
> savepoints compatibility will only be guaranteed for the previous release.
> > Although I think at the moment almost, if not all, of the current code
> still maintains compatibility for 1.1, in the long run these migration
> codes would definitely start to pile up and pollute the actual codebase if
> we try to always be compatible with all previous versions.
> >
> > Cheers,
> > Gordon
> >
> >
> > On 21 May 2017 at 2:24:53 AM, Kostas Kloudas (
> k.kloudas@data-artisans.com) wrote:
> >
> > Hi Chesnay,
> >
> > I believe that for APIs we already have a pretty clear policy with the
> annotations.
> > I was referring to savepoints and state related backwards compatibility.
> >
> >
> >> On May 20, 2017, at 7:20 PM, Chesnay Schepler <ch...@apache.org>
> wrote:
> >>
> >> I think it would be a good to clarify what kind of
> backwards-compatibilitiy we're talking about here. As in are we talking
> about APIs or savepoints?
> >>
> >> On 20.05.2017 19:09, Kostas Kloudas wrote:
> >>> Hi all,
> >>>
> >>> As we are getting closer to releasing Flink-1.3, I would like to open
> a discussion
> >>> on how far back we provide backwards compatibility for.
> >>>
> >>> The reason for opening the discussion is that i) for the users and for
> the
> >>> adoption of the project, it is good to have an explicitely stated
> policy that implies
> >>> certain guarantees, and ii) keeping code and tests for backwards
> compatibility with
> >>> Flink-1.1 does not offer much. On the contrary, I think that it leads
> to:
> >>>
> >>> 1) dead or ugly code in the codebase, e.g. deprecated class fields
> that could go away and
> >>> ugly if() loops (see aligned window operators that were deprecated in
> 1.2 and are now
> >>> normal windows), etc
> >>> 2) expensive tests (as, normally, they read from a savepoint)
> >>> 3) binary files in the codebase for holding the aforementioned
> savepoints
> >>>
> >>> My proposal for such a policy would be to offer backwards
> compatibility for one previous version.
> >>>
> >>> This means that 1.3 will be compatible with 1.2 (not 1.1). This still
> allows a clear
> >>> "backwards compatibility" path when jumping versions (a user that goes
> >>> from 1.1 to 1.3 can go initially 1.1 -> 1.2, take a savepoint, and
> then 1.2 -> 1.3),
> >>> while also allowing us to clean up the codebase a bit.
> >>>
> >>> What do you think?
> >>>
> >>> Kostas
> >>
> >>
> >
>
>

Re: [DISCUSS] Backwards compatibility policy.

Posted by 施晓罡 <sh...@gmail.com>.
Hi all,

Currently, we work a lot in the maintenance of compatibility.  
There exist much code in runtime to support the migration of savepoints (most of which are deprecated), making it hard to focus on the current implementation.
When more versions are released, much more efforts will be needed if we try to make these released versions compatible.

I agree with Tzu-Li that we should provide a method to let users upgrade Flink in a reasonable pace.
But i am against the proposal that we only offer backwards compatibility for one previous version.
According our time-based release model, a major version is released every four month. 
That means, users have to upgrade their versions every 8 months. Otherwise they will have difficulties in the migration of existing savepoints.

My suggestions include

(1) We can release Long-Term Support (LTS) versions which are widely adopted in other open-source projects. 
LTS versions should be stable and are free of found bugs. Savepoints in LTS versions are guaranteed to be back-compatible so that users can easily upgrade to newer LTS versions.

The releasing of LTS versions is slower than that of major versions (maybe once a year, determined by users’ upgrade frequency). 
Each LTS version will be supported a period of time and typically there are no more than three active LTS versions.
By encouraging users to use LTS versions, we can ease the maintenance of released versions (bug fixes, back compatibility, and critical performance improvement).

(2) We can provide a third-party tool to do the migration of old-versioned savepoints. 
When users upgrade their versions, they can use the provided tool to migrate existing savepoints.
This can help move the code for savepoint migration out of the actual codebase,  making code focuses on current implementation.

What do you think?

Regards,
Xiaogang


> 在 2017年5月22日,下午1:39,Tzu-Li (Gordon) Tai <tz...@apache.org> 写道:
> 
> Hi Kostas,
> 
> Thanks for bringing this up!
> I think it is reasonable to keep this coherent with our timely-based release model guarantees.
> 
> With the timely-based release model, there is a guarantee that the current latest major version and the previous one is supported.
> For example, upon releasing 1.3, only 1.3 and 1.2 will still be supported by the community for any required bug fixes.
> I think this was initially decided not only to ease old version maintenance efforts for the community, but also as a means to let users upgrade their Flink versions in a reasonable pace (at least every other major release.)
> 
> Therefore, I think its also reasonable to also clearly state that savepoints compatibility will only be guaranteed for the previous release.
> Although I think at the moment almost, if not all, of the current code still maintains compatibility for 1.1, in the long run these migration codes would definitely start to pile up and pollute the actual codebase if we try to always be compatible with all previous versions.
> 
> Cheers,
> Gordon
> 
> 
> On 21 May 2017 at 2:24:53 AM, Kostas Kloudas (k.kloudas@data-artisans.com) wrote:
> 
> Hi Chesnay, 
> 
> I believe that for APIs we already have a pretty clear policy with the annotations. 
> I was referring to savepoints and state related backwards compatibility. 
> 
> 
>> On May 20, 2017, at 7:20 PM, Chesnay Schepler <ch...@apache.org> wrote: 
>> 
>> I think it would be a good to clarify what kind of backwards-compatibilitiy we're talking about here. As in are we talking about APIs or savepoints? 
>> 
>> On 20.05.2017 19:09, Kostas Kloudas wrote: 
>>> Hi all, 
>>> 
>>> As we are getting closer to releasing Flink-1.3, I would like to open a discussion 
>>> on how far back we provide backwards compatibility for. 
>>> 
>>> The reason for opening the discussion is that i) for the users and for the 
>>> adoption of the project, it is good to have an explicitely stated policy that implies 
>>> certain guarantees, and ii) keeping code and tests for backwards compatibility with 
>>> Flink-1.1 does not offer much. On the contrary, I think that it leads to: 
>>> 
>>> 1) dead or ugly code in the codebase, e.g. deprecated class fields that could go away and 
>>> ugly if() loops (see aligned window operators that were deprecated in 1.2 and are now 
>>> normal windows), etc 
>>> 2) expensive tests (as, normally, they read from a savepoint) 
>>> 3) binary files in the codebase for holding the aforementioned savepoints 
>>> 
>>> My proposal for such a policy would be to offer backwards compatibility for one previous version. 
>>> 
>>> This means that 1.3 will be compatible with 1.2 (not 1.1). This still allows a clear 
>>> "backwards compatibility" path when jumping versions (a user that goes 
>>> from 1.1 to 1.3 can go initially 1.1 -> 1.2, take a savepoint, and then 1.2 -> 1.3), 
>>> while also allowing us to clean up the codebase a bit. 
>>> 
>>> What do you think? 
>>> 
>>> Kostas 
>> 
>> 
> 


Re: [DISCUSS] Backwards compatibility policy.

Posted by "Tzu-Li (Gordon) Tai" <tz...@apache.org>.
Hi Kostas,

Thanks for bringing this up!
I think it is reasonable to keep this coherent with our timely-based release model guarantees.

With the timely-based release model, there is a guarantee that the current latest major version and the previous one is supported.
For example, upon releasing 1.3, only 1.3 and 1.2 will still be supported by the community for any required bug fixes.
I think this was initially decided not only to ease old version maintenance efforts for the community, but also as a means to let users upgrade their Flink versions in a reasonable pace (at least every other major release.)

Therefore, I think its also reasonable to also clearly state that savepoints compatibility will only be guaranteed for the previous release.
Although I think at the moment almost, if not all, of the current code still maintains compatibility for 1.1, in the long run these migration codes would definitely start to pile up and pollute the actual codebase if we try to always be compatible with all previous versions.

Cheers,
Gordon


On 21 May 2017 at 2:24:53 AM, Kostas Kloudas (k.kloudas@data-artisans.com) wrote:

Hi Chesnay, 

I believe that for APIs we already have a pretty clear policy with the annotations. 
I was referring to savepoints and state related backwards compatibility. 


> On May 20, 2017, at 7:20 PM, Chesnay Schepler <ch...@apache.org> wrote: 
> 
> I think it would be a good to clarify what kind of backwards-compatibilitiy we're talking about here. As in are we talking about APIs or savepoints? 
> 
> On 20.05.2017 19:09, Kostas Kloudas wrote: 
>> Hi all, 
>> 
>> As we are getting closer to releasing Flink-1.3, I would like to open a discussion 
>> on how far back we provide backwards compatibility for. 
>> 
>> The reason for opening the discussion is that i) for the users and for the 
>> adoption of the project, it is good to have an explicitely stated policy that implies 
>> certain guarantees, and ii) keeping code and tests for backwards compatibility with 
>> Flink-1.1 does not offer much. On the contrary, I think that it leads to: 
>> 
>> 1) dead or ugly code in the codebase, e.g. deprecated class fields that could go away and 
>> ugly if() loops (see aligned window operators that were deprecated in 1.2 and are now 
>> normal windows), etc 
>> 2) expensive tests (as, normally, they read from a savepoint) 
>> 3) binary files in the codebase for holding the aforementioned savepoints 
>> 
>> My proposal for such a policy would be to offer backwards compatibility for one previous version. 
>> 
>> This means that 1.3 will be compatible with 1.2 (not 1.1). This still allows a clear 
>> "backwards compatibility" path when jumping versions (a user that goes 
>> from 1.1 to 1.3 can go initially 1.1 -> 1.2, take a savepoint, and then 1.2 -> 1.3), 
>> while also allowing us to clean up the codebase a bit. 
>> 
>> What do you think? 
>> 
>> Kostas 
> 
> 


Re: [DISCUSS] Backwards compatibility policy.

Posted by Kostas Kloudas <k....@data-artisans.com>.
Hi Chesnay,

I believe that for APIs we already have a pretty clear policy with the annotations.
I was referring to savepoints and state related backwards compatibility.


> On May 20, 2017, at 7:20 PM, Chesnay Schepler <ch...@apache.org> wrote:
> 
> I think it would be a good to clarify what kind of backwards-compatibilitiy we're talking about here. As in are we talking about APIs or savepoints?
> 
> On 20.05.2017 19:09, Kostas Kloudas wrote:
>> Hi all,
>> 
>> As we are getting closer to releasing Flink-1.3, I would like to open a discussion
>> on how far back we provide backwards compatibility for.
>> 
>> The reason for opening the discussion is that i) for the users and for the
>> adoption of the project, it is good to have an explicitely stated policy that implies
>> certain guarantees, and ii) keeping code and tests for backwards compatibility with
>> Flink-1.1 does not offer much. On the contrary, I think that it leads to:
>> 
>> 1) dead or ugly code in the codebase, e.g. deprecated class fields that could go away and
>> ugly if() loops (see aligned window operators that were deprecated in 1.2 and are now
>> normal windows), etc
>> 2) expensive tests (as, normally, they read from a savepoint)
>> 3) binary files in the codebase for holding the aforementioned savepoints
>> 
>> My proposal for such a policy would be to offer backwards compatibility for one previous version.
>> 
>> This means that 1.3 will be compatible with 1.2 (not 1.1). This still allows a clear
>> "backwards compatibility" path when jumping versions (a user that goes
>> from 1.1 to 1.3 can go initially 1.1 -> 1.2, take a savepoint, and then 1.2 -> 1.3),
>> while also allowing us to clean up the codebase a bit.
>> 
>> What do you think?
>> 
>> Kostas
> 
> 


Re: [DISCUSS] Backwards compatibility policy.

Posted by Chesnay Schepler <ch...@apache.org>.
I think it would be a good to clarify what kind of 
backwards-compatibilitiy we're talking about here. As in are we talking 
about APIs or savepoints?

On 20.05.2017 19:09, Kostas Kloudas wrote:
> Hi all,
>
> As we are getting closer to releasing Flink-1.3, I would like to open a discussion
> on how far back we provide backwards compatibility for.
>
> The reason for opening the discussion is that i) for the users and for the
> adoption of the project, it is good to have an explicitely stated policy that implies
> certain guarantees, and ii) keeping code and tests for backwards compatibility with
> Flink-1.1 does not offer much. On the contrary, I think that it leads to:
>
> 1) dead or ugly code in the codebase, e.g. deprecated class fields that could go away and
> ugly if() loops (see aligned window operators that were deprecated in 1.2 and are now
> normal windows), etc
> 2) expensive tests (as, normally, they read from a savepoint)
> 3) binary files in the codebase for holding the aforementioned savepoints
>
> My proposal for such a policy would be to offer backwards compatibility for one previous version.
>
> This means that 1.3 will be compatible with 1.2 (not 1.1). This still allows a clear
> "backwards compatibility" path when jumping versions (a user that goes
> from 1.1 to 1.3 can go initially 1.1 -> 1.2, take a savepoint, and then 1.2 -> 1.3),
> while also allowing us to clean up the codebase a bit.
>
> What do you think?
>
> Kostas