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 >