diff --git a/.github/workflows/flink_cdc_ci.yml b/.github/workflows/flink_cdc_ci.yml index e2e70d9096d..789313b1f11 100644 --- a/.github/workflows/flink_cdc_ci.yml +++ b/.github/workflows/flink_cdc_ci.yml @@ -86,9 +86,3 @@ jobs: java-version: "[8]" flink-version: "['1.19.2', '1.20.1']" module: source_e2e - migration_test: - name: Migration Tests - uses: ./.github/workflows/flink_cdc_migration_test_base.yml - with: - java-version: "[8]" - flink-version: "['1.19.2', '1.20.1']" \ No newline at end of file diff --git a/.github/workflows/flink_cdc_ci_nightly.yml b/.github/workflows/flink_cdc_ci_nightly.yml index 0b65cc0be71..5f54ecd1a28 100644 --- a/.github/workflows/flink_cdc_ci_nightly.yml +++ b/.github/workflows/flink_cdc_ci_nightly.yml @@ -79,10 +79,3 @@ jobs: java-version: "[11]" flink-version: "['1.19.2', '1.20.1']" module: source_e2e - migration_test: - if: github.repository == 'apache/flink-cdc' - name: Migration Tests - uses: ./.github/workflows/flink_cdc_migration_test_base.yml - with: - java-version: "[11]" - flink-version: "['1.19.2', '1.20.1']" diff --git a/.github/workflows/flink_cdc_migration_test_base.yml b/.github/workflows/flink_cdc_migration_test_base.yml deleted file mode 100644 index 058d89ca8ae..00000000000 --- a/.github/workflows/flink_cdc_migration_test_base.yml +++ /dev/null @@ -1,135 +0,0 @@ -# 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. - -name: Migration Tests - -on: - workflow_call: - inputs: - java-version: - description: "Jdk versions to test against, passed as a JSON array string." - required: false - type: string - default: "['8']" - flink-version: - description: "Flink versions to test against, passed as a JSON array string." - required: true - type: string - -jobs: - migration_test_ut: - runs-on: ubuntu-latest - strategy: - fail-fast: false - matrix: - java-version: ${{ fromJSON(inputs.java-version) }} - flink-version: ${{ fromJSON(inputs.flink-version) }} - steps: - - name: Check out repository code - uses: actions/checkout@v4 - with: - submodules: true - - uses: actions/setup-java@v4 - with: - java-version: ${{ matrix.java-version }} - distribution: temurin - - name: Compile snapshot CDC version - run: mvn --no-snapshot-updates -B install -DskipTests - - name: Run migration tests - run: cd flink-cdc-migration-tests && mvn clean verify - - pipeline_migration_test: - runs-on: ubuntu-latest - strategy: - fail-fast: false - matrix: - java-version: ${{ fromJSON(inputs.java-version) }} - flink-version: ${{ fromJSON(inputs.flink-version) }} - steps: - - uses: actions/checkout@v4 - - name: Set up Ruby - uses: ruby/setup-ruby@v1 - with: - ruby-version: 3.0 - bundler-cache: true # runs 'bundle install' and caches installed gems automatically - - uses: actions/setup-java@v4 - with: - java-version: ${{ matrix.java-version }} - distribution: temurin - cache: maven - - name: Install dependencies - run: gem install terminal-table - - name: Prepare CDC versions - run: CDC_SOURCE_HOME=$PWD ruby tools/mig-test/prepare_libs.rb - - name: Prepare Flink distro - run: wget https://dlcdn.apache.org/flink/flink-${{ matrix.flink-version }}/flink-${{ matrix.flink-version }}-bin-scala_2.12.tgz && tar -xzvf flink-${{ matrix.flink-version }}-bin-scala_2.12.tgz - working-directory: ./tools/mig-test - - name: Patch Flink configs - run: FLINK_HOME=./flink-${{ matrix.flink-version }}/ ruby misc/patch_flink_conf.rb - working-directory: ./tools/mig-test - - name: Start containers - run: cd conf && docker compose up -d - working-directory: ./tools/mig-test - - name: Run migration tests - run: FLINK_HOME=./flink-${{ matrix.flink-version }}/ ruby run_migration_test.rb ${{ matrix.flink-version }} - working-directory: ./tools/mig-test - - name: Stop containers - if: always() - run: cd conf && docker compose down - working-directory: ./tools/mig-test - - data_stream_migration_test: - runs-on: ubuntu-latest - strategy: - fail-fast: false - matrix: - java-version: ${{ fromJSON(inputs.java-version) }} - flink-version: [ '1.19.2', '1.20.1' ] - - steps: - - uses: actions/checkout@v4 - - name: Set up Ruby - uses: ruby/setup-ruby@v1 - with: - ruby-version: 3.0 - bundler-cache: true # runs 'bundle install' and caches installed gems automatically - - uses: actions/setup-java@v4 - with: - java-version: ${{ matrix.java-version }} - distribution: temurin - cache: maven - - name: Install dependencies - run: gem install terminal-table - - name: Prepare CDC versions - run: CDC_SOURCE_HOME=$PWD ruby tools/mig-test/prepare_libs.rb - - name: Prepare Flink distro - run: wget https://dlcdn.apache.org/flink/flink-${{ matrix.flink-version }}/flink-${{ matrix.flink-version }}-bin-scala_2.12.tgz && tar -xzvf flink-${{ matrix.flink-version }}-bin-scala_2.12.tgz - working-directory: ./tools/mig-test - - name: Patch Flink configs - run: FLINK_HOME=./flink-${{ matrix.flink-version }}/ ruby misc/patch_flink_conf.rb - working-directory: ./tools/mig-test - - name: Compile Dummy DataStream Jobs - run: cd datastream && ruby compile_jobs.rb - working-directory: ./tools/mig-test - - name: Start containers - run: cd conf && docker compose up -d - working-directory: ./tools/mig-test - - name: Run migration tests - run: cd datastream && FLINK_HOME=../flink-${{ matrix.flink-version }}/ ruby run_migration_test.rb - working-directory: ./tools/mig-test - - name: Stop containers - if: always() - run: cd conf && docker compose down - working-directory: ./tools/mig-test diff --git a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/MaxComputeE2eITCase.java b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/MaxComputeE2eITCase.java index bbb7beed46b..59d966bc2af 100644 --- a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/MaxComputeE2eITCase.java +++ b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/MaxComputeE2eITCase.java @@ -136,8 +136,7 @@ private void startTest(String testSet) throws Exception { + "pipeline:\n" + " parallelism: 4"; Path maxcomputeCdcJar = TestUtils.getResource("maxcompute-cdc-pipeline-connector.jar"); - Path valuesCdcJar = TestUtils.getResource("values-cdc-pipeline-connector.jar"); - submitPipelineJob(pipelineJob, maxcomputeCdcJar, valuesCdcJar); + submitPipelineJob(pipelineJob, maxcomputeCdcJar); waitUntilJobFinished(Duration.ofMinutes(10)); LOG.info("Pipeline job is running"); } diff --git a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/MySqlToDorisE2eITCase.java b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/MySqlToDorisE2eITCase.java index 722d8fd5380..434ccbcf62c 100644 --- a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/MySqlToDorisE2eITCase.java +++ b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/MySqlToDorisE2eITCase.java @@ -19,8 +19,6 @@ import org.apache.flink.cdc.common.test.utils.TestUtils; import org.apache.flink.cdc.connectors.doris.sink.utils.DorisContainer; -import org.apache.flink.cdc.connectors.mysql.testutils.MySqlContainer; -import org.apache.flink.cdc.connectors.mysql.testutils.MySqlVersion; import org.apache.flink.cdc.connectors.mysql.testutils.UniqueDatabase; import org.apache.flink.cdc.pipeline.tests.utils.PipelineTestEnvironment; @@ -32,6 +30,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.testcontainers.containers.wait.strategy.LogMessageWaitStrategy; +import org.testcontainers.junit.jupiter.Container; import org.testcontainers.lifecycle.Startables; import java.nio.file.Path; @@ -53,30 +52,8 @@ class MySqlToDorisE2eITCase extends PipelineTestEnvironment { private static final Logger LOG = LoggerFactory.getLogger(MySqlToDorisE2eITCase.class); - // ------------------------------------------------------------------------------------------ - // 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 MYSQL_DRIVER_CLASS = "com.mysql.cj.jdbc.Driver"; - public static final Duration DEFAULT_STARTUP_TIMEOUT = Duration.ofSeconds(240); - public static final Duration DEFAULT_RESULT_VERIFY_TIMEOUT = Duration.ofSeconds(30); - - @org.testcontainers.junit.jupiter.Container - public static final MySqlContainer MYSQL = - (MySqlContainer) - new MySqlContainer( - MySqlVersion.V8_0) // v8 support both ARM and AMD architectures - .withConfigurationOverride("docker/mysql/my.cnf") - .withSetupSQL("docker/mysql/setup.sql") - .withDatabaseName("flink-test") - .withUsername("flinkuser") - .withPassword("flinkpw") - .withNetwork(NETWORK) - .withNetworkAliases("mysql"); - - @org.testcontainers.junit.jupiter.Container - public static final DorisContainer DORIS = + @Container + protected static final DorisContainer DORIS = new DorisContainer(NETWORK).withNetworkAliases("doris"); protected final UniqueDatabase mysqlInventoryDatabase = @@ -96,13 +73,13 @@ public static void initializeContainers() { new LogMessageWaitStrategy() .withRegEx(".*get heartbeat from FE.*") .withTimes(1) - .withStartupTimeout(DEFAULT_STARTUP_TIMEOUT) + .withStartupTimeout(STARTUP_WAITING_TIMEOUT) .waitUntilReady(DORIS); while (!checkBackendAvailability()) { try { if (System.currentTimeMillis() - startWaitingTimestamp - > DEFAULT_STARTUP_TIMEOUT.toMillis()) { + > STARTUP_WAITING_TIMEOUT.toMillis()) { throw new RuntimeException("Doris backend startup timed out."); } LOG.info("Waiting for backends to be available"); @@ -189,10 +166,8 @@ void testSyncWholeDatabase() throws Exception { DORIS.getUsername(), DORIS.getPassword(), parallelism); - Path mysqlCdcJar = TestUtils.getResource("mysql-cdc-pipeline-connector.jar"); Path dorisCdcConnector = TestUtils.getResource("doris-cdc-pipeline-connector.jar"); - Path mysqlDriverJar = TestUtils.getResource("mysql-driver.jar"); - submitPipelineJob(pipelineJob, mysqlCdcJar, dorisCdcConnector, mysqlDriverJar); + submitPipelineJob(pipelineJob, dorisCdcConnector); waitUntilJobRunning(Duration.ofSeconds(30)); LOG.info("Pipeline job is running"); @@ -1011,7 +986,7 @@ private void validateSinkResult( "SELECT * FROM " + tableName, columnCount, expected, - DEFAULT_RESULT_VERIFY_TIMEOUT.toMillis(), + EVENT_WAITING_TIMEOUT.toMillis(), true); } @@ -1022,7 +997,7 @@ private void validateSinkSchema(String databaseName, String tableName, List results = Collections.emptyList(); while (System.currentTimeMillis() < deadline) { try { @@ -364,7 +364,7 @@ private void validateSinkResult( LOG.info( "Successfully verified {} records in {} seconds.", expected.size(), - (System.currentTimeMillis() - deadline + TESTCASE_TIMEOUT.toMillis()) + (System.currentTimeMillis() - deadline + EVENT_WAITING_TIMEOUT.toMillis()) / 1000); return; } catch (Exception e) { diff --git a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/MySqlToPaimonE2eITCase.java b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/MySqlToPaimonE2eITCase.java index 4a187eae367..d263dfd53e8 100644 --- a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/MySqlToPaimonE2eITCase.java +++ b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/MySqlToPaimonE2eITCase.java @@ -18,8 +18,6 @@ package org.apache.flink.cdc.pipeline.tests; 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.cdc.connectors.mysql.testutils.UniqueDatabase; import org.apache.flink.cdc.pipeline.tests.utils.PipelineTestEnvironment; @@ -31,7 +29,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.testcontainers.containers.Container; -import org.testcontainers.containers.output.Slf4jLogConsumer; import org.testcontainers.images.builder.Transferable; import org.testcontainers.lifecycle.Startables; import org.testcontainers.utility.MountableFile; @@ -54,27 +51,7 @@ class MySqlToPaimonE2eITCase extends PipelineTestEnvironment { private static final Logger LOG = LoggerFactory.getLogger(MySqlToPaimonE2eITCase.class); - public static final Duration TESTCASE_TIMEOUT = Duration.ofMinutes(3); - - // ------------------------------------------------------------------------------------------ - // 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"; - - @org.testcontainers.junit.jupiter.Container - static final MySqlContainer MYSQL = - (MySqlContainer) - new MySqlContainer( - MySqlVersion.V8_0) // v8 support both ARM and AMD architectures - .withConfigurationOverride("docker/mysql/my.cnf") - .withSetupSQL("docker/mysql/setup.sql") - .withDatabaseName("flink-test") - .withUsername("flinkuser") - .withPassword("flinkpw") - .withNetwork(NETWORK) - .withNetworkAliases("mysql") - .withLogConsumer(new Slf4jLogConsumer(LOG)); + private static final Duration PAIMON_TESTCASE_TIMEOUT = Duration.ofMinutes(3); protected final UniqueDatabase inventoryDatabase = new UniqueDatabase(MYSQL, "paimon_inventory", MYSQL_TEST_USER, MYSQL_TEST_PASSWORD); @@ -131,11 +108,9 @@ void testSyncWholeDatabase() throws Exception { + " schema.change.behavior: evolve\n" + " parallelism: 4", MYSQL_TEST_USER, MYSQL_TEST_PASSWORD, database, warehouse); - Path mysqlCdcJar = TestUtils.getResource("mysql-cdc-pipeline-connector.jar"); Path paimonCdcConnector = TestUtils.getResource("paimon-cdc-pipeline-connector.jar"); Path hadoopJar = TestUtils.getResource("flink-shade-hadoop.jar"); - Path mysqlDriverJar = TestUtils.getResource("mysql-driver.jar"); - submitPipelineJob(pipelineJob, mysqlCdcJar, paimonCdcConnector, mysqlDriverJar, hadoopJar); + submitPipelineJob(pipelineJob, paimonCdcConnector, hadoopJar); waitUntilJobRunning(Duration.ofSeconds(30)); LOG.info("Pipeline job is running"); validateSinkResult( @@ -238,10 +213,8 @@ public void testSinkToAppendOnlyTable() throws Exception { + " schema.change.behavior: evolve\n" + " parallelism: 4", MYSQL_TEST_USER, MYSQL_TEST_PASSWORD, database, database, warehouse); - Path mysqlCdcJar = TestUtils.getResource("mysql-cdc-pipeline-connector.jar"); Path paimonCdcConnector = TestUtils.getResource("paimon-cdc-pipeline-connector.jar"); Path hadoopJar = TestUtils.getResource("flink-shade-hadoop.jar"); - Path mysqlDriverJar = TestUtils.getResource("mysql-driver.jar"); String mysqlJdbcUrl = String.format( "jdbc:mysql://%s:%s/%s", @@ -274,7 +247,7 @@ public void testSinkToAppendOnlyTable() throws Exception { LOG.error("Create table for CDC failed.", e); throw e; } - submitPipelineJob(pipelineJob, mysqlCdcJar, paimonCdcConnector, mysqlDriverJar, hadoopJar); + submitPipelineJob(pipelineJob, paimonCdcConnector, hadoopJar); waitUntilJobRunning(Duration.ofSeconds(30)); LOG.info("Pipeline job is running"); validateSinkResult( @@ -495,7 +468,7 @@ private void validateSinkResult( String warehouse, String database, String table, List expected) throws InterruptedException { LOG.info("Verifying Paimon {}::{}::{} results...", warehouse, database, table); - long deadline = System.currentTimeMillis() + TESTCASE_TIMEOUT.toMillis(); + long deadline = System.currentTimeMillis() + PAIMON_TESTCASE_TIMEOUT.toMillis(); List results = Collections.emptyList(); while (System.currentTimeMillis() < deadline) { try { @@ -504,7 +477,7 @@ private void validateSinkResult( LOG.info( "Successfully verified {} records in {} seconds.", expected.size(), - (System.currentTimeMillis() - deadline + TESTCASE_TIMEOUT.toMillis()) + (System.currentTimeMillis() - deadline + PAIMON_TESTCASE_TIMEOUT.toMillis()) / 1000); return; } catch (Exception e) { diff --git a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/MysqlE2eITCase.java b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/MysqlE2eITCase.java index b649eb907c4..b85d77ebc52 100644 --- a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/MysqlE2eITCase.java +++ b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/MysqlE2eITCase.java @@ -17,10 +17,7 @@ package org.apache.flink.cdc.pipeline.tests; -import org.apache.flink.cdc.common.test.utils.TestUtils; import org.apache.flink.cdc.common.utils.Preconditions; -import org.apache.flink.cdc.connectors.mysql.testutils.MySqlContainer; -import org.apache.flink.cdc.connectors.mysql.testutils.MySqlVersion; import org.apache.flink.cdc.connectors.mysql.testutils.UniqueDatabase; import org.apache.flink.cdc.pipeline.tests.utils.PipelineTestEnvironment; @@ -29,48 +26,25 @@ import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.testcontainers.containers.output.Slf4jLogConsumer; -import org.testcontainers.junit.jupiter.Container; -import java.nio.file.Path; import java.sql.Connection; import java.sql.DriverManager; import java.sql.ResultSet; import java.sql.SQLException; import java.sql.Statement; import java.time.Duration; -import java.util.concurrent.TimeoutException; +import java.util.function.Function; /** End-to-end tests for mysql cdc pipeline job. */ class MysqlE2eITCase extends PipelineTestEnvironment { private static final Logger LOG = LoggerFactory.getLogger(MysqlE2eITCase.class); - // ------------------------------------------------------------------------------------------ - // 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 MYSQL_DRIVER_CLASS = "com.mysql.cj.jdbc.Driver"; - protected static final String INTER_CONTAINER_MYSQL_ALIAS = "mysql"; - protected static final long EVENT_WAITING_TIMEOUT = 60000L; - - @Container - public static final MySqlContainer MYSQL = - (MySqlContainer) - new MySqlContainer( - MySqlVersion.V8_0) // v8 support both ARM and AMD architectures - .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)); - protected final UniqueDatabase mysqlInventoryDatabase = new UniqueDatabase(MYSQL, "mysql_inventory", MYSQL_TEST_USER, MYSQL_TEST_PASSWORD); + private final Function dbNameFormatter = + (s) -> String.format(s, mysqlInventoryDatabase.getDatabaseName()); + @BeforeEach public void before() throws Exception { super.before(); @@ -107,22 +81,13 @@ void testSyncWholeDatabase() throws Exception { MYSQL_TEST_PASSWORD, mysqlInventoryDatabase.getDatabaseName(), parallelism); - Path mysqlCdcJar = TestUtils.getResource("mysql-cdc-pipeline-connector.jar"); - Path valuesCdcJar = TestUtils.getResource("values-cdc-pipeline-connector.jar"); - Path mysqlDriverJar = TestUtils.getResource("mysql-driver.jar"); - submitPipelineJob(pipelineJob, mysqlCdcJar, valuesCdcJar, mysqlDriverJar); + + submitPipelineJob(pipelineJob); waitUntilJobRunning(Duration.ofSeconds(30)); LOG.info("Pipeline job is running"); - waitUntilSpecificEvent( - String.format( - "DataChangeEvent{tableId=%s.customers, before=[], after=[104, user_4, Shanghai, 123567891234], op=INSERT, meta=()}", - mysqlInventoryDatabase.getDatabaseName())); - waitUntilSpecificEvent( - String.format( - "DataChangeEvent{tableId=%s.products, before=[], after=[109, spare tire, 24 inch spare tire, 22.2, null, null, null], op=INSERT, meta=()}", - mysqlInventoryDatabase.getDatabaseName())); validateResult( + dbNameFormatter, "CreateTableEvent{tableId=%s.customers, schema=columns={`id` INT NOT NULL,`name` VARCHAR(255) NOT NULL 'flink',`address` VARCHAR(1024),`phone_number` VARCHAR(512)}, primaryKeys=id, options=()}", "DataChangeEvent{tableId=%s.customers, before=[], after=[104, user_4, Shanghai, 123567891234], op=INSERT, meta=()}", "DataChangeEvent{tableId=%s.customers, before=[], after=[103, user_3, Shanghai, 123567891234], op=INSERT, meta=()}", @@ -175,12 +140,8 @@ void testSyncWholeDatabase() throws Exception { throw e; } - waitUntilSpecificEvent( - String.format( - "DataChangeEvent{tableId=%s.products, before=[111, scooter, Big 2-wheel scooter , 5.17, null, null, null, 1], after=[], op=DELETE, meta=()}", - mysqlInventoryDatabase.getDatabaseName())); - validateResult( + dbNameFormatter, "DataChangeEvent{tableId=%s.products, before=[106, hammer, 16oz carpenter's hammer, 1.0, null, null, null], after=[106, hammer, 18oz carpenter hammer, 1.0, null, null, null], op=UPDATE, meta=()}", "DataChangeEvent{tableId=%s.products, before=[107, rocks, box of assorted rocks, 5.3, null, null, null], after=[107, rocks, box of assorted rocks, 5.1, null, null, null], op=UPDATE, meta=()}", "AddColumnEvent{tableId=%s.products, addedColumns=[ColumnWithPosition{column=`new_col` INT, position=LAST, existedColumnName=null}]}", @@ -217,22 +178,11 @@ void testSyncWholeDatabaseInBatchMode() throws Exception { MYSQL_TEST_PASSWORD, mysqlInventoryDatabase.getDatabaseName(), parallelism); - Path mysqlCdcJar = TestUtils.getResource("mysql-cdc-pipeline-connector.jar"); - Path valuesCdcJar = TestUtils.getResource("values-cdc-pipeline-connector.jar"); - Path mysqlDriverJar = TestUtils.getResource("mysql-driver.jar"); - submitPipelineJob(pipelineJob, mysqlCdcJar, valuesCdcJar, mysqlDriverJar); + submitPipelineJob(pipelineJob); waitUntilJobRunning(Duration.ofSeconds(30)); - LOG.info("Pipeline job is running"); - waitUntilSpecificEvent( - String.format( - "DataChangeEvent{tableId=%s.customers, before=[], after=[104, user_4, Shanghai, 123567891234], op=INSERT, meta=()}", - mysqlInventoryDatabase.getDatabaseName())); - waitUntilSpecificEvent( - String.format( - "DataChangeEvent{tableId=%s.products, before=[], after=[109, spare tire, 24 inch spare tire, 22.2, null, null, null], op=INSERT, meta=()}", - mysqlInventoryDatabase.getDatabaseName())); validateResult( + dbNameFormatter, "CreateTableEvent{tableId=%s.customers, schema=columns={`id` INT NOT NULL,`name` VARCHAR(255) NOT NULL 'flink',`address` VARCHAR(1024),`phone_number` VARCHAR(512)}, primaryKeys=id, options=()}", "CreateTableEvent{tableId=%s.products, schema=columns={`id` INT NOT NULL,`name` VARCHAR(255) NOT NULL 'flink',`description` VARCHAR(512),`weight` FLOAT,`enum_c` STRING 'red',`json_c` STRING,`point_c` STRING}, primaryKeys=id, options=()}", "DataChangeEvent{tableId=%s.customers, before=[], after=[104, user_4, Shanghai, 123567891234], op=INSERT, meta=()}", @@ -275,22 +225,11 @@ void testSchemaChangeEvents() throws Exception { MYSQL_TEST_PASSWORD, mysqlInventoryDatabase.getDatabaseName(), parallelism); - Path mysqlCdcJar = TestUtils.getResource("mysql-cdc-pipeline-connector.jar"); - Path valuesCdcJar = TestUtils.getResource("values-cdc-pipeline-connector.jar"); - Path mysqlDriverJar = TestUtils.getResource("mysql-driver.jar"); - submitPipelineJob(pipelineJob, mysqlCdcJar, valuesCdcJar, mysqlDriverJar); + submitPipelineJob(pipelineJob); waitUntilJobRunning(Duration.ofSeconds(30)); - LOG.info("Pipeline job is running"); - waitUntilSpecificEvent( - String.format( - "DataChangeEvent{tableId=%s.customers, before=[], after=[104, user_4, Shanghai, 123567891234], op=INSERT, meta=()}", - mysqlInventoryDatabase.getDatabaseName())); - waitUntilSpecificEvent( - String.format( - "DataChangeEvent{tableId=%s.products, before=[], after=[109, spare tire, 24 inch spare tire, 22.2, null, null, null], op=INSERT, meta=()}", - mysqlInventoryDatabase.getDatabaseName())); validateResult( + dbNameFormatter, "CreateTableEvent{tableId=%s.customers, schema=columns={`id` INT NOT NULL,`name` VARCHAR(255) NOT NULL 'flink',`address` VARCHAR(1024),`phone_number` VARCHAR(512)}, primaryKeys=id, options=()}", "DataChangeEvent{tableId=%s.customers, before=[], after=[104, user_4, Shanghai, 123567891234], op=INSERT, meta=()}", "DataChangeEvent{tableId=%s.customers, before=[], after=[103, user_3, Shanghai, 123567891234], op=INSERT, meta=()}", @@ -307,8 +246,6 @@ void testSchemaChangeEvents() throws Exception { "DataChangeEvent{tableId=%s.products, before=[], after=[101, scooter, Small 2-wheel scooter, 3.14, red, {\"key1\": \"value1\"}, {\"coordinates\":[1,1],\"type\":\"Point\",\"srid\":0}], op=INSERT, meta=()}", "DataChangeEvent{tableId=%s.products, before=[], after=[102, car battery, 12V car battery, 8.1, white, {\"key2\": \"value2\"}, {\"coordinates\":[2,2],\"type\":\"Point\",\"srid\":0}], op=INSERT, meta=()}"); - LOG.info("Begin incremental reading stage."); - // generate binlogs String mysqlJdbcUrl = String.format( "jdbc:mysql://%s:%s/%s", @@ -372,12 +309,8 @@ void testSchemaChangeEvents() throws Exception { throw e; } - waitUntilSpecificEvent( - String.format( - "DropTableEvent{tableId=%s.products}", - mysqlInventoryDatabase.getDatabaseName())); - validateResult( + dbNameFormatter, "DataChangeEvent{tableId=%s.products, before=[106, hammer, 16oz carpenter's hammer, 1.0, null, null, null], after=[106, hammer, 18oz carpenter hammer, 1.0, null, null, null], op=UPDATE, meta=()}", "DataChangeEvent{tableId=%s.products, before=[107, rocks, box of assorted rocks, 5.3, null, null, null], after=[107, rocks, box of assorted rocks, 5.1, null, null, null], op=UPDATE, meta=()}", "AddColumnEvent{tableId=%s.products, addedColumns=[ColumnWithPosition{column=`new_col` INT, position=LAST, existedColumnName=null}]}", @@ -429,23 +362,12 @@ void testSoftDelete() throws Exception { MYSQL_TEST_PASSWORD, mysqlInventoryDatabase.getDatabaseName(), parallelism); - Path mysqlCdcJar = TestUtils.getResource("mysql-cdc-pipeline-connector.jar"); - Path valuesCdcJar = TestUtils.getResource("values-cdc-pipeline-connector.jar"); - Path mysqlDriverJar = TestUtils.getResource("mysql-driver.jar"); - submitPipelineJob(pipelineJob, mysqlCdcJar, valuesCdcJar, mysqlDriverJar); + submitPipelineJob(pipelineJob); waitUntilJobRunning(Duration.ofSeconds(30)); LOG.info("Pipeline job is running"); - waitUntilSpecificEvent( - String.format( - "DataChangeEvent{tableId=%s.customers, before=[], after=[104, user_4, Shanghai, 123567891234, +I], op=INSERT, meta=()}", - mysqlInventoryDatabase.getDatabaseName())); - waitUntilSpecificEvent( - String.format( - "DataChangeEvent{tableId=%s.products, before=[], after=[109, spare tire, 24 inch spare tire, 22.2, null, null, null, +I], op=INSERT, meta=()}", - mysqlInventoryDatabase.getDatabaseName())); - validateResult( + dbNameFormatter, "CreateTableEvent{tableId=%s.customers, schema=columns={`id` INT NOT NULL,`name` VARCHAR(255) NOT NULL 'flink',`address` VARCHAR(1024),`phone_number` VARCHAR(512),`op_type` STRING NOT NULL}, primaryKeys=id, options=()}", "DataChangeEvent{tableId=%s.customers, before=[], after=[104, user_4, Shanghai, 123567891234, +I], op=INSERT, meta=()}", "DataChangeEvent{tableId=%s.customers, before=[], after=[103, user_3, Shanghai, 123567891234, +I], op=INSERT, meta=()}", @@ -521,12 +443,8 @@ void testSoftDelete() throws Exception { throw e; } - waitUntilSpecificEvent( - String.format( - "DropTableEvent{tableId=%s.products}", - mysqlInventoryDatabase.getDatabaseName())); - validateResult( + dbNameFormatter, "DataChangeEvent{tableId=%s.products, before=[106, hammer, 16oz carpenter's hammer, 1.0, null, null, null, -U], after=[106, hammer, 18oz carpenter hammer, 1.0, null, null, null, +U], op=UPDATE, meta=()}", "DataChangeEvent{tableId=%s.products, before=[107, rocks, box of assorted rocks, 5.3, null, null, null, -U], after=[107, rocks, box of assorted rocks, 5.1, null, null, null, +U], op=UPDATE, meta=()}", "AddColumnEvent{tableId=%s.products, addedColumns=[ColumnWithPosition{column=`new_col` INT, position=AFTER, existedColumnName=point_c}]}", @@ -605,10 +523,7 @@ void testDanglingDropTableEventInBinlog() throws Exception { logFileName, logPosition, parallelism); - Path mysqlCdcJar = TestUtils.getResource("mysql-cdc-pipeline-connector.jar"); - Path valuesCdcJar = TestUtils.getResource("values-cdc-pipeline-connector.jar"); - Path mysqlDriverJar = TestUtils.getResource("mysql-driver.jar"); - submitPipelineJob(pipelineJob, mysqlCdcJar, valuesCdcJar, mysqlDriverJar); + submitPipelineJob(pipelineJob); waitUntilJobRunning(Duration.ofSeconds(30)); try (Connection connection = mysqlInventoryDatabase.getJdbcConnection(); Statement statement = connection.createStatement()) { @@ -616,38 +531,8 @@ void testDanglingDropTableEventInBinlog() throws Exception { "UPDATE products SET description='18oz carpenter hammer' WHERE id=106;"); } validateResult( - String.format( - "CreateTableEvent{tableId=%s.products, schema=columns={`id` INT NOT NULL,`name` VARCHAR(255) NOT NULL 'flink',`description` VARCHAR(512),`weight` FLOAT,`enum_c` STRING 'red',`json_c` STRING,`point_c` STRING}, primaryKeys=id, options=()}", - mysqlInventoryDatabase.getDatabaseName()), - String.format( - "DataChangeEvent{tableId=%s.products, before=[106, hammer, 16oz carpenter's hammer, 1.0, null, null, null], after=[106, hammer, 18oz carpenter hammer, 1.0, null, null, null], op=UPDATE, meta=()}", - mysqlInventoryDatabase.getDatabaseName())); - } - - private void validateResult(String... expectedEvents) throws Exception { - String dbName = mysqlInventoryDatabase.getDatabaseName(); - for (String event : expectedEvents) { - waitUntilSpecificEvent(String.format(event, dbName, dbName)); - } - } - - private void waitUntilSpecificEvent(String event) throws Exception { - boolean result = false; - long endTimeout = System.currentTimeMillis() + MysqlE2eITCase.EVENT_WAITING_TIMEOUT; - while (System.currentTimeMillis() < endTimeout) { - String stdout = taskManagerConsumer.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: " - + taskManagerConsumer.toUtf8String()); - } + dbNameFormatter, + "CreateTableEvent{tableId=%s.products, schema=columns={`id` INT NOT NULL,`name` VARCHAR(255) NOT NULL 'flink',`description` VARCHAR(512),`weight` FLOAT,`enum_c` STRING 'red',`json_c` STRING,`point_c` STRING}, primaryKeys=id, options=()}", + "DataChangeEvent{tableId=%s.products, before=[106, hammer, 16oz carpenter's hammer, 1.0, null, null, null], after=[106, hammer, 18oz carpenter hammer, 1.0, null, null, null], op=UPDATE, meta=()}"); } } diff --git a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/MysqlToKafkaE2eITCase.java b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/MysqlToKafkaE2eITCase.java index b612f682d41..a8093f0f7b6 100644 --- a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/MysqlToKafkaE2eITCase.java +++ b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/MysqlToKafkaE2eITCase.java @@ -20,8 +20,6 @@ import org.apache.flink.cdc.common.event.TableId; import org.apache.flink.cdc.common.test.utils.TestUtils; import org.apache.flink.cdc.connectors.kafka.sink.KafkaUtil; -import org.apache.flink.cdc.connectors.mysql.testutils.MySqlContainer; -import org.apache.flink.cdc.connectors.mysql.testutils.MySqlVersion; import org.apache.flink.cdc.connectors.mysql.testutils.UniqueDatabase; import org.apache.flink.cdc.pipeline.tests.utils.PipelineTestEnvironment; @@ -43,7 +41,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.testcontainers.containers.KafkaContainer; -import org.testcontainers.containers.output.Slf4jLogConsumer; import org.testcontainers.junit.jupiter.Container; import org.testcontainers.lifecycle.Startables; import org.testcontainers.shaded.org.apache.commons.lang3.StringUtils; @@ -77,15 +74,6 @@ class MysqlToKafkaE2eITCase extends PipelineTestEnvironment { private static final Logger LOG = LoggerFactory.getLogger(MysqlToKafkaE2eITCase.class); - // ------------------------------------------------------------------------------------------ - // 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 MYSQL_DRIVER_CLASS = "com.mysql.cj.jdbc.Driver"; - protected static final String INTER_CONTAINER_MYSQL_ALIAS = "mysql"; - protected static final long EVENT_WAITING_TIMEOUT = 60000L; - private static AdminClient admin; private static final String INTER_CONTAINER_KAFKA_ALIAS = "kafka"; private static final int ZK_TIMEOUT_MILLIS = 30000; @@ -94,20 +82,6 @@ class MysqlToKafkaE2eITCase extends PipelineTestEnvironment { private String topic; private KafkaConsumer consumer; - @Container - static final MySqlContainer MYSQL = - (MySqlContainer) - new MySqlContainer( - MySqlVersion.V8_0) // v8 support both ARM and AMD architectures - .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)); - @Container static final KafkaContainer KAFKA_CONTAINER = KafkaUtil.createKafkaContainer(KAFKA, LOG) @@ -177,10 +151,8 @@ void testSyncWholeDatabaseWithDebeziumJson() throws Exception { mysqlInventoryDatabase.getDatabaseName(), topic, parallelism); - Path mysqlCdcJar = TestUtils.getResource("mysql-cdc-pipeline-connector.jar"); Path kafkaCdcJar = TestUtils.getResource("kafka-cdc-pipeline-connector.jar"); - Path mysqlDriverJar = TestUtils.getResource("mysql-driver.jar"); - submitPipelineJob(pipelineJob, mysqlCdcJar, kafkaCdcJar, mysqlDriverJar); + submitPipelineJob(pipelineJob, kafkaCdcJar); waitUntilJobRunning(Duration.ofSeconds(30)); LOG.info("Pipeline job is running"); List> collectedRecords = new ArrayList<>(); @@ -253,10 +225,8 @@ public void testSyncWholeDatabaseWithCanalJson() throws Exception { mysqlInventoryDatabase.getDatabaseName(), topic, parallelism); - Path mysqlCdcJar = TestUtils.getResource("mysql-cdc-pipeline-connector.jar"); Path kafkaCdcJar = TestUtils.getResource("kafka-cdc-pipeline-connector.jar"); - Path mysqlDriverJar = TestUtils.getResource("mysql-driver.jar"); - submitPipelineJob(pipelineJob, mysqlCdcJar, kafkaCdcJar, mysqlDriverJar); + submitPipelineJob(pipelineJob, kafkaCdcJar); waitUntilJobRunning(Duration.ofSeconds(30)); LOG.info("Pipeline job is running"); List> collectedRecords = new ArrayList<>(); @@ -380,7 +350,8 @@ public void testSyncWholeDatabaseWithDebeziumJsonHasSchema() throws Exception { private void waitUntilSpecificEventCount( List> actualEvent, int expectedCount) throws Exception { boolean result = false; - long endTimeout = System.currentTimeMillis() + MysqlToKafkaE2eITCase.EVENT_WAITING_TIMEOUT; + long endTimeout = + System.currentTimeMillis() + MysqlToKafkaE2eITCase.EVENT_WAITING_TIMEOUT.toMillis(); while (System.currentTimeMillis() < endTimeout) { ConsumerRecords records = consumer.poll(Duration.ofSeconds(1)); records.forEach(actualEvent::add); diff --git a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/OceanBaseE2eITCase.java b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/OceanBaseE2eITCase.java index 62b232e6175..2bebedb3bbe 100644 --- a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/OceanBaseE2eITCase.java +++ b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/OceanBaseE2eITCase.java @@ -18,8 +18,6 @@ package org.apache.flink.cdc.pipeline.tests; 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.cdc.connectors.mysql.testutils.UniqueDatabase; import org.apache.flink.cdc.connectors.oceanbase.OceanBaseTestUtils; import org.apache.flink.cdc.connectors.oceanbase.testutils.OceanBaseContainer; @@ -56,31 +54,10 @@ class OceanBaseE2eITCase extends PipelineTestEnvironment { private static final Logger LOG = LoggerFactory.getLogger(OceanBaseE2eITCase.class); - // ------------------------------------------------------------------------------------------ - // 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"; - private static final String MYSQL_TEST_TABLE_NAME = "products"; @Container - static final MySqlContainer MYSQL = - (MySqlContainer) - new MySqlContainer( - MySqlVersion.V8_0) // v8 support both ARM and AMD architectures - .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)); - - @Container - static final OceanBaseContainer OB_SERVER = + private static final OceanBaseContainer OB_SERVER = OceanBaseTestUtils.createOceanBaseContainerForJdbc() .withNetwork(NETWORK) .withNetworkAliases("oceanbase") @@ -133,10 +110,8 @@ void testSyncWholeDatabase() throws Exception { getJdbcUrlInContainer("test", "oceanbase"), OB_SERVER.getUsername(), OB_SERVER.getPassword()); - Path mysqlCdcJar = TestUtils.getResource("mysql-cdc-pipeline-connector.jar"); Path oceanbaseCdcJar = TestUtils.getResource("oceanbase-cdc-pipeline-connector.jar"); - Path mysqlDriverJar = TestUtils.getResource("mysql-driver.jar"); - submitPipelineJob(pipelineJob, mysqlCdcJar, oceanbaseCdcJar, mysqlDriverJar); + submitPipelineJob(pipelineJob, oceanbaseCdcJar); waitUntilJobRunning(Duration.ofSeconds(30)); LOG.info("Pipeline job is running"); diff --git a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/RouteE2eITCase.java b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/RouteE2eITCase.java index 23b340cad4f..2514711c0d5 100644 --- a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/RouteE2eITCase.java +++ b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/RouteE2eITCase.java @@ -17,61 +17,37 @@ package org.apache.flink.cdc.pipeline.tests; -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.cdc.connectors.mysql.testutils.UniqueDatabase; import org.apache.flink.cdc.pipeline.tests.utils.PipelineTestEnvironment; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.testcontainers.containers.output.Slf4jLogConsumer; -import org.testcontainers.junit.jupiter.Container; -import java.nio.file.Path; import java.sql.Connection; import java.sql.DriverManager; import java.sql.SQLException; import java.sql.Statement; import java.time.Duration; -import java.util.concurrent.TimeoutException; +import java.util.function.Function; import java.util.stream.IntStream; /** E2e tests for routing features. */ class RouteE2eITCase extends PipelineTestEnvironment { private static final Logger LOG = LoggerFactory.getLogger(RouteE2eITCase.class); - // ------------------------------------------------------------------------------------------ - // 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 MYSQL_DRIVER_CLASS = "com.mysql.cj.jdbc.Driver"; - protected static final String INTER_CONTAINER_MYSQL_ALIAS = "mysql"; - protected static final long EVENT_DEFAULT_TIMEOUT = 60000L; protected static final int TEST_TABLE_NUMBER = 100; - @Container - public static final MySqlContainer MYSQL = - (MySqlContainer) - new MySqlContainer( - MySqlVersion.V8_0) // v8 support both ARM and AMD architectures - .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)); - protected final UniqueDatabase routeTestDatabase = new UniqueDatabase(MYSQL, "route_test", MYSQL_TEST_USER, MYSQL_TEST_PASSWORD); + private final Function routeDbNameFormatter = + (s) -> String.format(s, routeTestDatabase.getDatabaseName()); + protected final UniqueDatabase extremeRouteTestDatabase = new UniqueDatabase(MYSQL, "extreme_route_test", MYSQL_TEST_USER, MYSQL_TEST_PASSWORD); @@ -162,31 +138,14 @@ void testDefaultRoute() throws Exception { MYSQL_TEST_PASSWORD, routeTestDatabase.getDatabaseName(), parallelism); - Path mysqlCdcJar = TestUtils.getResource("mysql-cdc-pipeline-connector.jar"); - Path valuesCdcJar = TestUtils.getResource("values-cdc-pipeline-connector.jar"); - Path mysqlDriverJar = TestUtils.getResource("mysql-driver.jar"); - submitPipelineJob(pipelineJob, mysqlCdcJar, valuesCdcJar, mysqlDriverJar); + submitPipelineJob(pipelineJob); waitUntilJobRunning(Duration.ofSeconds(30)); - LOG.info("Pipeline job is running"); - - waitUntilSpecificEvent( - String.format( - "CreateTableEvent{tableId=%s.TABLEALPHA, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17)}, primaryKeys=ID, options=()}", - routeTestDatabase.getDatabaseName())); - waitUntilSpecificEvent( - String.format( - "CreateTableEvent{tableId=%s.TABLEBETA, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17)}, primaryKeys=ID, options=()}", - routeTestDatabase.getDatabaseName())); - waitUntilSpecificEvent( - String.format( - "CreateTableEvent{tableId=%s.TABLEGAMMA, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17)}, primaryKeys=ID, options=()}", - routeTestDatabase.getDatabaseName())); - waitUntilSpecificEvent( - String.format( - "CreateTableEvent{tableId=%s.TABLEDELTA, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17)}, primaryKeys=ID, options=()}", - routeTestDatabase.getDatabaseName())); - validateResult( + routeDbNameFormatter, + "CreateTableEvent{tableId=%s.TABLEALPHA, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17)}, primaryKeys=ID, options=()}", + "CreateTableEvent{tableId=%s.TABLEBETA, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17)}, primaryKeys=ID, options=()}", + "CreateTableEvent{tableId=%s.TABLEGAMMA, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17)}, primaryKeys=ID, options=()}", + "CreateTableEvent{tableId=%s.TABLEDELTA, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17)}, primaryKeys=ID, options=()}", "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[1008, 8], op=INSERT, meta=()}", "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[1009, 8.1], op=INSERT, meta=()}", "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[1010, 10], op=INSERT, meta=()}", @@ -206,20 +165,17 @@ void testDefaultRoute() throws Exception { "DataChangeEvent{tableId=%s.TABLEDELTA, before=[], after=[4023, Mojave], op=INSERT, meta=()}", "DataChangeEvent{tableId=%s.TABLEDELTA, before=[], after=[4024, Catalina], op=INSERT, meta=()}"); - LOG.info("Begin incremental reading stage."); - - // generate binlogs generateIncrementalChanges(); - validateResult( + routeDbNameFormatter, "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[3007, 7], op=INSERT, meta=()}", "DataChangeEvent{tableId=%s.TABLEBETA, before=[2014, 14], after=[2014, 2014], op=UPDATE, meta=()}", "DataChangeEvent{tableId=%s.TABLEGAMMA, before=[], after=[3019, Emerald], op=INSERT, meta=()}", "DataChangeEvent{tableId=%s.TABLEDELTA, before=[4024, Catalina], after=[], op=DELETE, meta=()}"); generateSchemaChanges(); - validateResult( + routeDbNameFormatter, "AddColumnEvent{tableId=%s.TABLEALPHA, addedColumns=[ColumnWithPosition{column=`NAME` VARCHAR(17), position=LAST, existedColumnName=null}]}", "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[10001, 12, Derrida], op=INSERT, meta=()}", "RenameColumnEvent{tableId=%s.TABLEBETA, nameMapping={VERSION=VERSION_EX}}", @@ -256,31 +212,17 @@ void testDefaultRouteInBatchMode() throws Exception { MYSQL_TEST_PASSWORD, routeTestDatabase.getDatabaseName(), parallelism); - Path mysqlCdcJar = TestUtils.getResource("mysql-cdc-pipeline-connector.jar"); - Path valuesCdcJar = TestUtils.getResource("values-cdc-pipeline-connector.jar"); - Path mysqlDriverJar = TestUtils.getResource("mysql-driver.jar"); - submitPipelineJob(pipelineJob, mysqlCdcJar, valuesCdcJar, mysqlDriverJar); + submitPipelineJob(pipelineJob); waitUntilJobRunning(Duration.ofSeconds(30)); - LOG.info("Pipeline job is running"); - - waitUntilSpecificEvent( - String.format( - "CreateTableEvent{tableId=%s.TABLEALPHA, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17)}, primaryKeys=ID, options=()}", - routeTestDatabase.getDatabaseName())); - waitUntilSpecificEvent( - String.format( - "CreateTableEvent{tableId=%s.TABLEBETA, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17)}, primaryKeys=ID, options=()}", - routeTestDatabase.getDatabaseName())); - waitUntilSpecificEvent( - String.format( - "CreateTableEvent{tableId=%s.TABLEGAMMA, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17)}, primaryKeys=ID, options=()}", - routeTestDatabase.getDatabaseName())); - waitUntilSpecificEvent( - String.format( - "CreateTableEvent{tableId=%s.TABLEDELTA, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17)}, primaryKeys=ID, options=()}", - routeTestDatabase.getDatabaseName())); + validateResult( + routeDbNameFormatter, + "CreateTableEvent{tableId=%s.TABLEALPHA, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17)}, primaryKeys=ID, options=()}", + "CreateTableEvent{tableId=%s.TABLEBETA, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17)}, primaryKeys=ID, options=()}", + "CreateTableEvent{tableId=%s.TABLEGAMMA, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17)}, primaryKeys=ID, options=()}", + "CreateTableEvent{tableId=%s.TABLEDELTA, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17)}, primaryKeys=ID, options=()}"); validateResult( + routeDbNameFormatter, "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[1008, 8], op=INSERT, meta=()}", "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[1009, 8.1], op=INSERT, meta=()}", "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[1010, 10], op=INSERT, meta=()}", @@ -330,19 +272,11 @@ void testMergeTableRoute() throws Exception { routeTestDatabase.getDatabaseName(), routeTestDatabase.getDatabaseName(), parallelism); - Path mysqlCdcJar = TestUtils.getResource("mysql-cdc-pipeline-connector.jar"); - Path valuesCdcJar = TestUtils.getResource("values-cdc-pipeline-connector.jar"); - Path mysqlDriverJar = TestUtils.getResource("mysql-driver.jar"); - submitPipelineJob(pipelineJob, mysqlCdcJar, valuesCdcJar, mysqlDriverJar); + submitPipelineJob(pipelineJob); waitUntilJobRunning(Duration.ofSeconds(30)); - LOG.info("Pipeline job is running"); - - waitUntilSpecificEvent( - String.format( - "CreateTableEvent{tableId=%s.ALL, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17)}, primaryKeys=ID, options=()}", - routeTestDatabase.getDatabaseName())); - validateResult( + routeDbNameFormatter, + "CreateTableEvent{tableId=%s.ALL, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17)}, primaryKeys=ID, options=()}", "DataChangeEvent{tableId=%s.ALL, before=[], after=[1008, 8], op=INSERT, meta=()}", "DataChangeEvent{tableId=%s.ALL, before=[], after=[1009, 8.1], op=INSERT, meta=()}", "DataChangeEvent{tableId=%s.ALL, before=[], after=[1010, 10], op=INSERT, meta=()}", @@ -362,20 +296,17 @@ void testMergeTableRoute() throws Exception { "DataChangeEvent{tableId=%s.ALL, before=[], after=[4023, Mojave], op=INSERT, meta=()}", "DataChangeEvent{tableId=%s.ALL, before=[], after=[4024, Catalina], op=INSERT, meta=()}"); - LOG.info("Begin incremental reading stage."); - - // generate binlogs generateIncrementalChanges(); - validateResult( + routeDbNameFormatter, "DataChangeEvent{tableId=%s.ALL, before=[], after=[3007, 7], op=INSERT, meta=()}", "DataChangeEvent{tableId=%s.ALL, before=[2014, 14], after=[2014, 2014], op=UPDATE, meta=()}", "DataChangeEvent{tableId=%s.ALL, before=[], after=[3019, Emerald], op=INSERT, meta=()}", "DataChangeEvent{tableId=%s.ALL, before=[4024, Catalina], after=[], op=DELETE, meta=()}"); generateSchemaChanges(); - validateResult( + routeDbNameFormatter, "AddColumnEvent{tableId=%s.ALL, addedColumns=[ColumnWithPosition{column=`NAME` VARCHAR(17), position=AFTER, existedColumnName=VERSION}]}", "DataChangeEvent{tableId=%s.ALL, before=[], after=[10001, 12, Derrida], op=INSERT, meta=()}", "AddColumnEvent{tableId=%s.ALL, addedColumns=[ColumnWithPosition{column=`VERSION_EX` VARCHAR(17), position=AFTER, existedColumnName=NAME}]}", @@ -416,19 +347,11 @@ void testMergeTableRouteInBatchMode() throws Exception { routeTestDatabase.getDatabaseName(), routeTestDatabase.getDatabaseName(), parallelism); - Path mysqlCdcJar = TestUtils.getResource("mysql-cdc-pipeline-connector.jar"); - Path valuesCdcJar = TestUtils.getResource("values-cdc-pipeline-connector.jar"); - Path mysqlDriverJar = TestUtils.getResource("mysql-driver.jar"); - submitPipelineJob(pipelineJob, mysqlCdcJar, valuesCdcJar, mysqlDriverJar); + submitPipelineJob(pipelineJob); waitUntilJobRunning(Duration.ofSeconds(30)); - LOG.info("Pipeline job is running"); - - waitUntilSpecificEvent( - String.format( - "CreateTableEvent{tableId=%s.ALL, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17)}, primaryKeys=ID, options=()}", - routeTestDatabase.getDatabaseName())); - validateResult( + routeDbNameFormatter, + "CreateTableEvent{tableId=%s.ALL, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17)}, primaryKeys=ID, options=()}", "DataChangeEvent{tableId=%s.ALL, before=[], after=[1008, 8], op=INSERT, meta=()}", "DataChangeEvent{tableId=%s.ALL, before=[], after=[1009, 8.1], op=INSERT, meta=()}", "DataChangeEvent{tableId=%s.ALL, before=[], after=[1010, 10], op=INSERT, meta=()}", @@ -478,29 +401,13 @@ void testPartialRoute() throws Exception { routeTestDatabase.getDatabaseName(), routeTestDatabase.getDatabaseName(), parallelism); - Path mysqlCdcJar = TestUtils.getResource("mysql-cdc-pipeline-connector.jar"); - Path valuesCdcJar = TestUtils.getResource("values-cdc-pipeline-connector.jar"); - Path mysqlDriverJar = TestUtils.getResource("mysql-driver.jar"); - submitPipelineJob(pipelineJob, mysqlCdcJar, valuesCdcJar, mysqlDriverJar); + submitPipelineJob(pipelineJob); waitUntilJobRunning(Duration.ofSeconds(30)); - LOG.info("Pipeline job is running"); - - waitUntilSpecificEvent( - String.format( - "CreateTableEvent{tableId=NEW_%s.ALPHABET, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17)}, primaryKeys=ID, options=()}", - routeTestDatabase.getDatabaseName())); - - waitUntilSpecificEvent( - String.format( - "CreateTableEvent{tableId=%s.TABLEGAMMA, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17)}, primaryKeys=ID, options=()}", - routeTestDatabase.getDatabaseName())); - - waitUntilSpecificEvent( - String.format( - "CreateTableEvent{tableId=%s.TABLEDELTA, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17)}, primaryKeys=ID, options=()}", - routeTestDatabase.getDatabaseName())); - validateResult( + routeDbNameFormatter, + "CreateTableEvent{tableId=NEW_%s.ALPHABET, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17)}, primaryKeys=ID, options=()}", + "CreateTableEvent{tableId=%s.TABLEGAMMA, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17)}, primaryKeys=ID, options=()}", + "CreateTableEvent{tableId=%s.TABLEDELTA, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17)}, primaryKeys=ID, options=()}", "DataChangeEvent{tableId=NEW_%s.ALPHABET, before=[], after=[1008, 8], op=INSERT, meta=()}", "DataChangeEvent{tableId=NEW_%s.ALPHABET, before=[], after=[1009, 8.1], op=INSERT, meta=()}", "DataChangeEvent{tableId=NEW_%s.ALPHABET, before=[], after=[1010, 10], op=INSERT, meta=()}", @@ -519,21 +426,17 @@ void testPartialRoute() throws Exception { "DataChangeEvent{tableId=%s.TABLEDELTA, before=[], after=[4022, High Sierra], op=INSERT, meta=()}", "DataChangeEvent{tableId=%s.TABLEDELTA, before=[], after=[4023, Mojave], op=INSERT, meta=()}", "DataChangeEvent{tableId=%s.TABLEDELTA, before=[], after=[4024, Catalina], op=INSERT, meta=()}"); - - LOG.info("Begin incremental reading stage."); - - // generate binlogs generateIncrementalChanges(); - validateResult( + routeDbNameFormatter, "DataChangeEvent{tableId=NEW_%s.ALPHABET, before=[], after=[3007, 7], op=INSERT, meta=()}", "DataChangeEvent{tableId=NEW_%s.ALPHABET, before=[2014, 14], after=[2014, 2014], op=UPDATE, meta=()}", "DataChangeEvent{tableId=%s.TABLEGAMMA, before=[], after=[3019, Emerald], op=INSERT, meta=()}", "DataChangeEvent{tableId=%s.TABLEDELTA, before=[4024, Catalina], after=[], op=DELETE, meta=()}"); generateSchemaChanges(); - validateResult( + routeDbNameFormatter, "AddColumnEvent{tableId=NEW_%s.ALPHABET, addedColumns=[ColumnWithPosition{column=`NAME` VARCHAR(17), position=AFTER, existedColumnName=VERSION}]}", "DataChangeEvent{tableId=NEW_%s.ALPHABET, before=[], after=[10001, 12, Derrida], op=INSERT, meta=()}", "AddColumnEvent{tableId=NEW_%s.ALPHABET, addedColumns=[ColumnWithPosition{column=`VERSION_EX` VARCHAR(17), position=AFTER, existedColumnName=NAME}]}", @@ -576,29 +479,14 @@ void testPartialRouteInBatchMode() throws Exception { routeTestDatabase.getDatabaseName(), routeTestDatabase.getDatabaseName(), parallelism); - Path mysqlCdcJar = TestUtils.getResource("mysql-cdc-pipeline-connector.jar"); - Path valuesCdcJar = TestUtils.getResource("values-cdc-pipeline-connector.jar"); - Path mysqlDriverJar = TestUtils.getResource("mysql-driver.jar"); - submitPipelineJob(pipelineJob, mysqlCdcJar, valuesCdcJar, mysqlDriverJar); + submitPipelineJob(pipelineJob); waitUntilJobRunning(Duration.ofSeconds(30)); - LOG.info("Pipeline job is running"); - - waitUntilSpecificEvent( - String.format( - "CreateTableEvent{tableId=NEW_%s.ALPHABET, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17)}, primaryKeys=ID, options=()}", - routeTestDatabase.getDatabaseName())); - - waitUntilSpecificEvent( - String.format( - "CreateTableEvent{tableId=%s.TABLEGAMMA, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17)}, primaryKeys=ID, options=()}", - routeTestDatabase.getDatabaseName())); - - waitUntilSpecificEvent( - String.format( - "CreateTableEvent{tableId=%s.TABLEDELTA, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17)}, primaryKeys=ID, options=()}", - routeTestDatabase.getDatabaseName())); validateResult( + routeDbNameFormatter, + "CreateTableEvent{tableId=NEW_%s.ALPHABET, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17)}, primaryKeys=ID, options=()}", + "CreateTableEvent{tableId=%s.TABLEGAMMA, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17)}, primaryKeys=ID, options=()}", + "CreateTableEvent{tableId=%s.TABLEDELTA, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17)}, primaryKeys=ID, options=()}", "DataChangeEvent{tableId=NEW_%s.ALPHABET, before=[], after=[1008, 8], op=INSERT, meta=()}", "DataChangeEvent{tableId=NEW_%s.ALPHABET, before=[], after=[1009, 8.1], op=INSERT, meta=()}", "DataChangeEvent{tableId=NEW_%s.ALPHABET, before=[], after=[1010, 10], op=INSERT, meta=()}", @@ -652,29 +540,13 @@ void testMultipleRoute() throws Exception { routeTestDatabase.getDatabaseName(), routeTestDatabase.getDatabaseName(), parallelism); - Path mysqlCdcJar = TestUtils.getResource("mysql-cdc-pipeline-connector.jar"); - Path valuesCdcJar = TestUtils.getResource("values-cdc-pipeline-connector.jar"); - Path mysqlDriverJar = TestUtils.getResource("mysql-driver.jar"); - submitPipelineJob(pipelineJob, mysqlCdcJar, valuesCdcJar, mysqlDriverJar); + submitPipelineJob(pipelineJob); waitUntilJobRunning(Duration.ofSeconds(30)); - LOG.info("Pipeline job is running"); - - waitUntilSpecificEvent( - String.format( - "CreateTableEvent{tableId=NEW_%s.ALPHABET, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17)}, primaryKeys=ID, options=()}", - routeTestDatabase.getDatabaseName())); - - waitUntilSpecificEvent( - String.format( - "CreateTableEvent{tableId=NEW_%s.BETAGAMM, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17)}, primaryKeys=ID, options=()}", - routeTestDatabase.getDatabaseName())); - - waitUntilSpecificEvent( - String.format( - "CreateTableEvent{tableId=%s.TABLEDELTA, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17)}, primaryKeys=ID, options=()}", - routeTestDatabase.getDatabaseName())); - validateResult( + routeDbNameFormatter, + "CreateTableEvent{tableId=NEW_%s.ALPHABET, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17)}, primaryKeys=ID, options=()}", + "CreateTableEvent{tableId=NEW_%s.BETAGAMM, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17)}, primaryKeys=ID, options=()}", + "CreateTableEvent{tableId=%s.TABLEDELTA, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17)}, primaryKeys=ID, options=()}", "DataChangeEvent{tableId=NEW_%s.ALPHABET, before=[], after=[1008, 8], op=INSERT, meta=()}", "DataChangeEvent{tableId=NEW_%s.ALPHABET, before=[], after=[1009, 8.1], op=INSERT, meta=()}", "DataChangeEvent{tableId=NEW_%s.ALPHABET, before=[], after=[1010, 10], op=INSERT, meta=()}", @@ -698,12 +570,9 @@ void testMultipleRoute() throws Exception { "DataChangeEvent{tableId=%s.TABLEDELTA, before=[], after=[4023, Mojave], op=INSERT, meta=()}", "DataChangeEvent{tableId=%s.TABLEDELTA, before=[], after=[4024, Catalina], op=INSERT, meta=()}"); - LOG.info("Begin incremental reading stage."); - - // generate binlogs generateIncrementalChanges(); - validateResult( + routeDbNameFormatter, "DataChangeEvent{tableId=NEW_%s.ALPHABET, before=[], after=[3007, 7], op=INSERT, meta=()}", "DataChangeEvent{tableId=NEW_%s.ALPHABET, before=[2014, 14], after=[2014, 2014], op=UPDATE, meta=()}", "DataChangeEvent{tableId=NEW_%s.BETAGAMM, before=[2014, 14], after=[2014, 2014], op=UPDATE, meta=()}", @@ -711,8 +580,8 @@ void testMultipleRoute() throws Exception { "DataChangeEvent{tableId=%s.TABLEDELTA, before=[4024, Catalina], after=[], op=DELETE, meta=()}"); generateSchemaChanges(); - validateResult( + routeDbNameFormatter, "AddColumnEvent{tableId=NEW_%s.ALPHABET, addedColumns=[ColumnWithPosition{column=`NAME` VARCHAR(17), position=AFTER, existedColumnName=VERSION}]}", "DataChangeEvent{tableId=NEW_%s.ALPHABET, before=[], after=[10001, 12, Derrida], op=INSERT, meta=()}", "AddColumnEvent{tableId=NEW_%s.ALPHABET, addedColumns=[ColumnWithPosition{column=`VERSION_EX` VARCHAR(17), position=AFTER, existedColumnName=NAME}]}", @@ -760,29 +629,14 @@ void testMultipleRouteInBatchMode() throws Exception { routeTestDatabase.getDatabaseName(), routeTestDatabase.getDatabaseName(), parallelism); - Path mysqlCdcJar = TestUtils.getResource("mysql-cdc-pipeline-connector.jar"); - Path valuesCdcJar = TestUtils.getResource("values-cdc-pipeline-connector.jar"); - Path mysqlDriverJar = TestUtils.getResource("mysql-driver.jar"); - submitPipelineJob(pipelineJob, mysqlCdcJar, valuesCdcJar, mysqlDriverJar); + submitPipelineJob(pipelineJob); waitUntilJobRunning(Duration.ofSeconds(30)); - LOG.info("Pipeline job is running"); - - waitUntilSpecificEvent( - String.format( - "CreateTableEvent{tableId=NEW_%s.ALPHABET, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17)}, primaryKeys=ID, options=()}", - routeTestDatabase.getDatabaseName())); - - waitUntilSpecificEvent( - String.format( - "CreateTableEvent{tableId=NEW_%s.BETAGAMM, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17)}, primaryKeys=ID, options=()}", - routeTestDatabase.getDatabaseName())); - - waitUntilSpecificEvent( - String.format( - "CreateTableEvent{tableId=%s.TABLEDELTA, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17)}, primaryKeys=ID, options=()}", - routeTestDatabase.getDatabaseName())); validateResult( + routeDbNameFormatter, + "CreateTableEvent{tableId=NEW_%s.ALPHABET, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17)}, primaryKeys=ID, options=()}", + "CreateTableEvent{tableId=NEW_%s.BETAGAMM, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17)}, primaryKeys=ID, options=()}", + "CreateTableEvent{tableId=%s.TABLEDELTA, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17)}, primaryKeys=ID, options=()}", "DataChangeEvent{tableId=NEW_%s.ALPHABET, before=[], after=[1008, 8], op=INSERT, meta=()}", "DataChangeEvent{tableId=NEW_%s.ALPHABET, before=[], after=[1009, 8.1], op=INSERT, meta=()}", "DataChangeEvent{tableId=NEW_%s.ALPHABET, before=[], after=[1010, 10], op=INSERT, meta=()}", @@ -844,29 +698,13 @@ void testOneToManyRoute() throws Exception { routeTestDatabase.getDatabaseName(), routeTestDatabase.getDatabaseName(), parallelism); - Path mysqlCdcJar = TestUtils.getResource("mysql-cdc-pipeline-connector.jar"); - Path valuesCdcJar = TestUtils.getResource("values-cdc-pipeline-connector.jar"); - Path mysqlDriverJar = TestUtils.getResource("mysql-driver.jar"); - submitPipelineJob(pipelineJob, mysqlCdcJar, valuesCdcJar, mysqlDriverJar); + submitPipelineJob(pipelineJob); waitUntilJobRunning(Duration.ofSeconds(30)); - LOG.info("Pipeline job is running"); - - waitUntilSpecificEvent( - String.format( - "CreateTableEvent{tableId=NEW_%s.TABLEA, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17)}, primaryKeys=ID, options=()}", - routeTestDatabase.getDatabaseName())); - - waitUntilSpecificEvent( - String.format( - "CreateTableEvent{tableId=NEW_%s.TABLEB, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17)}, primaryKeys=ID, options=()}", - routeTestDatabase.getDatabaseName())); - - waitUntilSpecificEvent( - String.format( - "CreateTableEvent{tableId=NEW_%s.TABLEC, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17)}, primaryKeys=ID, options=()}", - routeTestDatabase.getDatabaseName())); - validateResult( + routeDbNameFormatter, + "CreateTableEvent{tableId=NEW_%s.TABLEA, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17)}, primaryKeys=ID, options=()}", + "CreateTableEvent{tableId=NEW_%s.TABLEB, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17)}, primaryKeys=ID, options=()}", + "CreateTableEvent{tableId=NEW_%s.TABLEC, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17)}, primaryKeys=ID, options=()}", "DataChangeEvent{tableId=NEW_%s.TABLEA, before=[], after=[1008, 8], op=INSERT, meta=()}", "DataChangeEvent{tableId=NEW_%s.TABLEA, before=[], after=[1009, 8.1], op=INSERT, meta=()}", "DataChangeEvent{tableId=NEW_%s.TABLEA, before=[], after=[1010, 10], op=INSERT, meta=()}", @@ -880,8 +718,6 @@ void testOneToManyRoute() throws Exception { "DataChangeEvent{tableId=NEW_%s.TABLEC, before=[], after=[1010, 10], op=INSERT, meta=()}", "DataChangeEvent{tableId=NEW_%s.TABLEC, before=[], after=[1011, 11], op=INSERT, meta=()}"); - LOG.info("Begin incremental reading stage."); - // generate binlogs String mysqlJdbcUrl = String.format( "jdbc:mysql://%s:%s/%s", @@ -899,13 +735,14 @@ void testOneToManyRoute() throws Exception { } validateResult( + routeDbNameFormatter, "DataChangeEvent{tableId=NEW_%s.TABLEA, before=[], after=[3007, 7], op=INSERT, meta=()}", "DataChangeEvent{tableId=NEW_%s.TABLEB, before=[], after=[3007, 7], op=INSERT, meta=()}", "DataChangeEvent{tableId=NEW_%s.TABLEC, before=[], after=[3007, 7], op=INSERT, meta=()}"); generateSchemaChanges(); - validateResult( + routeDbNameFormatter, "AddColumnEvent{tableId=NEW_%s.TABLEA, addedColumns=[ColumnWithPosition{column=`NAME` VARCHAR(17), position=LAST, existedColumnName=null}]}", "AddColumnEvent{tableId=NEW_%s.TABLEB, addedColumns=[ColumnWithPosition{column=`NAME` VARCHAR(17), position=LAST, existedColumnName=null}]}", "AddColumnEvent{tableId=NEW_%s.TABLEC, addedColumns=[ColumnWithPosition{column=`NAME` VARCHAR(17), position=LAST, existedColumnName=null}]}", @@ -960,29 +797,17 @@ void testOneToManyRouteInBatchMode() throws Exception { routeTestDatabase.getDatabaseName(), routeTestDatabase.getDatabaseName(), parallelism); - Path mysqlCdcJar = TestUtils.getResource("mysql-cdc-pipeline-connector.jar"); - Path valuesCdcJar = TestUtils.getResource("values-cdc-pipeline-connector.jar"); - Path mysqlDriverJar = TestUtils.getResource("mysql-driver.jar"); - submitPipelineJob(pipelineJob, mysqlCdcJar, valuesCdcJar, mysqlDriverJar); + submitPipelineJob(pipelineJob); waitUntilJobRunning(Duration.ofSeconds(30)); - LOG.info("Pipeline job is running"); - - waitUntilSpecificEvent( - String.format( - "CreateTableEvent{tableId=NEW_%s.TABLEA, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17)}, primaryKeys=ID, options=()}", - routeTestDatabase.getDatabaseName())); - waitUntilSpecificEvent( - String.format( - "CreateTableEvent{tableId=NEW_%s.TABLEB, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17)}, primaryKeys=ID, options=()}", - routeTestDatabase.getDatabaseName())); - - waitUntilSpecificEvent( - String.format( - "CreateTableEvent{tableId=NEW_%s.TABLEC, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17)}, primaryKeys=ID, options=()}", - routeTestDatabase.getDatabaseName())); + validateResult( + routeDbNameFormatter, + "CreateTableEvent{tableId=NEW_%s.TABLEA, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17)}, primaryKeys=ID, options=()}", + "CreateTableEvent{tableId=NEW_%s.TABLEB, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17)}, primaryKeys=ID, options=()}", + "CreateTableEvent{tableId=NEW_%s.TABLEC, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17)}, primaryKeys=ID, options=()}"); validateResult( + routeDbNameFormatter, "DataChangeEvent{tableId=NEW_%s.TABLEA, before=[], after=[1008, 8], op=INSERT, meta=()}", "DataChangeEvent{tableId=NEW_%s.TABLEA, before=[], after=[1009, 8.1], op=INSERT, meta=()}", "DataChangeEvent{tableId=NEW_%s.TABLEA, before=[], after=[1010, 10], op=INSERT, meta=()}", @@ -1030,19 +855,11 @@ void testMergeTableRouteWithTransform() throws Exception { routeTestDatabase.getDatabaseName(), routeTestDatabase.getDatabaseName(), parallelism); - Path mysqlCdcJar = TestUtils.getResource("mysql-cdc-pipeline-connector.jar"); - Path valuesCdcJar = TestUtils.getResource("values-cdc-pipeline-connector.jar"); - Path mysqlDriverJar = TestUtils.getResource("mysql-driver.jar"); - submitPipelineJob(pipelineJob, mysqlCdcJar, valuesCdcJar, mysqlDriverJar); + submitPipelineJob(pipelineJob); waitUntilJobRunning(Duration.ofSeconds(30)); - LOG.info("Pipeline job is running"); - - waitUntilSpecificEvent( - String.format( - "CreateTableEvent{tableId=%s.ALL, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17),`EXTRAS` STRING}, primaryKeys=ID, options=()}", - routeTestDatabase.getDatabaseName())); - validateResult( + routeDbNameFormatter, + "CreateTableEvent{tableId=%s.ALL, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17),`EXTRAS` STRING}, primaryKeys=ID, options=()}", "DataChangeEvent{tableId=%s.ALL, before=[], after=[1008, 8, extras], op=INSERT, meta=()}", "DataChangeEvent{tableId=%s.ALL, before=[], after=[1009, 8.1, extras], op=INSERT, meta=()}", "DataChangeEvent{tableId=%s.ALL, before=[], after=[1010, 10, extras], op=INSERT, meta=()}", @@ -1062,21 +879,17 @@ void testMergeTableRouteWithTransform() throws Exception { "DataChangeEvent{tableId=%s.ALL, before=[], after=[4023, Mojave, extras], op=INSERT, meta=()}", "DataChangeEvent{tableId=%s.ALL, before=[], after=[4024, Catalina, extras], op=INSERT, meta=()}"); - LOG.info("Begin incremental reading stage."); - - // generate binlogs generateIncrementalChanges(); - validateResult( + routeDbNameFormatter, "DataChangeEvent{tableId=%s.ALL, before=[], after=[3007, 7, extras], op=INSERT, meta=()}", "DataChangeEvent{tableId=%s.ALL, before=[2014, 14, extras], after=[2014, 2014, extras], op=UPDATE, meta=()}", "DataChangeEvent{tableId=%s.ALL, before=[], after=[3019, Emerald, extras], op=INSERT, meta=()}", "DataChangeEvent{tableId=%s.ALL, before=[4024, Catalina, extras], after=[], op=DELETE, meta=()}"); - LOG.info("Begin schema changing stage."); generateSchemaChanges(); - validateResult( + routeDbNameFormatter, "AddColumnEvent{tableId=%s.ALL, addedColumns=[ColumnWithPosition{column=`NAME` VARCHAR(17), position=AFTER, existedColumnName=EXTRAS}]}", "DataChangeEvent{tableId=%s.ALL, before=[], after=[10001, 12, extras, Derrida], op=INSERT, meta=()}", "AddColumnEvent{tableId=%s.ALL, addedColumns=[ColumnWithPosition{column=`VERSION_EX` VARCHAR(17), position=AFTER, existedColumnName=NAME}]}", @@ -1121,19 +934,12 @@ void testMergeTableRouteWithTransformInBatchMode() throws Exception { routeTestDatabase.getDatabaseName(), routeTestDatabase.getDatabaseName(), parallelism); - Path mysqlCdcJar = TestUtils.getResource("mysql-cdc-pipeline-connector.jar"); - Path valuesCdcJar = TestUtils.getResource("values-cdc-pipeline-connector.jar"); - Path mysqlDriverJar = TestUtils.getResource("mysql-driver.jar"); - submitPipelineJob(pipelineJob, mysqlCdcJar, valuesCdcJar, mysqlDriverJar); + submitPipelineJob(pipelineJob); waitUntilJobRunning(Duration.ofSeconds(30)); - LOG.info("Pipeline job is running"); - - waitUntilSpecificEvent( - String.format( - "CreateTableEvent{tableId=%s.ALL, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17),`EXTRAS` STRING}, primaryKeys=ID, options=()}", - routeTestDatabase.getDatabaseName())); validateResult( + routeDbNameFormatter, + "CreateTableEvent{tableId=%s.ALL, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17),`EXTRAS` STRING}, primaryKeys=ID, options=()}", "DataChangeEvent{tableId=%s.ALL, before=[], after=[1008, 8, extras], op=INSERT, meta=()}", "DataChangeEvent{tableId=%s.ALL, before=[], after=[1009, 8.1, extras], op=INSERT, meta=()}", "DataChangeEvent{tableId=%s.ALL, before=[], after=[1010, 10, extras], op=INSERT, meta=()}", @@ -1184,31 +990,15 @@ void testReplacementSymbol() throws Exception { routeTestDatabase.getDatabaseName(), routeTestDatabase.getDatabaseName(), parallelism); - Path mysqlCdcJar = TestUtils.getResource("mysql-cdc-pipeline-connector.jar"); - Path valuesCdcJar = TestUtils.getResource("values-cdc-pipeline-connector.jar"); - Path mysqlDriverJar = TestUtils.getResource("mysql-driver.jar"); - submitPipelineJob(pipelineJob, mysqlCdcJar, valuesCdcJar, mysqlDriverJar); + submitPipelineJob(pipelineJob); waitUntilJobRunning(Duration.ofSeconds(30)); - LOG.info("Pipeline job is running"); - - waitUntilSpecificEvent( - String.format( - "CreateTableEvent{tableId=NEW_%s.NEW_TABLEALPHA, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17)}, primaryKeys=ID, options=()}", - routeTestDatabase.getDatabaseName())); - waitUntilSpecificEvent( - String.format( - "CreateTableEvent{tableId=NEW_%s.NEW_TABLEBETA, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17)}, primaryKeys=ID, options=()}", - routeTestDatabase.getDatabaseName())); - waitUntilSpecificEvent( - String.format( - "CreateTableEvent{tableId=NEW_%s.NEW_TABLEGAMMA, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17)}, primaryKeys=ID, options=()}", - routeTestDatabase.getDatabaseName())); - waitUntilSpecificEvent( - String.format( - "CreateTableEvent{tableId=NEW_%s.NEW_TABLEDELTA, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17)}, primaryKeys=ID, options=()}", - routeTestDatabase.getDatabaseName())); validateResult( + routeDbNameFormatter, + "CreateTableEvent{tableId=NEW_%s.NEW_TABLEALPHA, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17)}, primaryKeys=ID, options=()}", + "CreateTableEvent{tableId=NEW_%s.NEW_TABLEBETA, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17)}, primaryKeys=ID, options=()}", + "CreateTableEvent{tableId=NEW_%s.NEW_TABLEGAMMA, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17)}, primaryKeys=ID, options=()}", + "CreateTableEvent{tableId=NEW_%s.NEW_TABLEDELTA, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17)}, primaryKeys=ID, options=()}", "DataChangeEvent{tableId=NEW_%s.NEW_TABLEALPHA, before=[], after=[1008, 8], op=INSERT, meta=()}", "DataChangeEvent{tableId=NEW_%s.NEW_TABLEALPHA, before=[], after=[1009, 8.1], op=INSERT, meta=()}", "DataChangeEvent{tableId=NEW_%s.NEW_TABLEALPHA, before=[], after=[1010, 10], op=INSERT, meta=()}", @@ -1228,11 +1018,10 @@ void testReplacementSymbol() throws Exception { "DataChangeEvent{tableId=NEW_%s.NEW_TABLEDELTA, before=[], after=[4023, Mojave], op=INSERT, meta=()}", "DataChangeEvent{tableId=NEW_%s.NEW_TABLEDELTA, before=[], after=[4024, Catalina], op=INSERT, meta=()}"); - LOG.info("Begin incremental reading stage."); - generateIncrementalChanges(); validateResult( + routeDbNameFormatter, "DataChangeEvent{tableId=NEW_%s.NEW_TABLEALPHA, before=[], after=[3007, 7], op=INSERT, meta=()}", "DataChangeEvent{tableId=NEW_%s.NEW_TABLEBETA, before=[2014, 14], after=[2014, 2014], op=UPDATE, meta=()}", "DataChangeEvent{tableId=NEW_%s.NEW_TABLEGAMMA, before=[], after=[3019, Emerald], op=INSERT, meta=()}", @@ -1240,6 +1029,7 @@ void testReplacementSymbol() throws Exception { generateSchemaChanges(); validateResult( + routeDbNameFormatter, "AddColumnEvent{tableId=NEW_%s.NEW_TABLEALPHA, addedColumns=[ColumnWithPosition{column=`NAME` VARCHAR(17), position=LAST, existedColumnName=null}]}", "DataChangeEvent{tableId=NEW_%s.NEW_TABLEALPHA, before=[], after=[10001, 12, Derrida], op=INSERT, meta=()}", "RenameColumnEvent{tableId=NEW_%s.NEW_TABLEBETA, nameMapping={VERSION=VERSION_EX}}", @@ -1283,31 +1073,18 @@ void testReplacementSymbolInBatchMode() throws Exception { routeTestDatabase.getDatabaseName(), routeTestDatabase.getDatabaseName(), parallelism); - Path mysqlCdcJar = TestUtils.getResource("mysql-cdc-pipeline-connector.jar"); - Path valuesCdcJar = TestUtils.getResource("values-cdc-pipeline-connector.jar"); - Path mysqlDriverJar = TestUtils.getResource("mysql-driver.jar"); - submitPipelineJob(pipelineJob, mysqlCdcJar, valuesCdcJar, mysqlDriverJar); + submitPipelineJob(pipelineJob); waitUntilJobRunning(Duration.ofSeconds(30)); - LOG.info("Pipeline job is running"); - waitUntilSpecificEvent( - String.format( - "CreateTableEvent{tableId=NEW_%s.NEW_TABLEALPHA, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17)}, primaryKeys=ID, options=()}", - routeTestDatabase.getDatabaseName())); - waitUntilSpecificEvent( - String.format( - "CreateTableEvent{tableId=NEW_%s.NEW_TABLEBETA, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17)}, primaryKeys=ID, options=()}", - routeTestDatabase.getDatabaseName())); - waitUntilSpecificEvent( - String.format( - "CreateTableEvent{tableId=NEW_%s.NEW_TABLEGAMMA, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17)}, primaryKeys=ID, options=()}", - routeTestDatabase.getDatabaseName())); - waitUntilSpecificEvent( - String.format( - "CreateTableEvent{tableId=NEW_%s.NEW_TABLEDELTA, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17)}, primaryKeys=ID, options=()}", - routeTestDatabase.getDatabaseName())); + validateResult( + routeDbNameFormatter, + "CreateTableEvent{tableId=NEW_%s.NEW_TABLEALPHA, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17)}, primaryKeys=ID, options=()}", + "CreateTableEvent{tableId=NEW_%s.NEW_TABLEBETA, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17)}, primaryKeys=ID, options=()}", + "CreateTableEvent{tableId=NEW_%s.NEW_TABLEGAMMA, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17)}, primaryKeys=ID, options=()}", + "CreateTableEvent{tableId=NEW_%s.NEW_TABLEDELTA, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17)}, primaryKeys=ID, options=()}"); validateResult( + routeDbNameFormatter, "DataChangeEvent{tableId=NEW_%s.NEW_TABLEALPHA, before=[], after=[1008, 8], op=INSERT, meta=()}", "DataChangeEvent{tableId=NEW_%s.NEW_TABLEALPHA, before=[], after=[1009, 8.1], op=INSERT, meta=()}", "DataChangeEvent{tableId=NEW_%s.NEW_TABLEALPHA, before=[], after=[1010, 10], op=INSERT, meta=()}", @@ -1328,92 +1105,9 @@ void testReplacementSymbolInBatchMode() throws Exception { "DataChangeEvent{tableId=NEW_%s.NEW_TABLEDELTA, before=[], after=[4024, Catalina], op=INSERT, meta=()}"); } - @Test - void testExtremeMergeTableRoute() throws Exception { - final String databaseName = extremeRouteTestDatabase.getDatabaseName(); - try (Connection conn = - DriverManager.getConnection( - MYSQL.getJdbcUrl(), MYSQL_TEST_USER, MYSQL_TEST_PASSWORD); - Statement stat = conn.createStatement()) { - stat.execute(String.format("CREATE DATABASE %s;", databaseName)); - stat.execute(String.format("USE %s;", databaseName)); - for (int i = 1; i <= TEST_TABLE_NUMBER; i++) { - stat.execute(String.format("DROP TABLE IF EXISTS TABLE%d;", i)); - stat.execute( - String.format( - "CREATE TABLE TABLE%d (ID INT NOT NULL PRIMARY KEY,VERSION VARCHAR(17));", - i)); - stat.execute(String.format("INSERT INTO TABLE%d VALUES (%d, 'No.%d');", i, i, i)); - } - } catch (SQLException e) { - LOG.error("Initialize table failed.", e); - throw e; - } - LOG.info("Table initialized successfully."); - - String pipelineJob = - String.format( - "source:\n" - + " type: mysql\n" - + " hostname: %s\n" - + " port: 3306\n" - + " username: %s\n" - + " password: %s\n" - + " tables: %s.\\.*\n" - + " server-id: 5400-5404\n" - + " server-time-zone: UTC\n" - + "\n" - + "sink:\n" - + " type: values\n" - + "\n" - + "pipeline:\n" - + " parallelism: %d", - INTER_CONTAINER_MYSQL_ALIAS, - MYSQL_TEST_USER, - MYSQL_TEST_PASSWORD, - databaseName, - parallelism); - Path mysqlCdcJar = TestUtils.getResource("mysql-cdc-pipeline-connector.jar"); - Path valuesCdcJar = TestUtils.getResource("values-cdc-pipeline-connector.jar"); - Path mysqlDriverJar = TestUtils.getResource("mysql-driver.jar"); - submitPipelineJob(pipelineJob, mysqlCdcJar, valuesCdcJar, mysqlDriverJar); - waitUntilJobRunning(Duration.ofSeconds(30)); - - // In single parallelism mode, sink will not print out the "subTaskId>" prefix. - String prefix = parallelism > 1 ? "> " : ""; - - LOG.info("Verifying CreateTableEvents..."); - validateResult( - 180_000L, - IntStream.rangeClosed(1, TEST_TABLE_NUMBER) - .mapToObj( - i -> - String.format( - prefix - + "CreateTableEvent{tableId=%s.TABLE%d, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17)}, primaryKeys=ID, options=()}", - databaseName, - i)) - .toArray(String[]::new)); - - LOG.info("Verifying DataChangeEvents..."); - validateResult( - 180_000L, - IntStream.rangeClosed(1, TEST_TABLE_NUMBER) - .mapToObj( - i -> - String.format( - prefix - + "DataChangeEvent{tableId=%s.TABLE%d, before=[], after=[%d, No.%d], op=INSERT, meta=()}", - databaseName, - i, - i, - i)) - .toArray(String[]::new)); - extremeRouteTestDatabase.dropDatabase(); - } - - @Test - public void testExtremeMergeTableRouteInBatchMode() throws Exception { + @ParameterizedTest(name = "batchMode: {0}") + @ValueSource(booleans = {true, false}) + void testExtremeMergeTableRoute(boolean batchMode) throws Exception { final String databaseName = extremeRouteTestDatabase.getDatabaseName(); try (Connection conn = DriverManager.getConnection( @@ -1433,7 +1127,6 @@ public void testExtremeMergeTableRouteInBatchMode() throws Exception { LOG.error("Initialize table failed.", e); throw e; } - LOG.info("Table initialized successfully."); String pipelineJob = String.format( @@ -1446,31 +1139,25 @@ public void testExtremeMergeTableRouteInBatchMode() throws Exception { + " tables: %s.\\.*\n" + " server-id: 5400-5404\n" + " server-time-zone: UTC\n" - + " scan.startup.mode: snapshot\n" + + " scan.startup.mode: %s\n" + "\n" + "sink:\n" + " type: values\n" + "\n" + "pipeline:\n" + " parallelism: %d\n" - + " execution.runtime-mode: BATCH", + + (batchMode ? " execution.runtime-mode: BATCH" : ""), INTER_CONTAINER_MYSQL_ALIAS, MYSQL_TEST_USER, MYSQL_TEST_PASSWORD, databaseName, + batchMode ? "snapshot" : "initial", parallelism); - Path mysqlCdcJar = TestUtils.getResource("mysql-cdc-pipeline-connector.jar"); - Path valuesCdcJar = TestUtils.getResource("values-cdc-pipeline-connector.jar"); - Path mysqlDriverJar = TestUtils.getResource("mysql-driver.jar"); - submitPipelineJob(pipelineJob, mysqlCdcJar, valuesCdcJar, mysqlDriverJar); + submitPipelineJob(pipelineJob); waitUntilJobRunning(Duration.ofSeconds(30)); - // In single parallelism mode, sink will not print out the "subTaskId>" prefix. String prefix = parallelism > 1 ? "> " : ""; - - LOG.info("Verifying CreateTableEvents..."); validateResult( - 180_000L, IntStream.rangeClosed(1, TEST_TABLE_NUMBER) .mapToObj( i -> @@ -1480,10 +1167,7 @@ public void testExtremeMergeTableRouteInBatchMode() throws Exception { databaseName, i)) .toArray(String[]::new)); - - LOG.info("Verifying DataChangeEvents..."); validateResult( - 180_000L, IntStream.rangeClosed(1, TEST_TABLE_NUMBER) .mapToObj( i -> @@ -1497,43 +1181,4 @@ public void testExtremeMergeTableRouteInBatchMode() throws Exception { .toArray(String[]::new)); extremeRouteTestDatabase.dropDatabase(); } - - private void validateResult(String... expectedEvents) throws Exception { - validateResult(EVENT_DEFAULT_TIMEOUT, expectedEvents); - } - - private void validateResult(long timeout, String... expectedEvents) throws Exception { - for (String event : expectedEvents) { - waitUntilSpecificEvent( - timeout, String.format(event, routeTestDatabase.getDatabaseName())); - } - } - - private void waitUntilSpecificEvent(String event) throws Exception { - waitUntilSpecificEvent(EVENT_DEFAULT_TIMEOUT, event); - } - - private void waitUntilSpecificEvent(long timeout, String event) throws Exception { - boolean result = false; - long endTimeout = System.currentTimeMillis() + timeout; - while (System.currentTimeMillis() < endTimeout) { - String stdout = taskManagerConsumer.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: " - + taskManagerConsumer.toUtf8String()); - } - } - - private void assertNotExists(String event) { - Assertions.assertThat(taskManagerConsumer.toUtf8String()).doesNotContain(event); - } } diff --git a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/SchemaEvolveE2eITCase.java b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/SchemaEvolveE2eITCase.java index 8fc16d23ea9..a6ade384ca5 100644 --- a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/SchemaEvolveE2eITCase.java +++ b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/SchemaEvolveE2eITCase.java @@ -17,9 +17,6 @@ package org.apache.flink.cdc.pipeline.tests; -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.cdc.connectors.mysql.testutils.UniqueDatabase; import org.apache.flink.cdc.pipeline.tests.utils.PipelineTestEnvironment; @@ -29,11 +26,7 @@ import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.testcontainers.containers.output.Slf4jLogConsumer; -import org.testcontainers.containers.output.ToStringConsumer; -import org.testcontainers.junit.jupiter.Container; -import java.nio.file.Path; import java.sql.Connection; import java.sql.DriverManager; import java.sql.Statement; @@ -41,36 +34,11 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; -import java.util.concurrent.TimeoutException; -import java.util.stream.Collectors; -import java.util.stream.Stream; /** E2e tests for Schema Evolution cases. */ class SchemaEvolveE2eITCase extends PipelineTestEnvironment { private static final Logger LOG = LoggerFactory.getLogger(SchemaEvolveE2eITCase.class); - // ------------------------------------------------------------------------------------------ - // 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 long EVENT_WAITING_TIMEOUT = 60000L; - - @Container - public static final MySqlContainer MYSQL = - (MySqlContainer) - new MySqlContainer( - MySqlVersion.V8_0) // v8 support both ARM and AMD architectures - .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)); - protected final UniqueDatabase schemaEvolveDatabase = new UniqueDatabase(MYSQL, "schema_evolve", MYSQL_TEST_USER, MYSQL_TEST_PASSWORD); @@ -203,10 +171,11 @@ void testLenientSchemaEvolution() throws Exception { "DataChangeEvent{tableId=%s.members, before=[], after=[1013, Fiona, null, null, 16.0, null], op=INSERT, meta=()}", "DataChangeEvent{tableId=%s.members, before=[], after=[1014, Gem, null, null, 17.0, null], op=INSERT, meta=()}")); - assertNotExists( - Collections.singletonList( - "Applied schema change event DropTableEvent{tableId=%s.members}"), - taskManagerConsumer); + Assertions.assertThat(taskManagerConsumer.toString()) + .doesNotContain( + String.format( + "Applied schema change event DropTableEvent{tableId=%s.members}", + schemaEvolveDatabase.getDatabaseName())); } @Test @@ -263,10 +232,7 @@ void testLenientWithRoute() throws Exception { dbName, dbName, parallelism); - Path mysqlCdcJar = TestUtils.getResource("mysql-cdc-pipeline-connector.jar"); - Path valuesCdcJar = TestUtils.getResource("values-cdc-pipeline-connector.jar"); - Path mysqlDriverJar = TestUtils.getResource("mysql-driver.jar"); - submitPipelineJob(pipelineJob, mysqlCdcJar, valuesCdcJar, mysqlDriverJar); + submitPipelineJob(pipelineJob); waitUntilJobRunning(Duration.ofSeconds(30)); LOG.info("Pipeline job is running"); validateSnapshotData(dbName, "redirect"); @@ -315,12 +281,12 @@ void testLenientWithRoute() throws Exception { "DataChangeEvent{tableId=%s.redirect, before=[], after=[1013, Fiona, null, null, 16.0, null], op=INSERT, meta=()}", "DataChangeEvent{tableId=%s.redirect, before=[], after=[1014, Gem, null, null, 17.0, null], op=INSERT, meta=()}"); - List expectedTmEvents = + String[] expectedTmEvents = expectedTaskManagerEvents.stream() .map(s -> String.format(s, dbName, dbName)) - .collect(Collectors.toList()); + .toArray(String[]::new); - validateResult(expectedTmEvents, taskManagerConsumer); + validateResult(expectedTmEvents); } @Test @@ -348,15 +314,9 @@ void testUnexpectedBehavior() { MYSQL_TEST_PASSWORD, schemaEvolveDatabase.getDatabaseName(), parallelism); - Path mysqlCdcJar = TestUtils.getResource("mysql-cdc-pipeline-connector.jar"); - Path valuesCdcJar = TestUtils.getResource("values-cdc-pipeline-connector.jar"); - Path mysqlDriverJar = TestUtils.getResource("mysql-driver.jar"); // Submitting job should fail given an unknown schema change behavior configuration - Assertions.assertThatThrownBy( - () -> - submitPipelineJob( - pipelineJob, mysqlCdcJar, valuesCdcJar, mysqlDriverJar)) + Assertions.assertThatThrownBy(() -> submitPipelineJob(pipelineJob)) .isExactlyInstanceOf(AssertionError.class); } @@ -393,10 +353,7 @@ void testByDefaultTransform() throws Exception { MYSQL_TEST_PASSWORD, dbName, parallelism); - Path mysqlCdcJar = TestUtils.getResource("mysql-cdc-pipeline-connector.jar"); - Path valuesCdcJar = TestUtils.getResource("values-cdc-pipeline-connector.jar"); - Path mysqlDriverJar = TestUtils.getResource("mysql-driver.jar"); - submitPipelineJob(pipelineJob, mysqlCdcJar, valuesCdcJar, mysqlDriverJar); + submitPipelineJob(pipelineJob); waitUntilJobRunning(Duration.ofSeconds(30)); LOG.info("Pipeline job is running"); validateSnapshotData(dbName, "members"); @@ -448,12 +405,12 @@ void testByDefaultTransform() throws Exception { "DataChangeEvent{tableId=%s.members, before=[], after=[1014, Gem, 17.0], op=INSERT, meta=()}", "DropTableEvent{tableId=%s.members}"); - List expectedTmEvents = + String[] expectedTmEvents = expectedTaskManagerEvents.stream() .map(s -> String.format(s, dbName, dbName)) - .collect(Collectors.toList()); + .toArray(String[]::new); - validateResult(expectedTmEvents, taskManagerConsumer); + validateResult(expectedTmEvents); } private void testGenericSchemaEvolution( @@ -519,10 +476,7 @@ private void testGenericSchemaEvolution( mergeTable ? "(members|new_members)" : "members", behavior, parallelism); - Path mysqlCdcJar = TestUtils.getResource("mysql-cdc-pipeline-connector.jar"); - Path valuesCdcJar = TestUtils.getResource("values-cdc-pipeline-connector.jar"); - Path mysqlDriverJar = TestUtils.getResource("mysql-driver.jar"); - submitPipelineJob(pipelineJob, mysqlCdcJar, valuesCdcJar, mysqlDriverJar); + submitPipelineJob(pipelineJob); waitUntilJobRunning(Duration.ofSeconds(30)); LOG.info("Pipeline job is running"); validateSnapshotData(dbName, mergeTable ? "merged" : "members"); @@ -561,33 +515,29 @@ private void testGenericSchemaEvolution( stmt.execute("DROP TABLE members;"); } - List expectedTmEvents = + String[] expectedTmEvents = expectedTaskManagerEvents.stream() .map(s -> String.format(s, dbName, dbName)) - .collect(Collectors.toList()); + .toArray(String[]::new); - validateResult(expectedTmEvents, taskManagerConsumer); + validateResult(taskManagerConsumer, expectedTmEvents); - List expectedJmEvents = + String[] expectedJmEvents = expectedJobManagerEvents.stream() .map(s -> String.format(s, dbName, dbName, dbName)) - .collect(Collectors.toList()); + .toArray(String[]::new); - validateResult(expectedJmEvents, jobManagerConsumer); + validateResult(jobManagerConsumer, expectedJmEvents); } private void validateSnapshotData(String dbName, String tableName) throws Exception { - List expected = - Stream.of( - "CreateTableEvent{tableId=%s.%s, schema=columns={`id` INT NOT NULL,`name` VARCHAR(17),`age` INT}, primaryKeys=id, options=()}", - "DataChangeEvent{tableId=%s.%s, before=[], after=[1008, Alice, 21], op=INSERT, meta=()}", - "DataChangeEvent{tableId=%s.%s, before=[], after=[1009, Bob, 20], op=INSERT, meta=()}", - "DataChangeEvent{tableId=%s.%s, before=[], after=[1010, Carol, 19], op=INSERT, meta=()}", - "DataChangeEvent{tableId=%s.%s, before=[], after=[1011, Derrida, 18], op=INSERT, meta=()}") - .map(s -> String.format(s, dbName, tableName)) - .collect(Collectors.toList()); - - validateResult(expected, taskManagerConsumer); + validateResult( + s -> String.format(s, dbName, tableName), + "CreateTableEvent{tableId=%s.%s, schema=columns={`id` INT NOT NULL,`name` VARCHAR(17),`age` INT}, primaryKeys=id, options=()}", + "DataChangeEvent{tableId=%s.%s, before=[], after=[1008, Alice, 21], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.%s, before=[], after=[1009, Bob, 20], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.%s, before=[], after=[1010, Carol, 19], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.%s, before=[], after=[1011, Derrida, 18], op=INSERT, meta=()}"); } private void waitForIncrementalStage(String dbName, String tableName, Statement stmt) @@ -596,44 +546,9 @@ private void waitForIncrementalStage(String dbName, String tableName, Statement // Ensure we change schema after incremental stage waitUntilSpecificEvent( + taskManagerConsumer, String.format( "DataChangeEvent{tableId=%s.%s, before=[], after=[0, __fence__, 0], op=INSERT, meta=()}", - dbName, tableName), - taskManagerConsumer); - } - - private void validateResult(List expectedEvents, ToStringConsumer consumer) - throws Exception { - for (String event : expectedEvents) { - waitUntilSpecificEvent(event, consumer); - } - } - - private void assertNotExists(List unexpectedEvents, ToStringConsumer consumer) { - String consumerLog = consumer.toUtf8String(); - for (String event : unexpectedEvents) { - Assertions.assertThat(consumerLog) - .doesNotContain(String.format(event, schemaEvolveDatabase.getDatabaseName())); - } - } - - private void waitUntilSpecificEvent(String event, ToStringConsumer consumer) throws Exception { - boolean result = false; - long endTimeout = System.currentTimeMillis() + SchemaEvolveE2eITCase.EVENT_WAITING_TIMEOUT; - 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()); - } + dbName, tableName)); } } diff --git a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/SchemaEvolvingTransformE2eITCase.java b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/SchemaEvolvingTransformE2eITCase.java index 5a671fc87dc..130dba0c4b1 100644 --- a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/SchemaEvolvingTransformE2eITCase.java +++ b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/SchemaEvolvingTransformE2eITCase.java @@ -17,9 +17,6 @@ package org.apache.flink.cdc.pipeline.tests; -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.cdc.connectors.mysql.testutils.UniqueDatabase; import org.apache.flink.cdc.pipeline.tests.utils.PipelineTestEnvironment; @@ -29,11 +26,7 @@ import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.testcontainers.containers.output.Slf4jLogConsumer; -import org.testcontainers.containers.output.ToStringConsumer; -import org.testcontainers.junit.jupiter.Container; -import java.nio.file.Path; import java.sql.Connection; import java.sql.DriverManager; import java.sql.Statement; @@ -41,37 +34,12 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; -import java.util.concurrent.TimeoutException; -import java.util.stream.Collectors; -import java.util.stream.Stream; /** E2e tests for Schema Evolution cases. */ class SchemaEvolvingTransformE2eITCase extends PipelineTestEnvironment { private static final Logger LOG = LoggerFactory.getLogger(SchemaEvolvingTransformE2eITCase.class); - // ------------------------------------------------------------------------------------------ - // 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 long EVENT_WAITING_TIMEOUT = 60000L; - - @Container - public static final MySqlContainer MYSQL = - (MySqlContainer) - new MySqlContainer( - MySqlVersion.V8_0) // v8 support both ARM and AMD architectures - .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)); - protected final UniqueDatabase schemaEvolveDatabase = new UniqueDatabase(MYSQL, "schema_evolve", MYSQL_TEST_USER, MYSQL_TEST_PASSWORD); @@ -247,15 +215,9 @@ void testUnexpectedBehavior() { MYSQL_TEST_PASSWORD, schemaEvolveDatabase.getDatabaseName(), parallelism); - Path mysqlCdcJar = TestUtils.getResource("mysql-cdc-pipeline-connector.jar"); - Path valuesCdcJar = TestUtils.getResource("values-cdc-pipeline-connector.jar"); - Path mysqlDriverJar = TestUtils.getResource("mysql-driver.jar"); // Submitting job should fail given an unknown schema change behavior configuration - Assertions.assertThatThrownBy( - () -> - submitPipelineJob( - pipelineJob, mysqlCdcJar, valuesCdcJar, mysqlDriverJar)) + Assertions.assertThatThrownBy(() -> submitPipelineJob(pipelineJob)) .isExactlyInstanceOf(AssertionError.class); } @@ -331,10 +293,7 @@ private void testGenericSchemaEvolution( dbName, behavior, parallelism); - Path mysqlCdcJar = TestUtils.getResource("mysql-cdc-pipeline-connector.jar"); - Path valuesCdcJar = TestUtils.getResource("values-cdc-pipeline-connector.jar"); - Path mysqlDriverJar = TestUtils.getResource("mysql-driver.jar"); - submitPipelineJob(pipelineJob, mysqlCdcJar, valuesCdcJar, mysqlDriverJar); + submitPipelineJob(pipelineJob); waitUntilJobRunning(Duration.ofSeconds(30)); LOG.info("Pipeline job is running"); validateSnapshotData(dbName, mergeTable ? "merged" : "members"); @@ -373,33 +332,29 @@ private void testGenericSchemaEvolution( stmt.execute("DROP TABLE members;"); } - List expectedTmEvents = + String[] expectedTmEvents = expectedTaskManagerEvents.stream() .map(s -> String.format(s, dbName, dbName)) - .collect(Collectors.toList()); + .toArray(String[]::new); - validateResult(expectedTmEvents, taskManagerConsumer); + validateResult(taskManagerConsumer, expectedTmEvents); - List expectedJmEvents = + String[] expectedJmEvents = expectedJobManagerEvents.stream() .map(s -> String.format(s, dbName, dbName, dbName)) - .collect(Collectors.toList()); + .toArray(String[]::new); - validateResult(expectedJmEvents, jobManagerConsumer); + validateResult(jobManagerConsumer, expectedJmEvents); } private void validateSnapshotData(String dbName, String tableName) throws Exception { - List expected = - Stream.of( - "CreateTableEvent{tableId=%s.%s, schema=columns={`uid` STRING,`id` INT NOT NULL,`name` VARCHAR(17),`age` INT,`id_square` INT,`tag` STRING}, primaryKeys=id, options=()}", - "DataChangeEvent{tableId=%s.%s, before=[], after=[1009 -> Bob, 1009, Bob, 20, -1018081, age >= 20], op=INSERT, meta=()}", - "DataChangeEvent{tableId=%s.%s, before=[], after=[1008 -> Alice, 1008, Alice, 21, -1016064, age >= 20], op=INSERT, meta=()}", - "DataChangeEvent{tableId=%s.%s, before=[], after=[1011 -> Derrida, 1011, Derrida, 18, 1022121, age < 20], op=INSERT, meta=()}", - "DataChangeEvent{tableId=%s.%s, before=[], after=[1010 -> Carol, 1010, Carol, 19, 1020100, age < 20], op=INSERT, meta=()}") - .map(s -> String.format(s, dbName, tableName)) - .collect(Collectors.toList()); - - validateResult(expected, taskManagerConsumer); + validateResult( + s -> String.format(s, dbName, tableName), + "CreateTableEvent{tableId=%s.%s, schema=columns={`uid` STRING,`id` INT NOT NULL,`name` VARCHAR(17),`age` INT,`id_square` INT,`tag` STRING}, primaryKeys=id, options=()}", + "DataChangeEvent{tableId=%s.%s, before=[], after=[1009 -> Bob, 1009, Bob, 20, -1018081, age >= 20], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.%s, before=[], after=[1008 -> Alice, 1008, Alice, 21, -1016064, age >= 20], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.%s, before=[], after=[1011 -> Derrida, 1011, Derrida, 18, 1022121, age < 20], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.%s, before=[], after=[1010 -> Carol, 1010, Carol, 19, 1020100, age < 20], op=INSERT, meta=()}"); } private void waitForIncrementalStage(String dbName, String tableName, Statement stmt) @@ -408,37 +363,9 @@ private void waitForIncrementalStage(String dbName, String tableName, Statement // Ensure we change schema after incremental stage waitUntilSpecificEvent( + taskManagerConsumer, String.format( "DataChangeEvent{tableId=%s.%s, before=[], after=[0 -> __fence__, 0, __fence__, 0, 0, age < 20], op=INSERT, meta=()}", - dbName, tableName), - taskManagerConsumer); - } - - private void validateResult(List expectedEvents, ToStringConsumer consumer) - throws Exception { - for (String event : expectedEvents) { - waitUntilSpecificEvent(event, consumer); - } - } - - private void waitUntilSpecificEvent(String event, ToStringConsumer consumer) throws Exception { - boolean result = false; - long endTimeout = - System.currentTimeMillis() + SchemaEvolvingTransformE2eITCase.EVENT_WAITING_TIMEOUT; - 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()); - } + dbName, tableName)); } } diff --git a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/TransformE2eITCase.java b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/TransformE2eITCase.java index 53391183d08..d685d283b05 100644 --- a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/TransformE2eITCase.java +++ b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/TransformE2eITCase.java @@ -17,9 +17,6 @@ package org.apache.flink.cdc.pipeline.tests; -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.cdc.connectors.mysql.testutils.UniqueDatabase; import org.apache.flink.cdc.pipeline.tests.utils.PipelineTestEnvironment; import org.apache.flink.cdc.runtime.operators.transform.PostTransformOperator; @@ -33,10 +30,7 @@ import org.junit.jupiter.params.provider.ValueSource; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.testcontainers.containers.output.Slf4jLogConsumer; -import org.testcontainers.junit.jupiter.Container; -import java.nio.file.Path; import java.sql.Connection; import java.sql.DriverManager; import java.sql.SQLException; @@ -49,6 +43,7 @@ import java.util.Arrays; import java.util.List; import java.util.concurrent.TimeoutException; +import java.util.function.Function; import java.util.regex.Pattern; import java.util.stream.Stream; @@ -56,31 +51,15 @@ class TransformE2eITCase extends PipelineTestEnvironment { private static final Logger LOG = LoggerFactory.getLogger(TransformE2eITCase.class); - // ------------------------------------------------------------------------------------------ - // 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 MYSQL_DRIVER_CLASS = "com.mysql.cj.jdbc.Driver"; - protected static final String INTER_CONTAINER_MYSQL_ALIAS = "mysql"; - - @Container - public static final MySqlContainer MYSQL = - (MySqlContainer) - new MySqlContainer( - MySqlVersion.V8_0) // v8 support both ARM and AMD architectures - .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)); - protected final UniqueDatabase transformTestDatabase = new UniqueDatabase(MYSQL, "transform_test", MYSQL_TEST_USER, MYSQL_TEST_PASSWORD); + private final Function dbNameFormatter = + (s) -> { + String databaseName = transformTestDatabase.getDatabaseName(); + return String.format(s, databaseName, databaseName, databaseName); + }; + @BeforeEach public void before() throws Exception { super.before(); @@ -135,20 +114,13 @@ void testHeteroSchemaTransform(boolean batchMode) throws Exception { transformTestDatabase.getDatabaseName(), runtimeMode, parallelism); - Path mysqlCdcJar = TestUtils.getResource("mysql-cdc-pipeline-connector.jar"); - Path valuesCdcJar = TestUtils.getResource("values-cdc-pipeline-connector.jar"); - Path mysqlDriverJar = TestUtils.getResource("mysql-driver.jar"); - submitPipelineJob(pipelineJob, mysqlCdcJar, valuesCdcJar, mysqlDriverJar); + submitPipelineJob(pipelineJob); waitUntilJobRunning(Duration.ofSeconds(30)); LOG.info("Pipeline job is running"); - waitUntilSpecificEvent( - String.format( - "CreateTableEvent{tableId=%s.terminus, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17)}, primaryKeys=ID, options=()}", - transformTestDatabase.getDatabaseName()), - 60000L); - - validateEvents( + validateResult( + dbNameFormatter, + "CreateTableEvent{tableId=%s.terminus, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17)}, primaryKeys=ID, options=()}", "DataChangeEvent{tableId=%s.terminus, before=[], after=[1009, 8.1], op=INSERT, meta=()}", "DataChangeEvent{tableId=%s.terminus, before=[], after=[1010, 10], op=INSERT, meta=()}", "DataChangeEvent{tableId=%s.terminus, before=[], after=[1011, 11], op=INSERT, meta=()}", @@ -181,7 +153,8 @@ void testHeteroSchemaTransform(boolean batchMode) throws Exception { throw e; } - validateEvents( + validateResult( + dbNameFormatter, "DataChangeEvent{tableId=%s.terminus, before=[], after=[3007, 7], op=INSERT, meta=()}", "DataChangeEvent{tableId=%s.terminus, before=[1009, 8.1], after=[1009, 100], op=UPDATE, meta=()}", "DataChangeEvent{tableId=%s.terminus, before=[2011, 11], after=[], op=DELETE, meta=()}"); @@ -228,26 +201,14 @@ void testMultipleTransformRule(boolean batchMode) throws Exception { transformTestDatabase.getDatabaseName(), runtimeMode, parallelism); - Path mysqlCdcJar = TestUtils.getResource("mysql-cdc-pipeline-connector.jar"); - Path valuesCdcJar = TestUtils.getResource("values-cdc-pipeline-connector.jar"); - Path mysqlDriverJar = TestUtils.getResource("mysql-driver.jar"); - submitPipelineJob(pipelineJob, mysqlCdcJar, valuesCdcJar, mysqlDriverJar); + submitPipelineJob(pipelineJob); waitUntilJobRunning(Duration.ofSeconds(30)); LOG.info("Pipeline job is running"); - waitUntilSpecificEvent( - String.format( - "CreateTableEvent{tableId=%s.TABLEALPHA, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17),`CATEGORY` STRING}, primaryKeys=ID, options=()}", - transformTestDatabase.getDatabaseName()), - 60000L); - - waitUntilSpecificEvent( - String.format( - "CreateTableEvent{tableId=%s.TABLEBETA, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17),`CATEGORY` STRING}, primaryKeys=ID, options=()}", - transformTestDatabase.getDatabaseName()), - 60000L); - - validateEvents( + validateResult( + dbNameFormatter, + "CreateTableEvent{tableId=%s.TABLEALPHA, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17),`CATEGORY` STRING}, primaryKeys=ID, options=()}", + "CreateTableEvent{tableId=%s.TABLEBETA, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17),`CATEGORY` STRING}, primaryKeys=ID, options=()}", "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[1008, 8, Type-B], op=INSERT, meta=()}", "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[1009, 8.1, Type-A], op=INSERT, meta=()}", "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[1010, 10, Type-A], op=INSERT, meta=()}", @@ -281,7 +242,8 @@ void testMultipleTransformRule(boolean batchMode) throws Exception { throw e; } - validateEvents( + validateResult( + dbNameFormatter, "DataChangeEvent{tableId=%s.TABLEALPHA, before=[1009, 8.1, Type-A], after=[1009, 100, Type-A], op=UPDATE, meta=()}", "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[3007, 7, Type-A], op=INSERT, meta=()}", "DataChangeEvent{tableId=%s.TABLEBETA, before=[2011, 11, Type-A], after=[], op=DELETE, meta=()}"); @@ -329,20 +291,13 @@ void testAssortedSchemaTransform(boolean batchMode) throws Exception { transformTestDatabase.getDatabaseName(), runtimeMode, parallelism); - Path mysqlCdcJar = TestUtils.getResource("mysql-cdc-pipeline-connector.jar"); - Path valuesCdcJar = TestUtils.getResource("values-cdc-pipeline-connector.jar"); - Path mysqlDriverJar = TestUtils.getResource("mysql-driver.jar"); - submitPipelineJob(pipelineJob, mysqlCdcJar, valuesCdcJar, mysqlDriverJar); + submitPipelineJob(pipelineJob); waitUntilJobRunning(Duration.ofSeconds(30)); LOG.info("Pipeline job is running"); - waitUntilSpecificEvent( - String.format( - "CreateTableEvent{tableId=%s.terminus, schema=columns={`ID` INT NOT NULL,`VERSION` STRING,`NAME` STRING}, primaryKeys=ID, options=()}", - transformTestDatabase.getDatabaseName()), - 60000L); - - validateEvents( + validateResult( + dbNameFormatter, + "CreateTableEvent{tableId=%s.terminus, schema=columns={`ID` INT NOT NULL,`VERSION` STRING,`NAME` STRING}, primaryKeys=ID, options=()}", "DataChangeEvent{tableId=%s.terminus, before=[], after=[1008, v8, alice], op=INSERT, meta=()}", "DataChangeEvent{tableId=%s.terminus, before=[], after=[1009, v8.1, bob], op=INSERT, meta=()}", "DataChangeEvent{tableId=%s.terminus, before=[], after=[2011, v11, eva], op=INSERT, meta=()}", @@ -374,7 +329,8 @@ void testAssortedSchemaTransform(boolean batchMode) throws Exception { throw e; } - validateEvents( + validateResult( + dbNameFormatter, "DataChangeEvent{tableId=%s.terminus, before=[1009, v8.1, bob], after=[1009, v100, bob], op=UPDATE, meta=()}", "DataChangeEvent{tableId=%s.terminus, before=[], after=[3007, v7, iina], op=INSERT, meta=()}", "DataChangeEvent{tableId=%s.terminus, before=[2011, v11, eva], after=[], op=DELETE, meta=()}"); @@ -417,26 +373,14 @@ void testWildcardSchemaTransform(boolean batchMode) throws Exception { transformTestDatabase.getDatabaseName(), runtimeMode, parallelism); - Path mysqlCdcJar = TestUtils.getResource("mysql-cdc-pipeline-connector.jar"); - Path valuesCdcJar = TestUtils.getResource("values-cdc-pipeline-connector.jar"); - Path mysqlDriverJar = TestUtils.getResource("mysql-driver.jar"); - submitPipelineJob(pipelineJob, mysqlCdcJar, valuesCdcJar, mysqlDriverJar); + submitPipelineJob(pipelineJob); waitUntilJobRunning(Duration.ofSeconds(30)); LOG.info("Pipeline job is running"); - waitUntilSpecificEvent( - String.format( - "CreateTableEvent{tableId=%s.TABLEALPHA, schema=columns={`ID` INT NOT NULL,`VERSION` STRING,`PRICEALPHA` INT,`AGEALPHA` INT,`NAMEALPHA` VARCHAR(128),`NAME` STRING}, primaryKeys=ID, options=()}", - transformTestDatabase.getDatabaseName()), - 60000L); - - waitUntilSpecificEvent( - String.format( - "CreateTableEvent{tableId=%s.TABLEBETA, schema=columns={`ID` INT NOT NULL,`VERSION` STRING,`CODENAMESBETA` VARCHAR(17),`AGEBETA` INT,`NAMEBETA` VARCHAR(128),`NAME` STRING}, primaryKeys=ID, options=()}", - transformTestDatabase.getDatabaseName()), - 60000L); - - validateEvents( + validateResult( + dbNameFormatter, + "CreateTableEvent{tableId=%s.TABLEALPHA, schema=columns={`ID` INT NOT NULL,`VERSION` STRING,`PRICEALPHA` INT,`AGEALPHA` INT,`NAMEALPHA` VARCHAR(128),`NAME` STRING}, primaryKeys=ID, options=()}", + "CreateTableEvent{tableId=%s.TABLEBETA, schema=columns={`ID` INT NOT NULL,`VERSION` STRING,`CODENAMESBETA` VARCHAR(17),`AGEBETA` INT,`NAMEBETA` VARCHAR(128),`NAME` STRING}, primaryKeys=ID, options=()}", "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[1008, v8, 199, 17, Alice, alice], op=INSERT, meta=()}", "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[1009, v8.1, 0, 18, Bob, bob], op=INSERT, meta=()}", "DataChangeEvent{tableId=%s.TABLEBETA, before=[], after=[2011, v11, Big Sur, 21, Eva, eva], op=INSERT, meta=()}", @@ -468,7 +412,8 @@ void testWildcardSchemaTransform(boolean batchMode) throws Exception { throw e; } - validateEvents( + validateResult( + dbNameFormatter, "DataChangeEvent{tableId=%s.TABLEALPHA, before=[1009, v8.1, 0, 18, Bob, bob], after=[1009, v100, 0, 18, Bob, bob], op=UPDATE, meta=()}", "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[3007, v7, 79, 16, IINA, iina], op=INSERT, meta=()}", "DataChangeEvent{tableId=%s.TABLEBETA, before=[2011, v11, Big Sur, 21, Eva, eva], after=[], op=DELETE, meta=()}"); @@ -511,26 +456,14 @@ void testWildcardWithMetadataColumnTransform(boolean batchMode) throws Exception transformTestDatabase.getDatabaseName(), runtimeMode, parallelism); - Path mysqlCdcJar = TestUtils.getResource("mysql-cdc-pipeline-connector.jar"); - Path valuesCdcJar = TestUtils.getResource("values-cdc-pipeline-connector.jar"); - Path mysqlDriverJar = TestUtils.getResource("mysql-driver.jar"); - submitPipelineJob(pipelineJob, mysqlCdcJar, valuesCdcJar, mysqlDriverJar); + submitPipelineJob(pipelineJob); waitUntilJobRunning(Duration.ofSeconds(30)); LOG.info("Pipeline job is running"); - waitUntilSpecificEvent( - String.format( - "CreateTableEvent{tableId=%s.TABLEALPHA, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17),`PRICEALPHA` INT,`AGEALPHA` INT,`NAMEALPHA` VARCHAR(128),`identifier_name` STRING,`type` STRING NOT NULL,`opts` BIGINT NOT NULL}, primaryKeys=ID, options=()}", - transformTestDatabase.getDatabaseName()), - 60000L); - - waitUntilSpecificEvent( - String.format( - "CreateTableEvent{tableId=%s.TABLEBETA, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17),`CODENAMESBETA` VARCHAR(17),`AGEBETA` INT,`NAMEBETA` VARCHAR(128),`identifier_name` STRING,`type` STRING NOT NULL,`opts` BIGINT NOT NULL}, primaryKeys=ID, options=()}", - transformTestDatabase.getDatabaseName()), - 60000L); - - validateEvents( + validateResult( + dbNameFormatter, + "CreateTableEvent{tableId=%s.TABLEALPHA, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17),`PRICEALPHA` INT,`AGEALPHA` INT,`NAMEALPHA` VARCHAR(128),`identifier_name` STRING,`type` STRING NOT NULL,`opts` BIGINT NOT NULL}, primaryKeys=ID, options=()}", + "CreateTableEvent{tableId=%s.TABLEBETA, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17),`CODENAMESBETA` VARCHAR(17),`AGEBETA` INT,`NAMEBETA` VARCHAR(128),`identifier_name` STRING,`type` STRING NOT NULL,`opts` BIGINT NOT NULL}, primaryKeys=ID, options=()}", "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[1008, 8, 199, 17, Alice, null.%s.TABLEALPHA, +I, 0], op=INSERT, meta=({op_ts=0})}", "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[1010, 10, 99, 19, Carol, null.%s.TABLEALPHA, +I, 0], op=INSERT, meta=({op_ts=0})}", "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[1009, 8.1, 0, 18, Bob, null.%s.TABLEALPHA, +I, 0], op=INSERT, meta=({op_ts=0})}", @@ -606,26 +539,14 @@ void testMultipleHittingTable(boolean batchMode) throws Exception { transformTestDatabase.getDatabaseName(), runtimeMode, parallelism); - Path mysqlCdcJar = TestUtils.getResource("mysql-cdc-pipeline-connector.jar"); - Path valuesCdcJar = TestUtils.getResource("values-cdc-pipeline-connector.jar"); - Path mysqlDriverJar = TestUtils.getResource("mysql-driver.jar"); - submitPipelineJob(pipelineJob, mysqlCdcJar, valuesCdcJar, mysqlDriverJar); + submitPipelineJob(pipelineJob); waitUntilJobRunning(Duration.ofSeconds(30)); LOG.info("Pipeline job is running"); - waitUntilSpecificEvent( - String.format( - "CreateTableEvent{tableId=%s.TABLEALPHA, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17),`PRICEALPHA` INT,`AGEALPHA` INT,`NAMEALPHA` VARCHAR(128),`UID` INT,`NEWVERSION` VARCHAR(17)}, primaryKeys=ID, options=()}", - transformTestDatabase.getDatabaseName()), - 60000L); - - waitUntilSpecificEvent( - String.format( - "CreateTableEvent{tableId=%s.TABLEBETA, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17),`CODENAMESBETA` VARCHAR(17),`AGEBETA` INT,`NAMEBETA` VARCHAR(128),`UID` INT,`NEWVERSION` VARCHAR(17)}, primaryKeys=ID, options=()}", - transformTestDatabase.getDatabaseName()), - 60000L); - - validateEvents( + validateResult( + dbNameFormatter, + "CreateTableEvent{tableId=%s.TABLEALPHA, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17),`PRICEALPHA` INT,`AGEALPHA` INT,`NAMEALPHA` VARCHAR(128),`UID` INT,`NEWVERSION` VARCHAR(17)}, primaryKeys=ID, options=()}", + "CreateTableEvent{tableId=%s.TABLEBETA, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17),`CODENAMESBETA` VARCHAR(17),`AGEBETA` INT,`NAMEBETA` VARCHAR(128),`UID` INT,`NEWVERSION` VARCHAR(17)}, primaryKeys=ID, options=()}", "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[1008, 8, 199, 17, Alice, 2008, 8], op=INSERT, meta=()}", "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[1009, 8.1, 0, 18, Bob, 2009, 8.1], op=INSERT, meta=()}", "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[1010, 10, 99, 19, Carol, 2010, 10], op=INSERT, meta=()}", @@ -658,7 +579,8 @@ void testMultipleHittingTable(boolean batchMode) throws Exception { throw e; } - validateEvents( + validateResult( + dbNameFormatter, "DataChangeEvent{tableId=%s.TABLEALPHA, before=[1009, 8.1, 0, 18, Bob, 2009, 8.1], after=[1009, 100, 0, 18, Bob, 2009, 100], op=UPDATE, meta=()}", "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[3007, 7, 79, 25, IINA, 4007, 7], op=INSERT, meta=()}", "DataChangeEvent{tableId=%s.TABLEBETA, before=[2011, 11, Big Sur, 21, Eva, 3011, 11], after=[], op=DELETE, meta=()}"); @@ -702,20 +624,13 @@ void testMultipleTransformWithDiffRefColumn(boolean batchMode) throws Exception transformTestDatabase.getDatabaseName(), runtimeMode, parallelism); - Path mysqlCdcJar = TestUtils.getResource("mysql-cdc-pipeline-connector.jar"); - Path valuesCdcJar = TestUtils.getResource("values-cdc-pipeline-connector.jar"); - Path mysqlDriverJar = TestUtils.getResource("mysql-driver.jar"); - submitPipelineJob(pipelineJob, mysqlCdcJar, valuesCdcJar, mysqlDriverJar); + submitPipelineJob(pipelineJob); waitUntilJobRunning(Duration.ofSeconds(30)); LOG.info("Pipeline job is running"); - waitUntilSpecificEvent( - String.format( - "CreateTableEvent{tableId=%s.TABLEALPHA, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17),`PRICEALPHA` INT,`AGEALPHA` INT,`ROLENAME` STRING}, primaryKeys=ID, options=()}", - transformTestDatabase.getDatabaseName()), - 60000L); - - validateEvents( + validateResult( + dbNameFormatter, + "CreateTableEvent{tableId=%s.TABLEALPHA, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17),`PRICEALPHA` INT,`AGEALPHA` INT,`ROLENAME` STRING}, primaryKeys=ID, options=()}", "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[1008, 8, 199, 17, Juvenile], op=INSERT, meta=()}", "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[1009, 8.1, 0, 18, Bob], op=INSERT, meta=()}", "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[1010, 10, 99, 19, Carol], op=INSERT, meta=()}", @@ -744,7 +659,8 @@ void testMultipleTransformWithDiffRefColumn(boolean batchMode) throws Exception throw e; } - validateEvents( + validateResult( + dbNameFormatter, "DataChangeEvent{tableId=%s.TABLEALPHA, before=[1009, 8.1, 0, 18, Bob], after=[1009, 100, 0, 18, Bob], op=UPDATE, meta=()}", "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[3007, 7, 79, 25, IINA], op=INSERT, meta=()}", "DataChangeEvent{tableId=%s.TABLEALPHA, before=[1011, 11, 59, 20, Dave], after=[], op=DELETE, meta=()}"); @@ -787,28 +703,17 @@ void testTransformWithCast(boolean batchMode) throws Exception { transformTestDatabase.getDatabaseName(), runtimeMode, parallelism); - Path mysqlCdcJar = TestUtils.getResource("mysql-cdc-pipeline-connector.jar"); - Path valuesCdcJar = TestUtils.getResource("values-cdc-pipeline-connector.jar"); - Path mysqlDriverJar = TestUtils.getResource("mysql-driver.jar"); - submitPipelineJob(pipelineJob, mysqlCdcJar, valuesCdcJar, mysqlDriverJar); + submitPipelineJob(pipelineJob); waitUntilJobRunning(Duration.ofSeconds(30)); if (batchMode) { return; } - waitUntilSpecificEvent( - String.format( - "CreateTableEvent{tableId=%s.TABLEALPHA, schema=columns={`ID` INT NOT NULL,`VERSION` DOUBLE,`IDENTIFIER` STRING}, primaryKeys=ID, options=()}", - transformTestDatabase.getDatabaseName()), - 60000L); - waitUntilSpecificEvent( - String.format( - "CreateTableEvent{tableId=%s.TABLEBETA, schema=columns={`ID` INT NOT NULL,`VERSION` DOUBLE,`IDENTIFIER` STRING}, primaryKeys=ID, options=()}", - transformTestDatabase.getDatabaseName()), - 60000L); - - validateEvents( + validateResult( + dbNameFormatter, + "CreateTableEvent{tableId=%s.TABLEALPHA, schema=columns={`ID` INT NOT NULL,`VERSION` DOUBLE,`IDENTIFIER` STRING}, primaryKeys=ID, options=()}", + "CreateTableEvent{tableId=%s.TABLEBETA, schema=columns={`ID` INT NOT NULL,`VERSION` DOUBLE,`IDENTIFIER` STRING}, primaryKeys=ID, options=()}", "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[1008, 108.0, 17 - Alice], op=INSERT, meta=()}", "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[1009, 108.1, 18 - Bob], op=INSERT, meta=()}", "DataChangeEvent{tableId=%s.TABLEBETA, before=[], after=[2011, 111.0, 21 - Eva], op=INSERT, meta=()}", @@ -836,7 +741,8 @@ void testTransformWithCast(boolean batchMode) throws Exception { throw e; } - validateEvents( + validateResult( + dbNameFormatter, "DataChangeEvent{tableId=%s.TABLEALPHA, before=[1009, 108.1, 18 - Bob], after=[1009, 200.0, 18 - Bob], op=UPDATE, meta=()}", "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[3007, 107.0, 16 - IINA], op=INSERT, meta=()}", "DataChangeEvent{tableId=%s.TABLEBETA, before=[2011, 111.0, 21 - Eva], after=[], op=DELETE, meta=()}"); @@ -878,10 +784,7 @@ void testTemporalFunctions(boolean batchMode) throws Exception { transformTestDatabase.getDatabaseName(), runtimeMode, parallelism); - Path mysqlCdcJar = TestUtils.getResource("mysql-cdc-pipeline-connector.jar"); - Path valuesCdcJar = TestUtils.getResource("values-cdc-pipeline-connector.jar"); - Path mysqlDriverJar = TestUtils.getResource("mysql-driver.jar"); - submitPipelineJob(pipelineJob, mysqlCdcJar, valuesCdcJar, mysqlDriverJar); + submitPipelineJob(pipelineJob); waitUntilJobRunning(Duration.ofSeconds(30)); LOG.info("Pipeline job is running"); @@ -916,20 +819,13 @@ void testTransformWithSchemaEvolution() throws Exception { transformTestDatabase.getDatabaseName(), transformTestDatabase.getDatabaseName(), parallelism); - Path mysqlCdcJar = TestUtils.getResource("mysql-cdc-pipeline-connector.jar"); - Path valuesCdcJar = TestUtils.getResource("values-cdc-pipeline-connector.jar"); - Path mysqlDriverJar = TestUtils.getResource("mysql-driver.jar"); - submitPipelineJob(pipelineJob, mysqlCdcJar, valuesCdcJar, mysqlDriverJar); + submitPipelineJob(pipelineJob); waitUntilJobRunning(Duration.ofSeconds(30)); LOG.info("Pipeline job is running"); - waitUntilSpecificEvent( - String.format( - "CreateTableEvent{tableId=%s.TABLEALPHA, schema=columns={`ID` INT NOT NULL,`UID` STRING,`PRICE` INT}, primaryKeys=ID, options=()}", - transformTestDatabase.getDatabaseName()), - 60000L); - - validateEvents( + validateResult( + dbNameFormatter, + "CreateTableEvent{tableId=%s.TABLEALPHA, schema=columns={`ID` INT NOT NULL,`UID` STRING,`PRICE` INT}, primaryKeys=ID, options=()}", "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[1009, id -> 1009, 0], op=INSERT, meta=()}", "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[1010, id -> 1010, 99], op=INSERT, meta=()}", "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[1011, id -> 1011, 59], op=INSERT, meta=()}"); @@ -954,7 +850,8 @@ void testTransformWithSchemaEvolution() throws Exception { throw e; } - validateEvents( + validateResult( + dbNameFormatter, "DataChangeEvent{tableId=%s.TABLEALPHA, before=[1009, id -> 1009, 0], after=[1009, id -> 1009, 0], op=UPDATE, meta=()}", "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[3007, id -> 3007, 79], op=INSERT, meta=()}"); @@ -981,7 +878,8 @@ void testTransformWithSchemaEvolution() throws Exception { stmt.execute("INSERT INTO TABLEALPHA VALUES (3010, '10', 10, 97, 19, 'Lynx');"); } - validateEvents( + validateResult( + dbNameFormatter, "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[3008, id -> 3008, 80], op=INSERT, meta=()}", "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[3009, id -> 3009, 90], op=INSERT, meta=()}", "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[3010, id -> 3010, 10], op=INSERT, meta=()}"); @@ -1015,20 +913,13 @@ void testTransformWildcardPrefixWithSchemaEvolution() throws Exception { transformTestDatabase.getDatabaseName(), transformTestDatabase.getDatabaseName(), parallelism); - Path mysqlCdcJar = TestUtils.getResource("mysql-cdc-pipeline-connector.jar"); - Path valuesCdcJar = TestUtils.getResource("values-cdc-pipeline-connector.jar"); - Path mysqlDriverJar = TestUtils.getResource("mysql-driver.jar"); - submitPipelineJob(pipelineJob, mysqlCdcJar, valuesCdcJar, mysqlDriverJar); + submitPipelineJob(pipelineJob); waitUntilJobRunning(Duration.ofSeconds(30)); LOG.info("Pipeline job is running"); - waitUntilSpecificEvent( - String.format( - "CreateTableEvent{tableId=%s.TABLEALPHA, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17),`PRICEALPHA` INT,`AGEALPHA` INT,`NAMEALPHA` VARCHAR(128),`UID` STRING}, primaryKeys=ID, options=()}", - transformTestDatabase.getDatabaseName()), - 60000L); - - validateEvents( + validateResult( + dbNameFormatter, + "CreateTableEvent{tableId=%s.TABLEALPHA, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17),`PRICEALPHA` INT,`AGEALPHA` INT,`NAMEALPHA` VARCHAR(128),`UID` STRING}, primaryKeys=ID, options=()}", "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[1009, 8.1, 0, 18, Bob, id -> 1009], op=INSERT, meta=()}", "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[1010, 10, 99, 19, Carol, id -> 1010], op=INSERT, meta=()}", "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[1011, 11, 59, 20, Dave, id -> 1011], op=INSERT, meta=()}"); @@ -1053,7 +944,8 @@ void testTransformWildcardPrefixWithSchemaEvolution() throws Exception { throw e; } - validateEvents( + validateResult( + dbNameFormatter, "DataChangeEvent{tableId=%s.TABLEALPHA, before=[1009, 8.1, 0, 18, Bob, id -> 1009], after=[1009, 100, 0, 18, Bob, id -> 1009], op=UPDATE, meta=()}", "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[3007, 7, 79, 16, IINA, id -> 3007], op=INSERT, meta=()}"); @@ -1083,7 +975,8 @@ void testTransformWildcardPrefixWithSchemaEvolution() throws Exception { throw e; } - validateEvents( + validateResult( + dbNameFormatter, "AddColumnEvent{tableId=%s.TABLEALPHA, addedColumns=[ColumnWithPosition{column=`LAST` VARCHAR(17), position=AFTER, existedColumnName=NAMEALPHA}]}", "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[3008, 8, 8, 80, 17, Jazz, Last, id -> 3008], op=INSERT, meta=()}", "AlterColumnTypeEvent{tableId=%s.TABLEALPHA, typeMapping={CODENAME=DOUBLE}, oldTypeMapping={CODENAME=TINYINT}}", @@ -1122,20 +1015,13 @@ void testTransformWildcardSuffixWithSchemaEvolution() throws Exception { transformTestDatabase.getDatabaseName(), transformTestDatabase.getDatabaseName(), parallelism); - Path mysqlCdcJar = TestUtils.getResource("mysql-cdc-pipeline-connector.jar"); - Path valuesCdcJar = TestUtils.getResource("values-cdc-pipeline-connector.jar"); - Path mysqlDriverJar = TestUtils.getResource("mysql-driver.jar"); - submitPipelineJob(pipelineJob, mysqlCdcJar, valuesCdcJar, mysqlDriverJar); + submitPipelineJob(pipelineJob); waitUntilJobRunning(Duration.ofSeconds(30)); LOG.info("Pipeline job is running"); - waitUntilSpecificEvent( - String.format( - "CreateTableEvent{tableId=%s.TABLEALPHA, schema=columns={`UID` STRING,`ID` INT NOT NULL,`VERSION` VARCHAR(17),`PRICEALPHA` INT,`AGEALPHA` INT,`NAMEALPHA` VARCHAR(128)}, primaryKeys=ID, options=()}", - transformTestDatabase.getDatabaseName()), - 60000L); - - validateEvents( + validateResult( + dbNameFormatter, + "CreateTableEvent{tableId=%s.TABLEALPHA, schema=columns={`UID` STRING,`ID` INT NOT NULL,`VERSION` VARCHAR(17),`PRICEALPHA` INT,`AGEALPHA` INT,`NAMEALPHA` VARCHAR(128)}, primaryKeys=ID, options=()}", "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[1009 <- id, 1009, 8.1, 0, 18, Bob], op=INSERT, meta=()}", "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[1010 <- id, 1010, 10, 99, 19, Carol], op=INSERT, meta=()}", "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[1011 <- id, 1011, 11, 59, 20, Dave], op=INSERT, meta=()}"); @@ -1160,7 +1046,8 @@ void testTransformWildcardSuffixWithSchemaEvolution() throws Exception { throw e; } - validateEvents( + validateResult( + dbNameFormatter, "DataChangeEvent{tableId=%s.TABLEALPHA, before=[1009 <- id, 1009, 8.1, 0, 18, Bob], after=[1009 <- id, 1009, 100, 0, 18, Bob], op=UPDATE, meta=()}", "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[3007 <- id, 3007, 7, 79, 16, IINA], op=INSERT, meta=()}"); @@ -1191,7 +1078,8 @@ void testTransformWildcardSuffixWithSchemaEvolution() throws Exception { throw e; } - validateEvents( + validateResult( + dbNameFormatter, "AddColumnEvent{tableId=%s.TABLEALPHA, addedColumns=[ColumnWithPosition{column=`CODENAME` TINYINT, position=AFTER, existedColumnName=VERSION}]}", "AddColumnEvent{tableId=%s.TABLEALPHA, addedColumns=[ColumnWithPosition{column=`FIRST` VARCHAR(17), position=BEFORE, existedColumnName=ID}]}", "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[3008 <- id, First, 3008, 8, 8, 80, 17, Jazz], op=INSERT, meta=()}", @@ -1203,18 +1091,6 @@ void testTransformWildcardSuffixWithSchemaEvolution() throws Exception { "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[3010 <- id, Beginning, 3010, 10, 10, 97, Lemon], op=INSERT, meta=()}"); } - private void validateEvents(String... expectedEvents) throws Exception { - for (String event : expectedEvents) { - waitUntilSpecificEvent( - String.format( - event, - transformTestDatabase.getDatabaseName(), - transformTestDatabase.getDatabaseName(), - transformTestDatabase.getDatabaseName()), - 20000L); - } - } - private void validateEventsWithPattern(String... patterns) throws Exception { for (String pattern : patterns) { waitUntilSpecificEventWithPattern( @@ -1249,26 +1125,6 @@ private void waitUntilSpecificEventWithPattern(String patternStr, long timeout) } } - private void waitUntilSpecificEvent(String event, long timeout) throws Exception { - boolean result = false; - long endTimeout = System.currentTimeMillis() + timeout; - while (System.currentTimeMillis() < endTimeout) { - String stdout = taskManagerConsumer.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: " - + taskManagerConsumer.toUtf8String()); - } - } - private int validateTemporaryRecords() { int validRecordCount = 0; for (String line : taskManagerConsumer.toUtf8String().split("\n")) { diff --git a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/UdfE2eITCase.java b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/UdfE2eITCase.java index 7f27507261c..ca8e3d76400 100644 --- a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/UdfE2eITCase.java +++ b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/UdfE2eITCase.java @@ -18,8 +18,6 @@ package org.apache.flink.cdc.pipeline.tests; 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.cdc.connectors.mysql.testutils.UniqueDatabase; import org.apache.flink.cdc.pipeline.tests.utils.PipelineTestEnvironment; @@ -30,8 +28,6 @@ import org.junit.jupiter.params.provider.MethodSource; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.testcontainers.containers.output.Slf4jLogConsumer; -import org.testcontainers.junit.jupiter.Container; import java.nio.file.Path; import java.sql.Connection; @@ -39,50 +35,29 @@ import java.sql.SQLException; import java.sql.Statement; import java.time.Duration; -import java.util.Arrays; -import java.util.List; -import java.util.concurrent.TimeoutException; +import java.util.function.Function; import java.util.stream.Stream; /** E2e tests for User-defined functions. */ class UdfE2eITCase extends PipelineTestEnvironment { private static final Logger LOG = LoggerFactory.getLogger(TransformE2eITCase.class); - // ------------------------------------------------------------------------------------------ - // 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 MYSQL_DRIVER_CLASS = "com.mysql.cj.jdbc.Driver"; - protected static final String INTER_CONTAINER_MYSQL_ALIAS = "mysql"; - - @Container - public static final MySqlContainer MYSQL = - (MySqlContainer) - new MySqlContainer( - MySqlVersion.V8_0) // v8 support both ARM and AMD architectures - .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)); - - protected final UniqueDatabase transformRenameDatabase = + protected final UniqueDatabase udfTestDatabase = new UniqueDatabase(MYSQL, "transform_test", MYSQL_TEST_USER, MYSQL_TEST_PASSWORD); + private final Function dbNameFormatter = + (s) -> String.format(s, udfTestDatabase.getDatabaseName()); + @BeforeEach public void before() throws Exception { super.before(); - transformRenameDatabase.createAndInitialize(); + udfTestDatabase.createAndInitialize(); } @AfterEach public void after() { super.after(); - transformRenameDatabase.dropDatabase(); + udfTestDatabase.dropDatabase(); } private static Stream variants() { @@ -138,9 +113,9 @@ void testUserDefinedFunctions(String language, boolean batchMode) throws Excepti MYSQL_TEST_USER, MYSQL_TEST_PASSWORD, startupMode, - transformRenameDatabase.getDatabaseName(), - transformRenameDatabase.getDatabaseName(), - transformRenameDatabase.getDatabaseName(), + udfTestDatabase.getDatabaseName(), + udfTestDatabase.getDatabaseName(), + udfTestDatabase.getDatabaseName(), parallelism, runtimeMode, language, @@ -148,73 +123,34 @@ void testUserDefinedFunctions(String language, boolean batchMode) throws Excepti language, language, language); - Path mysqlCdcJar = TestUtils.getResource("mysql-cdc-pipeline-connector.jar"); - Path valuesCdcJar = TestUtils.getResource("values-cdc-pipeline-connector.jar"); - Path mysqlDriverJar = TestUtils.getResource("mysql-driver.jar"); Path udfJar = TestUtils.getResource("udf-examples.jar"); Path scalaLibJar = TestUtils.getResource("scala-library.jar"); - submitPipelineJob( - pipelineJob, mysqlCdcJar, valuesCdcJar, mysqlDriverJar, udfJar, scalaLibJar); + submitPipelineJob(pipelineJob, udfJar, scalaLibJar); waitUntilJobRunning(Duration.ofSeconds(30)); - LOG.info("Pipeline job is running"); - - waitUntilSpecificEvent("[ LifecycleFunction ] opened.", 60000L); - - waitUntilSpecificEvent( - String.format( - "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[1011, 11, 1013, <11>], op=INSERT, meta=()}", - transformRenameDatabase.getDatabaseName()), - 60000L); - - waitUntilSpecificEvent( - String.format( - "DataChangeEvent{tableId=%s.TABLEBETA, before=[], after=[2014, 14, Forty-two, Integer: 2014], op=INSERT, meta=()}", - transformRenameDatabase.getDatabaseName()), - 60000L); + waitUntilSpecificEvent("[ LifecycleFunction ] opened."); - List expectedEvents = - Arrays.asList( - String.format( - "CreateTableEvent{tableId=%s.TABLEALPHA, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17),`INC_ID` STRING,`FMT_VER` STRING}, primaryKeys=ID, options=()}", - transformRenameDatabase.getDatabaseName()), - String.format( - "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[1009, 8.1, 1011, <8.1>], op=INSERT, meta=()}", - transformRenameDatabase.getDatabaseName()), - String.format( - "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[1010, 10, 1012, <10>], op=INSERT, meta=()}", - transformRenameDatabase.getDatabaseName()), - String.format( - "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[1011, 11, 1013, <11>], op=INSERT, meta=()}", - transformRenameDatabase.getDatabaseName()), - String.format( - "CreateTableEvent{tableId=%s.TABLEBETA, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17),`ANS` STRING,`TYP` STRING}, primaryKeys=ID, options=()}", - transformRenameDatabase.getDatabaseName()), - String.format( - "DataChangeEvent{tableId=%s.TABLEBETA, before=[], after=[2011, 11, Forty-two, Integer: 2011], op=INSERT, meta=()}", - transformRenameDatabase.getDatabaseName()), - String.format( - "DataChangeEvent{tableId=%s.TABLEBETA, before=[], after=[2012, 12, Forty-two, Integer: 2012], op=INSERT, meta=()}", - transformRenameDatabase.getDatabaseName()), - String.format( - "DataChangeEvent{tableId=%s.TABLEBETA, before=[], after=[2013, 13, Forty-two, Integer: 2013], op=INSERT, meta=()}", - transformRenameDatabase.getDatabaseName()), - String.format( - "DataChangeEvent{tableId=%s.TABLEBETA, before=[], after=[2014, 14, Forty-two, Integer: 2014], op=INSERT, meta=()}", - transformRenameDatabase.getDatabaseName())); - validateResult(expectedEvents); + validateResult( + dbNameFormatter, + "CreateTableEvent{tableId=%s.TABLEALPHA, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17),`INC_ID` STRING,`FMT_VER` STRING}, primaryKeys=ID, options=()}", + "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[1009, 8.1, 1011, <8.1>], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[1010, 10, 1012, <10>], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[1011, 11, 1013, <11>], op=INSERT, meta=()}", + "CreateTableEvent{tableId=%s.TABLEBETA, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17),`ANS` STRING,`TYP` STRING}, primaryKeys=ID, options=()}", + "DataChangeEvent{tableId=%s.TABLEBETA, before=[], after=[2011, 11, Forty-two, Integer: 2011], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.TABLEBETA, before=[], after=[2012, 12, Forty-two, Integer: 2012], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.TABLEBETA, before=[], after=[2013, 13, Forty-two, Integer: 2013], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.TABLEBETA, before=[], after=[2014, 14, Forty-two, Integer: 2014], op=INSERT, meta=()}"); if (batchMode) { return; } - LOG.info("Begin incremental reading stage."); - // generate binlogs String mysqlJdbcUrl = String.format( "jdbc:mysql://%s:%s/%s", MYSQL.getHost(), MYSQL.getDatabasePort(), - transformRenameDatabase.getDatabaseName()); + udfTestDatabase.getDatabaseName()); try (Connection conn = DriverManager.getConnection( mysqlJdbcUrl, MYSQL_TEST_USER, MYSQL_TEST_PASSWORD); @@ -227,23 +163,11 @@ void testUserDefinedFunctions(String language, boolean batchMode) throws Excepti throw e; } - waitUntilSpecificEvent( - String.format( - "DataChangeEvent{tableId=%s.TABLEALPHA, before=[1009, 8.1, 1011, <8.1>], after=[1009, 100, 1011, <100>], op=UPDATE, meta=()}", - transformRenameDatabase.getDatabaseName()), - 20000L); - - waitUntilSpecificEvent( - String.format( - "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[3007, 7, 3009, <7>], op=INSERT, meta=()}", - transformRenameDatabase.getDatabaseName()), - 20000L); - - waitUntilSpecificEvent( - String.format( - "DataChangeEvent{tableId=%s.TABLEBETA, before=[2011, 11, Forty-two, Integer: 2011], after=[], op=DELETE, meta=()}", - transformRenameDatabase.getDatabaseName()), - 20000L); + validateResult( + dbNameFormatter, + "DataChangeEvent{tableId=%s.TABLEALPHA, before=[1009, 8.1, 1011, <8.1>], after=[1009, 100, 1011, <100>], op=UPDATE, meta=()}", + "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[3007, 7, 3009, <7>], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.TABLEBETA, before=[2011, 11, Forty-two, Integer: 2011], after=[], op=DELETE, meta=()}"); } @ParameterizedTest(name = "language: {0}, batchMode: {1}") @@ -287,78 +211,40 @@ void testFlinkCompatibleScalarFunctions(String language, boolean batchMode) thro MYSQL_TEST_USER, MYSQL_TEST_PASSWORD, startupMode, - transformRenameDatabase.getDatabaseName(), - transformRenameDatabase.getDatabaseName(), - transformRenameDatabase.getDatabaseName(), + udfTestDatabase.getDatabaseName(), + udfTestDatabase.getDatabaseName(), + udfTestDatabase.getDatabaseName(), parallelism, runtimeMode, language, language, language); - Path mysqlCdcJar = TestUtils.getResource("mysql-cdc-pipeline-connector.jar"); - Path valuesCdcJar = TestUtils.getResource("values-cdc-pipeline-connector.jar"); - Path mysqlDriverJar = TestUtils.getResource("mysql-driver.jar"); Path udfJar = TestUtils.getResource("udf-examples.jar"); Path scalaLibJar = TestUtils.getResource("scala-library.jar"); - submitPipelineJob( - pipelineJob, mysqlCdcJar, valuesCdcJar, mysqlDriverJar, udfJar, scalaLibJar); + submitPipelineJob(pipelineJob, udfJar, scalaLibJar); waitUntilJobRunning(Duration.ofSeconds(30)); - LOG.info("Pipeline job is running"); - - waitUntilSpecificEvent( - String.format( - "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[1011, 11, 1013, <11>], op=INSERT, meta=()}", - transformRenameDatabase.getDatabaseName()), - 60000L); - - waitUntilSpecificEvent( - String.format( - "DataChangeEvent{tableId=%s.TABLEBETA, before=[], after=[2014, 14, Integer: 2014], op=INSERT, meta=()}", - transformRenameDatabase.getDatabaseName()), - 60000L); - - List expectedEvents = - Arrays.asList( - String.format( - "CreateTableEvent{tableId=%s.TABLEALPHA, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17),`INC_ID` STRING,`FMT_VER` STRING}, primaryKeys=ID, options=()}", - transformRenameDatabase.getDatabaseName()), - String.format( - "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[1009, 8.1, 1011, <8.1>], op=INSERT, meta=()}", - transformRenameDatabase.getDatabaseName()), - String.format( - "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[1010, 10, 1012, <10>], op=INSERT, meta=()}", - transformRenameDatabase.getDatabaseName()), - String.format( - "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[1011, 11, 1013, <11>], op=INSERT, meta=()}", - transformRenameDatabase.getDatabaseName()), - String.format( - "CreateTableEvent{tableId=%s.TABLEBETA, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17),`TYP` STRING}, primaryKeys=ID, options=()}", - transformRenameDatabase.getDatabaseName()), - String.format( - "DataChangeEvent{tableId=%s.TABLEBETA, before=[], after=[2011, 11, Integer: 2011], op=INSERT, meta=()}", - transformRenameDatabase.getDatabaseName()), - String.format( - "DataChangeEvent{tableId=%s.TABLEBETA, before=[], after=[2012, 12, Integer: 2012], op=INSERT, meta=()}", - transformRenameDatabase.getDatabaseName()), - String.format( - "DataChangeEvent{tableId=%s.TABLEBETA, before=[], after=[2013, 13, Integer: 2013], op=INSERT, meta=()}", - transformRenameDatabase.getDatabaseName()), - String.format( - "DataChangeEvent{tableId=%s.TABLEBETA, before=[], after=[2014, 14, Integer: 2014], op=INSERT, meta=()}", - transformRenameDatabase.getDatabaseName())); - validateResult(expectedEvents); + validateResult( + dbNameFormatter, + "CreateTableEvent{tableId=%s.TABLEALPHA, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17),`INC_ID` STRING,`FMT_VER` STRING}, primaryKeys=ID, options=()}", + "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[1009, 8.1, 1011, <8.1>], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[1010, 10, 1012, <10>], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[1011, 11, 1013, <11>], op=INSERT, meta=()}", + "CreateTableEvent{tableId=%s.TABLEBETA, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17),`TYP` STRING}, primaryKeys=ID, options=()}", + "DataChangeEvent{tableId=%s.TABLEBETA, before=[], after=[2011, 11, Integer: 2011], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.TABLEBETA, before=[], after=[2012, 12, Integer: 2012], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.TABLEBETA, before=[], after=[2013, 13, Integer: 2013], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.TABLEBETA, before=[], after=[2014, 14, Integer: 2014], op=INSERT, meta=()}"); if (batchMode) { return; } - LOG.info("Begin incremental reading stage."); - // generate binlogs + String mysqlJdbcUrl = String.format( "jdbc:mysql://%s:%s/%s", MYSQL.getHost(), MYSQL.getDatabasePort(), - transformRenameDatabase.getDatabaseName()); + udfTestDatabase.getDatabaseName()); try (Connection conn = DriverManager.getConnection( mysqlJdbcUrl, MYSQL_TEST_USER, MYSQL_TEST_PASSWORD); @@ -371,48 +257,10 @@ void testFlinkCompatibleScalarFunctions(String language, boolean batchMode) thro throw e; } - waitUntilSpecificEvent( - String.format( - "DataChangeEvent{tableId=%s.TABLEALPHA, before=[1009, 8.1, 1011, <8.1>], after=[1009, 100, 1011, <100>], op=UPDATE, meta=()}", - transformRenameDatabase.getDatabaseName()), - 20000L); - - waitUntilSpecificEvent( - String.format( - "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[3007, 7, 3009, <7>], op=INSERT, meta=()}", - transformRenameDatabase.getDatabaseName()), - 20000L); - - waitUntilSpecificEvent( - String.format( - "DataChangeEvent{tableId=%s.TABLEBETA, before=[2011, 11, Integer: 2011], after=[], op=DELETE, meta=()}", - transformRenameDatabase.getDatabaseName()), - 20000L); - } - - private void validateResult(List expectedEvents) throws Exception { - for (String event : expectedEvents) { - waitUntilSpecificEvent(event, 6000L); - } - } - - private void waitUntilSpecificEvent(String event, long timeout) throws Exception { - boolean result = false; - long endTimeout = System.currentTimeMillis() + timeout; - while (System.currentTimeMillis() < endTimeout) { - String stdout = taskManagerConsumer.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: " - + taskManagerConsumer.toUtf8String()); - } + validateResult( + dbNameFormatter, + "DataChangeEvent{tableId=%s.TABLEALPHA, before=[1009, 8.1, 1011, <8.1>], after=[1009, 100, 1011, <100>], op=UPDATE, meta=()}", + "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[3007, 7, 3009, <7>], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.TABLEBETA, before=[2011, 11, Integer: 2011], after=[], op=DELETE, meta=()}"); } } diff --git a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/migration/YamlJobMigrationITCase.java b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/migration/YamlJobMigrationITCase.java new file mode 100644 index 00000000000..ca349dfd21d --- /dev/null +++ b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/migration/YamlJobMigrationITCase.java @@ -0,0 +1,235 @@ +/* + * 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.migration; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.cdc.connectors.mysql.testutils.MySqlContainer; +import org.apache.flink.cdc.connectors.mysql.testutils.UniqueDatabase; +import org.apache.flink.cdc.pipeline.tests.utils.PipelineTestEnvironment; +import org.apache.flink.cdc.pipeline.tests.utils.TarballFetcher; + +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.EnumSource; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.SQLException; +import java.sql.Statement; +import java.time.Duration; +import java.util.function.Function; + +/** E2e cases for stopping & restarting jobs from previous state. */ +class YamlJobMigrationITCase extends PipelineTestEnvironment { + + private static final Logger LOG = LoggerFactory.getLogger(YamlJobMigrationITCase.class); + + protected UniqueDatabase mysqlInventoryDatabase; + private final Function dbNameFormatter = + (s) -> String.format(s, mysqlInventoryDatabase.getDatabaseName()); + + @BeforeEach + public void before() throws Exception { + super.before(); + mysqlInventoryDatabase = + new UniqueDatabase(MYSQL, "mysql_inventory", MYSQL_TEST_USER, MYSQL_TEST_PASSWORD); + mysqlInventoryDatabase.createAndInitialize(); + } + + @AfterEach + public void after() { + super.after(); + if (mysqlInventoryDatabase != null) { + mysqlInventoryDatabase.dropDatabase(); + } + } + + @Test + void testBasicJobSubmitting() throws Exception { + String content = + String.format( + "source:\n" + + " type: values\n" + + "\n" + + "sink:\n" + + " type: values\n" + + "\n" + + "pipeline:\n" + + " parallelism: %d\n", + parallelism); + JobID jobID = submitPipelineJob(content); + Assertions.assertThat(jobID).isNotNull(); + LOG.info("Submitted Job ID is {} ", jobID); + + validateResult( + dbNameFormatter, + "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=()}"); + LOG.info("Snapshot phase successfully finished."); + + waitUntilJobFinished(Duration.ofSeconds(30)); + LOG.info("Job gracefully stopped."); + } + + @ParameterizedTest(name = "{0} -> SNAPSHOT") + @EnumSource(names = {"V3_2_1", "V3_3_0", "SNAPSHOT"}) + void testStartingJobFromSavepoint(TarballFetcher.CdcVersion migrateFromVersion) + throws Exception { + TarballFetcher.fetch(jobManager, migrateFromVersion); + LOG.info("Successfully fetched CDC {}.", migrateFromVersion); + + String content = + String.format( + "source:\n" + + " type: mysql\n" + + " hostname: %s\n" + + " port: %d\n" + + " username: %s\n" + + " password: %s\n" + + " tables: %s.\\.*\n" + + " server-id: 5400-5404\n" + + " server-time-zone: UTC\n" + + "\n" + + "sink:\n" + + " type: values\n" + + "\n" + + "pipeline:\n" + + " parallelism: %d", + INTER_CONTAINER_MYSQL_ALIAS, + MySqlContainer.MYSQL_PORT, + MYSQL_TEST_USER, + MYSQL_TEST_PASSWORD, + mysqlInventoryDatabase.getDatabaseName(), + parallelism); + JobID jobID = submitPipelineJob(migrateFromVersion, content); + Assertions.assertThat(jobID).isNotNull(); + LOG.info("Submitted Job ID is {} ", jobID); + + validateResult( + dbNameFormatter, + "CreateTableEvent{tableId=%s.customers, schema=columns={`id` INT NOT NULL,`name` VARCHAR(255) NOT NULL 'flink',`address` VARCHAR(1024),`phone_number` VARCHAR(512)}, primaryKeys=id, options=()}", + "CreateTableEvent{tableId=%s.customers, schema=columns={`id` INT NOT NULL,`name` VARCHAR(255) NOT NULL 'flink',`address` VARCHAR(1024),`phone_number` VARCHAR(512)}, primaryKeys=id, options=()}", + "CreateTableEvent{tableId=%s.customers, schema=columns={`id` INT NOT NULL,`name` VARCHAR(255) NOT NULL 'flink',`address` VARCHAR(1024),`phone_number` VARCHAR(512)}, primaryKeys=id, options=()}", + "DataChangeEvent{tableId=%s.customers, before=[], after=[103, user_3, Shanghai, 123567891234], op=INSERT, meta=()}", + "CreateTableEvent{tableId=%s.customers, schema=columns={`id` INT NOT NULL,`name` VARCHAR(255) NOT NULL 'flink',`address` VARCHAR(1024),`phone_number` VARCHAR(512)}, primaryKeys=id, options=()}", + "DataChangeEvent{tableId=%s.customers, before=[], after=[101, user_1, Shanghai, 123567891234], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.customers, before=[], after=[104, user_4, Shanghai, 123567891234], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.customers, before=[], after=[102, user_2, Shanghai, 123567891234], op=INSERT, meta=()}", + "CreateTableEvent{tableId=%s.products, schema=columns={`id` INT NOT NULL,`name` VARCHAR(255) NOT NULL 'flink',`description` VARCHAR(512),`weight` FLOAT,`enum_c` STRING 'red',`json_c` STRING,`point_c` STRING}, primaryKeys=id, options=()}", + "DataChangeEvent{tableId=%s.products, before=[], after=[104, hammer, 12oz carpenter's hammer, 0.75, white, {\"key4\": \"value4\"}, {\"coordinates\":[4,4],\"type\":\"Point\",\"srid\":0}], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.products, before=[], after=[108, jacket, water resistent black wind breaker, 0.1, null, null, null], op=INSERT, meta=()}", + "CreateTableEvent{tableId=%s.products, schema=columns={`id` INT NOT NULL,`name` VARCHAR(255) NOT NULL 'flink',`description` VARCHAR(512),`weight` FLOAT,`enum_c` STRING 'red',`json_c` STRING,`point_c` STRING}, primaryKeys=id, options=()}", + "DataChangeEvent{tableId=%s.products, before=[], after=[103, 12-pack drill bits, 12-pack of drill bits with sizes ranging from #40 to #3, 0.8, red, {\"key3\": \"value3\"}, {\"coordinates\":[3,3],\"type\":\"Point\",\"srid\":0}], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.products, before=[], after=[107, rocks, box of assorted rocks, 5.3, null, null, null], op=INSERT, meta=()}", + "CreateTableEvent{tableId=%s.products, schema=columns={`id` INT NOT NULL,`name` VARCHAR(255) NOT NULL 'flink',`description` VARCHAR(512),`weight` FLOAT,`enum_c` STRING 'red',`json_c` STRING,`point_c` STRING}, primaryKeys=id, options=()}", + "CreateTableEvent{tableId=%s.products, schema=columns={`id` INT NOT NULL,`name` VARCHAR(255) NOT NULL 'flink',`description` VARCHAR(512),`weight` FLOAT,`enum_c` STRING 'red',`json_c` STRING,`point_c` STRING}, primaryKeys=id, options=()}", + "DataChangeEvent{tableId=%s.products, before=[], after=[102, car battery, 12V car battery, 8.1, white, {\"key2\": \"value2\"}, {\"coordinates\":[2,2],\"type\":\"Point\",\"srid\":0}], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.products, before=[], after=[106, hammer, 16oz carpenter's hammer, 1.0, null, null, null], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.products, before=[], after=[101, scooter, Small 2-wheel scooter, 3.14, red, {\"key1\": \"value1\"}, {\"coordinates\":[1,1],\"type\":\"Point\",\"srid\":0}], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.products, before=[], after=[105, hammer, 14oz carpenter's hammer, 0.875, red, {\"k1\": \"v1\", \"k2\": \"v2\"}, {\"coordinates\":[5,5],\"type\":\"Point\",\"srid\":0}], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.products, before=[], after=[109, spare tire, 24 inch spare tire, 22.2, null, null, null], op=INSERT, meta=()}"); + LOG.info("Snapshot stage finished successfully."); + + generateIncrementalEventsPhaseOne(); + validateResult( + dbNameFormatter, + "DataChangeEvent{tableId=%s.products, before=[106, hammer, 16oz carpenter's hammer, 1.0, null, null, null], after=[106, hammer, 18oz carpenter hammer, 1.0, null, null, null], op=UPDATE, meta=()}", + "DataChangeEvent{tableId=%s.products, before=[107, rocks, box of assorted rocks, 5.3, null, null, null], after=[107, rocks, box of assorted rocks, 5.1, null, null, null], op=UPDATE, meta=()}"); + LOG.info("Incremental stage 1 finished successfully."); + + generateIncrementalEventsPhaseTwo(); + validateResult( + dbNameFormatter, + "AddColumnEvent{tableId=%s.products, addedColumns=[ColumnWithPosition{column=`new_col` INT, position=LAST, existedColumnName=null}]}", + "DataChangeEvent{tableId=%s.products, before=[], after=[110, jacket, water resistent white wind breaker, 0.2, null, null, null, 1], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.products, before=[], after=[111, scooter, Big 2-wheel scooter , 5.18, null, null, null, 1], op=INSERT, meta=()}"); + LOG.info("Incremental stage 2 finished successfully."); + + String savepointPath = stopJobWithSavepoint(jobID); + LOG.info("Stopped Job {} and created a savepoint at {}.", jobID, savepointPath); + + JobID newJobID = submitPipelineJob(content, savepointPath, true); + LOG.info("Reincarnated Job {} has been submitted successfully.", newJobID); + + generateIncrementalEventsPhaseThree(); + validateResult( + dbNameFormatter, + "DataChangeEvent{tableId=%s.products, before=[110, jacket, water resistent white wind breaker, 0.2, null, null, null, 1], after=[110, jacket, new water resistent white wind breaker, 0.5, null, null, null, 1], op=UPDATE, meta=()}", + "DataChangeEvent{tableId=%s.products, before=[111, scooter, Big 2-wheel scooter , 5.18, null, null, null, 1], after=[111, scooter, Big 2-wheel scooter , 5.17, null, null, null, 1], op=UPDATE, meta=()}", + "DataChangeEvent{tableId=%s.products, before=[111, scooter, Big 2-wheel scooter , 5.17, null, null, null, 1], after=[], op=DELETE, meta=()}"); + cancelJob(newJobID); + } + + private void generateIncrementalEventsPhaseOne() { + executeMySqlStatements( + mysqlInventoryDatabase, + "UPDATE products SET description='18oz carpenter hammer' WHERE id=106;", + "UPDATE products SET weight='5.1' WHERE id=107;"); + } + + private void generateIncrementalEventsPhaseTwo() { + executeMySqlStatements( + mysqlInventoryDatabase, + "UPDATE products SET description='18oz carpenter hammer' WHERE id=106;", + "UPDATE products SET weight='5.1' WHERE id=107;", + "ALTER TABLE products ADD COLUMN new_col INT;", + "INSERT INTO products VALUES (default,'jacket','water resistent white wind breaker',0.2, null, null, null, 1);", + "INSERT INTO products VALUES (default,'scooter','Big 2-wheel scooter ',5.18, null, null, null, 1);"); + } + + private void generateIncrementalEventsPhaseThree() { + executeMySqlStatements( + mysqlInventoryDatabase, + "UPDATE products SET description='new water resistent white wind breaker', weight='0.5' WHERE id=110;", + "UPDATE products SET weight='5.17' WHERE id=111;", + "DELETE FROM products WHERE id=111;"); + } + + private void executeMySqlStatements(UniqueDatabase database, String... statements) { + String mysqlJdbcUrl = + String.format( + "jdbc:mysql://%s:%s/%s", + MYSQL.getHost(), MYSQL.getDatabasePort(), database.getDatabaseName()); + try (Connection conn = + DriverManager.getConnection( + mysqlJdbcUrl, MYSQL_TEST_USER, MYSQL_TEST_PASSWORD); + Statement stat = conn.createStatement()) { + for (String sql : statements) { + try { + stat.execute(sql); + } catch (SQLException e) { + throw new RuntimeException("Failed to execute SQL statement " + sql, e); + } + } + } catch (SQLException e) { + throw new RuntimeException("Failed to execute MySQL statements.", e); + } + } +} diff --git a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/utils/PipelineTestEnvironment.java b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/utils/PipelineTestEnvironment.java index 962fb704f5b..f37acf7b753 100644 --- a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/utils/PipelineTestEnvironment.java +++ b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/utils/PipelineTestEnvironment.java @@ -17,9 +17,12 @@ 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; @@ -28,13 +31,11 @@ import org.apache.flink.table.api.ValidationException; import org.apache.flink.util.TestLogger; -import com.fasterxml.jackson.core.Version; 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.junit.jupiter.api.io.TempDir; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.testcontainers.DockerClientFactory; @@ -44,7 +45,10 @@ 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; @@ -57,12 +61,14 @@ 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.List; import java.util.Objects; import java.util.concurrent.TimeUnit; -import java.util.stream.Collectors; +import java.util.concurrent.TimeoutException; +import java.util.function.Function; import java.util.stream.Stream; import static org.apache.flink.util.Preconditions.checkState; @@ -70,9 +76,10 @@ /** Test environment running pipeline job on Flink containers. */ @Testcontainers public abstract class PipelineTestEnvironment extends TestLogger { + private static final Logger LOG = LoggerFactory.getLogger(PipelineTestEnvironment.class); - public Integer parallelism = getParallelism(); + protected Integer parallelism = getParallelism(); private int getParallelism() { try { @@ -85,13 +92,37 @@ private int getParallelism() { } } + // ------------------------------------------------------------------------------------------ + // 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("mysql") + .withLogConsumer(new Slf4jLogConsumer(LOG)); + // ------------------------------------------------------------------------------------------ // Flink Variables // ------------------------------------------------------------------------------------------ - public static final int JOB_MANAGER_REST_PORT = 8081; - public static final String INTER_CONTAINER_JM_ALIAS = "jobmanager"; - public static final String INTER_CONTAINER_TM_ALIAS = "taskmanager"; - public static final List EXTERNAL_PROPS = + 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", @@ -106,13 +137,10 @@ private int getParallelism() { "execution.checkpointing.interval: 300", "state.backend.type: hashmap", "env.java.opts.all: -Doracle.jdbc.timezoneAsRegion=false", + "execution.checkpointing.savepoint-dir: file:///opt/flink", "restart-strategy.type: off"); public static final String FLINK_PROPERTIES = String.join("\n", EXTERNAL_PROPS); - public static final Network NETWORK = Network.newNetwork(); - - @TempDir public Path temporaryFolder; - @Nullable protected RestClusterClient restClusterClient; protected GenericContainer jobManager; @@ -164,6 +192,9 @@ public void before() throws Exception { 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 @@ -179,51 +210,114 @@ public void after() { } } - /** Allow overriding the default flink properties. */ - public void overrideFlinkProperties(String properties) { - jobManager.withEnv("FLINK_PROPERTIES", properties); - taskManager.withEnv("FLINK_PROPERTIES", properties); + /** + * 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 SQL job to the running cluster. - * - *

