diff --git a/.github/workflows/flink_cdc_base.yml b/.github/workflows/flink_cdc_base.yml index c5157abc924..982ea81e3e3 100644 --- a/.github/workflows/flink_cdc_base.yml +++ b/.github/workflows/flink_cdc_base.yml @@ -108,7 +108,7 @@ jobs: build_maven_parameter="${build_maven_parameter:+$build_maven_parameter }${{ inputs.custom-maven-parameter }}" - mvn --no-snapshot-updates -B -DskipTests -pl $compile_modules -am install && mvn --no-snapshot-updates -B $build_maven_parameter -pl $modules -DspecifiedParallelism=${{ inputs.parallelism }} -Duser.timezone=$jvm_timezone verify + mvn --no-snapshot-updates -B -DskipTests ${{ inputs.custom-maven-parameter }} -pl $compile_modules -am install && mvn --no-snapshot-updates -B $build_maven_parameter -pl $modules -DspecifiedParallelism=${{ inputs.parallelism }} -Duser.timezone=$jvm_timezone verify - name: Print JVM thread dumps when cancelled if: ${{ failure() }} diff --git a/.github/workflows/flink_cdc_ci.yml b/.github/workflows/flink_cdc_ci.yml index 5398ca43ec9..66d2f71be5a 100644 --- a/.github/workflows/flink_cdc_ci.yml +++ b/.github/workflows/flink_cdc_ci.yml @@ -63,6 +63,14 @@ jobs: with: java-versions: "[11]" modules: "['core']" + common_2_x: + name: Common Unit Tests 2.x + uses: ./.github/workflows/flink_cdc_base.yml + with: + java-versions: "[11]" + flink-versions: "['2.2.0']" + custom-maven-parameter: "-Pflink2" + modules: "['core_2.x']" pipeline-ut: name: Pipeline Unit Tests uses: ./.github/workflows/flink_cdc_base.yml @@ -88,6 +96,20 @@ jobs: flink-versions: "['1.19.3', '1.20.3']" modules: "['pipeline_e2e']" parallelism: ${{ matrix.parallelism }} + pipeline_e2e_2_x: + strategy: + max-parallel: 2 + fail-fast: false + matrix: + parallelism: [ 1, 4 ] + name: Pipeline E2E Tests 2.x (${{ matrix.parallelism }}-Parallelism) + uses: ./.github/workflows/flink_cdc_base.yml + with: + java-versions: "[11]" + flink-versions: "['2.2.0']" + custom-maven-parameter: "-Pflink2" + modules: "['pipeline_e2e_2.x']" + parallelism: ${{ matrix.parallelism }} source_e2e: name: Source E2E Tests uses: ./.github/workflows/flink_cdc_base.yml diff --git a/.github/workflows/flink_cdc_ci_nightly.yml b/.github/workflows/flink_cdc_ci_nightly.yml index d4801676421..5b01fc62cb1 100644 --- a/.github/workflows/flink_cdc_ci_nightly.yml +++ b/.github/workflows/flink_cdc_ci_nightly.yml @@ -54,6 +54,15 @@ jobs: with: java-versions: "[17]" modules: "['core']" + common_2_x: + if: github.repository == 'apache/flink-cdc' + name: Common Unit Tests 2.x + uses: ./.github/workflows/flink_cdc_base.yml + with: + java-versions: "[17]" + flink-versions: "['2.2.0']" + custom-maven-parameter: "-Pflink2" + modules: "['core_2.x']" pipeline-ut: if: github.repository == 'apache/flink-cdc' name: Pipeline Unit Tests @@ -82,6 +91,21 @@ jobs: flink-versions: "['1.19.3', '1.20.3']" modules: "['pipeline_e2e']" parallelism: ${{ matrix.parallelism }} + pipeline_e2e_2_x: + if: github.repository == 'apache/flink-cdc' + strategy: + max-parallel: 2 + fail-fast: false + matrix: + parallelism: [ 1, 4 ] + name: Pipeline E2E Tests 2.x (${{ matrix.parallelism }}-Parallelism) + uses: ./.github/workflows/flink_cdc_base.yml + with: + java-versions: "[17]" + custom-maven-parameter: "-Pflink2" + flink-versions: "['2.2.0']" + modules: "['pipeline_e2e_2.x']" + parallelism: ${{ matrix.parallelism }} source_e2e: if: github.repository == 'apache/flink-cdc' name: Source E2E Tests diff --git a/.github/workflows/modules.py b/.github/workflows/modules.py index d2d051c7677..d06c36d02dc 100755 --- a/.github/workflows/modules.py +++ b/.github/workflows/modules.py @@ -17,6 +17,14 @@ "flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-values" ] +MODULES_CORE_2_X = [ + "flink-cdc-cli", + "flink-cdc-common", + "flink-cdc-composer", + "flink-cdc-runtime", + "flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-values" +] + MODULES_PIPELINE_CONNECTORS = [ "flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-doris", "flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-elasticsearch", @@ -136,12 +144,17 @@ "flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests" ] +MODULES_PIPELINE_E2E_2_X = [ + "flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests-2.x" +] + MODULES_SOURCE_E2E = [ "flink-cdc-e2e-tests/flink-cdc-source-e2e-tests" ] ALL_MODULES = set( MODULES_CORE + + MODULES_CORE_2_X + MODULES_PIPELINE_CONNECTORS + MODULES_MYSQL_SOURCE + MODULES_MYSQL_PIPELINE + @@ -169,13 +182,18 @@ MODULES_SOURCE_E2E ) +# Modules that require the flink2 Maven profile to be activated +ALL_MODULES_FLINK2 = ALL_MODULES | set(MODULES_PIPELINE_E2E_2_X) + test_modules = set() compile_modules = set() for module in INPUT_MODULES.split(', '): - module_list = set(globals()['MODULES_' + module.upper().replace('-', '_')]) + module_list = set(globals()['MODULES_' + module.upper().replace('-', '_').replace('.', '_')]) test_modules |= module_list - if module == 'source_e2e' or module == 'pipeline_e2e': + if module == 'pipeline_e2e_2.x': + compile_modules |= ALL_MODULES_FLINK2 + elif module == 'source_e2e' or module == 'pipeline_e2e': compile_modules |= ALL_MODULES else: compile_modules |= module_list diff --git a/flink-cdc-cli/pom.xml b/flink-cdc-cli/pom.xml index 35a69cbd26f..6230f22a15f 100644 --- a/flink-cdc-cli/pom.xml +++ b/flink-cdc-cli/pom.xml @@ -82,4 +82,33 @@ limitations under the License. + + + flink2 + + ${flink.2.x.version} + + + + org.apache.flink + flink-cdc-flink2.2-compat + ${project.version} + provided + + + + + + org.apache.maven.plugins + maven-dependency-plugin + + + org.apache.maven.plugins + maven-surefire-plugin + + + + + + \ No newline at end of file diff --git a/flink-cdc-common/pom.xml b/flink-cdc-common/pom.xml index 95f7d5fe054..4101ad51b2e 100644 --- a/flink-cdc-common/pom.xml +++ b/flink-cdc-common/pom.xml @@ -44,4 +44,33 @@ limitations under the License. + + + flink2 + + ${flink.2.x.version} + + + + + org.apache.maven.plugins + maven-dependency-plugin + + + org.apache.maven.plugins + maven-surefire-plugin + + + + + + org.apache.flink + flink-cdc-flink2.2-compat + ${project.version} + provided + + + + + \ No newline at end of file diff --git a/flink-cdc-composer/pom.xml b/flink-cdc-composer/pom.xml index b5f1d8a187a..08bb6bc396e 100644 --- a/flink-cdc-composer/pom.xml +++ b/flink-cdc-composer/pom.xml @@ -95,4 +95,33 @@ limitations under the License. + + + flink2 + + ${flink.2.x.version} + + + + + org.apache.maven.plugins + maven-dependency-plugin + + + org.apache.maven.plugins + maven-surefire-plugin + + + + + + org.apache.flink + flink-cdc-flink2.2-compat + ${project.version} + provided + + + + + \ No newline at end of file diff --git a/flink-cdc-composer/src/test/java/org/apache/flink/cdc/composer/testsource/source/DistributedSourceFunction.java b/flink-cdc-composer/src/test/java/org/apache/flink/cdc/composer/testsource/source/DistributedSourceFunction.java index 58bbe3f42e1..d95034e0be6 100644 --- a/flink-cdc-composer/src/test/java/org/apache/flink/cdc/composer/testsource/source/DistributedSourceFunction.java +++ b/flink-cdc-composer/src/test/java/org/apache/flink/cdc/composer/testsource/source/DistributedSourceFunction.java @@ -79,8 +79,8 @@ public DistributedSourceFunction(int numOfTables, boolean distributedTables) { public void open(Configuration parameters) throws Exception { super.open(parameters); iotaCounter = 0; - subTaskId = getRuntimeContext().getIndexOfThisSubtask(); - parallelism = getRuntimeContext().getNumberOfParallelSubtasks(); + subTaskId = getRuntimeContext().getTaskInfo().getIndexOfThisSubtask(); + parallelism = getRuntimeContext().getTaskInfo().getNumberOfParallelSubtasks(); if (distributedTables) { tables = IntStream.range(0, numOfTables) diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-values/pom.xml b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-values/pom.xml index 3eda26dade8..96052ee640d 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-values/pom.xml +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-values/pom.xml @@ -107,4 +107,39 @@ limitations under the License. + + + flink2 + + ${flink.2.x.version} + + + + org.apache.flink + flink-streaming-java + ${flink.2.x.version} + provided + + + org.apache.flink + flink-cdc-flink2.2-compat + ${project.version} + provided + + + + + + org.apache.maven.plugins + maven-dependency-plugin + + + org.apache.maven.plugins + maven-surefire-plugin + + + + + + \ No newline at end of file diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-values/src/main/java/org/apache/flink/cdc/connectors/values/sink/ValuesDataSink.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-values/src/main/java/org/apache/flink/cdc/connectors/values/sink/ValuesDataSink.java index d5511b7bac6..c155946225d 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-values/src/main/java/org/apache/flink/cdc/connectors/values/sink/ValuesDataSink.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-values/src/main/java/org/apache/flink/cdc/connectors/values/sink/ValuesDataSink.java @@ -19,6 +19,7 @@ import org.apache.flink.api.connector.sink2.Sink; import org.apache.flink.api.connector.sink2.SinkWriter; +import org.apache.flink.api.connector.sink2.WriterInitContext; import org.apache.flink.cdc.common.annotation.Internal; import org.apache.flink.cdc.common.data.RecordData; import org.apache.flink.cdc.common.event.ChangeEvent; @@ -36,6 +37,7 @@ import org.apache.flink.cdc.common.utils.SchemaUtils; import org.apache.flink.cdc.connectors.values.ValuesDatabase; +import java.io.IOException; import java.io.Serializable; import java.util.HashMap; import java.util.List; @@ -101,8 +103,17 @@ public SinkWriter createWriter(InitContext context) { return new ValuesSinkWriter( materializedInMemory, print, - context.getSubtaskId(), - context.getNumberOfParallelSubtasks()); + context.getTaskInfo().getIndexOfThisSubtask(), + context.getTaskInfo().getNumberOfParallelSubtasks()); + } + + @Override + public SinkWriter createWriter(WriterInitContext context) throws IOException { + return new ValuesSinkWriter( + materializedInMemory, + print, + context.getTaskInfo().getIndexOfThisSubtask(), + context.getTaskInfo().getNumberOfParallelSubtasks()); } } diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-values/src/test/java/org/apache/flink/cdc/connectors/values/source/ValuesDataSourceITCase.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-values/src/test/java/org/apache/flink/cdc/connectors/values/source/ValuesDataSourceITCase.java index 7bea12a182e..640fa9cc359 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-values/src/test/java/org/apache/flink/cdc/connectors/values/source/ValuesDataSourceITCase.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-values/src/test/java/org/apache/flink/cdc/connectors/values/source/ValuesDataSourceITCase.java @@ -18,7 +18,6 @@ package org.apache.flink.cdc.connectors.values.source; import org.apache.flink.api.common.eventtime.WatermarkStrategy; -import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.cdc.common.data.binary.BinaryStringData; import org.apache.flink.cdc.common.event.CreateTableEvent; import org.apache.flink.cdc.common.event.DataChangeEvent; @@ -40,6 +39,7 @@ import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import java.util.ArrayList; import java.util.List; @@ -49,6 +49,7 @@ * different enumeration situations of {@link * org.apache.flink.cdc.connectors.values.source.ValuesDataSourceHelper}. */ +@Timeout(value = 60, unit = java.util.concurrent.TimeUnit.SECONDS) class ValuesDataSourceITCase { @BeforeEach @@ -68,7 +69,6 @@ public void after() { private void executeDataStreamJob(ValuesDataSourceHelper.EventSetId type) throws Exception { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.enableCheckpointing(3000); - env.setRestartStrategy(RestartStrategies.noRestart()); FlinkSourceProvider sourceProvider = (FlinkSourceProvider) new ValuesDataSource(type).getEventSourceProvider(); CloseableIterator events = diff --git a/flink-cdc-dist/pom.xml b/flink-cdc-dist/pom.xml index 132bea2cda2..5e9890bfeaa 100644 --- a/flink-cdc-dist/pom.xml +++ b/flink-cdc-dist/pom.xml @@ -54,6 +54,24 @@ limitations under the License. + + + flink2 + + + org.apache.flink + flink-cdc-flink2.2-compat + ${project.version} + + + org.apache.flink + flink-shaded-guava + ${flink.2.x.shaded.guava.version} + + + + + diff --git a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests-2.x/pom.xml b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests-2.x/pom.xml new file mode 100644 index 00000000000..2d2a1125919 --- /dev/null +++ b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests-2.x/pom.xml @@ -0,0 +1,454 @@ + + + + + flink-cdc-e2e-tests + org.apache.flink + ${revision} + + 4.0.0 + + flink-cdc-pipeline-e2e-tests-2.x + + + ${flink.2.x.version} + 8.0.27 + true + true + flink-${flink.version}-bin-scala_${scala.binary.version}.tgz + https://dlcdn.apache.org/flink/flink-${flink.version} + 1.6.8 + 1.37 + + + + + org.apache.flink + flink-cdc-e2e-utils + ${project.version} + test-jar + test + + + + + mysql + mysql-connector-java + + + com.google.protobuf + protobuf-java + + + ${mysql.driver.version} + test + + + + + org.apache.flink + flink-cdc-dist + ${project.version} + test + + + org.apache.flink + flink-connector-mysql-cdc + ${project.version} + test-jar + test + + + org.apache.flink + flink-cdc-pipeline-connector-values + ${project.version} + test + + + org.apache.flink + flink-connector-test-util + ${project.version} + test + + + org.apache.flink + flink-cdc-pipeline-udf-examples + ${project.version} + test + + + + + org.testcontainers + testcontainers + ${testcontainers.version} + test + + + org.testcontainers + junit-jupiter + ${testcontainers.version} + test + + + org.testcontainers + mysql + ${testcontainers.version} + test + + + + + org.openjdk.jmh + jmh-core + ${jmh.version} + test + + + + org.openjdk.jmh + jmh-generator-annprocess + ${jmh.version} + test + + + + + org.testcontainers + postgresql + ${testcontainers.version} + test + + + + + org.scala-lang + scala-library + ${scala.version} + test + + + + org.testcontainers + kafka + ${testcontainers.version} + test + + + + + org.apache.hadoop + hadoop-yarn-api + ${hadoop.version} + test + + + + jdk.tools + jdk.tools + + + ch.qos.reload4j + reload4j + + + org.slf4j + slf4j-reload4j + + + + + + org.apache.hadoop + hadoop-common + ${hadoop.version} + test + + + jdk.tools + jdk.tools + + + log4j + log4j + + + org.slf4j + slf4j-log4j12 + + + + + + org.apache.hadoop + hadoop-yarn-client + ${hadoop.version} + test + + + jdk.tools + jdk.tools + + + log4j + log4j + + + slf4j-log4j12 + org.slf4j + + + + + + org.apache.hadoop + hadoop-yarn-server-tests + ${hadoop.version} + test + test-jar + + + jdk.tools + jdk.tools + + + log4j + log4j + + + slf4j-log4j12 + org.slf4j + + + + + + org.apache.hadoop + hadoop-common + ${hadoop.version} + test + test-jar + + + jdk.tools + jdk.tools + + + log4j + log4j + + + slf4j-log4j12 + org.slf4j + + + + + + org.apache.flink + flink-yarn + ${flink.version} + test + + + + + + + src/test/resources + + **/flink-cdc.sh + **/flink-cdc.yaml + + + + + + + org.apache.maven.plugins + maven-surefire-plugin + + true + + + + + + org.apache.maven.plugins + maven-failsafe-plugin + 3.0.0-M5 + + + end-to-end-tests + + integration-test + verify + + + + **/*.* + + + **/MysqlE2eWithYarnApplicationITCase.java + + 1 + + ${project.basedir} + + + + + run-last-test + + integration-test + verify + + + + **/MysqlE2eWithYarnApplicationITCase.java + + + + + + + com.googlecode.maven-download-plugin + download-maven-plugin + 1.6.8 + + ${maven.plugin.download.cache.path} + ${project.build.directory} + 60000 + 3 + true + + + + download-flink-release + + wget + + test + + ${flink.release.download.skip} + ${flink.release.mirror}/${flink.release.name} + + + + + + + org.apache.maven.plugins + maven-dependency-plugin + + + copy-jars + test + + copy + + + + store-classpath-in-target-for-tests + process-test-resources + + build-classpath + + + ${project.build.directory}/yarn.classpath + org.apache.flink + + + + + + + + mysql + mysql-connector-java + ${mysql.driver.version} + mysql-driver.jar + jar + ${project.build.directory}/dependencies + + + + + org.apache.flink + flink-cdc-dist + ${project.version} + flink-cdc-dist.jar + jar + ${project.build.directory}/dependencies + + + + + org.apache.flink + flink-cdc-pipeline-connector-values + ${project.version} + values-cdc-pipeline-connector.jar + jar + ${project.build.directory}/dependencies + + + + + org.apache.flink + flink-cdc-pipeline-udf-examples + ${project.version} + udf-examples.jar + jar + ${project.build.directory}/dependencies + + + + + org.scala-lang + scala-library + ${scala.version} + scala-library.jar + jar + ${project.build.directory}/dependencies + + + + + + + + + + + flink2 + + false + false + + + + \ No newline at end of file diff --git a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests-2.x/src/test/java/org/apache/flink/cdc/pipeline/tests/ValuesE2eITCase.java b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests-2.x/src/test/java/org/apache/flink/cdc/pipeline/tests/ValuesE2eITCase.java new file mode 100644 index 00000000000..7d863333dfa --- /dev/null +++ b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests-2.x/src/test/java/org/apache/flink/cdc/pipeline/tests/ValuesE2eITCase.java @@ -0,0 +1,260 @@ +/* + * 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.pipeline.tests; + +import org.apache.flink.cdc.pipeline.tests.utils.PipelineTestEnvironment; + +import org.junit.jupiter.api.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.time.Duration; + +/** End-to-end tests for values cdc pipeline job. */ +class ValuesE2eITCase extends PipelineTestEnvironment { + private static final Logger LOG = LoggerFactory.getLogger(ValuesE2eITCase.class); + + @Test + void testValuesSingleSplitSingleTable() throws Exception { + String pipelineJob = + String.format( + "source:\n" + + " type: values\n" + + " event-set.id: SINGLE_SPLIT_SINGLE_TABLE\n" + + "\n" + + "sink:\n" + + " type: values\n" + + " print.enabled: true\n" + + "\n" + + "pipeline:\n" + + " parallelism: %d", + parallelism); + + submitPipelineJob(pipelineJob); + waitUntilJobFinished(Duration.ofSeconds(30)); + LOG.info("Pipeline job is running"); + + validateResult( + "CreateTableEvent{tableId=default_namespace.default_schema.table1, schema=columns={`col1` STRING,`col2` STRING}, primaryKeys=col1, options=()}", + "DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[], after=[1, 1], op=INSERT, meta=()}", + "DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[], after=[2, 2], op=INSERT, meta=()}", + "DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[], after=[3, 3], op=INSERT, meta=()}", + "AddColumnEvent{tableId=default_namespace.default_schema.table1, addedColumns=[ColumnWithPosition{column=`col3` STRING, position=LAST, existedColumnName=null}]}", + "RenameColumnEvent{tableId=default_namespace.default_schema.table1, nameMapping={col2=newCol2, col3=newCol3}}", + "DropColumnEvent{tableId=default_namespace.default_schema.table1, droppedColumnNames=[newCol2]}", + "DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[1, 1], after=[], op=DELETE, meta=()}", + "DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[2, ], after=[2, x], op=UPDATE, meta=()}"); + } + + @Test + void testValuesSingleSplitMultiTables() throws Exception { + String pipelineJob = + String.format( + "source:\n" + + " type: values\n" + + " event-set.id: SINGLE_SPLIT_MULTI_TABLES\n" + + "\n" + + "sink:\n" + + " type: values\n" + + " print.enabled: true\n" + + "\n" + + "pipeline:\n" + + " parallelism: %d", + parallelism); + + submitPipelineJob(pipelineJob); + waitUntilJobFinished(Duration.ofSeconds(30)); + LOG.info("Pipeline job is running"); + + validateResult( + "CreateTableEvent{tableId=default_namespace.default_schema.table1, schema=columns={`col1` STRING,`col2` STRING}, primaryKeys=col1, options=()}", + "CreateTableEvent{tableId=default_namespace.default_schema.table2, schema=columns={`col1` STRING,`col2` STRING}, primaryKeys=col1, options=()}", + "DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[], after=[1, 1], op=INSERT, meta=()}", + "DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[], after=[2, 2], op=INSERT, meta=()}", + "DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[], after=[3, 3], op=INSERT, meta=()}", + "AddColumnEvent{tableId=default_namespace.default_schema.table1, addedColumns=[ColumnWithPosition{column=`col3` STRING, position=LAST, existedColumnName=null}]}", + "DataChangeEvent{tableId=default_namespace.default_schema.table2, before=[], after=[1, 1], op=INSERT, meta=()}", + "DataChangeEvent{tableId=default_namespace.default_schema.table2, before=[], after=[2, 2], op=INSERT, meta=()}", + "DataChangeEvent{tableId=default_namespace.default_schema.table2, before=[], after=[3, 3], op=INSERT, meta=()}", + "RenameColumnEvent{tableId=default_namespace.default_schema.table1, nameMapping={col2=newCol2, col3=newCol3}}", + "DropColumnEvent{tableId=default_namespace.default_schema.table1, droppedColumnNames=[newCol2]}", + "DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[1, 1], after=[], op=DELETE, meta=()}", + "DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[2, 2], after=[2, x], op=UPDATE, meta=()}"); + } + + @Test + void testValuesMultiSplitsSingleTable() throws Exception { + String pipelineJob = + String.format( + "source:\n" + + " type: values\n" + + " event-set.id: MULTI_SPLITS_SINGLE_TABLE\n" + + "\n" + + "sink:\n" + + " type: values\n" + + " print.enabled: true\n" + + "\n" + + "pipeline:\n" + + " parallelism: %d", + parallelism); + + submitPipelineJob(pipelineJob); + waitUntilJobFinished(Duration.ofSeconds(30)); + LOG.info("Pipeline job is running"); + + validateResult( + "CreateTableEvent{tableId=default_namespace.default_schema.table1, schema=columns={`col1` STRING,`col2` STRING}, primaryKeys=col1, options=()}", + "DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[], after=[1, 1], op=INSERT, meta=()}", + "DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[], after=[2, 2], op=INSERT, meta=()}", + "DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[], after=[3, 3], op=INSERT, meta=()}", + "DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[], after=[4, 4], op=INSERT, meta=()}", + "DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[], after=[5, 5], op=INSERT, meta=()}", + "DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[], after=[6, 6], op=INSERT, meta=()}", + "DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[2, 2], after=[], op=DELETE, meta=()}", + "DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[4, 4], after=[], op=DELETE, meta=()}", + "DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[6, 6], after=[], op=DELETE, meta=()}", + "AddColumnEvent{tableId=default_namespace.default_schema.table1, addedColumns=[ColumnWithPosition{column=`col3` STRING, position=LAST, existedColumnName=null}]}", + "DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[1, 1, ], after=[1, 1, x], op=UPDATE, meta=()}", + "DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[3, 3, ], after=[3, 3, x], op=UPDATE, meta=()}"); + } + + @Test + void testValuesSingleSplitSingleTableWithDefaultValue() throws Exception { + String pipelineJob = + String.format( + "source:\n" + + " type: values\n" + + " event-set.id: SINGLE_SPLIT_SINGLE_TABLE_WITH_DEFAULT_VALUE\n" + + "\n" + + "sink:\n" + + " type: values\n" + + " print.enabled: true\n" + + "\n" + + "pipeline:\n" + + " parallelism: %d", + parallelism); + + submitPipelineJob(pipelineJob); + waitUntilJobFinished(Duration.ofSeconds(30)); + LOG.info("Pipeline job is running"); + + validateResult( + "CreateTableEvent{tableId=default_namespace.default_schema.table1, schema=columns={`col1` STRING,`col2` STRING}, primaryKeys=col1, options=()}", + "DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[], after=[1, 1], op=INSERT, meta=()}", + "DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[], after=[2, 2], op=INSERT, meta=()}", + "DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[], after=[3, 3], op=INSERT, meta=()}", + "AddColumnEvent{tableId=default_namespace.default_schema.table1, addedColumns=[ColumnWithPosition{column=`col3` STRING, position=LAST, existedColumnName=null}]}", + "RenameColumnEvent{tableId=default_namespace.default_schema.table1, nameMapping={col2=newCol2, col3=newCol3}}", + "DropColumnEvent{tableId=default_namespace.default_schema.table1, droppedColumnNames=[newCol2]}", + "DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[1, 1], after=[], op=DELETE, meta=()}", + "DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[2, ], after=[2, x], op=UPDATE, meta=()}", + "AddColumnEvent{tableId=default_namespace.default_schema.table1, addedColumns=[ColumnWithPosition{column=`colWithDefault` STRING 'flink', position=LAST, existedColumnName=null}]}", + "RenameColumnEvent{tableId=default_namespace.default_schema.table1, nameMapping={colWithDefault=newColWithDefault}}"); + } + + @Test + void testValuesSingleSplitSingleBatchTable() throws Exception { + String pipelineJob = + String.format( + "source:\n" + + " type: values\n" + + " event-set.id: SINGLE_SPLIT_SINGLE_BATCH_TABLE\n" + + "\n" + + "sink:\n" + + " type: values\n" + + " print.enabled: true\n" + + "\n" + + "pipeline:\n" + + " parallelism: %d\n" + + " execution.runtime-mode: BATCH", + parallelism); + + submitPipelineJob(pipelineJob); + waitUntilJobFinished(Duration.ofSeconds(30)); + LOG.info("Pipeline job is running"); + + validateResult( + "CreateTableEvent{tableId=default_namespace.default_schema.table1, schema=columns={`col1` STRING,`col2` STRING}, primaryKeys=col1, options=()}", + "DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[], after=[1, 1], op=INSERT, meta=()}", + "DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[], after=[2, 2], op=INSERT, meta=()}", + "DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[], after=[3, 3], op=INSERT, meta=()}"); + } + + @Test + void testValuesSingleSplitMultiBatchTable() throws Exception { + String pipelineJob = + String.format( + "source:\n" + + " type: values\n" + + " event-set.id: SINGLE_SPLIT_MULTI_BATCH_TABLE\n" + + "\n" + + "sink:\n" + + " type: values\n" + + " print.enabled: true\n" + + "\n" + + "pipeline:\n" + + " parallelism: %d\n" + + " execution.runtime-mode: BATCH", + parallelism); + + submitPipelineJob(pipelineJob); + waitUntilJobFinished(Duration.ofSeconds(30)); + LOG.info("Pipeline job is running"); + + validateResult( + "CreateTableEvent{tableId=default_namespace.default_schema.table1, schema=columns={`col1` STRING,`col2` STRING}, primaryKeys=col1, options=()}", + "CreateTableEvent{tableId=default_namespace.default_schema.table2, schema=columns={`col1` STRING,`col2` STRING}, primaryKeys=col1, options=()}", + "DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[], after=[1, 1], op=INSERT, meta=()}", + "DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[], after=[2, 2], op=INSERT, meta=()}", + "DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[], after=[3, 3], op=INSERT, meta=()}", + "DataChangeEvent{tableId=default_namespace.default_schema.table2, before=[], after=[1, 1], op=INSERT, meta=()}", + "DataChangeEvent{tableId=default_namespace.default_schema.table2, before=[], after=[2, 2], op=INSERT, meta=()}", + "DataChangeEvent{tableId=default_namespace.default_schema.table2, before=[], after=[3, 3], op=INSERT, meta=()}"); + } + + @Test + void testValuesMultiSplitsSingleBatchTable() throws Exception { + String pipelineJob = + String.format( + "source:\n" + + " type: values\n" + + " event-set.id: MULTI_SPLITS_SINGLE_BATCH_TABLE\n" + + "\n" + + "sink:\n" + + " type: values\n" + + " print.enabled: true\n" + + "\n" + + "pipeline:\n" + + " parallelism: %d\n" + + " execution.runtime-mode: BATCH", + parallelism); + + submitPipelineJob(pipelineJob); + waitUntilJobFinished(Duration.ofSeconds(30)); + LOG.info("Pipeline job is running"); + + validateResult( + "CreateTableEvent{tableId=default_namespace.default_schema.table1, schema=columns={`col1` STRING,`col2` STRING}, primaryKeys=col1, options=()}", + "DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[], after=[1, 1], op=INSERT, meta=()}", + "DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[], after=[2, 2], op=INSERT, meta=()}", + "DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[], after=[3, 3], op=INSERT, meta=()}", + "DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[], after=[4, 4], op=INSERT, meta=()}", + "DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[], after=[5, 5], op=INSERT, meta=()}", + "DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[], after=[6, 6], op=INSERT, meta=()}"); + } +} diff --git a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests-2.x/src/test/java/org/apache/flink/cdc/pipeline/tests/utils/PipelineTestEnvironment.java b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests-2.x/src/test/java/org/apache/flink/cdc/pipeline/tests/utils/PipelineTestEnvironment.java new file mode 100644 index 00000000000..b1d57c77830 --- /dev/null +++ b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests-2.x/src/test/java/org/apache/flink/cdc/pipeline/tests/utils/PipelineTestEnvironment.java @@ -0,0 +1,505 @@ +/* + * 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.pipeline.tests.utils; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.JobStatus; +import org.apache.flink.api.common.time.Deadline; +import org.apache.flink.cdc.common.test.utils.TestUtils; +import org.apache.flink.cdc.connectors.mysql.testutils.MySqlContainer; +import org.apache.flink.cdc.connectors.mysql.testutils.MySqlVersion; +import org.apache.flink.client.deployment.StandaloneClusterId; +import org.apache.flink.client.program.rest.RestClusterClient; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.RestOptions; +import org.apache.flink.runtime.client.JobStatusMessage; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.util.TestLogger; + +import com.github.dockerjava.api.DockerClient; +import com.github.dockerjava.api.command.ExecCreateCmdResponse; +import com.github.dockerjava.api.model.Volume; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testcontainers.DockerClientFactory; +import org.testcontainers.containers.BindMode; +import org.testcontainers.containers.Container.ExecResult; +import org.testcontainers.containers.GenericContainer; +import org.testcontainers.containers.Network; +import org.testcontainers.containers.output.FrameConsumerResultCallback; +import org.testcontainers.containers.output.OutputFrame; +import org.testcontainers.containers.output.Slf4jLogConsumer; +import org.testcontainers.containers.output.ToStringConsumer; +import org.testcontainers.images.builder.Transferable; +import org.testcontainers.junit.jupiter.Container; +import org.testcontainers.junit.jupiter.Testcontainers; +import org.testcontainers.lifecycle.Startables; +import org.testcontainers.utility.MountableFile; + +import javax.annotation.Nullable; + +import java.io.File; +import java.io.IOException; +import java.net.URL; +import java.nio.file.Files; +import java.nio.file.Path; +import java.time.Duration; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.function.Function; +import java.util.stream.Stream; + +import static org.apache.flink.util.Preconditions.checkState; + +/** Test environment running pipeline job on Flink containers. */ +@Testcontainers +public abstract class PipelineTestEnvironment extends TestLogger { + + private static final Logger LOG = LoggerFactory.getLogger(PipelineTestEnvironment.class); + + protected Integer parallelism = getParallelism(); + + private int getParallelism() { + try { + return Integer.parseInt(System.getProperty("specifiedParallelism")); + } catch (NumberFormatException ex) { + LOG.warn( + "Unable to parse specified parallelism configuration ({} provided). Use 4 by default.", + System.getProperty("specifiedParallelism")); + return 4; + } + } + + // ------------------------------------------------------------------------------------------ + // MySQL Variables (we always use MySQL as the data source for easier verifying) + // ------------------------------------------------------------------------------------------ + protected static final String MYSQL_TEST_USER = "mysqluser"; + protected static final String MYSQL_TEST_PASSWORD = "mysqlpw"; + protected static final String INTER_CONTAINER_MYSQL_ALIAS = "mysql"; + protected static final Duration EVENT_WAITING_TIMEOUT = Duration.ofMinutes(3); + protected static final Duration STARTUP_WAITING_TIMEOUT = Duration.ofMinutes(5); + + public static final Network NETWORK = Network.newNetwork(); + + @Container + protected static final MySqlContainer MYSQL = + (MySqlContainer) + new MySqlContainer(MySqlVersion.V8_0) + .withConfigurationOverride("docker/mysql/my.cnf") + .withSetupSQL("docker/mysql/setup.sql") + .withDatabaseName("flink-test") + .withUsername("flinkuser") + .withPassword("flinkpw") + .withNetwork(NETWORK) + .withNetworkAliases(INTER_CONTAINER_MYSQL_ALIAS) + .withLogConsumer(new Slf4jLogConsumer(LOG)); + + // ------------------------------------------------------------------------------------------ + // Flink Variables + // ------------------------------------------------------------------------------------------ + protected static final int JOB_MANAGER_REST_PORT = 8081; + protected static final String INTER_CONTAINER_JM_ALIAS = "jobmanager"; + protected static final String INTER_CONTAINER_TM_ALIAS = "taskmanager"; + protected static final List EXTERNAL_PROPS = + Arrays.asList( + String.format("jobmanager.rpc.address: %s", INTER_CONTAINER_JM_ALIAS), + "jobmanager.bind-host: 0.0.0.0", + "taskmanager.bind-host: 0.0.0.0", + "rest.bind-address: 0.0.0.0", + "rest.address: 0.0.0.0", + "jobmanager.memory.process.size: 1GB", + "query.server.port: 6125", + "blob.server.port: 6124", + "taskmanager.numberOfTaskSlots: 10", + "parallelism.default: 4", + "execution.checkpointing.interval: 300", + "state.backend.type: hashmap", + "env.java.default-opts.all: --add-exports=java.base/sun.net.util=ALL-UNNAMED --add-exports=java.rmi/sun.rmi.registry=ALL-UNNAMED --add-exports=jdk.compiler/com.sun.tools.javac.api=ALL-UNNAMED --add-exports=jdk.compiler/com.sun.tools.javac.file=ALL-UNNAMED --add-exports=jdk.compiler/com.sun.tools.javac.parser=ALL-UNNAMED --add-exports=jdk.compiler/com.sun.tools.javac.tree=ALL-UNNAMED --add-exports=jdk.compiler/com.sun.tools.javac.util=ALL-UNNAMED --add-exports=java.security.jgss/sun.security.krb5=ALL-UNNAMED --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.text=ALL-UNNAMED --add-opens=java.base/java.time=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.locks=ALL-UNNAMED --add-opens=java.base/jdk.internal.loader=ALL-UNNAMED --add-opens=java.base/java.security=ALL-UNNAMED --add-exports=java.base/sun.net.www=ALL-UNNAMED -Doracle.jdbc.timezoneAsRegion=false", + "execution.checkpointing.savepoint-dir: file:///opt/flink", + "restart-strategy.type: off", + "pekko.ask.timeout: 60s", + // Set off-heap memory explicitly to avoid "java.lang.OutOfMemoryError: Direct + // buffer memory" error. + "taskmanager.memory.task.off-heap.size: 128mb", + // Fix `java.lang.OutOfMemoryError: Metaspace. The metaspace out-of-memory error + // has occurred` error. + "taskmanager.memory.jvm-metaspace.size: 512mb"); + public static final String FLINK_PROPERTIES = String.join("\n", EXTERNAL_PROPS); + + @Nullable protected RestClusterClient restClusterClient; + + protected GenericContainer jobManager; + protected GenericContainer taskManager; + protected Volume sharedVolume = new Volume("/tmp/shared"); + + protected ToStringConsumer jobManagerConsumer; + + protected ToStringConsumer taskManagerConsumer; + + protected String flinkVersion = getFlinkVersion(); + + public static String getFlinkVersion() { + return "2.2.0"; + } + + protected List copyJarToFlinkLib() { + return Collections.emptyList(); + } + + @BeforeEach + public void before() throws Exception { + LOG.info("Starting containers..."); + jobManagerConsumer = new ToStringConsumer(); + jobManager = + new GenericContainer<>(getFlinkDockerImageTag()) + .withCommand("jobmanager") + .withNetwork(NETWORK) + .withNetworkAliases(INTER_CONTAINER_JM_ALIAS) + .withExposedPorts(JOB_MANAGER_REST_PORT) + .withEnv("FLINK_PROPERTIES", FLINK_PROPERTIES) + .withCreateContainerCmdModifier(cmd -> cmd.withVolumes(sharedVolume)) + .withLogConsumer(jobManagerConsumer); + + List jarToCopy = copyJarToFlinkLib(); + if (!jarToCopy.isEmpty()) { + for (String jar : jarToCopy) { + jobManager.withCopyFileToContainer( + MountableFile.forHostPath(TestUtils.getResource(jar)), "/opt/flink/lib/"); + } + } + + Startables.deepStart(Stream.of(jobManager)).join(); + runInContainerAsRoot(jobManager, "chmod", "0777", "-R", sharedVolume.toString()); + LOG.info("JobManager is started."); + + taskManagerConsumer = new ToStringConsumer(); + taskManager = + new GenericContainer<>(getFlinkDockerImageTag()) + .withCommand("taskmanager") + .withNetwork(NETWORK) + .withNetworkAliases(INTER_CONTAINER_TM_ALIAS) + .withEnv("FLINK_PROPERTIES", FLINK_PROPERTIES) + .dependsOn(jobManager) + .withVolumesFrom(jobManager, BindMode.READ_WRITE) + .withLogConsumer(taskManagerConsumer); + Startables.deepStart(Stream.of(taskManager)).join(); + runInContainerAsRoot(taskManager, "chmod", "0777", "-R", sharedVolume.toString()); + LOG.info("TaskManager is started."); + + TarballFetcher.fetchLatest(jobManager); + LOG.info("CDC executables deployed."); + } + + @AfterEach + public void after() { + if (restClusterClient != null) { + restClusterClient.close(); + } + if (jobManager != null) { + jobManager.stop(); + } + if (taskManager != null) { + taskManager.stop(); + } + } + + /** + * Submits a YAML job to the running cluster with latest CDC version, without from previous + * savepoint states. + */ + public JobID submitPipelineJob(String pipelineJob, Path... jars) throws Exception { + return submitPipelineJob( + TarballFetcher.CdcVersion.SNAPSHOT, pipelineJob, null, false, jars); + } + + /** + * Submits a YAML job to the running cluster with specific CDC version, without from previous + * savepoint states. + */ + public JobID submitPipelineJob( + TarballFetcher.CdcVersion version, String pipelineJob, Path... jars) throws Exception { + return submitPipelineJob(version, pipelineJob, null, false, jars); + } + + /** Submits a YAML job to the running cluster with latest CDC version. */ + public JobID submitPipelineJob( + String pipelineJob, + @Nullable String savepointPath, + boolean allowNonRestoredState, + Path... jars) + throws Exception { + return submitPipelineJob( + TarballFetcher.CdcVersion.SNAPSHOT, + pipelineJob, + savepointPath, + allowNonRestoredState, + jars); + } + + public JobID submitPipelineJob( + TarballFetcher.CdcVersion version, + String pipelineJob, + @Nullable String savepointPath, + boolean allowNonRestoredState, + Path... jars) + throws Exception { + + // Prepare external JAR dependencies + List paths = new ArrayList<>(Arrays.asList(jars)); + List containerPaths = new ArrayList<>(); + paths.add(TestUtils.getResource("mysql-driver.jar")); + + for (Path jar : paths) { + String containerPath = version.workDir() + "/lib/" + jar.getFileName(); + jobManager.copyFileToContainer(MountableFile.forHostPath(jar), containerPath); + containerPaths.add(containerPath); + } + + containerPaths.add(version.workDir() + "/lib/values-cdc-pipeline-connector.jar"); + + StringBuilder sb = new StringBuilder(); + for (String containerPath : containerPaths) { + sb.append(" --jar ").append(containerPath); + } + + jobManager.copyFileToContainer( + Transferable.of(pipelineJob), version.workDir() + "/conf/pipeline.yaml"); + + String commands = + version.workDir() + + "/bin/flink-cdc.sh " + + version.workDir() + + "/conf/pipeline.yaml --flink-home /opt/flink" + + sb; + + if (savepointPath != null) { + commands += " --from-savepoint " + savepointPath; + if (allowNonRestoredState) { + commands += " --allow-nonRestored-state"; + } + } + LOG.info("Execute command: {}", commands); + ExecResult execResult = executeAndCheck(jobManager, commands); + return Arrays.stream(execResult.getStdout().split("\n")) + .filter(line -> line.startsWith("Job ID: ")) + .findFirst() + .map(line -> line.split(": ")[1]) + .map(JobID::fromHexString) + .orElse(null); + } + + public String stopJobWithSavepoint(JobID jobID) { + String savepointPath = "/opt/flink/"; + ExecResult result = + executeAndCheck( + jobManager, + "flink", + "stop", + jobID.toHexString(), + "--savepointPath", + savepointPath); + + return Arrays.stream(result.getStdout().split("\n")) + .filter(line -> line.startsWith("Savepoint completed.")) + .findFirst() + .map(line -> line.split("Path: file:")[1]) + .orElseThrow( + () -> new RuntimeException("Failed to parse savepoint path from stdout.")); + } + + public void cancelJob(JobID jobID) { + executeAndCheck(jobManager, "flink", "cancel", jobID.toHexString()); + } + + /** + * Get {@link RestClusterClient} connected to this FlinkContainer. + * + *

