Hi All!

Based on the discussion above, I feel that the most reasonable approach
from both developers and users perspective at this point is what Becket
lists as Option 1:

Revert the naming change to the backward compatible version and accept that
the names are not perfect (treat it as legacy).

On a different note, I agree that the current sink v2 interface is very
difficult to evolve and structuring the interfaces the way they are now is
not a good design in the long run.
For new functionality or changes we can make easily, we should switch to
the decorative/mixin interface approach used successfully in the source and
table interfaces. Let's try to do this as much as possible within the v2
and compatibility boundaries and we should only introduce a v3 if we really
must.

So from my side, +1 to reverting the naming to keep backward compatibility.

Cheers,
Gyula


On Fri, Dec 1, 2023 at 10:43 AM Péter Váry <peter.vary.apa...@gmail.com>
wrote:

> Thanks Becket for your reply!
>
> *On Option 1:*
> - I personally consider API inconsistencies more important, since they will
> remain with us "forever", but this is up to the community. I can implement
> whichever solution we decide upon.
>
> *Option 2:*
> - I don't think this specific issue merits a rewrite, but if we decide to
> change our approach, then it's a different story.
>
> *Evolvability:*
> This discussion reminds me of a similar discussion on FLIP-372 [1], where
> we are trying to decide if we should use mixin interfaces, or use interface
> inheritance.
> With the mixin approach, we have a more flexible interface, but we can't
> check the generic types of the interfaces/classes on compile time, or even
> when we create the DAG. The first issue happens when we call the method and
> fail.
> The issue here is similar:
> - *StatefulSink* needs a writer with a method to `*snapshotState*`
> - *TwoPhaseCommittingSink* needs a writer with `*prepareCommit*`
> - If there is a Sink which is stateful and needs to commit, then it needs
> both of these methods.
>
> If we use the mixin solution here, we lose the possibility to check the
> types in compile time. We could do the type check in runtime using `
> *instanceof*`, so we are better off than with the FLIP-372 example above,
> but still lose any important possibility. I personally prefer the mixin
> approach, but that would mean we rewrite the Sink API again - likely a
> SinkV3. Are we ready to move down that path?
>
> Thanks,
> Peter
>
> [1] - https://lists.apache.org/thread/344pzbrqbbb4w0sfj67km25msp7hxlyd
>
> On Thu, Nov 30, 2023, 14:53 Becket Qin <becket....@gmail.com> wrote:
>
> > Hi folks,
> >
> > Sorry for replying late on the thread.
> >
> > For this particular FLIP, I see two solutions:
> >
> > Option 1:
> > 1. On top of the the current status, rename
> > *org.apache.flink.api.connector.sink2.InitContext *to
> > *CommonInitContext (*should
> > probably be package private*)*.
> > 2. Change the name *WriterInitContext* back to *InitContext, *and revert
> > the deprecation. We can change the parameter name to writerContext if we
> > want to.
> > Admittedly, this does not have full symmetric naming of the InitContexts
> -
> > we will have CommonInitContext / InitContext / CommitterInitContext
> instead
> > of InitContext / WriterInitContext / CommitterInitContext. However, the
> > naming seems clear without much confusion. Personally, I can live with
> > that, treating the class InitContext as a non-ideal legacy class name
> > without much material harm.
> >
> > Option 2:
> > Theoretically speaking, if we really want to reach the perfect state
> while
> > being backwards compatible, we can create a brand new set of Sink
> > interfaces and deprecate the old ones. But I feel this is an overkill
> here.
> >
> > The solution to this particular issue aside, the evolvability of the
> > current interface hierarchy seems a more fundamental issue and worries me
> > more. I haven't completely thought it through, but there are two
> noticeable
> > differences between the interface design principles between Source and
> > Sink.
> > 1. Source uses decorative interfaces. For example, we have a
> > SupportsFilterPushdown interface, instead of a subclass of
> > FilterableSource. This seems provides better flexibility.
> > 2. Source tends to have a more coarse-grained interface. For example,
> > SourceReader always has the methods of snapshotState(),
> > notifyCheckpointComplete(). Even if they may not be always required, we
> do
> > not separate them into different interfaces.
> > My hunch is that if we follow similar approach as Source, the
> evolvability
> > might be better. If we want to do this, we'd better to do it before 2.0.
> > What do you think?
> >
> > Process wise,
> > - I agree that if there is a change to the passed FLIP during
> > implementation, it should be brought back to the mailing list.
> > - There might be value for the connector nightly build to depend on the
> > latest snapshot of the same Flink major version. It helps catching
> > unexpected breaking changes sooner.
> > - I'll update the website to reflect the latest API stability policy.
> > Apologies for the confusion caused by the stale doc.
> >
> > Thanks,
> >
> > Jiangjie (Becket) Qin
> >
> >
> >
> > On Wed, Nov 29, 2023 at 10:55 PM Márton Balassi <
> balassi.mar...@gmail.com>
> > wrote:
> >
> > > Thanks, Martijn and Peter.
> > >
> > > In terms of the concrete issue:
> > >
> > >    - I am following up with the author of FLIP-321 [1] (Becket) to
> update
> > >    the docs [2] to reflect the right state.
> > >    - I see two reasonable approaches in terms of proceeding with the
> > >    specific changeset:
> > >
> > >
> > >    1. We allow the exception from FLIP-321 for this change and let the
> > >       PublicEvolving API change happen between Flink 1.18 and 1.19,
> which
> > > is
> > >       consistent with current state of the relevant documentation. [2]
> > > We commit
> > >       to helping the connector repos make the necessary (one liner)
> > > changes.
> > >       2. We revert back to the original implementation plan as
> explicitly
> > >       voted on in FLIP-371 [3]. That has no API breaking changes.
> > > However we end
> > >       up with an inconsistently named API with duplicated internal
> > > methods. Peter
> > >       has also discovered additional bad patterns during his work in
> > > FLIP-372
> > >       [3], the total of these changes could be handled in a separate
> FLIP
> > > that
> > >       would do multiple PublicEvolving breaking changes to clean up the
> > > API.
> > >
> > > In terms of the general issues:
> > >
> > >    - I agree that if a PR review of an accepted FLIP newly introduces a
> > >    breaking API change that warrants an update to the mailing list
> > > discussion
> > >    and possibly even a new vote.
> > >    - I agree with the general sentiment of FLIP-321 to provide stronger
> > API
> > >    guarantees with the minor note that if we have changes in mind we
> > should
> > >    prioritize them now such that they can be validated by Flink 2.0.
> > >    - I agree that ideally the connector repos should build against the
> > >    latest release and not the master branch.
> > >
> > > [1]
> > >
> > >
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-321%3A+Introduce+an+API+deprecation+process
> > > [2]
> > >
> > >
> >
> https://nightlies.apache.org/flink/flink-docs-master/docs/ops/upgrading/#api-compatibility-guarantees
> > > [3]
> > >
> > >
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-371%3A+Provide+initialization+context+for+Committer+creation+in+TwoPhaseCommittingSink
> > > [4]
> > >
> > >
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-372%3A+Allow+TwoPhaseCommittingSink+WithPreCommitTopology+to+alter+the+type+of+the+Committable
> > >
> > > Best,
> > > Marton
> > >
> > > On Mon, Nov 27, 2023 at 7:23 PM Péter Váry <
> peter.vary.apa...@gmail.com>
> > > wrote:
> > >
> > > > I think we should try to separate the discussion in a few different
> > > topics:
> > > >
> > > >    - Concrete issue
> > > >       - How to solve this problem in 1.19 and wrt the affected
> > > createWriter
> > > >       interface
> > > >       - Update the documentation [1], so FLIP-321 is visible for
> every
> > > >       contributor
> > > >    - Generic issue
> > > >       - API stability
> > > >       - Connector dependencies
> > > >
> > > >
> > > > *CreateWriter interface*
> > > > The change on the createWriter is not strictly required for the
> > > > functionality defined by the requirements on the FLIP.
> > > > If the only goal is only to have a backward compatible API, we can
> > simply
> > > > create a separate `*CommitterInitContext*` object and do not touch
> the
> > > > writer `*InitContext*`, like it was done in the original PR [2].
> > > > The issue is that this would result in an implementation which has
> > > > duplicated methods/implementations (internal issue only), and has
> > > > inconsistent naming (issue for external users).
> > > >
> > > > If we want to create an API which is consistent (and I agree with the
> > > > reviewer's comments), then we need to rename the parameter type (
> > > > *WriterInitContext*) for the createWriter method.
> > > > I have tried to keep the backward compatibility with creating a new
> > > method
> > > > and providing a default implementation for this new method which
> would
> > > call
> > > > the original method after converting the WriterInitContext to
> > > InitContext.
> > > >
> > > > This is failed because the following details:
> > > >
> > > >    - *org.apache.flink.api.connector.sink2.Sink* defines
> > > > `*SinkWriter<InputT>
> > > >    createWriter(InitContext context)`*
> > > >    - *org.apache.flink.api.connector.sink2.StatefulSink* narrows it
> > > > down to *`StatefulSinkWriter<InputT,
> > > >    WriterStateT> createWriter(InitContext context)`*
> > > >    - *org.apache.flink.api.connector.sink2.TwoPhaseCommittingSink*
> > > narrows
> > > >    it down to *`PrecommittingSinkWriter<InputT, CommT>
> > > >    createWriter(WriterInitContext context)`*
> > > >    -
> > > >
> > > >
> > >
> >
> *org.apache.flink.streaming.runtime.operators.sink.TestSinkV2.TestStatefulSinkV2*
> > > >    implements *StatefulSink* and *TwoPhaseCommittingSink* too
> > > >
> > > > *TestStatefulSinkV2* is a good example where we can not achieve
> > backward
> > > > compatibility, since the the compiler will fail with unrelated
> default
> > > > methods [3]
> > > >
> > > > I am open for any suggestions how to move to the new API, and keep
> the
> > > > backward compatibility. If we do not find a way to keep backward
> > > > compatibility, and we decide that we would like to honour FLIP-321,
> > then
> > > we
> > > > can reverting to the original solution and keep only the changes for
> > the
> > > `
> > > > *createCommitter*` method.
> > > >
> > > > *Update the documentation*
> > > > I have not found only one place in the docs [1], where we talk about
> > the
> > > > compatibility guarantees.
> > > > Based FLIP-321 and the result of the discussion here, we should
> update
> > > this
> > > > page.
> > > >
> > > > *API stability*
> > > > I agree with the general sentiment of FLIP-321 to keep the changes
> > > backward
> > > > compatible as much as possible. But the issue above highlights that
> > there
> > > > could be situations where it is not possible to achieve backward
> > > > compatibility. Probably we should provide exceptions to handle this
> > kind
> > > of
> > > > situations - minimally for PublicEvolving interfaces. After we agree
> on
> > > > long term goals - allowing exceptions or to be more lenient on
> backward
> > > > compatibility guarantees, or sticking to FLIP-321 by the letter - we
> > > could
> > > > discuss how to apply it to the current situation.
> > > >
> > > > *Connector dependencies*
> > > > I think it is generally a good practice to depend on the stable
> version
> > > of
> > > > Flink (or any other downstream project). This is how we do it in
> > Iceberg,
> > > > and how it was implemented in the Kafka connector as well. This would
> > > > result in more stable connector builds. The only issue I see, that
> the
> > > > situations like this would take longer to surface, but I fully expect
> > us
> > > to
> > > > get better at compatibility after we wetted the process.
> > > >
> > > > [1] -
> > > >
> > > >
> > >
> >
> https://nightlies.apache.org/flink/flink-docs-master/docs/ops/upgrading/#api-compatibility-guarantees
> > > > [2] -
> > > >
> > > >
> > >
> >
> https://github.com/apache/flink/pull/23555/commits/2b9adeb20e55c33a623115efa97d3149c11e9ca4
> > > > [3] -
> > https://github.com/apache/flink/pull/23555#discussion_r1371740397
> > > >
> > > > Martijn Visser <martijnvis...@apache.org> ezt írta (időpont: 2023.
> > nov.
> > > > 27., H, 11:21):
> > > >
> > > > > Hi all,
> > > > >
> > > > > I'm opening this discussion thread to bring a discussion that's
> > > > > happening on a completed Jira ticket back to the mailing list [1]
> > > > >
> > > > > In summary:
> > > > >
> > > > > * There was a discussion and a vote on FLIP-371 [2]
> > > > > * During implementation, it was determined that there's a diamond
> > > > > inheritance problem on the Sink.createWriter method, making a
> > > > > backwards compatible change hard/impossible (I think this is where
> > the
> > > > > main discussion point actually is) [3]
> > > > > * The PR was merged, causing a backwards incompatible change
> without
> > a
> > > > > discussion on the Dev mailing list
> > > > >
> > > > > I think that in hindsight, even though there was a FLIP on this
> > topic,
> > > > > the finding of the diamond inheritance issue should have been
> brought
> > > > > back to the Dev mailing list in order to agree on how to resolve
> it.
> > > > > Since 1.19 is still under way, we still have time to fix this.
> > > > >
> > > > > I think there's two things we can improve:
> > > > >
> > > > > 1) Next time during implementation of a FLIP/PR which involves a
> > > > > non-backward compatible change of an API that wasn't accounted for,
> > > > > the discussion should be brought back to the Dev mailing list. I
> > think
> > > > > we can just add that to the FLIP bylaws.
> > > > > 2) How do we actually resolve the problem: is there anyone who has
> an
> > > > > idea on how we could introduce the proposed change while
> maintaining
> > > > > backwards compatibility, or do we agree that while this is an non
> > > > > desired situation, there is no better alternative unfortunately?
> > > > >
> > > > > Best regards,
> > > > >
> > > > > Martijn
> > > > >
> > > > > [1] https://issues.apache.org/jira/browse/FLINK-25857
> > > > > [2]
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-371%3A+Provide+initialization+context+for+Committer+creation+in+TwoPhaseCommittingSink
> > > > > [3]
> > https://github.com/apache/flink/pull/23555#discussion_r1371740397
> > > > >
> > > >
> > >
> >
>

Reply via email to