diff --git a/.github/workflows/flink_cdc_base.yml b/.github/workflows/flink_cdc_base.yml index 204878c4e14..dcb461ee239 100644 --- a/.github/workflows/flink_cdc_base.yml +++ b/.github/workflows/flink_cdc_base.yml @@ -108,6 +108,7 @@ jobs: runs-on: ubuntu-latest timeout-minutes: 120 strategy: + fail-fast: false matrix: java-version: ${{ fromJSON(inputs.java-version) }} flink-version: ${{ fromJSON(inputs.flink-version) }} @@ -148,7 +149,7 @@ jobs: maven-version: 3.8.6 - name: Compile and test - timeout-minutes: 90 + timeout-minutes: 60 run: | set -o pipefail diff --git a/.github/workflows/flink_cdc_ci.yml b/.github/workflows/flink_cdc_ci.yml index 76e7691e3f8..47f1c77c27f 100644 --- a/.github/workflows/flink_cdc_ci.yml +++ b/.github/workflows/flink_cdc_ci.yml @@ -59,6 +59,7 @@ jobs: run: gem install rubyzip -v 2.3.0 && ./tools/ci/license_check.rb ut: strategy: + fail-fast: false matrix: module: [ 'core', 'pipeline_connectors', 'mysql', 'postgres', 'oracle', 'mongodb6', 'mongodb7', 'sqlserver', 'tidb', 'oceanbase', 'db2', 'vitess' ] name: Unit Tests @@ -68,6 +69,7 @@ jobs: module: ${{ matrix.module }} pipeline_e2e: strategy: + fail-fast: false matrix: parallelism: [ 1, 4 ] name: Pipeline E2E Tests (${{ matrix.parallelism }}-Parallelism) diff --git a/.github/workflows/flink_cdc_ci_nightly.yml b/.github/workflows/flink_cdc_ci_nightly.yml index 456c5f8926c..c69238f422e 100644 --- a/.github/workflows/flink_cdc_ci_nightly.yml +++ b/.github/workflows/flink_cdc_ci_nightly.yml @@ -48,6 +48,7 @@ jobs: run: gem install rubyzip -v 2.3.0 && ./tools/ci/license_check.rb ut: strategy: + fail-fast: false matrix: module: [ 'core', 'pipeline_connectors', 'mysql', 'postgres', 'oracle', 'mongodb6', 'mongodb7', 'sqlserver', 'tidb', 'oceanbase', 'db2', 'vitess' ] name: Unit Tests @@ -57,6 +58,7 @@ jobs: module: ${{ matrix.module }} pipeline_e2e: strategy: + fail-fast: false matrix: parallelism: [ 1, 4 ] name: Pipeline E2E Tests (${{ matrix.parallelism }} Parallelism) diff --git a/.github/workflows/flink_cdc_migration_test_base.yml b/.github/workflows/flink_cdc_migration_test_base.yml index fa8012fd3fa..ff707c0fd8e 100644 --- a/.github/workflows/flink_cdc_migration_test_base.yml +++ b/.github/workflows/flink_cdc_migration_test_base.yml @@ -32,6 +32,7 @@ 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) }} @@ -52,6 +53,7 @@ jobs: pipeline_migration_test: runs-on: ubuntu-latest strategy: + fail-fast: false matrix: java-version: ${{ fromJSON(inputs.java-version) }} flink-version: ${{ fromJSON(inputs.flink-version) }} @@ -91,6 +93,7 @@ jobs: data_stream_migration_test: runs-on: ubuntu-latest strategy: + fail-fast: false matrix: java-version: ${{ fromJSON(inputs.java-version) }} flink-version: [ '1.19.1', '1.20.0' ] diff --git a/flink-cdc-common/pom.xml b/flink-cdc-common/pom.xml index 95f7d5fe054..d3413dbe79a 100644 --- a/flink-cdc-common/pom.xml +++ b/flink-cdc-common/pom.xml @@ -23,6 +23,17 @@ limitations under the License. org.apache.flink ${revision} + + + + org.apache.flink + flink-table-planner_${scala.binary.version} + ${flink.version} + test-jar + test + + + 4.0.0 flink-cdc-common diff --git a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/utils/TestCaseUtils.java b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/utils/TestCaseUtils.java deleted file mode 100644 index f179e779ce6..00000000000 --- a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/utils/TestCaseUtils.java +++ /dev/null @@ -1,94 +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.common.utils; - -import org.apache.flink.util.function.SupplierWithException; - -import java.time.Duration; -import java.util.Collections; -import java.util.List; -import java.util.function.Predicate; -import java.util.function.Supplier; - -/** Some utility methods for creating repeated-checking test cases. */ -public class TestCaseUtils { - - public static final Duration DEFAULT_TIMEOUT = Duration.ofSeconds(30); - public static final Duration DEFAULT_INTERVAL = Duration.ofSeconds(1); - - /** Fetch with a ({@code timeout}, {@code interval}) duration. */ - public static void repeatedCheck(Supplier fetcher) { - repeatedCheck(fetcher, DEFAULT_TIMEOUT); - } - - /** Fetch with a ({@code timeout}, {@code interval}) duration. */ - public static void repeatedCheck(Supplier fetcher, Duration timeout) { - repeatedCheck(fetcher, timeout, DEFAULT_INTERVAL); - } - - /** Fetch with a ({@code timeout}, {@code interval}) duration. */ - public static void repeatedCheck( - Supplier fetcher, Duration timeout, Duration interval) { - repeatedCheck(fetcher::get, timeout, interval, Collections.emptyList()); - } - - /** Fetch and wait with a ({@code timeout}, {@code interval}) duration. */ - public static void repeatedCheck( - Supplier fetcher, Predicate validator, Duration timeout, Duration interval) { - repeatedCheckAndValidate( - fetcher::get, validator, timeout, interval, Collections.emptyList()); - } - - /** Waiting for fetching values with a ({@code timeout}, {@code interval}) duration. */ - public static void repeatedCheck( - SupplierWithException fetcher, - Duration timeout, - Duration interval, - List> allowedThrowsList) { - repeatedCheckAndValidate(fetcher, b -> b, timeout, interval, allowedThrowsList); - } - - /** Fetch and validate, with a ({@code timeout}, {@code interval}) duration. */ - public static void repeatedCheckAndValidate( - SupplierWithException fetcher, - Predicate validator, - Duration timeout, - Duration interval, - List> allowedThrowsList) { - - long start = System.currentTimeMillis(); - while (System.currentTimeMillis() - start < timeout.toMillis()) { - try { - if (validator.test(fetcher.get())) { - return; - } - } catch (Throwable t) { - if (allowedThrowsList.stream() - .noneMatch(clazz -> clazz.isAssignableFrom(t.getClass()))) { - throw new RuntimeException("Fetcher has thrown an unexpected exception: ", t); - } - } - try { - Thread.sleep(interval.toMillis()); - } catch (InterruptedException ignored) { - // ignored - } - } - throw new RuntimeException("Timeout when waiting for state to be ready."); - } -} diff --git a/flink-cdc-common/src/test/java/org/apache/flink/cdc/common/testutils/TestCaseUtils.java b/flink-cdc-common/src/test/java/org/apache/flink/cdc/common/testutils/TestCaseUtils.java new file mode 100644 index 00000000000..8cef6f3f3b5 --- /dev/null +++ b/flink-cdc-common/src/test/java/org/apache/flink/cdc/common/testutils/TestCaseUtils.java @@ -0,0 +1,227 @@ +/* + * 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.common.testutils; + +import org.apache.flink.table.planner.factories.TestValuesTableFactory; +import org.apache.flink.util.function.SupplierWithException; + +import java.time.Duration; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Function; +import java.util.function.Predicate; +import java.util.function.Supplier; +import java.util.stream.Collectors; + +/** Some utility methods for creating repeated-checking test cases. */ +public class TestCaseUtils { + + public static final Duration DEFAULT_TIMEOUT = Duration.ofSeconds(30); + public static final Duration DEFAULT_INTERVAL = Duration.ofSeconds(1); + + /** Fetch with a ({@code timeout}, {@code interval}) duration. */ + public static void repeatedCheck(Supplier fetcher) { + repeatedCheck(fetcher, DEFAULT_TIMEOUT); + } + + /** Fetch with a ({@code timeout}, {@code interval}) duration. */ + public static void repeatedCheck(Supplier fetcher, Duration timeout) { + repeatedCheck(fetcher, timeout, DEFAULT_INTERVAL); + } + + /** Fetch with a ({@code timeout}, {@code interval}) duration. */ + public static void repeatedCheck( + Supplier fetcher, Duration timeout, Duration interval) { + repeatedCheck(fetcher::get, timeout, interval, Collections.emptyList()); + } + + /** Fetch and wait with a ({@code timeout}, {@code interval}) duration. */ + public static void repeatedCheck( + Supplier fetcher, Predicate validator, Duration timeout, Duration interval) { + repeatedCheckAndValidate( + fetcher::get, validator, timeout, interval, Collections.emptyList()); + } + + /** Waiting for fetching values with a ({@code timeout}, {@code interval}) duration. */ + public static void repeatedCheck( + SupplierWithException fetcher, + Duration timeout, + Duration interval, + List> allowedThrowsList) { + repeatedCheckAndValidate(fetcher, b -> b, timeout, interval, allowedThrowsList); + } + + /** Fetch and validate, with a ({@code timeout}, {@code interval}) duration. */ + public static void repeatedCheckAndValidate( + SupplierWithException fetcher, + Predicate validator, + Duration timeout, + Duration interval, + List> allowedThrowsList) { + + long start = System.currentTimeMillis(); + while (System.currentTimeMillis() - start < timeout.toMillis()) { + try { + if (validator.test(fetcher.get())) { + return; + } + } catch (Throwable t) { + if (allowedThrowsList.stream() + .noneMatch(clazz -> clazz.isAssignableFrom(t.getClass()))) { + throw new RuntimeException("Fetcher has thrown an unexpected exception: ", t); + } + } + try { + Thread.sleep(interval.toMillis()); + } catch (InterruptedException ignored) { + // ignored + } + } + throw new RuntimeException("Timeout when waiting for state to be ready."); + } + + public static List fetch(Iterator iter, final int size) throws InterruptedException { + return fetch(iter, size, DEFAULT_TIMEOUT); + } + + public static List fetch(Iterator iter, final int size, Duration timeout) + throws InterruptedException { + return fetch(iter, size, timeout, DEFAULT_INTERVAL); + } + + /** + * Fetches at most {@code size} entries from {@link Iterator} {@code iter}.
+ * It may return a list with less than {@code size} elements, if {@code iter} doesn't provide + * results or {@code timeout} exceeds. + */ + public static List fetch( + Iterator iter, final int size, Duration timeout, Duration interval) + throws InterruptedException { + long deadline = System.currentTimeMillis() + timeout.toMillis(); + + ConcurrentLinkedQueue results = new ConcurrentLinkedQueue<>(); + AtomicReference fetchException = new AtomicReference<>(); + + Thread thread = + new Thread( + () -> { + try { + int remainingSize = size; + while (remainingSize > 0 && iter.hasNext()) { + T row = iter.next(); + results.add(row); + remainingSize--; + } + } catch (Throwable t) { + fetchException.set(t); + } + }); + + thread.start(); + + while (true) { + // Raise any exception thrown by the fetching thread + if (fetchException.get() != null) { + throw (RuntimeException) fetchException.get(); + } + + // Stop if fetching thread has exited + if (!thread.isAlive()) { + break; + } + + // Stop waiting if deadline has arrived + if (System.currentTimeMillis() > deadline) { + thread.interrupt(); + break; + } + + Thread.sleep(interval.toMillis()); + } + + return new ArrayList<>(results); + } + + public static List fetchAndConvert( + Iterator iter, int size, Function converter) throws InterruptedException { + return fetch(iter, size).stream().map(converter).collect(Collectors.toList()); + } + + public static List fetchAndConvert( + Iterator iter, int size, Duration timeout, Function converter) + throws InterruptedException { + return fetch(iter, size, timeout).stream().map(converter).collect(Collectors.toList()); + } + + public static void waitForSnapshotStarted(Iterator iter) { + repeatedCheck(iter::hasNext); + } + + public static void waitForSnapshotStarted(String sinkName) throws InterruptedException { + waitForSinkSize(sinkName, false, 1); + } + + public static void waitForSinkSize(String sinkName, boolean upsertMode, int expectedSize) + throws InterruptedException { + waitForSinkSize(sinkName, upsertMode, DEFAULT_TIMEOUT, expectedSize); + } + + public static void waitForSinkSize( + String sinkName, boolean upsertMode, Duration timeout, int expectedSize) + throws InterruptedException { + waitForSinkSize(sinkName, upsertMode, expectedSize, timeout, DEFAULT_INTERVAL); + } + + public static void waitForSinkSize( + String sinkName, + boolean upsertMode, + int expectedSize, + Duration timeout, + Duration interval) + throws InterruptedException { + long deadline = System.currentTimeMillis() + timeout.toMillis(); + while (sinkSize(sinkName, upsertMode) < expectedSize) { + if (System.currentTimeMillis() > deadline) { + throw new RuntimeException( + String.format( + "Wait for sink size timeout. Expected %s, got actual %s", + expectedSize, sinkSize(sinkName, upsertMode))); + } + Thread.sleep(interval.toMillis()); + } + } + + public static int sinkSize(String sinkName, boolean upsertMode) { + synchronized (TestValuesTableFactory.class) { + try { + if (upsertMode) { + return TestValuesTableFactory.getResultsAsStrings(sinkName).size(); + } else { + return TestValuesTableFactory.getRawResultsAsStrings(sinkName).size(); + } + } catch (IllegalArgumentException e) { + // job is not started yet + return 0; + } + } + } +} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-db2-cdc/pom.xml b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-db2-cdc/pom.xml index 0dc256531a0..32257a8fa6f 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-db2-cdc/pom.xml +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-db2-cdc/pom.xml @@ -62,6 +62,14 @@ limitations under the License. + + org.apache.flink + flink-cdc-common + ${project.version} + test-jar + test + + org.testcontainers db2 diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-db2-cdc/src/test/java/org/apache/flink/cdc/connectors/db2/source/Db2SourceITCase.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-db2-cdc/src/test/java/org/apache/flink/cdc/connectors/db2/source/Db2SourceITCase.java index c3221aed864..6e2fabc34d5 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-db2-cdc/src/test/java/org/apache/flink/cdc/connectors/db2/source/Db2SourceITCase.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-db2-cdc/src/test/java/org/apache/flink/cdc/connectors/db2/source/Db2SourceITCase.java @@ -38,10 +38,10 @@ import java.util.ArrayList; import java.util.Arrays; -import java.util.Iterator; import java.util.List; import static java.lang.String.format; +import static org.apache.flink.cdc.common.testutils.TestCaseUtils.fetchAndConvert; import static org.testcontainers.containers.Db2Container.DB2_PORT; /** IT tests for {@link Db2IncrementalSource}. */ @@ -217,7 +217,8 @@ private void testDb2ParallelSource( } assertEqualsInAnyOrder( - expectedSnapshotData, fetchRows(iterator, expectedSnapshotData.size())); + expectedSnapshotData, + fetchAndConvert(iterator, expectedSnapshotData.size(), Row::toString)); // second step: check the change stream data for (String tableId : captureCustomerTables) { @@ -250,7 +251,8 @@ private void testDb2ParallelSource( expectedRedoLogsData.addAll(Arrays.asList(redoLogsForSingleTable)); } assertEqualsInAnyOrder( - expectedRedoLogsData, fetchRows(iterator, expectedRedoLogsData.size())); + expectedRedoLogsData, + fetchAndConvert(iterator, expectedRedoLogsData.size(), Row::toString)); tableResult.getJobClient().get().cancel().get(); } @@ -275,16 +277,6 @@ private void sleepMs(long millis) { } } - private static List fetchRows(Iterator iter, int size) { - List rows = new ArrayList<>(size); - while (size > 0 && iter.hasNext()) { - Row row = iter.next(); - rows.add(row.toString()); - size--; - } - return rows; - } - /** The type of failover. */ protected enum FailoverType { TM, diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-db2-cdc/src/test/java/org/apache/flink/cdc/connectors/db2/table/Db2ConnectorITCase.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-db2-cdc/src/test/java/org/apache/flink/cdc/connectors/db2/table/Db2ConnectorITCase.java index 6cde68ee47f..c9c31ca6806 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-db2-cdc/src/test/java/org/apache/flink/cdc/connectors/db2/table/Db2ConnectorITCase.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-db2-cdc/src/test/java/org/apache/flink/cdc/connectors/db2/table/Db2ConnectorITCase.java @@ -47,6 +47,8 @@ import java.util.concurrent.ExecutionException; import static org.apache.flink.api.common.JobStatus.RUNNING; +import static org.apache.flink.cdc.common.testutils.TestCaseUtils.waitForSinkSize; +import static org.apache.flink.cdc.common.testutils.TestCaseUtils.waitForSnapshotStarted; import static org.hamcrest.Matchers.containsInAnyOrder; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThat; @@ -174,7 +176,7 @@ public void testConsumingAllEvents() statement.execute("DELETE FROM DB2INST1.PRODUCTS WHERE ID=111;"); } - waitForSinkSize("sink", 20); + waitForSinkSize("sink", false, 20); /* *
@@ -296,7 +298,7 @@ public void testAllTypes() throws Exception {
             statement.execute("UPDATE DB2INST1.FULL_TYPES SET SMALL_C=0 WHERE ID=1;");
         }
 
-        waitForSinkSize("sink", 2);
+        waitForSinkSize("sink", false, 2);
 
         List expected =
                 Arrays.asList(
@@ -368,7 +370,7 @@ public void testStartupFromLatestOffset() throws Exception {
             statement.execute("DELETE FROM DB2INST1.PRODUCTS WHERE ID=111");
         }
 
-        waitForSinkSize("sink", 5);
+        waitForSinkSize("sink", false, 5);
 
         String[] expected =
                 new String[] {"110,jacket,new water resistent white wind breaker,0.500"};
@@ -449,7 +451,7 @@ public void testMetadataColumns() throws Throwable {
             statement.execute("DELETE FROM DB2INST1.PRODUCTS WHERE ID=111;");
         }
 
-        waitForSinkSize("sink", 16);
+        waitForSinkSize("sink", false, 16);
 
         List expected =
                 Arrays.asList(
@@ -476,28 +478,4 @@ public void testMetadataColumns() throws Throwable {
         assertEquals(expected, actual);
         cancelJobIfRunning(result);
     }
-
-    private static void waitForSnapshotStarted(String sinkName) throws InterruptedException {
-        while (sinkSize(sinkName) == 0) {
-            Thread.sleep(1000L);
-        }
-    }
-
-    private static void waitForSinkSize(String sinkName, int expectedSize)
-            throws InterruptedException {
-        while (sinkSize(sinkName) < expectedSize) {
-            Thread.sleep(1000L);
-        }
-    }
-
-    private static int sinkSize(String sinkName) {
-        synchronized (TestValuesTableFactory.class) {
-            try {
-                return TestValuesTableFactory.getRawResultsAsStrings(sinkName).size();
-            } catch (IllegalArgumentException e) {
-                // job is not started yet
-                return 0;
-            }
-        }
-    }
 }
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mongodb-cdc/pom.xml b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mongodb-cdc/pom.xml
index 2b58e677872..ffdfb1f4d15 100644
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mongodb-cdc/pom.xml
+++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mongodb-cdc/pom.xml
@@ -73,6 +73,14 @@ limitations under the License.
         
 
         
+        
+            org.apache.flink
+            flink-cdc-common
+            ${project.version}
+            test-jar
+            test
+        
+
         
             org.apache.flink
             flink-connector-test-util
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mongodb-cdc/src/test/java/org/apache/flink/cdc/connectors/mongodb/source/MongoDBFullChangelogITCase.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mongodb-cdc/src/test/java/org/apache/flink/cdc/connectors/mongodb/source/MongoDBFullChangelogITCase.java
index eebaacafce7..575d7fce44b 100644
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mongodb-cdc/src/test/java/org/apache/flink/cdc/connectors/mongodb/source/MongoDBFullChangelogITCase.java
+++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mongodb-cdc/src/test/java/org/apache/flink/cdc/connectors/mongodb/source/MongoDBFullChangelogITCase.java
@@ -56,11 +56,10 @@
 import java.util.Random;
 import java.util.stream.Collectors;
 
+import static org.apache.flink.cdc.common.testutils.TestCaseUtils.fetchAndConvert;
 import static org.apache.flink.cdc.connectors.mongodb.utils.MongoDBAssertUtils.assertEqualsInAnyOrder;
 import static org.apache.flink.cdc.connectors.mongodb.utils.MongoDBContainer.FLINK_USER;
 import static org.apache.flink.cdc.connectors.mongodb.utils.MongoDBContainer.FLINK_USER_PASSWORD;
-import static org.apache.flink.cdc.connectors.mongodb.utils.MongoDBTestUtils.fetchRowData;
-import static org.apache.flink.cdc.connectors.mongodb.utils.MongoDBTestUtils.fetchRows;
 import static org.apache.flink.cdc.connectors.mongodb.utils.MongoDBTestUtils.triggerFailover;
 import static org.apache.flink.table.api.DataTypes.BIGINT;
 import static org.apache.flink.table.api.DataTypes.STRING;
@@ -584,7 +583,7 @@ private List testBackfillWhenWritingEvents(
         try (CloseableIterator iterator =
                 env.fromSource(source, WatermarkStrategy.noWatermarks(), "Backfill Skipped Source")
                         .executeAndCollect()) {
-            records = fetchRowData(iterator, fetchSize, customerTable::stringify);
+            records = fetchAndConvert(iterator, fetchSize, customerTable::stringify);
             env.close();
         }
         return records;
@@ -713,7 +712,8 @@ private void testMongoDBParallelSource(
         }
 
         assertEqualsInAnyOrder(
-                expectedSnapshotData, fetchRows(iterator, expectedSnapshotData.size()));
+                expectedSnapshotData,
+                fetchAndConvert(iterator, expectedSnapshotData.size(), Row::toString));
 
         // second step: check the change stream data
         for (String collectionName : captureCustomerCollections) {
@@ -747,7 +747,8 @@ private void testMongoDBParallelSource(
         for (int i = 0; i < captureCustomerCollections.length; i++) {
             expectedChangeStreamData.addAll(Arrays.asList(changeEventsForSingleTable));
         }
-        List actualChangeStreamData = fetchRows(iterator, expectedChangeStreamData.size());
+        List actualChangeStreamData =
+                fetchAndConvert(iterator, expectedChangeStreamData.size(), Row::toString);
         assertEqualsInAnyOrder(expectedChangeStreamData, actualChangeStreamData);
         tableResult.getJobClient().get().cancel().get();
     }
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mongodb-cdc/src/test/java/org/apache/flink/cdc/connectors/mongodb/source/MongoDBParallelSourceITCase.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mongodb-cdc/src/test/java/org/apache/flink/cdc/connectors/mongodb/source/MongoDBParallelSourceITCase.java
index 461bf76af0e..20d347c3d81 100644
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mongodb-cdc/src/test/java/org/apache/flink/cdc/connectors/mongodb/source/MongoDBParallelSourceITCase.java
+++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mongodb-cdc/src/test/java/org/apache/flink/cdc/connectors/mongodb/source/MongoDBParallelSourceITCase.java
@@ -56,11 +56,10 @@
 import java.util.stream.Collectors;
 import java.util.stream.Stream;
 
+import static org.apache.flink.cdc.common.testutils.TestCaseUtils.fetchAndConvert;
 import static org.apache.flink.cdc.connectors.mongodb.utils.MongoDBAssertUtils.assertEqualsInAnyOrder;
 import static org.apache.flink.cdc.connectors.mongodb.utils.MongoDBContainer.FLINK_USER;
 import static org.apache.flink.cdc.connectors.mongodb.utils.MongoDBContainer.FLINK_USER_PASSWORD;
-import static org.apache.flink.cdc.connectors.mongodb.utils.MongoDBTestUtils.fetchRowData;
-import static org.apache.flink.cdc.connectors.mongodb.utils.MongoDBTestUtils.fetchRows;
 import static org.apache.flink.cdc.connectors.mongodb.utils.MongoDBTestUtils.triggerFailover;
 import static org.apache.flink.table.api.DataTypes.BIGINT;
 import static org.apache.flink.table.api.DataTypes.STRING;
@@ -496,7 +495,7 @@ private List testBackfillWhenWritingEvents(
         try (CloseableIterator iterator =
                 env.fromSource(source, WatermarkStrategy.noWatermarks(), "Backfill Skipped Source")
                         .executeAndCollect()) {
-            records = fetchRowData(iterator, fetchSize, customerTable::stringify);
+            records = fetchAndConvert(iterator, fetchSize, customerTable::stringify);
             env.close();
         }
         return records;
@@ -605,7 +604,8 @@ private void testMongoDBParallelSource(
         }
 
         assertEqualsInAnyOrder(
-                expectedSnapshotData, fetchRows(iterator, expectedSnapshotData.size()));
+                expectedSnapshotData,
+                fetchAndConvert(iterator, expectedSnapshotData.size(), Row::toString));
 
         // second step: check the change stream data
         for (String collectionName : captureCustomerCollections) {
@@ -639,7 +639,8 @@ private void testMongoDBParallelSource(
         for (int i = 0; i < captureCustomerCollections.length; i++) {
             expectedChangeStreamData.addAll(Arrays.asList(changeEventsForSingleTable));
         }
-        List actualChangeStreamData = fetchRows(iterator, expectedChangeStreamData.size());
+        List actualChangeStreamData =
+                fetchAndConvert(iterator, expectedChangeStreamData.size(), Row::toString);
         assertEqualsInAnyOrder(expectedChangeStreamData, actualChangeStreamData);
         tableResult.getJobClient().get().cancel().get();
     }
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mongodb-cdc/src/test/java/org/apache/flink/cdc/connectors/mongodb/source/NewlyAddedTableITCase.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mongodb-cdc/src/test/java/org/apache/flink/cdc/connectors/mongodb/source/NewlyAddedTableITCase.java
index 0f6b5a62509..52164562178 100644
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mongodb-cdc/src/test/java/org/apache/flink/cdc/connectors/mongodb/source/NewlyAddedTableITCase.java
+++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mongodb-cdc/src/test/java/org/apache/flink/cdc/connectors/mongodb/source/NewlyAddedTableITCase.java
@@ -60,6 +60,7 @@
 import java.util.stream.Stream;
 
 import static java.lang.String.format;
+import static org.apache.flink.cdc.common.testutils.TestCaseUtils.waitForSinkSize;
 import static org.apache.flink.cdc.connectors.mongodb.utils.MongoDBContainer.FLINK_USER;
 import static org.apache.flink.cdc.connectors.mongodb.utils.MongoDBContainer.FLINK_USER_PASSWORD;
 import static org.apache.flink.util.Preconditions.checkState;
@@ -398,8 +399,7 @@ private void testRemoveAndAddCollectionsOneByOne(
                             format(
                                     "+I[%s, 417022095255614379, China, %s, %s West Town address 3]",
                                     collection0, cityName0, cityName0)));
-
-            MongoDBTestUtils.waitForSinkSize("sink", fetchedDataList.size());
+            waitForSinkSize("sink", false, fetchedDataList.size());
             MongoDBAssertUtils.assertEqualsInAnyOrder(
                     fetchedDataList, TestValuesTableFactory.getRawResultsAsStrings("sink"));
 
@@ -416,7 +416,7 @@ private void testRemoveAndAddCollectionsOneByOne(
                             format(
                                     "+I[%s, %d, China, %s, %s West Town address 4]",
                                     collection0, 417022095255614380L, cityName0, cityName0)));
-            MongoDBTestUtils.waitForSinkSize("sink", fetchedDataList.size());
+            waitForSinkSize("sink", false, fetchedDataList.size());
             MongoDBAssertUtils.assertEqualsInAnyOrder(
                     fetchedDataList, TestValuesTableFactory.getRawResultsAsStrings("sink"));
             finishedSavePointPath = triggerSavepointWithRetry(jobClient, savepointDirectory);
@@ -464,7 +464,7 @@ private void testRemoveAndAddCollectionsOneByOne(
                             format(
                                     "+I[%s, 417022095255614379, China, %s, %s West Town address 3]",
                                     captureTableThisRound, cityName, cityName)));
-            MongoDBTestUtils.waitForSinkSize("sink", fetchedDataList.size());
+            waitForSinkSize("sink", false, fetchedDataList.size());
             MongoDBAssertUtils.assertEqualsInAnyOrder(
                     fetchedDataList, TestValuesTableFactory.getRawResultsAsStrings("sink"));
 
@@ -509,7 +509,7 @@ private void testRemoveAndAddCollectionsOneByOne(
                                     cityName)));
 
             // assert fetched changelog data in this round
-            MongoDBTestUtils.waitForSinkSize("sink", fetchedDataList.size());
+            waitForSinkSize("sink", false, fetchedDataList.size());
 
             MongoDBAssertUtils.assertEqualsInAnyOrder(
                     fetchedDataList, TestValuesTableFactory.getRawResultsAsStrings("sink"));
@@ -588,7 +588,7 @@ private void testRemoveCollectionsOneByOne(
                         miniClusterResource.getMiniCluster(),
                         () -> sleepMs(100));
             }
-            MongoDBTestUtils.waitForSinkSize("sink", fetchedDataList.size());
+            waitForSinkSize("sink", false, fetchedDataList.size());
             MongoDBAssertUtils.assertEqualsInAnyOrder(
                     fetchedDataList, TestValuesTableFactory.getRawResultsAsStrings("sink"));
             finishedSavePointPath = triggerSavepointWithRetry(jobClient, savepointDirectory);
@@ -627,7 +627,7 @@ private void testRemoveCollectionsOneByOne(
                             "insert into sink select collection_name, cid, country, city, detail_address from address");
             JobClient jobClient = tableResult.getJobClient().get();
 
-            MongoDBTestUtils.waitForSinkSize("sink", fetchedDataList.size());
+            waitForSinkSize("sink", false, fetchedDataList.size());
             MongoDBAssertUtils.assertEqualsInAnyOrder(
                     fetchedDataList, TestValuesTableFactory.getRawResultsAsStrings("sink"));
 
@@ -676,7 +676,7 @@ private void testRemoveCollectionsOneByOne(
 
             fetchedDataList.addAll(expectedOplogDataThisRound);
             // step 4: assert fetched oplog data in this round
-            MongoDBTestUtils.waitForSinkSize("sink", fetchedDataList.size());
+            waitForSinkSize("sink", false, fetchedDataList.size());
 
             MongoDBAssertUtils.assertEqualsInAnyOrder(
                     fetchedDataList, TestValuesTableFactory.getRawResultsAsStrings("sink"));
@@ -796,7 +796,7 @@ private void testNewlyAddedCollectionOneByOne(
                         () -> sleepMs(100));
             }
             fetchedDataList.addAll(expectedSnapshotDataThisRound);
-            waitForUpsertSinkSize("sink", fetchedDataList.size());
+            waitForSinkSize("sink", true, fetchedDataList.size());
             MongoDBAssertUtils.assertEqualsInAnyOrder(
                     fetchedDataList, TestValuesTableFactory.getResultsAsStrings("sink"));
 
@@ -838,7 +838,7 @@ private void testNewlyAddedCollectionOneByOne(
             // step 5: assert fetched changelog data in this round
             fetchedDataList.addAll(expectedOplogUpsertDataThisRound);
 
-            waitForUpsertSinkSize("sink", fetchedDataList.size());
+            waitForSinkSize("sink", true, fetchedDataList.size());
             // the result size of sink may arrive fetchedDataList.size() with old data, wait one
             // checkpoint to wait retract old record and send new record
             Thread.sleep(1000);
@@ -1013,24 +1013,6 @@ private String getCreateTableStatement(
                                         .collect(Collectors.joining(",")));
     }
 
-    protected static void waitForUpsertSinkSize(String sinkName, int expectedSize)
-            throws InterruptedException {
-        while (upsertSinkSize(sinkName) < expectedSize) {
-            Thread.sleep(100);
-        }
-    }
-
-    protected static int upsertSinkSize(String sinkName) {
-        synchronized (TestValuesTableFactory.class) {
-            try {
-                return TestValuesTableFactory.getResultsAsStrings(sinkName).size();
-            } catch (IllegalArgumentException e) {
-                // job is not started yet
-                return 0;
-            }
-        }
-    }
-
     private String getCollectionNameRegex(String database, String[] captureCustomerCollections) {
         checkState(captureCustomerCollections.length > 0);
         if (captureCustomerCollections.length == 1) {
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mongodb-cdc/src/test/java/org/apache/flink/cdc/connectors/mongodb/table/MongoDBConnectorITCase.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mongodb-cdc/src/test/java/org/apache/flink/cdc/connectors/mongodb/table/MongoDBConnectorITCase.java
index edb8c27d756..a02801b2dc4 100644
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mongodb-cdc/src/test/java/org/apache/flink/cdc/connectors/mongodb/table/MongoDBConnectorITCase.java
+++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mongodb-cdc/src/test/java/org/apache/flink/cdc/connectors/mongodb/table/MongoDBConnectorITCase.java
@@ -46,10 +46,10 @@
 import java.util.stream.Collectors;
 import java.util.stream.Stream;
 
+import static org.apache.flink.cdc.common.testutils.TestCaseUtils.waitForSinkSize;
+import static org.apache.flink.cdc.common.testutils.TestCaseUtils.waitForSnapshotStarted;
 import static org.apache.flink.cdc.connectors.mongodb.utils.MongoDBContainer.FLINK_USER;
 import static org.apache.flink.cdc.connectors.mongodb.utils.MongoDBContainer.FLINK_USER_PASSWORD;
-import static org.apache.flink.cdc.connectors.mongodb.utils.MongoDBTestUtils.waitForSinkSize;
-import static org.apache.flink.cdc.connectors.mongodb.utils.MongoDBTestUtils.waitForSnapshotStarted;
 import static org.hamcrest.Matchers.containsInAnyOrder;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertThat;
@@ -175,11 +175,11 @@ public void testConsumingAllEvents() throws ExecutionException, InterruptedExcep
                 Updates.set("weight", 5.17));
 
         // Delay delete operations to avoid unstable tests.
-        waitForSinkSize("sink", 19);
+        waitForSinkSize("sink", false, 19);
 
         products.deleteOne(Filters.eq("_id", new ObjectId("100000000000000000000111")));
 
-        waitForSinkSize("sink", 20);
+        waitForSinkSize("sink", false, 20);
 
         // The final database table looks like this:
         //
@@ -296,7 +296,7 @@ public void testStartupFromTimestamp() throws Exception {
         products.insertOne(
                 productDocOf("100000000000000000000111", "scooter", "Big 2-wheel scooter", 5.18));
 
-        waitForSinkSize("sink", 2);
+        waitForSinkSize("sink", false, 2);
 
         String[] expected = new String[] {"jacket,0.200", "scooter,5.180"};
 
@@ -438,7 +438,7 @@ public void testAllTypes() throws Throwable {
                 Filters.eq("_id", new ObjectId("5d505646cf6d4fe581014ab2")),
                 Updates.set("int64Field", 510L));
 
-        waitForSinkSize("sink", 3);
+        waitForSinkSize("sink", false, 3);
 
         // 2021-09-03T18:36:04.123Z
         BsonDateTime updatedDateTime = new BsonDateTime(1630694164123L);
@@ -454,7 +454,7 @@ public void testAllTypes() throws Throwable {
                         Updates.set("timestampField", updatedTimestamp),
                         Updates.set("timestampToLocalTimestampField", updatedTimestamp)));
 
-        waitForSinkSize("sink", 5);
+        waitForSinkSize("sink", false, 5);
 
         List expected =
                 Arrays.asList(
@@ -520,7 +520,7 @@ public void testMetadataColumns() throws Exception {
         TableResult result = tEnv.executeSql("INSERT INTO meta_sink SELECT * FROM mongodb_source");
 
         // wait for snapshot finished and start change stream
-        waitForSinkSize("meta_sink", 9);
+        waitForSinkSize("meta_sink", false, 9);
 
         MongoCollection products =
                 mongodbClient.getDatabase(database).getCollection("products");
@@ -554,11 +554,11 @@ public void testMetadataColumns() throws Exception {
                 Updates.set("weight", 5.17));
 
         // Delay delete operations to avoid unstable tests.
-        waitForSinkSize("meta_sink", 15);
+        waitForSinkSize("meta_sink", false, 15);
 
         products.deleteOne(Filters.eq("_id", new ObjectId("100000000000000000000111")));
 
-        waitForSinkSize("meta_sink", 16);
+        waitForSinkSize("meta_sink", false, 16);
 
         List expected =
                 Stream.of(
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mongodb-cdc/src/test/java/org/apache/flink/cdc/connectors/mongodb/table/MongoDBRegexFilterITCase.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mongodb-cdc/src/test/java/org/apache/flink/cdc/connectors/mongodb/table/MongoDBRegexFilterITCase.java
index 8610c29b8c0..6c1a7e386cc 100644
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mongodb-cdc/src/test/java/org/apache/flink/cdc/connectors/mongodb/table/MongoDBRegexFilterITCase.java
+++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mongodb-cdc/src/test/java/org/apache/flink/cdc/connectors/mongodb/table/MongoDBRegexFilterITCase.java
@@ -33,10 +33,10 @@
 
 import java.util.List;
 
+import static org.apache.flink.cdc.common.testutils.TestCaseUtils.waitForSinkSize;
+import static org.apache.flink.cdc.common.testutils.TestCaseUtils.waitForSnapshotStarted;
 import static org.apache.flink.cdc.connectors.mongodb.utils.MongoDBContainer.FLINK_USER;
 import static org.apache.flink.cdc.connectors.mongodb.utils.MongoDBContainer.FLINK_USER_PASSWORD;
-import static org.apache.flink.cdc.connectors.mongodb.utils.MongoDBTestUtils.waitForSinkSize;
-import static org.apache.flink.cdc.connectors.mongodb.utils.MongoDBTestUtils.waitForSnapshotStarted;
 import static org.hamcrest.Matchers.containsInAnyOrder;
 import static org.junit.Assert.assertThat;
 
@@ -92,7 +92,7 @@ public void testMatchMultipleDatabasesAndCollections() throws Exception {
         TableResult result = submitTestCase(null, collectionRegex);
 
         // 3. Wait snapshot finished
-        waitForSinkSize("mongodb_sink", 4);
+        waitForSinkSize("mongodb_sink", false, 4);
 
         // 4. Insert new records in database: [coll_a1.A102, coll_a2.A202, coll_b1.B102,
         // coll_b1.B102]
@@ -100,7 +100,7 @@ public void testMatchMultipleDatabasesAndCollections() throws Exception {
         insertRecordsInDatabase(db1);
 
         // 5. Wait change stream records come
-        waitForSinkSize("mongodb_sink", 8);
+        waitForSinkSize("mongodb_sink", false, 8);
 
         // 6. Check results
         String[] expected =
@@ -137,7 +137,7 @@ public void testMatchMultipleDatabases() throws Exception {
         TableResult result = submitTestCase(databaseRegex, null);
 
         // 3. Wait snapshot finished
-        waitForSinkSize("mongodb_sink", 8);
+        waitForSinkSize("mongodb_sink", false, 8);
 
         // 4. Insert new records in database: [coll_a1.A102, coll_a2.A202, coll_b1.B102,
         // coll_b1.B102]
@@ -146,7 +146,7 @@ public void testMatchMultipleDatabases() throws Exception {
         insertRecordsInDatabase(db2);
 
         // 5. Wait change stream records come
-        waitForSinkSize("mongodb_sink", 16);
+        waitForSinkSize("mongodb_sink", false, 16);
 
         // 6. Check results
         String[] expected =
@@ -189,7 +189,7 @@ public void testMatchSingleQualifiedCollectionPattern() throws Exception {
         TableResult result = submitTestCase(null, collectionRegex);
 
         // 3. Wait snapshot finished
-        waitForSinkSize("mongodb_sink", 2);
+        waitForSinkSize("mongodb_sink", false, 2);
 
         // 4. Insert new records in database: [coll_a1.A102, coll_a2.A202, coll_b1.B102,
         // coll_b1.B102]
@@ -197,7 +197,7 @@ public void testMatchSingleQualifiedCollectionPattern() throws Exception {
         insertRecordsInDatabase(db1);
 
         // 5. Wait change stream records come
-        waitForSinkSize("mongodb_sink", 4);
+        waitForSinkSize("mongodb_sink", false, 4);
 
         // 6. Check results
         String[] expected =
@@ -228,7 +228,7 @@ public void testMatchSingleDatabaseWithCollectionPattern() throws Exception {
         TableResult result = submitTestCase(db0, collectionRegex);
 
         // 3. Wait snapshot finished
-        waitForSinkSize("mongodb_sink", 2);
+        waitForSinkSize("mongodb_sink", false, 2);
 
         // 4. Insert new records in database: [coll_a1.A102, coll_a2.A202, coll_b1.B102,
         // coll_b1.B102]
@@ -236,7 +236,7 @@ public void testMatchSingleDatabaseWithCollectionPattern() throws Exception {
         insertRecordsInDatabase(db1);
 
         // 5. Wait change stream records come
-        waitForSinkSize("mongodb_sink", 4);
+        waitForSinkSize("mongodb_sink", false, 4);
 
         // 6. Check results
         String[] expected =
@@ -262,7 +262,7 @@ public void testMatchDatabaseAndCollectionContainsDash() throws Exception {
         TableResult result = submitTestCase(db0, "coll-a1");
 
         // 2. Wait change stream records come
-        waitForSinkSize("mongodb_sink", 1);
+        waitForSinkSize("mongodb_sink", false, 1);
 
         // 3. Check results
         String[] expected = new String[] {String.format("+I[%s, coll-a1, A101]", db0)};
@@ -282,7 +282,7 @@ public void testMatchCollectionWithDots() throws Exception {
         TableResult result = submitTestCase(db, db + "[.]coll[.]name");
 
         // 2. Wait change stream records come
-        waitForSinkSize("mongodb_sink", 3);
+        waitForSinkSize("mongodb_sink", false, 3);
 
         // 3. Check results
         String[] expected =
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mongodb-cdc/src/test/java/org/apache/flink/cdc/connectors/mongodb/table/MongoDBTimeZoneITCase.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mongodb-cdc/src/test/java/org/apache/flink/cdc/connectors/mongodb/table/MongoDBTimeZoneITCase.java
index 96ab145a9c5..a31520225ff 100644
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mongodb-cdc/src/test/java/org/apache/flink/cdc/connectors/mongodb/table/MongoDBTimeZoneITCase.java
+++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mongodb-cdc/src/test/java/org/apache/flink/cdc/connectors/mongodb/table/MongoDBTimeZoneITCase.java
@@ -17,6 +17,7 @@
 
 package org.apache.flink.cdc.connectors.mongodb.table;
 
+import org.apache.flink.cdc.common.testutils.TestCaseUtils;
 import org.apache.flink.cdc.connectors.mongodb.source.MongoDBSourceTestBase;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.table.api.EnvironmentSettings;
@@ -33,7 +34,6 @@
 
 import java.time.ZoneId;
 import java.util.ArrayList;
-import java.util.Iterator;
 import java.util.List;
 
 import static org.apache.flink.cdc.connectors.mongodb.utils.MongoDBContainer.FLINK_USER;
@@ -154,7 +154,8 @@ public void testTemporalTypesWithTimeZone() throws Exception {
                 break;
         }
 
-        List actualSnapshot = fetchRows(iterator, expectedSnapshot.length);
+        List actualSnapshot =
+                TestCaseUtils.fetchAndConvert(iterator, expectedSnapshot.length, Row::toString);
         assertThat(actualSnapshot, containsInAnyOrder(expectedSnapshot));
 
         result.getJobClient().get().cancel().get();
@@ -217,19 +218,10 @@ public void testDateAndTimestampToStringWithTimeZone() throws Exception {
                 break;
         }
 
-        List actualSnapshot = fetchRows(iterator, expectedSnapshot.length);
+        List actualSnapshot =
+                TestCaseUtils.fetchAndConvert(iterator, expectedSnapshot.length, Row::toString);
         assertThat(actualSnapshot, containsInAnyOrder(expectedSnapshot));
 
         result.getJobClient().get().cancel().get();
     }
-
-    private static List fetchRows(Iterator iter, int size) {
-        List rows = new ArrayList<>(size);
-        while (size > 0 && iter.hasNext()) {
-            Row row = iter.next();
-            rows.add(row.toString());
-            size--;
-        }
-        return rows;
-    }
 }
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mongodb-cdc/src/test/java/org/apache/flink/cdc/connectors/mongodb/utils/MongoDBTestUtils.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mongodb-cdc/src/test/java/org/apache/flink/cdc/connectors/mongodb/utils/MongoDBTestUtils.java
index 7b826dd717e..1f826fe383d 100644
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mongodb-cdc/src/test/java/org/apache/flink/cdc/connectors/mongodb/utils/MongoDBTestUtils.java
+++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mongodb-cdc/src/test/java/org/apache/flink/cdc/connectors/mongodb/utils/MongoDBTestUtils.java
@@ -20,81 +20,10 @@
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.runtime.highavailability.nonha.embedded.HaLeadershipControl;
 import org.apache.flink.runtime.minicluster.MiniCluster;
-import org.apache.flink.table.data.RowData;
-import org.apache.flink.table.planner.factories.TestValuesTableFactory;
-import org.apache.flink.types.Row;
-
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-import java.util.concurrent.TimeUnit;
-import java.util.function.Function;
-import java.util.stream.Collectors;
-
-import static org.junit.Assert.fail;
 
 /** MongoDB test utilities. */
 public class MongoDBTestUtils {
 
-    public static void waitForSnapshotStarted(String sinkName) throws InterruptedException {
-        while (sinkSize(sinkName) == 0) {
-            Thread.sleep(100);
-        }
-    }
-
-    public static void waitForSinkSize(String sinkName, int expectedSize)
-            throws InterruptedException {
-        waitForSinkSize(sinkName, expectedSize, 10, TimeUnit.MINUTES);
-    }
-
-    public static void waitForSinkSize(
-            String sinkName, int expectedSize, long timeout, TimeUnit timeUnit)
-            throws InterruptedException {
-        long deadline = System.nanoTime() + timeUnit.toNanos(timeout);
-        while (sinkSize(sinkName) < expectedSize) {
-            if (System.nanoTime() > deadline) {
-                fail(
-                        "Wait for sink size timeout, raw results: \n"
-                                + String.join(
-                                        "\n",
-                                        TestValuesTableFactory.getRawResultsAsStrings(sinkName)));
-            }
-            Thread.sleep(100);
-        }
-    }
-
-    public static int sinkSize(String sinkName) {
-        synchronized (TestValuesTableFactory.class) {
-            try {
-                return TestValuesTableFactory.getRawResultsAsStrings(sinkName).size();
-            } catch (IllegalArgumentException e) {
-                // job is not started yet
-                return 0;
-            }
-        }
-    }
-
-    public static List fetchRowData(
-            Iterator iter, int size, Function stringifier) {
-        List rows = new ArrayList<>(size);
-        while (size > 0 && iter.hasNext()) {
-            RowData row = iter.next();
-            rows.add(row);
-            size--;
-        }
-        return rows.stream().map(stringifier).collect(Collectors.toList());
-    }
-
-    public static List fetchRows(Iterator iter, int size) {
-        List rows = new ArrayList<>(size);
-        while (size > 0 && iter.hasNext()) {
-            Row row = iter.next();
-            rows.add(row.toString());
-            size--;
-        }
-        return rows;
-    }
-
     /** The type of failover. */
     public enum FailoverType {
         TM,
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/pom.xml b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/pom.xml
index 8e6886e156a..94bb73efdd7 100644
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/pom.xml
+++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/pom.xml
@@ -98,15 +98,16 @@ limitations under the License.
             test
         
 
+        
+
         
-            com.alibaba
-            fastjson
-            1.2.83
+            org.apache.flink
+            flink-cdc-common
+            ${project.version}
+            test-jar
             test
         
 
-        
-
         
             org.apache.flink
             flink-table-planner_${scala.binary.version}
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/org/apache/flink/cdc/connectors/mysql/LegacyMySqlSourceITCase.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/org/apache/flink/cdc/connectors/mysql/LegacyMySqlSourceITCase.java
index de7acd86a71..070b48c211c 100644
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/org/apache/flink/cdc/connectors/mysql/LegacyMySqlSourceITCase.java
+++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/org/apache/flink/cdc/connectors/mysql/LegacyMySqlSourceITCase.java
@@ -28,8 +28,11 @@
 import org.apache.flink.types.Row;
 import org.apache.flink.util.CloseableIterator;
 
-import com.alibaba.fastjson.JSONObject;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
 import org.apache.kafka.connect.json.JsonConverterConfig;
+import org.assertj.core.api.Assertions;
 import org.junit.Test;
 
 import java.io.IOException;
@@ -39,14 +42,12 @@
 import java.nio.file.Paths;
 import java.sql.Connection;
 import java.sql.Statement;
-import java.util.ArrayList;
 import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
 import java.util.Map;
 import java.util.Objects;
 
-import static org.junit.Assert.assertTrue;
+import static org.apache.flink.cdc.common.testutils.TestCaseUtils.fetchAndConvert;
+import static org.apache.flink.cdc.common.testutils.TestCaseUtils.waitForSnapshotStarted;
 
 /** Integration tests for the legacy {@link MySqlSource}. */
 public class LegacyMySqlSourceITCase extends LegacyMySqlTestBase {
@@ -99,9 +100,9 @@ private void testConsumingAllEventsWithJsonFormat(
                 StreamTableEnvironment.create(
                         env, EnvironmentSettings.newInstance().inStreamingMode().build());
 
-        final JSONObject expected =
-                JSONObject.parseObject(readLines(expectedFile), JSONObject.class);
-        JSONObject expectSnapshot = expected.getJSONObject("expected_snapshot");
+        final JsonNode expected =
+                new ObjectMapper().readValue(readLines(expectedFile), JsonNode.class);
+        JsonNode expectSnapshot = expected.get("expected_snapshot");
 
         DataStreamSource source = env.addSource(sourceFunction);
         tEnv.createTemporaryView("full_types", source);
@@ -110,9 +111,10 @@ private void testConsumingAllEventsWithJsonFormat(
         // check the snapshot result
         CloseableIterator snapshot = result.collect();
         waitForSnapshotStarted(snapshot);
-        assertTrue(
-                dataInJsonIsEquals(
-                        fetchRows(snapshot, 1).get(0).toString(), expectSnapshot.toString()));
+
+        assertJsonEquals(
+                fetchAndConvert(snapshot, 1, LegacyMySqlSourceITCase::extractJsonBody).get(0),
+                expectSnapshot);
         try (Connection connection = fullTypesDatabase.getJdbcConnection();
                 Statement statement = connection.createStatement()) {
             statement.execute(
@@ -121,10 +123,10 @@ private void testConsumingAllEventsWithJsonFormat(
 
         // check the binlog result
         CloseableIterator binlog = result.collect();
-        JSONObject expectBinlog = expected.getJSONObject("expected_binlog");
-        assertTrue(
-                dataInJsonIsEquals(
-                        fetchRows(binlog, 1).get(0).toString(), expectBinlog.toString()));
+        JsonNode expectBinlog = expected.get("expected_binlog");
+        assertJsonEquals(
+                fetchAndConvert(binlog, 1, LegacyMySqlSourceITCase::extractJsonBody).get(0),
+                expectBinlog);
         result.getJobClient().get().cancel().get();
     }
 
@@ -136,23 +138,6 @@ private void testConsumingAllEventsWithJsonFormat(Boolean includeSchema) throws
         testConsumingAllEventsWithJsonFormat(includeSchema, null, expectedFile);
     }
 
-    private static List fetchRows(Iterator iter, int size) {
-        List rows = new ArrayList<>(size);
-        while (size > 0 && iter.hasNext()) {
-            Row row = iter.next();
-            // ignore rowKind marker
-            rows.add(row.getField(0));
-            size--;
-        }
-        return rows;
-    }
-
-    private static void waitForSnapshotStarted(CloseableIterator iterator) throws Exception {
-        while (!iterator.hasNext()) {
-            Thread.sleep(100);
-        }
-    }
-
     private static byte[] readLines(String resource) throws IOException, URISyntaxException {
         Path path =
                 Paths.get(
@@ -164,25 +149,23 @@ private static byte[] readLines(String resource) throws IOException, URISyntaxEx
         return Files.readAllBytes(path);
     }
 
-    private static boolean dataInJsonIsEquals(String actual, String expect) {
-        JSONObject actualJsonObject = JSONObject.parseObject(actual);
-        JSONObject expectJsonObject = JSONObject.parseObject(expect);
-
-        if (expectJsonObject.getJSONObject("payload") != null
-                && actualJsonObject.getJSONObject("payload") != null) {
-            expectJsonObject = expectJsonObject.getJSONObject("payload");
-            actualJsonObject = actualJsonObject.getJSONObject("payload");
+    private static void assertJsonEquals(JsonNode actual, JsonNode expect) throws Exception {
+        if (actual.get("payload") != null && expect.get("payload") != null) {
+            actual = actual.get("payload");
+            expect = expect.get("payload");
         }
-        return jsonObjectEquals(
-                        expectJsonObject.getJSONObject("after"),
-                        actualJsonObject.getJSONObject("after"))
-                && jsonObjectEquals(
-                        expectJsonObject.getJSONObject("before"),
-                        actualJsonObject.getJSONObject("before"))
-                && Objects.equals(expectJsonObject.get("op"), actualJsonObject.get("op"));
+        Assertions.assertThat(actual.get("after")).isEqualTo(expect.get("after"));
+        Assertions.assertThat(actual.get("before")).isEqualTo(expect.get("before"));
+        Assertions.assertThat(actual.get("op")).isEqualTo(expect.get("op"));
     }
 
-    private static boolean jsonObjectEquals(JSONObject a, JSONObject b) {
-        return (a == b) || (a != null && a.toString().equals(b.toString()));
+    private static JsonNode extractJsonBody(Row row) {
+        try {
+            String body = row.toString();
+            return new ObjectMapper()
+                    .readValue(body.substring(3, body.length() - 1), JsonNode.class);
+        } catch (JsonProcessingException e) {
+            throw new RuntimeException("Invalid JSON format.", e);
+        }
     }
 }
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/org/apache/flink/cdc/connectors/mysql/debezium/converters/MysqlDebeziumTimeConverterITCase.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/org/apache/flink/cdc/connectors/mysql/debezium/converters/MysqlDebeziumTimeConverterITCase.java
index 8adb92454ae..bef00cc1685 100644
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/org/apache/flink/cdc/connectors/mysql/debezium/converters/MysqlDebeziumTimeConverterITCase.java
+++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/org/apache/flink/cdc/connectors/mysql/debezium/converters/MysqlDebeziumTimeConverterITCase.java
@@ -18,6 +18,7 @@
 package org.apache.flink.cdc.connectors.mysql.debezium.converters;
 
 import org.apache.flink.api.common.eventtime.WatermarkStrategy;
+import org.apache.flink.cdc.common.testutils.TestCaseUtils;
 import org.apache.flink.cdc.connectors.mysql.MySqlValidatorTest;
 import org.apache.flink.cdc.connectors.mysql.source.MySqlSource;
 import org.apache.flink.cdc.connectors.mysql.source.MySqlSourceBuilder;
@@ -60,7 +61,6 @@
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
-import java.util.Iterator;
 import java.util.List;
 import java.util.Objects;
 import java.util.Properties;
@@ -227,7 +227,7 @@ private Properties getDebeziumConfigurations(String timezone) {
         return debeziumProperties;
     }
 
-    private void checkData(TableResult tableResult) {
+    private void checkData(TableResult tableResult) throws Exception {
         String[] snapshotForSingleTable =
                 new String[] {
                     "+I[1, 14:23:00, 2023-04-01 14:24:00, 2023-04-01, 14:25:00]",
@@ -240,17 +240,8 @@ private void checkData(TableResult tableResult) {
         CloseableIterator collect = tableResult.collect();
         tableResult.getJobClient().get().getJobID();
         assertEqualsInAnyOrder(
-                expectedSnapshotData, fetchRows(collect, expectedSnapshotData.size()));
-    }
-
-    private static List fetchRows(Iterator iter, int size) {
-        List rows = new ArrayList<>(size);
-        while (size > 0 && iter.hasNext()) {
-            Row row = iter.next();
-            rows.add(row.toString());
-            size--;
-        }
-        return rows;
+                expectedSnapshotData,
+                TestCaseUtils.fetchAndConvert(collect, expectedSnapshotData.size(), Row::toString));
     }
 
     protected MySqlContainer createMySqlContainer(String timezone) {
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/org/apache/flink/cdc/connectors/mysql/source/MySqlOnLineSchemaMigrationSourceITCase.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/org/apache/flink/cdc/connectors/mysql/source/MySqlOnLineSchemaMigrationSourceITCase.java
index 84886978392..ad64b2e52ab 100644
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/org/apache/flink/cdc/connectors/mysql/source/MySqlOnLineSchemaMigrationSourceITCase.java
+++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/org/apache/flink/cdc/connectors/mysql/source/MySqlOnLineSchemaMigrationSourceITCase.java
@@ -24,8 +24,8 @@
 import org.apache.flink.cdc.common.event.Event;
 import org.apache.flink.cdc.common.event.TableId;
 import org.apache.flink.cdc.common.schema.Schema;
+import org.apache.flink.cdc.common.testutils.TestCaseUtils;
 import org.apache.flink.cdc.common.types.DataType;
-import org.apache.flink.cdc.common.utils.TestCaseUtils;
 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;
@@ -36,7 +36,6 @@
 import org.apache.flink.table.api.EnvironmentSettings;
 import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
 import org.apache.flink.table.planner.factories.TestValuesTableFactory;
-import org.apache.flink.types.Row;
 
 import org.junit.After;
 import org.junit.AfterClass;
@@ -54,9 +53,7 @@
 import java.io.PrintStream;
 import java.sql.Connection;
 import java.sql.Statement;
-import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Iterator;
 import java.util.List;
 import java.util.NoSuchElementException;
 import java.util.Random;
@@ -64,8 +61,8 @@
 import java.util.stream.Stream;
 
 import static org.apache.flink.api.common.JobStatus.RUNNING;
-import static org.apache.flink.cdc.common.utils.TestCaseUtils.DEFAULT_INTERVAL;
-import static org.apache.flink.cdc.common.utils.TestCaseUtils.DEFAULT_TIMEOUT;
+import static org.apache.flink.cdc.common.testutils.TestCaseUtils.DEFAULT_INTERVAL;
+import static org.apache.flink.cdc.common.testutils.TestCaseUtils.DEFAULT_TIMEOUT;
 
 /**
  * IT case for Evolving MySQL schema with gh-ost/pt-osc utility. See  sinkSize(sinkName) >= expectedSize);
-    }
-
-    private static int sinkSize(String sinkName) {
-        synchronized (TestValuesTableFactory.class) {
-            try {
-                return TestValuesTableFactory.getRawResults(sinkName).size();
-            } catch (IllegalArgumentException e) {
-                // job is not started yet
-                return 0;
-            }
-        }
-    }
-
-    private static List fetchRows(Iterator iter, int size) {
-        List rows = new ArrayList<>(size);
-        while (size > 0 && iter.hasNext()) {
-            Row row = iter.next();
-            rows.add(row.toString());
-            size--;
-        }
-        return rows;
-    }
 }
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/org/apache/flink/cdc/connectors/mysql/source/MySqlSourceITCase.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/org/apache/flink/cdc/connectors/mysql/source/MySqlSourceITCase.java
index 7be090a8fef..777c4b77e4f 100644
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/org/apache/flink/cdc/connectors/mysql/source/MySqlSourceITCase.java
+++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/org/apache/flink/cdc/connectors/mysql/source/MySqlSourceITCase.java
@@ -84,7 +84,6 @@
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
-import java.util.Iterator;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
@@ -97,12 +96,12 @@
 import java.util.concurrent.FutureTask;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
-import java.util.function.Function;
-import java.util.stream.Collectors;
 
 import static java.lang.String.format;
 import static org.apache.flink.api.common.JobStatus.RUNNING;
+import static org.apache.flink.cdc.common.testutils.TestCaseUtils.fetchAndConvert;
 import static org.apache.flink.util.Preconditions.checkState;
+import static org.assertj.core.api.Assertions.assertThat;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
@@ -347,7 +346,10 @@ public void testSnapshotSplitReadingFailCrossCheckpoints() throws Exception {
         // Check all snapshot records are sent with exactly-once semantics
         assertEqualsInAnyOrder(
                 Arrays.asList(expectedSnapshotData),
-                fetchRowData(iterator, expectedSnapshotData.length));
+                fetchAndConvert(
+                        iterator,
+                        expectedSnapshotData.length,
+                        MySqlSourceITCase::convertRowDataToRowString));
         assertTrue(!hasNextData(iterator));
         jobClient.cancel().get();
     }
@@ -638,7 +640,7 @@ private List testBackfillWhenWritingEvents(
         try (CloseableIterator iterator =
                 env.fromSource(source, WatermarkStrategy.noWatermarks(), "Backfill Skipped Source")
                         .executeAndCollect()) {
-            List records = fetchRowData(iterator, fetchSize, customerTable::stringify);
+            List records = fetchAndConvert(iterator, fetchSize, customerTable::stringify);
             return records;
         }
     }
@@ -693,7 +695,9 @@ private void testStartingOffset(
         DataStreamSource source =
                 env.fromSource(mySqlSource, WatermarkStrategy.noWatermarks(), "MySQL CDC Source");
         try (CloseableIterator iterator = source.executeAndCollect()) {
-            List rows = fetchRowData(iterator, expectedChangelogAfterStart.size());
+            List rows =
+                    fetchAndConvert(
+                            iterator, expectedChangelogAfterStart.size(), RowData::toString);
             assertEqualsInAnyOrder(expectedChangelogAfterStart, rows);
         }
     }
@@ -1027,7 +1031,8 @@ private void checkSnapshotData(
         }
 
         assertEqualsInAnyOrder(
-                expectedSnapshotData, fetchRows(iterator, expectedSnapshotData.size()));
+                expectedSnapshotData,
+                fetchAndConvert(iterator, expectedSnapshotData.size(), Row::toString));
     }
 
     private void checkBinlogData(
@@ -1064,61 +1069,25 @@ private void checkBinlogData(
             expectedBinlogData.addAll(secondPartBinlogEvents);
         }
 
-        assertEqualsInAnyOrder(expectedBinlogData, fetchRows(iterator, expectedBinlogData.size()));
-        assertTrue(!hasNextData(iterator));
+        assertEqualsInAnyOrder(
+                expectedBinlogData,
+                fetchAndConvert(iterator, expectedBinlogData.size(), Row::toString));
+        assertThat(iterator.hasNext()).isFalse();
     }
 
-    private static List convertRowDataToRowString(List rows) {
+    private static String convertRowDataToRowString(RowData row) {
         LinkedHashMap map = new LinkedHashMap<>();
         map.put("id", 0);
         map.put("name", 1);
         map.put("address", 2);
         map.put("phone_number", 3);
-        return rows.stream()
-                .map(
-                        row ->
-                                RowUtils.createRowWithNamedPositions(
-                                                row.getRowKind(),
-                                                new Object[] {
-                                                    row.getLong(0),
-                                                    row.getString(1),
-                                                    row.getString(2),
-                                                    row.getString(3)
-                                                },
-                                                map)
-                                        .toString())
-                .collect(Collectors.toList());
-    }
-
-    private static List fetchRows(Iterator iter, int size) {
-        List rows = new ArrayList<>(size);
-        while (size > 0 && iter.hasNext()) {
-            Row row = iter.next();
-            rows.add(row.toString());
-            size--;
-        }
-        return rows;
-    }
-
-    private List fetchRowData(
-            Iterator iter, int size, Function stringifier) {
-        List rows = new ArrayList<>(size);
-        while (size > 0 && iter.hasNext()) {
-            RowData row = iter.next();
-            rows.add(row);
-            size--;
-        }
-        return rows.stream().map(stringifier).collect(Collectors.toList());
-    }
-
-    private static List fetchRowData(Iterator iter, int size) {
-        List rows = new ArrayList<>(size);
-        while (size > 0 && iter.hasNext()) {
-            RowData row = iter.next();
-            rows.add(row);
-            size--;
-        }
-        return convertRowDataToRowString(rows);
+        return RowUtils.createRowWithNamedPositions(
+                        row.getRowKind(),
+                        new Object[] {
+                            row.getLong(0), row.getString(1), row.getString(2), row.getString(3)
+                        },
+                        map)
+                .toString();
     }
 
     private String getTableNameRegex(String[] captureCustomerTables) {
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/org/apache/flink/cdc/connectors/mysql/source/MySqlSourceTestBase.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/org/apache/flink/cdc/connectors/mysql/source/MySqlSourceTestBase.java
index 2b8e76500ee..f1c2cc6b907 100644
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/org/apache/flink/cdc/connectors/mysql/source/MySqlSourceTestBase.java
+++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/org/apache/flink/cdc/connectors/mysql/source/MySqlSourceTestBase.java
@@ -26,7 +26,6 @@
 import org.apache.flink.runtime.minicluster.RpcServiceSharing;
 import org.apache.flink.runtime.testutils.InMemoryReporter;
 import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration;
-import org.apache.flink.table.planner.factories.TestValuesTableFactory;
 import org.apache.flink.test.util.MiniClusterWithClientResource;
 import org.apache.flink.util.TestLogger;
 
@@ -174,22 +173,4 @@ protected static void restartTaskManager(MiniCluster miniCluster, Runnable after
         afterFailAction.run();
         miniCluster.startTaskManager();
     }
-
-    protected static void waitForUpsertSinkSize(String sinkName, int expectedSize)
-            throws InterruptedException {
-        while (upsertSinkSize(sinkName) < expectedSize) {
-            Thread.sleep(100);
-        }
-    }
-
-    protected static int upsertSinkSize(String sinkName) {
-        synchronized (TestValuesTableFactory.class) {
-            try {
-                return TestValuesTableFactory.getResultsAsStrings(sinkName).size();
-            } catch (IllegalArgumentException e) {
-                // job is not started yet
-                return 0;
-            }
-        }
-    }
 }
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/org/apache/flink/cdc/connectors/mysql/source/NewlyAddedTableITCase.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/org/apache/flink/cdc/connectors/mysql/source/NewlyAddedTableITCase.java
index 1f3fa0d1aab..429b0015215 100644
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/org/apache/flink/cdc/connectors/mysql/source/NewlyAddedTableITCase.java
+++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/org/apache/flink/cdc/connectors/mysql/source/NewlyAddedTableITCase.java
@@ -62,7 +62,6 @@
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.HashMap;
-import java.util.Iterator;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
@@ -79,6 +78,8 @@
 
 import static java.lang.String.format;
 import static org.apache.flink.api.common.restartstrategy.RestartStrategies.noRestart;
+import static org.apache.flink.cdc.common.testutils.TestCaseUtils.fetchAndConvert;
+import static org.apache.flink.cdc.common.testutils.TestCaseUtils.waitForSinkSize;
 import static org.apache.flink.util.Preconditions.checkState;
 
 /** IT tests to cover various newly added tables during capture process. */
@@ -487,7 +488,11 @@ public void testRemoveAndAddNewTable() throws Exception {
                                             expectedCustomersResult.stream())
                                     .collect(Collectors.toList())
                             : expectedCustomersResult;
-            List rows = fetchRowData(iterator, expectedSnapshotResult.size());
+            List rows =
+                    fetchAndConvert(
+                            iterator,
+                            expectedSnapshotResult.size(),
+                            NewlyAddedTableITCase::convertRowDataToRowString);
             assertEqualsInAnyOrder(expectedSnapshotResult, rows);
 
             // make binlog events
@@ -503,7 +508,11 @@ public void testRemoveAndAddNewTable() throws Exception {
                         "UPDATE " + tableId + " SET address = 'Update2' where id = 103");
                 connection.commit();
             }
-            rows = fetchRowData(iterator, expectedBinlogResult.size());
+            rows =
+                    fetchAndConvert(
+                            iterator,
+                            expectedBinlogResult.size(),
+                            NewlyAddedTableITCase::convertRowDataToRowString);
             assertEqualsInAnyOrder(expectedBinlogResult, rows);
 
             finishedSavePointPath = triggerSavepointWithRetry(jobClient, savepointDirectory);
@@ -540,38 +549,24 @@ protected CollectResultIterator addCollectSink(DataStream stre
         return iterator;
     }
 
-    private List fetchRowData(Iterator iter, int size) {
-        List rows = new ArrayList<>(size);
-        while (size > 0 && iter.hasNext()) {
-            RowData row = iter.next();
-            rows.add(row);
-            size--;
-        }
-        return convertRowDataToRowString(rows);
-    }
-
-    private static List convertRowDataToRowString(List rows) {
+    private static String convertRowDataToRowString(RowData row) {
         LinkedHashMap map = new LinkedHashMap<>();
         map.put("id", 0);
         map.put("name", 1);
         map.put("address", 2);
         map.put("phone_number", 3);
         map.put("_table_name", 4);
-        return rows.stream()
-                .map(
-                        row ->
-                                RowUtils.createRowWithNamedPositions(
-                                                row.getRowKind(),
-                                                new Object[] {
-                                                    row.getLong(0),
-                                                    row.getString(1),
-                                                    row.getString(2),
-                                                    row.getString(3),
-                                                    row.getString(4)
-                                                },
-                                                map)
-                                        .toString())
-                .collect(Collectors.toList());
+        return RowUtils.createRowWithNamedPositions(
+                        row.getRowKind(),
+                        new Object[] {
+                            row.getLong(0),
+                            row.getString(1),
+                            row.getString(2),
+                            row.getString(3),
+                            row.getString(4)
+                        },
+                        map)
+                .toString();
     }
 
     private void testRemoveTablesOneByOne(
@@ -638,7 +633,7 @@ private void testRemoveTablesOneByOne(
                         miniClusterResource.getMiniCluster(),
                         () -> sleepMs(100));
             }
-            waitForSinkSize("sink", fetchedDataList.size());
+            waitForSinkSize("sink", false, fetchedDataList.size());
             assertEqualsInAnyOrder(
                     fetchedDataList, TestValuesTableFactory.getRawResultsAsStrings("sink"));
             finishedSavePointPath = triggerSavepointWithRetry(jobClient, savepointDirectory);
@@ -674,7 +669,7 @@ private void testRemoveTablesOneByOne(
             TableResult tableResult = tEnv.executeSql("insert into sink select * from address");
             JobClient jobClient = tableResult.getJobClient().get();
 
-            waitForSinkSize("sink", fetchedDataList.size());
+            waitForSinkSize("sink", false, fetchedDataList.size());
             assertEqualsInAnyOrder(
                     fetchedDataList, TestValuesTableFactory.getRawResultsAsStrings("sink"));
 
@@ -715,7 +710,7 @@ private void testRemoveTablesOneByOne(
 
             fetchedDataList.addAll(expectedBinlogDataThisRound);
             // step 4: assert fetched binlog data in this round
-            waitForSinkSize("sink", fetchedDataList.size());
+            waitForSinkSize("sink", false, fetchedDataList.size());
             assertEqualsInAnyOrder(
                     fetchedDataList, TestValuesTableFactory.getRawResultsAsStrings("sink"));
 
@@ -830,7 +825,7 @@ private void testNewlyAddedTableOneByOne(
                         () -> sleepMs(100));
             }
             fetchedDataList.addAll(expectedSnapshotDataThisRound);
-            waitForUpsertSinkSize("sink", fetchedDataList.size());
+            waitForSinkSize("sink", true, fetchedDataList.size());
             assertEqualsInAnyOrder(
                     fetchedDataList, TestValuesTableFactory.getResultsAsStrings("sink"));
 
@@ -870,7 +865,7 @@ private void testNewlyAddedTableOneByOne(
             // step 5: assert fetched binlog data in this round
             fetchedDataList.addAll(expectedBinlogUpsertDataThisRound);
 
-            waitForUpsertSinkSize("sink", fetchedDataList.size());
+            waitForSinkSize("sink", true, fetchedDataList.size());
             // the result size of sink may arrive fetchedDataList.size() with old data, wait one
             // checkpoint to wait retract old record and send new record
             Thread.sleep(1000);
@@ -1104,24 +1099,6 @@ private void makeBinlogForAddressTable(JdbcConnection connection, String tableNa
         }
     }
 
-    private static void waitForSinkSize(String sinkName, int expectedSize)
-            throws InterruptedException {
-        while (sinkSize(sinkName) < expectedSize) {
-            Thread.sleep(100);
-        }
-    }
-
-    private static int sinkSize(String sinkName) {
-        synchronized (TestValuesTableFactory.class) {
-            try {
-                return TestValuesTableFactory.getRawResultsAsStrings(sinkName).size();
-            } catch (IllegalArgumentException e) {
-                // job is not started yet
-                return 0;
-            }
-        }
-    }
-
     private void testNewlyAddedTableOneByOneWithCreateBeforeStart(
             int parallelism, Map sourceOptions, String... captureAddressTables)
             throws Exception {
@@ -1181,7 +1158,7 @@ private void testNewlyAddedTableOneByOneWithCreateBeforeStart(
                                     "+I[%s, 417022095255614379, China, %s, %s West Town address 3]",
                                     newlyAddedTable, cityName, cityName));
             fetchedDataList.addAll(expectedSnapshotDataThisRound);
-            waitForUpsertSinkSize("sink", fetchedDataList.size());
+            waitForSinkSize("sink", true, fetchedDataList.size());
             assertEqualsInAnyOrder(
                     fetchedDataList, TestValuesTableFactory.getResultsAsStrings("sink"));
             // step 3: make some binlog data for this round
@@ -1209,7 +1186,7 @@ private void testNewlyAddedTableOneByOneWithCreateBeforeStart(
                                     newlyAddedTable, cityName, cityName));
             // step 5: assert fetched binlog data in this round
             fetchedDataList.addAll(expectedBinlogUpsertDataThisRound);
-            waitForUpsertSinkSize("sink", fetchedDataList.size());
+            waitForSinkSize("sink", true, fetchedDataList.size());
             // the result size of sink may arrive fetchedDataList.size() with old data, wait one
             // checkpoint to wait retract old record and send new record
             Thread.sleep(1000);
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/org/apache/flink/cdc/connectors/mysql/source/SpecificStartingOffsetITCase.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/org/apache/flink/cdc/connectors/mysql/source/SpecificStartingOffsetITCase.java
index 7a178842e76..875e90a96f1 100644
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/org/apache/flink/cdc/connectors/mysql/source/SpecificStartingOffsetITCase.java
+++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/org/apache/flink/cdc/connectors/mysql/source/SpecificStartingOffsetITCase.java
@@ -62,18 +62,15 @@
 import java.nio.file.Paths;
 import java.nio.file.StandardOpenOption;
 import java.time.ZoneId;
-import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
-import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Objects;
 import java.util.Properties;
 import java.util.UUID;
-import java.util.function.Function;
-import java.util.stream.Collectors;
 
+import static org.apache.flink.cdc.common.testutils.TestCaseUtils.fetchAndConvert;
 import static org.assertj.core.api.Assertions.assertThat;
 
 /** Integration test for validating specifying starting offset. */
@@ -150,7 +147,7 @@ void testStartingFromEarliestOffset() throws Exception {
         // Execute job and validate results
         JobClient jobClient = env.executeAsync();
         iterator.setJobClient(jobClient);
-        List rows = fetchRowData(iterator, 3, customers::stringify);
+        List rows = fetchAndConvert(iterator, 3, customers::stringify);
         assertThat(rows)
                 .containsExactly(
                         "+I[15213, Alice, Rome, 123456987]",
@@ -176,7 +173,7 @@ void testStartingFromEarliestOffset() throws Exception {
         setupSavepoint(restoredEnv, savepointPath);
         JobClient restoredJobClient = restoredEnv.executeAsync();
         iterator.setJobClient(restoredJobClient);
-        List rowsAfterRestored = fetchRowData(iterator, 2, customers::stringify);
+        List rowsAfterRestored = fetchAndConvert(iterator, 2, customers::stringify);
         assertThat(rowsAfterRestored)
                 .containsExactly(
                         "-U[15213, Alice, Rome, 123456987]", "+U[15213, Alicia, Rome, 123456987]");
@@ -225,7 +222,7 @@ void testStartingFromSpecificOffset() throws Exception {
         // Execute job and validate results
         JobClient jobClient = env.executeAsync();
         iterator.setJobClient(jobClient);
-        List rows = fetchRowData(iterator, 3, customers::stringify);
+        List rows = fetchAndConvert(iterator, 3, customers::stringify);
         assertThat(rows)
                 .containsExactly(
                         "+I[15213, Alice, Rome, 123456987]",
@@ -251,7 +248,7 @@ void testStartingFromSpecificOffset() throws Exception {
         setupSavepoint(restoredEnv, savepointPath);
         JobClient restoredJobClient = restoredEnv.executeAsync("snapshotSplitTest");
         iterator.setJobClient(restoredJobClient);
-        List rowsAfterRestored = fetchRowData(iterator, 2, customers::stringify);
+        List rowsAfterRestored = fetchAndConvert(iterator, 2, customers::stringify);
         assertThat(rowsAfterRestored)
                 .containsExactly(
                         "-U[15213, Alice, Rome, 123456987]", "+U[15213, Alicia, Rome, 123456987]");
@@ -398,7 +395,7 @@ void testStartingFromTimestampOffset() throws Exception {
         // Execute job and validate results
         JobClient jobClient = env.executeAsync();
         iterator.setJobClient(jobClient);
-        List rows = fetchRowData(iterator, 3, customers::stringify);
+        List rows = fetchAndConvert(iterator, 3, customers::stringify);
         assertThat(rows)
                 .containsExactly(
                         "+I[19613, Tom, NewYork, 123456987]",
@@ -424,7 +421,7 @@ void testStartingFromTimestampOffset() throws Exception {
         setupSavepoint(restoredEnv, savepointPath);
         JobClient restoredJobClient = restoredEnv.executeAsync("snapshotSplitTest");
         iterator.setJobClient(restoredJobClient);
-        List rowsAfterRestored = fetchRowData(iterator, 2, customers::stringify);
+        List rowsAfterRestored = fetchAndConvert(iterator, 2, customers::stringify);
         assertThat(rowsAfterRestored)
                 .containsExactly(
                         "-U[18213, Charlie, Paris, 123456987]",
@@ -500,17 +497,6 @@ private  CollectResultIterator addCollector(
         return iterator;
     }
 
-    private List fetchRowData(
-            Iterator iter, int size, Function stringifier) {
-        List rows = new ArrayList<>(size);
-        while (size > 0 && iter.hasNext()) {
-            RowData row = iter.next();
-            rows.add(row);
-            size--;
-        }
-        return rows.stream().map(stringifier).collect(Collectors.toList());
-    }
-
     private static String buildMySqlConfigWithTimezone(File resourceDirectory, String timezone) {
         try {
             TemporaryFolder tempFolder = new TemporaryFolder(resourceDirectory);
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/org/apache/flink/cdc/connectors/mysql/table/MySqlCompatibilityITCase.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/org/apache/flink/cdc/connectors/mysql/table/MySqlCompatibilityITCase.java
index 23a4b25a89f..ef61471a9f8 100644
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/org/apache/flink/cdc/connectors/mysql/table/MySqlCompatibilityITCase.java
+++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/org/apache/flink/cdc/connectors/mysql/table/MySqlCompatibilityITCase.java
@@ -44,16 +44,14 @@
 import java.nio.file.StandardOpenOption;
 import java.sql.Connection;
 import java.sql.Statement;
-import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
-import java.util.Iterator;
-import java.util.List;
 import java.util.Objects;
 import java.util.Random;
 import java.util.UUID;
 import java.util.stream.Stream;
 
+import static org.apache.flink.cdc.common.testutils.TestCaseUtils.fetchAndConvert;
 import static org.apache.flink.cdc.connectors.mysql.source.MySqlSourceTestBase.assertEqualsInAnyOrder;
 import static org.apache.flink.cdc.connectors.mysql.source.MySqlSourceTestBase.assertEqualsInOrder;
 
@@ -185,7 +183,8 @@ private void testDifferentMySqlVersion(MySqlVersion version, boolean enableGtid)
                     "+I[109, spare tire, 24 inch spare tire, 22.200]"
                 };
         assertEqualsInAnyOrder(
-                Arrays.asList(expectedSnapshot), fetchRows(iterator, expectedSnapshot.length));
+                Arrays.asList(expectedSnapshot),
+                fetchAndConvert(iterator, expectedSnapshot.length, Row::toString));
 
         try (Connection connection = testDatabase.getJdbcConnection();
                 Statement statement = connection.createStatement()) {
@@ -218,7 +217,8 @@ private void testDifferentMySqlVersion(MySqlVersion version, boolean enableGtid)
                 };
 
         assertEqualsInOrder(
-                Arrays.asList(expectedBinlog), fetchRows(iterator, expectedBinlog.length));
+                Arrays.asList(expectedBinlog),
+                fetchAndConvert(iterator, expectedBinlog.length, Row::toString));
         result.getJobClient().get().cancel().get();
         mySqlContainer.stop();
     }
@@ -229,16 +229,6 @@ private String getServerId() {
         return serverId + "-" + (serverId + env.getParallelism());
     }
 
-    private static List fetchRows(Iterator iter, int size) {
-        List rows = new ArrayList<>(size);
-        while (size > 0 && iter.hasNext()) {
-            Row row = iter.next();
-            rows.add(row.toString());
-            size--;
-        }
-        return rows;
-    }
-
     private String buildCustomMySqlConfig(MySqlVersion version, boolean enableGtid) {
         try {
             File folder = tempFolder.newFolder(String.valueOf(UUID.randomUUID()));
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/org/apache/flink/cdc/connectors/mysql/table/MySqlConnectorITCase.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/org/apache/flink/cdc/connectors/mysql/table/MySqlConnectorITCase.java
index 799e96ffa60..001a85be595 100644
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/org/apache/flink/cdc/connectors/mysql/table/MySqlConnectorITCase.java
+++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/org/apache/flink/cdc/connectors/mysql/table/MySqlConnectorITCase.java
@@ -57,7 +57,6 @@
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
-import java.util.Iterator;
 import java.util.List;
 import java.util.Optional;
 import java.util.Random;
@@ -65,6 +64,9 @@
 import java.util.stream.Stream;
 
 import static org.apache.flink.api.common.JobStatus.RUNNING;
+import static org.apache.flink.cdc.common.testutils.TestCaseUtils.fetchAndConvert;
+import static org.apache.flink.cdc.common.testutils.TestCaseUtils.waitForSinkSize;
+import static org.apache.flink.cdc.common.testutils.TestCaseUtils.waitForSnapshotStarted;
 import static org.apache.flink.cdc.connectors.mysql.LegacyMySqlSourceTest.currentMySqlLatestOffset;
 import static org.apache.flink.cdc.connectors.mysql.MySqlTestUtils.assertContainsErrorMsg;
 import static org.apache.flink.cdc.connectors.mysql.MySqlTestUtils.waitForJobStatus;
@@ -236,7 +238,7 @@ private void runConsumingAllEventsTest(String otherTableOptions) throws Exceptio
             statement.execute("DELETE FROM products WHERE id=111;");
         }
 
-        waitForSinkSize("sink", 20);
+        waitForSinkSize("sink", false, 20);
 
         /*
          * 
@@ -341,7 +343,7 @@ private void runConsumingForNoPKTableTest(String otherTableOptions) throws Excep
         // async submit job
         TableResult result = tEnv.executeSql("INSERT INTO sink SELECT * FROM debezium_source");
         // wait until the snapshot phase finished
-        waitForSinkSize("sink", 11);
+        waitForSinkSize("sink", false, 11);
 
         try (Connection connection = inventoryDatabase.getJdbcConnection();
                 Statement statement = connection.createStatement()) {
@@ -360,7 +362,7 @@ private void runConsumingForNoPKTableTest(String otherTableOptions) throws Excep
             statement.execute("DELETE FROM products_no_pk WHERE type=111;");
         }
 
-        waitForSinkSize("sink", incrementalSnapshot ? 25 : 29);
+        waitForSinkSize("sink", false, incrementalSnapshot ? 25 : 29);
 
         /*
          * 
@@ -473,7 +475,8 @@ public void testCheckpointIsOptionalUnderSingleParallelism() throws Exception {
                     "+I[109, spare tire, 24 inch spare tire, 22.200]"
                 };
         assertEqualsInAnyOrder(
-                Arrays.asList(expectedSnapshot), fetchRows(iterator, expectedSnapshot.length));
+                Arrays.asList(expectedSnapshot),
+                fetchAndConvert(iterator, expectedSnapshot.length, Row::toString));
 
         try (Connection connection = inventoryDatabase.getJdbcConnection();
                 Statement statement = connection.createStatement()) {
@@ -499,7 +502,8 @@ public void testCheckpointIsOptionalUnderSingleParallelism() throws Exception {
                     "-D[111, scooter, Big 2-wheel scooter , 5.170]"
                 };
         assertEqualsInOrder(
-                Arrays.asList(expectedBinlog), fetchRows(iterator, expectedBinlog.length));
+                Arrays.asList(expectedBinlog),
+                fetchAndConvert(iterator, expectedBinlog.length, Row::toString));
         result.getJobClient().get().cancel().get();
     }
 
@@ -760,7 +764,8 @@ public void testAllDataTypes(MySqlContainer mySqlContainer, UniqueDatabase datab
                             + "]",
                 };
 
-        assertEqualsInAnyOrder(Arrays.asList(expected), fetchRows(iterator, expected.length));
+        assertEqualsInAnyOrder(
+                Arrays.asList(expected), fetchAndConvert(iterator, expected.length, Row::toString));
         result.getJobClient().get().cancel().get();
     }
 
@@ -830,7 +835,8 @@ public void testWideTable() throws Exception {
                     "+U[0, 1024, " + getIntegerSeqString(2, tableColumnCount) + "]"
                 };
 
-        assertEqualsInAnyOrder(Arrays.asList(expected), fetchRows(iterator, expected.length));
+        assertEqualsInAnyOrder(
+                Arrays.asList(expected), fetchAndConvert(iterator, expected.length, Row::toString));
         result.getJobClient().get().cancel().get();
     }
 
@@ -897,7 +903,7 @@ public void testBigTableWithHugeSplits() throws Exception {
         TableResult result = tEnv.executeSql("INSERT INTO sink SELECT * FROM big_table");
 
         // wait for snapshot finished and begin binlog
-        waitForSinkSize("sink", tableRowNumber * 2);
+        waitForSinkSize("sink", false, tableRowNumber * 2);
 
         try (Connection connection = fullTypesMySql57Database.getJdbcConnection();
                 Statement statement = connection.createStatement()) {
@@ -907,7 +913,7 @@ public void testBigTableWithHugeSplits() throws Exception {
             statement.execute("UPDATE big_table2 SET str = '2049' WHERE id=3;");
         }
         // wait for snapshot finished and begin binlog
-        waitForSinkSize("sink", tableRowNumber * 2 + 4);
+        waitForSinkSize("sink", false, tableRowNumber * 2 + 4);
 
         List expected = new ArrayList<>();
         // snapshot result after upsert into the sink
@@ -988,7 +994,7 @@ public void testMetadataColumns() throws Exception {
         TableResult result = tEnv.executeSql("INSERT INTO sink SELECT * FROM mysql_users");
 
         // wait for snapshot finished and begin binlog
-        waitForSinkSize("sink", 2);
+        waitForSinkSize("sink", false, 2);
 
         try (Connection connection = userDatabase1.getJdbcConnection();
                 Statement statement = connection.createStatement()) {
@@ -1003,7 +1009,7 @@ public void testMetadataColumns() throws Exception {
         }
 
         // waiting for binlog finished (5 more events)
-        waitForSinkSize("sink", 7);
+        waitForSinkSize("sink", false, 7);
 
         List expected =
                 Stream.of(
@@ -1096,7 +1102,8 @@ public void testStartupFromLatestOffset() throws Exception {
                     "+U[111, scooter, Big 2-wheel scooter , 5.170]",
                     "-D[111, scooter, Big 2-wheel scooter , 5.170]"
                 };
-        assertEqualsInAnyOrder(Arrays.asList(expected), fetchRows(iterator, expected.length));
+        assertEqualsInAnyOrder(
+                Arrays.asList(expected), fetchAndConvert(iterator, expected.length, Row::toString));
         result.getJobClient().get().cancel().get();
     }
 
@@ -1174,7 +1181,8 @@ public void testPrimaryKeyWithVarbinaryType() throws Exception {
                     "+U[[4, 4, 4, 4, 4, 4, 4, 5], 2021-03-08, 50, 500, flink]",
                     "-D[[4, 4, 4, 4, 4, 4, 4, 6], 2021-03-08, 30, 500, flink-sql]"
                 };
-        assertEqualsInAnyOrder(Arrays.asList(expected), fetchRows(iterator, expected.length));
+        assertEqualsInAnyOrder(
+                Arrays.asList(expected), fetchAndConvert(iterator, expected.length, Row::toString));
         result.getJobClient().get().cancel().get();
     }
 
@@ -1241,7 +1249,8 @@ public void testPrimaryKeyWithSnowflakeAlgorithm() throws Exception {
                     "+U[416927583791428523, China, Hangzhou, West Town address 2]",
                     "+I[418257940021724075, Germany, Berlin, West Town address 3]"
                 };
-        assertEqualsInAnyOrder(Arrays.asList(expected), fetchRows(iterator, expected.length));
+        assertEqualsInAnyOrder(
+                Arrays.asList(expected), fetchAndConvert(iterator, expected.length, Row::toString));
         result.getJobClient().get().cancel().get();
     }
 
@@ -1307,7 +1316,8 @@ public void testReadingWithDotTableName() throws Exception {
                     "+U[103, user_3, Hangzhou, 123567891234]",
                     "+I[110, newCustomer, Berlin, 12345678]"
                 };
-        assertEqualsInAnyOrder(Arrays.asList(expected), fetchRows(iterator, expected.length));
+        assertEqualsInAnyOrder(
+                Arrays.asList(expected), fetchAndConvert(iterator, expected.length, Row::toString));
         result.getJobClient().get().cancel().get();
         customer3_0Database.dropDatabase();
     }
@@ -1381,7 +1391,8 @@ public void testReadingWithRegexPattern() throws Exception {
                     "+I[1019, user_20, Shanghai, 123567891234]",
                     "+I[2000, user_21, Shanghai, 123567891234]"
                 };
-        assertEqualsInAnyOrder(Arrays.asList(expected), fetchRows(iterator, expected.length));
+        assertEqualsInAnyOrder(
+                Arrays.asList(expected), fetchAndConvert(iterator, expected.length, Row::toString));
         result.getJobClient().get().cancel().get();
     }
 
@@ -1519,7 +1530,8 @@ public void testDdlWithDefaultStringValue() throws Exception {
                             + "     tiny_un_c TINYINT UNSIGNED DEFAULT ' 28 '"
                             + " );");
         }
-        assertEqualsInAnyOrder(Arrays.asList(expected), fetchRows(iterator, expected.length));
+        assertEqualsInAnyOrder(
+                Arrays.asList(expected), fetchAndConvert(iterator, expected.length, Row::toString));
         jobClient.cancel().get();
     }
 
@@ -1588,7 +1600,8 @@ public void testAlterWithDefaultStringValue() throws Exception {
             statement.execute(
                     "alter table default_value_test add column `int_test` INT DEFAULT ' 30 ';");
         }
-        assertEqualsInAnyOrder(Arrays.asList(expected), fetchRows(iterator, expected.length));
+        assertEqualsInAnyOrder(
+                Arrays.asList(expected), fetchAndConvert(iterator, expected.length, Row::toString));
         jobClient.cancel().get();
     }
 
@@ -1667,7 +1680,7 @@ public void testStartupFromSpecificBinlogFilePos() throws Exception {
         }
 
         // We only expect 5 records here as all UPDATE_BEFOREs are ignored with primary key defined
-        waitForSinkSize("sink", 5);
+        waitForSinkSize("sink", false, 5);
 
         String[] expected =
                 new String[] {"+I[110, jacket, new water resistent white wind breaker, 0.500]"};
@@ -1768,7 +1781,7 @@ public void testStartupFromSpecificGtidSet() throws Exception {
         }
 
         // We only expect 5 records here as all UPDATE_BEFOREs are ignored with primary key defined
-        waitForSinkSize("sink", 5);
+        waitForSinkSize("sink", false, 5);
 
         String[] expected =
                 new String[] {"+I[110, jacket, new water resistent white wind breaker, 0.500]"};
@@ -1837,7 +1850,7 @@ public void testStartupFromEarliestOffset() throws Exception {
         // async submit job
         TableResult result = tEnv.executeSql("INSERT INTO sink SELECT * FROM debezium_source");
 
-        waitForSinkSize("sink", 16);
+        waitForSinkSize("sink", false, 16);
 
         String[] expected =
                 new String[] {
@@ -1923,7 +1936,7 @@ public void testStartupFromTimestamp() throws Exception {
             statement.execute("DELETE FROM products WHERE id=111;");
         }
 
-        waitForSinkSize("sink", 5);
+        waitForSinkSize("sink", false, 5);
 
         String[] expected =
                 new String[] {"+I[110, jacket, new water resistent white wind breaker, 0.500]"};
@@ -1986,7 +1999,8 @@ public void testColumnOptionalWithDefaultValue() throws Exception {
                     "+I[123458.6789, KIND_003, user_3, my shopping cart]",
                     "+I[123459.1234, KIND_004, user_4, null]"
                 };
-        assertEqualsInAnyOrder(Arrays.asList(expected), fetchRows(iterator, expected.length));
+        assertEqualsInAnyOrder(
+                Arrays.asList(expected), fetchAndConvert(iterator, expected.length, Row::toString));
         result.getJobClient().get().cancel().get();
     }
 
@@ -2051,7 +2065,8 @@ public void testReadingWithMultiMaxValue() throws Exception {
                     "+I[E, 3, flink]",
                     "+I[e, 4, flink]"
                 };
-        assertEqualsInAnyOrder(Arrays.asList(expected), fetchRows(iterator, expected.length));
+        assertEqualsInAnyOrder(
+                Arrays.asList(expected), fetchAndConvert(iterator, expected.length, Row::toString));
         result.getJobClient().get().cancel().get();
     }
 
@@ -2183,7 +2198,8 @@ public void testBinlogTableMetadataDeserialization() throws Exception {
                     "+I[1, 127, 255, 255, 32767, 65535, 65535, 2023]",
                     "+I[2, 127, 255, 255, 32767, 65535, 65535, 2024]"
                 };
-        assertEqualsInAnyOrder(Arrays.asList(expected), fetchRows(iterator, expected.length));
+        assertEqualsInAnyOrder(
+                Arrays.asList(expected), fetchAndConvert(iterator, expected.length, Row::toString));
         result.getJobClient().get().cancel().get();
     }
 
@@ -2231,46 +2247,6 @@ private static String getIntegerSeqString(int start, int end) {
         return stringBuilder.toString();
     }
 
-    private static void waitForSnapshotStarted(String sinkName) throws InterruptedException {
-        while (sinkSize(sinkName) == 0) {
-            Thread.sleep(100);
-        }
-    }
-
-    private static void waitForSinkSize(String sinkName, int expectedSize)
-            throws InterruptedException {
-        while (sinkSize(sinkName) < expectedSize) {
-            Thread.sleep(100);
-        }
-    }
-
-    private static int sinkSize(String sinkName) {
-        synchronized (TestValuesTableFactory.class) {
-            try {
-                return TestValuesTableFactory.getRawResultsAsStrings(sinkName).size();
-            } catch (IllegalArgumentException e) {
-                // job is not started yet
-                return 0;
-            }
-        }
-    }
-
-    private static List fetchRows(Iterator iter, int size) {
-        List rows = new ArrayList<>(size);
-        while (size > 0 && iter.hasNext()) {
-            Row row = iter.next();
-            rows.add(row.toString());
-            size--;
-        }
-        return rows;
-    }
-
-    private static void waitForSnapshotStarted(CloseableIterator iterator) throws Exception {
-        while (!iterator.hasNext()) {
-            Thread.sleep(100);
-        }
-    }
-
     @Test
     public void testBinaryHandlingModeWithBase64() throws Exception {
         if (!incrementalSnapshot) {
@@ -2348,7 +2324,8 @@ public void testBinaryHandlingModeWithBase64() throws Exception {
                     "+U[6, BAQEBAQEBAU=, 2021-03-08, 50, 500, flink]",
                     "-D[7, BAQEBAQEBAY=, 2021-03-08, 30, 500, flink-sql]"
                 };
-        assertEqualsInAnyOrder(Arrays.asList(expected), fetchRows(iterator, expected.length));
+        assertEqualsInAnyOrder(
+                Arrays.asList(expected), fetchAndConvert(iterator, expected.length, Row::toString));
         result.getJobClient().get().cancel().get();
     }
 }
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/org/apache/flink/cdc/connectors/mysql/table/MySqlConnectorShardingTableITCase.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/org/apache/flink/cdc/connectors/mysql/table/MySqlConnectorShardingTableITCase.java
index 8ee1e9f28cb..b8dfa23edfa 100644
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/org/apache/flink/cdc/connectors/mysql/table/MySqlConnectorShardingTableITCase.java
+++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/org/apache/flink/cdc/connectors/mysql/table/MySqlConnectorShardingTableITCase.java
@@ -41,13 +41,15 @@
 
 import java.sql.Connection;
 import java.sql.Statement;
-import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Iterator;
 import java.util.List;
 import java.util.Random;
 import java.util.stream.Stream;
 
+import static org.apache.flink.cdc.common.testutils.TestCaseUtils.fetchAndConvert;
+import static org.apache.flink.cdc.common.testutils.TestCaseUtils.waitForSinkSize;
+import static org.apache.flink.cdc.common.testutils.TestCaseUtils.waitForSnapshotStarted;
+
 /** Integration tests for MySQL shardding tables. */
 @RunWith(Parameterized.class)
 public class MySqlConnectorShardingTableITCase extends MySqlSourceTestBase {
@@ -176,7 +178,7 @@ public void testShardingTablesWithTinyInt1() throws Exception {
             statement.execute("INSERT INTO sharding_table_1 values(3, true),(4, false)");
         }
         // wait for snapshot finished and begin binlog
-        waitForSinkSize("sink", 4);
+        waitForSinkSize("sink", false, 4);
 
         try (Connection connection = fullTypesMySql57Database.getJdbcConnection();
                 Statement statement = connection.createStatement()) {
@@ -191,7 +193,7 @@ public void testShardingTablesWithTinyInt1() throws Exception {
             statement.execute("INSERT INTO sharding_table_2 values(5, true),(6, false)");
         }
 
-        waitForSinkSize("sink", 6);
+        waitForSinkSize("sink", false, 6);
         String[] expected =
                 new String[] {
                     "+I[1, 1]", "+I[2, 0]", "+I[3, 1]", "+I[4, 0]", "+I[5, 1]", "+I[6, 0]",
@@ -269,7 +271,8 @@ public void testShardingTablesWithInconsistentSchema() throws Exception {
                     "+U[221, user_221, Shanghai, 123567891234, null, 20]",
                 };
 
-        assertEqualsInAnyOrder(Arrays.asList(expected), fetchRows(iterator, expected.length));
+        assertEqualsInAnyOrder(
+                Arrays.asList(expected), fetchAndConvert(iterator, expected.length, Row::toString));
         result.getJobClient().get().cancel().get();
     }
 
@@ -316,44 +319,4 @@ private static String getIntegerSeqString(int start, int end) {
         stringBuilder.append(end - 1);
         return stringBuilder.toString();
     }
-
-    private static void waitForSnapshotStarted(String sinkName) throws InterruptedException {
-        while (sinkSize(sinkName) == 0) {
-            Thread.sleep(100);
-        }
-    }
-
-    private static void waitForSinkSize(String sinkName, int expectedSize)
-            throws InterruptedException {
-        while (sinkSize(sinkName) < expectedSize) {
-            Thread.sleep(100);
-        }
-    }
-
-    private static int sinkSize(String sinkName) {
-        synchronized (TestValuesTableFactory.class) {
-            try {
-                return TestValuesTableFactory.getRawResults(sinkName).size();
-            } catch (IllegalArgumentException e) {
-                // job is not started yet
-                return 0;
-            }
-        }
-    }
-
-    private static List fetchRows(Iterator iter, int size) {
-        List rows = new ArrayList<>(size);
-        while (size > 0 && iter.hasNext()) {
-            Row row = iter.next();
-            rows.add(row.toString());
-            size--;
-        }
-        return rows;
-    }
-
-    private static void waitForSnapshotStarted(CloseableIterator iterator) throws Exception {
-        while (!iterator.hasNext()) {
-            Thread.sleep(100);
-        }
-    }
 }
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/org/apache/flink/cdc/connectors/mysql/table/MySqlJsonArrayAsKeyIndexITCase.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/org/apache/flink/cdc/connectors/mysql/table/MySqlJsonArrayAsKeyIndexITCase.java
index 94d654e21a6..059e6700451 100644
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/org/apache/flink/cdc/connectors/mysql/table/MySqlJsonArrayAsKeyIndexITCase.java
+++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/org/apache/flink/cdc/connectors/mysql/table/MySqlJsonArrayAsKeyIndexITCase.java
@@ -39,14 +39,12 @@
 
 import java.sql.Connection;
 import java.sql.Statement;
-import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Iterator;
-import java.util.List;
 import java.util.Random;
 import java.util.stream.Stream;
 
 import static org.apache.flink.api.common.JobStatus.RUNNING;
+import static org.apache.flink.cdc.common.testutils.TestCaseUtils.fetchAndConvert;
 
 /** Integration tests for MySQL Table source. */
 @RunWith(Parameterized.class)
@@ -100,7 +98,7 @@ public void after() {
     }
 
     @Test
-    public void testJsonArrayAsKeyIndex() {
+    public void testJsonArrayAsKeyIndex() throws InterruptedException {
         UniqueDatabase jaakiDatabase =
                 new UniqueDatabase(container, "json_array_as_key", TEST_USER, TEST_PASSWORD);
         jaakiDatabase.createAndInitialize();
@@ -161,7 +159,8 @@ public void testJsonArrayAsKeyIndex() {
                     "+I[17]", "+I[18]", "+I[19]", "-D[19]",
                 };
 
-        assertEqualsInAnyOrder(Arrays.asList(expected), fetchRows(iterator, expected.length));
+        assertEqualsInAnyOrder(
+                Arrays.asList(expected), fetchAndConvert(iterator, expected.length, Row::toString));
 
         try {
             result.getJobClient().get().cancel().get();
@@ -170,16 +169,6 @@ public void testJsonArrayAsKeyIndex() {
         }
     }
 
-    private static List fetchRows(Iterator iter, int size) {
-        List rows = new ArrayList<>(size);
-        while (size > 0 && iter.hasNext()) {
-            Row row = iter.next();
-            rows.add(row.toString());
-            size--;
-        }
-        return rows;
-    }
-
     private String getServerId() {
         final Random random = new Random();
         int serverId = random.nextInt(100) + 5400;
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/org/apache/flink/cdc/connectors/mysql/table/MySqlOnLineSchemaMigrationTableITCase.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/org/apache/flink/cdc/connectors/mysql/table/MySqlOnLineSchemaMigrationTableITCase.java
index 92e5fc566df..a3ecd97af6c 100644
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/org/apache/flink/cdc/connectors/mysql/table/MySqlOnLineSchemaMigrationTableITCase.java
+++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/org/apache/flink/cdc/connectors/mysql/table/MySqlOnLineSchemaMigrationTableITCase.java
@@ -23,8 +23,8 @@
 import org.apache.flink.cdc.common.event.Event;
 import org.apache.flink.cdc.common.event.TableId;
 import org.apache.flink.cdc.common.schema.Schema;
+import org.apache.flink.cdc.common.testutils.TestCaseUtils;
 import org.apache.flink.cdc.common.types.DataType;
-import org.apache.flink.cdc.common.utils.TestCaseUtils;
 import org.apache.flink.cdc.connectors.mysql.source.MySqlSourceTestBase;
 import org.apache.flink.cdc.connectors.mysql.testutils.MySqlContainer;
 import org.apache.flink.cdc.connectors.mysql.testutils.MySqlVersion;
@@ -52,9 +52,7 @@
 import java.io.IOException;
 import java.sql.Connection;
 import java.sql.Statement;
-import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Iterator;
 import java.util.List;
 import java.util.NoSuchElementException;
 import java.util.Random;
@@ -62,8 +60,9 @@
 import java.util.stream.Stream;
 
 import static org.apache.flink.api.common.JobStatus.RUNNING;
-import static org.apache.flink.cdc.common.utils.TestCaseUtils.DEFAULT_INTERVAL;
-import static org.apache.flink.cdc.common.utils.TestCaseUtils.DEFAULT_TIMEOUT;
+import static org.apache.flink.cdc.common.testutils.TestCaseUtils.DEFAULT_INTERVAL;
+import static org.apache.flink.cdc.common.testutils.TestCaseUtils.DEFAULT_TIMEOUT;
+import static org.apache.flink.cdc.common.testutils.TestCaseUtils.fetchAndConvert;
 
 /**
  * IT case for Evolving MySQL schema with gh-ost/pt-osc utility. See  fetchRows(Iterator iter, int size) {
-        List rows = new ArrayList<>(size);
-        while (size > 0 && iter.hasNext()) {
-            Row row = iter.next();
-            rows.add(row.toString());
-            size--;
-        }
-        return rows;
-    }
 }
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/org/apache/flink/cdc/connectors/mysql/table/MySqlTimezoneITCase.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/org/apache/flink/cdc/connectors/mysql/table/MySqlTimezoneITCase.java
index 7d994e8001d..3d0ac03008b 100644
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/org/apache/flink/cdc/connectors/mysql/table/MySqlTimezoneITCase.java
+++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/org/apache/flink/cdc/connectors/mysql/table/MySqlTimezoneITCase.java
@@ -45,16 +45,15 @@
 import java.nio.file.StandardOpenOption;
 import java.sql.Connection;
 import java.sql.Statement;
-import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
-import java.util.Iterator;
 import java.util.List;
 import java.util.Objects;
 import java.util.Random;
 import java.util.UUID;
 import java.util.stream.Stream;
 
+import static org.apache.flink.cdc.common.testutils.TestCaseUtils.fetchAndConvert;
 import static org.apache.flink.cdc.connectors.mysql.source.MySqlSourceTestBase.assertEqualsInAnyOrder;
 import static org.apache.flink.cdc.connectors.mysql.source.MySqlSourceTestBase.assertEqualsInOrder;
 
@@ -189,7 +188,8 @@ private void testTemporalTypesWithMySqlServerTimezone(String timezone) throws Ex
                     "+I[2020-07-17, 18:00:22, 2020-07-17T18:00:22.123, 2020-07-17T18:00:22.123456, 2020-07-17T18:00:22]"
                 };
         assertEqualsInAnyOrder(
-                Arrays.asList(expectedSnapshot), fetchRows(iterator, expectedSnapshot.length));
+                Arrays.asList(expectedSnapshot),
+                fetchAndConvert(iterator, expectedSnapshot.length, Row::toString));
 
         try (Connection connection = fullTypesDatabase.getJdbcConnection();
                 Statement statement = connection.createStatement()) {
@@ -205,7 +205,8 @@ private void testTemporalTypesWithMySqlServerTimezone(String timezone) throws Ex
                 };
 
         assertEqualsInOrder(
-                Arrays.asList(expectedBinlog), fetchRows(iterator, expectedBinlog.length));
+                Arrays.asList(expectedBinlog),
+                fetchAndConvert(iterator, expectedBinlog.length, Row::toString));
 
         result.getJobClient().get().cancel().get();
         mySqlContainer.stop();
@@ -228,16 +229,6 @@ private int getSplitSize() {
         return 0;
     }
 
-    private static List fetchRows(Iterator iter, int size) {
-        List rows = new ArrayList<>(size);
-        while (size > 0 && iter.hasNext()) {
-            Row row = iter.next();
-            rows.add(row.toString());
-            size--;
-        }
-        return rows;
-    }
-
     private String buildMySqlConfigWithTimezone(String timezone) {
         try {
             File folder = tempFolder.newFolder(String.valueOf(UUID.randomUUID()));
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/org/apache/flink/cdc/connectors/mysql/table/MysqlConnectorCharsetITCase.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/org/apache/flink/cdc/connectors/mysql/table/MysqlConnectorCharsetITCase.java
index 197452389de..d7b9de15535 100644
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/org/apache/flink/cdc/connectors/mysql/table/MysqlConnectorCharsetITCase.java
+++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/org/apache/flink/cdc/connectors/mysql/table/MysqlConnectorCharsetITCase.java
@@ -35,12 +35,12 @@
 
 import java.sql.Connection;
 import java.sql.Statement;
-import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Iterator;
-import java.util.List;
 import java.util.Random;
 
+import static org.apache.flink.cdc.common.testutils.TestCaseUtils.fetchAndConvert;
+import static org.apache.flink.cdc.common.testutils.TestCaseUtils.waitForSnapshotStarted;
+
 /** Test supporting different column charsets for MySQL Table source. */
 @RunWith(Parameterized.class)
 public class MysqlConnectorCharsetITCase extends MySqlSourceTestBase {
@@ -379,7 +379,8 @@ public void testCharset() throws Exception {
         CloseableIterator iterator = result.collect();
         waitForSnapshotStarted(iterator);
         assertEqualsInAnyOrder(
-                Arrays.asList(snapshotExpected), fetchRows(iterator, snapshotExpected.length));
+                Arrays.asList(snapshotExpected),
+                fetchAndConvert(iterator, snapshotExpected.length, Row::toString));
 
         // test binlog phase
         try (Connection connection = charsetTestDatabase.getJdbcConnection();
@@ -387,7 +388,8 @@ public void testCharset() throws Exception {
             statement.execute(String.format("UPDATE %s SET table_id = table_id + 10;", testName));
         }
         assertEqualsInAnyOrder(
-                Arrays.asList(binlogExpected), fetchRows(iterator, binlogExpected.length));
+                Arrays.asList(binlogExpected),
+                fetchAndConvert(iterator, binlogExpected.length, Row::toString));
         result.getJobClient().get().cancel().get();
     }
 
@@ -396,20 +398,4 @@ private String getServerId() {
         int serverId = random.nextInt(100) + 5400;
         return serverId + "-" + (serverId + env.getParallelism());
     }
-
-    private static List fetchRows(Iterator iter, int size) {
-        List rows = new ArrayList<>(size);
-        while (size > 0 && iter.hasNext()) {
-            Row row = iter.next();
-            rows.add(row.toString());
-            size--;
-        }
-        return rows;
-    }
-
-    private static void waitForSnapshotStarted(CloseableIterator iterator) throws Exception {
-        while (!iterator.hasNext()) {
-            Thread.sleep(100);
-        }
-    }
 }
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/org/apache/flink/cdc/connectors/polardbx/PolardbxCharsetITCase.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/org/apache/flink/cdc/connectors/polardbx/PolardbxCharsetITCase.java
index d4ecb5d974f..1e8f7d3d50f 100644
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/org/apache/flink/cdc/connectors/polardbx/PolardbxCharsetITCase.java
+++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/org/apache/flink/cdc/connectors/polardbx/PolardbxCharsetITCase.java
@@ -36,6 +36,9 @@
 import java.sql.Statement;
 import java.util.Arrays;
 
+import static org.apache.flink.cdc.common.testutils.TestCaseUtils.fetchAndConvert;
+import static org.apache.flink.cdc.common.testutils.TestCaseUtils.waitForSnapshotStarted;
+
 /** Test supporting different column charsets for Polardbx. */
 @RunWith(Parameterized.class)
 public class PolardbxCharsetITCase extends PolardbxSourceTestBase {
@@ -167,8 +170,8 @@ public void testCharset() throws Exception {
                                 + " 'scan.incremental.snapshot.chunk.size' = '%s'"
                                 + ")",
                         testName,
-                        HOST_NAME,
-                        PORT,
+                        getHost(),
+                        getPort(),
                         USER_NAME,
                         PASSWORD,
                         DATABASE,
@@ -185,7 +188,8 @@ public void testCharset() throws Exception {
         CloseableIterator iterator = result.collect();
         waitForSnapshotStarted(iterator);
         assertEqualsInAnyOrder(
-                Arrays.asList(snapshotExpected), fetchRows(iterator, snapshotExpected.length));
+                Arrays.asList(snapshotExpected),
+                fetchAndConvert(iterator, snapshotExpected.length, WAITING_TIMEOUT, Row::toString));
 
         // test binlog phase
         try (Connection connection = getJdbcConnection();
@@ -196,13 +200,8 @@ public void testCharset() throws Exception {
                             DATABASE, testName));
         }
         assertEqualsInAnyOrder(
-                Arrays.asList(binlogExpected), fetchRows(iterator, binlogExpected.length));
+                Arrays.asList(binlogExpected),
+                fetchAndConvert(iterator, binlogExpected.length, WAITING_TIMEOUT, Row::toString));
         result.getJobClient().get().cancel().get();
     }
-
-    private static void waitForSnapshotStarted(CloseableIterator iterator) throws Exception {
-        while (!iterator.hasNext()) {
-            Thread.sleep(100);
-        }
-    }
 }
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/org/apache/flink/cdc/connectors/polardbx/PolardbxSourceITCase.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/org/apache/flink/cdc/connectors/polardbx/PolardbxSourceITCase.java
index db87a98acb4..e0ec69ae404 100644
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/org/apache/flink/cdc/connectors/polardbx/PolardbxSourceITCase.java
+++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/org/apache/flink/cdc/connectors/polardbx/PolardbxSourceITCase.java
@@ -36,6 +36,8 @@
 import java.util.List;
 
 import static java.lang.String.format;
+import static org.apache.flink.cdc.common.testutils.TestCaseUtils.fetchAndConvert;
+import static org.apache.flink.cdc.common.testutils.TestCaseUtils.waitForSinkSize;
 
 /**
  * Database Polardbx supported the mysql protocol, but there are some different features in ddl. So
@@ -83,8 +85,8 @@ public void testSingleKey() throws Exception {
                                 + " 'server-time-zone' = 'UTC',"
                                 + " 'server-id' = '%s'"
                                 + ")",
-                        HOST_NAME,
-                        PORT,
+                        getHost(),
+                        getPort(),
                         USER_NAME,
                         PASSWORD,
                         DATABASE,
@@ -108,7 +110,9 @@ public void testSingleKey() throws Exception {
             expectedSnapshotData.addAll(Arrays.asList(snapshotForSingleTable));
         }
 
-        List realSnapshotData = fetchRows(iterator, expectedSnapshotData.size());
+        List realSnapshotData =
+                fetchAndConvert(
+                        iterator, expectedSnapshotData.size(), WAITING_TIMEOUT, Row::toString);
         assertEqualsInAnyOrder(expectedSnapshotData, realSnapshotData);
 
         // second step: check the sink data
@@ -126,7 +130,7 @@ public void testSingleKey() throws Exception {
                         + ")");
         tableResult = tEnv.executeSql("insert into sink select * from orders_source");
 
-        waitForSinkSize("sink", realSnapshotData.size());
+        waitForSinkSize("sink", false, realSnapshotData.size());
         assertEqualsInAnyOrder(
                 expectedSnapshotData, TestValuesTableFactory.getRawResultsAsStrings("sink"));
 
@@ -155,7 +159,8 @@ public void testSingleKey() throws Exception {
         for (int i = 0; i < captureCustomerTables.length; i++) {
             expectedBinlogData.addAll(Arrays.asList(expectedBinlog));
         }
-        List realBinlog = fetchRows(iterator, expectedBinlog.length);
+        List realBinlog =
+                fetchAndConvert(iterator, expectedBinlog.length, WAITING_TIMEOUT, Row::toString);
         assertEqualsInOrder(expectedBinlogData, realBinlog);
         tableResult.getJobClient().get().cancel().get();
     }
@@ -234,8 +239,8 @@ public void testFullTypesDdl() throws Exception {
                                 + " 'server-time-zone' = 'UTC',"
                                 + " 'server-id' = '%s'"
                                 + ")",
-                        HOST_NAME,
-                        PORT,
+                        getHost(),
+                        getPort(),
                         USER_NAME,
                         PASSWORD,
                         DATABASE,
@@ -245,7 +250,8 @@ public void testFullTypesDdl() throws Exception {
 
         TableResult tableResult = tEnv.executeSql("select * from polardbx_full_types");
         CloseableIterator iterator = tableResult.collect();
-        List realSnapshotData = fetchRows(iterator, 1);
+        List realSnapshotData =
+                fetchAndConvert(iterator, 1, WAITING_TIMEOUT, Row::toString);
         String[] expectedSnapshotData =
                 new String[] {
                     "+I[100001, 127, 255, 32767, 65535, 8388607, 16777215, 2147483647, 4294967295, 2147483647, "
@@ -301,8 +307,8 @@ public void testMultiKeys() throws Exception {
                                 + " 'server-time-zone' = 'UTC',"
                                 + " 'server-id' = '%s'"
                                 + ")",
-                        HOST_NAME,
-                        PORT,
+                        getHost(),
+                        getPort(),
                         USER_NAME,
                         PASSWORD,
                         DATABASE,
@@ -326,7 +332,9 @@ public void testMultiKeys() throws Exception {
             expectedSnapshotData.addAll(Arrays.asList(snapshotForSingleTable));
         }
 
-        List realSnapshotData = fetchRows(iterator, expectedSnapshotData.size());
+        List realSnapshotData =
+                fetchAndConvert(
+                        iterator, expectedSnapshotData.size(), WAITING_TIMEOUT, Row::toString);
         assertEqualsInAnyOrder(expectedSnapshotData, realSnapshotData);
 
         // second step: check the sink data
@@ -345,7 +353,7 @@ public void testMultiKeys() throws Exception {
 
         tEnv.executeSql("insert into multi_key_sink select * from orders_with_multi_pks");
 
-        waitForSinkSize("multi_key_sink", realSnapshotData.size());
+        waitForSinkSize("multi_key_sink", false, realSnapshotData.size());
         assertEqualsInAnyOrder(
                 expectedSnapshotData,
                 TestValuesTableFactory.getRawResultsAsStrings("multi_key_sink"));
@@ -375,7 +383,8 @@ public void testMultiKeys() throws Exception {
                     "+I[7, 9999, 9999, 1007, 2022-01-17T00:00]",
                     "-D[7, 9999, 9999, 1007, 2022-01-17T00:00]"
                 };
-        List realBinlog = fetchRows(iterator, expectedBinlog.length);
+        List realBinlog =
+                fetchAndConvert(iterator, expectedBinlog.length, WAITING_TIMEOUT, Row::toString);
         assertEqualsInAnyOrder(Arrays.asList(expectedBinlog), realBinlog);
         tableResult.getJobClient().get().cancel().get();
     }
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/org/apache/flink/cdc/connectors/polardbx/PolardbxSourceTestBase.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/org/apache/flink/cdc/connectors/polardbx/PolardbxSourceTestBase.java
index 43c3c678393..a4e0a985a7d 100644
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/org/apache/flink/cdc/connectors/polardbx/PolardbxSourceTestBase.java
+++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/org/apache/flink/cdc/connectors/polardbx/PolardbxSourceTestBase.java
@@ -17,13 +17,9 @@
 
 package org.apache.flink.cdc.connectors.polardbx;
 
-import org.apache.flink.table.planner.factories.TestValuesTableFactory;
+import org.apache.flink.cdc.common.testutils.TestCaseUtils;
 import org.apache.flink.test.util.AbstractTestBase;
-import org.apache.flink.types.Row;
 
-import com.github.dockerjava.api.model.ExposedPort;
-import com.github.dockerjava.api.model.PortBinding;
-import com.github.dockerjava.api.model.Ports;
 import org.apache.commons.lang3.StringUtils;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -42,9 +38,7 @@
 import java.sql.SQLException;
 import java.sql.Statement;
 import java.time.Duration;
-import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Iterator;
 import java.util.List;
 import java.util.Random;
 import java.util.function.Function;
@@ -64,36 +58,37 @@
 public abstract class PolardbxSourceTestBase extends AbstractTestBase {
     private static final Logger LOG = LoggerFactory.getLogger(PolardbxSourceTestBase.class);
     private static final Pattern COMMENT_PATTERN = Pattern.compile("^(.*)--.*$");
-    protected static final Integer PORT = 8527;
-    protected static final String HOST_NAME = "127.0.0.1";
-    protected static final String USER_NAME = "polardbx_root";
-    protected static final String PASSWORD = "123456";
+
     private static final String IMAGE_VERSION = "2.1.0";
     private static final DockerImageName POLARDBX_IMAGE =
             DockerImageName.parse("polardbx/polardb-x:" + IMAGE_VERSION);
 
+    protected static final Integer INNER_PORT = 8527;
+    protected static final String USER_NAME = "polardbx_root";
+    protected static final String PASSWORD = "123456";
+    protected static final Duration WAITING_TIMEOUT = Duration.ofMinutes(1);
+
     protected static final GenericContainer POLARDBX_CONTAINER =
             new GenericContainer<>(POLARDBX_IMAGE)
-                    .withExposedPorts(PORT)
+                    .withExposedPorts(INNER_PORT)
                     .withLogConsumer(new Slf4jLogConsumer(LOG))
-                    .withStartupTimeout(Duration.ofMinutes(3))
-                    .withCreateContainerCmdModifier(
-                            c ->
-                                    c.withPortBindings(
-                                            new PortBinding(
-                                                    Ports.Binding.bindPort(PORT),
-                                                    new ExposedPort(PORT))));
+                    .withStartupTimeout(Duration.ofMinutes(3));
+
+    protected static String getHost() {
+        return POLARDBX_CONTAINER.getHost();
+    }
+
+    protected static int getPort() {
+        return POLARDBX_CONTAINER.getMappedPort(INNER_PORT);
+    }
 
     @BeforeClass
-    public static void startContainers() throws InterruptedException {
-        // no need to start container when the port 8527 is listening
-        if (!checkConnection()) {
-            LOG.info("Polardbx connection is not valid, so try to start containers...");
-            Startables.deepStart(Stream.of(POLARDBX_CONTAINER)).join();
-            LOG.info("Containers are started.");
-            // here should wait 10s that make sure the polardbx is ready
-            Thread.sleep(10 * 1000);
-        }
+    public static void startContainers() {
+        Startables.deepStart(Stream.of(POLARDBX_CONTAINER)).join();
+        LOG.info("Containers are started.");
+
+        TestCaseUtils.repeatedCheck(
+                PolardbxSourceTestBase::checkConnection, WAITING_TIMEOUT, Duration.ofSeconds(1));
     }
 
     @AfterClass
@@ -104,7 +99,7 @@ public static void stopContainers() {
     }
 
     protected static String getJdbcUrl() {
-        return String.format("jdbc:mysql://%s:%s", HOST_NAME, PORT);
+        return String.format("jdbc:mysql://%s:%s", getHost(), getPort());
     }
 
     protected static Connection getJdbcConnection() throws SQLException {
@@ -160,16 +155,6 @@ protected static void initializePolardbxTables(
         }
     }
 
-    protected static List fetchRows(Iterator iter, int size) {
-        List rows = new ArrayList<>(size);
-        while (size > 0 && iter.hasNext()) {
-            Row row = iter.next();
-            rows.add(row.toString());
-            size--;
-        }
-        return rows;
-    }
-
     protected String getTableNameRegex(String[] captureCustomerTables) {
         checkState(captureCustomerTables.length > 0);
         if (captureCustomerTables.length == 1) {
@@ -190,24 +175,6 @@ protected String getServerId() {
     //  test utilities
     // ------------------------------------------------------------------------
 
-    protected static void waitForSinkSize(String sinkName, int expectedSize)
-            throws InterruptedException {
-        while (sinkSize(sinkName) < expectedSize) {
-            Thread.sleep(100);
-        }
-    }
-
-    protected static int sinkSize(String sinkName) {
-        synchronized (TestValuesTableFactory.class) {
-            try {
-                return TestValuesTableFactory.getRawResultsAsStrings(sinkName).size();
-            } catch (IllegalArgumentException e) {
-                // job is not started yet
-                return 0;
-            }
-        }
-    }
-
     protected static void assertEqualsInAnyOrder(List expected, List actual) {
         assertTrue(expected != null && actual != null);
         assertEqualsInOrder(
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/resources/file/debezium-data-schema-exclude-with-numeric-decimal.json b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/resources/file/debezium-data-schema-exclude-with-numeric-decimal.json
index f11a4cb3962..20c641df522 100644
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/resources/file/debezium-data-schema-exclude-with-numeric-decimal.json
+++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/resources/file/debezium-data-schema-exclude-with-numeric-decimal.json
@@ -52,14 +52,14 @@
             "enum_c": "red",
             "set_c": "a,b",
             "json_c": "{\"key1\": \"value1\"}",
-            "point_c": {"x":1.0,"y":1.0,"wkb":"AQEAAAAAAAAAAADwPwAAAAAAAPA/"},
-            "geometry_c":{"wkb":"AQMAAAABAAAABQAAAAAAAAAAAPA/AAAAAAAA8D8AAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAADwPwAAAAAAAPA/"},
-            "linestring_c": {"wkb":"AQIAAAADAAAAAAAAAAAACEAAAAAAAAAAAAAAAAAAAAhAAAAAAAAACEAAAAAAAAAIQAAAAAAAABRA"},
-            "polygon_c": {"wkb":"AQMAAAABAAAABQAAAAAAAAAAAPA/AAAAAAAA8D8AAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAADwPwAAAAAAAPA/"},
-            "multipoint_c": {"wkb":"AQQAAAACAAAAAQEAAAAAAAAAAADwPwAAAAAAAPA/AQEAAAAAAAAAAAAAQAAAAAAAAABA"},
-            "multiline_c": {"wkb":"AQUAAAACAAAAAQIAAAADAAAAAAAAAAAA8D8AAAAAAADwPwAAAAAAAABAAAAAAAAAAEAAAAAAAAAIQAAAAAAAAAhAAQIAAAACAAAAAAAAAAAAEEAAAAAAAAAQQAAAAAAAABRAAAAAAAAAFEA="},
-            "multipolygon_c": {"wkb":"AQYAAAACAAAAAQMAAAABAAAABQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAkQAAAAAAAAAAAAAAAAAAAJEAAAAAAAAAkQAAAAAAAAAAAAAAAAAAAJEAAAAAAAAAAAAAAAAAAAAAAAQMAAAABAAAABQAAAAAAAAAAABRAAAAAAAAAFEAAAAAAAAAcQAAAAAAAABRAAAAAAAAAHEAAAAAAAAAcQAAAAAAAABRAAAAAAAAAHEAAAAAAAAAUQAAAAAAAABRA"},
-            "geometrycollection_c": {"wkb":"AQcAAAADAAAAAQEAAAAAAAAAAAAkQAAAAAAAACRAAQEAAAAAAAAAAAA+QAAAAAAAAD5AAQIAAAACAAAAAAAAAAAALkAAAAAAAAAuQAAAAAAAADRAAAAAAAAANEA="}
+            "point_c": {"x":1.0,"y":1.0,"wkb":"AQEAAAAAAAAAAADwPwAAAAAAAPA/","srid":null},
+            "geometry_c":{"wkb":"AQMAAAABAAAABQAAAAAAAAAAAPA/AAAAAAAA8D8AAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAADwPwAAAAAAAPA/","srid":null},
+            "linestring_c": {"wkb":"AQIAAAADAAAAAAAAAAAACEAAAAAAAAAAAAAAAAAAAAhAAAAAAAAACEAAAAAAAAAIQAAAAAAAABRA","srid":null},
+            "polygon_c": {"wkb":"AQMAAAABAAAABQAAAAAAAAAAAPA/AAAAAAAA8D8AAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAADwPwAAAAAAAPA/","srid":null},
+            "multipoint_c": {"wkb":"AQQAAAACAAAAAQEAAAAAAAAAAADwPwAAAAAAAPA/AQEAAAAAAAAAAAAAQAAAAAAAAABA","srid":null},
+            "multiline_c": {"wkb":"AQUAAAACAAAAAQIAAAADAAAAAAAAAAAA8D8AAAAAAADwPwAAAAAAAABAAAAAAAAAAEAAAAAAAAAIQAAAAAAAAAhAAQIAAAACAAAAAAAAAAAAEEAAAAAAAAAQQAAAAAAAABRAAAAAAAAAFEA=","srid":null},
+            "multipolygon_c": {"wkb":"AQYAAAACAAAAAQMAAAABAAAABQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAkQAAAAAAAAAAAAAAAAAAAJEAAAAAAAAAkQAAAAAAAAAAAAAAAAAAAJEAAAAAAAAAAAAAAAAAAAAAAAQMAAAABAAAABQAAAAAAAAAAABRAAAAAAAAAFEAAAAAAAAAcQAAAAAAAABRAAAAAAAAAHEAAAAAAAAAcQAAAAAAAABRAAAAAAAAAHEAAAAAAAAAUQAAAAAAAABRA","srid":null},
+            "geometrycollection_c": {"wkb":"AQcAAAADAAAAAQEAAAAAAAAAAAAkQAAAAAAAACRAAQEAAAAAAAAAAAA+QAAAAAAAAD5AAQIAAAACAAAAAAAAAAAALkAAAAAAAAAuQAAAAAAAADRAAAAAAAAANEA=","srid":null}
         },
         "op": "r",
         "transaction": null
@@ -116,14 +116,14 @@
             "enum_c": "red",
             "set_c": "a,b",
             "json_c": "{\"key1\":\"value1\"}",
-            "point_c": {"x":1.0,"y":1.0,"wkb":"AQEAAAAAAAAAAADwPwAAAAAAAPA/"},
-            "geometry_c":{"wkb":"AQMAAAABAAAABQAAAAAAAAAAAPA/AAAAAAAA8D8AAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAADwPwAAAAAAAPA/"},
-            "linestring_c": {"wkb":"AQIAAAADAAAAAAAAAAAACEAAAAAAAAAAAAAAAAAAAAhAAAAAAAAACEAAAAAAAAAIQAAAAAAAABRA"},
-            "polygon_c": {"wkb":"AQMAAAABAAAABQAAAAAAAAAAAPA/AAAAAAAA8D8AAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAADwPwAAAAAAAPA/"},
-            "multipoint_c": {"wkb":"AQQAAAACAAAAAQEAAAAAAAAAAADwPwAAAAAAAPA/AQEAAAAAAAAAAAAAQAAAAAAAAABA"},
-            "multiline_c": {"wkb":"AQUAAAACAAAAAQIAAAADAAAAAAAAAAAA8D8AAAAAAADwPwAAAAAAAABAAAAAAAAAAEAAAAAAAAAIQAAAAAAAAAhAAQIAAAACAAAAAAAAAAAAEEAAAAAAAAAQQAAAAAAAABRAAAAAAAAAFEA="},
-            "multipolygon_c": {"wkb":"AQYAAAACAAAAAQMAAAABAAAABQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAkQAAAAAAAAAAAAAAAAAAAJEAAAAAAAAAkQAAAAAAAAAAAAAAAAAAAJEAAAAAAAAAAAAAAAAAAAAAAAQMAAAABAAAABQAAAAAAAAAAABRAAAAAAAAAFEAAAAAAAAAcQAAAAAAAABRAAAAAAAAAHEAAAAAAAAAcQAAAAAAAABRAAAAAAAAAHEAAAAAAAAAUQAAAAAAAABRA"},
-            "geometrycollection_c": {"wkb":"AQcAAAADAAAAAQEAAAAAAAAAAAAkQAAAAAAAACRAAQEAAAAAAAAAAAA+QAAAAAAAAD5AAQIAAAACAAAAAAAAAAAALkAAAAAAAAAuQAAAAAAAADRAAAAAAAAANEA="}
+            "point_c": {"x":1.0,"y":1.0,"wkb":"AQEAAAAAAAAAAADwPwAAAAAAAPA/","srid":null},
+            "geometry_c":{"wkb":"AQMAAAABAAAABQAAAAAAAAAAAPA/AAAAAAAA8D8AAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAADwPwAAAAAAAPA/","srid":null},
+            "linestring_c": {"wkb":"AQIAAAADAAAAAAAAAAAACEAAAAAAAAAAAAAAAAAAAAhAAAAAAAAACEAAAAAAAAAIQAAAAAAAABRA","srid":null},
+            "polygon_c": {"wkb":"AQMAAAABAAAABQAAAAAAAAAAAPA/AAAAAAAA8D8AAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAADwPwAAAAAAAPA/","srid":null},
+            "multipoint_c": {"wkb":"AQQAAAACAAAAAQEAAAAAAAAAAADwPwAAAAAAAPA/AQEAAAAAAAAAAAAAQAAAAAAAAABA","srid":null},
+            "multiline_c": {"wkb":"AQUAAAACAAAAAQIAAAADAAAAAAAAAAAA8D8AAAAAAADwPwAAAAAAAABAAAAAAAAAAEAAAAAAAAAIQAAAAAAAAAhAAQIAAAACAAAAAAAAAAAAEEAAAAAAAAAQQAAAAAAAABRAAAAAAAAAFEA=","srid":null},
+            "multipolygon_c": {"wkb":"AQYAAAACAAAAAQMAAAABAAAABQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAkQAAAAAAAAAAAAAAAAAAAJEAAAAAAAAAkQAAAAAAAAAAAAAAAAAAAJEAAAAAAAAAAAAAAAAAAAAAAAQMAAAABAAAABQAAAAAAAAAAABRAAAAAAAAAFEAAAAAAAAAcQAAAAAAAABRAAAAAAAAAHEAAAAAAAAAcQAAAAAAAABRAAAAAAAAAHEAAAAAAAAAUQAAAAAAAABRA","srid":null},
+            "geometrycollection_c": {"wkb":"AQcAAAADAAAAAQEAAAAAAAAAAAAkQAAAAAAAACRAAQEAAAAAAAAAAAA+QAAAAAAAAD5AAQIAAAACAAAAAAAAAAAALkAAAAAAAAAuQAAAAAAAADRAAAAAAAAANEA=","srid":null}
         },
         "after": {
             "id": 1,
@@ -176,14 +176,14 @@
             "enum_c": "red",
             "set_c": "a,b",
             "json_c": "{\"key1\":\"value1\"}",
-            "point_c": {"x":1.0,"y":1.0,"wkb":"AQEAAAAAAAAAAADwPwAAAAAAAPA/"},
-            "geometry_c":{"wkb":"AQMAAAABAAAABQAAAAAAAAAAAPA/AAAAAAAA8D8AAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAADwPwAAAAAAAPA/"},
-            "linestring_c": {"wkb":"AQIAAAADAAAAAAAAAAAACEAAAAAAAAAAAAAAAAAAAAhAAAAAAAAACEAAAAAAAAAIQAAAAAAAABRA"},
-            "polygon_c": {"wkb":"AQMAAAABAAAABQAAAAAAAAAAAPA/AAAAAAAA8D8AAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAADwPwAAAAAAAPA/"},
-            "multipoint_c": {"wkb":"AQQAAAACAAAAAQEAAAAAAAAAAADwPwAAAAAAAPA/AQEAAAAAAAAAAAAAQAAAAAAAAABA"},
-            "multiline_c": {"wkb":"AQUAAAACAAAAAQIAAAADAAAAAAAAAAAA8D8AAAAAAADwPwAAAAAAAABAAAAAAAAAAEAAAAAAAAAIQAAAAAAAAAhAAQIAAAACAAAAAAAAAAAAEEAAAAAAAAAQQAAAAAAAABRAAAAAAAAAFEA="},
-            "multipolygon_c": {"wkb":"AQYAAAACAAAAAQMAAAABAAAABQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAkQAAAAAAAAAAAAAAAAAAAJEAAAAAAAAAkQAAAAAAAAAAAAAAAAAAAJEAAAAAAAAAAAAAAAAAAAAAAAQMAAAABAAAABQAAAAAAAAAAABRAAAAAAAAAFEAAAAAAAAAcQAAAAAAAABRAAAAAAAAAHEAAAAAAAAAcQAAAAAAAABRAAAAAAAAAHEAAAAAAAAAUQAAAAAAAABRA"},
-            "geometrycollection_c": {"wkb":"AQcAAAADAAAAAQEAAAAAAAAAAAAkQAAAAAAAACRAAQEAAAAAAAAAAAA+QAAAAAAAAD5AAQIAAAACAAAAAAAAAAAALkAAAAAAAAAuQAAAAAAAADRAAAAAAAAANEA="}
+            "point_c": {"x":1.0,"y":1.0,"wkb":"AQEAAAAAAAAAAADwPwAAAAAAAPA/","srid":null},
+            "geometry_c":{"wkb":"AQMAAAABAAAABQAAAAAAAAAAAPA/AAAAAAAA8D8AAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAADwPwAAAAAAAPA/","srid":null},
+            "linestring_c": {"wkb":"AQIAAAADAAAAAAAAAAAACEAAAAAAAAAAAAAAAAAAAAhAAAAAAAAACEAAAAAAAAAIQAAAAAAAABRA","srid":null},
+            "polygon_c": {"wkb":"AQMAAAABAAAABQAAAAAAAAAAAPA/AAAAAAAA8D8AAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAADwPwAAAAAAAPA/","srid":null},
+            "multipoint_c": {"wkb":"AQQAAAACAAAAAQEAAAAAAAAAAADwPwAAAAAAAPA/AQEAAAAAAAAAAAAAQAAAAAAAAABA","srid":null},
+            "multiline_c": {"wkb":"AQUAAAACAAAAAQIAAAADAAAAAAAAAAAA8D8AAAAAAADwPwAAAAAAAABAAAAAAAAAAEAAAAAAAAAIQAAAAAAAAAhAAQIAAAACAAAAAAAAAAAAEEAAAAAAAAAQQAAAAAAAABRAAAAAAAAAFEA=","srid":null},
+            "multipolygon_c": {"wkb":"AQYAAAACAAAAAQMAAAABAAAABQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAkQAAAAAAAAAAAAAAAAAAAJEAAAAAAAAAkQAAAAAAAAAAAAAAAAAAAJEAAAAAAAAAAAAAAAAAAAAAAAQMAAAABAAAABQAAAAAAAAAAABRAAAAAAAAAFEAAAAAAAAAcQAAAAAAAABRAAAAAAAAAHEAAAAAAAAAcQAAAAAAAABRAAAAAAAAAHEAAAAAAAAAUQAAAAAAAABRA","srid":null},
+            "geometrycollection_c": {"wkb":"AQcAAAADAAAAAQEAAAAAAAAAAAAkQAAAAAAAACRAAQEAAAAAAAAAAAA+QAAAAAAAAD5AAQIAAAACAAAAAAAAAAAALkAAAAAAAAAuQAAAAAAAADRAAAAAAAAANEA=","srid":null}
         },
         "op": "u",
         "transaction": null
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/resources/file/debezium-data-schema-exclude.json b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/resources/file/debezium-data-schema-exclude.json
index 3e5dedd08ae..41cc94b3728 100644
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/resources/file/debezium-data-schema-exclude.json
+++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/resources/file/debezium-data-schema-exclude.json
@@ -52,14 +52,14 @@
             "enum_c": "red",
             "set_c": "a,b",
             "json_c": "{\"key1\": \"value1\"}",
-            "point_c": {"x":1.0,"y":1.0,"wkb":"AQEAAAAAAAAAAADwPwAAAAAAAPA/"},
-            "geometry_c":{"wkb":"AQMAAAABAAAABQAAAAAAAAAAAPA/AAAAAAAA8D8AAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAADwPwAAAAAAAPA/"},
-            "linestring_c": {"wkb":"AQIAAAADAAAAAAAAAAAACEAAAAAAAAAAAAAAAAAAAAhAAAAAAAAACEAAAAAAAAAIQAAAAAAAABRA"},
-            "polygon_c": {"wkb":"AQMAAAABAAAABQAAAAAAAAAAAPA/AAAAAAAA8D8AAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAADwPwAAAAAAAPA/"},
-            "multipoint_c": {"wkb":"AQQAAAACAAAAAQEAAAAAAAAAAADwPwAAAAAAAPA/AQEAAAAAAAAAAAAAQAAAAAAAAABA"},
-            "multiline_c": {"wkb":"AQUAAAACAAAAAQIAAAADAAAAAAAAAAAA8D8AAAAAAADwPwAAAAAAAABAAAAAAAAAAEAAAAAAAAAIQAAAAAAAAAhAAQIAAAACAAAAAAAAAAAAEEAAAAAAAAAQQAAAAAAAABRAAAAAAAAAFEA="},
-            "multipolygon_c": {"wkb":"AQYAAAACAAAAAQMAAAABAAAABQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAkQAAAAAAAAAAAAAAAAAAAJEAAAAAAAAAkQAAAAAAAAAAAAAAAAAAAJEAAAAAAAAAAAAAAAAAAAAAAAQMAAAABAAAABQAAAAAAAAAAABRAAAAAAAAAFEAAAAAAAAAcQAAAAAAAABRAAAAAAAAAHEAAAAAAAAAcQAAAAAAAABRAAAAAAAAAHEAAAAAAAAAUQAAAAAAAABRA"},
-            "geometrycollection_c": {"wkb":"AQcAAAADAAAAAQEAAAAAAAAAAAAkQAAAAAAAACRAAQEAAAAAAAAAAAA+QAAAAAAAAD5AAQIAAAACAAAAAAAAAAAALkAAAAAAAAAuQAAAAAAAADRAAAAAAAAANEA="}
+            "point_c": {"x":1.0,"y":1.0,"wkb":"AQEAAAAAAAAAAADwPwAAAAAAAPA/","srid":null},
+            "geometry_c":{"wkb":"AQMAAAABAAAABQAAAAAAAAAAAPA/AAAAAAAA8D8AAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAADwPwAAAAAAAPA/","srid":null},
+            "linestring_c": {"wkb":"AQIAAAADAAAAAAAAAAAACEAAAAAAAAAAAAAAAAAAAAhAAAAAAAAACEAAAAAAAAAIQAAAAAAAABRA","srid":null},
+            "polygon_c": {"wkb":"AQMAAAABAAAABQAAAAAAAAAAAPA/AAAAAAAA8D8AAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAADwPwAAAAAAAPA/","srid":null},
+            "multipoint_c": {"wkb":"AQQAAAACAAAAAQEAAAAAAAAAAADwPwAAAAAAAPA/AQEAAAAAAAAAAAAAQAAAAAAAAABA","srid":null},
+            "multiline_c": {"wkb":"AQUAAAACAAAAAQIAAAADAAAAAAAAAAAA8D8AAAAAAADwPwAAAAAAAABAAAAAAAAAAEAAAAAAAAAIQAAAAAAAAAhAAQIAAAACAAAAAAAAAAAAEEAAAAAAAAAQQAAAAAAAABRAAAAAAAAAFEA=","srid":null},
+            "multipolygon_c": {"wkb":"AQYAAAACAAAAAQMAAAABAAAABQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAkQAAAAAAAAAAAAAAAAAAAJEAAAAAAAAAkQAAAAAAAAAAAAAAAAAAAJEAAAAAAAAAAAAAAAAAAAAAAAQMAAAABAAAABQAAAAAAAAAAABRAAAAAAAAAFEAAAAAAAAAcQAAAAAAAABRAAAAAAAAAHEAAAAAAAAAcQAAAAAAAABRAAAAAAAAAHEAAAAAAAAAUQAAAAAAAABRA","srid":null},
+            "geometrycollection_c": {"wkb":"AQcAAAADAAAAAQEAAAAAAAAAAAAkQAAAAAAAACRAAQEAAAAAAAAAAAA+QAAAAAAAAD5AAQIAAAACAAAAAAAAAAAALkAAAAAAAAAuQAAAAAAAADRAAAAAAAAANEA=","srid":null}
         },
         "op": "r",
         "transaction": null
@@ -116,14 +116,14 @@
             "enum_c": "red",
             "set_c": "a,b",
             "json_c": "{\"key1\":\"value1\"}",
-            "point_c": {"x":1.0,"y":1.0,"wkb":"AQEAAAAAAAAAAADwPwAAAAAAAPA/"},
-            "geometry_c":{"wkb":"AQMAAAABAAAABQAAAAAAAAAAAPA/AAAAAAAA8D8AAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAADwPwAAAAAAAPA/"},
-            "linestring_c": {"wkb":"AQIAAAADAAAAAAAAAAAACEAAAAAAAAAAAAAAAAAAAAhAAAAAAAAACEAAAAAAAAAIQAAAAAAAABRA"},
-            "polygon_c": {"wkb":"AQMAAAABAAAABQAAAAAAAAAAAPA/AAAAAAAA8D8AAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAADwPwAAAAAAAPA/"},
-            "multipoint_c": {"wkb":"AQQAAAACAAAAAQEAAAAAAAAAAADwPwAAAAAAAPA/AQEAAAAAAAAAAAAAQAAAAAAAAABA"},
-            "multiline_c": {"wkb":"AQUAAAACAAAAAQIAAAADAAAAAAAAAAAA8D8AAAAAAADwPwAAAAAAAABAAAAAAAAAAEAAAAAAAAAIQAAAAAAAAAhAAQIAAAACAAAAAAAAAAAAEEAAAAAAAAAQQAAAAAAAABRAAAAAAAAAFEA="},
-            "multipolygon_c": {"wkb":"AQYAAAACAAAAAQMAAAABAAAABQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAkQAAAAAAAAAAAAAAAAAAAJEAAAAAAAAAkQAAAAAAAAAAAAAAAAAAAJEAAAAAAAAAAAAAAAAAAAAAAAQMAAAABAAAABQAAAAAAAAAAABRAAAAAAAAAFEAAAAAAAAAcQAAAAAAAABRAAAAAAAAAHEAAAAAAAAAcQAAAAAAAABRAAAAAAAAAHEAAAAAAAAAUQAAAAAAAABRA"},
-            "geometrycollection_c": {"wkb":"AQcAAAADAAAAAQEAAAAAAAAAAAAkQAAAAAAAACRAAQEAAAAAAAAAAAA+QAAAAAAAAD5AAQIAAAACAAAAAAAAAAAALkAAAAAAAAAuQAAAAAAAADRAAAAAAAAANEA="}
+            "point_c": {"x":1.0,"y":1.0,"wkb":"AQEAAAAAAAAAAADwPwAAAAAAAPA/","srid":null},
+            "geometry_c":{"wkb":"AQMAAAABAAAABQAAAAAAAAAAAPA/AAAAAAAA8D8AAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAADwPwAAAAAAAPA/","srid":null},
+            "linestring_c": {"wkb":"AQIAAAADAAAAAAAAAAAACEAAAAAAAAAAAAAAAAAAAAhAAAAAAAAACEAAAAAAAAAIQAAAAAAAABRA","srid":null},
+            "polygon_c": {"wkb":"AQMAAAABAAAABQAAAAAAAAAAAPA/AAAAAAAA8D8AAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAADwPwAAAAAAAPA/","srid":null},
+            "multipoint_c": {"wkb":"AQQAAAACAAAAAQEAAAAAAAAAAADwPwAAAAAAAPA/AQEAAAAAAAAAAAAAQAAAAAAAAABA","srid":null},
+            "multiline_c": {"wkb":"AQUAAAACAAAAAQIAAAADAAAAAAAAAAAA8D8AAAAAAADwPwAAAAAAAABAAAAAAAAAAEAAAAAAAAAIQAAAAAAAAAhAAQIAAAACAAAAAAAAAAAAEEAAAAAAAAAQQAAAAAAAABRAAAAAAAAAFEA=","srid":null},
+            "multipolygon_c": {"wkb":"AQYAAAACAAAAAQMAAAABAAAABQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAkQAAAAAAAAAAAAAAAAAAAJEAAAAAAAAAkQAAAAAAAAAAAAAAAAAAAJEAAAAAAAAAAAAAAAAAAAAAAAQMAAAABAAAABQAAAAAAAAAAABRAAAAAAAAAFEAAAAAAAAAcQAAAAAAAABRAAAAAAAAAHEAAAAAAAAAcQAAAAAAAABRAAAAAAAAAHEAAAAAAAAAUQAAAAAAAABRA","srid":null},
+            "geometrycollection_c": {"wkb":"AQcAAAADAAAAAQEAAAAAAAAAAAAkQAAAAAAAACRAAQEAAAAAAAAAAAA+QAAAAAAAAD5AAQIAAAACAAAAAAAAAAAALkAAAAAAAAAuQAAAAAAAADRAAAAAAAAANEA=","srid":null}
         },
         "after": {
             "id": "AQ==",
@@ -176,14 +176,14 @@
             "enum_c": "red",
             "set_c": "a,b",
             "json_c": "{\"key1\":\"value1\"}",
-            "point_c": {"x":1.0,"y":1.0,"wkb":"AQEAAAAAAAAAAADwPwAAAAAAAPA/"},
-            "geometry_c":{"wkb":"AQMAAAABAAAABQAAAAAAAAAAAPA/AAAAAAAA8D8AAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAADwPwAAAAAAAPA/"},
-            "linestring_c": {"wkb":"AQIAAAADAAAAAAAAAAAACEAAAAAAAAAAAAAAAAAAAAhAAAAAAAAACEAAAAAAAAAIQAAAAAAAABRA"},
-            "polygon_c": {"wkb":"AQMAAAABAAAABQAAAAAAAAAAAPA/AAAAAAAA8D8AAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAADwPwAAAAAAAPA/"},
-            "multipoint_c": {"wkb":"AQQAAAACAAAAAQEAAAAAAAAAAADwPwAAAAAAAPA/AQEAAAAAAAAAAAAAQAAAAAAAAABA"},
-            "multiline_c": {"wkb":"AQUAAAACAAAAAQIAAAADAAAAAAAAAAAA8D8AAAAAAADwPwAAAAAAAABAAAAAAAAAAEAAAAAAAAAIQAAAAAAAAAhAAQIAAAACAAAAAAAAAAAAEEAAAAAAAAAQQAAAAAAAABRAAAAAAAAAFEA="},
-            "multipolygon_c": {"wkb":"AQYAAAACAAAAAQMAAAABAAAABQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAkQAAAAAAAAAAAAAAAAAAAJEAAAAAAAAAkQAAAAAAAAAAAAAAAAAAAJEAAAAAAAAAAAAAAAAAAAAAAAQMAAAABAAAABQAAAAAAAAAAABRAAAAAAAAAFEAAAAAAAAAcQAAAAAAAABRAAAAAAAAAHEAAAAAAAAAcQAAAAAAAABRAAAAAAAAAHEAAAAAAAAAUQAAAAAAAABRA"},
-            "geometrycollection_c": {"wkb":"AQcAAAADAAAAAQEAAAAAAAAAAAAkQAAAAAAAACRAAQEAAAAAAAAAAAA+QAAAAAAAAD5AAQIAAAACAAAAAAAAAAAALkAAAAAAAAAuQAAAAAAAADRAAAAAAAAANEA="}
+            "point_c": {"x":1.0,"y":1.0,"wkb":"AQEAAAAAAAAAAADwPwAAAAAAAPA/","srid":null},
+            "geometry_c":{"wkb":"AQMAAAABAAAABQAAAAAAAAAAAPA/AAAAAAAA8D8AAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAADwPwAAAAAAAPA/","srid":null},
+            "linestring_c": {"wkb":"AQIAAAADAAAAAAAAAAAACEAAAAAAAAAAAAAAAAAAAAhAAAAAAAAACEAAAAAAAAAIQAAAAAAAABRA","srid":null},
+            "polygon_c": {"wkb":"AQMAAAABAAAABQAAAAAAAAAAAPA/AAAAAAAA8D8AAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAADwPwAAAAAAAPA/","srid":null},
+            "multipoint_c": {"wkb":"AQQAAAACAAAAAQEAAAAAAAAAAADwPwAAAAAAAPA/AQEAAAAAAAAAAAAAQAAAAAAAAABA","srid":null},
+            "multiline_c": {"wkb":"AQUAAAACAAAAAQIAAAADAAAAAAAAAAAA8D8AAAAAAADwPwAAAAAAAABAAAAAAAAAAEAAAAAAAAAIQAAAAAAAAAhAAQIAAAACAAAAAAAAAAAAEEAAAAAAAAAQQAAAAAAAABRAAAAAAAAAFEA=","srid":null},
+            "multipolygon_c": {"wkb":"AQYAAAACAAAAAQMAAAABAAAABQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAkQAAAAAAAAAAAAAAAAAAAJEAAAAAAAAAkQAAAAAAAAAAAAAAAAAAAJEAAAAAAAAAAAAAAAAAAAAAAAQMAAAABAAAABQAAAAAAAAAAABRAAAAAAAAAFEAAAAAAAAAcQAAAAAAAABRAAAAAAAAAHEAAAAAAAAAcQAAAAAAAABRAAAAAAAAAHEAAAAAAAAAUQAAAAAAAABRA","srid":null},
+            "geometrycollection_c": {"wkb":"AQcAAAADAAAAAQEAAAAAAAAAAAAkQAAAAAAAACRAAQEAAAAAAAAAAAA+QAAAAAAAAD5AAQIAAAACAAAAAAAAAAAALkAAAAAAAAAuQAAAAAAAADRAAAAAAAAANEA=","srid":null}
         },
         "op": "u",
         "transaction": null
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/resources/file/debezium-data-schema-include.json b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/resources/file/debezium-data-schema-include.json
index fed8c09a940..74dc65456d4 100644
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/resources/file/debezium-data-schema-include.json
+++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/resources/file/debezium-data-schema-include.json
@@ -725,14 +725,14 @@
                 "enum_c": "red",
                 "set_c": "a,b",
                 "json_c": "{\"key1\": \"value1\"}",
-                "point_c": {"x":1.0,"y":1.0,"wkb":"AQEAAAAAAAAAAADwPwAAAAAAAPA/"},
-                "geometry_c":{"wkb":"AQMAAAABAAAABQAAAAAAAAAAAPA/AAAAAAAA8D8AAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAADwPwAAAAAAAPA/"},
-                "linestring_c": {"wkb":"AQIAAAADAAAAAAAAAAAACEAAAAAAAAAAAAAAAAAAAAhAAAAAAAAACEAAAAAAAAAIQAAAAAAAABRA"},
-                "polygon_c": {"wkb":"AQMAAAABAAAABQAAAAAAAAAAAPA/AAAAAAAA8D8AAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAADwPwAAAAAAAPA/"},
-                "multipoint_c": {"wkb":"AQQAAAACAAAAAQEAAAAAAAAAAADwPwAAAAAAAPA/AQEAAAAAAAAAAAAAQAAAAAAAAABA"},
-                "multiline_c": {"wkb":"AQUAAAACAAAAAQIAAAADAAAAAAAAAAAA8D8AAAAAAADwPwAAAAAAAABAAAAAAAAAAEAAAAAAAAAIQAAAAAAAAAhAAQIAAAACAAAAAAAAAAAAEEAAAAAAAAAQQAAAAAAAABRAAAAAAAAAFEA="},
-                "multipolygon_c": {"wkb":"AQYAAAACAAAAAQMAAAABAAAABQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAkQAAAAAAAAAAAAAAAAAAAJEAAAAAAAAAkQAAAAAAAAAAAAAAAAAAAJEAAAAAAAAAAAAAAAAAAAAAAAQMAAAABAAAABQAAAAAAAAAAABRAAAAAAAAAFEAAAAAAAAAcQAAAAAAAABRAAAAAAAAAHEAAAAAAAAAcQAAAAAAAABRAAAAAAAAAHEAAAAAAAAAUQAAAAAAAABRA"},
-                "geometrycollection_c": {"wkb":"AQcAAAADAAAAAQEAAAAAAAAAAAAkQAAAAAAAACRAAQEAAAAAAAAAAAA+QAAAAAAAAD5AAQIAAAACAAAAAAAAAAAALkAAAAAAAAAuQAAAAAAAADRAAAAAAAAANEA="}
+                "point_c": {"x":1.0,"y":1.0,"wkb":"AQEAAAAAAAAAAADwPwAAAAAAAPA/","srid":null},
+                "geometry_c":{"wkb":"AQMAAAABAAAABQAAAAAAAAAAAPA/AAAAAAAA8D8AAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAADwPwAAAAAAAPA/","srid":null},
+                "linestring_c": {"wkb":"AQIAAAADAAAAAAAAAAAACEAAAAAAAAAAAAAAAAAAAAhAAAAAAAAACEAAAAAAAAAIQAAAAAAAABRA","srid":null},
+                "polygon_c": {"wkb":"AQMAAAABAAAABQAAAAAAAAAAAPA/AAAAAAAA8D8AAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAADwPwAAAAAAAPA/","srid":null},
+                "multipoint_c": {"wkb":"AQQAAAACAAAAAQEAAAAAAAAAAADwPwAAAAAAAPA/AQEAAAAAAAAAAAAAQAAAAAAAAABA","srid":null},
+                "multiline_c": {"wkb":"AQUAAAACAAAAAQIAAAADAAAAAAAAAAAA8D8AAAAAAADwPwAAAAAAAABAAAAAAAAAAEAAAAAAAAAIQAAAAAAAAAhAAQIAAAACAAAAAAAAAAAAEEAAAAAAAAAQQAAAAAAAABRAAAAAAAAAFEA=","srid":null},
+                "multipolygon_c": {"wkb":"AQYAAAACAAAAAQMAAAABAAAABQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAkQAAAAAAAAAAAAAAAAAAAJEAAAAAAAAAkQAAAAAAAAAAAAAAAAAAAJEAAAAAAAAAAAAAAAAAAAAAAAQMAAAABAAAABQAAAAAAAAAAABRAAAAAAAAAFEAAAAAAAAAcQAAAAAAAABRAAAAAAAAAHEAAAAAAAAAcQAAAAAAAABRAAAAAAAAAHEAAAAAAAAAUQAAAAAAAABRA","srid":null},
+                "geometrycollection_c": {"wkb":"AQcAAAADAAAAAQEAAAAAAAAAAAAkQAAAAAAAACRAAQEAAAAAAAAAAAA+QAAAAAAAAD5AAQIAAAACAAAAAAAAAAAALkAAAAAAAAAuQAAAAAAAADRAAAAAAAAANEA=","srid":null}
             },
             "source": {
                 "version": "1.5.2.Final",
@@ -1080,14 +1080,14 @@
                 "enum_c": "red",
                 "set_c": "a,b",
                 "json_c": "{\"key1\":\"value1\"}",
-                "point_c": {"x":1.0,"y":1.0,"wkb":"AQEAAAAAAAAAAADwPwAAAAAAAPA/"},
-                "geometry_c":{"wkb":"AQMAAAABAAAABQAAAAAAAAAAAPA/AAAAAAAA8D8AAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAADwPwAAAAAAAPA/"},
-                "linestring_c": {"wkb":"AQIAAAADAAAAAAAAAAAACEAAAAAAAAAAAAAAAAAAAAhAAAAAAAAACEAAAAAAAAAIQAAAAAAAABRA"},
-                "polygon_c": {"wkb":"AQMAAAABAAAABQAAAAAAAAAAAPA/AAAAAAAA8D8AAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAADwPwAAAAAAAPA/"},
-                "multipoint_c": {"wkb":"AQQAAAACAAAAAQEAAAAAAAAAAADwPwAAAAAAAPA/AQEAAAAAAAAAAAAAQAAAAAAAAABA"},
-                "multiline_c": {"wkb":"AQUAAAACAAAAAQIAAAADAAAAAAAAAAAA8D8AAAAAAADwPwAAAAAAAABAAAAAAAAAAEAAAAAAAAAIQAAAAAAAAAhAAQIAAAACAAAAAAAAAAAAEEAAAAAAAAAQQAAAAAAAABRAAAAAAAAAFEA="},
-                "multipolygon_c": {"wkb":"AQYAAAACAAAAAQMAAAABAAAABQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAkQAAAAAAAAAAAAAAAAAAAJEAAAAAAAAAkQAAAAAAAAAAAAAAAAAAAJEAAAAAAAAAAAAAAAAAAAAAAAQMAAAABAAAABQAAAAAAAAAAABRAAAAAAAAAFEAAAAAAAAAcQAAAAAAAABRAAAAAAAAAHEAAAAAAAAAcQAAAAAAAABRAAAAAAAAAHEAAAAAAAAAUQAAAAAAAABRA"},
-                "geometrycollection_c": {"wkb":"AQcAAAADAAAAAQEAAAAAAAAAAAAkQAAAAAAAACRAAQEAAAAAAAAAAAA+QAAAAAAAAD5AAQIAAAACAAAAAAAAAAAALkAAAAAAAAAuQAAAAAAAADRAAAAAAAAANEA="}
+                "point_c": {"x":1.0,"y":1.0,"wkb":"AQEAAAAAAAAAAADwPwAAAAAAAPA/","srid":null},
+                "geometry_c":{"wkb":"AQMAAAABAAAABQAAAAAAAAAAAPA/AAAAAAAA8D8AAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAADwPwAAAAAAAPA/","srid":null},
+                "linestring_c": {"wkb":"AQIAAAADAAAAAAAAAAAACEAAAAAAAAAAAAAAAAAAAAhAAAAAAAAACEAAAAAAAAAIQAAAAAAAABRA","srid":null},
+                "polygon_c": {"wkb":"AQMAAAABAAAABQAAAAAAAAAAAPA/AAAAAAAA8D8AAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAADwPwAAAAAAAPA/","srid":null},
+                "multipoint_c": {"wkb":"AQQAAAACAAAAAQEAAAAAAAAAAADwPwAAAAAAAPA/AQEAAAAAAAAAAAAAQAAAAAAAAABA","srid":null},
+                "multiline_c": {"wkb":"AQUAAAACAAAAAQIAAAADAAAAAAAAAAAA8D8AAAAAAADwPwAAAAAAAABAAAAAAAAAAEAAAAAAAAAIQAAAAAAAAAhAAQIAAAACAAAAAAAAAAAAEEAAAAAAAAAQQAAAAAAAABRAAAAAAAAAFEA=","srid":null},
+                "multipolygon_c": {"wkb":"AQYAAAACAAAAAQMAAAABAAAABQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAkQAAAAAAAAAAAAAAAAAAAJEAAAAAAAAAkQAAAAAAAAAAAAAAAAAAAJEAAAAAAAAAAAAAAAAAAAAAAAQMAAAABAAAABQAAAAAAAAAAABRAAAAAAAAAFEAAAAAAAAAcQAAAAAAAABRAAAAAAAAAHEAAAAAAAAAcQAAAAAAAABRAAAAAAAAAHEAAAAAAAAAUQAAAAAAAABRA","srid":null},
+                "geometrycollection_c": {"wkb":"AQcAAAADAAAAAQEAAAAAAAAAAAAkQAAAAAAAACRAAQEAAAAAAAAAAAA+QAAAAAAAAD5AAQIAAAACAAAAAAAAAAAALkAAAAAAAAAuQAAAAAAAADRAAAAAAAAANEA=","srid":null}
             },
             "after": {
                 "id": "AQ==",
@@ -1140,14 +1140,14 @@
                 "enum_c": "red",
                 "set_c": "a,b",
                 "json_c": "{\"key1\":\"value1\"}",
-                "point_c": {"x":1.0,"y":1.0,"wkb":"AQEAAAAAAAAAAADwPwAAAAAAAPA/"},
-                "geometry_c":{"wkb":"AQMAAAABAAAABQAAAAAAAAAAAPA/AAAAAAAA8D8AAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAADwPwAAAAAAAPA/"},
-                "linestring_c": {"wkb":"AQIAAAADAAAAAAAAAAAACEAAAAAAAAAAAAAAAAAAAAhAAAAAAAAACEAAAAAAAAAIQAAAAAAAABRA"},
-                "polygon_c": {"wkb":"AQMAAAABAAAABQAAAAAAAAAAAPA/AAAAAAAA8D8AAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAADwPwAAAAAAAPA/"},
-                "multipoint_c": {"wkb":"AQQAAAACAAAAAQEAAAAAAAAAAADwPwAAAAAAAPA/AQEAAAAAAAAAAAAAQAAAAAAAAABA"},
-                "multiline_c": {"wkb":"AQUAAAACAAAAAQIAAAADAAAAAAAAAAAA8D8AAAAAAADwPwAAAAAAAABAAAAAAAAAAEAAAAAAAAAIQAAAAAAAAAhAAQIAAAACAAAAAAAAAAAAEEAAAAAAAAAQQAAAAAAAABRAAAAAAAAAFEA="},
-                "multipolygon_c": {"wkb":"AQYAAAACAAAAAQMAAAABAAAABQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAkQAAAAAAAAAAAAAAAAAAAJEAAAAAAAAAkQAAAAAAAAAAAAAAAAAAAJEAAAAAAAAAAAAAAAAAAAAAAAQMAAAABAAAABQAAAAAAAAAAABRAAAAAAAAAFEAAAAAAAAAcQAAAAAAAABRAAAAAAAAAHEAAAAAAAAAcQAAAAAAAABRAAAAAAAAAHEAAAAAAAAAUQAAAAAAAABRA"},
-                "geometrycollection_c": {"wkb":"AQcAAAADAAAAAQEAAAAAAAAAAAAkQAAAAAAAACRAAQEAAAAAAAAAAAA+QAAAAAAAAD5AAQIAAAACAAAAAAAAAAAALkAAAAAAAAAuQAAAAAAAADRAAAAAAAAANEA="}
+                "point_c": {"x":1.0,"y":1.0,"wkb":"AQEAAAAAAAAAAADwPwAAAAAAAPA/","srid":null},
+                "geometry_c":{"wkb":"AQMAAAABAAAABQAAAAAAAAAAAPA/AAAAAAAA8D8AAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAADwPwAAAAAAAPA/","srid":null},
+                "linestring_c": {"wkb":"AQIAAAADAAAAAAAAAAAACEAAAAAAAAAAAAAAAAAAAAhAAAAAAAAACEAAAAAAAAAIQAAAAAAAABRA","srid":null},
+                "polygon_c": {"wkb":"AQMAAAABAAAABQAAAAAAAAAAAPA/AAAAAAAA8D8AAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAAAAQAAAAAAAAPA/AAAAAAAAAEAAAAAAAADwPwAAAAAAAPA/","srid":null},
+                "multipoint_c": {"wkb":"AQQAAAACAAAAAQEAAAAAAAAAAADwPwAAAAAAAPA/AQEAAAAAAAAAAAAAQAAAAAAAAABA","srid":null},
+                "multiline_c": {"wkb":"AQUAAAACAAAAAQIAAAADAAAAAAAAAAAA8D8AAAAAAADwPwAAAAAAAABAAAAAAAAAAEAAAAAAAAAIQAAAAAAAAAhAAQIAAAACAAAAAAAAAAAAEEAAAAAAAAAQQAAAAAAAABRAAAAAAAAAFEA=","srid":null},
+                "multipolygon_c": {"wkb":"AQYAAAACAAAAAQMAAAABAAAABQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAkQAAAAAAAAAAAAAAAAAAAJEAAAAAAAAAkQAAAAAAAAAAAAAAAAAAAJEAAAAAAAAAAAAAAAAAAAAAAAQMAAAABAAAABQAAAAAAAAAAABRAAAAAAAAAFEAAAAAAAAAcQAAAAAAAABRAAAAAAAAAHEAAAAAAAAAcQAAAAAAAABRAAAAAAAAAHEAAAAAAAAAUQAAAAAAAABRA","srid":null},
+                "geometrycollection_c": {"wkb":"AQcAAAADAAAAAQEAAAAAAAAAAAAkQAAAAAAAACRAAQEAAAAAAAAAAAA+QAAAAAAAAD5AAQIAAAACAAAAAAAAAAAALkAAAAAAAAAuQAAAAAAAADRAAAAAAAAANEA=","srid":null}
             },
             "source": {
                 "version": "1.5.2.Final",
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/pom.xml b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/pom.xml
index 726d7f43822..163db5555a2 100644
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/pom.xml
+++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/pom.xml
@@ -74,6 +74,14 @@ limitations under the License.
         
 
         
+        
+            org.apache.flink
+            flink-cdc-common
+            ${project.version}
+            test-jar
+            test
+        
+
         
             org.apache.flink
             flink-table-planner_${scala.binary.version}
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseTestBase.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseTestBase.java
index 9766da91601..2af771fddff 100644
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseTestBase.java
+++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseTestBase.java
@@ -18,7 +18,6 @@
 package org.apache.flink.cdc.connectors.oceanbase;
 
 import org.apache.flink.cdc.connectors.oceanbase.testutils.OceanBaseCdcMetadata;
-import org.apache.flink.table.planner.factories.TestValuesTableFactory;
 import org.apache.flink.table.utils.LegacyRowResource;
 import org.apache.flink.test.util.AbstractTestBase;
 
@@ -34,7 +33,6 @@
 import java.time.Duration;
 import java.util.Arrays;
 import java.util.List;
-import java.util.concurrent.TimeoutException;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 import java.util.stream.Collectors;
@@ -133,33 +131,6 @@ protected void initializeTable(String sqlFile) {
         }
     }
 
-    public static void waitForSinkSize(String sinkName, int expectedSize)
-            throws InterruptedException, TimeoutException {
-        long deadlineTimestamp = System.currentTimeMillis() + FETCH_TIMEOUT.toMillis();
-        while (System.currentTimeMillis() < deadlineTimestamp) {
-            if (sinkSize(sinkName) < expectedSize) {
-                Thread.sleep(100);
-            } else {
-                return;
-            }
-        }
-        throw new TimeoutException(
-                String.format(
-                        "Failed to fetch enough records in sink.\nExpected size: %d\nActual values: %s",
-                        expectedSize, TestValuesTableFactory.getRawResults(sinkName)));
-    }
-
-    public static int sinkSize(String sinkName) {
-        synchronized (TestValuesTableFactory.class) {
-            try {
-                return TestValuesTableFactory.getRawResultsAsStrings(sinkName).size();
-            } catch (IllegalArgumentException e) {
-                // job is not started yet
-                return 0;
-            }
-        }
-    }
-
     public static void assertContainsInAnyOrder(List expected, List actual) {
         assertTrue(expected != null && actual != null);
         assertTrue(
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseMySQLModeITCase.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseMySQLModeITCase.java
index a74bf035ec0..eca8ff9b96f 100644
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseMySQLModeITCase.java
+++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseMySQLModeITCase.java
@@ -44,6 +44,8 @@
 import java.util.stream.Collectors;
 import java.util.stream.Stream;
 
+import static org.apache.flink.cdc.common.testutils.TestCaseUtils.sinkSize;
+import static org.apache.flink.cdc.common.testutils.TestCaseUtils.waitForSinkSize;
 import static org.apache.flink.cdc.connectors.oceanbase.OceanBaseTestUtils.createLogProxyContainer;
 import static org.apache.flink.cdc.connectors.oceanbase.OceanBaseTestUtils.createOceanBaseContainerForCDC;
 
@@ -155,8 +157,8 @@ public void testTableList() throws Exception {
 
         TableResult result = tEnv.executeSql("INSERT INTO sink SELECT * FROM ob_source");
 
-        waitForSinkSize("sink", 9);
-        int snapshotSize = sinkSize("sink");
+        waitForSinkSize("sink", false, 9);
+        int snapshotSize = sinkSize("sink", false);
 
         try (Connection connection = getJdbcConnection();
                 Statement statement = connection.createStatement()) {
@@ -174,7 +176,7 @@ public void testTableList() throws Exception {
             statement.execute("DELETE FROM products WHERE id=111;");
         }
 
-        waitForSinkSize("sink", snapshotSize + 7);
+        waitForSinkSize("sink", false, snapshotSize + 7);
 
         /*
          * 
@@ -269,8 +271,8 @@ public void testMetadataColumns() throws Exception {
         // async submit job
         TableResult result = tEnv.executeSql("INSERT INTO sink SELECT * FROM ob_source");
 
-        waitForSinkSize("sink", 9);
-        int snapshotSize = sinkSize("sink");
+        waitForSinkSize("sink", false, 9);
+        int snapshotSize = sinkSize("sink", false);
 
         try (Connection connection = getJdbcConnection();
                 Statement statement = connection.createStatement()) {
@@ -279,7 +281,7 @@ public void testMetadataColumns() throws Exception {
                     "UPDATE products SET description='18oz carpenter hammer' WHERE id=106;");
         }
 
-        waitForSinkSize("sink", snapshotSize + 1);
+        waitForSinkSize("sink", false, snapshotSize + 1);
 
         String tenant = metadata().getTenantName();
 
@@ -467,8 +469,8 @@ public void testAllDataTypes() throws Exception {
                                 + "json_c\n"
                                 + " FROM ob_source");
 
-        waitForSinkSize("sink", 1);
-        int snapshotSize = sinkSize("sink");
+        waitForSinkSize("sink", false, 1);
+        int snapshotSize = sinkSize("sink", false);
 
         try (Connection connection = getJdbcConnection();
                 Statement statement = connection.createStatement()) {
@@ -477,7 +479,7 @@ public void testAllDataTypes() throws Exception {
                     "UPDATE full_types SET timestamp_c = '2020-07-17 18:33:22' WHERE id=1;");
         }
 
-        waitForSinkSize("sink", snapshotSize + 1);
+        waitForSinkSize("sink", false, snapshotSize + 1);
 
         List expected =
                 Arrays.asList(
@@ -551,8 +553,8 @@ public void testTimeDataTypes(String serverTimeZone) throws Exception {
                         "INSERT INTO sink SELECT `id`, date_c, time_c, datetime3_c, datetime6_c, cast(timestamp_c as timestamp) FROM ob_source");
 
         // wait for snapshot finished and begin binlog
-        waitForSinkSize("sink", 1);
-        int snapshotSize = sinkSize("sink");
+        waitForSinkSize("sink", false, 1);
+        int snapshotSize = sinkSize("sink", false);
 
         try (Connection connection = getJdbcConnection();
                 Statement statement = connection.createStatement()) {
@@ -561,7 +563,7 @@ public void testTimeDataTypes(String serverTimeZone) throws Exception {
                     "UPDATE full_types SET timestamp_c = '2020-07-17 18:33:22' WHERE id=1;");
         }
 
-        waitForSinkSize("sink", snapshotSize + 1);
+        waitForSinkSize("sink", false, snapshotSize + 1);
 
         List expected =
                 Arrays.asList(
@@ -612,7 +614,7 @@ public void testSnapshotOnly() throws Exception {
 
         TableResult result = tEnv.executeSql("INSERT INTO sink SELECT * FROM ob_source");
 
-        waitForSinkSize("sink", 9);
+        waitForSinkSize("sink", false, 9);
 
         List expected =
                 Arrays.asList(
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseOracleModeITCase.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseOracleModeITCase.java
index b9f9fa0e0dc..162cc85b63c 100644
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseOracleModeITCase.java
+++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseOracleModeITCase.java
@@ -34,6 +34,8 @@
 import java.util.Arrays;
 import java.util.List;
 
+import static org.apache.flink.cdc.common.testutils.TestCaseUtils.waitForSinkSize;
+
 /** Integration tests for OceanBase Oracle mode table source. */
 @Ignore("Test ignored before oceanbase-xe docker image is available")
 public class OceanBaseOracleModeITCase extends OceanBaseTestBase {
@@ -205,7 +207,7 @@ public void testAllDataTypes() throws Exception {
                                 + " DECODE(VAL_BLOB_INLINE, 'UTF-8')"
                                 + " FROM full_types");
 
-        waitForSinkSize("sink", 1);
+        waitForSinkSize("sink", false, 1);
 
         try (Connection connection = getJdbcConnection();
                 Statement statement = connection.createStatement()) {
@@ -213,7 +215,7 @@ public void testAllDataTypes() throws Exception {
                     "UPDATE FULL_TYPES SET VAL_TS = '2022-10-30 12:34:56.12545' WHERE id=1;");
         }
 
-        waitForSinkSize("sink", 2);
+        waitForSinkSize("sink", false, 2);
 
         List expected =
                 Arrays.asList(
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/pom.xml b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/pom.xml
index 9b258cceb39..1cf686f9aff 100644
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/pom.xml
+++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/pom.xml
@@ -77,9 +77,16 @@ limitations under the License.
         
 
         
-
         
             org.apache.flink
+            flink-cdc-common
+            ${project.version}
+            test-jar
+            test
+        
+        
+
+        org.apache.flink
             flink-table-planner_${scala.binary.version}
             ${flink.version}
             test
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/test/java/org/apache/flink/cdc/connectors/oracle/source/NewlyAddedTableITCase.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/test/java/org/apache/flink/cdc/connectors/oracle/source/NewlyAddedTableITCase.java
index 122c44ac44c..f24f03ec756 100644
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/test/java/org/apache/flink/cdc/connectors/oracle/source/NewlyAddedTableITCase.java
+++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/test/java/org/apache/flink/cdc/connectors/oracle/source/NewlyAddedTableITCase.java
@@ -54,10 +54,9 @@
 import java.util.stream.Collectors;
 
 import static java.lang.String.format;
+import static org.apache.flink.cdc.common.testutils.TestCaseUtils.waitForSinkSize;
 import static org.apache.flink.cdc.connectors.oracle.testutils.OracleTestUtils.getTableNameRegex;
 import static org.apache.flink.cdc.connectors.oracle.testutils.OracleTestUtils.triggerFailover;
-import static org.apache.flink.cdc.connectors.oracle.testutils.OracleTestUtils.waitForSinkSize;
-import static org.apache.flink.cdc.connectors.oracle.testutils.OracleTestUtils.waitForUpsertSinkSize;
 
 /** IT tests to cover various newly added tables during capture process. */
 public class NewlyAddedTableITCase extends OracleSourceTestBase {
@@ -393,7 +392,7 @@ private void testRemoveAndAddTablesOneByOne(int parallelism, String... captureAd
                             format(
                                     "+I[%s, 417022095255614379, China, %s, %s West Town address 3]",
                                     captureTableThisRound, cityName, cityName)));
-            waitForSinkSize("sink", fetchedDataList.size());
+            waitForSinkSize("sink", false, fetchedDataList.size());
             assertEqualsInAnyOrder(
                     fetchedDataList, TestValuesTableFactory.getRawResultsAsStrings("sink"));
 
@@ -418,7 +417,7 @@ private void testRemoveAndAddTablesOneByOne(int parallelism, String... captureAd
                                     cityName)));
 
             // step 3: assert fetched redo log data in this round
-            waitForSinkSize("sink", fetchedDataList.size());
+            waitForSinkSize("sink", false, fetchedDataList.size());
 
             assertEqualsInAnyOrder(
                     fetchedDataList, TestValuesTableFactory.getRawResultsAsStrings("sink"));
@@ -492,7 +491,7 @@ private void testRemoveTablesOneByOne(
                         miniClusterResource.getMiniCluster(),
                         () -> sleepMs(100));
             }
-            waitForSinkSize("sink", fetchedDataList.size());
+            waitForSinkSize("sink", false, fetchedDataList.size());
             assertEqualsInAnyOrder(
                     fetchedDataList, TestValuesTableFactory.getRawResultsAsStrings("sink"));
             // wait task to stream phase
@@ -530,7 +529,7 @@ private void testRemoveTablesOneByOne(
             TableResult tableResult = tEnv.executeSql("insert into sink select * from address");
             JobClient jobClient = tableResult.getJobClient().get();
 
-            waitForSinkSize("sink", fetchedDataList.size());
+            waitForSinkSize("sink", false, fetchedDataList.size());
             assertEqualsInAnyOrder(
                     fetchedDataList, TestValuesTableFactory.getRawResultsAsStrings("sink"));
 
@@ -572,7 +571,7 @@ private void testRemoveTablesOneByOne(
 
             fetchedDataList.addAll(expectedRedoLogDataThisRound);
             // step 4: assert fetched redo log data in this round
-            waitForSinkSize("sink", fetchedDataList.size());
+            waitForSinkSize("sink", false, fetchedDataList.size());
             assertEqualsInAnyOrder(
                     fetchedDataList, TestValuesTableFactory.getRawResultsAsStrings("sink"));
 
@@ -687,7 +686,7 @@ private void testNewlyAddedTableOneByOne(
                         () -> sleepMs(100));
             }
             fetchedDataList.addAll(expectedSnapshotDataThisRound);
-            waitForUpsertSinkSize("sink", fetchedDataList.size());
+            waitForSinkSize("sink", true, fetchedDataList.size());
             assertEqualsInAnyOrder(
                     fetchedDataList, TestValuesTableFactory.getResultsAsStrings("sink"));
 
@@ -726,7 +725,7 @@ private void testNewlyAddedTableOneByOne(
             // step 5: assert fetched redo log data in this round
             fetchedDataList.addAll(expectedRedoLogUpsertDataThisRound);
 
-            waitForUpsertSinkSize("sink", fetchedDataList.size());
+            waitForSinkSize("sink", true, fetchedDataList.size());
             // the result size of sink may arrive fetchedDataList.size() with old data, wait one
             // checkpoint to wait retract old record and send new record
             Thread.sleep(1000);
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/test/java/org/apache/flink/cdc/connectors/oracle/source/OracleSourceITCase.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/test/java/org/apache/flink/cdc/connectors/oracle/source/OracleSourceITCase.java
index 1b0cba5f281..f187dc943cd 100644
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/test/java/org/apache/flink/cdc/connectors/oracle/source/OracleSourceITCase.java
+++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/test/java/org/apache/flink/cdc/connectors/oracle/source/OracleSourceITCase.java
@@ -20,6 +20,7 @@
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.api.common.eventtime.WatermarkStrategy;
 import org.apache.flink.api.common.restartstrategy.RestartStrategies;
+import org.apache.flink.cdc.common.testutils.TestCaseUtils;
 import org.apache.flink.cdc.connectors.base.config.JdbcSourceConfig;
 import org.apache.flink.cdc.connectors.base.options.StartupOptions;
 import org.apache.flink.cdc.connectors.base.source.utils.hooks.SnapshotPhaseHook;
@@ -52,12 +53,10 @@
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
-import java.util.Iterator;
 import java.util.List;
-import java.util.function.Function;
-import java.util.stream.Collectors;
 
 import static java.lang.String.format;
+import static org.apache.flink.cdc.common.testutils.TestCaseUtils.fetchAndConvert;
 import static org.apache.flink.cdc.connectors.oracle.testutils.OracleTestUtils.triggerFailover;
 import static org.apache.flink.table.api.DataTypes.BIGINT;
 import static org.apache.flink.table.api.DataTypes.STRING;
@@ -490,7 +489,7 @@ private List testBackfillWhenWritingEvents(
         try (CloseableIterator iterator =
                 env.fromSource(source, WatermarkStrategy.noWatermarks(), "Backfill Skipped Source")
                         .executeAndCollect()) {
-            records = fetchRowData(iterator, fetchSize, customerTable::stringify);
+            records = fetchAndConvert(iterator, fetchSize, customerTable::stringify);
             env.close();
         }
         return records;
@@ -617,7 +616,9 @@ private void testOracleParallelSource(
 
         LOG.info("snapshot data start");
         assertEqualsInAnyOrder(
-                expectedSnapshotData, fetchRows(iterator, expectedSnapshotData.size()));
+                expectedSnapshotData,
+                TestCaseUtils.fetchAndConvert(
+                        iterator, expectedSnapshotData.size(), Row::toString));
 
         // second step: check the redo log data
         for (String tableId : captureCustomerTables) {
@@ -650,7 +651,8 @@ private void testOracleParallelSource(
             expectedRedoLogData.addAll(Arrays.asList(redoLogForSingleTable));
         }
         assertEqualsInAnyOrder(
-                expectedRedoLogData, fetchRows(iterator, expectedRedoLogData.size()));
+                expectedRedoLogData,
+                TestCaseUtils.fetchAndConvert(iterator, expectedRedoLogData.size(), Row::toString));
         tableResult.getJobClient().get().cancel().get();
     }
 
@@ -675,27 +677,6 @@ private void sleepMs(long millis) {
         }
     }
 
-    private static List fetchRowData(
-            Iterator iter, int size, Function stringifier) {
-        List rows = new ArrayList<>(size);
-        while (size > 0 && iter.hasNext()) {
-            RowData row = iter.next();
-            rows.add(row);
-            size--;
-        }
-        return rows.stream().map(stringifier).collect(Collectors.toList());
-    }
-
-    private static List fetchRows(Iterator iter, int size) {
-        List rows = new ArrayList<>(size);
-        while (size > 0 && iter.hasNext()) {
-            Row row = iter.next();
-            rows.add(row.toString());
-            size--;
-        }
-        return rows;
-    }
-
     private String getTableNameRegex(String[] captureCustomerTables) {
         checkState(captureCustomerTables.length > 0);
         if (captureCustomerTables.length == 1) {
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/test/java/org/apache/flink/cdc/connectors/oracle/table/OracleConnectorITCase.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/test/java/org/apache/flink/cdc/connectors/oracle/table/OracleConnectorITCase.java
index a501d0c59ae..4c5431ab441 100644
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/test/java/org/apache/flink/cdc/connectors/oracle/table/OracleConnectorITCase.java
+++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/test/java/org/apache/flink/cdc/connectors/oracle/table/OracleConnectorITCase.java
@@ -54,6 +54,8 @@
 import java.util.stream.IntStream;
 import java.util.stream.Stream;
 
+import static org.apache.flink.cdc.common.testutils.TestCaseUtils.waitForSinkSize;
+import static org.apache.flink.cdc.common.testutils.TestCaseUtils.waitForSnapshotStarted;
 import static org.apache.flink.cdc.connectors.oracle.source.OracleSourceTestBase.CONNECTOR_PWD;
 import static org.apache.flink.cdc.connectors.oracle.source.OracleSourceTestBase.CONNECTOR_USER;
 import static org.apache.flink.cdc.connectors.oracle.source.OracleSourceTestBase.ORACLE_CONTAINER;
@@ -166,7 +168,7 @@ public void testConsumingAllEvents() throws Exception {
                         "INSERT INTO sink SELECT NAME, SUM(WEIGHT) FROM debezium_source GROUP BY NAME");
 
         // There are 9 records in the table, wait until the snapshot phase finished
-        waitForSinkSize("sink", 9);
+        waitForSinkSize("sink", false, 9);
 
         try (Connection connection = getJdbcConnection();
                 Statement statement = connection.createStatement()) {
@@ -186,7 +188,7 @@ public void testConsumingAllEvents() throws Exception {
             statement.execute("DELETE FROM debezium.products WHERE ID=112");
         }
 
-        waitForSinkSize("sink", 20);
+        waitForSinkSize("sink", false, 20);
 
         /*
          * 
@@ -303,7 +305,7 @@ public void testSkipNestedTables() throws Exception {
                         "INSERT INTO sink SELECT NAME, SUM(WEIGHT) FROM debezium_source GROUP BY NAME");
 
         // There are 9 records in the table, wait until the snapshot phase finished
-        waitForSinkSize("sink", 9);
+        waitForSinkSize("sink", false, 9);
 
         try (Connection connection = getJdbcConnection();
                 Statement statement = connection.createStatement()) {
@@ -320,7 +322,7 @@ public void testSkipNestedTables() throws Exception {
             statement.execute("DELETE FROM debezium.products WHERE ID=112");
         }
 
-        waitForSinkSize("sink", 20);
+        waitForSinkSize("sink", false, 20);
 
         /*
          * 
@@ -419,7 +421,7 @@ public void testConsumingAllEventsByChunkKeyColumn() throws Exception {
                 tEnv.executeSql(
                         "INSERT INTO sink SELECT NAME, SUM(WEIGHT) FROM debezium_source GROUP BY NAME");
 
-        waitForSinkSize("sink", 9);
+        waitForSinkSize("sink", false, 9);
 
         try (Connection connection = getJdbcConnection();
                 Statement statement = connection.createStatement()) {
@@ -437,7 +439,7 @@ public void testConsumingAllEventsByChunkKeyColumn() throws Exception {
             statement.execute("DELETE FROM debezium.products WHERE ID=112");
         }
 
-        waitForSinkSize("sink", 20);
+        waitForSinkSize("sink", false, 20);
 
         String[] expected =
                 new String[] {
@@ -517,7 +519,7 @@ public void testMetadataColumns() throws Throwable {
         // async submit job
         TableResult result = tEnv.executeSql("INSERT INTO sink SELECT * FROM debezium_source");
 
-        waitForSinkSize("sink", 9);
+        waitForSinkSize("sink", false, 9);
 
         try (Connection connection = getJdbcConnection();
                 Statement statement = connection.createStatement()) {
@@ -534,7 +536,7 @@ public void testMetadataColumns() throws Throwable {
             statement.execute("UPDATE debezium.products SET WEIGHT=5.17 WHERE ID=112");
             statement.execute("DELETE FROM debezium.products WHERE ID=112");
         }
-        waitForSinkSize("sink", 16);
+        waitForSinkSize("sink", false, 16);
         List expected =
                 Arrays.asList(
                         "+I[ORCLCDB, DEBEZIUM, PRODUCTS, 101, scooter, Small 2-wheel scooter, 3.140]",
@@ -620,7 +622,7 @@ public void testStartupFromLatestOffset() throws Exception {
             statement.execute("DELETE FROM debezium.products WHERE id=111");
         }
 
-        waitForSinkSize("sink", 7);
+        waitForSinkSize("sink", false, 7);
 
         String[] expected =
                 new String[] {"+I[110, jacket, new water resistent white wind breaker, 0.500]"};
@@ -721,7 +723,7 @@ public void testConsumingNumericColumns() throws Exception {
         waitForSnapshotStarted("test_numeric_sink");
 
         // waiting for change events finished.
-        waitForSinkSize("test_numeric_sink", 2);
+        waitForSinkSize("test_numeric_sink", false, 2);
 
         List expected =
                 Arrays.asList(
@@ -823,7 +825,7 @@ public void testAllDataTypes() throws Throwable {
         TableResult result = tEnv.executeSql("INSERT INTO sink SELECT * FROM full_types");
 
         // waiting for change events finished.
-        waitForSinkSize("sink", 1);
+        waitForSinkSize("sink", false, 1);
 
         String[] expected =
                 new String[] {
@@ -902,7 +904,7 @@ public void testSnapshotToStreamingSwitchPendingTransactions() throws Exception
         finishFuture.get(10, TimeUnit.MINUTES);
         LOG.info("all async runners were finished");
 
-        waitForSinkSize("sink", RECORDS_COUNT);
+        waitForSinkSize("sink", false, RECORDS_COUNT);
 
         List actual =
                 TestValuesTableFactory.getResultsAsStrings("sink").stream()
@@ -982,32 +984,6 @@ private  Supplier createRandomSupplier(List possibleValues) {
         return () -> possibleValues.get(ThreadLocalRandom.current().nextInt(size));
     }
 
-    // ------------------------------------------------------------------------------------
-
-    private static void waitForSnapshotStarted(String sinkName) throws InterruptedException {
-        while (sinkSize(sinkName) == 0) {
-            Thread.sleep(100);
-        }
-    }
-
-    private static void waitForSinkSize(String sinkName, int expectedSize)
-            throws InterruptedException {
-        while (sinkSize(sinkName) < expectedSize) {
-            Thread.sleep(100);
-        }
-    }
-
-    private static int sinkSize(String sinkName) {
-        synchronized (TestValuesTableFactory.class) {
-            try {
-                return TestValuesTableFactory.getRawResultsAsStrings(sinkName).size();
-            } catch (IllegalArgumentException e) {
-                // job is not started yet
-                return 0;
-            }
-        }
-    }
-
     @Test
     public void testCompositePkTableSplitsUnevenlyWithChunkKeyColumn() throws Exception {
         if (parallelismSnapshot) {
@@ -1070,7 +1046,7 @@ private void testUseChunkColumn(String chunkColumn) throws Exception {
 
         // async submit job
         TableResult result = tEnv.executeSql("INSERT INTO sink SELECT * FROM evenly_shopping_cart");
-        waitForSinkSize("sink", 12);
+        waitForSinkSize("sink", false, 12);
         result.getJobClient().get().cancel().get();
     }
 }
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/test/java/org/apache/flink/cdc/connectors/oracle/testutils/OracleTestUtils.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/test/java/org/apache/flink/cdc/connectors/oracle/testutils/OracleTestUtils.java
index b43927712d0..6f6e2b3bd2c 100644
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/test/java/org/apache/flink/cdc/connectors/oracle/testutils/OracleTestUtils.java
+++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/test/java/org/apache/flink/cdc/connectors/oracle/testutils/OracleTestUtils.java
@@ -20,7 +20,6 @@
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.runtime.highavailability.nonha.embedded.HaLeadershipControl;
 import org.apache.flink.runtime.minicluster.MiniCluster;
-import org.apache.flink.table.planner.factories.TestValuesTableFactory;
 
 import org.apache.commons.lang3.StringUtils;
 
@@ -76,42 +75,6 @@ public static void restartTaskManager(MiniCluster miniCluster, Runnable afterFai
         miniCluster.startTaskManager();
     }
 
-    public static void waitForSinkSize(String sinkName, int expectedSize)
-            throws InterruptedException {
-        while (sinkSize(sinkName) < expectedSize) {
-            Thread.sleep(100);
-        }
-    }
-
-    public static int sinkSize(String sinkName) {
-        synchronized (TestValuesTableFactory.class) {
-            try {
-                return TestValuesTableFactory.getRawResultsAsStrings(sinkName).size();
-            } catch (IllegalArgumentException e) {
-                // job is not started yet
-                return 0;
-            }
-        }
-    }
-
-    public static void waitForUpsertSinkSize(String sinkName, int expectedSize)
-            throws InterruptedException {
-        while (upsertSinkSize(sinkName) < expectedSize) {
-            Thread.sleep(100);
-        }
-    }
-
-    public static int upsertSinkSize(String sinkName) {
-        synchronized (TestValuesTableFactory.class) {
-            try {
-                return TestValuesTableFactory.getResultsAsStrings(sinkName).size();
-            } catch (IllegalArgumentException e) {
-                // job is not started yet
-                return 0;
-            }
-        }
-    }
-
     public static String getTableNameRegex(String[] captureCustomerTables) {
         checkState(captureCustomerTables.length > 0);
         if (captureCustomerTables.length == 1) {
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/pom.xml b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/pom.xml
index 0b12465a2d0..ebe7a34d30b 100644
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/pom.xml
+++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/pom.xml
@@ -94,6 +94,13 @@ limitations under the License.
         
 
         
+        
+            org.apache.flink
+            flink-cdc-common
+            ${project.version}
+            test-jar
+            test
+        
 
         
             org.apache.flink
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/test/java/org/apache/flink/cdc/connectors/postgres/PostgresTestBase.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/test/java/org/apache/flink/cdc/connectors/postgres/PostgresTestBase.java
index c63dec1e7a2..f896addb151 100644
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/test/java/org/apache/flink/cdc/connectors/postgres/PostgresTestBase.java
+++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/test/java/org/apache/flink/cdc/connectors/postgres/PostgresTestBase.java
@@ -20,9 +20,7 @@
 import org.apache.flink.cdc.connectors.postgres.source.PostgresConnectionPoolFactory;
 import org.apache.flink.cdc.connectors.postgres.source.config.PostgresSourceConfigFactory;
 import org.apache.flink.cdc.connectors.postgres.testutils.UniqueDatabase;
-import org.apache.flink.table.planner.factories.TestValuesTableFactory;
 import org.apache.flink.test.util.AbstractTestBase;
-import org.apache.flink.types.Row;
 
 import io.debezium.config.Configuration;
 import io.debezium.connector.postgresql.connection.PostgresConnection;
@@ -43,9 +41,7 @@
 import java.sql.DriverManager;
 import java.sql.SQLException;
 import java.sql.Statement;
-import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Random;
@@ -161,42 +157,6 @@ protected PostgresConnection createConnection(Map properties) {
         return new PostgresConnection(JdbcConfiguration.adapt(config), "test-connection");
     }
 
-    protected void waitForSnapshotStarted(String sinkName) throws InterruptedException {
-        while (sinkSize(sinkName) == 0) {
-            Thread.sleep(300);
-        }
-    }
-
-    protected void waitForSinkResult(String sinkName, List expected)
-            throws InterruptedException {
-        List actual = TestValuesTableFactory.getResultsAsStrings(sinkName);
-        actual = actual.stream().sorted().collect(Collectors.toList());
-        while (actual.size() != expected.size() || !actual.equals(expected)) {
-            actual =
-                    TestValuesTableFactory.getResultsAsStrings(sinkName).stream()
-                            .sorted()
-                            .collect(Collectors.toList());
-            Thread.sleep(1000);
-        }
-    }
-
-    protected void waitForSinkSize(String sinkName, int expectedSize) throws InterruptedException {
-        while (sinkSize(sinkName) < expectedSize) {
-            Thread.sleep(100);
-        }
-    }
-
-    protected int sinkSize(String sinkName) {
-        synchronized (TestValuesTableFactory.class) {
-            try {
-                return TestValuesTableFactory.getRawResultsAsStrings(sinkName).size();
-            } catch (IllegalArgumentException e) {
-                // job is not started yet
-                return 0;
-            }
-        }
-    }
-
     protected PostgresSourceConfigFactory getMockPostgresSourceConfigFactory(
             UniqueDatabase database, String schemaName, String tableName, int splitSize) {
         return getMockPostgresSourceConfigFactory(
@@ -224,16 +184,6 @@ protected PostgresSourceConfigFactory getMockPostgresSourceConfigFactory(
         return postgresSourceConfigFactory;
     }
 
-    public static List fetchRows(Iterator iter, int size) {
-        List rows = new ArrayList<>(size);
-        while (size > 0 && iter.hasNext()) {
-            Row row = iter.next();
-            rows.add(row.toString());
-            size--;
-        }
-        return rows;
-    }
-
     public static void assertEqualsInAnyOrder(List expected, List actual) {
         assertTrue(expected != null && actual != null);
         assertEqualsInOrder(
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/test/java/org/apache/flink/cdc/connectors/postgres/source/NewlyAddedTableITCase.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/test/java/org/apache/flink/cdc/connectors/postgres/source/NewlyAddedTableITCase.java
index e97ce65ab18..39e85625c1e 100644
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/test/java/org/apache/flink/cdc/connectors/postgres/source/NewlyAddedTableITCase.java
+++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/test/java/org/apache/flink/cdc/connectors/postgres/source/NewlyAddedTableITCase.java
@@ -57,6 +57,7 @@
 import java.util.stream.Collectors;
 
 import static java.lang.String.format;
+import static org.apache.flink.cdc.common.testutils.TestCaseUtils.waitForSinkSize;
 
 /**
  * IT tests to cover various newly added tables during capture process. Ignore this test because
@@ -421,7 +422,7 @@ private void testRemoveAndAddTablesOneByOne(int parallelism, String... captureAd
                             format(
                                     "+I[%s, 417022095255614379, China, %s, %s West Town address 3]",
                                     captureTableThisRound, cityName, cityName)));
-            waitForSinkSize("sink", fetchedDataList.size());
+            waitForSinkSize("sink", false, fetchedDataList.size());
             assertEqualsInAnyOrder(
                     fetchedDataList, TestValuesTableFactory.getRawResultsAsStrings("sink"));
 
@@ -448,7 +449,7 @@ private void testRemoveAndAddTablesOneByOne(int parallelism, String... captureAd
                                     cityName)));
 
             // step 3: assert fetched wal log data in this round
-            waitForSinkSize("sink", fetchedDataList.size());
+            waitForSinkSize("sink", false, fetchedDataList.size());
 
             assertEqualsInAnyOrder(
                     fetchedDataList, TestValuesTableFactory.getRawResultsAsStrings("sink"));
@@ -522,7 +523,7 @@ private void testRemoveTablesOneByOne(
                         miniClusterResource.getMiniCluster(),
                         () -> sleepMs(100));
             }
-            waitForSinkSize("sink", fetchedDataList.size());
+            waitForSinkSize("sink", false, fetchedDataList.size());
             assertEqualsInAnyOrder(
                     fetchedDataList, TestValuesTableFactory.getRawResultsAsStrings("sink"));
             finishedSavePointPath = triggerSavepointWithRetry(jobClient, savepointDirectory);
@@ -558,7 +559,7 @@ private void testRemoveTablesOneByOne(
             TableResult tableResult = tEnv.executeSql("insert into sink select * from address");
             JobClient jobClient = tableResult.getJobClient().get();
 
-            waitForSinkSize("sink", fetchedDataList.size());
+            waitForSinkSize("sink", false, fetchedDataList.size());
             assertEqualsInAnyOrder(
                     fetchedDataList, TestValuesTableFactory.getRawResultsAsStrings("sink"));
 
@@ -605,7 +606,7 @@ private void testRemoveTablesOneByOne(
 
             fetchedDataList.addAll(expectedWalLogDataThisRound);
             // step 4: assert fetched wal log data in this round
-            waitForSinkSize("sink", fetchedDataList.size());
+            waitForSinkSize("sink", false, fetchedDataList.size());
             assertEqualsInAnyOrder(
                     fetchedDataList, TestValuesTableFactory.getRawResultsAsStrings("sink"));
 
@@ -720,7 +721,7 @@ private void testNewlyAddedTableOneByOne(
                         () -> sleepMs(100));
             }
             fetchedDataList.addAll(expectedSnapshotDataThisRound);
-            PostgresTestUtils.waitForUpsertSinkSize("sink", fetchedDataList.size());
+            waitForSinkSize("sink", true, fetchedDataList.size());
             assertEqualsInAnyOrder(
                     fetchedDataList, TestValuesTableFactory.getResultsAsStrings("sink"));
 
@@ -759,7 +760,7 @@ private void testNewlyAddedTableOneByOne(
             // step 5: assert fetched wal log data in this round
             fetchedDataList.addAll(expectedWalLogUpsertDataThisRound);
 
-            PostgresTestUtils.waitForUpsertSinkSize("sink", fetchedDataList.size());
+            waitForSinkSize("sink", true, fetchedDataList.size());
             // the result size of sink may arrive fetchedDataList.size() with old data, wait one
             // checkpoint to wait retract old record and send new record
             Thread.sleep(1000);
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/test/java/org/apache/flink/cdc/connectors/postgres/source/PostgresSourceITCase.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/test/java/org/apache/flink/cdc/connectors/postgres/source/PostgresSourceITCase.java
index 7d2c6dde8ff..52ecb5d10ec 100644
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/test/java/org/apache/flink/cdc/connectors/postgres/source/PostgresSourceITCase.java
+++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/test/java/org/apache/flink/cdc/connectors/postgres/source/PostgresSourceITCase.java
@@ -20,6 +20,7 @@
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.api.common.eventtime.WatermarkStrategy;
 import org.apache.flink.api.common.restartstrategy.RestartStrategies;
+import org.apache.flink.cdc.common.testutils.TestCaseUtils;
 import org.apache.flink.cdc.connectors.base.options.StartupOptions;
 import org.apache.flink.cdc.connectors.base.source.utils.hooks.SnapshotPhaseHook;
 import org.apache.flink.cdc.connectors.base.source.utils.hooks.SnapshotPhaseHooks;
@@ -59,15 +60,14 @@
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashMap;
-import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.atomic.AtomicLong;
-import java.util.function.Function;
 import java.util.stream.Collectors;
 
 import static java.lang.String.format;
+import static org.apache.flink.cdc.common.testutils.TestCaseUtils.fetchAndConvert;
 import static org.apache.flink.cdc.connectors.postgres.testutils.PostgresTestUtils.hasNextData;
 import static org.apache.flink.cdc.connectors.postgres.testutils.PostgresTestUtils.triggerFailover;
 import static org.apache.flink.cdc.connectors.postgres.testutils.PostgresTestUtils.waitUntilJobRunning;
@@ -773,7 +773,7 @@ private List testBackfillWhenWritingEvents(
         try (CloseableIterator iterator =
                 env.fromSource(source, WatermarkStrategy.noWatermarks(), "Backfill Skipped Source")
                         .executeAndCollect()) {
-            records = fetchRowData(iterator, fetchSize, customerTable::stringify);
+            records = fetchAndConvert(iterator, fetchSize, customerTable::stringify);
             env.close();
         }
         return records;
@@ -943,7 +943,9 @@ private void checkSnapshotData(
         }
 
         assertEqualsInAnyOrder(
-                expectedSnapshotData, fetchRows(iterator, expectedSnapshotData.size()));
+                expectedSnapshotData,
+                TestCaseUtils.fetchAndConvert(
+                        iterator, expectedSnapshotData.size(), Row::toString));
     }
 
     private void checkStreamData(
@@ -984,7 +986,9 @@ private void checkStreamData(
         // wait for the stream reading
         Thread.sleep(2000L);
 
-        assertEqualsInAnyOrder(expectedStreamData, fetchRows(iterator, expectedStreamData.size()));
+        assertEqualsInAnyOrder(
+                expectedStreamData,
+                TestCaseUtils.fetchAndConvert(iterator, expectedStreamData.size(), Row::toString));
         assertTrue(!hasNextData(iterator));
     }
 
@@ -1053,7 +1057,9 @@ private void checkStreamDataWithHook(
         // wait for the stream reading
         Thread.sleep(2000L);
 
-        assertEqualsInAnyOrder(expectedStreamData, fetchRows(iterator, expectedStreamData.size()));
+        assertEqualsInAnyOrder(
+                expectedStreamData,
+                TestCaseUtils.fetchAndConvert(iterator, expectedStreamData.size(), Row::toString));
         assertTrue(!hasNextData(iterator));
     }
 
@@ -1109,7 +1115,9 @@ private void checkStreamDataWithDDLDuringFailover(
         // wait for the stream reading
         Thread.sleep(2000L);
 
-        assertEqualsInAnyOrder(expectedStreamData, fetchRows(iterator, expectedStreamData.size()));
+        assertEqualsInAnyOrder(
+                expectedStreamData,
+                TestCaseUtils.fetchAndConvert(iterator, expectedStreamData.size(), Row::toString));
         assertTrue(!hasNextData(iterator));
     }
 
@@ -1130,27 +1138,6 @@ private String getTableNameRegex(String[] captureCustomerTables) {
         }
     }
 
-    private static List fetchRowData(
-            Iterator iter, int size, Function stringifier) {
-        List rows = new ArrayList<>(size);
-        while (size > 0 && iter.hasNext()) {
-            RowData row = iter.next();
-            rows.add(row);
-            size--;
-        }
-        return rows.stream().map(stringifier).collect(Collectors.toList());
-    }
-
-    public static List fetchRows(Iterator iter, int size) {
-        List rows = new ArrayList<>(size);
-        while (size > 0 && iter.hasNext()) {
-            Row row = iter.next();
-            rows.add(row.toString());
-            size--;
-        }
-        return rows;
-    }
-
     /**
      * Make some changes on the specified customer table. Changelog in string could be accessed by
      * {@link #firstPartStreamEvents}.
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/test/java/org/apache/flink/cdc/connectors/postgres/table/PostgreSQLConnectorITCase.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/test/java/org/apache/flink/cdc/connectors/postgres/table/PostgreSQLConnectorITCase.java
index 4b355cac027..9d5a810d9e2 100644
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/test/java/org/apache/flink/cdc/connectors/postgres/table/PostgreSQLConnectorITCase.java
+++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/test/java/org/apache/flink/cdc/connectors/postgres/table/PostgreSQLConnectorITCase.java
@@ -45,6 +45,9 @@
 import java.util.List;
 import java.util.concurrent.ExecutionException;
 
+import static org.apache.flink.cdc.common.testutils.TestCaseUtils.fetchAndConvert;
+import static org.apache.flink.cdc.common.testutils.TestCaseUtils.waitForSinkSize;
+import static org.apache.flink.cdc.common.testutils.TestCaseUtils.waitForSnapshotStarted;
 import static org.hamcrest.Matchers.containsInAnyOrder;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertThat;
@@ -158,7 +161,7 @@ public void testConsumingAllEvents()
             statement.execute("DELETE FROM inventory.products WHERE id=111;");
         }
 
-        waitForSinkSize("sink", 20);
+        waitForSinkSize("sink", false, 20);
 
         /*
          * 
@@ -266,7 +269,7 @@ public void testStartupFromLatestOffset() throws Exception {
             statement.execute("DELETE FROM inventory.products WHERE id=111;");
         }
 
-        waitForSinkSize("sink", 5);
+        waitForSinkSize("sink", false, 5);
 
         String[] expected =
                 new String[] {"110,jacket,new water resistent white wind breaker,0.500"};
@@ -442,7 +445,7 @@ public void testAllTypes() throws Throwable {
         // async submit job
         TableResult result = tEnv.executeSql("INSERT INTO sink SELECT * FROM full_types");
 
-        waitForSinkSize("sink", 1);
+        waitForSinkSize("sink", false, 1);
         // wait a bit to make sure the replication slot is ready
         Thread.sleep(5000);
 
@@ -452,7 +455,7 @@ public void testAllTypes() throws Throwable {
             statement.execute("UPDATE inventory.full_types SET small_c=0 WHERE id=1;");
         }
 
-        waitForSinkSize("sink", 3);
+        waitForSinkSize("sink", false, 3);
 
         List expected =
                 Arrays.asList(
@@ -546,7 +549,7 @@ public void testMetadataColumns() throws Throwable {
         }
 
         // waiting for change events finished.
-        waitForSinkSize("sink", 16);
+        waitForSinkSize("sink", false, 16);
         String databaseName = POSTGRES_CONTAINER.getDatabaseName();
 
         List expected =
@@ -679,7 +682,7 @@ public void testUpsertMode() throws Exception {
             statement.execute("DELETE FROM inventory.products WHERE id=111;");
         }
 
-        waitForSinkSize("sink", 20);
+        waitForSinkSize("sink", false, 20);
 
         /*
          * 
@@ -784,7 +787,7 @@ public void testUniqueIndexIncludingFunction() throws Exception {
 
         expected.addAll(Arrays.asList("-U[1, a]", "+U[1, null]"));
         CloseableIterator iterator = tableResult.collect();
-        assertEqualsInAnyOrder(expected, fetchRows(iterator, expected.size()));
+        assertEqualsInAnyOrder(expected, fetchAndConvert(iterator, expected.size(), Row::toString));
         tableResult.getJobClient().get().cancel().get();
         RowUtils.USE_LEGACY_TO_STRING = true;
     }
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/test/java/org/apache/flink/cdc/connectors/postgres/table/PostgreSQLSavepointITCase.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/test/java/org/apache/flink/cdc/connectors/postgres/table/PostgreSQLSavepointITCase.java
index 24b9ec8ff3f..58ebb1cb733 100644
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/test/java/org/apache/flink/cdc/connectors/postgres/table/PostgreSQLSavepointITCase.java
+++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/test/java/org/apache/flink/cdc/connectors/postgres/table/PostgreSQLSavepointITCase.java
@@ -41,6 +41,7 @@
 import java.util.Optional;
 import java.util.concurrent.ExecutionException;
 
+import static org.apache.flink.cdc.common.testutils.TestCaseUtils.waitForSinkSize;
 import static org.hamcrest.Matchers.containsInAnyOrder;
 import static org.junit.Assert.assertThat;
 import static org.testcontainers.containers.PostgreSQLContainer.POSTGRESQL_PORT;
@@ -123,19 +124,24 @@ private void testRestartFromSavepoint() throws Exception {
 
         // wait for the source startup, we don't have a better way to wait it, use sleep for now
         Thread.sleep(10000L);
-        waitForSinkResult(
-                "sink",
-                Arrays.asList(
-                        "+I[101, scooter, Small 2-wheel scooter, 3.140]",
-                        "+I[102, car battery, 12V car battery, 8.100]",
-                        "+I[103, 12-pack drill bits, 12-pack of drill bits with sizes ranging from #40 to #3, 0.800]",
-                        "+I[104, hammer, 12oz carpenter's hammer, 0.750]",
-                        "+I[105, hammer, 14oz carpenter's hammer, 0.875]",
-                        "+I[106, hammer, 16oz carpenter's hammer, 1.000]",
-                        "+I[107, rocks, box of assorted rocks, 5.300]",
-                        "+I[108, jacket, water resistent black wind breaker, 0.100]",
-                        "+I[109, spare tire, 24 inch spare tire, 22.200]",
-                        "+I[110, jacket, new water resistent white wind breaker, 0.500]"));
+
+        waitForSinkSize("sink", false, 10);
+
+        List actual = TestValuesTableFactory.getResultsAsStrings("sink");
+        assertThat(
+                actual,
+                containsInAnyOrder(
+                        Arrays.asList(
+                                "+I[101, scooter, Small 2-wheel scooter, 3.140]",
+                                "+I[102, car battery, 12V car battery, 8.100]",
+                                "+I[103, 12-pack drill bits, 12-pack of drill bits with sizes ranging from #40 to #3, 0.800]",
+                                "+I[104, hammer, 12oz carpenter's hammer, 0.750]",
+                                "+I[105, hammer, 14oz carpenter's hammer, 0.875]",
+                                "+I[106, hammer, 16oz carpenter's hammer, 1.000]",
+                                "+I[107, rocks, box of assorted rocks, 5.300]",
+                                "+I[108, jacket, water resistent black wind breaker, 0.100]",
+                                "+I[109, spare tire, 24 inch spare tire, 22.200]",
+                                "+I[110, jacket, new water resistent white wind breaker, 0.500]")));
 
         finishedSavePointPath = triggerSavepointWithRetry(jobClient, savepointDirectory);
         jobClient.cancel().get();
@@ -162,7 +168,7 @@ private void testRestartFromSavepoint() throws Exception {
         result = tEnv.executeSql("INSERT INTO sink SELECT * FROM debezium_source");
         jobClient = result.getJobClient().get();
 
-        waitForSinkSize("sink", 15);
+        waitForSinkSize("sink", false, 15);
 
         String[] expected =
                 new String[] {
@@ -179,7 +185,7 @@ private void testRestartFromSavepoint() throws Exception {
                     "+I[112, jacket, new water resistent white wind breaker, 0.500]"
                 };
 
-        List actual = TestValuesTableFactory.getResultsAsStrings("sink");
+        actual = TestValuesTableFactory.getResultsAsStrings("sink");
         assertThat(actual, containsInAnyOrder(expected));
 
         jobClient.cancel().get();
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/test/java/org/apache/flink/cdc/connectors/postgres/testutils/PostgresTestUtils.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/test/java/org/apache/flink/cdc/connectors/postgres/testutils/PostgresTestUtils.java
index fce9ece5e3a..34eba11fdfc 100644
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/test/java/org/apache/flink/cdc/connectors/postgres/testutils/PostgresTestUtils.java
+++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/test/java/org/apache/flink/cdc/connectors/postgres/testutils/PostgresTestUtils.java
@@ -21,7 +21,6 @@
 import org.apache.flink.runtime.highavailability.nonha.embedded.HaLeadershipControl;
 import org.apache.flink.runtime.minicluster.MiniCluster;
 import org.apache.flink.table.api.TableResult;
-import org.apache.flink.table.planner.factories.TestValuesTableFactory;
 import org.apache.flink.util.CloseableIterator;
 
 import org.apache.commons.lang3.StringUtils;
@@ -86,24 +85,6 @@ public static void restartTaskManager(MiniCluster miniCluster, Runnable afterFai
         miniCluster.startTaskManager();
     }
 
-    public static void waitForUpsertSinkSize(String sinkName, int expectedSize)
-            throws InterruptedException {
-        while (upsertSinkSize(sinkName) < expectedSize) {
-            Thread.sleep(100);
-        }
-    }
-
-    public static int upsertSinkSize(String sinkName) {
-        synchronized (TestValuesTableFactory.class) {
-            try {
-                return TestValuesTableFactory.getResultsAsStrings(sinkName).size();
-            } catch (IllegalArgumentException e) {
-                // job is not started yet
-                return 0;
-            }
-        }
-    }
-
     public static String getTableNameRegex(String[] captureCustomerTables) {
         checkState(captureCustomerTables.length > 0);
         if (captureCustomerTables.length == 1) {
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/pom.xml b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/pom.xml
index a96d3eb57c2..a5996d96180 100644
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/pom.xml
+++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/pom.xml
@@ -76,6 +76,14 @@ limitations under the License.
 
         
 
+        
+            org.apache.flink
+            flink-cdc-common
+            ${project.version}
+            test-jar
+            test
+        
+
         
             org.apache.flink
             flink-table-planner_${scala.binary.version}
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/test/java/org/apache/flink/cdc/connectors/sqlserver/SqlServerTestBase.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/test/java/org/apache/flink/cdc/connectors/sqlserver/SqlServerTestBase.java
index d107ea5f489..bf5fdb93f31 100644
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/test/java/org/apache/flink/cdc/connectors/sqlserver/SqlServerTestBase.java
+++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/test/java/org/apache/flink/cdc/connectors/sqlserver/SqlServerTestBase.java
@@ -17,7 +17,6 @@
 
 package org.apache.flink.cdc.connectors.sqlserver;
 
-import org.apache.flink.table.planner.factories.TestValuesTableFactory;
 import org.apache.flink.test.util.AbstractTestBase;
 
 import org.awaitility.Awaitility;
@@ -198,28 +197,4 @@ protected void initializeSqlServerTable(String sqlFile) {
             throw new RuntimeException(e);
         }
     }
-
-    protected static void waitForSnapshotStarted(String sinkName) throws InterruptedException {
-        while (sinkSize(sinkName) == 0) {
-            Thread.sleep(100);
-        }
-    }
-
-    protected static void waitForSinkSize(String sinkName, int expectedSize)
-            throws InterruptedException {
-        while (sinkSize(sinkName) < expectedSize) {
-            Thread.sleep(100);
-        }
-    }
-
-    protected static int sinkSize(String sinkName) {
-        synchronized (TestValuesTableFactory.class) {
-            try {
-                return TestValuesTableFactory.getRawResultsAsStrings(sinkName).size();
-            } catch (IllegalArgumentException e) {
-                // job is not started yet
-                return 0;
-            }
-        }
-    }
 }
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/test/java/org/apache/flink/cdc/connectors/sqlserver/source/SqlServerSourceITCase.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/test/java/org/apache/flink/cdc/connectors/sqlserver/source/SqlServerSourceITCase.java
index e8b231aac2b..f41550e4f7c 100644
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/test/java/org/apache/flink/cdc/connectors/sqlserver/source/SqlServerSourceITCase.java
+++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/test/java/org/apache/flink/cdc/connectors/sqlserver/source/SqlServerSourceITCase.java
@@ -47,12 +47,10 @@
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
-import java.util.Iterator;
 import java.util.List;
-import java.util.function.Function;
-import java.util.stream.Collectors;
 
 import static java.lang.String.format;
+import static org.apache.flink.cdc.common.testutils.TestCaseUtils.fetchAndConvert;
 import static org.apache.flink.table.api.DataTypes.BIGINT;
 import static org.apache.flink.table.api.DataTypes.STRING;
 import static org.apache.flink.table.catalog.Column.physical;
@@ -368,7 +366,7 @@ private List testBackfillWhenWritingEvents(
         try (CloseableIterator iterator =
                 env.fromSource(source, WatermarkStrategy.noWatermarks(), "Backfill Skipped Source")
                         .executeAndCollect()) {
-            records = fetchRowData(iterator, fetchSize, customerTable::stringify);
+            records = fetchAndConvert(iterator, fetchSize, customerTable::stringify);
             env.close();
         }
         return records;
@@ -493,7 +491,8 @@ private void testSqlServerParallelSource(
 
         LOG.info("snapshot data start");
         assertEqualsInAnyOrder(
-                expectedSnapshotData, fetchRows(iterator, expectedSnapshotData.size()));
+                expectedSnapshotData,
+                fetchAndConvert(iterator, expectedSnapshotData.size(), Row::toString));
 
         // second step: check the change stream data
         for (String tableId : captureCustomerTables) {
@@ -525,7 +524,9 @@ private void testSqlServerParallelSource(
         for (int i = 0; i < captureCustomerTables.length; i++) {
             expectedBinlogData.addAll(Arrays.asList(binlogForSingleTable));
         }
-        assertEqualsInAnyOrder(expectedBinlogData, fetchRows(iterator, expectedBinlogData.size()));
+        assertEqualsInAnyOrder(
+                expectedBinlogData,
+                fetchAndConvert(iterator, expectedBinlogData.size(), Row::toString));
         tableResult.getJobClient().get().cancel().get();
     }
 
@@ -550,27 +551,6 @@ private void sleepMs(long millis) {
         }
     }
 
-    public static List fetchRowData(
-            Iterator iter, int size, Function stringifier) {
-        List rows = new ArrayList<>(size);
-        while (size > 0 && iter.hasNext()) {
-            RowData row = iter.next();
-            rows.add(row);
-            size--;
-        }
-        return rows.stream().map(stringifier).collect(Collectors.toList());
-    }
-
-    private static List fetchRows(Iterator iter, int size) {
-        List rows = new ArrayList<>(size);
-        while (size > 0 && iter.hasNext()) {
-            Row row = iter.next();
-            rows.add(row.toString());
-            size--;
-        }
-        return rows;
-    }
-
     private String getTableNameRegex(String[] captureCustomerTables) {
         checkState(captureCustomerTables.length > 0);
         if (captureCustomerTables.length == 1) {
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/test/java/org/apache/flink/cdc/connectors/sqlserver/source/SqlServerSourceTestBase.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/test/java/org/apache/flink/cdc/connectors/sqlserver/source/SqlServerSourceTestBase.java
index cd670792e97..1ca9870deba 100644
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/test/java/org/apache/flink/cdc/connectors/sqlserver/source/SqlServerSourceTestBase.java
+++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/test/java/org/apache/flink/cdc/connectors/sqlserver/source/SqlServerSourceTestBase.java
@@ -24,7 +24,6 @@
 import org.apache.flink.runtime.minicluster.MiniCluster;
 import org.apache.flink.runtime.minicluster.RpcServiceSharing;
 import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration;
-import org.apache.flink.table.planner.factories.TestValuesTableFactory;
 import org.apache.flink.test.util.MiniClusterWithClientResource;
 import org.apache.flink.util.TestLogger;
 
@@ -198,30 +197,6 @@ protected static void assertEqualsInOrder(List expected, List ac
         assertArrayEquals(expected.toArray(new String[0]), actual.toArray(new String[0]));
     }
 
-    protected static void waitForSnapshotStarted(String sinkName) throws InterruptedException {
-        while (sinkSize(sinkName) == 0) {
-            Thread.sleep(100);
-        }
-    }
-
-    protected static void waitForSinkSize(String sinkName, int expectedSize)
-            throws InterruptedException {
-        while (sinkSize(sinkName) < expectedSize) {
-            Thread.sleep(100);
-        }
-    }
-
-    protected static int sinkSize(String sinkName) {
-        synchronized (TestValuesTableFactory.class) {
-            try {
-                return TestValuesTableFactory.getRawResultsAsStrings(sinkName).size();
-            } catch (IllegalArgumentException e) {
-                // job is not started yet
-                return 0;
-            }
-        }
-    }
-
     protected Connection getJdbcConnection() throws SQLException {
         return DriverManager.getConnection(
                 MSSQL_SERVER_CONTAINER.getJdbcUrl(),
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/test/java/org/apache/flink/cdc/connectors/sqlserver/table/SqlServerConnectorITCase.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/test/java/org/apache/flink/cdc/connectors/sqlserver/table/SqlServerConnectorITCase.java
index 0b3465e0a7a..122a6a6e0d2 100644
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/test/java/org/apache/flink/cdc/connectors/sqlserver/table/SqlServerConnectorITCase.java
+++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/test/java/org/apache/flink/cdc/connectors/sqlserver/table/SqlServerConnectorITCase.java
@@ -40,6 +40,8 @@
 import java.util.concurrent.ExecutionException;
 
 import static org.apache.flink.api.common.JobStatus.RUNNING;
+import static org.apache.flink.cdc.common.testutils.TestCaseUtils.waitForSinkSize;
+import static org.apache.flink.cdc.common.testutils.TestCaseUtils.waitForSnapshotStarted;
 import static org.hamcrest.Matchers.containsInAnyOrder;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertThat;
@@ -165,7 +167,7 @@ public void testConsumingAllEvents()
             statement.execute("UPDATE inventory.dbo.products SET volume='1.2' WHERE id=110;");
         }
 
-        waitForSinkSize("sink", 20);
+        waitForSinkSize("sink", false, 20);
 
         /*
          * 
@@ -267,7 +269,7 @@ public void testStartupFromLatestOffset() throws Exception {
         statement.execute(
                 "INSERT INTO inventory.dbo.products (name,description,weight) VALUES ('scooter','Big 3-wheel scooter',5.20);");
 
-        waitForSinkSize("sink", 2);
+        waitForSinkSize("sink", false, 2);
 
         String[] expected =
                 new String[] {
@@ -379,7 +381,7 @@ public void testAllTypes() throws Throwable {
                     "UPDATE column_type_test.dbo.full_types SET val_int=8888 WHERE id=0;");
         }
 
-        waitForSinkSize("sink", 2);
+        waitForSinkSize("sink", false, 2);
 
         List expected =
                 Arrays.asList(
@@ -463,7 +465,7 @@ public void testMetadataColumns() throws Throwable {
         }
 
         // waiting for change events finished.
-        waitForSinkSize("sink", 16);
+        waitForSinkSize("sink", false, 16);
 
         List expected =
                 Arrays.asList(
@@ -549,7 +551,7 @@ private void testUseChunkColumn(String chunkColumn)
 
         // async submit job
         TableResult result = tEnv.executeSql("INSERT INTO sink SELECT * FROM evenly_shopping_cart");
-        waitForSinkSize("sink", 12);
+        waitForSinkSize("sink", false, 12);
         result.getJobClient().get().cancel().get();
     }
 }
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/test/java/org/apache/flink/cdc/connectors/sqlserver/table/SqlServerTimezoneITCase.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/test/java/org/apache/flink/cdc/connectors/sqlserver/table/SqlServerTimezoneITCase.java
index c3d9f985515..f56882db6be 100644
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/test/java/org/apache/flink/cdc/connectors/sqlserver/table/SqlServerTimezoneITCase.java
+++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/test/java/org/apache/flink/cdc/connectors/sqlserver/table/SqlServerTimezoneITCase.java
@@ -36,6 +36,7 @@
 import java.util.TimeZone;
 import java.util.concurrent.ExecutionException;
 
+import static org.apache.flink.cdc.common.testutils.TestCaseUtils.waitForSnapshotStarted;
 import static org.junit.Assert.assertEquals;
 import static org.testcontainers.containers.MSSQLServerContainer.MS_SQL_SERVER_PORT;
 
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-tidb-cdc/pom.xml b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-tidb-cdc/pom.xml
index 05b1e63ed63..869b41f35ba 100644
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-tidb-cdc/pom.xml
+++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-tidb-cdc/pom.xml
@@ -56,6 +56,15 @@ limitations under the License.
         
 
         
+
+        
+            org.apache.flink
+            flink-cdc-common
+            ${project.version}
+            test-jar
+            test
+        
+
         
             org.apache.flink
             flink-table-planner_${scala.binary.version}
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-tidb-cdc/src/test/java/org/apache/flink/cdc/connectors/tidb/table/TiDBConnectorITCase.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-tidb-cdc/src/test/java/org/apache/flink/cdc/connectors/tidb/table/TiDBConnectorITCase.java
index dcc48817804..7fcdeedee37 100644
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-tidb-cdc/src/test/java/org/apache/flink/cdc/connectors/tidb/table/TiDBConnectorITCase.java
+++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-tidb-cdc/src/test/java/org/apache/flink/cdc/connectors/tidb/table/TiDBConnectorITCase.java
@@ -38,6 +38,7 @@
 import java.util.List;
 import java.util.stream.Collectors;
 
+import static org.apache.flink.cdc.common.testutils.TestCaseUtils.waitForSinkSize;
 import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
@@ -100,7 +101,7 @@ public void testConsumingAllEvents() throws Exception {
         TableResult result = tEnv.executeSql("INSERT INTO sink SELECT * FROM tidb_source");
 
         // wait for snapshot finished and begin binlog
-        waitForSinkSize("sink", 9);
+        waitForSinkSize("sink", false, 9);
 
         try (Connection connection = getJdbcConnection("inventory");
                 Statement statement = connection.createStatement()) {
@@ -118,7 +119,7 @@ public void testConsumingAllEvents() throws Exception {
             statement.execute("DELETE FROM products WHERE id=111;");
         }
 
-        waitForSinkSize("sink", 16);
+        waitForSinkSize("sink", false, 16);
 
         /*
          * 
@@ -205,7 +206,7 @@ public void testDeleteColumn() throws Exception {
         TableResult result = tEnv.executeSql("INSERT INTO sink SELECT * FROM tidb_source");
 
         // wait for snapshot finished and begin binlog
-        waitForSinkSize("sink", 9);
+        waitForSinkSize("sink", false, 9);
 
         try (Connection connection = getJdbcConnection("inventory");
                 Statement statement = connection.createStatement()) {
@@ -220,7 +221,7 @@ public void testDeleteColumn() throws Exception {
             statement.execute("DELETE FROM products WHERE id=111;");
         }
 
-        waitForSinkSize("sink", 15);
+        waitForSinkSize("sink", false, 15);
 
         List expected =
                 Arrays.asList(
@@ -284,7 +285,7 @@ public void testAddColumn() throws Exception {
         TableResult result = tEnv.executeSql("INSERT INTO sink SELECT * FROM tidb_source");
 
         // wait for snapshot finished and begin binlog
-        waitForSinkSize("sink", 9);
+        waitForSinkSize("sink", false, 9);
 
         try (Connection connection = getJdbcConnection("inventory");
                 Statement statement = connection.createStatement()) {
@@ -304,7 +305,7 @@ public void testAddColumn() throws Exception {
             statement.execute("DELETE FROM products WHERE id=111;");
         }
 
-        waitForSinkSize("sink", 16);
+        waitForSinkSize("sink", false, 16);
 
         List expected =
                 Arrays.asList(
@@ -374,7 +375,7 @@ public void testMetadataColumns() throws Exception {
         TableResult result = tEnv.executeSql("INSERT INTO sink SELECT * FROM tidb_source");
 
         // wait for snapshot finished and begin binlog
-        waitForSinkSize("sink", 9);
+        waitForSinkSize("sink", false, 9);
 
         try (Connection connection = getJdbcConnection("inventory");
                 Statement statement = connection.createStatement()) {
@@ -382,7 +383,7 @@ public void testMetadataColumns() throws Exception {
                     "UPDATE products SET description='18oz carpenter hammer' WHERE id=106;");
         }
 
-        waitForSinkSize("sink", 10);
+        waitForSinkSize("sink", false, 10);
 
         List expected =
                 Arrays.asList(
@@ -519,7 +520,7 @@ public void testAllDataTypes() throws Throwable {
                         "INSERT INTO sink SELECT id, tiny_c, tiny_un_c, small_c, small_un_c, medium_c, medium_un_c, int_c, int_un_c, int11_c, big_c, big_un_c, varchar_c, char_c, real_c, float_c, double_c, decimal_c, numeric_c, big_decimal_c, bit1_c, tiny1_c, boolean_c, date_c, time_c, datetime3_c, datetime6_c, cast(timestamp_c as timestamp), file_uuid, bit_c, text_c, tiny_blob_c, blob_c, medium_blob_c, long_blob_c, year_c, enum_c, set_c, json_c FROM tidb_source");
 
         // wait for snapshot finished and begin binlog
-        waitForSinkSize("sink", 1);
+        waitForSinkSize("sink", false, 1);
 
         try (Connection connection = getJdbcConnection("column_type_test");
                 Statement statement = connection.createStatement()) {
@@ -527,7 +528,7 @@ public void testAllDataTypes() throws Throwable {
                     "UPDATE full_types SET timestamp_c = '2020-07-17 18:33:22' WHERE id=1;");
         }
 
-        waitForSinkSize("sink", 2);
+        waitForSinkSize("sink", false, 2);
 
         List expected =
                 Arrays.asList(
@@ -599,7 +600,7 @@ public void testTiDBServerTimezone(String timezone) throws Exception {
                         "INSERT INTO sink select `id`, date_c, time_c,datetime3_c, datetime6_c, cast(timestamp_c as timestamp) FROM tidb_source t");
 
         // wait for snapshot finished and begin binlog
-        waitForSinkSize("sink", 1);
+        waitForSinkSize("sink", false, 1);
 
         try (Connection connection = getJdbcConnection("column_type_test");
                 Statement statement = connection.createStatement()) {
@@ -607,7 +608,7 @@ public void testTiDBServerTimezone(String timezone) throws Exception {
                     "UPDATE full_types SET timestamp_c = '2020-07-17 18:33:22' WHERE id=1;");
         }
 
-        waitForSinkSize("sink", 2);
+        waitForSinkSize("sink", false, 2);
 
         List expected =
                 Arrays.asList(
@@ -619,24 +620,6 @@ public void testTiDBServerTimezone(String timezone) throws Exception {
         result.getJobClient().get().cancel().get();
     }
 
-    private static void waitForSinkSize(String sinkName, int expectedSize)
-            throws InterruptedException {
-        while (sinkSize(sinkName) < expectedSize) {
-            Thread.sleep(100);
-        }
-    }
-
-    private static int sinkSize(String sinkName) {
-        synchronized (TestValuesTableFactory.class) {
-            try {
-                return TestValuesTableFactory.getRawResultsAsStrings(sinkName).size();
-            } catch (IllegalArgumentException e) {
-                // job is not started yet
-                return 0;
-            }
-        }
-    }
-
     public static void assertEqualsInAnyOrder(List expected, List actual) {
         assertTrue(expected != null && actual != null);
         assertEqualsInOrder(
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-tidb-cdc/src/test/java/org/apache/flink/cdc/connectors/tidb/table/TiDBConnectorRegionITCase.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-tidb-cdc/src/test/java/org/apache/flink/cdc/connectors/tidb/table/TiDBConnectorRegionITCase.java
index 47c7ac366a2..7169492c0da 100644
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-tidb-cdc/src/test/java/org/apache/flink/cdc/connectors/tidb/table/TiDBConnectorRegionITCase.java
+++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-tidb-cdc/src/test/java/org/apache/flink/cdc/connectors/tidb/table/TiDBConnectorRegionITCase.java
@@ -35,6 +35,8 @@
 import java.sql.ResultSet;
 import java.sql.Statement;
 
+import static org.apache.flink.cdc.common.testutils.TestCaseUtils.waitForSinkSize;
+
 /** Integration tests for TiDB change stream event SQL source. */
 public class TiDBConnectorRegionITCase extends TiDBTestBase {
 
@@ -87,7 +89,7 @@ public void testRegionChange() throws Exception {
         // async submit job
         TableResult result = tEnv.executeSql("INSERT INTO sink SELECT * FROM tidb_source");
 
-        waitForSinkSize("sink", 1);
+        waitForSinkSize("sink", false, 1);
 
         int count = 0;
 
@@ -125,25 +127,7 @@ public void testRegionChange() throws Exception {
             LOG.info("count: {}", count);
         }
 
-        waitForSinkSize("sink", count);
+        waitForSinkSize("sink", false, count);
         result.getJobClient().get().cancel().get();
     }
-
-    private static void waitForSinkSize(String sinkName, int expectedSize)
-            throws InterruptedException {
-        while (sinkSize(sinkName) < expectedSize) {
-            Thread.sleep(100);
-        }
-    }
-
-    private static int sinkSize(String sinkName) {
-        synchronized (TestValuesTableFactory.class) {
-            try {
-                return TestValuesTableFactory.getRawResultsAsStrings(sinkName).size();
-            } catch (IllegalArgumentException e) {
-                // job is not started yet
-                return 0;
-            }
-        }
-    }
 }
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-vitess-cdc/pom.xml b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-vitess-cdc/pom.xml
index 39354246ed8..0c9667850c2 100644
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-vitess-cdc/pom.xml
+++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-vitess-cdc/pom.xml
@@ -83,6 +83,14 @@ limitations under the License.
 
         
 
+        
+            org.apache.flink
+            flink-cdc-common
+            ${project.version}
+            test-jar
+            test
+        
+
         
             org.apache.flink
             flink-table-planner_${scala.binary.version}
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-vitess-cdc/src/test/java/org/apache/flink/cdc/connectors/vitess/table/VitessConnectorITCase.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-vitess-cdc/src/test/java/org/apache/flink/cdc/connectors/vitess/table/VitessConnectorITCase.java
index c660fa67e84..eb1798239ab 100644
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-vitess-cdc/src/test/java/org/apache/flink/cdc/connectors/vitess/table/VitessConnectorITCase.java
+++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-vitess-cdc/src/test/java/org/apache/flink/cdc/connectors/vitess/table/VitessConnectorITCase.java
@@ -24,7 +24,6 @@
 import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
 import org.apache.flink.table.planner.factories.TestValuesTableFactory;
 import org.apache.flink.types.Row;
-import org.apache.flink.util.CloseableIterator;
 
 import org.junit.Before;
 import org.junit.Test;
@@ -32,13 +31,14 @@
 import java.sql.Connection;
 import java.sql.SQLException;
 import java.sql.Statement;
-import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Iterator;
 import java.util.List;
 import java.util.concurrent.ExecutionException;
 import java.util.stream.Collectors;
 
+import static org.apache.flink.cdc.common.testutils.TestCaseUtils.fetchAndConvert;
+import static org.apache.flink.cdc.common.testutils.TestCaseUtils.waitForSinkSize;
+import static org.apache.flink.cdc.common.testutils.TestCaseUtils.waitForSnapshotStarted;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
@@ -129,7 +129,7 @@ public void testConsumingAllEvents()
             statement.execute("DELETE FROM test.products WHERE id=111;");
         }
 
-        waitForSinkSize("sink", 20);
+        waitForSinkSize("sink", false, 20);
 
         List expected =
                 Arrays.asList(
@@ -207,49 +207,15 @@ public void testAllTypes() throws Throwable {
                         "-U[1, 127, 255, 32767, 65535, 2147483647, 4294967295, 2147483647, 9223372036854775807, Hello World, abc, 123.102, 404.4443, 123.4567, 346, true]",
                         "+U[1, 127, 255, 32767, 65535, 2147483647, 4294967295, 2147483647, 9223372036854775807, Bye World, abc, 123.102, 404.4443, 123.4567, 346, true]");
 
-        List actual = fetchRows(result.collect(), expected.size());
+        List actual = fetchAndConvert(result.collect(), expected.size(), Row::toString);
         assertEquals(expected, actual);
         result.getJobClient().get().cancel().get();
     }
 
-    private static List fetchRows(Iterator iter, int size) {
-        List rows = new ArrayList<>(size);
-        while (size > 0 && iter.hasNext()) {
-            Row row = iter.next();
-            rows.add(row.toString());
-            size--;
-        }
-        return rows;
-    }
-
     public static void assertEqualsInAnyOrder(List actual, List expected) {
         assertTrue(actual != null && expected != null);
         assertEquals(
                 actual.stream().sorted().collect(Collectors.toList()),
                 expected.stream().sorted().collect(Collectors.toList()));
     }
-
-    private static void waitForSnapshotStarted(CloseableIterator iterator) throws Exception {
-        while (!iterator.hasNext()) {
-            Thread.sleep(100);
-        }
-    }
-
-    private static void waitForSinkSize(String sinkName, int expectedSize)
-            throws InterruptedException {
-        while (sinkSize(sinkName) < expectedSize) {
-            Thread.sleep(100);
-        }
-    }
-
-    private static int sinkSize(String sinkName) {
-        synchronized (TestValuesTableFactory.class) {
-            try {
-                return TestValuesTableFactory.getRawResultsAsStrings(sinkName).size();
-            } catch (IllegalArgumentException e) {
-                // job is not started yet
-                return 0;
-            }
-        }
-    }
 }
diff --git a/tools/mig-test/datastream/compile_jobs.rb b/tools/mig-test/datastream/compile_jobs.rb
index 5c906e5bf31..2326a454059 100644
--- a/tools/mig-test/datastream/compile_jobs.rb
+++ b/tools/mig-test/datastream/compile_jobs.rb
@@ -20,7 +20,7 @@
 
 JOB_VERSIONS.each do |version|
   puts "Compiling DataStream job for CDC #{version}"
-  `cd datastream-#{version} && mvn clean package -DskipTests`
+  system "cd datastream-#{version} && mvn clean package -DskipTests"
 end
 
 puts 'Done'
\ No newline at end of file
diff --git a/tools/mig-test/datastream/datastream-3.2.0/pom.xml b/tools/mig-test/datastream/datastream-3.2.0/pom.xml
index c1f556033d1..9eb0212f977 100644
--- a/tools/mig-test/datastream/datastream-3.2.0/pom.xml
+++ b/tools/mig-test/datastream/datastream-3.2.0/pom.xml
@@ -29,7 +29,7 @@ limitations under the License.
         UTF-8
         1.18.1
         3.2.0
-        1.9.7.Final
+        1.9.8.Final
         2.12
         2.0.13
         UTF-8
@@ -136,20 +136,96 @@ limitations under the License.
                 
             
             
-                maven-assembly-plugin
+                org.apache.maven.plugins
+                maven-shade-plugin
                 
                     
+                        shade-flink
                         package
                         
-                            single
+                            shade
                         
+                        
+                            
+                            false
+                            false
+                            true
+                            
+                                ${project.basedir}/target/dependency-reduced-pom.xml
+                            
+                            
+                                
+                                    *:*
+                                    
+                                        module-info.class
+                                        META-INF/*.SF
+                                        META-INF/*.DSA
+                                        META-INF/*.RSA
+                                    
+                                
+                            
+                            
+                                
+                                    
+                                    io.debezium:debezium-api
+                                    io.debezium:debezium-embedded
+                                    io.debezium:debezium-core
+                                    io.debezium:debezium-ddl-parser
+                                    io.debezium:debezium-connector-mysql
+                                    org.apache.flink:flink-connector-debezium
+                                    org.apache.flink:flink-connector-mysql-cdc
+                                    org.antlr:antlr4-runtime
+                                    org.apache.kafka:*
+                                    mysql:mysql-connector-java
+                                    com.zendesk:mysql-binlog-connector-java
+                                    com.fasterxml.*:*
+                                    com.google.guava:*
+                                    com.esri.geometry:esri-geometry-api
+                                    com.zaxxer:HikariCP
+                                    
+                                    org.apache.flink:flink-shaded-guava
+                                
+                            
+                            
+                                
+                                    org.apache.kafka
+                                    
+                                        org.apache.flink.cdc.connectors.shaded.org.apache.kafka
+                                    
+                                
+                                
+                                    org.antlr
+                                    
+                                        org.apache.flink.cdc.connectors.shaded.org.antlr
+                                    
+                                
+                                
+                                    com.fasterxml
+                                    
+                                        org.apache.flink.cdc.connectors.shaded.com.fasterxml
+                                    
+                                
+                                
+                                    com.google
+                                    
+                                        org.apache.flink.cdc.connectors.shaded.com.google
+                                    
+                                
+                                
+                                    com.esri.geometry
+                                    org.apache.flink.cdc.connectors.shaded.com.esri.geometry
+                                
+                                
+                                    com.zaxxer
+                                    
+                                        org.apache.flink.cdc.connectors.shaded.com.zaxxer
+                                    
+                                
+                            
+                        
                     
                 
-                
-                    
-                        jar-with-dependencies
-                    
-                
             
         
     
diff --git a/tools/mig-test/datastream/datastream-3.2.0/src/main/java/DataStreamJob.java b/tools/mig-test/datastream/datastream-3.2.0/src/main/java/DataStreamJob.java
index f821ac0a2de..bfaa2d529bf 100644
--- a/tools/mig-test/datastream/datastream-3.2.0/src/main/java/DataStreamJob.java
+++ b/tools/mig-test/datastream/datastream-3.2.0/src/main/java/DataStreamJob.java
@@ -23,7 +23,7 @@
 
 public class DataStreamJob {
 
-    public static void main(String[] args) {
+    public static void main(String[] args) throws Exception {
         MySqlSource mySqlSource = MySqlSource.builder()
                 .hostname("localhost")
                 .port(3306)
@@ -45,10 +45,6 @@ public static void main(String[] args) {
                 .print()
                 .setParallelism(1);
 
-        try {
-            env.execute();
-        } catch (Exception e) {
-            // ... unfortunately
-        }
+        env.execute();
     }
 }
diff --git a/tools/mig-test/datastream/datastream-3.2.1/pom.xml b/tools/mig-test/datastream/datastream-3.2.1/pom.xml
index c7d680a2f3b..b3e2117720b 100644
--- a/tools/mig-test/datastream/datastream-3.2.1/pom.xml
+++ b/tools/mig-test/datastream/datastream-3.2.1/pom.xml
@@ -136,20 +136,96 @@ limitations under the License.
                 
             
             
-                maven-assembly-plugin
+                org.apache.maven.plugins
+                maven-shade-plugin
                 
                     
+                        shade-flink
                         package
                         
-                            single
+                            shade
                         
+                        
+                            
+                            false
+                            false
+                            true
+                            
+                                ${project.basedir}/target/dependency-reduced-pom.xml
+                            
+                            
+                                
+                                    *:*
+                                    
+                                        module-info.class
+                                        META-INF/*.SF
+                                        META-INF/*.DSA
+                                        META-INF/*.RSA
+                                    
+                                
+                            
+                            
+                                
+                                    
+                                    io.debezium:debezium-api
+                                    io.debezium:debezium-embedded
+                                    io.debezium:debezium-core
+                                    io.debezium:debezium-ddl-parser
+                                    io.debezium:debezium-connector-mysql
+                                    org.apache.flink:flink-connector-debezium
+                                    org.apache.flink:flink-connector-mysql-cdc
+                                    org.antlr:antlr4-runtime
+                                    org.apache.kafka:*
+                                    mysql:mysql-connector-java
+                                    com.zendesk:mysql-binlog-connector-java
+                                    com.fasterxml.*:*
+                                    com.google.guava:*
+                                    com.esri.geometry:esri-geometry-api
+                                    com.zaxxer:HikariCP
+                                    
+                                    org.apache.flink:flink-shaded-guava
+                                
+                            
+                            
+                                
+                                    org.apache.kafka
+                                    
+                                        org.apache.flink.cdc.connectors.shaded.org.apache.kafka
+                                    
+                                
+                                
+                                    org.antlr
+                                    
+                                        org.apache.flink.cdc.connectors.shaded.org.antlr
+                                    
+                                
+                                
+                                    com.fasterxml
+                                    
+                                        org.apache.flink.cdc.connectors.shaded.com.fasterxml
+                                    
+                                
+                                
+                                    com.google
+                                    
+                                        org.apache.flink.cdc.connectors.shaded.com.google
+                                    
+                                
+                                
+                                    com.esri.geometry
+                                    org.apache.flink.cdc.connectors.shaded.com.esri.geometry
+                                
+                                
+                                    com.zaxxer
+                                    
+                                        org.apache.flink.cdc.connectors.shaded.com.zaxxer
+                                    
+                                
+                            
+                        
                     
                 
-                
-                    
-                        jar-with-dependencies
-                    
-                
             
         
     
diff --git a/tools/mig-test/datastream/datastream-3.2.1/src/main/java/DataStreamJob.java b/tools/mig-test/datastream/datastream-3.2.1/src/main/java/DataStreamJob.java
index f821ac0a2de..bfaa2d529bf 100644
--- a/tools/mig-test/datastream/datastream-3.2.1/src/main/java/DataStreamJob.java
+++ b/tools/mig-test/datastream/datastream-3.2.1/src/main/java/DataStreamJob.java
@@ -23,7 +23,7 @@
 
 public class DataStreamJob {
 
-    public static void main(String[] args) {
+    public static void main(String[] args) throws Exception {
         MySqlSource mySqlSource = MySqlSource.builder()
                 .hostname("localhost")
                 .port(3306)
@@ -45,10 +45,6 @@ public static void main(String[] args) {
                 .print()
                 .setParallelism(1);
 
-        try {
-            env.execute();
-        } catch (Exception e) {
-            // ... unfortunately
-        }
+        env.execute();
     }
 }
diff --git a/tools/mig-test/datastream/datastream-3.3.0/pom.xml b/tools/mig-test/datastream/datastream-3.3.0/pom.xml
index cc65c6c7869..e3ba1b846d9 100644
--- a/tools/mig-test/datastream/datastream-3.3.0/pom.xml
+++ b/tools/mig-test/datastream/datastream-3.3.0/pom.xml
@@ -29,7 +29,7 @@ limitations under the License.
         UTF-8
         1.19.1
         3.3.0
-        1.9.7.Final
+        1.9.8.Final
         2.12
         2.0.13
         UTF-8
@@ -136,20 +136,96 @@ limitations under the License.
                 
             
             
-                maven-assembly-plugin
+                org.apache.maven.plugins
+                maven-shade-plugin
                 
                     
+                        shade-flink
                         package
                         
-                            single
+                            shade
                         
+                        
+                            
+                            false
+                            false
+                            true
+                            
+                                ${project.basedir}/target/dependency-reduced-pom.xml
+                            
+                            
+                                
+                                    *:*
+                                    
+                                        module-info.class
+                                        META-INF/*.SF
+                                        META-INF/*.DSA
+                                        META-INF/*.RSA
+                                    
+                                
+                            
+                            
+                                
+                                    
+                                    io.debezium:debezium-api
+                                    io.debezium:debezium-embedded
+                                    io.debezium:debezium-core
+                                    io.debezium:debezium-ddl-parser
+                                    io.debezium:debezium-connector-mysql
+                                    org.apache.flink:flink-connector-debezium
+                                    org.apache.flink:flink-connector-mysql-cdc
+                                    org.antlr:antlr4-runtime
+                                    org.apache.kafka:*
+                                    mysql:mysql-connector-java
+                                    com.zendesk:mysql-binlog-connector-java
+                                    com.fasterxml.*:*
+                                    com.google.guava:*
+                                    com.esri.geometry:esri-geometry-api
+                                    com.zaxxer:HikariCP
+                                    
+                                    org.apache.flink:flink-shaded-guava
+                                
+                            
+                            
+                                
+                                    org.apache.kafka
+                                    
+                                        org.apache.flink.cdc.connectors.shaded.org.apache.kafka
+                                    
+                                
+                                
+                                    org.antlr
+                                    
+                                        org.apache.flink.cdc.connectors.shaded.org.antlr
+                                    
+                                
+                                
+                                    com.fasterxml
+                                    
+                                        org.apache.flink.cdc.connectors.shaded.com.fasterxml
+                                    
+                                
+                                
+                                    com.google
+                                    
+                                        org.apache.flink.cdc.connectors.shaded.com.google
+                                    
+                                
+                                
+                                    com.esri.geometry
+                                    org.apache.flink.cdc.connectors.shaded.com.esri.geometry
+                                
+                                
+                                    com.zaxxer
+                                    
+                                        org.apache.flink.cdc.connectors.shaded.com.zaxxer
+                                    
+                                
+                            
+                        
                     
                 
-                
-                    
-                        jar-with-dependencies
-                    
-                
             
         
     
diff --git a/tools/mig-test/datastream/datastream-3.3.0/src/main/java/DataStreamJob.java b/tools/mig-test/datastream/datastream-3.3.0/src/main/java/DataStreamJob.java
index f821ac0a2de..bfaa2d529bf 100644
--- a/tools/mig-test/datastream/datastream-3.3.0/src/main/java/DataStreamJob.java
+++ b/tools/mig-test/datastream/datastream-3.3.0/src/main/java/DataStreamJob.java
@@ -23,7 +23,7 @@
 
 public class DataStreamJob {
 
-    public static void main(String[] args) {
+    public static void main(String[] args) throws Exception {
         MySqlSource mySqlSource = MySqlSource.builder()
                 .hostname("localhost")
                 .port(3306)
@@ -45,10 +45,6 @@ public static void main(String[] args) {
                 .print()
                 .setParallelism(1);
 
-        try {
-            env.execute();
-        } catch (Exception e) {
-            // ... unfortunately
-        }
+        env.execute();
     }
 }
diff --git a/tools/mig-test/datastream/datastream-3.4-SNAPSHOT/pom.xml b/tools/mig-test/datastream/datastream-3.4-SNAPSHOT/pom.xml
index e174d5583cf..f83f6804295 100644
--- a/tools/mig-test/datastream/datastream-3.4-SNAPSHOT/pom.xml
+++ b/tools/mig-test/datastream/datastream-3.4-SNAPSHOT/pom.xml
@@ -29,7 +29,7 @@ limitations under the License.
         UTF-8
         1.19.1
         3.4-SNAPSHOT
-        1.9.7.Final
+        1.9.8.Final
         2.12
         2.0.13
         UTF-8
@@ -136,20 +136,96 @@ limitations under the License.
                 
             
             
-                maven-assembly-plugin
+                org.apache.maven.plugins
+                maven-shade-plugin
                 
                     
+                        shade-flink
                         package
                         
-                            single
+                            shade
                         
+                        
+                            
+                            false
+                            false
+                            true
+                            
+                                ${project.basedir}/target/dependency-reduced-pom.xml
+                            
+                            
+                                
+                                    *:*
+                                    
+                                        module-info.class
+                                        META-INF/*.SF
+                                        META-INF/*.DSA
+                                        META-INF/*.RSA
+                                    
+                                
+                            
+                            
+                                
+                                    
+                                    io.debezium:debezium-api
+                                    io.debezium:debezium-embedded
+                                    io.debezium:debezium-core
+                                    io.debezium:debezium-ddl-parser
+                                    io.debezium:debezium-connector-mysql
+                                    org.apache.flink:flink-connector-debezium
+                                    org.apache.flink:flink-connector-mysql-cdc
+                                    org.antlr:antlr4-runtime
+                                    org.apache.kafka:*
+                                    mysql:mysql-connector-java
+                                    com.zendesk:mysql-binlog-connector-java
+                                    com.fasterxml.*:*
+                                    com.google.guava:*
+                                    com.esri.geometry:esri-geometry-api
+                                    com.zaxxer:HikariCP
+                                    
+                                    org.apache.flink:flink-shaded-guava
+                                
+                            
+                            
+                                
+                                    org.apache.kafka
+                                    
+                                        org.apache.flink.cdc.connectors.shaded.org.apache.kafka
+                                    
+                                
+                                
+                                    org.antlr
+                                    
+                                        org.apache.flink.cdc.connectors.shaded.org.antlr
+                                    
+                                
+                                
+                                    com.fasterxml
+                                    
+                                        org.apache.flink.cdc.connectors.shaded.com.fasterxml
+                                    
+                                
+                                
+                                    com.google
+                                    
+                                        org.apache.flink.cdc.connectors.shaded.com.google
+                                    
+                                
+                                
+                                    com.esri.geometry
+                                    org.apache.flink.cdc.connectors.shaded.com.esri.geometry
+                                
+                                
+                                    com.zaxxer
+                                    
+                                        org.apache.flink.cdc.connectors.shaded.com.zaxxer
+                                    
+                                
+                            
+                        
                     
                 
-                
-                    
-                        jar-with-dependencies
-                    
-                
             
         
     
diff --git a/tools/mig-test/datastream/datastream-3.4-SNAPSHOT/src/main/java/DataStreamJob.java b/tools/mig-test/datastream/datastream-3.4-SNAPSHOT/src/main/java/DataStreamJob.java
index f821ac0a2de..bfaa2d529bf 100644
--- a/tools/mig-test/datastream/datastream-3.4-SNAPSHOT/src/main/java/DataStreamJob.java
+++ b/tools/mig-test/datastream/datastream-3.4-SNAPSHOT/src/main/java/DataStreamJob.java
@@ -23,7 +23,7 @@
 
 public class DataStreamJob {
 
-    public static void main(String[] args) {
+    public static void main(String[] args) throws Exception {
         MySqlSource mySqlSource = MySqlSource.builder()
                 .hostname("localhost")
                 .port(3306)
@@ -45,10 +45,6 @@ public static void main(String[] args) {
                 .print()
                 .setParallelism(1);
 
-        try {
-            env.execute();
-        } catch (Exception e) {
-            // ... unfortunately
-        }
+        env.execute();
     }
 }
diff --git a/tools/mig-test/datastream/run_migration_test.rb b/tools/mig-test/datastream/run_migration_test.rb
index deb16b0f074..9b355c1c3dd 100644
--- a/tools/mig-test/datastream/run_migration_test.rb
+++ b/tools/mig-test/datastream/run_migration_test.rb
@@ -32,12 +32,18 @@ def exec_sql_source(sql)
   `mysql -h 127.0.0.1 -P#{SOURCE_PORT} -uroot --skip-password -e "USE #{DATABASE_NAME}; #{sql}"`
 end
 
+def extract_job_id(output)
+  current_job_id = output.split("\n").filter { _1.start_with?('Job has been submitted with JobID ') }.first&.split&.last
+  raise StandardError, "Failed to submit Flink job. Output: #{output}" unless current_job_id&.length == 32
+  current_job_id
+end
+
 def put_mystery_data(mystery)
   exec_sql_source("REPLACE INTO girl(id, name) VALUES (17, '#{mystery}');")
 end
 
 def ensure_mystery_data(mystery)
-  throw StandardError, 'Failed to get specific mystery string' unless `cat #{FLINK_HOME}/log/*.out`.include? mystery
+  raise StandardError, 'Failed to get specific mystery string' unless `cat #{FLINK_HOME}/log/*.out`.include? mystery
 end
 
 puts '   Waiting for source to start up...'
@@ -52,8 +58,8 @@ def test_migration_chore(from_version, to_version)
   # Clear previous savepoints and logs
   `rm -rf savepoints`
 
-  old_job_id = `#{FLINK_HOME}/bin/flink run -p 1 -c DataStreamJob --detached datastream-#{from_version}/target/datastream-job-#{from_version}-jar-with-dependencies.jar`.split.last
-  raise StandardError, 'Failed to submit Flink job' unless old_job_id.length == 32
+  old_output = `#{FLINK_HOME}/bin/flink run -p 1 -c DataStreamJob --detached datastream-#{from_version}/target/datastream-job-#{from_version}.jar`
+  old_job_id = extract_job_id(old_output)
 
   puts "Submitted job at #{from_version} as #{old_job_id}"
 
@@ -64,8 +70,8 @@ def test_migration_chore(from_version, to_version)
 
   puts `#{FLINK_HOME}/bin/flink stop --savepointPath #{Dir.pwd}/savepoints #{old_job_id}`
   savepoint_file = `ls savepoints`.split("\n").last
-  new_job_id = `#{FLINK_HOME}/bin/flink run --fromSavepoint #{Dir.pwd}/savepoints/#{savepoint_file} -p 1 -c DataStreamJob --detached datastream-#{to_version}/target/datastream-job-#{to_version}-jar-with-dependencies.jar`.split.last
-  raise StandardError, 'Failed to submit Flink job' unless new_job_id.length == 32
+  new_output = `#{FLINK_HOME}/bin/flink run --fromSavepoint #{Dir.pwd}/savepoints/#{savepoint_file} -p 1 -c DataStreamJob --detached datastream-#{to_version}/target/datastream-job-#{to_version}.jar`
+  new_job_id = extract_job_id(new_output)
 
   puts "Submitted job at #{to_version} as #{new_job_id}"
   random_string_2 = SecureRandom.hex(8)
diff --git a/tools/mig-test/run_migration_test.rb b/tools/mig-test/run_migration_test.rb
index bb111252cbf..4ee9e20d897 100644
--- a/tools/mig-test/run_migration_test.rb
+++ b/tools/mig-test/run_migration_test.rb
@@ -37,7 +37,7 @@ def put_mystery_data(mystery)
 end
 
 def ensure_mystery_data(mystery)
-  throw StandardError, 'Failed to get specific mystery string' unless `cat #{FLINK_HOME}/log/*.out`.include? mystery
+  raise StandardError, 'Failed to get specific mystery string' unless `cat #{FLINK_HOME}/log/*.out`.include? mystery
 end
 
 def extract_job_id(output)