diff --git a/flink-cdc-composer/src/main/java/org/apache/flink/cdc/composer/flink/translator/DataSinkTranslator.java b/flink-cdc-composer/src/main/java/org/apache/flink/cdc/composer/flink/translator/DataSinkTranslator.java index ec165dd5635..a6188d2df71 100644 --- a/flink-cdc-composer/src/main/java/org/apache/flink/cdc/composer/flink/translator/DataSinkTranslator.java +++ b/flink-cdc-composer/src/main/java/org/apache/flink/cdc/composer/flink/translator/DataSinkTranslator.java @@ -21,6 +21,7 @@ import org.apache.flink.api.connector.sink2.Sink; import org.apache.flink.api.connector.sink2.TwoPhaseCommittingSink; import org.apache.flink.cdc.common.annotation.Internal; +import org.apache.flink.cdc.common.annotation.VisibleForTesting; import org.apache.flink.cdc.common.event.Event; import org.apache.flink.cdc.common.sink.DataSink; import org.apache.flink.cdc.common.sink.EventSinkProvider; @@ -71,7 +72,8 @@ public void translate( } } - private void sinkTo( + @VisibleForTesting + void sinkTo( DataStream input, Sink sink, String sinkName, @@ -85,7 +87,7 @@ private void sinkTo( if (sink instanceof TwoPhaseCommittingSink) { addCommittingTopology(sink, stream, sinkName, schemaOperatorID); } else { - input.transform( + stream.transform( SINK_WRITER_PREFIX + sinkName, CommittableMessageTypeInfo.noOutput(), new DataSinkWriterOperatorFactory<>(sink, schemaOperatorID)); diff --git a/flink-cdc-composer/src/test/java/org/apache/flink/cdc/composer/flink/translator/DataSinkTranslatorTest.java b/flink-cdc-composer/src/test/java/org/apache/flink/cdc/composer/flink/translator/DataSinkTranslatorTest.java new file mode 100644 index 00000000000..1970f3dfaec --- /dev/null +++ b/flink-cdc-composer/src/test/java/org/apache/flink/cdc/composer/flink/translator/DataSinkTranslatorTest.java @@ -0,0 +1,89 @@ +/* + * 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.composer.flink.translator; + +import org.apache.flink.api.connector.sink2.SinkWriter; +import org.apache.flink.api.dag.Transformation; +import org.apache.flink.cdc.common.event.Event; +import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.streaming.api.connector.sink2.WithPreWriteTopology; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.transformations.OneInputTransformation; + +import org.apache.flink.shaded.guava31.com.google.common.collect.Lists; + +import org.junit.Assert; +import org.junit.jupiter.api.Test; + +import java.io.IOException; +import java.util.ArrayList; + +/** A test for {@link DataSinkTranslator}. */ +public class DataSinkTranslatorTest { + + @Test + public void testPreWriteWithoutCommitSink() { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + ArrayList mockEvents = Lists.newArrayList(new EmptyEvent(), new EmptyEvent()); + DataStreamSource inputStream = env.fromCollection(mockEvents); + DataSinkTranslator translator = new DataSinkTranslator(); + + // Node hash must be a 32 character String that describes a hex code + String uid = "aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa"; + MockPreWriteWithoutCommitSink mockPreWriteWithoutCommitSink = + new MockPreWriteWithoutCommitSink(uid); + translator.sinkTo( + inputStream, + mockPreWriteWithoutCommitSink, + "testPreWriteWithoutCommitSink", + new OperatorID()); + + // Check if the `addPreWriteTopology` is called, and the uid is set when the transformation + // added + OneInputTransformation oneInputTransformation = + (OneInputTransformation) env.getTransformations().get(0); + Transformation reblanceTransformation = oneInputTransformation.getInputs().get(0); + Assert.assertEquals(uid, reblanceTransformation.getUserProvidedNodeHash()); + } + + private static class EmptyEvent implements Event {} + + private static class MockPreWriteWithoutCommitSink implements WithPreWriteTopology { + + private final String uid; + + public MockPreWriteWithoutCommitSink(String uid) { + this.uid = uid; + } + + @Override + public DataStream addPreWriteTopology(DataStream inputDataStream) { + // return a new DataSteam with specified uid + DataStream rebalance = inputDataStream.rebalance(); + rebalance.getTransformation().setUidHash(uid); + return rebalance; + } + + @Override + public SinkWriter createWriter(InitContext context) throws IOException { + return null; + } + } +} diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-starrocks/src/main/java/org/apache/flink/cdc/connectors/starrocks/sink/StarRocksUtils.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-starrocks/src/main/java/org/apache/flink/cdc/connectors/starrocks/sink/StarRocksUtils.java index 642a9dd3db1..ccab99c7a36 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-starrocks/src/main/java/org/apache/flink/cdc/connectors/starrocks/sink/StarRocksUtils.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-starrocks/src/main/java/org/apache/flink/cdc/connectors/starrocks/sink/StarRocksUtils.java @@ -305,10 +305,15 @@ public StarRocksColumn.Builder visit(DecimalType decimalType) { builder.setDecimalDigits(decimalType.getScale()); } else { builder.setDataType(VARCHAR); - // For a DecimalType with precision N, we may need N + 1 or N + 2 characters to store it as a + // For a DecimalType with precision N, we may need N + 1 or N + 2 characters to + // store it as a // string (one for negative sign, and one for decimal point) - builder.setColumnSize(Math.min( - decimalType.getScale() != 0? decimalType.getPrecision() + 2:decimalType.getPrecision() + 1, MAX_VARCHAR_SIZE)); + builder.setColumnSize( + Math.min( + decimalType.getScale() != 0 + ? decimalType.getPrecision() + 2 + : decimalType.getPrecision() + 1, + MAX_VARCHAR_SIZE)); } builder.setNullable(decimalType.isNullable()); return builder;