NOTE: You should not use {@code '\t'}. + * Submits a YAML job to the running cluster with specific CDC version, without from previous + * savepoint states. */ - public void submitPipelineJob(String pipelineJob, Path... jars) - throws IOException, InterruptedException { - for (Path jar : jars) { - jobManager.copyFileToContainer( - MountableFile.forHostPath(jar), "/tmp/flinkCDC/lib/" + jar.getFileName()); + 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); } - jobManager.copyFileToContainer( - MountableFile.forHostPath( - TestUtils.getResource("flink-cdc.sh", "flink-cdc-dist", "src"), 755), - "/tmp/flinkCDC/bin/flink-cdc.sh"); - jobManager.copyFileToContainer( - MountableFile.forHostPath( - TestUtils.getResource("flink-cdc.yaml", "flink-cdc-dist", "src"), 755), - "/tmp/flinkCDC/conf/flink-cdc.yaml"); - jobManager.copyFileToContainer( - MountableFile.forHostPath(TestUtils.getResource("flink-cdc-dist.jar")), - "/tmp/flinkCDC/lib/flink-cdc-dist.jar"); - Path script = Files.createFile(temporaryFolder.resolve("pipeline.yaml")); - Files.write(script, pipelineJob.getBytes()); - jobManager.copyFileToContainer( - MountableFile.forHostPath(script), "/tmp/flinkCDC/conf/pipeline.yaml"); + + // Attach default MySQL and Values connectors + containerPaths.add(version.workDir() + "/lib/mysql-cdc-pipeline-connector.jar"); + containerPaths.add(version.workDir() + "/lib/values-cdc-pipeline-connector.jar"); + StringBuilder sb = new StringBuilder(); - for (Path jar : jars) { - sb.append(" --jar /tmp/flinkCDC/lib/").append(jar.getFileName()); + for (String containerPath : containerPaths) { + sb.append(" --jar ").append(containerPath); } + + jobManager.copyFileToContainer( + Transferable.of(pipelineJob), version.workDir() + "/conf/pipeline.yaml"); + String commands = - "/tmp/flinkCDC/bin/flink-cdc.sh /tmp/flinkCDC/conf/pipeline.yaml --flink-home /opt/flink" + version.workDir() + + "/bin/flink-cdc.sh " + + version.workDir() + + "/conf/pipeline.yaml --flink-home /opt/flink" + sb; - ExecResult execResult = jobManager.execInContainer("bash", "-c", commands); - LOG.info(execResult.getStdout()); - LOG.error(execResult.getStderr()); - if (execResult.getExitCode() != 0) { - throw new AssertionError("Failed when submitting the pipeline job."); + + 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()); } /** @@ -253,6 +347,14 @@ public RestClusterClient getRestClusterClient() { } 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()) { @@ -266,14 +368,14 @@ public void waitUntilJobRunning(Duration timeout) { if (jobStatusMessages != null && !jobStatusMessages.isEmpty()) { JobStatusMessage message = jobStatusMessages.iterator().next(); JobStatus jobStatus = message.getJobState(); - if (jobStatus.isTerminalState()) { + 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 == JobStatus.RUNNING) { + } else if (jobStatus == expectedStatus) { return; } } @@ -284,26 +386,26 @@ protected String getFlinkDockerImageTag() { return String.format("flink:%s-scala_2.12", flinkVersion); } - private static Version parseVersion(String version) { - List versionParts = - Arrays.stream(version.split("\\.")) - .map(Integer::valueOf) - .limit(3) - .collect(Collectors.toList()); - return new Version( - versionParts.get(0), versionParts.get(1), versionParts.get(2), null, null, null); - } - - private static String getFlinkProperties() { - return String.join( - "\n", - Arrays.asList( - "restart-strategy.type: off", - "jobmanager.rpc.address: jobmanager", - "taskmanager.numberOfTaskSlots: 10", - "parallelism.default: 4", - "execution.checkpointing.interval: 300", - "env.java.opts.all: -Doracle.jdbc.timezoneAsRegion=false")); + 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. Exit code: " + + execResult.getExitCode()); + } + } catch (Exception e) { + throw new RuntimeException( + "Failed to execute command " + joinedCommand + " in container " + container); + } } public void runInContainerAsRoot(GenericContainer container, String... command) @@ -329,4 +431,46 @@ protected List readLines(String resource) throws IOException { 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 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/src/test/java/org/apache/flink/cdc/pipeline/tests/utils/TarballFetcher.java b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/utils/TarballFetcher.java new file mode 100644 index 00000000000..573a4797c87 --- /dev/null +++ b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/utils/TarballFetcher.java @@ -0,0 +1,157 @@ +/* + * 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"), 755), + version.workDir() + "/bin/flink-cdc.sh"); + container.copyFileToContainer( + MountableFile.forHostPath( + TestUtils.getResource("flink-cdc.yaml", "flink-cdc-dist", "src"), 755), + 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("mysql-cdc-pipeline-connector.jar")), + version.workDir() + "/lib/mysql-cdc-pipeline-connector.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("mysql"), + version.workDir() + "/lib/mysql-cdc-pipeline-connector.jar"); + 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"), + 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-migration-tests/flink-cdc-migration-testcases/pom.xml b/flink-cdc-migration-tests/flink-cdc-migration-testcases/pom.xml deleted file mode 100644 index 58c3cae6763..00000000000 --- a/flink-cdc-migration-tests/flink-cdc-migration-testcases/pom.xml +++ /dev/null @@ -1,54 +0,0 @@ - - - - 4.0.0 - - org.apache.flink - flink-cdc-migration-tests - ${revision} - - - flink-cdc-migration-testcases - flink-cdc-migration-testcases - - - UTF-8 - - - - org.apache.flink - flink-cdc-release-3.2.0 - ${revision} - compile - - - org.apache.flink - flink-cdc-release-3.2.1 - ${revision} - compile - - - org.apache.flink - flink-cdc-release-snapshot - ${revision} - compile - - - \ No newline at end of file diff --git a/flink-cdc-migration-tests/flink-cdc-migration-testcases/src/test/java/org/apache/flink/cdc/migration/tests/MigrationTestBase.java b/flink-cdc-migration-tests/flink-cdc-migration-testcases/src/test/java/org/apache/flink/cdc/migration/tests/MigrationTestBase.java deleted file mode 100644 index dc7c841a8ed..00000000000 --- a/flink-cdc-migration-tests/flink-cdc-migration-testcases/src/test/java/org/apache/flink/cdc/migration/tests/MigrationTestBase.java +++ /dev/null @@ -1,116 +0,0 @@ -/* - * 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.migration.tests; - -import org.assertj.core.api.Assertions; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.Arrays; -import java.util.List; -import java.util.stream.Collectors; - -/** Utilities for migration tests. */ -public class MigrationTestBase { - - private static final Logger LOG = LoggerFactory.getLogger(MigrationTestBase.class); - - /** Flink CDC versions since 3.0. */ - public enum FlinkCdcVersion { - v3_2_0, - v3_2_1, - SNAPSHOT; - - public String getShadedClassPrefix() { - switch (this) { - case v3_2_0: - return "org.apache.flink.cdc.v3_2_0"; - case v3_2_1: - return "org.apache.flink.cdc.v3_2_1"; - case SNAPSHOT: - return "org.apache.flink.cdc.snapshot"; - default: - throw new RuntimeException("Unknown Flink CDC version: " + this); - } - } - } - - private static final List versions = - Arrays.asList(FlinkCdcVersion.v3_2_0, FlinkCdcVersion.v3_2_1, FlinkCdcVersion.SNAPSHOT); - - public static List getAllVersions() { - return versions.subList(0, versions.size()); - } - - public static List getVersionSince(FlinkCdcVersion sinceVersion) { - return versions.subList(versions.indexOf(sinceVersion), versions.size()); - } - - public static List getAllVersionExcept(FlinkCdcVersion... excludedVersions) { - List excluded = Arrays.asList(excludedVersions); - return versions.stream().filter(e -> !excluded.contains(e)).collect(Collectors.toList()); - } - - public static FlinkCdcVersion getSnapshotVersion() { - return versions.get(versions.size() - 1); - } - - private static Class getMockClass(FlinkCdcVersion version, String caseName) - throws Exception { - return Class.forName(version.getShadedClassPrefix() + ".migration.tests." + caseName); - } - - protected void testMigrationFromTo( - FlinkCdcVersion fromVersion, FlinkCdcVersion toVersion, String caseName) - throws Exception { - - LOG.info("Testing {} compatibility case from {} -> {}", caseName, fromVersion, toVersion); - - // Serialize dummy object to bytes in early versions - Class fromVersionMockClass = getMockClass(fromVersion, caseName); - Object fromVersionMockObject = fromVersionMockClass.newInstance(); - - int serializerVersion = - (int) - fromVersionMockClass - .getDeclaredMethod("getSerializerVersion") - .invoke(fromVersionMockObject); - byte[] serializedObject = - (byte[]) - fromVersionMockClass - .getDeclaredMethod("serializeObject") - .invoke(fromVersionMockObject); - - // Deserialize object in latest versions - Class toVersionMockClass = getMockClass(toVersion, caseName); - Object toVersionMockObject = toVersionMockClass.newInstance(); - - Assertions.assertThat( - (boolean) - toVersionMockClass - .getDeclaredMethod( - "deserializeAndCheckObject", - int.class, - byte[].class) - .invoke( - toVersionMockObject, - serializerVersion, - serializedObject)) - .isTrue(); - } -} diff --git a/flink-cdc-migration-tests/flink-cdc-migration-testcases/src/test/java/org/apache/flink/cdc/migration/tests/SchemaManagerMigrationTest.java b/flink-cdc-migration-tests/flink-cdc-migration-testcases/src/test/java/org/apache/flink/cdc/migration/tests/SchemaManagerMigrationTest.java deleted file mode 100644 index a3a9eba8d7e..00000000000 --- a/flink-cdc-migration-tests/flink-cdc-migration-testcases/src/test/java/org/apache/flink/cdc/migration/tests/SchemaManagerMigrationTest.java +++ /dev/null @@ -1,35 +0,0 @@ -/* - * 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.migration.tests; - -import org.apache.flink.cdc.runtime.operators.schema.coordinator.SchemaManager; - -import org.junit.jupiter.api.Test; - -/** Migration test cases for {@link SchemaManager}. */ -class SchemaManagerMigrationTest extends MigrationTestBase { - - public static String mockCaseName = "SchemaManagerMigrationMock"; - - @Test - void testMigration() throws Exception { - for (FlinkCdcVersion version : getAllVersions()) { - testMigrationFromTo(version, getSnapshotVersion(), mockCaseName); - } - } -} diff --git a/flink-cdc-migration-tests/flink-cdc-migration-testcases/src/test/java/org/apache/flink/cdc/migration/tests/SchemaRegistryMigrationTest.java b/flink-cdc-migration-tests/flink-cdc-migration-testcases/src/test/java/org/apache/flink/cdc/migration/tests/SchemaRegistryMigrationTest.java deleted file mode 100644 index 11a11f69f30..00000000000 --- a/flink-cdc-migration-tests/flink-cdc-migration-testcases/src/test/java/org/apache/flink/cdc/migration/tests/SchemaRegistryMigrationTest.java +++ /dev/null @@ -1,34 +0,0 @@ -/* - * 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.migration.tests; - -import org.apache.flink.cdc.runtime.operators.schema.coordinator.SchemaRegistry; - -import org.junit.jupiter.api.Test; - -/** Migration test cases for {@link SchemaRegistry}. */ -class SchemaRegistryMigrationTest extends MigrationTestBase { - static String mockCaseName = "SchemaRegistryMigrationMock"; - - @Test - void testMigration() throws Exception { - for (FlinkCdcVersion version : getAllVersions()) { - testMigrationFromTo(version, getSnapshotVersion(), mockCaseName); - } - } -} diff --git a/flink-cdc-migration-tests/flink-cdc-migration-testcases/src/test/java/org/apache/flink/cdc/migration/tests/TableChangeInfoMigrationTest.java b/flink-cdc-migration-tests/flink-cdc-migration-testcases/src/test/java/org/apache/flink/cdc/migration/tests/TableChangeInfoMigrationTest.java deleted file mode 100644 index a2540db288d..00000000000 --- a/flink-cdc-migration-tests/flink-cdc-migration-testcases/src/test/java/org/apache/flink/cdc/migration/tests/TableChangeInfoMigrationTest.java +++ /dev/null @@ -1,33 +0,0 @@ -/* - * 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.migration.tests; - -import org.junit.jupiter.api.Test; - -/** Migration test cases for {@code PreTransformChangeInfo}. */ -class TableChangeInfoMigrationTest extends MigrationTestBase { - - static String mockCaseName = "TableChangeInfoMigrationMock"; - - @Test - void testMigration() throws Exception { - for (FlinkCdcVersion version : getAllVersions()) { - testMigrationFromTo(version, getSnapshotVersion(), mockCaseName); - } - } -} diff --git a/flink-cdc-migration-tests/flink-cdc-migration-testcases/src/test/resources/log4j2-test.properties b/flink-cdc-migration-tests/flink-cdc-migration-testcases/src/test/resources/log4j2-test.properties deleted file mode 100644 index 32df1c0251c..00000000000 --- a/flink-cdc-migration-tests/flink-cdc-migration-testcases/src/test/resources/log4j2-test.properties +++ /dev/null @@ -1,25 +0,0 @@ -################################################################################ -# 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-migration-tests/flink-cdc-release-3.2.0/pom.xml b/flink-cdc-migration-tests/flink-cdc-release-3.2.0/pom.xml deleted file mode 100644 index b26e016c378..00000000000 --- a/flink-cdc-migration-tests/flink-cdc-release-3.2.0/pom.xml +++ /dev/null @@ -1,86 +0,0 @@ - - - - 4.0.0 - - org.apache.flink - flink-cdc-migration-tests - ${revision} - - - flink-cdc-release-3.2.0 - flink-cdc-release-3.2.0 - - - - org.apache.flink - flink-cdc-base - 3.2.0 - - - org.apache.flink - flink-cdc-common - 3.2.0 - - - org.apache.flink - flink-cdc-runtime - 3.2.0 - - - - - - - org.apache.maven.plugins - maven-shade-plugin - ${maven.shade.plugin.version} - - - shade-flink-cdc - package - - shade - - - - - *:* - - META-INF/*.SF - META-INF/*.DSA - META-INF/*.RSA - - - - - - org.apache.flink.cdc - org.apache.flink.cdc.v3_2_0 - META-INF/*.SF,META-INF/*.DSA,META-INF/*.RSA - - - - - - - - - \ No newline at end of file diff --git a/flink-cdc-migration-tests/flink-cdc-release-3.2.0/src/main/java/org/apache/flink/cdc/migration/tests/MigrationMockBase.java b/flink-cdc-migration-tests/flink-cdc-release-3.2.0/src/main/java/org/apache/flink/cdc/migration/tests/MigrationMockBase.java deleted file mode 100644 index 3f52615dbde..00000000000 --- a/flink-cdc-migration-tests/flink-cdc-release-3.2.0/src/main/java/org/apache/flink/cdc/migration/tests/MigrationMockBase.java +++ /dev/null @@ -1,27 +0,0 @@ -/* - * 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.migration.tests; - -/** Base classes for migration test cases. */ -public interface MigrationMockBase { - int getSerializerVersion(); - - byte[] serializeObject() throws Exception; - - boolean deserializeAndCheckObject(int v, byte[] b) throws Exception; -} diff --git a/flink-cdc-migration-tests/flink-cdc-release-3.2.0/src/main/java/org/apache/flink/cdc/migration/tests/SchemaManagerMigrationMock.java b/flink-cdc-migration-tests/flink-cdc-release-3.2.0/src/main/java/org/apache/flink/cdc/migration/tests/SchemaManagerMigrationMock.java deleted file mode 100644 index 87f7d03ad4c..00000000000 --- a/flink-cdc-migration-tests/flink-cdc-release-3.2.0/src/main/java/org/apache/flink/cdc/migration/tests/SchemaManagerMigrationMock.java +++ /dev/null @@ -1,91 +0,0 @@ -/* - * 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.migration.tests; - -import org.apache.flink.cdc.common.event.TableId; -import org.apache.flink.cdc.common.pipeline.SchemaChangeBehavior; -import org.apache.flink.cdc.common.schema.Schema; -import org.apache.flink.cdc.common.types.DataTypes; -import org.apache.flink.cdc.runtime.operators.schema.coordinator.SchemaManager; - -import java.util.HashMap; -import java.util.Map; -import java.util.SortedMap; -import java.util.TreeMap; - -/** Dummy classes for migration test. Called via reflection. */ -public class SchemaManagerMigrationMock implements MigrationMockBase { - private static final TableId TABLE_1 = TableId.tableId("ns", "scm", "tbl1"); - private static final TableId TABLE_2 = TableId.tableId("ns", "scm", "tbl2"); - - private static final String SCHEMA_MANAGER = - "runtime.operators.schema.coordinator.SchemaManager"; - - private static Schema genSchema(String identifier) { - return Schema.newBuilder() - .physicalColumn("id", DataTypes.INT()) - .physicalColumn("col_" + identifier, DataTypes.STRING()) - .primaryKey("id") - .build(); - } - - private static final Map> ORIGINAL_SCHEMA_MAP; - private static final Map> EVOLVED_SCHEMA_MAP; - - static { - SortedMap originalSchemas = new TreeMap<>(); - originalSchemas.put(1, genSchema("upstream_1")); - originalSchemas.put(2, genSchema("upstream_2")); - originalSchemas.put(3, genSchema("upstream_3")); - - SortedMap evolvedSchemas = new TreeMap<>(); - evolvedSchemas.put(1, genSchema("evolved_1")); - evolvedSchemas.put(2, genSchema("evolved_2")); - evolvedSchemas.put(3, genSchema("evolved_3")); - - ORIGINAL_SCHEMA_MAP = new HashMap<>(); - ORIGINAL_SCHEMA_MAP.put(TABLE_1, originalSchemas); - ORIGINAL_SCHEMA_MAP.put(TABLE_2, originalSchemas); - - EVOLVED_SCHEMA_MAP = new HashMap<>(); - EVOLVED_SCHEMA_MAP.put(TABLE_1, evolvedSchemas); - EVOLVED_SCHEMA_MAP.put(TABLE_2, evolvedSchemas); - } - - public SchemaManager generateDummyObject() { - return new SchemaManager( - ORIGINAL_SCHEMA_MAP, EVOLVED_SCHEMA_MAP, SchemaChangeBehavior.TRY_EVOLVE); - } - - @Override - public int getSerializerVersion() { - return SchemaManager.SERIALIZER.getVersion(); - } - - @Override - public byte[] serializeObject() throws Exception { - return SchemaManager.SERIALIZER.serialize(generateDummyObject()); - } - - @Override - public boolean deserializeAndCheckObject(int version, byte[] serialized) throws Exception { - Object expected = generateDummyObject(); - Object actual = SchemaManager.SERIALIZER.deserialize(version, serialized); - return expected.equals(actual); - } -} diff --git a/flink-cdc-migration-tests/flink-cdc-release-3.2.0/src/main/java/org/apache/flink/cdc/migration/tests/SchemaRegistryMigrationMock.java b/flink-cdc-migration-tests/flink-cdc-release-3.2.0/src/main/java/org/apache/flink/cdc/migration/tests/SchemaRegistryMigrationMock.java deleted file mode 100644 index 41f0d8ac0bd..00000000000 --- a/flink-cdc-migration-tests/flink-cdc-release-3.2.0/src/main/java/org/apache/flink/cdc/migration/tests/SchemaRegistryMigrationMock.java +++ /dev/null @@ -1,181 +0,0 @@ -/* - * 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.migration.tests; - -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.cdc.common.event.TableId; -import org.apache.flink.cdc.common.pipeline.SchemaChangeBehavior; -import org.apache.flink.cdc.common.schema.Schema; -import org.apache.flink.cdc.common.schema.Selectors; -import org.apache.flink.cdc.common.types.DataTypes; -import org.apache.flink.cdc.runtime.operators.schema.coordinator.SchemaDerivation; -import org.apache.flink.cdc.runtime.operators.schema.coordinator.SchemaManager; -import org.apache.flink.cdc.runtime.operators.schema.coordinator.SchemaRegistry; - -import java.lang.reflect.Field; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.SortedMap; -import java.util.TreeMap; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.Executors; - -/** Dummy classes for migration test. Called via reflection. */ -public class SchemaRegistryMigrationMock implements MigrationMockBase { - private static final TableId TABLE_1 = TableId.tableId("ns", "scm", "tbl1"); - private static final TableId TABLE_2 = TableId.tableId("ns", "scm", "tbl2"); - - private static Schema genSchema(String identifier) { - return Schema.newBuilder() - .physicalColumn("id", DataTypes.INT()) - .physicalColumn("col_" + identifier, DataTypes.STRING()) - .primaryKey("id") - .build(); - } - - private static final Map> ORIGINAL_SCHEMA_MAP; - private static final Map> EVOLVED_SCHEMA_MAP; - - static { - SortedMap originalSchemas = new TreeMap<>(); - originalSchemas.put(1, genSchema("upstream_1")); - originalSchemas.put(2, genSchema("upstream_2")); - originalSchemas.put(3, genSchema("upstream_3")); - - SortedMap evolvedSchemas = new TreeMap<>(); - evolvedSchemas.put(1, genSchema("evolved_1")); - evolvedSchemas.put(2, genSchema("evolved_2")); - evolvedSchemas.put(3, genSchema("evolved_3")); - - ORIGINAL_SCHEMA_MAP = new HashMap<>(); - ORIGINAL_SCHEMA_MAP.put(TABLE_1, originalSchemas); - ORIGINAL_SCHEMA_MAP.put(TABLE_2, originalSchemas); - - EVOLVED_SCHEMA_MAP = new HashMap<>(); - EVOLVED_SCHEMA_MAP.put(TABLE_1, evolvedSchemas); - EVOLVED_SCHEMA_MAP.put(TABLE_2, evolvedSchemas); - } - - public SchemaManager generateDummySchemaManager() { - return new SchemaManager(new HashMap<>(), new HashMap<>(), SchemaChangeBehavior.EVOLVE); - } - - public SchemaRegistry generateSchemaRegistry() { - return new SchemaRegistry( - "Dummy Name", - null, - Executors.newSingleThreadExecutor(), - e -> {}, - new ArrayList<>()); - } - - private SchemaManager getSchemaManager(SchemaRegistry schemaRegistry) throws Exception { - Field managerField = SchemaRegistry.class.getDeclaredField("schemaManager"); - managerField.setAccessible(true); - return (SchemaManager) managerField.get(schemaRegistry); - } - - @SuppressWarnings("unchecked") - private Map> getOriginalSchemaMap( - SchemaRegistry schemaRegistry) throws Exception { - SchemaManager schemaManager = getSchemaManager(schemaRegistry); - Field originalField = SchemaManager.class.getDeclaredField("originalSchemas"); - originalField.setAccessible(true); - return (Map>) originalField.get(schemaManager); - } - - private void setOriginalSchemaMap( - SchemaRegistry schemaRegistry, - Map> originalSchemaMap) - throws Exception { - SchemaManager schemaManager = getSchemaManager(schemaRegistry); - Field field = SchemaManager.class.getDeclaredField("originalSchemas"); - field.setAccessible(true); - field.set(schemaManager, originalSchemaMap); - } - - @SuppressWarnings("unchecked") - private Map> getEvolvedSchemaMap( - SchemaRegistry schemaRegistry) throws Exception { - SchemaManager schemaManager = getSchemaManager(schemaRegistry); - Field originalField = SchemaManager.class.getDeclaredField("evolvedSchemas"); - originalField.setAccessible(true); - return (Map>) originalField.get(schemaManager); - } - - private void setEvolvedSchemaMap( - SchemaRegistry schemaRegistry, - Map> evolvedSchemaMap) - throws Exception { - SchemaManager schemaManager = getSchemaManager(schemaRegistry); - Field field = SchemaManager.class.getDeclaredField("evolvedSchemas"); - field.setAccessible(true); - field.set(schemaManager, evolvedSchemaMap); - } - - private SchemaDerivation getSchemaDerivation(SchemaRegistry schemaRegistry) throws Exception { - Field field = SchemaRegistry.class.getDeclaredField("schemaDerivation"); - field.setAccessible(true); - return (SchemaDerivation) field.get(schemaRegistry); - } - - @SuppressWarnings("unchecked") - private List> getSchemaRoutes(SchemaRegistry schemaRegistry) - throws Exception { - SchemaDerivation schemaDerivation = getSchemaDerivation(schemaRegistry); - Field field = SchemaDerivation.class.getDeclaredField("routes"); - field.setAccessible(true); - return (List>) field.get(schemaDerivation); - } - - @Override - public int getSerializerVersion() { - return -1; - } - - @Override - public byte[] serializeObject() throws Exception { - CompletableFuture future = new CompletableFuture<>(); - SchemaRegistry registry = generateSchemaRegistry(); - setOriginalSchemaMap(registry, ORIGINAL_SCHEMA_MAP); - setEvolvedSchemaMap(registry, EVOLVED_SCHEMA_MAP); - - registry.checkpointCoordinator(0, future); - - while (!future.isDone()) { - Thread.sleep(1000); - } - return future.get(); - } - - @Override - public boolean deserializeAndCheckObject(int v, byte[] b) throws Exception { - SchemaRegistry expected = generateSchemaRegistry(); - setOriginalSchemaMap(expected, ORIGINAL_SCHEMA_MAP); - setEvolvedSchemaMap(expected, EVOLVED_SCHEMA_MAP); - - SchemaRegistry actual = generateSchemaRegistry(); - actual.resetToCheckpoint(0, b); - - return getOriginalSchemaMap(expected).equals(getOriginalSchemaMap(actual)) - && getEvolvedSchemaMap(expected).equals(getEvolvedSchemaMap(actual)) - && getSchemaRoutes(expected).equals(getSchemaRoutes(actual)); - } -} diff --git a/flink-cdc-migration-tests/flink-cdc-release-3.2.0/src/main/java/org/apache/flink/cdc/migration/tests/TableChangeInfoMigrationMock.java b/flink-cdc-migration-tests/flink-cdc-release-3.2.0/src/main/java/org/apache/flink/cdc/migration/tests/TableChangeInfoMigrationMock.java deleted file mode 100644 index 1317c73ebf4..00000000000 --- a/flink-cdc-migration-tests/flink-cdc-release-3.2.0/src/main/java/org/apache/flink/cdc/migration/tests/TableChangeInfoMigrationMock.java +++ /dev/null @@ -1,61 +0,0 @@ -/* - * 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.migration.tests; - -import org.apache.flink.cdc.common.event.TableId; -import org.apache.flink.cdc.common.schema.Schema; -import org.apache.flink.cdc.common.types.DataTypes; -import org.apache.flink.cdc.runtime.operators.transform.PreTransformChangeInfo; - -/** Dummy classes for migration test. Called via reflection. */ -public class TableChangeInfoMigrationMock implements MigrationMockBase { - private static final TableId DUMMY_TABLE_ID = - TableId.tableId("dummyNamespace", "dummySchema", "dummyTable"); - private static final Schema DUMMY_SCHEMA = - Schema.newBuilder() - .physicalColumn("id", DataTypes.INT()) - .physicalColumn("name", DataTypes.STRING()) - .physicalColumn("age", DataTypes.DOUBLE()) - .primaryKey("id", "name") - .build(); - - public PreTransformChangeInfo generateDummyObject() { - return PreTransformChangeInfo.of(DUMMY_TABLE_ID, DUMMY_SCHEMA, DUMMY_SCHEMA); - } - - @Override - public int getSerializerVersion() { - return PreTransformChangeInfo.SERIALIZER.getVersion(); - } - - @Override - public byte[] serializeObject() throws Exception { - return PreTransformChangeInfo.SERIALIZER.serialize(generateDummyObject()); - } - - @Override - public boolean deserializeAndCheckObject(int version, byte[] bytes) throws Exception { - PreTransformChangeInfo expected = generateDummyObject(); - PreTransformChangeInfo actual = - PreTransformChangeInfo.SERIALIZER.deserialize(version, bytes); - - return expected.getTableId().equals(actual.getTableId()) - && expected.getSourceSchema().equals(actual.getSourceSchema()) - && expected.getPreTransformedSchema().equals(actual.getPreTransformedSchema()); - } -} diff --git a/flink-cdc-migration-tests/flink-cdc-release-3.2.1/pom.xml b/flink-cdc-migration-tests/flink-cdc-release-3.2.1/pom.xml deleted file mode 100644 index 545542ff20d..00000000000 --- a/flink-cdc-migration-tests/flink-cdc-release-3.2.1/pom.xml +++ /dev/null @@ -1,86 +0,0 @@ - - - - 4.0.0 - - org.apache.flink - flink-cdc-migration-tests - ${revision} - - - flink-cdc-release-3.2.1 - flink-cdc-release-3.2.1 - - - - org.apache.flink - flink-cdc-base - 3.2.1 - - - org.apache.flink - flink-cdc-common - 3.2.1 - - - org.apache.flink - flink-cdc-runtime - 3.2.1 - - - - - - - org.apache.maven.plugins - maven-shade-plugin - ${maven.shade.plugin.version} - - - shade-flink-cdc - package - - shade - - - - - *:* - - META-INF/*.SF - META-INF/*.DSA - META-INF/*.RSA - - - - - - org.apache.flink.cdc - org.apache.flink.cdc.v3_2_1 - META-INF/*.SF,META-INF/*.DSA,META-INF/*.RSA - - - - - - - - - \ No newline at end of file diff --git a/flink-cdc-migration-tests/flink-cdc-release-3.2.1/src/main/java/org/apache/flink/cdc/migration/tests/MigrationMockBase.java b/flink-cdc-migration-tests/flink-cdc-release-3.2.1/src/main/java/org/apache/flink/cdc/migration/tests/MigrationMockBase.java deleted file mode 100644 index 3f52615dbde..00000000000 --- a/flink-cdc-migration-tests/flink-cdc-release-3.2.1/src/main/java/org/apache/flink/cdc/migration/tests/MigrationMockBase.java +++ /dev/null @@ -1,27 +0,0 @@ -/* - * 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.migration.tests; - -/** Base classes for migration test cases. */ -public interface MigrationMockBase { - int getSerializerVersion(); - - byte[] serializeObject() throws Exception; - - boolean deserializeAndCheckObject(int v, byte[] b) throws Exception; -} diff --git a/flink-cdc-migration-tests/flink-cdc-release-3.2.1/src/main/java/org/apache/flink/cdc/migration/tests/SchemaManagerMigrationMock.java b/flink-cdc-migration-tests/flink-cdc-release-3.2.1/src/main/java/org/apache/flink/cdc/migration/tests/SchemaManagerMigrationMock.java deleted file mode 100644 index 87f7d03ad4c..00000000000 --- a/flink-cdc-migration-tests/flink-cdc-release-3.2.1/src/main/java/org/apache/flink/cdc/migration/tests/SchemaManagerMigrationMock.java +++ /dev/null @@ -1,91 +0,0 @@ -/* - * 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.migration.tests; - -import org.apache.flink.cdc.common.event.TableId; -import org.apache.flink.cdc.common.pipeline.SchemaChangeBehavior; -import org.apache.flink.cdc.common.schema.Schema; -import org.apache.flink.cdc.common.types.DataTypes; -import org.apache.flink.cdc.runtime.operators.schema.coordinator.SchemaManager; - -import java.util.HashMap; -import java.util.Map; -import java.util.SortedMap; -import java.util.TreeMap; - -/** Dummy classes for migration test. Called via reflection. */ -public class SchemaManagerMigrationMock implements MigrationMockBase { - private static final TableId TABLE_1 = TableId.tableId("ns", "scm", "tbl1"); - private static final TableId TABLE_2 = TableId.tableId("ns", "scm", "tbl2"); - - private static final String SCHEMA_MANAGER = - "runtime.operators.schema.coordinator.SchemaManager"; - - private static Schema genSchema(String identifier) { - return Schema.newBuilder() - .physicalColumn("id", DataTypes.INT()) - .physicalColumn("col_" + identifier, DataTypes.STRING()) - .primaryKey("id") - .build(); - } - - private static final Map> ORIGINAL_SCHEMA_MAP; - private static final Map> EVOLVED_SCHEMA_MAP; - - static { - SortedMap originalSchemas = new TreeMap<>(); - originalSchemas.put(1, genSchema("upstream_1")); - originalSchemas.put(2, genSchema("upstream_2")); - originalSchemas.put(3, genSchema("upstream_3")); - - SortedMap evolvedSchemas = new TreeMap<>(); - evolvedSchemas.put(1, genSchema("evolved_1")); - evolvedSchemas.put(2, genSchema("evolved_2")); - evolvedSchemas.put(3, genSchema("evolved_3")); - - ORIGINAL_SCHEMA_MAP = new HashMap<>(); - ORIGINAL_SCHEMA_MAP.put(TABLE_1, originalSchemas); - ORIGINAL_SCHEMA_MAP.put(TABLE_2, originalSchemas); - - EVOLVED_SCHEMA_MAP = new HashMap<>(); - EVOLVED_SCHEMA_MAP.put(TABLE_1, evolvedSchemas); - EVOLVED_SCHEMA_MAP.put(TABLE_2, evolvedSchemas); - } - - public SchemaManager generateDummyObject() { - return new SchemaManager( - ORIGINAL_SCHEMA_MAP, EVOLVED_SCHEMA_MAP, SchemaChangeBehavior.TRY_EVOLVE); - } - - @Override - public int getSerializerVersion() { - return SchemaManager.SERIALIZER.getVersion(); - } - - @Override - public byte[] serializeObject() throws Exception { - return SchemaManager.SERIALIZER.serialize(generateDummyObject()); - } - - @Override - public boolean deserializeAndCheckObject(int version, byte[] serialized) throws Exception { - Object expected = generateDummyObject(); - Object actual = SchemaManager.SERIALIZER.deserialize(version, serialized); - return expected.equals(actual); - } -} diff --git a/flink-cdc-migration-tests/flink-cdc-release-3.2.1/src/main/java/org/apache/flink/cdc/migration/tests/SchemaRegistryMigrationMock.java b/flink-cdc-migration-tests/flink-cdc-release-3.2.1/src/main/java/org/apache/flink/cdc/migration/tests/SchemaRegistryMigrationMock.java deleted file mode 100644 index 41f0d8ac0bd..00000000000 --- a/flink-cdc-migration-tests/flink-cdc-release-3.2.1/src/main/java/org/apache/flink/cdc/migration/tests/SchemaRegistryMigrationMock.java +++ /dev/null @@ -1,181 +0,0 @@ -/* - * 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.migration.tests; - -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.cdc.common.event.TableId; -import org.apache.flink.cdc.common.pipeline.SchemaChangeBehavior; -import org.apache.flink.cdc.common.schema.Schema; -import org.apache.flink.cdc.common.schema.Selectors; -import org.apache.flink.cdc.common.types.DataTypes; -import org.apache.flink.cdc.runtime.operators.schema.coordinator.SchemaDerivation; -import org.apache.flink.cdc.runtime.operators.schema.coordinator.SchemaManager; -import org.apache.flink.cdc.runtime.operators.schema.coordinator.SchemaRegistry; - -import java.lang.reflect.Field; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.SortedMap; -import java.util.TreeMap; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.Executors; - -/** Dummy classes for migration test. Called via reflection. */ -public class SchemaRegistryMigrationMock implements MigrationMockBase { - private static final TableId TABLE_1 = TableId.tableId("ns", "scm", "tbl1"); - private static final TableId TABLE_2 = TableId.tableId("ns", "scm", "tbl2"); - - private static Schema genSchema(String identifier) { - return Schema.newBuilder() - .physicalColumn("id", DataTypes.INT()) - .physicalColumn("col_" + identifier, DataTypes.STRING()) - .primaryKey("id") - .build(); - } - - private static final Map> ORIGINAL_SCHEMA_MAP; - private static final Map> EVOLVED_SCHEMA_MAP; - - static { - SortedMap originalSchemas = new TreeMap<>(); - originalSchemas.put(1, genSchema("upstream_1")); - originalSchemas.put(2, genSchema("upstream_2")); - originalSchemas.put(3, genSchema("upstream_3")); - - SortedMap evolvedSchemas = new TreeMap<>(); - evolvedSchemas.put(1, genSchema("evolved_1")); - evolvedSchemas.put(2, genSchema("evolved_2")); - evolvedSchemas.put(3, genSchema("evolved_3")); - - ORIGINAL_SCHEMA_MAP = new HashMap<>(); - ORIGINAL_SCHEMA_MAP.put(TABLE_1, originalSchemas); - ORIGINAL_SCHEMA_MAP.put(TABLE_2, originalSchemas); - - EVOLVED_SCHEMA_MAP = new HashMap<>(); - EVOLVED_SCHEMA_MAP.put(TABLE_1, evolvedSchemas); - EVOLVED_SCHEMA_MAP.put(TABLE_2, evolvedSchemas); - } - - public SchemaManager generateDummySchemaManager() { - return new SchemaManager(new HashMap<>(), new HashMap<>(), SchemaChangeBehavior.EVOLVE); - } - - public SchemaRegistry generateSchemaRegistry() { - return new SchemaRegistry( - "Dummy Name", - null, - Executors.newSingleThreadExecutor(), - e -> {}, - new ArrayList<>()); - } - - private SchemaManager getSchemaManager(SchemaRegistry schemaRegistry) throws Exception { - Field managerField = SchemaRegistry.class.getDeclaredField("schemaManager"); - managerField.setAccessible(true); - return (SchemaManager) managerField.get(schemaRegistry); - } - - @SuppressWarnings("unchecked") - private Map> getOriginalSchemaMap( - SchemaRegistry schemaRegistry) throws Exception { - SchemaManager schemaManager = getSchemaManager(schemaRegistry); - Field originalField = SchemaManager.class.getDeclaredField("originalSchemas"); - originalField.setAccessible(true); - return (Map>) originalField.get(schemaManager); - } - - private void setOriginalSchemaMap( - SchemaRegistry schemaRegistry, - Map> originalSchemaMap) - throws Exception { - SchemaManager schemaManager = getSchemaManager(schemaRegistry); - Field field = SchemaManager.class.getDeclaredField("originalSchemas"); - field.setAccessible(true); - field.set(schemaManager, originalSchemaMap); - } - - @SuppressWarnings("unchecked") - private Map> getEvolvedSchemaMap( - SchemaRegistry schemaRegistry) throws Exception { - SchemaManager schemaManager = getSchemaManager(schemaRegistry); - Field originalField = SchemaManager.class.getDeclaredField("evolvedSchemas"); - originalField.setAccessible(true); - return (Map>) originalField.get(schemaManager); - } - - private void setEvolvedSchemaMap( - SchemaRegistry schemaRegistry, - Map> evolvedSchemaMap) - throws Exception { - SchemaManager schemaManager = getSchemaManager(schemaRegistry); - Field field = SchemaManager.class.getDeclaredField("evolvedSchemas"); - field.setAccessible(true); - field.set(schemaManager, evolvedSchemaMap); - } - - private SchemaDerivation getSchemaDerivation(SchemaRegistry schemaRegistry) throws Exception { - Field field = SchemaRegistry.class.getDeclaredField("schemaDerivation"); - field.setAccessible(true); - return (SchemaDerivation) field.get(schemaRegistry); - } - - @SuppressWarnings("unchecked") - private List> getSchemaRoutes(SchemaRegistry schemaRegistry) - throws Exception { - SchemaDerivation schemaDerivation = getSchemaDerivation(schemaRegistry); - Field field = SchemaDerivation.class.getDeclaredField("routes"); - field.setAccessible(true); - return (List>) field.get(schemaDerivation); - } - - @Override - public int getSerializerVersion() { - return -1; - } - - @Override - public byte[] serializeObject() throws Exception { - CompletableFuture future = new CompletableFuture<>(); - SchemaRegistry registry = generateSchemaRegistry(); - setOriginalSchemaMap(registry, ORIGINAL_SCHEMA_MAP); - setEvolvedSchemaMap(registry, EVOLVED_SCHEMA_MAP); - - registry.checkpointCoordinator(0, future); - - while (!future.isDone()) { - Thread.sleep(1000); - } - return future.get(); - } - - @Override - public boolean deserializeAndCheckObject(int v, byte[] b) throws Exception { - SchemaRegistry expected = generateSchemaRegistry(); - setOriginalSchemaMap(expected, ORIGINAL_SCHEMA_MAP); - setEvolvedSchemaMap(expected, EVOLVED_SCHEMA_MAP); - - SchemaRegistry actual = generateSchemaRegistry(); - actual.resetToCheckpoint(0, b); - - return getOriginalSchemaMap(expected).equals(getOriginalSchemaMap(actual)) - && getEvolvedSchemaMap(expected).equals(getEvolvedSchemaMap(actual)) - && getSchemaRoutes(expected).equals(getSchemaRoutes(actual)); - } -} diff --git a/flink-cdc-migration-tests/flink-cdc-release-3.2.1/src/main/java/org/apache/flink/cdc/migration/tests/TableChangeInfoMigrationMock.java b/flink-cdc-migration-tests/flink-cdc-release-3.2.1/src/main/java/org/apache/flink/cdc/migration/tests/TableChangeInfoMigrationMock.java deleted file mode 100644 index 1317c73ebf4..00000000000 --- a/flink-cdc-migration-tests/flink-cdc-release-3.2.1/src/main/java/org/apache/flink/cdc/migration/tests/TableChangeInfoMigrationMock.java +++ /dev/null @@ -1,61 +0,0 @@ -/* - * 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.migration.tests; - -import org.apache.flink.cdc.common.event.TableId; -import org.apache.flink.cdc.common.schema.Schema; -import org.apache.flink.cdc.common.types.DataTypes; -import org.apache.flink.cdc.runtime.operators.transform.PreTransformChangeInfo; - -/** Dummy classes for migration test. Called via reflection. */ -public class TableChangeInfoMigrationMock implements MigrationMockBase { - private static final TableId DUMMY_TABLE_ID = - TableId.tableId("dummyNamespace", "dummySchema", "dummyTable"); - private static final Schema DUMMY_SCHEMA = - Schema.newBuilder() - .physicalColumn("id", DataTypes.INT()) - .physicalColumn("name", DataTypes.STRING()) - .physicalColumn("age", DataTypes.DOUBLE()) - .primaryKey("id", "name") - .build(); - - public PreTransformChangeInfo generateDummyObject() { - return PreTransformChangeInfo.of(DUMMY_TABLE_ID, DUMMY_SCHEMA, DUMMY_SCHEMA); - } - - @Override - public int getSerializerVersion() { - return PreTransformChangeInfo.SERIALIZER.getVersion(); - } - - @Override - public byte[] serializeObject() throws Exception { - return PreTransformChangeInfo.SERIALIZER.serialize(generateDummyObject()); - } - - @Override - public boolean deserializeAndCheckObject(int version, byte[] bytes) throws Exception { - PreTransformChangeInfo expected = generateDummyObject(); - PreTransformChangeInfo actual = - PreTransformChangeInfo.SERIALIZER.deserialize(version, bytes); - - return expected.getTableId().equals(actual.getTableId()) - && expected.getSourceSchema().equals(actual.getSourceSchema()) - && expected.getPreTransformedSchema().equals(actual.getPreTransformedSchema()); - } -} diff --git a/flink-cdc-migration-tests/flink-cdc-release-snapshot/pom.xml b/flink-cdc-migration-tests/flink-cdc-release-snapshot/pom.xml deleted file mode 100644 index cbea929f401..00000000000 --- a/flink-cdc-migration-tests/flink-cdc-release-snapshot/pom.xml +++ /dev/null @@ -1,90 +0,0 @@ - - - - 4.0.0 - - org.apache.flink - flink-cdc-migration-tests - ${revision} - - - flink-cdc-release-snapshot - flink-cdc-release-snapshot - - - - org.apache.flink - flink-cdc-base - ${revision} - compile - - - org.apache.flink - flink-cdc-common - ${revision} - compile - - - org.apache.flink - flink-cdc-runtime - ${revision} - compile - - - - - - - org.apache.maven.plugins - maven-shade-plugin - ${maven.shade.plugin.version} - - - shade-flink-cdc - package - - shade - - - - - *:* - - META-INF/*.SF - META-INF/*.DSA - META-INF/*.RSA - - - - - - org.apache.flink.cdc - org.apache.flink.cdc.snapshot - META-INF/*.SF,META-INF/*.DSA,META-INF/*.RSA - - - - - - - - - - \ No newline at end of file diff --git a/flink-cdc-migration-tests/flink-cdc-release-snapshot/src/main/java/org/apache/flink/cdc/migration/tests/MigrationMockBase.java b/flink-cdc-migration-tests/flink-cdc-release-snapshot/src/main/java/org/apache/flink/cdc/migration/tests/MigrationMockBase.java deleted file mode 100644 index 3f52615dbde..00000000000 --- a/flink-cdc-migration-tests/flink-cdc-release-snapshot/src/main/java/org/apache/flink/cdc/migration/tests/MigrationMockBase.java +++ /dev/null @@ -1,27 +0,0 @@ -/* - * 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.migration.tests; - -/** Base classes for migration test cases. */ -public interface MigrationMockBase { - int getSerializerVersion(); - - byte[] serializeObject() throws Exception; - - boolean deserializeAndCheckObject(int v, byte[] b) throws Exception; -} diff --git a/flink-cdc-migration-tests/flink-cdc-release-snapshot/src/main/java/org/apache/flink/cdc/migration/tests/SchemaManagerMigrationMock.java b/flink-cdc-migration-tests/flink-cdc-release-snapshot/src/main/java/org/apache/flink/cdc/migration/tests/SchemaManagerMigrationMock.java deleted file mode 100644 index 16d8ad75363..00000000000 --- a/flink-cdc-migration-tests/flink-cdc-release-snapshot/src/main/java/org/apache/flink/cdc/migration/tests/SchemaManagerMigrationMock.java +++ /dev/null @@ -1,91 +0,0 @@ -/* - * 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.migration.tests; - -import org.apache.flink.cdc.common.event.TableId; -import org.apache.flink.cdc.common.pipeline.SchemaChangeBehavior; -import org.apache.flink.cdc.common.schema.Schema; -import org.apache.flink.cdc.common.types.DataTypes; -import org.apache.flink.cdc.runtime.operators.schema.common.SchemaManager; - -import java.util.HashMap; -import java.util.Map; -import java.util.SortedMap; -import java.util.TreeMap; - -/** Dummy classes for migration test. Called via reflection. */ -public class SchemaManagerMigrationMock implements MigrationMockBase { - private static final TableId TABLE_1 = TableId.tableId("ns", "scm", "tbl1"); - private static final TableId TABLE_2 = TableId.tableId("ns", "scm", "tbl2"); - - private static final String SCHEMA_MANAGER = - "runtime.operators.schema.coordinator.SchemaManager"; - - private static Schema genSchema(String identifier) { - return Schema.newBuilder() - .physicalColumn("id", DataTypes.INT()) - .physicalColumn("col_" + identifier, DataTypes.STRING()) - .primaryKey("id") - .build(); - } - - private static final Map> ORIGINAL_SCHEMA_MAP; - private static final Map> EVOLVED_SCHEMA_MAP; - - static { - SortedMap originalSchemas = new TreeMap<>(); - originalSchemas.put(1, genSchema("upstream_1")); - originalSchemas.put(2, genSchema("upstream_2")); - originalSchemas.put(3, genSchema("upstream_3")); - - SortedMap evolvedSchemas = new TreeMap<>(); - evolvedSchemas.put(1, genSchema("evolved_1")); - evolvedSchemas.put(2, genSchema("evolved_2")); - evolvedSchemas.put(3, genSchema("evolved_3")); - - ORIGINAL_SCHEMA_MAP = new HashMap<>(); - ORIGINAL_SCHEMA_MAP.put(TABLE_1, originalSchemas); - ORIGINAL_SCHEMA_MAP.put(TABLE_2, originalSchemas); - - EVOLVED_SCHEMA_MAP = new HashMap<>(); - EVOLVED_SCHEMA_MAP.put(TABLE_1, evolvedSchemas); - EVOLVED_SCHEMA_MAP.put(TABLE_2, evolvedSchemas); - } - - public SchemaManager generateDummyObject() { - return new SchemaManager( - ORIGINAL_SCHEMA_MAP, EVOLVED_SCHEMA_MAP, SchemaChangeBehavior.TRY_EVOLVE); - } - - @Override - public int getSerializerVersion() { - return SchemaManager.SERIALIZER.getVersion(); - } - - @Override - public byte[] serializeObject() throws Exception { - return SchemaManager.SERIALIZER.serialize(generateDummyObject()); - } - - @Override - public boolean deserializeAndCheckObject(int version, byte[] serialized) throws Exception { - Object expected = generateDummyObject(); - Object actual = SchemaManager.SERIALIZER.deserialize(version, serialized); - return expected.equals(actual); - } -} diff --git a/flink-cdc-migration-tests/flink-cdc-release-snapshot/src/main/java/org/apache/flink/cdc/migration/tests/SchemaRegistryMigrationMock.java b/flink-cdc-migration-tests/flink-cdc-release-snapshot/src/main/java/org/apache/flink/cdc/migration/tests/SchemaRegistryMigrationMock.java deleted file mode 100644 index b9c8c2d1408..00000000000 --- a/flink-cdc-migration-tests/flink-cdc-release-snapshot/src/main/java/org/apache/flink/cdc/migration/tests/SchemaRegistryMigrationMock.java +++ /dev/null @@ -1,222 +0,0 @@ -/* - * 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.migration.tests; - -import org.apache.flink.cdc.common.event.TableId; -import org.apache.flink.cdc.common.pipeline.SchemaChangeBehavior; -import org.apache.flink.cdc.common.schema.Schema; -import org.apache.flink.cdc.common.types.DataTypes; -import org.apache.flink.cdc.runtime.operators.schema.common.SchemaManager; -import org.apache.flink.cdc.runtime.operators.schema.regular.SchemaCoordinator; -import org.apache.flink.metrics.groups.OperatorCoordinatorMetricGroup; -import org.apache.flink.runtime.checkpoint.CheckpointCoordinator; -import org.apache.flink.runtime.jobgraph.OperatorID; -import org.apache.flink.runtime.operators.coordination.CoordinatorStore; -import org.apache.flink.runtime.operators.coordination.OperatorCoordinator; - -import javax.annotation.Nullable; - -import java.lang.reflect.Field; -import java.time.Duration; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.Map; -import java.util.SortedMap; -import java.util.TreeMap; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.Executors; - -/** Dummy classes for migration test. Called via reflection. */ -public class SchemaRegistryMigrationMock implements MigrationMockBase { - private static final TableId TABLE_1 = TableId.tableId("ns", "scm", "tbl1"); - private static final TableId TABLE_2 = TableId.tableId("ns", "scm", "tbl2"); - - private static Schema genSchema(String identifier) { - return Schema.newBuilder() - .physicalColumn("id", DataTypes.INT()) - .physicalColumn("col_" + identifier, DataTypes.STRING()) - .primaryKey("id") - .build(); - } - - private static final Map> ORIGINAL_SCHEMA_MAP; - private static final Map> EVOLVED_SCHEMA_MAP; - - static { - SortedMap originalSchemas = new TreeMap<>(); - originalSchemas.put(1, genSchema("upstream_1")); - originalSchemas.put(2, genSchema("upstream_2")); - originalSchemas.put(3, genSchema("upstream_3")); - - SortedMap evolvedSchemas = new TreeMap<>(); - evolvedSchemas.put(1, genSchema("evolved_1")); - evolvedSchemas.put(2, genSchema("evolved_2")); - evolvedSchemas.put(3, genSchema("evolved_3")); - - ORIGINAL_SCHEMA_MAP = new HashMap<>(); - ORIGINAL_SCHEMA_MAP.put(TABLE_1, originalSchemas); - ORIGINAL_SCHEMA_MAP.put(TABLE_2, originalSchemas); - - EVOLVED_SCHEMA_MAP = new HashMap<>(); - EVOLVED_SCHEMA_MAP.put(TABLE_1, evolvedSchemas); - EVOLVED_SCHEMA_MAP.put(TABLE_2, evolvedSchemas); - } - - public SchemaManager generateDummySchemaManager() { - return new SchemaManager(new HashMap<>(), new HashMap<>(), SchemaChangeBehavior.EVOLVE); - } - - public SchemaCoordinator generateSchemaRegistry() { - SchemaCoordinator coordinator = - new SchemaCoordinator( - "Dummy Name", - MOCKED_CONTEXT, - Executors.newSingleThreadExecutor(), - e -> {}, - new ArrayList<>(), - SchemaChangeBehavior.EVOLVE, - Duration.ofMinutes(3)); - try { - coordinator.start(); - } catch (Exception e) { - throw new RuntimeException(e); - } - return coordinator; - } - - private SchemaManager getSchemaManager(SchemaCoordinator schemaCoordinator) throws Exception { - Field managerField = - SchemaCoordinator.class.getSuperclass().getDeclaredField("schemaManager"); - managerField.setAccessible(true); - return (SchemaManager) managerField.get(schemaCoordinator); - } - - @SuppressWarnings("unchecked") - private Map> getOriginalSchemaMap( - SchemaCoordinator schemaRegistry) throws Exception { - SchemaManager schemaManager = getSchemaManager(schemaRegistry); - Field originalField = SchemaManager.class.getDeclaredField("originalSchemas"); - originalField.setAccessible(true); - return (Map>) originalField.get(schemaManager); - } - - private void setOriginalSchemaMap( - SchemaCoordinator schemaRegistry, - Map> originalSchemaMap) - throws Exception { - SchemaManager schemaManager = getSchemaManager(schemaRegistry); - Field field = SchemaManager.class.getDeclaredField("originalSchemas"); - field.setAccessible(true); - field.set(schemaManager, originalSchemaMap); - } - - @SuppressWarnings("unchecked") - private Map> getEvolvedSchemaMap( - SchemaCoordinator schemaRegistry) throws Exception { - SchemaManager schemaManager = getSchemaManager(schemaRegistry); - Field originalField = SchemaManager.class.getDeclaredField("evolvedSchemas"); - originalField.setAccessible(true); - return (Map>) originalField.get(schemaManager); - } - - private void setEvolvedSchemaMap( - SchemaCoordinator schemaRegistry, - Map> evolvedSchemaMap) - throws Exception { - SchemaManager schemaManager = getSchemaManager(schemaRegistry); - Field field = SchemaManager.class.getDeclaredField("evolvedSchemas"); - field.setAccessible(true); - field.set(schemaManager, evolvedSchemaMap); - } - - @Override - public int getSerializerVersion() { - return -1; - } - - @Override - public byte[] serializeObject() throws Exception { - CompletableFuture future = new CompletableFuture<>(); - SchemaCoordinator registry = generateSchemaRegistry(); - setOriginalSchemaMap(registry, ORIGINAL_SCHEMA_MAP); - setEvolvedSchemaMap(registry, EVOLVED_SCHEMA_MAP); - - registry.checkpointCoordinator(0, future); - - while (!future.isDone()) { - Thread.sleep(1000); - } - return future.get(); - } - - @Override - public boolean deserializeAndCheckObject(int v, byte[] b) throws Exception { - SchemaCoordinator expected = generateSchemaRegistry(); - setOriginalSchemaMap(expected, ORIGINAL_SCHEMA_MAP); - setEvolvedSchemaMap(expected, EVOLVED_SCHEMA_MAP); - - SchemaCoordinator actual = generateSchemaRegistry(); - actual.resetToCheckpoint(0, b); - - return getOriginalSchemaMap(expected).equals(getOriginalSchemaMap(actual)) - && getEvolvedSchemaMap(expected).equals(getEvolvedSchemaMap(actual)); - } - - private static final OperatorCoordinator.Context MOCKED_CONTEXT = - new OperatorCoordinator.Context() { - - @Override - public OperatorID getOperatorId() { - return null; - } - - @Override - public OperatorCoordinatorMetricGroup metricGroup() { - return null; - } - - @Override - public void failJob(Throwable throwable) {} - - @Override - public int currentParallelism() { - return 0; - } - - @Override - public ClassLoader getUserCodeClassloader() { - return null; - } - - @Override - public CoordinatorStore getCoordinatorStore() { - return null; - } - - @Override - public boolean isConcurrentExecutionAttemptsSupported() { - return false; - } - - @Nullable - @Override - public CheckpointCoordinator getCheckpointCoordinator() { - return null; - } - }; -} diff --git a/flink-cdc-migration-tests/flink-cdc-release-snapshot/src/main/java/org/apache/flink/cdc/migration/tests/TableChangeInfoMigrationMock.java b/flink-cdc-migration-tests/flink-cdc-release-snapshot/src/main/java/org/apache/flink/cdc/migration/tests/TableChangeInfoMigrationMock.java deleted file mode 100644 index 1317c73ebf4..00000000000 --- a/flink-cdc-migration-tests/flink-cdc-release-snapshot/src/main/java/org/apache/flink/cdc/migration/tests/TableChangeInfoMigrationMock.java +++ /dev/null @@ -1,61 +0,0 @@ -/* - * 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.migration.tests; - -import org.apache.flink.cdc.common.event.TableId; -import org.apache.flink.cdc.common.schema.Schema; -import org.apache.flink.cdc.common.types.DataTypes; -import org.apache.flink.cdc.runtime.operators.transform.PreTransformChangeInfo; - -/** Dummy classes for migration test. Called via reflection. */ -public class TableChangeInfoMigrationMock implements MigrationMockBase { - private static final TableId DUMMY_TABLE_ID = - TableId.tableId("dummyNamespace", "dummySchema", "dummyTable"); - private static final Schema DUMMY_SCHEMA = - Schema.newBuilder() - .physicalColumn("id", DataTypes.INT()) - .physicalColumn("name", DataTypes.STRING()) - .physicalColumn("age", DataTypes.DOUBLE()) - .primaryKey("id", "name") - .build(); - - public PreTransformChangeInfo generateDummyObject() { - return PreTransformChangeInfo.of(DUMMY_TABLE_ID, DUMMY_SCHEMA, DUMMY_SCHEMA); - } - - @Override - public int getSerializerVersion() { - return PreTransformChangeInfo.SERIALIZER.getVersion(); - } - - @Override - public byte[] serializeObject() throws Exception { - return PreTransformChangeInfo.SERIALIZER.serialize(generateDummyObject()); - } - - @Override - public boolean deserializeAndCheckObject(int version, byte[] bytes) throws Exception { - PreTransformChangeInfo expected = generateDummyObject(); - PreTransformChangeInfo actual = - PreTransformChangeInfo.SERIALIZER.deserialize(version, bytes); - - return expected.getTableId().equals(actual.getTableId()) - && expected.getSourceSchema().equals(actual.getSourceSchema()) - && expected.getPreTransformedSchema().equals(actual.getPreTransformedSchema()); - } -} diff --git a/flink-cdc-migration-tests/pom.xml b/flink-cdc-migration-tests/pom.xml deleted file mode 100644 index 0abd4e4a156..00000000000 --- a/flink-cdc-migration-tests/pom.xml +++ /dev/null @@ -1,54 +0,0 @@ - - - - 4.0.0 - - org.apache.flink - flink-cdc-parent - ${revision} - - - flink-cdc-migration-tests - flink-cdc-migration-tests - pom - - - flink-cdc-release-3.2.0 - flink-cdc-release-3.2.1 - flink-cdc-release-snapshot - flink-cdc-migration-testcases - - - - UTF-8 - - - - - - org.apache.maven.plugins - maven-deploy-plugin - - true - - - - - \ No newline at end of file