leonardBang commented on code in PR #3348:
URL: https://github.com/apache/flink-cdc/pull/3348#discussion_r1626875255
##########
flink-cdc-composer/src/main/java/org/apache/flink/cdc/composer/flink/translator/DataSinkTranslator.java:
##########
@@ -152,4 +152,37 @@ private String generateSinkName(SinkDef sinkDef) {
return sinkDef.getName()
.orElse(String.format("Flink CDC Event Sink: %s",
sinkDef.getType()));
}
+
+ private static <CommT> SimpleVersionedSerializer<CommT>
getCommittableSerializer(Object sink) {
+ // TwoPhaseCommittingSink has been deprecated, and its signature has
changed
+ // during Flink 1.18 to 1.19. Remove this when Flink 1.18 is no longer
supported.
+ try {
+ return (SimpleVersionedSerializer<CommT>)
+
sink.getClass().getDeclaredMethod("getCommittableSerializer").invoke(sink);
+ } catch (NoSuchMethodException | IllegalAccessException |
InvocationTargetException e) {
+ throw new RuntimeException("Failed to get CommittableSerializer",
e);
+ }
+ }
+
+ private static <CommT>
+ OneInputStreamOperatorFactory<CommittableMessage<CommT>,
CommittableMessage<CommT>>
+ getCommitterOperatorFactory(
+ Sink<Event> sink, boolean isBatchMode, boolean
isCheckpointingEnabled) {
+ // OneInputStreamOperatorFactory is an @Internal class, and its
signature has changed
Review Comment:
ditto
##########
flink-cdc-composer/src/main/java/org/apache/flink/cdc/composer/flink/translator/DataSinkTranslator.java:
##########
@@ -150,4 +150,37 @@ private String generateSinkName(SinkDef sinkDef) {
return sinkDef.getName()
.orElse(String.format("Flink CDC Event Sink: %s",
sinkDef.getType()));
}
+
+ private static <CommT> SimpleVersionedSerializer<CommT>
getCommittableSerializer(Object sink) {
+ // TwoPhaseCommittingSink has been deprecated, and its signature has
changed
+ // during Flink 1.18 to 1.19. Remove this when Flink 1.18 is no longer
supported.
+ try {
+ return (SimpleVersionedSerializer<CommT>)
+
sink.getClass().getDeclaredMethod("getCommittableSerializer").invoke(sink);
Review Comment:
+1, distribute CDC with various Flink versions in the future is recommended
just like other flink external connectors
##########
flink-cdc-composer/src/main/java/org/apache/flink/cdc/composer/flink/translator/DataSinkTranslator.java:
##########
@@ -152,4 +152,37 @@ private String generateSinkName(SinkDef sinkDef) {
return sinkDef.getName()
.orElse(String.format("Flink CDC Event Sink: %s",
sinkDef.getType()));
}
+
+ private static <CommT> SimpleVersionedSerializer<CommT>
getCommittableSerializer(Object sink) {
+ // TwoPhaseCommittingSink has been deprecated, and its signature has
changed
Review Comment:
we can add a //FIXME note
--
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]