This method lazily initializes the REST client on-demand. + */ + public RestClusterClient getRestClusterClient() { + if (restClusterClient != null) { + return restClusterClient; + } + checkState( + jobManager.isRunning(), + "Cluster client should only be retrieved for a running cluster"); + try { + final Configuration clientConfiguration = new Configuration(); + clientConfiguration.set(RestOptions.ADDRESS, jobManager.getHost()); + clientConfiguration.set( + RestOptions.PORT, jobManager.getMappedPort(JOB_MANAGER_REST_PORT)); + this.restClusterClient = + new RestClusterClient<>(clientConfiguration, StandaloneClusterId.getInstance()); + } catch (Exception e) { + throw new IllegalStateException( + "Failed to create client for Flink container cluster", e); + } + return restClusterClient; + } + + public void waitUntilJobRunning(Duration timeout) { + waitUntilJobState(timeout, JobStatus.RUNNING); + } + + public void waitUntilJobFinished(Duration timeout) { + waitUntilJobState(timeout, JobStatus.FINISHED); + } + + public void waitUntilJobState(Duration timeout, JobStatus expectedStatus) { + RestClusterClient clusterClient = getRestClusterClient(); + Deadline deadline = Deadline.fromNow(timeout); + while (deadline.hasTimeLeft()) { + Collection jobStatusMessages; + try { + jobStatusMessages = clusterClient.listJobs().get(10, TimeUnit.SECONDS); + } catch (Exception e) { + LOG.warn("Error when fetching job status.", e); + continue; + } + if (jobStatusMessages != null && !jobStatusMessages.isEmpty()) { + JobStatusMessage message = jobStatusMessages.iterator().next(); + JobStatus jobStatus = message.getJobState(); + if (!expectedStatus.isTerminalState() && jobStatus.isTerminalState()) { + throw new ValidationException( + String.format( + "Job has been terminated! JobName: %s, JobID: %s, Status: %s", + message.getJobName(), + message.getJobId(), + message.getJobState())); + } else if (jobStatus == expectedStatus) { + return; + } + } + } + } + + protected String getFlinkDockerImageTag() { + if (System.getProperty("java.specification.version").equals("17")) { + return String.format("flink:%s-scala_2.12-java17", flinkVersion); + } + return String.format("flink:%s-scala_2.12-java11", flinkVersion); + } + + private ExecResult executeAndCheck(GenericContainer container, String... command) { + String joinedCommand = String.join(" ", command); + try { + LOG.info("Executing command {}", joinedCommand); + ExecResult execResult = + container.execInContainer("bash", "-c", String.join(" ", command)); + LOG.info(execResult.getStdout()); + if (execResult.getExitCode() == 0) { + LOG.info("Command executed successfully."); + return execResult; + } else { + LOG.error(execResult.getStderr()); + throw new AssertionError( + "Failed when submitting the pipeline job.\n" + + "Exit code: " + + execResult.getExitCode() + + "\n" + + "StdOut: " + + execResult.getStdout() + + "\n" + + "StdErr: " + + execResult.getStderr()); + } + } catch (Exception e) { + throw new RuntimeException( + "Failed to execute command " + joinedCommand + " in container " + container); + } + } + + public void runInContainerAsRoot(GenericContainer container, String... command) + throws InterruptedException { + ToStringConsumer stdoutConsumer = new ToStringConsumer(); + ToStringConsumer stderrConsumer = new ToStringConsumer(); + DockerClient dockerClient = DockerClientFactory.instance().client(); + ExecCreateCmdResponse execCreateCmdResponse = + dockerClient + .execCreateCmd(container.getContainerId()) + .withUser("root") + .withCmd(command) + .exec(); + FrameConsumerResultCallback callback = new FrameConsumerResultCallback(); + callback.addConsumer(OutputFrame.OutputType.STDOUT, stdoutConsumer); + callback.addConsumer(OutputFrame.OutputType.STDERR, stderrConsumer); + dockerClient.execStartCmd(execCreateCmdResponse.getId()).exec(callback).awaitCompletion(); + } + + protected List readLines(String resource) throws IOException { + final URL url = PipelineTestEnvironment.class.getClassLoader().getResource(resource); + assert url != null; + Path path = new File(url.getFile()).toPath(); + return Files.readAllLines(path); + } + + protected void validateResult(String... expectedEvents) throws Exception { + validateResult(Function.identity(), expectedEvents); + } + + protected void validateResult(Function mapper, String... expectedEvents) + throws Exception { + validateResult( + taskManagerConsumer, Stream.of(expectedEvents).map(mapper).toArray(String[]::new)); + } + + protected void validateResult(ToStringConsumer consumer, String... expectedEvents) + throws Exception { + for (String event : expectedEvents) { + waitUntilSpecificEvent(consumer, event); + } + } + + protected void validateResult( + ToStringConsumer consumer, Function mapper, String... expectedEvents) + throws Exception { + validateResult(consumer, Stream.of(expectedEvents).map(mapper).toArray(String[]::new)); + } + + protected void waitUntilSpecificEvent(String event) throws Exception { + waitUntilSpecificEvent(taskManagerConsumer, event); + } + + protected void waitUntilSpecificEvent(ToStringConsumer consumer, String event) + throws Exception { + boolean result = false; + long endTimeout = System.currentTimeMillis() + EVENT_WAITING_TIMEOUT.toMillis(); + while (System.currentTimeMillis() < endTimeout) { + String stdout = consumer.toUtf8String(); + if (stdout.contains(event + "\n")) { + result = true; + break; + } + Thread.sleep(1000); + } + if (!result) { + throw new TimeoutException( + "failed to get specific event: " + + event + + " from stdout: " + + consumer.toUtf8String()); + } + } +} diff --git a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests-2.x/src/test/java/org/apache/flink/cdc/pipeline/tests/utils/TarballFetcher.java b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests-2.x/src/test/java/org/apache/flink/cdc/pipeline/tests/utils/TarballFetcher.java new file mode 100644 index 00000000000..8bd16d805f2 --- /dev/null +++ b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests-2.x/src/test/java/org/apache/flink/cdc/pipeline/tests/utils/TarballFetcher.java @@ -0,0 +1,151 @@ +/* + * 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.pipeline.tests.utils; + +import org.apache.flink.cdc.common.test.utils.TestUtils; + +import org.apache.commons.io.FileUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testcontainers.containers.GenericContainer; +import org.testcontainers.utility.MountableFile; + +import java.net.URL; +import java.nio.file.Files; +import java.nio.file.Path; +import java.time.Duration; +import java.util.Arrays; +import java.util.List; + +/** Obtain and downloads corresponding Flink CDC tarball files. */ +public abstract class TarballFetcher { + + private static final Logger LOG = LoggerFactory.getLogger(TarballFetcher.class); + + public static void fetchAll(GenericContainer container) throws Exception { + fetch(container, CdcVersion.values()); + } + + public static void fetchLatest(GenericContainer container) throws Exception { + fetch(container, CdcVersion.SNAPSHOT); + } + + public static void fetch(GenericContainer container, CdcVersion... versions) + throws Exception { + for (CdcVersion version : versions) { + TarballFetcher.fetchInternal(container, version); + } + } + + private static void fetchInternal(GenericContainer container, CdcVersion version) + throws Exception { + LOG.info("Trying to download CDC tarball @ {}...", version); + if (CdcVersion.SNAPSHOT.equals(version)) { + LOG.info("CDC {} is a snapshot version, we should fetch it locally...", version); + + container.copyFileToContainer( + MountableFile.forHostPath( + TestUtils.getResource("flink-cdc.sh", "flink-cdc-dist", "src"), 0755), + version.workDir() + "/bin/flink-cdc.sh"); + container.copyFileToContainer( + MountableFile.forHostPath( + TestUtils.getResource("flink-cdc.yaml", "flink-cdc-dist", "src"), 0755), + version.workDir() + "/conf/flink-cdc.yaml"); + container.copyFileToContainer( + MountableFile.forHostPath(TestUtils.getResource("flink-cdc-dist.jar")), + version.workDir() + "/lib/flink-cdc-dist.jar"); + container.copyFileToContainer( + MountableFile.forHostPath( + TestUtils.getResource("values-cdc-pipeline-connector.jar")), + version.workDir() + "/lib/values-cdc-pipeline-connector.jar"); + + } else { + LOG.info("CDC {} is a released version, download it from the Internet...", version); + + String containerPath = "/tmp/tarball/" + version.getVersion() + ".tar.gz"; + downloadAndCopyToContainer(container, version.tarballUrl(), containerPath); + container.execInContainer("mkdir", "-p", version.workDir()); + container.execInContainer( + "tar", "-xzvf", containerPath, "-C", version.workDir(), "--strip-components=1"); + + downloadAndCopyToContainer( + container, + version.connectorJarUrl("values"), + version.workDir() + "/lib/values-cdc-pipeline-connector.jar"); + } + } + + private static void downloadAndCopyToContainer( + GenericContainer container, String url, String containerPath) throws Exception { + Path tempFile = Files.createTempFile("download-", ".tmp"); + FileUtils.copyURLToFile( + new URL(url), + tempFile.toFile(), + (int) Duration.ofMinutes(1).toMillis(), + (int) Duration.ofMinutes(5).toMillis()); + container.copyFileToContainer(MountableFile.forHostPath(tempFile), containerPath); + } + + /** Enum for all released Flink CDC version tags. */ + public enum CdcVersion { + V3_1_1("3.1.1"), + V3_2_0("3.2.0"), + V3_2_1("3.2.1"), + V3_3_0("3.3.0"), + V3_4_0("3.4.0"), + V3_5_0("3.5.0"), + SNAPSHOT("SNAPSHOT"); + + private final String version; + + CdcVersion(String version) { + this.version = version; + } + + public String getVersion() { + return version; + } + + public static List getAllVersions() { + return Arrays.asList(CdcVersion.values()); + } + + public static List getVersionsSince(CdcVersion version) { + return getAllVersions() + .subList(getAllVersions().indexOf(version), getAllVersions().size()); + } + + public String tarballUrl() { + return "https://dlcdn.apache.org/flink/flink-cdc-" + + version + + "/flink-cdc-" + + version + + "-bin.tar.gz"; + } + + public String workDir() { + return "/tmp/cdc/" + version; + } + + public String connectorJarUrl(String name) { + return String.format( + "https://repo1.maven.org/maven2/org/apache/flink/flink-cdc-pipeline-connector-%s/%s/flink-cdc-pipeline-connector-%s-%s.jar", + name, version, name, version); + } + } +} diff --git a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests-2.x/src/test/resources/docker/mysql/my.cnf b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests-2.x/src/test/resources/docker/mysql/my.cnf new file mode 100644 index 00000000000..26d1abad6d3 --- /dev/null +++ b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests-2.x/src/test/resources/docker/mysql/my.cnf @@ -0,0 +1,64 @@ +# 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. + +# For advice on how to change settings please see +# http://dev.mysql.com/doc/refman/5.7/en/server-configuration-defaults.html + +[mysqld] +# +# Remove leading # and set to the amount of RAM for the most important data +# cache in MySQL. Start at 70% of total RAM for dedicated server, else 10%. +# innodb_buffer_pool_size = 128M +# +# Remove leading # to turn on a very important data integrity option: logging +# changes to the binary log between backups. +# log_bin +# +# Remove leading # to set options mainly useful for reporting servers. +# The server defaults are faster for transactions and fast SELECTs. +# Adjust sizes as needed, experiment to find the optimal values. +# join_buffer_size = 128M +# sort_buffer_size = 2M +# read_rnd_buffer_size = 2M +skip-host-cache +skip-name-resolve +#datadir=/var/lib/mysql +#socket=/var/lib/mysql/mysql.sock +#secure-file-priv=/var/lib/mysql-files +secure-file-priv=/var/lib/mysql +user=mysql + +# Disabling symbolic-links is recommended to prevent assorted security risks +symbolic-links=0 + +#log-error=/var/log/mysqld.log +#pid-file=/var/run/mysqld/mysqld.pid + +# ---------------------------------------------- +# Enable the binlog for replication & CDC +# ---------------------------------------------- + +# Enable binary replication log and set the prefix, expiration, and log format. +# The prefix is arbitrary, expiration can be short for integration tests but would +# be longer on a production system. Row-level info is required for ingest to work. +# Server ID is required, but this will vary on production systems +server-id = 223344 +log_bin = mysql-bin +expire_logs_days = 1 +binlog_format = row + +# enable gtid mode +gtid_mode = on +enforce_gtid_consistency = on \ No newline at end of file diff --git a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests-2.x/src/test/resources/docker/mysql/setup.sql b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests-2.x/src/test/resources/docker/mysql/setup.sql new file mode 100644 index 00000000000..3c0cccb4bca --- /dev/null +++ b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests-2.x/src/test/resources/docker/mysql/setup.sql @@ -0,0 +1,30 @@ +-- 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. + +-- In production you would almost certainly limit the replication user must be on the follower (slave) machine, +-- to prevent other clients accessing the log from other machines. For example, 'replicator'@'follower.acme.com'. +-- However, in this database we'll grant 2 users different privileges: +-- +-- 1) 'flinkuser' - all privileges required by the snapshot reader AND binlog reader (used for testing) +-- 2) 'mysqluser' - all privileges +-- +GRANT SELECT, RELOAD, SHOW DATABASES, REPLICATION SLAVE, REPLICATION CLIENT, LOCK TABLES ON *.* TO 'flinkuser'@'%'; +CREATE USER 'mysqluser' IDENTIFIED BY 'mysqlpw'; +GRANT ALL PRIVILEGES ON *.* TO 'mysqluser'@'%'; + +-- ---------------------------------------------------------------------------------------------------------------- +-- DATABASE: emptydb +-- ---------------------------------------------------------------------------------------------------------------- +CREATE DATABASE emptydb; diff --git a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests-2.x/src/test/resources/log4j2-test.properties b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests-2.x/src/test/resources/log4j2-test.properties new file mode 100644 index 00000000000..32df1c0251c --- /dev/null +++ b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests-2.x/src/test/resources/log4j2-test.properties @@ -0,0 +1,25 @@ +################################################################################ +# 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. +################################################################################ + +# Set root logger level to ERROR to not flood build logs +# set manually to INFO for debugging purposes +rootLogger.level=ERROR +rootLogger.appenderRef.test.ref = TestLogger + +appender.testlogger.name = TestLogger +appender.testlogger.type = CONSOLE +appender.testlogger.target = SYSTEM_ERR +appender.testlogger.layout.type = PatternLayout +appender.testlogger.layout.pattern = %-4r [%t] %-5p %c - %m%n diff --git a/flink-cdc-e2e-tests/pom.xml b/flink-cdc-e2e-tests/pom.xml index 28d16beee70..155c6b73563 100644 --- a/flink-cdc-e2e-tests/pom.xml +++ b/flink-cdc-e2e-tests/pom.xml @@ -33,6 +33,7 @@ limitations under the License. flink-cdc-e2e-utils flink-cdc-source-e2e-tests flink-cdc-pipeline-e2e-tests + flink-cdc-pipeline-e2e-tests-2.x diff --git a/flink-cdc-flink2.2-compat/pom.xml b/flink-cdc-flink2.2-compat/pom.xml new file mode 100644 index 00000000000..ffed8f2dc00 --- /dev/null +++ b/flink-cdc-flink2.2-compat/pom.xml @@ -0,0 +1,95 @@ + + + + + flink-cdc-parent + org.apache.flink + ${revision} + + 4.0.0 + + flink-cdc-flink2.2-compat + + + ${flink.2.x.version} + ${flink.2.x.shaded.guava.version} + + + + + org.apache.flink + flink-table-common + ${flink.version} + provided + + + org.apache.flink + flink-connector-base + ${flink.version} + provided + + + org.apache.flink + flink-table-api-java-bridge + ${flink.version} + provided + + + org.apache.flink + flink-core + ${flink.version} + provided + + + org.apache.flink + flink-streaming-java + ${flink.version} + provided + + + org.slf4j + slf4j-api + ${slf4j.version} + + + org.apache.flink + flink-shaded-guava + ${shaded.guava.version} + + + + + + + org.apache.maven.plugins + maven-jar-plugin + + + test-jar + + test-jar + + + + + + + + \ No newline at end of file diff --git a/flink-cdc-flink2.2-compat/src/main/java/org/apache/flink/api/connector/sink2/Sink.java b/flink-cdc-flink2.2-compat/src/main/java/org/apache/flink/api/connector/sink2/Sink.java new file mode 100644 index 00000000000..f4b5d6d010d --- /dev/null +++ b/flink-cdc-flink2.2-compat/src/main/java/org/apache/flink/api/connector/sink2/Sink.java @@ -0,0 +1,72 @@ +/* + * 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.api.connector.sink2; + +import org.apache.flink.annotation.Experimental; +import org.apache.flink.annotation.Public; +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.api.common.operators.MailboxExecutor; +import org.apache.flink.api.common.operators.ProcessingTimeService; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.metrics.groups.SinkWriterMetricGroup; +import org.apache.flink.util.UserCodeClassLoader; + +import java.io.IOException; +import java.io.Serializable; +import java.util.Optional; +import java.util.function.Consumer; + +/** Compatibility adapter to add class that was present in Flink 1.20 but removed in Flink 2.2. */ +@Public +public interface Sink extends Serializable { + + SinkWriter createWriter(WriterInitContext var1) throws IOException; + + /** + * @deprecated + */ + @Deprecated + SinkWriter createWriter(InitContext var1) throws IOException; + + /** + * @deprecated + */ + @Deprecated + @PublicEvolving + public interface InitContext extends org.apache.flink.api.connector.sink2.InitContext { + UserCodeClassLoader getUserCodeClassLoader(); + + MailboxExecutor getMailboxExecutor(); + + ProcessingTimeService getProcessingTimeService(); + + SinkWriterMetricGroup metricGroup(); + + SerializationSchema.InitializationContext asSerializationSchemaInitializationContext(); + + boolean isObjectReuseEnabled(); + + TypeSerializer createInputSerializer(); + + @Experimental + default Optional> metadataConsumer() { + return Optional.empty(); + } + } +} diff --git a/flink-cdc-flink2.2-compat/src/main/java/org/apache/flink/api/connector/sink2/TwoPhaseCommittingSink.java b/flink-cdc-flink2.2-compat/src/main/java/org/apache/flink/api/connector/sink2/TwoPhaseCommittingSink.java new file mode 100644 index 00000000000..86d0636a586 --- /dev/null +++ b/flink-cdc-flink2.2-compat/src/main/java/org/apache/flink/api/connector/sink2/TwoPhaseCommittingSink.java @@ -0,0 +1,22 @@ +/* + * 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.api.connector.sink2; + +/** Compatibility adapter to add class that was present in Flink 1.20 but removed in Flink 2.2. */ +public interface TwoPhaseCommittingSink + extends Sink, SupportsCommitter {} diff --git a/flink-cdc-flink2.2-compat/src/main/java/org/apache/flink/core/execution/RestoreMode.java b/flink-cdc-flink2.2-compat/src/main/java/org/apache/flink/core/execution/RestoreMode.java new file mode 100644 index 00000000000..80227ea9bb8 --- /dev/null +++ b/flink-cdc-flink2.2-compat/src/main/java/org/apache/flink/core/execution/RestoreMode.java @@ -0,0 +1,54 @@ +/* + * 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.core.execution; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.configuration.DescribedEnum; +import org.apache.flink.configuration.description.InlineElement; +import org.apache.flink.configuration.description.TextElement; + +/** + * Copy from .... + */ +@PublicEvolving +public enum RestoreMode implements DescribedEnum { + CLAIM( + "Flink will take ownership of the given snapshot. It will clean the snapshot once it is subsumed by newer ones."), + NO_CLAIM( + "Flink will not claim ownership of the snapshot files. However it will make sure it does not depend on any artefacts from the restored snapshot. In order to do that, Flink will take the first checkpoint as a full one, which means it might reupload/duplicate files that are part of the restored checkpoint."), + /** + * @deprecated + */ + @Deprecated + LEGACY( + "This is the mode in which Flink worked until 1.15. It will not claim ownership of the snapshot and will not delete the files. However, it can directly depend on the existence of the files of the restored checkpoint. It might not be safe to delete checkpoints that were restored in legacy mode. This mode is deprecated, please use CLAIM or NO_CLAIM mode to get a clear state file ownership."); + + private final String description; + public static final RestoreMode DEFAULT = NO_CLAIM; + + private RestoreMode(String description) { + this.description = description; + } + + @Internal + public InlineElement getDescription() { + return TextElement.text(this.description); + } +} diff --git a/flink-cdc-flink2.2-compat/src/main/java/org/apache/flink/runtime/jobgraph/SavepointConfigOptions.java b/flink-cdc-flink2.2-compat/src/main/java/org/apache/flink/runtime/jobgraph/SavepointConfigOptions.java new file mode 100644 index 00000000000..f281c0ee41b --- /dev/null +++ b/flink-cdc-flink2.2-compat/src/main/java/org/apache/flink/runtime/jobgraph/SavepointConfigOptions.java @@ -0,0 +1,56 @@ +/* + * 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.runtime.jobgraph; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.annotation.docs.Documentation; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; +import org.apache.flink.core.execution.RestoreMode; + +/** + * Copy from .... + */ +@Deprecated +@Documentation.ExcludeFromDocumentation("Hidden for deprecated.") +@PublicEvolving +public class SavepointConfigOptions { + public static final ConfigOption SAVEPOINT_PATH = + ConfigOptions.key("execution.savepoint.path") + .stringType() + .noDefaultValue() + .withDescription( + "Path to a savepoint to restore the job from (for example hdfs:///flink/savepoint-1537)."); + public static final ConfigOption SAVEPOINT_IGNORE_UNCLAIMED_STATE = + ConfigOptions.key("execution.savepoint.ignore-unclaimed-state") + .booleanType() + .defaultValue(false) + .withDescription( + "Allow to skip savepoint state that cannot be restored. Allow this if you removed an operator from your pipeline after the savepoint was triggered."); + public static final ConfigOption RESTORE_MODE; + + static { + RESTORE_MODE = + ConfigOptions.key("execution.savepoint-restore-mode") + .enumType(RestoreMode.class) + .defaultValue(RestoreMode.DEFAULT) + .withDescription( + "Describes the mode how Flink should restore from the given savepoint or retained checkpoint."); + } +} diff --git a/flink-cdc-flink2.2-compat/src/main/java/org/apache/flink/streaming/api/connector/sink2/WithPostCommitTopology.java b/flink-cdc-flink2.2-compat/src/main/java/org/apache/flink/streaming/api/connector/sink2/WithPostCommitTopology.java new file mode 100644 index 00000000000..1f0cb5c8667 --- /dev/null +++ b/flink-cdc-flink2.2-compat/src/main/java/org/apache/flink/streaming/api/connector/sink2/WithPostCommitTopology.java @@ -0,0 +1,21 @@ +/* + * 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.streaming.api.connector.sink2; + +/** Compatibility adapter to add class that was present in Flink 1.20 but removed in Flink 2.2. */ +public interface WithPostCommitTopology extends SupportsPostCommitTopology {} diff --git a/flink-cdc-flink2.2-compat/src/main/java/org/apache/flink/streaming/api/connector/sink2/WithPreCommitTopology.java b/flink-cdc-flink2.2-compat/src/main/java/org/apache/flink/streaming/api/connector/sink2/WithPreCommitTopology.java new file mode 100644 index 00000000000..eda582e47f3 --- /dev/null +++ b/flink-cdc-flink2.2-compat/src/main/java/org/apache/flink/streaming/api/connector/sink2/WithPreCommitTopology.java @@ -0,0 +1,22 @@ +/* + * 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.streaming.api.connector.sink2; + +/** Compatibility adapter to add class that was present in Flink 1.20 but removed in Flink 2.2. */ +public interface WithPreCommitTopology + extends SupportsPreCommitTopology {} diff --git a/flink-cdc-flink2.2-compat/src/main/java/org/apache/flink/streaming/api/connector/sink2/WithPreWriteTopology.java b/flink-cdc-flink2.2-compat/src/main/java/org/apache/flink/streaming/api/connector/sink2/WithPreWriteTopology.java new file mode 100644 index 00000000000..719c4745940 --- /dev/null +++ b/flink-cdc-flink2.2-compat/src/main/java/org/apache/flink/streaming/api/connector/sink2/WithPreWriteTopology.java @@ -0,0 +1,24 @@ +/* + * 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.streaming.api.connector.sink2; + +import org.apache.flink.api.connector.sink2.Sink; + +/** Compatibility adapter to add class that was present in Flink 1.20 but removed in Flink 2.2. */ +public interface WithPreWriteTopology + extends Sink, SupportsPreWriteTopology {} diff --git a/flink-cdc-flink2.2-compat/src/main/java/org/apache/flink/streaming/api/functions/sink/SinkFunction.java b/flink-cdc-flink2.2-compat/src/main/java/org/apache/flink/streaming/api/functions/sink/SinkFunction.java new file mode 100644 index 00000000000..e4f45737484 --- /dev/null +++ b/flink-cdc-flink2.2-compat/src/main/java/org/apache/flink/streaming/api/functions/sink/SinkFunction.java @@ -0,0 +1,22 @@ +/* + * 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.streaming.api.functions.sink; + +/** Compatibility adapter to add class that was present in Flink 1.20 but removed in Flink 2.2. */ +public interface SinkFunction + extends org.apache.flink.streaming.api.functions.sink.legacy.SinkFunction {} diff --git a/flink-cdc-flink2.2-compat/src/main/java/org/apache/flink/streaming/api/functions/source/RichParallelSourceFunction.java b/flink-cdc-flink2.2-compat/src/main/java/org/apache/flink/streaming/api/functions/source/RichParallelSourceFunction.java new file mode 100644 index 00000000000..127ab3cc9b4 --- /dev/null +++ b/flink-cdc-flink2.2-compat/src/main/java/org/apache/flink/streaming/api/functions/source/RichParallelSourceFunction.java @@ -0,0 +1,40 @@ +/* + * 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.streaming.api.functions.source; + +import org.apache.flink.annotation.Public; +import org.apache.flink.api.common.functions.AbstractRichFunction; +import org.apache.flink.api.common.functions.OpenContext; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.functions.source.legacy.ParallelSourceFunction; + +/** Compatibility adapter to add class that was present in Flink 1.20 but removed in Flink 2.2. */ +@Deprecated +@Public +public abstract class RichParallelSourceFunction extends AbstractRichFunction + implements ParallelSourceFunction, SourceFunction { + private static final long serialVersionUID = 1L; + + @Override + public void open(OpenContext openContext) throws Exception { + super.open(openContext); + open(new Configuration()); + } + + public void open(Configuration parameters) throws Exception {} +} diff --git a/flink-cdc-flink2.2-compat/src/main/java/org/apache/flink/streaming/api/functions/source/SourceFunction.java b/flink-cdc-flink2.2-compat/src/main/java/org/apache/flink/streaming/api/functions/source/SourceFunction.java new file mode 100644 index 00000000000..d61e8a99145 --- /dev/null +++ b/flink-cdc-flink2.2-compat/src/main/java/org/apache/flink/streaming/api/functions/source/SourceFunction.java @@ -0,0 +1,22 @@ +/* + * 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.streaming.api.functions.source; + +/** Compatibility adapter to add class that was present in Flink 1.20 but removed in Flink 2.2. */ +public interface SourceFunction + extends org.apache.flink.streaming.api.functions.source.legacy.SourceFunction {} diff --git a/flink-cdc-flink2.2-compat/src/main/java/org/apache/flink/table/api/ValidationException.java b/flink-cdc-flink2.2-compat/src/main/java/org/apache/flink/table/api/ValidationException.java new file mode 100644 index 00000000000..fc6a8e37ca0 --- /dev/null +++ b/flink-cdc-flink2.2-compat/src/main/java/org/apache/flink/table/api/ValidationException.java @@ -0,0 +1,35 @@ +/* + * 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.table.api; + +import org.apache.flink.annotation.PublicEvolving; + +import javax.annotation.Nullable; + +/** Compatibility adapter to add class that was present in Flink 1.20 but removed in Flink 2.2. */ +@PublicEvolving +public class ValidationException extends RuntimeException { + + public ValidationException(String message) { + super(message); + } + + public ValidationException(String message, @Nullable Throwable cause) { + super(message, cause); + } +} diff --git a/flink-cdc-runtime/pom.xml b/flink-cdc-runtime/pom.xml index bb1c78fbba5..2a2072f0b2c 100644 --- a/flink-cdc-runtime/pom.xml +++ b/flink-cdc-runtime/pom.xml @@ -103,4 +103,32 @@ limitations under the License. + + + flink2 + + ${flink.2.x.version} + + + + + org.apache.maven.plugins + maven-dependency-plugin + + + org.apache.maven.plugins + maven-surefire-plugin + + + + + + org.apache.flink + flink-cdc-flink2.2-compat + ${project.version} + provided + + + + \ No newline at end of file diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/distributed/SchemaOperator.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/distributed/SchemaOperator.java index 15c0571ba16..45f91afaf98 100755 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/distributed/SchemaOperator.java +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/distributed/SchemaOperator.java @@ -97,7 +97,7 @@ public SchemaOperator( @Override public void open() throws Exception { super.open(); - subTaskId = getRuntimeContext().getIndexOfThisSubtask(); + subTaskId = getRuntimeContext().getTaskInfo().getIndexOfThisSubtask(); upstreamSchemaTable = HashBasedTable.create(); evolvedSchemaMap = new HashMap<>(); tableIdRouter = new TableIdRouter(routingRules); diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/regular/BatchSchemaOperator.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/regular/BatchSchemaOperator.java index df990425afc..da089e6c5cc 100644 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/regular/BatchSchemaOperator.java +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/regular/BatchSchemaOperator.java @@ -30,6 +30,7 @@ import org.apache.flink.cdc.common.sink.MetadataApplier; import org.apache.flink.cdc.runtime.operators.schema.common.SchemaDerivator; import org.apache.flink.cdc.runtime.operators.schema.common.SchemaManager; +import org.apache.flink.cdc.runtime.utils.FlinkCompatibilityUtils; import org.apache.flink.streaming.api.graph.StreamConfig; import org.apache.flink.streaming.api.operators.AbstractStreamOperator; import org.apache.flink.streaming.api.operators.ChainingStrategy; @@ -70,7 +71,7 @@ public class BatchSchemaOperator extends AbstractStreamOperator public BatchSchemaOperator( List routingRules, MetadataApplier metadataApplier, String timezone) { - this.chainingStrategy = ChainingStrategy.ALWAYS; + FlinkCompatibilityUtils.setChainingStrategyIfAvailable(this, ChainingStrategy.ALWAYS); this.timezone = timezone; this.routingRules = routingRules; this.metadataApplier = metadataApplier; diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/regular/SchemaOperator.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/regular/SchemaOperator.java index 901ba168db0..f35d62ce631 100644 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/regular/SchemaOperator.java +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/regular/SchemaOperator.java @@ -34,6 +34,7 @@ import org.apache.flink.cdc.runtime.operators.schema.common.metrics.SchemaOperatorMetrics; import org.apache.flink.cdc.runtime.operators.schema.regular.event.SchemaChangeRequest; import org.apache.flink.cdc.runtime.operators.schema.regular.event.SchemaChangeResponse; +import org.apache.flink.cdc.runtime.utils.FlinkCompatibilityUtils; import org.apache.flink.runtime.jobgraph.tasks.TaskOperatorEventGateway; import org.apache.flink.runtime.operators.coordination.CoordinationRequest; import org.apache.flink.runtime.operators.coordination.CoordinationResponse; @@ -109,7 +110,7 @@ public SchemaOperator( Duration rpcTimeOut, SchemaChangeBehavior schemaChangeBehavior, String timezone) { - this.chainingStrategy = ChainingStrategy.ALWAYS; + FlinkCompatibilityUtils.setChainingStrategyIfAvailable(this, ChainingStrategy.ALWAYS); this.rpcTimeout = rpcTimeOut; this.schemaChangeBehavior = schemaChangeBehavior; this.timezone = timezone; @@ -131,7 +132,7 @@ public void open() throws Exception { this.schemaOperatorMetrics = new SchemaOperatorMetrics( getRuntimeContext().getMetricGroup(), schemaChangeBehavior); - this.subTaskId = getRuntimeContext().getIndexOfThisSubtask(); + this.subTaskId = getRuntimeContext().getTaskInfo().getIndexOfThisSubtask(); this.originalSchemaMap = new HashMap<>(); this.evolvedSchemaMap = new HashMap<>(); this.router = new TableIdRouter(routingRules); diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/sink/BatchDataSinkFunctionOperator.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/sink/BatchDataSinkFunctionOperator.java index 560909006d3..73a57f32689 100644 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/sink/BatchDataSinkFunctionOperator.java +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/sink/BatchDataSinkFunctionOperator.java @@ -20,6 +20,7 @@ import org.apache.flink.cdc.common.annotation.Internal; import org.apache.flink.cdc.common.event.Event; import org.apache.flink.cdc.runtime.operators.sink.exception.SinkWrapperException; +import org.apache.flink.cdc.runtime.utils.FlinkCompatibilityUtils; import org.apache.flink.streaming.api.functions.sink.SinkFunction; import org.apache.flink.streaming.api.operators.ChainingStrategy; import org.apache.flink.streaming.api.operators.StreamSink; @@ -37,7 +38,7 @@ public class BatchDataSinkFunctionOperator extends StreamSink { public BatchDataSinkFunctionOperator(SinkFunction userFunction) { super(userFunction); - this.chainingStrategy = ChainingStrategy.ALWAYS; + FlinkCompatibilityUtils.setChainingStrategyIfAvailable(this, ChainingStrategy.ALWAYS); } @Override diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/sink/BatchDataSinkWriterOperator.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/sink/BatchDataSinkWriterOperator.java index 52900fef740..5f4bd9acf35 100644 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/sink/BatchDataSinkWriterOperator.java +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/sink/BatchDataSinkWriterOperator.java @@ -30,7 +30,6 @@ import org.apache.flink.streaming.api.graph.StreamConfig; import org.apache.flink.streaming.api.operators.AbstractStreamOperator; import org.apache.flink.streaming.api.operators.BoundedOneInput; -import org.apache.flink.streaming.api.operators.ChainingStrategy; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.operators.Output; import org.apache.flink.streaming.api.watermark.Watermark; @@ -41,6 +40,7 @@ import java.lang.reflect.Constructor; import java.lang.reflect.Field; +import java.lang.reflect.Method; /** * An operator that processes records to be written into a {@link Sink} in batch mode. @@ -78,7 +78,6 @@ public BatchDataSinkWriterOperator( this.sink = sink; this.processingTimeService = processingTimeService; this.mailboxExecutor = mailboxExecutor; - this.chainingStrategy = ChainingStrategy.ALWAYS; } @Override @@ -88,40 +87,43 @@ public void setup( Output>> output) { super.setup(containingTask, config, output); flinkWriterOperator = createFlinkWriterOperator(); - this.>>getFlinkWriterOperator() - .setup(containingTask, config, output); + invokeFlinkWriterOperatorMethod( + "setup", + new Class[] {StreamTask.class, StreamConfig.class, Output.class}, + containingTask, + config, + output); } @Override public void open() throws Exception { - this.>>getFlinkWriterOperator().open(); + invokeFlinkWriterOperatorMethod("open", new Class[0]); copySinkWriter = getFieldValue("sinkWriter"); } @Override public void initializeState(StateInitializationContext context) throws Exception { - this.>>getFlinkWriterOperator() - .initializeState(context); + invokeFlinkWriterOperatorMethod( + "initializeState", new Class[] {StateInitializationContext.class}, context); } @Override public void snapshotState(StateSnapshotContext context) throws Exception { - this.>>getFlinkWriterOperator() - .snapshotState(context); + invokeFlinkWriterOperatorMethod( + "snapshotState", new Class[] {StateSnapshotContext.class}, context); } @Override public void processWatermark(Watermark mark) throws Exception { super.processWatermark(mark); - this.>>getFlinkWriterOperator() - .processWatermark(mark); + invokeFlinkWriterOperatorMethod("processWatermark", new Class[] {Watermark.class}, mark); } @Override public void processWatermarkStatus(WatermarkStatus watermarkStatus) throws Exception { super.processWatermarkStatus(watermarkStatus); - this.>>getFlinkWriterOperator() - .processWatermarkStatus(watermarkStatus); + invokeFlinkWriterOperatorMethod( + "processWatermarkStatus", new Class[] {WatermarkStatus.class}, watermarkStatus); } @Override @@ -138,19 +140,18 @@ public void processElement(StreamRecord element) throws Exception { @Override public void prepareSnapshotPreBarrier(long checkpointId) throws Exception { - this.>>getFlinkWriterOperator() - .prepareSnapshotPreBarrier(checkpointId); + invokeFlinkWriterOperatorMethod( + "prepareSnapshotPreBarrier", new Class[] {long.class}, checkpointId); } @Override public void close() throws Exception { - this.>>getFlinkWriterOperator() - .close(); + invokeFlinkWriterOperatorMethod("close", new Class[0]); } @Override public void endInput() throws Exception { - this.getFlinkWriterOperator().endInput(); + invokeFlinkWriterOperatorMethod("endInput", new Class[0]); } // ----------------------------- Helper functions ------------------------------- @@ -168,11 +169,68 @@ private Object createFlinkWriterOperator() { .getUserCodeClassLoader() .loadClass( "org.apache.flink.streaming.runtime.operators.sink.SinkWriterOperator"); - Constructor constructor = - flinkWriterClass.getDeclaredConstructor( - Sink.class, ProcessingTimeService.class, MailboxExecutor.class); - constructor.setAccessible(true); - return constructor.newInstance(sink, processingTimeService, mailboxExecutor); + + // Try to find a constructor whose first three parameters are compatible with + // (Sink, ProcessingTimeService, MailboxExecutor). This makes the code resilient + // against Flink 2.x adding extra parameters to the constructor. + Constructor target = null; + for (Constructor c : flinkWriterClass.getDeclaredConstructors()) { + Class[] p = c.getParameterTypes(); + if (p.length >= 3 + && Sink.class.isAssignableFrom(p[0]) + && ProcessingTimeService.class.isAssignableFrom(p[1]) + && MailboxExecutor.class.isAssignableFrom(p[2])) { + target = c; + break; + } + } + + if (target == null) { + // Fallback: use the first declared constructor and best-effort argument mapping + // below. This covers Flink 2.2 where the constructor signature may have changed. + Constructor[] all = flinkWriterClass.getDeclaredConstructors(); + if (all.length == 0) { + throw new RuntimeException( + "No constructors found on SinkWriterOperator in Flink runtime"); + } + target = all[0]; + } + + target.setAccessible(true); + Class[] paramTypes = target.getParameterTypes(); + Object[] args = new Object[paramTypes.length]; + for (int i = 0; i < paramTypes.length; i++) { + Class t = paramTypes[i]; + if (Sink.class.isAssignableFrom(t)) { + args[i] = sink; + } else if (ProcessingTimeService.class.isAssignableFrom(t)) { + args[i] = processingTimeService; + } else if (MailboxExecutor.class.isAssignableFrom(t)) { + args[i] = mailboxExecutor; + } else if (t.isPrimitive()) { + if (t == boolean.class) { + args[i] = false; + } else if (t == char.class) { + args[i] = '\0'; + } else if (t == byte.class) { + args[i] = (byte) 0; + } else if (t == short.class) { + args[i] = (short) 0; + } else if (t == int.class) { + args[i] = 0; + } else if (t == long.class) { + args[i] = 0L; + } else if (t == float.class) { + args[i] = 0.0f; + } else if (t == double.class) { + args[i] = 0.0d; + } + } else { + // Best effort: pass null for any extra, unknown parameters. + args[i] = null; + } + } + return target.newInstance(args); } catch (Exception e) { throw new RuntimeException("Failed to create SinkWriterOperator in Flink", e); } @@ -204,4 +262,36 @@ private T getFieldValue(String fieldName) throws IllegalAccessException { private T getFlinkWriterOperator() { return (T) flinkWriterOperator; } + + private void invokeFlinkWriterOperatorMethod( + String methodName, Class[] parameterTypes, Object... args) { + try { + Method m = findMethod(flinkWriterOperator.getClass(), methodName, parameterTypes); + if (m == null) { + // Method does not exist in this Flink version (for example, open() signature + // changed); ignore for compatibility. + return; + } + m.setAccessible(true); + m.invoke(flinkWriterOperator, args); + } catch (Exception e) { + throw new RuntimeException( + "Failed to invoke method " + + methodName + + " on wrapped flink writer operator " + + flinkWriterOperator.getClass().getName(), + e); + } + } + + private static Method findMethod(Class clazz, String methodName, Class[] parameterTypes) { + while (clazz != null) { + try { + return clazz.getDeclaredMethod(methodName, parameterTypes); + } catch (NoSuchMethodException e) { + clazz = clazz.getSuperclass(); + } + } + return null; + } } diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/sink/DataSinkFunctionOperator.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/sink/DataSinkFunctionOperator.java index df0e7802b27..505f131e1ff 100644 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/sink/DataSinkFunctionOperator.java +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/sink/DataSinkFunctionOperator.java @@ -26,6 +26,7 @@ import org.apache.flink.cdc.common.event.TableId; import org.apache.flink.cdc.common.schema.Schema; import org.apache.flink.cdc.runtime.operators.sink.exception.SinkWrapperException; +import org.apache.flink.cdc.runtime.utils.FlinkCompatibilityUtils; import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.state.StateInitializationContext; import org.apache.flink.streaming.api.functions.sink.SinkFunction; @@ -62,7 +63,7 @@ public DataSinkFunctionOperator(SinkFunction userFunction, OperatorID sch super(userFunction); this.schemaOperatorID = schemaOperatorID; processedTableIds = new HashSet<>(); - this.chainingStrategy = ChainingStrategy.ALWAYS; + FlinkCompatibilityUtils.setChainingStrategyIfAvailable(this, ChainingStrategy.ALWAYS); } @Override @@ -79,7 +80,8 @@ public void setup( @Override public void initializeState(StateInitializationContext context) throws Exception { - schemaEvolutionClient.registerSubtask(getRuntimeContext().getIndexOfThisSubtask()); + schemaEvolutionClient.registerSubtask( + getRuntimeContext().getTaskInfo().getIndexOfThisSubtask()); super.initializeState(context); } @@ -135,7 +137,8 @@ private void handleFlushEvent(FlushEvent event) throws Exception { }); } schemaEvolutionClient.notifyFlushSuccess( - getRuntimeContext().getIndexOfThisSubtask(), event.getSourceSubTaskId()); + getRuntimeContext().getTaskInfo().getIndexOfThisSubtask(), + event.getSourceSubTaskId()); } private void emitLatestSchema(TableId tableId) throws Exception { diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/sink/DataSinkWriterOperator.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/sink/DataSinkWriterOperator.java index 0432d1f54cf..79261bc43b7 100644 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/sink/DataSinkWriterOperator.java +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/sink/DataSinkWriterOperator.java @@ -36,7 +36,6 @@ import org.apache.flink.streaming.api.graph.StreamConfig; import org.apache.flink.streaming.api.operators.AbstractStreamOperator; import org.apache.flink.streaming.api.operators.BoundedOneInput; -import org.apache.flink.streaming.api.operators.ChainingStrategy; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.operators.Output; import org.apache.flink.streaming.api.watermark.Watermark; @@ -45,8 +44,12 @@ import org.apache.flink.streaming.runtime.tasks.StreamTask; import org.apache.flink.streaming.runtime.watermarkstatus.WatermarkStatus; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import java.lang.reflect.Constructor; import java.lang.reflect.Field; +import java.lang.reflect.Method; import java.util.HashSet; import java.util.Optional; import java.util.Set; @@ -65,6 +68,8 @@ public class DataSinkWriterOperator extends AbstractStreamOperator> implements OneInputStreamOperator>, BoundedOneInput { + private static final Logger LOG = LoggerFactory.getLogger(DataSinkWriterOperator.class); + private SchemaEvolutionClient schemaEvolutionClient; private final OperatorID schemaOperatorID; @@ -97,7 +102,6 @@ public DataSinkWriterOperator( this.mailboxExecutor = mailboxExecutor; this.schemaOperatorID = schemaOperatorID; this.processedTableIds = new HashSet<>(); - this.chainingStrategy = ChainingStrategy.ALWAYS; } @Override @@ -107,8 +111,14 @@ public void setup( Output>> output) { super.setup(containingTask, config, output); flinkWriterOperator = createFlinkWriterOperator(); - this.>>getFlinkWriterOperator() - .setup(containingTask, config, output); + // Call protected AbstractStreamOperator.setup(...) on the wrapped operator via reflection + // to avoid JVM verifier issues with protected access across packages. + invokeFlinkWriterOperatorMethod( + "setup", + new Class[] {StreamTask.class, StreamConfig.class, Output.class}, + containingTask, + config, + output); schemaEvolutionClient = new SchemaEvolutionClient( containingTask.getEnvironment().getOperatorCoordinatorEventGateway(), @@ -117,35 +127,109 @@ public void setup( @Override public void open() throws Exception { - this.>>getFlinkWriterOperator().open(); + invokeFlinkWriterOperatorMethod("open", new Class[0]); copySinkWriter = getFieldValue("sinkWriter"); + ensureEmitDownstream(); + } + + /** + * Flink 2.2 {@code SinkWriterOperator} sets {@code emitDownstream = sink instanceof + * SupportsCommitter}. Sinks like Paimon implement the older {@code TwoPhaseCommittingSink} + * rather than Flink 2.x {@code SupportsCommitter}, so committables are silently discarded. This + * method forces the flag to {@code true} and fills in the committable serializer so that the + * wrapped operator properly emits committables to the downstream committer. + * + *

