lvyanquan commented on code in PR #3348:
URL: https://github.com/apache/flink-cdc/pull/3348#discussion_r1611558989


##########
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:
   Considering that we have had similar operations before in 
DataSinkWriterOperator, this change is acceptable for me.



##########
flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-doris/src/test/java/org/apache/flink/cdc/connectors/doris/sink/DorisMetadataApplierITCase.java:
##########
@@ -0,0 +1,341 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.cdc.connectors.doris.sink;
+
+import org.apache.flink.api.common.restartstrategy.RestartStrategies;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.cdc.common.configuration.Configuration;
+import org.apache.flink.cdc.common.event.AddColumnEvent;
+import org.apache.flink.cdc.common.event.AlterColumnTypeEvent;
+import org.apache.flink.cdc.common.event.CreateTableEvent;
+import org.apache.flink.cdc.common.event.DropColumnEvent;
+import org.apache.flink.cdc.common.event.Event;
+import org.apache.flink.cdc.common.event.RenameColumnEvent;
+import org.apache.flink.cdc.common.event.TableId;
+import org.apache.flink.cdc.common.pipeline.SchemaChangeBehavior;
+import org.apache.flink.cdc.common.schema.PhysicalColumn;
+import org.apache.flink.cdc.common.schema.Schema;
+import org.apache.flink.cdc.common.sink.DataSink;
+import org.apache.flink.cdc.common.types.DataTypes;
+import org.apache.flink.cdc.composer.definition.SinkDef;
+import org.apache.flink.cdc.composer.flink.coordination.OperatorIDGenerator;
+import org.apache.flink.cdc.composer.flink.translator.DataSinkTranslator;
+import org.apache.flink.cdc.composer.flink.translator.SchemaOperatorTranslator;
+import org.apache.flink.cdc.connectors.doris.sink.utils.DorisContainer;
+import org.apache.flink.cdc.connectors.doris.sink.utils.DorisSinkTestBase;
+import org.apache.flink.runtime.client.JobExecutionException;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Ignore;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+import static 
org.apache.flink.cdc.common.pipeline.PipelineOptions.DEFAULT_SCHEMA_OPERATOR_RPC_TIMEOUT;
+import static 
org.apache.flink.cdc.connectors.doris.sink.DorisDataSinkOptions.BENODES;
+import static 
org.apache.flink.cdc.connectors.doris.sink.DorisDataSinkOptions.FENODES;
+import static 
org.apache.flink.cdc.connectors.doris.sink.DorisDataSinkOptions.PASSWORD;
+import static 
org.apache.flink.cdc.connectors.doris.sink.DorisDataSinkOptions.SINK_ENABLE_BATCH_MODE;
+import static 
org.apache.flink.cdc.connectors.doris.sink.DorisDataSinkOptions.SINK_ENABLE_DELETE;
+import static 
org.apache.flink.cdc.connectors.doris.sink.DorisDataSinkOptions.USERNAME;
+
+/** IT tests for {@link DorisMetadataApplier}. */
+@RunWith(Parameterized.class)
+public class DorisMetadataApplierITCase extends DorisSinkTestBase {

Review Comment:
   It's better to separate it into two issues, Or more simply, split it into 
two commitments.



-- 
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: issues-unsubscr...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to