pvary commented on code in PR #23912:
URL: https://github.com/apache/flink/pull/23912#discussion_r1433804590
##########
flink-core/src/main/java/org/apache/flink/api/connector/sink2/TwoPhaseCommittingSink.java:
##########
@@ -40,17 +37,9 @@
* @param <CommT> The type of the committables.
*/
@PublicEvolving
-public interface TwoPhaseCommittingSink<InputT, CommT> extends Sink<InputT> {
-
- /**
- * Creates a {@link PrecommittingSinkWriter} that creates committables on
checkpoint or end of
- * input.
- *
- * @param context the runtime context.
- * @return A sink writer for the two-phase commit protocol.
- * @throws IOException for any failure during creation.
- */
- PrecommittingSinkWriter<InputT, CommT> createWriter(InitContext context)
throws IOException;
Review Comment:
@leonardBang: Thanks for the review!
This was the exact issue which prevented us to evolve the API.
`TwoPhaseCommittingSink` and `StatefulSink` overrides the same
`createWriter` method with different return types. If we need to evolve the
`createWriter` method for whatever reason in the future, then we need to
provide a default implementation for it, and the new method to allow backward
compatibility.
If there is a sink which implements both `TwoPhaseCommittingSink` and
`StatefulSink`, then it will not implement the new method, so the default
implementation will be inherited from both `TwoPhaseCommittingSink` and
`StatefulSink` which will cause a java compilation error.
That is why we need to remove this method, and use `instanceof` instead in
the implementation, to check the types.
Of course, I am open for any suggestion, but sadly nothing viable come up
during the discussions on the dev list [1], [2]
Thanks,
[1] - https://lists.apache.org/thread/344pzbrqbbb4w0sfj67km25msp7hxlyd
[2] - https://lists.apache.org/thread/h6nkgth838dlh5s90sd95zd6hlsxwx57
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]