This is only applied when the sink actually supports committing (implements either {@code + * TwoPhaseCommittingSink} or {@code SupportsCommitter}). For sinks that do not support + * committing, forcing {@code emitDownstream = true} would cause a {@link ClassCastException} + * when the wrapped operator tries to cast the writer to {@code CommittingSinkWriter}. + */ + private void ensureEmitDownstream() { + if (!sinkSupportsCommitting()) { + return; + } + try { + Field emitField = findField(flinkWriterOperator.getClass(), "emitDownstream"); + if (emitField == null) { + return; + } + emitField.setAccessible(true); + if (!emitField.getBoolean(flinkWriterOperator)) { + emitField.setBoolean(flinkWriterOperator, true); + + Field serField = findField(flinkWriterOperator.getClass(), "committableSerializer"); + if (serField != null) { + serField.setAccessible(true); + if (serField.get(flinkWriterOperator) == null) { + try { + Method getSerializer = + sink.getClass().getMethod("getCommittableSerializer"); + getSerializer.setAccessible(true); + serField.set(flinkWriterOperator, getSerializer.invoke(sink)); + } catch (NoSuchMethodException ignored) { + } + } + } + } + } catch (Exception e) { + LOG.warn("Could not force emitDownstream on wrapped SinkWriterOperator", e); + } + } + + private boolean sinkSupportsCommitting() { + try { + Class tpc = + Class.forName("org.apache.flink.api.connector.sink2.TwoPhaseCommittingSink"); + if (tpc.isInstance(sink)) { + return true; + } + } catch (ClassNotFoundException ignored) { + } + try { + Class sc = Class.forName("org.apache.flink.api.connector.sink2.SupportsCommitter"); + if (sc.isInstance(sink)) { + return true; + } + } catch (ClassNotFoundException ignored) { + } + return false; + } + + private static Field findField(Class clazz, String name) { + while (clazz != null) { + try { + return clazz.getDeclaredField(name); + } catch (NoSuchFieldException e) { + clazz = clazz.getSuperclass(); + } + } + return null; } @Override public void initializeState(StateInitializationContext context) throws Exception { - schemaEvolutionClient.registerSubtask(getRuntimeContext().getIndexOfThisSubtask()); - this.>>getFlinkWriterOperator() - .initializeState(context); + schemaEvolutionClient.registerSubtask(getSubtaskIndexCompat()); + invokeFlinkWriterOperatorMethod( + "initializeState", new Class[] {StateInitializationContext.class}, context); } @Override public void snapshotState(StateSnapshotContext context) throws Exception { - this.>>getFlinkWriterOperator() - .snapshotState(context); + invokeFlinkWriterOperatorMethod( + "snapshotState", new Class[] {StateSnapshotContext.class}, context); } @Override public void processWatermark(Watermark mark) throws Exception { super.processWatermark(mark); - this.>>getFlinkWriterOperator() - .processWatermark(mark); + invokeFlinkWriterOperatorMethod("processWatermark", new Class[] {Watermark.class}, mark); } @Override public void processWatermarkStatus(WatermarkStatus watermarkStatus) throws Exception { super.processWatermarkStatus(watermarkStatus); - this.>>getFlinkWriterOperator() - .processWatermarkStatus(watermarkStatus); + invokeFlinkWriterOperatorMethod( + "processWatermarkStatus", new Class[] {WatermarkStatus.class}, watermarkStatus); } @Override @@ -188,19 +272,18 @@ public void processElement(StreamRecord element) throws Exception { @Override public void prepareSnapshotPreBarrier(long checkpointId) throws Exception { - this.>>getFlinkWriterOperator() - .prepareSnapshotPreBarrier(checkpointId); + invokeFlinkWriterOperatorMethod( + "prepareSnapshotPreBarrier", new Class[] {long.class}, checkpointId); } @Override public void close() throws Exception { - this.>>getFlinkWriterOperator() - .close(); + invokeFlinkWriterOperatorMethod("close", new Class[0]); } @Override public void endInput() throws Exception { - this.getFlinkWriterOperator().endInput(); + invokeFlinkWriterOperatorMethod("endInput", new Class[0]); } // ----------------------------- Helper functions ------------------------------- @@ -223,7 +306,7 @@ private void handleFlushEvent(FlushEvent event) throws Exception { }); } schemaEvolutionClient.notifyFlushSuccess( - getRuntimeContext().getIndexOfThisSubtask(), event.getSourceSubTaskId()); + getSubtaskIndexCompat(), event.getSourceSubTaskId()); } private void emitLatestSchema(TableId tableId) throws Exception { @@ -250,16 +333,106 @@ private Object createFlinkWriterOperator() { .getUserCodeClassLoader() .loadClass( "org.apache.flink.streaming.runtime.operators.sink.SinkWriterOperator"); - Constructor constructor = - flinkWriterClass.getDeclaredConstructor( - Sink.class, ProcessingTimeService.class, MailboxExecutor.class); - constructor.setAccessible(true); - return constructor.newInstance(sink, processingTimeService, mailboxExecutor); + + // Try to find a constructor whose first three parameters are compatible with + // (Sink, ProcessingTimeService, MailboxExecutor). This makes the code resilient + // against Flink 2.x adding extra parameters to the constructor. + Constructor target = null; + for (Constructor c : flinkWriterClass.getDeclaredConstructors()) { + Class[] p = c.getParameterTypes(); + if (p.length >= 3 + && Sink.class.isAssignableFrom(p[0]) + && ProcessingTimeService.class.isAssignableFrom(p[1]) + && MailboxExecutor.class.isAssignableFrom(p[2])) { + target = c; + break; + } + } + + if (target == null) { + // Fallback: use the first declared constructor and best-effort argument mapping + // below. This covers Flink 2.2 where the constructor signature may have changed. + Constructor[] all = flinkWriterClass.getDeclaredConstructors(); + if (all.length == 0) { + throw new RuntimeException( + "No constructors found on SinkWriterOperator in Flink runtime"); + } + target = all[0]; + } + + target.setAccessible(true); + Class[] paramTypes = target.getParameterTypes(); + Object[] args = new Object[paramTypes.length]; + for (int i = 0; i < paramTypes.length; i++) { + Class t = paramTypes[i]; + if (Sink.class.isAssignableFrom(t)) { + args[i] = sink; + } else if (ProcessingTimeService.class.isAssignableFrom(t)) { + args[i] = processingTimeService; + } else if (MailboxExecutor.class.isAssignableFrom(t)) { + args[i] = mailboxExecutor; + } else if (t.isPrimitive()) { + if (t == boolean.class) { + args[i] = false; + } else if (t == char.class) { + args[i] = '\0'; + } else if (t == byte.class) { + args[i] = (byte) 0; + } else if (t == short.class) { + args[i] = (short) 0; + } else if (t == int.class) { + args[i] = 0; + } else if (t == long.class) { + args[i] = 0L; + } else if (t == float.class) { + args[i] = 0.0f; + } else if (t == double.class) { + args[i] = 0.0d; + } + } else { + // Best effort: pass null for any extra, unknown parameters. + args[i] = null; + } + } + return target.newInstance(args); } catch (Exception e) { throw new RuntimeException("Failed to create SinkWriterOperator in Flink", e); } } + /** + * Obtains the subtask index in a way that is compatible with both Flink 1.x and 2.x. + * + *

Flink 2.x removed {@code StreamingRuntimeContext.getIndexOfThisSubtask()}. The correct + * replacement is {@code getRuntimeContext().getTaskInfo().getIndexOfThisSubtask()}. + * + *

IMPORTANT: Returning a wrong (fixed) index here causes ALL sink subtasks to report + * the same ID to SchemaCoordinator. The coordinator then waits forever for the missing subtask + * IDs, causing a 3-minute timeout in SchemaOperator. + */ + private int getSubtaskIndexCompat() { + try { + Object ctx = getRuntimeContext(); + // Flink 2.x: getTaskInfo().getIndexOfThisSubtask() + try { + Method getTaskInfo = ctx.getClass().getMethod("getTaskInfo"); + getTaskInfo.setAccessible(true); + Object taskInfo = getTaskInfo.invoke(ctx); + Method getIndex = taskInfo.getClass().getMethod("getIndexOfThisSubtask"); + getIndex.setAccessible(true); + return (Integer) getIndex.invoke(taskInfo); + } catch (NoSuchMethodException ignored) { + // fall through to Flink 1.x path + } + // Flink 1.x: direct getIndexOfThisSubtask() + Method m = ctx.getClass().getMethod("getIndexOfThisSubtask"); + m.setAccessible(true); + return (Integer) m.invoke(ctx); + } catch (Exception e) { + throw new RuntimeException("Failed to obtain subtask index from RuntimeContext", e); + } + } + /** * Finds a field by name from its declaring class. This also searches for the field in super * classes. @@ -286,4 +459,36 @@ private T getFieldValue(String fieldName) throws IllegalAccessException { private T getFlinkWriterOperator() { return (T) flinkWriterOperator; } + + private void invokeFlinkWriterOperatorMethod( + String methodName, Class[] parameterTypes, Object... args) { + try { + Method m = findMethod(flinkWriterOperator.getClass(), methodName, parameterTypes); + if (m == null) { + // Method does not exist in this Flink version (for example, open() signature + // changed); ignore for compatibility. + return; + } + m.setAccessible(true); + m.invoke(flinkWriterOperator, args); + } catch (Exception e) { + throw new RuntimeException( + "Failed to invoke method " + + methodName + + " on wrapped flink writer operator " + + flinkWriterOperator.getClass().getName(), + e); + } + } + + private static Method findMethod(Class clazz, String methodName, Class[] parameterTypes) { + while (clazz != null) { + try { + return clazz.getDeclaredMethod(methodName, parameterTypes); + } catch (NoSuchMethodException e) { + clazz = clazz.getSuperclass(); + } + } + return null; + } } diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/sink/DataSinkWriterOperatorFactory.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/sink/DataSinkWriterOperatorFactory.java index 278518f976e..ab64f9c2ba0 100644 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/sink/DataSinkWriterOperatorFactory.java +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/sink/DataSinkWriterOperatorFactory.java @@ -17,23 +17,25 @@ package org.apache.flink.cdc.runtime.operators.sink; +import org.apache.flink.api.common.operators.MailboxExecutor; import org.apache.flink.api.connector.sink2.Sink; import org.apache.flink.cdc.common.annotation.Internal; import org.apache.flink.cdc.common.event.Event; import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.streaming.api.connector.sink2.CommittableMessage; import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory; +import org.apache.flink.streaming.api.operators.ChainingStrategy; import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; import org.apache.flink.streaming.api.operators.StreamOperator; import org.apache.flink.streaming.api.operators.StreamOperatorParameters; -import org.apache.flink.streaming.api.operators.YieldingOperatorFactory; + +import java.lang.reflect.Method; /** Operator factory for {@link DataSinkWriterOperator}. */ @Internal public class DataSinkWriterOperatorFactory extends AbstractStreamOperatorFactory> - implements OneInputStreamOperatorFactory>, - YieldingOperatorFactory> { + implements OneInputStreamOperatorFactory> { private final Sink sink; private final boolean isBounded; @@ -41,6 +43,7 @@ public class DataSinkWriterOperatorFactory public DataSinkWriterOperatorFactory( Sink sink, boolean isBounded, OperatorID schemaOperatorID) { + setChainingStrategy(ChainingStrategy.ALWAYS); this.sink = sink; this.isBounded = isBounded; this.schemaOperatorID = schemaOperatorID; @@ -50,11 +53,10 @@ public DataSinkWriterOperatorFactory( @Override public >> T createStreamOperator( StreamOperatorParameters> parameters) { - + MailboxExecutor mailboxExecutor = getMailboxExecutor(parameters); if (isBounded) { BatchDataSinkWriterOperator writerOperator = - new BatchDataSinkWriterOperator<>( - sink, processingTimeService, getMailboxExecutor()); + new BatchDataSinkWriterOperator<>(sink, processingTimeService, mailboxExecutor); writerOperator.setup( parameters.getContainingTask(), parameters.getStreamConfig(), @@ -63,7 +65,7 @@ public >> T createStreamOpera } DataSinkWriterOperator writerOperator = new DataSinkWriterOperator<>( - sink, processingTimeService, getMailboxExecutor(), schemaOperatorID); + sink, processingTimeService, mailboxExecutor, schemaOperatorID); writerOperator.setup( parameters.getContainingTask(), parameters.getStreamConfig(), @@ -71,6 +73,41 @@ public >> T createStreamOpera return (T) writerOperator; } + /** + * Obtains the {@link MailboxExecutor} in a way compatible with both Flink 1.19 and 1.20+. + * + *

Flink 1.20+ added {@code StreamOperatorParameters.getMailboxExecutor()}. In Flink 1.19 and + * earlier, the executor must be obtained via {@code + * StreamTask.getMailboxExecutorFactory().createExecutor(chainIndex)}. + */ + private MailboxExecutor getMailboxExecutor( + StreamOperatorParameters> parameters) { + // Try Flink 1.20+ / 2.x API: StreamOperatorParameters.getMailboxExecutor() + try { + Method m = parameters.getClass().getMethod("getMailboxExecutor"); + return (MailboxExecutor) m.invoke(parameters); + } catch (NoSuchMethodException ignored) { + // Fall through to Flink 1.19 path + } catch (Exception e) { + throw new RuntimeException( + "Failed to invoke getMailboxExecutor on StreamOperatorParameters", e); + } + + // Flink 1.19 and earlier: obtain from containingTask.getMailboxExecutorFactory() + try { + Object containingTask = parameters.getContainingTask(); + Method getFactory = containingTask.getClass().getMethod("getMailboxExecutorFactory"); + Object factory = getFactory.invoke(containingTask); + int chainIndex = parameters.getStreamConfig().getChainIndex(); + Method createExecutor = factory.getClass().getMethod("createExecutor", int.class); + return (MailboxExecutor) createExecutor.invoke(factory, chainIndex); + } catch (Exception e) { + throw new RuntimeException( + "Failed to obtain MailboxExecutor from StreamTask for Flink 1.19 compatibility", + e); + } + } + @Override public Class getStreamOperatorClass(ClassLoader classLoader) { if (isBounded) { diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/PreTransformOperator.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/PreTransformOperator.java index ff6cc512f44..fc350919653 100644 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/PreTransformOperator.java +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/PreTransformOperator.java @@ -35,6 +35,7 @@ import org.apache.flink.cdc.common.utils.SchemaUtils; import org.apache.flink.cdc.runtime.operators.transform.exceptions.TransformException; import org.apache.flink.cdc.runtime.parser.TransformParser; +import org.apache.flink.cdc.runtime.utils.FlinkCompatibilityUtils; import org.apache.flink.runtime.state.StateInitializationContext; import org.apache.flink.runtime.state.StateSnapshotContext; import org.apache.flink.streaming.api.graph.StreamConfig; @@ -85,7 +86,7 @@ public static PreTransformOperatorBuilder newBuilder() { this.preTransformChangeInfoMap = new HashMap<>(); this.preTransformProcessorMap = new HashMap<>(); this.schemaMetadataTransformers = new ArrayList<>(); - this.chainingStrategy = ChainingStrategy.ALWAYS; + FlinkCompatibilityUtils.setChainingStrategyIfAvailable(this, ChainingStrategy.ALWAYS); this.transformRules = transformRules; this.udfFunctions = udfFunctions; @@ -161,6 +162,12 @@ public void processElement(StreamRecord element) throws Exception { try { processEvent(event); } catch (Exception e) { + // If this exception originates from a chained downstream operator, rethrow as-is + // to avoid misleadingly wrapping it as a "pre-transform" error. + if (isChainedOperatorException(e)) { + throw e; + } + TableId tableId = null; Schema schemaBefore = null; Schema schemaAfter = null; @@ -398,4 +405,21 @@ private void clearOperator() { this.transforms = null; this.preTransformProcessorMap = null; } + + /** + * Checks if the given exception originates from a chained downstream operator. In Flink 2.x, + * all operators default to {@code ChainingStrategy.ALWAYS}, which means downstream operators + * like PostTransformOperator may be chained with PreTransformOperator. When a chained operator + * fails, the exception propagates back through the chain and should not be re-wrapped as a + * "pre-transform" error. + */ + private static boolean isChainedOperatorException(Exception e) { + try { + return Class.forName( + "org.apache.flink.streaming.runtime.tasks.ExceptionInChainedOperatorException") + .isInstance(e); + } catch (ClassNotFoundException ignored) { + return false; + } + } } diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/partitioning/BatchRegularPrePartitionOperator.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/partitioning/BatchRegularPrePartitionOperator.java index d6142868f17..e273bd694a8 100644 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/partitioning/BatchRegularPrePartitionOperator.java +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/partitioning/BatchRegularPrePartitionOperator.java @@ -27,6 +27,7 @@ import org.apache.flink.cdc.common.schema.Schema; import org.apache.flink.cdc.runtime.operators.schema.regular.SchemaOperator; import org.apache.flink.cdc.runtime.serializer.event.EventSerializer; +import org.apache.flink.cdc.runtime.utils.FlinkCompatibilityUtils; import org.apache.flink.runtime.state.StateSnapshotContext; import org.apache.flink.streaming.api.operators.AbstractStreamOperator; import org.apache.flink.streaming.api.operators.ChainingStrategy; @@ -58,7 +59,7 @@ public class BatchRegularPrePartitionOperator extends AbstractStreamOperator hashFunctionProvider) { - this.chainingStrategy = ChainingStrategy.ALWAYS; + FlinkCompatibilityUtils.setChainingStrategyIfAvailable(this, ChainingStrategy.ALWAYS); this.downstreamParallelism = downstreamParallelism; this.hashFunctionProvider = hashFunctionProvider; } diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/partitioning/DistributedPrePartitionOperator.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/partitioning/DistributedPrePartitionOperator.java index bedbe8e8e09..8b8fcf32225 100644 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/partitioning/DistributedPrePartitionOperator.java +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/partitioning/DistributedPrePartitionOperator.java @@ -28,6 +28,7 @@ import org.apache.flink.cdc.common.utils.SchemaUtils; import org.apache.flink.cdc.runtime.operators.schema.regular.SchemaOperator; import org.apache.flink.cdc.runtime.serializer.event.EventSerializer; +import org.apache.flink.cdc.runtime.utils.FlinkCompatibilityUtils; import org.apache.flink.runtime.state.StateSnapshotContext; import org.apache.flink.streaming.api.operators.AbstractStreamOperator; import org.apache.flink.streaming.api.operators.ChainingStrategy; @@ -55,7 +56,7 @@ public class DistributedPrePartitionOperator extends AbstractStreamOperator hashFunctionProvider) { - this.chainingStrategy = ChainingStrategy.ALWAYS; + FlinkCompatibilityUtils.setChainingStrategyIfAvailable(this, ChainingStrategy.ALWAYS); this.downstreamParallelism = downstreamParallelism; this.hashFunctionProvider = hashFunctionProvider; } @@ -63,7 +64,7 @@ public DistributedPrePartitionOperator( @Override public void open() throws Exception { super.open(); - subTaskId = getRuntimeContext().getIndexOfThisSubtask(); + subTaskId = getRuntimeContext().getTaskInfo().getIndexOfThisSubtask(); schemaMap = new HashMap<>(); hashFunctionMap = new HashMap<>(); } diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/partitioning/RegularPrePartitionOperator.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/partitioning/RegularPrePartitionOperator.java index 38f177472fb..d65e3d7552a 100644 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/partitioning/RegularPrePartitionOperator.java +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/partitioning/RegularPrePartitionOperator.java @@ -29,6 +29,7 @@ import org.apache.flink.cdc.runtime.operators.schema.regular.SchemaOperator; import org.apache.flink.cdc.runtime.operators.sink.SchemaEvolutionClient; import org.apache.flink.cdc.runtime.serializer.event.EventSerializer; +import org.apache.flink.cdc.runtime.utils.FlinkCompatibilityUtils; import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.jobgraph.tasks.TaskOperatorEventGateway; import org.apache.flink.runtime.state.StateSnapshotContext; @@ -67,7 +68,7 @@ public RegularPrePartitionOperator( OperatorID schemaOperatorId, int downstreamParallelism, HashFunctionProvider hashFunctionProvider) { - this.chainingStrategy = ChainingStrategy.ALWAYS; + FlinkCompatibilityUtils.setChainingStrategyIfAvailable(this, ChainingStrategy.ALWAYS); this.schemaOperatorId = schemaOperatorId; this.downstreamParallelism = downstreamParallelism; this.hashFunctionProvider = hashFunctionProvider; diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/EnumSerializer.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/EnumSerializer.java index 5b93c320d4f..c7828108960 100644 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/EnumSerializer.java +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/EnumSerializer.java @@ -238,7 +238,8 @@ public TypeSerializer restoreSerializer() { return new EnumSerializer<>(enumClass, previousEnums); } - @Override + // Flink 1.x abstract method - removed in Flink 2.x, @Override omitted for cross-version + // compatibility public TypeSerializerSchemaCompatibility resolveSchemaCompatibility( TypeSerializer newSerializer) { if (!(newSerializer instanceof EnumSerializer)) { @@ -269,6 +270,32 @@ public TypeSerializerSchemaCompatibility resolveSchemaCompatibility( return TypeSerializerSchemaCompatibility.compatibleWithReconfiguredSerializer( reconfiguredSerializer); } + + // Flink 2.x new abstract method - no @Override so this also compiles against Flink 1.x + public TypeSerializerSchemaCompatibility resolveSchemaCompatibility( + TypeSerializerSnapshot oldSerializerSnapshot) { + if (!(oldSerializerSnapshot instanceof EnumSerializerSnapshot)) { + return TypeSerializerSchemaCompatibility.incompatible(); + } + @SuppressWarnings("unchecked") + EnumSerializerSnapshot oldSnapshot = + (EnumSerializerSnapshot) oldSerializerSnapshot; + if (!enumClass.equals(oldSnapshot.enumClass)) { + return TypeSerializerSchemaCompatibility.incompatible(); + } + if (Arrays.equals(oldSnapshot.previousEnums, previousEnums)) { + return TypeSerializerSchemaCompatibility.compatibleAsIs(); + } + Set reconfiguredEnumSet = + new LinkedHashSet<>(Arrays.asList(oldSnapshot.previousEnums)); + reconfiguredEnumSet.addAll(Arrays.asList(previousEnums)); + @SuppressWarnings("unchecked") + T[] reconfiguredEnums = + reconfiguredEnumSet.toArray( + (T[]) Array.newInstance(enumClass, reconfiguredEnumSet.size())); + return TypeSerializerSchemaCompatibility.compatibleWithReconfiguredSerializer( + new EnumSerializer<>(enumClass, reconfiguredEnums)); + } } // -------------------------------------------------------------------------------------------- diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/NestedSerializersSnapshotDelegate.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/NestedSerializersSnapshotDelegate.java index 66c62630dac..8eedaeef14e 100644 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/NestedSerializersSnapshotDelegate.java +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/NestedSerializersSnapshotDelegate.java @@ -184,7 +184,14 @@ public static NestedSerializersSnapshotDelegate readNestedSerializerSnapshots( // Utilities // ------------------------------------------------------------------------ - /** Utility method to conjure up a new scope for the generic parameters. */ + /** + * Utility method to conjure up a new scope for the generic parameters. + * + *

In Flink 1.x, TypeSerializerSnapshot.resolveSchemaCompatibility(TypeSerializer) is the + * abstract method. In Flink 2.x, it was replaced by + * resolveSchemaCompatibility(TypeSerializerSnapshot). This method uses reflection to call the + * right variant at runtime. + */ @SuppressWarnings("unchecked") private static TypeSerializerSchemaCompatibility resolveCompatibility( TypeSerializer serializer, TypeSerializerSnapshot snapshot) { @@ -192,7 +199,32 @@ private static TypeSerializerSchemaCompatibility resolveCompatibility( TypeSerializer typedSerializer = (TypeSerializer) serializer; TypeSerializerSnapshot typedSnapshot = (TypeSerializerSnapshot) snapshot; - return typedSnapshot.resolveSchemaCompatibility(typedSerializer); + // Try Flink 1.x approach: oldSnapshot.resolveSchemaCompatibility(newSerializer) + try { + java.lang.reflect.Method m = + typedSnapshot + .getClass() + .getMethod("resolveSchemaCompatibility", TypeSerializer.class); + return (TypeSerializerSchemaCompatibility) m.invoke(typedSnapshot, typedSerializer); + } catch (NoSuchMethodException ignored) { + // fall through to Flink 2.x approach + } catch (Exception e) { + throw new RuntimeException( + "Failed to resolve schema compatibility (Flink 1.x path)", e); + } + + // Flink 2.x approach: newSnapshot.resolveSchemaCompatibility(oldSnapshot) + try { + TypeSerializerSnapshot newSnapshot = typedSerializer.snapshotConfiguration(); + java.lang.reflect.Method m = + newSnapshot + .getClass() + .getMethod("resolveSchemaCompatibility", TypeSerializerSnapshot.class); + return (TypeSerializerSchemaCompatibility) m.invoke(newSnapshot, typedSnapshot); + } catch (Exception e) { + throw new RuntimeException( + "Failed to resolve schema compatibility (Flink 2.x path)", e); + } } private static TypeSerializer[] snapshotsToRestoreSerializers( diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/NullableSerializerWrapper.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/NullableSerializerWrapper.java index c230c8c9a8c..33eb92742fd 100644 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/NullableSerializerWrapper.java +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/NullableSerializerWrapper.java @@ -172,7 +172,8 @@ public TypeSerializer restoreSerializer() { return new NullableSerializerWrapper<>(innerSerializer); } - @Override + // Flink 1.x abstract method - removed in Flink 2.x, @Override omitted for cross-version + // compatibility public TypeSerializerSchemaCompatibility resolveSchemaCompatibility( TypeSerializer newSerializer) { if (!(newSerializer instanceof NullableSerializerWrapper)) { @@ -187,5 +188,20 @@ public TypeSerializerSchemaCompatibility resolveSchemaCompatibility( return TypeSerializerSchemaCompatibility.compatibleAsIs(); } } + + // Flink 2.x new abstract method - no @Override so this also compiles against Flink 1.x + @SuppressWarnings("unchecked") + public TypeSerializerSchemaCompatibility resolveSchemaCompatibility( + TypeSerializerSnapshot oldSerializerSnapshot) { + if (!(oldSerializerSnapshot instanceof NullableSerializerWrapperSnapshot)) { + return TypeSerializerSchemaCompatibility.incompatible(); + } + NullableSerializerWrapperSnapshot snapshot = + (NullableSerializerWrapperSnapshot) oldSerializerSnapshot; + if (!innerSerializer.equals(snapshot.innerSerializer)) { + return TypeSerializerSchemaCompatibility.incompatible(); + } + return TypeSerializerSchemaCompatibility.compatibleAsIs(); + } } } diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/data/ArrayDataSerializer.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/data/ArrayDataSerializer.java index 39939375d50..ecbc03d7e9d 100644 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/data/ArrayDataSerializer.java +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/data/ArrayDataSerializer.java @@ -265,7 +265,8 @@ public TypeSerializer restoreSerializer() { return new ArrayDataSerializer(previousType, previousEleSer); } - @Override + // Flink 1.x abstract method - removed in Flink 2.x, @Override omitted for cross-version + // compatibility public TypeSerializerSchemaCompatibility resolveSchemaCompatibility( TypeSerializer newSerializer) { if (!(newSerializer instanceof ArrayDataSerializer)) { @@ -280,5 +281,20 @@ public TypeSerializerSchemaCompatibility resolveSchemaCompatibility( return TypeSerializerSchemaCompatibility.compatibleAsIs(); } } + + // Flink 2.x new abstract method - no @Override so this also compiles against Flink 1.x + public TypeSerializerSchemaCompatibility resolveSchemaCompatibility( + TypeSerializerSnapshot oldSerializerSnapshot) { + if (!(oldSerializerSnapshot instanceof ArrayDataSerializerSnapshot)) { + return TypeSerializerSchemaCompatibility.incompatible(); + } + ArrayDataSerializerSnapshot snapshot = + (ArrayDataSerializerSnapshot) oldSerializerSnapshot; + if (!previousType.equals(snapshot.previousType) + || !previousEleSer.equals(snapshot.previousEleSer)) { + return TypeSerializerSchemaCompatibility.incompatible(); + } + return TypeSerializerSchemaCompatibility.compatibleAsIs(); + } } } diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/data/DecimalDataSerializer.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/data/DecimalDataSerializer.java index 4872330f37e..97d2e356147 100644 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/data/DecimalDataSerializer.java +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/data/DecimalDataSerializer.java @@ -178,7 +178,8 @@ public TypeSerializer restoreSerializer() { return new DecimalDataSerializer(previousPrecision, previousScale); } - @Override + // Flink 1.x abstract method - removed in Flink 2.x, @Override omitted for cross-version + // compatibility public TypeSerializerSchemaCompatibility resolveSchemaCompatibility( TypeSerializer newSerializer) { if (!(newSerializer instanceof DecimalDataSerializer)) { @@ -193,5 +194,21 @@ public TypeSerializerSchemaCompatibility resolveSchemaCompatibility return TypeSerializerSchemaCompatibility.compatibleAsIs(); } } + + // Flink 2.x changed the abstract method to + // resolveSchemaCompatibility(TypeSerializerSnapshot). + // No @Override so this also compiles against Flink 1.x where the method does not exist. + public TypeSerializerSchemaCompatibility resolveSchemaCompatibility( + TypeSerializerSnapshot oldSerializerSnapshot) { + if (!(oldSerializerSnapshot instanceof DecimalSerializerSnapshot)) { + return TypeSerializerSchemaCompatibility.incompatible(); + } + DecimalSerializerSnapshot snapshot = (DecimalSerializerSnapshot) oldSerializerSnapshot; + if (previousPrecision != snapshot.previousPrecision + || previousScale != snapshot.previousScale) { + return TypeSerializerSchemaCompatibility.incompatible(); + } + return TypeSerializerSchemaCompatibility.compatibleAsIs(); + } } } diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/data/LocalZonedTimestampDataSerializer.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/data/LocalZonedTimestampDataSerializer.java index c3622db2340..987f2082353 100644 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/data/LocalZonedTimestampDataSerializer.java +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/data/LocalZonedTimestampDataSerializer.java @@ -177,7 +177,8 @@ public TypeSerializer restoreSerializer() { return new LocalZonedTimestampDataSerializer(previousPrecision); } - @Override + // Flink 1.x abstract method - removed in Flink 2.x, @Override omitted for cross-version + // compatibility public TypeSerializerSchemaCompatibility resolveSchemaCompatibility(TypeSerializer newSerializer) { if (!(newSerializer instanceof LocalZonedTimestampDataSerializer)) { @@ -192,5 +193,22 @@ public TypeSerializer restoreSerializer() { return TypeSerializerSchemaCompatibility.compatibleAsIs(); } } + + // Flink 2.x changed the abstract method to + // resolveSchemaCompatibility(TypeSerializerSnapshot). + // No @Override so this also compiles against Flink 1.x where the method does not exist. + public TypeSerializerSchemaCompatibility + resolveSchemaCompatibility( + TypeSerializerSnapshot oldSerializerSnapshot) { + if (!(oldSerializerSnapshot instanceof LocalZonedTimestampDataSerializerSnapshot)) { + return TypeSerializerSchemaCompatibility.incompatible(); + } + LocalZonedTimestampDataSerializerSnapshot snapshot = + (LocalZonedTimestampDataSerializerSnapshot) oldSerializerSnapshot; + if (previousPrecision != snapshot.previousPrecision) { + return TypeSerializerSchemaCompatibility.incompatible(); + } + return TypeSerializerSchemaCompatibility.compatibleAsIs(); + } } } diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/data/MapDataSerializer.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/data/MapDataSerializer.java index d8f92ee29f0..8b2e19cdef7 100644 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/data/MapDataSerializer.java +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/data/MapDataSerializer.java @@ -306,7 +306,8 @@ public TypeSerializer restoreSerializer() { return new MapDataSerializer(keyType, valueType, keySerializer, valueSerializer); } - @Override + // Flink 1.x abstract method - removed in Flink 2.x, @Override omitted for cross-version + // compatibility public TypeSerializerSchemaCompatibility resolveSchemaCompatibility( TypeSerializer newSerializer) { if (!(newSerializer instanceof MapDataSerializer)) { @@ -323,5 +324,21 @@ public TypeSerializerSchemaCompatibility resolveSchemaCompatibility( return TypeSerializerSchemaCompatibility.compatibleAsIs(); } } + + // Flink 2.x new abstract method - no @Override so this also compiles against Flink 1.x + public TypeSerializerSchemaCompatibility resolveSchemaCompatibility( + TypeSerializerSnapshot oldSerializerSnapshot) { + if (!(oldSerializerSnapshot instanceof MapDataSerializerSnapshot)) { + return TypeSerializerSchemaCompatibility.incompatible(); + } + MapDataSerializerSnapshot snapshot = (MapDataSerializerSnapshot) oldSerializerSnapshot; + if (!keyType.equals(snapshot.keyType) + || !valueType.equals(snapshot.valueType) + || !keySerializer.equals(snapshot.keySerializer) + || !valueSerializer.equals(snapshot.valueSerializer)) { + return TypeSerializerSchemaCompatibility.incompatible(); + } + return TypeSerializerSchemaCompatibility.compatibleAsIs(); + } } } diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/data/TimestampDataSerializer.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/data/TimestampDataSerializer.java index babc6345bc5..10dfc060631 100644 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/data/TimestampDataSerializer.java +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/data/TimestampDataSerializer.java @@ -171,7 +171,8 @@ public TypeSerializer restoreSerializer() { return new TimestampDataSerializer(previousPrecision); } - @Override + // Flink 1.x abstract method - removed in Flink 2.x, @Override omitted for cross-version + // compatibility public TypeSerializerSchemaCompatibility resolveSchemaCompatibility( TypeSerializer newSerializer) { if (!(newSerializer instanceof TimestampDataSerializer)) { @@ -186,5 +187,21 @@ public TypeSerializerSchemaCompatibility resolveSchemaCompatibili return TypeSerializerSchemaCompatibility.compatibleAsIs(); } } + + // Flink 2.x changed the abstract method to + // resolveSchemaCompatibility(TypeSerializerSnapshot). + // No @Override so this also compiles against Flink 1.x where the method does not exist. + public TypeSerializerSchemaCompatibility resolveSchemaCompatibility( + TypeSerializerSnapshot oldSerializerSnapshot) { + if (!(oldSerializerSnapshot instanceof TimestampDataSerializerSnapshot)) { + return TypeSerializerSchemaCompatibility.incompatible(); + } + TimestampDataSerializerSnapshot snapshot = + (TimestampDataSerializerSnapshot) oldSerializerSnapshot; + if (previousPrecision != snapshot.previousPrecision) { + return TypeSerializerSchemaCompatibility.incompatible(); + } + return TypeSerializerSchemaCompatibility.compatibleAsIs(); + } } } diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/data/ZonedTimestampDataSerializer.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/data/ZonedTimestampDataSerializer.java index ca9861761cf..8676d903f75 100644 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/data/ZonedTimestampDataSerializer.java +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/data/ZonedTimestampDataSerializer.java @@ -181,7 +181,8 @@ public TypeSerializer restoreSerializer() { return new ZonedTimestampDataSerializer(previousPrecision); } - @Override + // Flink 1.x abstract method - removed in Flink 2.x, @Override omitted for cross-version + // compatibility public TypeSerializerSchemaCompatibility resolveSchemaCompatibility( TypeSerializer newSerializer) { if (!(newSerializer instanceof ZonedTimestampDataSerializer)) { @@ -196,5 +197,19 @@ public TypeSerializerSchemaCompatibility resolveSchemaCompat return TypeSerializerSchemaCompatibility.compatibleAsIs(); } } + + // Flink 2.x new abstract method - no @Override so this also compiles against Flink 1.x + public TypeSerializerSchemaCompatibility resolveSchemaCompatibility( + TypeSerializerSnapshot oldSerializerSnapshot) { + if (!(oldSerializerSnapshot instanceof ZonedTimestampDataSerializerSnapshot)) { + return TypeSerializerSchemaCompatibility.incompatible(); + } + ZonedTimestampDataSerializerSnapshot snapshot = + (ZonedTimestampDataSerializerSnapshot) oldSerializerSnapshot; + if (previousPrecision != snapshot.previousPrecision) { + return TypeSerializerSchemaCompatibility.incompatible(); + } + return TypeSerializerSchemaCompatibility.compatibleAsIs(); + } } } diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/typeutils/EventTypeInfo.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/typeutils/EventTypeInfo.java index 8087746cdec..c126abca628 100644 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/typeutils/EventTypeInfo.java +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/typeutils/EventTypeInfo.java @@ -18,6 +18,7 @@ package org.apache.flink.cdc.runtime.typeutils; import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.serialization.SerializerConfig; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.cdc.common.event.Event; @@ -58,11 +59,18 @@ public boolean isKeyType() { return false; } - @Override + // Flink 1.x abstract method - not abstract in Flink 2.x so @Override omitted for cross-version + // compatibility public TypeSerializer createSerializer(ExecutionConfig config) { return EventSerializer.INSTANCE; } + /** For Flink 2.2+ (TypeInformation.createSerializer now takes SerializerConfig). */ + @Override + public TypeSerializer createSerializer(SerializerConfig config) { + return EventSerializer.INSTANCE; + } + @Override public String toString() { return "Event"; diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/typeutils/PartitioningEventTypeInfo.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/typeutils/PartitioningEventTypeInfo.java index 4a3018fb98a..1de112319e4 100644 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/typeutils/PartitioningEventTypeInfo.java +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/typeutils/PartitioningEventTypeInfo.java @@ -18,6 +18,7 @@ package org.apache.flink.cdc.runtime.typeutils; import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.serialization.SerializerConfig; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.cdc.common.annotation.Internal; @@ -58,11 +59,18 @@ public boolean isKeyType() { return false; } - @Override + // Flink 1.x abstract method - not abstract in Flink 2.x so @Override omitted for cross-version + // compatibility public TypeSerializer createSerializer(ExecutionConfig config) { return PartitioningEventSerializer.INSTANCE; } + /** For Flink 2.2+ (TypeInformation.createSerializer now takes SerializerConfig). */ + @Override + public TypeSerializer createSerializer(SerializerConfig config) { + return PartitioningEventSerializer.INSTANCE; + } + @Override public String toString() { return "PartitioningEvent"; diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/utils/FlinkCompatibilityUtils.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/utils/FlinkCompatibilityUtils.java new file mode 100644 index 00000000000..974ad8641ab --- /dev/null +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/utils/FlinkCompatibilityUtils.java @@ -0,0 +1,153 @@ +/* + * 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.runtime.utils; + +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.cdc.common.annotation.Internal; +import org.apache.flink.streaming.api.operators.AbstractStreamOperator; +import org.apache.flink.streaming.api.operators.ChainingStrategy; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.lang.reflect.Field; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; + +/** Utility class for handling Flink version compatibility issues. */ +@Internal +public class FlinkCompatibilityUtils { + + private static final Logger LOG = LoggerFactory.getLogger(FlinkCompatibilityUtils.class); + + private FlinkCompatibilityUtils() { + // Private constructor to prevent instantiation + } + + /** + * Sets the chaining strategy for an operator via reflection. + * + *

This method is needed because Flink 2.x removed the {@code chainingStrategy} field from + * {@link AbstractStreamOperator}. This method attempts to set the field via reflection for + * backward compatibility with Flink 1.x, and silently ignores the failure in Flink 2.x. + * + * @param operator the operator to set the chaining strategy for + * @param strategy the chaining strategy to set + */ + public static void setChainingStrategyIfAvailable( + AbstractStreamOperator operator, ChainingStrategy strategy) { + try { + Field field = AbstractStreamOperator.class.getDeclaredField("chainingStrategy"); + field.setAccessible(true); + field.set(operator, strategy); + LOG.debug("Successfully set chainingStrategy to {} via reflection", strategy); + } catch (NoSuchFieldException e) { + LOG.debug( + "chainingStrategy field not available (likely Flink 2.x), skipping for operator {}", + operator.getClass().getSimpleName()); + } catch (Exception e) { + LOG.warn( + "Failed to set chainingStrategy via reflection for operator {}", + operator.getClass().getSimpleName(), + e); + } + } + + /** + * Resolves schema compatibility between a serializer snapshot and a new serializer, handling + * API differences between Flink 1.x and 2.x. + * + *

In Flink 1.x, the method is {@code snapshot.resolveSchemaCompatibility(serializer)}. In + * Flink 2.x, the method was changed to {@code + * newSnapshot.resolveSchemaCompatibility(oldSnapshot)}. + */ + @SuppressWarnings("unchecked") + public static TypeSerializerSchemaCompatibility resolveSchemaCompatibility( + TypeSerializerSnapshot snapshot, TypeSerializer serializer) { + // Try Flink 1.x approach: snapshot.resolveSchemaCompatibility(serializer) + try { + Method m = + snapshot.getClass() + .getMethod("resolveSchemaCompatibility", TypeSerializer.class); + return (TypeSerializerSchemaCompatibility) m.invoke(snapshot, serializer); + } catch (NoSuchMethodException ignored) { + // fall through to Flink 2.x approach + } catch (InvocationTargetException e) { + Throwable cause = e.getCause(); + if (cause instanceof RuntimeException) { + throw (RuntimeException) cause; + } + throw new RuntimeException( + "Failed to resolve schema compatibility (Flink 1.x path)", cause); + } catch (Exception e) { + throw new RuntimeException( + "Failed to resolve schema compatibility (Flink 1.x path)", e); + } + + // Flink 2.x approach: newSnapshot.resolveSchemaCompatibility(oldSnapshot) + try { + TypeSerializerSnapshot newSnapshot = serializer.snapshotConfiguration(); + Method m = + newSnapshot + .getClass() + .getMethod("resolveSchemaCompatibility", TypeSerializerSnapshot.class); + return (TypeSerializerSchemaCompatibility) m.invoke(newSnapshot, snapshot); + } catch (InvocationTargetException e) { + Throwable cause = e.getCause(); + if (cause instanceof RuntimeException) { + throw (RuntimeException) cause; + } + throw new RuntimeException( + "Failed to resolve schema compatibility (Flink 2.x path)", cause); + } catch (Exception e) { + throw new RuntimeException( + "Failed to resolve schema compatibility (Flink 2.x path)", e); + } + } + + /** + * Calls {@link AbstractStreamOperator#setup} via reflection to handle access level changes + * between Flink 1.x (public) and Flink 2.x (protected). + */ + public static void setupOperator( + AbstractStreamOperator operator, Object containingTask, Object config, Object output) + throws Exception { + Method setupMethod = null; + for (Method m : AbstractStreamOperator.class.getDeclaredMethods()) { + if ("setup".equals(m.getName()) && m.getParameterCount() == 3) { + setupMethod = m; + break; + } + } + if (setupMethod == null) { + throw new NoSuchMethodException("Cannot find setup method in AbstractStreamOperator"); + } + setupMethod.setAccessible(true); + try { + setupMethod.invoke(operator, containingTask, config, output); + } catch (InvocationTargetException e) { + Throwable cause = e.getCause(); + if (cause instanceof Exception) { + throw (Exception) cause; + } + throw new RuntimeException(cause); + } + } +} diff --git a/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/operators/sink/DataSinkOperatorAdapter.java b/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/operators/sink/DataSinkOperatorAdapter.java index 8fd60f632f2..0a9e60b7f9f 100644 --- a/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/operators/sink/DataSinkOperatorAdapter.java +++ b/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/operators/sink/DataSinkOperatorAdapter.java @@ -59,7 +59,15 @@ public class DataSinkOperatorAdapter extends AbstractStreamOperator public DataSinkOperatorAdapter() { this.schemaOperatorID = new OperatorID(); this.processedTableIds = new HashSet<>(); - this.chainingStrategy = ChainingStrategy.ALWAYS; + // Try to set chainingStrategy via reflection for backward compatibility with Flink 1.x + try { + java.lang.reflect.Field field = + AbstractStreamOperator.class.getDeclaredField("chainingStrategy"); + field.setAccessible(true); + field.set(this, ChainingStrategy.ALWAYS); + } catch (Exception e) { + // Ignore if chainingStrategy doesn't exist in Flink 2.x + } } @Override @@ -79,7 +87,8 @@ public void open() throws Exception {} @Override public void initializeState(StateInitializationContext context) throws Exception { - schemaEvolutionClient.registerSubtask(getRuntimeContext().getIndexOfThisSubtask()); + schemaEvolutionClient.registerSubtask( + getRuntimeContext().getTaskInfo().getIndexOfThisSubtask()); } @Override @@ -149,7 +158,8 @@ private void handleFlushEvent(FlushEvent event) throws Exception { }); } schemaEvolutionClient.notifyFlushSuccess( - getRuntimeContext().getIndexOfThisSubtask(), event.getSourceSubTaskId()); + getRuntimeContext().getTaskInfo().getIndexOfThisSubtask(), + event.getSourceSubTaskId()); } private void emitLatestSchema(TableId tableId) throws Exception { diff --git a/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/serializer/EnumSerializerTest.java b/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/serializer/EnumSerializerTest.java index e101492cdd0..c7fe72b1b8a 100644 --- a/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/serializer/EnumSerializerTest.java +++ b/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/serializer/EnumSerializerTest.java @@ -22,6 +22,7 @@ import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; import org.apache.flink.api.common.typeutils.TypeSerializerSnapshotSerializationUtil; import org.apache.flink.cdc.common.utils.InstantiationUtil; +import org.apache.flink.cdc.runtime.utils.FlinkCompatibilityUtils; import org.apache.flink.core.memory.DataInputViewStreamWrapper; import org.apache.flink.core.memory.DataOutputViewStreamWrapper; import org.apache.flink.util.TestLogger; @@ -85,7 +86,7 @@ void testReconfiguration() { EnumSerializer.EnumSerializerSnapshot serializerSnapshot = new EnumSerializer.EnumSerializerSnapshot(PublicEnum.class, mockPreviousOrder); TypeSerializerSchemaCompatibility compatibility = - serializerSnapshot.resolveSchemaCompatibility(serializer); + FlinkCompatibilityUtils.resolveSchemaCompatibility(serializerSnapshot, serializer); Assertions.assertThat(compatibility.isCompatibleWithReconfiguredSerializer()).isTrue(); // after reconfiguration, the order should be first the original BAR, PAULA, NATHANIEL, @@ -131,7 +132,7 @@ void testConfigurationSnapshotSerialization() throws Exception { } TypeSerializerSchemaCompatibility compatResult = - restoredConfig.resolveSchemaCompatibility(serializer); + FlinkCompatibilityUtils.resolveSchemaCompatibility(restoredConfig, serializer); Assertions.assertThat(compatResult.isCompatibleAsIs()).isTrue(); Assertions.assertThat(serializer.getValueToOrdinal().get(PublicEnum.FOO).intValue()) @@ -217,7 +218,7 @@ void testSerializeReconfiguredEnumSerializer() throws Exception { EnumSerializer.EnumSerializerSnapshot serializerSnapshot = new EnumSerializer.EnumSerializerSnapshot(PublicEnum.class, mockPreviousOrder); TypeSerializerSchemaCompatibility compatibility = - serializerSnapshot.resolveSchemaCompatibility(serializer); + FlinkCompatibilityUtils.resolveSchemaCompatibility(serializerSnapshot, serializer); Assertions.assertThat(compatibility.isCompatibleWithReconfiguredSerializer()).isTrue(); // verify that after the serializer was read, the reconfigured constant ordering is diff --git a/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/serializer/SerializerTestBase.java b/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/serializer/SerializerTestBase.java index 79566763bc9..d50bb8a3910 100644 --- a/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/serializer/SerializerTestBase.java +++ b/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/serializer/SerializerTestBase.java @@ -23,6 +23,7 @@ import org.apache.flink.api.common.typeutils.TypeSerializerSnapshotSerializationUtil; import org.apache.flink.api.java.typeutils.runtime.NullableSerializer; import org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializer; +import org.apache.flink.cdc.runtime.utils.FlinkCompatibilityUtils; import org.apache.flink.core.memory.DataInputDeserializer; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataInputViewStreamWrapper; @@ -152,7 +153,7 @@ protected void testSnapshotConfigurationAndReconfigure() throws Exception { } TypeSerializerSchemaCompatibility strategy = - restoredConfig.resolveSchemaCompatibility(getSerializer()); + FlinkCompatibilityUtils.resolveSchemaCompatibility(restoredConfig, getSerializer()); final TypeSerializer restoreSerializer; if (strategy.isCompatibleAsIs()) { restoreSerializer = restoredConfig.restoreSerializer(); diff --git a/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/testutils/operators/DistributedEventOperatorTestHarness.java b/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/testutils/operators/DistributedEventOperatorTestHarness.java index 2b9030bc458..ab86809e1e0 100644 --- a/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/testutils/operators/DistributedEventOperatorTestHarness.java +++ b/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/testutils/operators/DistributedEventOperatorTestHarness.java @@ -30,20 +30,16 @@ import org.apache.flink.cdc.runtime.operators.sink.SchemaEvolutionClient; import org.apache.flink.cdc.runtime.testutils.schema.CollectingMetadataApplier; import org.apache.flink.cdc.runtime.testutils.schema.TestingSchemaRegistryGateway; +import org.apache.flink.cdc.runtime.utils.FlinkCompatibilityUtils; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.jobgraph.tasks.TaskOperatorEventGateway; import org.apache.flink.runtime.operators.testutils.DummyEnvironment; import org.apache.flink.streaming.api.operators.AbstractStreamOperator; import org.apache.flink.streaming.api.operators.Output; -import org.apache.flink.streaming.api.watermark.Watermark; -import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker; -import org.apache.flink.streaming.runtime.streamrecord.RecordAttributes; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.runtime.tasks.StreamTask; -import org.apache.flink.streaming.runtime.watermarkstatus.WatermarkStatus; import org.apache.flink.streaming.util.MockStreamConfig; -import org.apache.flink.util.OutputTag; import org.apache.flink.util.SerializedValue; import java.io.IOException; @@ -154,68 +150,73 @@ public void close() throws Exception { // -------------------------------------- Helper functions ------------------------------- + @SuppressWarnings("unchecked") private void initializeOperator() throws Exception { - operator.setup( + Output> output = + (Output>) + java.lang.reflect.Proxy.newProxyInstance( + Output.class.getClassLoader(), + new Class[] {Output.class}, + new EventCollectingOutputHandler<>( + outputRecords, schemaRegistryGateway)); + FlinkCompatibilityUtils.setupOperator( + operator, new MockStreamTask(schemaRegistryGateway), new MockStreamConfig(new Configuration(), numOutputs), - new EventCollectingOutput<>(outputRecords, schemaRegistryGateway)); + output); schemaRegistryGateway.sendOperatorEventToCoordinator( SINK_OPERATOR_ID, new SerializedValue<>(new SinkWriterRegisterEvent(0))); } // ---------------------------------------- Helper classes --------------------------------- - private static class EventCollectingOutput implements Output> { + private static class EventCollectingOutputHandler + implements java.lang.reflect.InvocationHandler { private final LinkedList> outputRecords; - private final TestingSchemaRegistryGateway schemaRegistryGateway; - public EventCollectingOutput( + EventCollectingOutputHandler( LinkedList> outputRecords, TestingSchemaRegistryGateway schemaRegistryGateway) { this.outputRecords = outputRecords; this.schemaRegistryGateway = schemaRegistryGateway; } + @SuppressWarnings("unchecked") @Override - public void collect(StreamRecord record) { - outputRecords.add(record); - Event event = record.getValue(); - if (event instanceof FlushEvent) { - try { - schemaRegistryGateway.sendOperatorEventToCoordinator( - SINK_OPERATOR_ID, new SerializedValue<>(new FlushSuccessEvent(0, 0))); - } catch (IOException e) { - throw new RuntimeException(e); - } + public Object invoke(Object proxy, java.lang.reflect.Method method, Object[] args) + throws Throwable { + switch (method.getName()) { + case "collect": + if (args != null && args.length == 1 && args[0] instanceof StreamRecord) { + StreamRecord record = (StreamRecord) args[0]; + outputRecords.add(record); + Event event = record.getValue(); + if (event instanceof FlushEvent) { + try { + schemaRegistryGateway.sendOperatorEventToCoordinator( + SINK_OPERATOR_ID, + new SerializedValue<>(new FlushSuccessEvent(0, 0))); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + return null; + } + throw new UnsupportedOperationException(); + case "emitRecordAttributes": + case "close": + return null; + case "toString": + return "EventCollectingOutput"; + case "hashCode": + return System.identityHashCode(proxy); + case "equals": + return proxy == args[0]; + default: + throw new UnsupportedOperationException(method.getName()); } } - - @Override - public void emitWatermark(Watermark mark) { - throw new UnsupportedOperationException(); - } - - @Override - public void emitWatermarkStatus(WatermarkStatus watermarkStatus) { - throw new UnsupportedOperationException(); - } - - @Override - public void collect(OutputTag outputTag, StreamRecord record) { - throw new UnsupportedOperationException(); - } - - @Override - public void emitLatencyMarker(LatencyMarker latencyMarker) { - throw new UnsupportedOperationException(); - } - - @Override - public void emitRecordAttributes(RecordAttributes recordAttributes) {} - - @Override - public void close() {} } private static class MockStreamTask extends StreamTask> { diff --git a/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/testutils/operators/RegularEventOperatorTestHarness.java b/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/testutils/operators/RegularEventOperatorTestHarness.java index ea4de7a14e7..6b4facdbb8c 100644 --- a/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/testutils/operators/RegularEventOperatorTestHarness.java +++ b/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/testutils/operators/RegularEventOperatorTestHarness.java @@ -39,20 +39,16 @@ import org.apache.flink.cdc.runtime.operators.sink.SchemaEvolutionClient; import org.apache.flink.cdc.runtime.testutils.schema.CollectingMetadataApplier; import org.apache.flink.cdc.runtime.testutils.schema.TestingSchemaRegistryGateway; +import org.apache.flink.cdc.runtime.utils.FlinkCompatibilityUtils; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.jobgraph.tasks.TaskOperatorEventGateway; import org.apache.flink.runtime.operators.testutils.DummyEnvironment; import org.apache.flink.streaming.api.operators.AbstractStreamOperator; import org.apache.flink.streaming.api.operators.Output; -import org.apache.flink.streaming.api.watermark.Watermark; -import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker; -import org.apache.flink.streaming.runtime.streamrecord.RecordAttributes; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.runtime.tasks.StreamTask; -import org.apache.flink.streaming.runtime.watermarkstatus.WatermarkStatus; import org.apache.flink.streaming.util.MockStreamConfig; -import org.apache.flink.util.OutputTag; import org.apache.flink.util.SerializedValue; import org.slf4j.Logger; @@ -285,68 +281,73 @@ public void close() throws Exception { // -------------------------------------- Helper functions ------------------------------- + @SuppressWarnings("unchecked") private void initializeOperator() throws Exception { - operator.setup( + Output> output = + (Output>) + java.lang.reflect.Proxy.newProxyInstance( + Output.class.getClassLoader(), + new Class[] {Output.class}, + new EventCollectingOutputHandler<>( + outputRecords, schemaRegistryGateway)); + FlinkCompatibilityUtils.setupOperator( + operator, new MockStreamTask(schemaRegistryGateway), new MockStreamConfig(new Configuration(), numOutputs), - new EventCollectingOutput<>(outputRecords, schemaRegistryGateway)); + output); schemaRegistryGateway.sendOperatorEventToCoordinator( SINK_OPERATOR_ID, new SerializedValue<>(new SinkWriterRegisterEvent(0))); } // ---------------------------------------- Helper classes --------------------------------- - private static class EventCollectingOutput implements Output> { + private static class EventCollectingOutputHandler + implements java.lang.reflect.InvocationHandler { private final LinkedList> outputRecords; - private final TestingSchemaRegistryGateway schemaRegistryGateway; - public EventCollectingOutput( + EventCollectingOutputHandler( LinkedList> outputRecords, TestingSchemaRegistryGateway schemaRegistryGateway) { this.outputRecords = outputRecords; this.schemaRegistryGateway = schemaRegistryGateway; } + @SuppressWarnings("unchecked") @Override - public void collect(StreamRecord record) { - outputRecords.add(record); - Event event = record.getValue(); - if (event instanceof FlushEvent) { - try { - schemaRegistryGateway.sendOperatorEventToCoordinator( - SINK_OPERATOR_ID, new SerializedValue<>(new FlushSuccessEvent(0, 0))); - } catch (IOException e) { - throw new RuntimeException(e); - } + public Object invoke(Object proxy, java.lang.reflect.Method method, Object[] args) + throws Throwable { + switch (method.getName()) { + case "collect": + if (args != null && args.length == 1 && args[0] instanceof StreamRecord) { + StreamRecord record = (StreamRecord) args[0]; + outputRecords.add(record); + Event event = record.getValue(); + if (event instanceof FlushEvent) { + try { + schemaRegistryGateway.sendOperatorEventToCoordinator( + SINK_OPERATOR_ID, + new SerializedValue<>(new FlushSuccessEvent(0, 0))); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + return null; + } + throw new UnsupportedOperationException(); + case "emitRecordAttributes": + case "close": + return null; + case "toString": + return "EventCollectingOutput"; + case "hashCode": + return System.identityHashCode(proxy); + case "equals": + return proxy == args[0]; + default: + throw new UnsupportedOperationException(method.getName()); } } - - @Override - public void emitWatermark(Watermark mark) { - throw new UnsupportedOperationException(); - } - - @Override - public void emitWatermarkStatus(WatermarkStatus watermarkStatus) { - throw new UnsupportedOperationException(); - } - - @Override - public void collect(OutputTag outputTag, StreamRecord record) { - throw new UnsupportedOperationException(); - } - - @Override - public void emitLatencyMarker(LatencyMarker latencyMarker) { - throw new UnsupportedOperationException(); - } - - @Override - public void emitRecordAttributes(RecordAttributes recordAttributes) {} - - @Override - public void close() {} } private static class MockStreamTask extends StreamTask> { diff --git a/pom.xml b/pom.xml index 3d87b69054f..295538df596 100644 --- a/pom.xml +++ b/pom.xml @@ -33,6 +33,7 @@ limitations under the License. pom + flink-cdc-flink2.2-compat flink-cdc-cli flink-cdc-common flink-cdc-composer @@ -67,8 +68,13 @@ limitations under the License. true - 1.20.3 + 1.20.3 + 2.2.0 + ${flink.1.x.version} 1.20 + 31.1-jre-17.0 + 33.4.0-jre-20.0 + ${flink.1.x.shaded.guava.version} 17.0 1.9.8.Final 3.2.0 @@ -289,7 +295,7 @@ limitations under the License. org.apache.flink flink-shaded-guava - 31.1-jre-${flink.shaded.version} + ${shaded.guava.version} @@ -842,6 +848,47 @@ limitations under the License. + + flink2 + + + + + org.apache.maven.plugins + maven-dependency-plugin + + + copy-flink2-guava + process-test-resources + + copy + + + + + org.apache.flink + flink-shaded-guava + ${flink.2.x.shaded.guava.version} + ${project.build.directory}/flink2-extra-libs + + + + + + + + org.apache.maven.plugins + maven-surefire-plugin + + + ${project.build.directory}/flink2-extra-libs/flink-shaded-guava-${flink.2.x.shaded.guava.version}.jar + + + + + + + java-17-target