From 6938558dca67f5328d5f514fbfa1e41722326c60 Mon Sep 17 00:00:00 2001 From: yuxiqian <34335406+yuxiqian@users.noreply.github.com> Date: Wed, 11 Dec 2024 14:11:56 +0800 Subject: [PATCH] [hotfix] Remove broken MySQL incremental sources used in tests Signed-off-by: yuxiqian <34335406+yuxiqian@users.noreply.github.com> --- .github/workflows/flink_cdc_base.yml | 3 +- ...st.java => GenericConnectionPoolTest.java} | 30 +- .../MySqlChangeEventSourceExampleTest.java | 286 ----------- .../base/MySqlSourceMetricsTest.java | 396 ---------------- .../EmbeddedFlinkDatabaseHistory.java | 159 ------- .../base/experimental/MySqlChunkSplitter.java | 64 --- .../base/experimental/MySqlDialect.java | 182 ------- .../base/experimental/MySqlSourceBuilder.java | 261 ----------- .../config/MySqlSourceConfigFactory.java | 135 ------ .../experimental/config/ServerIdRange.java | 114 ----- .../fetch/MySqlScanFetchTask.java | 335 ------------- .../fetch/MySqlSourceFetchTaskContext.java | 364 -------------- .../fetch/MySqlStreamFetchTask.java | 188 -------- .../MySqlSchemaChangeEventHandler.java | 50 -- .../experimental/offset/BinlogOffset.java | 217 --------- .../offset/BinlogOffsetFactory.java | 60 --- .../utils/MySqlConnectionUtils.java | 164 ------- .../base/experimental/utils/MySqlSchema.java | 110 ----- .../experimental/utils/MySqlTypeUtils.java | 133 ------ .../base/experimental/utils/MySqlUtils.java | 372 --------------- .../utils/TableDiscoveryUtils.java | 88 ---- .../MockedPooledDataSourceFactory.java} | 4 +- .../MockedSourceConfig.java} | 6 +- .../base/source/MySqlEventDeserializer.java | 121 ----- .../source/MySqlSchemaDataTypeInference.java | 45 -- .../PendingSplitsStateSerializerTest.java | 297 ------------ .../HybridPendingSplitsStateVersion5.java | 45 -- .../PendingSplitsStateSerializerVersion5.java | 161 ------- .../SnapshotPendingSplitsStateVersion5.java | 133 ------ .../HybridPendingSplitsStateVersion6.java | 41 -- .../PendingSplitsStateSerializerVersion6.java | 157 ------- .../SnapshotPendingSplitsStateVersion6.java | 130 ----- .../meta/split/SourceSplitSerializerTest.java | 154 ------ .../LegacySourceSplitSerializierVersion4.java | 130 ----- .../split/version4/StreamSplitVersion4.java | 86 ---- .../CustomAlterTableParserListener.java | 443 ------------------ .../CustomColumnDefinitionParserListener.java | 333 ------------- .../parser/CustomMySqlAntlrDdlParser.java | 288 ------------ .../CustomMySqlAntlrDdlParserListener.java | 143 ------ .../base/testutils/MySqlContainer.java | 176 ------- .../base/testutils/MySqlVersion.java | 41 -- .../base/testutils/UniqueDatabase.java | 152 ------ 42 files changed, 22 insertions(+), 6775 deletions(-) rename flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/{JdbcConnectionPoolTest.java => GenericConnectionPoolTest.java} (82%) delete mode 100644 flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/MySqlChangeEventSourceExampleTest.java delete mode 100644 flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/MySqlSourceMetricsTest.java delete mode 100644 flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/experimental/EmbeddedFlinkDatabaseHistory.java delete mode 100644 flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/experimental/MySqlChunkSplitter.java delete mode 100644 flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/experimental/MySqlDialect.java delete mode 100644 flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/experimental/MySqlSourceBuilder.java delete mode 100644 flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/experimental/config/MySqlSourceConfigFactory.java delete mode 100644 flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/experimental/config/ServerIdRange.java delete mode 100644 flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/experimental/fetch/MySqlScanFetchTask.java delete mode 100644 flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/experimental/fetch/MySqlSourceFetchTaskContext.java delete mode 100644 flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/experimental/fetch/MySqlStreamFetchTask.java delete mode 100644 flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/experimental/handler/MySqlSchemaChangeEventHandler.java delete mode 100644 flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/experimental/offset/BinlogOffset.java delete mode 100644 flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/experimental/offset/BinlogOffsetFactory.java delete mode 100644 flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/experimental/utils/MySqlConnectionUtils.java delete mode 100644 flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/experimental/utils/MySqlSchema.java delete mode 100644 flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/experimental/utils/MySqlTypeUtils.java delete mode 100644 flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/experimental/utils/MySqlUtils.java delete mode 100644 flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/experimental/utils/TableDiscoveryUtils.java rename flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/{experimental/MysqlPooledDataSourceFactory.java => mocked/MockedPooledDataSourceFactory.java} (91%) rename flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/{experimental/config/MySqlSourceConfig.java => mocked/MockedSourceConfig.java} (95%) delete mode 100644 flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/source/MySqlEventDeserializer.java delete mode 100644 flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/source/MySqlSchemaDataTypeInference.java delete mode 100644 flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/source/assigner/state/PendingSplitsStateSerializerTest.java delete mode 100644 flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/source/assigner/state/version5/HybridPendingSplitsStateVersion5.java delete mode 100644 flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/source/assigner/state/version5/PendingSplitsStateSerializerVersion5.java delete mode 100644 flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/source/assigner/state/version5/SnapshotPendingSplitsStateVersion5.java delete mode 100644 flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/source/assigner/state/version6/HybridPendingSplitsStateVersion6.java delete mode 100644 flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/source/assigner/state/version6/PendingSplitsStateSerializerVersion6.java delete mode 100644 flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/source/assigner/state/version6/SnapshotPendingSplitsStateVersion6.java delete mode 100644 flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/source/meta/split/SourceSplitSerializerTest.java delete mode 100644 flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/source/meta/split/version4/LegacySourceSplitSerializierVersion4.java delete mode 100644 flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/source/meta/split/version4/StreamSplitVersion4.java delete mode 100644 flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/source/parser/CustomAlterTableParserListener.java delete mode 100644 flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/source/parser/CustomColumnDefinitionParserListener.java delete mode 100644 flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/source/parser/CustomMySqlAntlrDdlParser.java delete mode 100644 flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/source/parser/CustomMySqlAntlrDdlParserListener.java delete mode 100644 flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/testutils/MySqlContainer.java delete mode 100644 flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/testutils/MySqlVersion.java delete mode 100644 flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/testutils/UniqueDatabase.java diff --git a/.github/workflows/flink_cdc_base.yml b/.github/workflows/flink_cdc_base.yml index 74260ef1404..707282fa177 100644 --- a/.github/workflows/flink_cdc_base.yml +++ b/.github/workflows/flink_cdc_base.yml @@ -41,7 +41,8 @@ env: flink-cdc-cli,\ flink-cdc-common,\ flink-cdc-composer,\ - flink-cdc-runtime" + flink-cdc-runtime,\ + flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base" MODULES_PIPELINE_CONNECTORS: "\ flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-values,\ diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/JdbcConnectionPoolTest.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/GenericConnectionPoolTest.java similarity index 82% rename from flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/JdbcConnectionPoolTest.java rename to flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/GenericConnectionPoolTest.java index ee4a078753a..7ec295c1e1a 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/JdbcConnectionPoolTest.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/GenericConnectionPoolTest.java @@ -18,8 +18,8 @@ package org.apache.flink.cdc.connectors.base; import org.apache.flink.cdc.connectors.base.config.JdbcSourceConfig; -import org.apache.flink.cdc.connectors.base.experimental.MysqlPooledDataSourceFactory; -import org.apache.flink.cdc.connectors.base.experimental.config.MySqlSourceConfig; +import org.apache.flink.cdc.connectors.base.mocked.MockedPooledDataSourceFactory; +import org.apache.flink.cdc.connectors.base.mocked.MockedSourceConfig; import org.apache.flink.cdc.connectors.base.options.StartupOptions; import org.apache.flink.cdc.connectors.base.relational.connection.ConnectionPoolId; import org.apache.flink.cdc.connectors.base.relational.connection.JdbcConnectionPoolFactory; @@ -34,7 +34,7 @@ import java.util.Properties; /** Tests for {@link JdbcConnectionPools}. */ -public class JdbcConnectionPoolTest { +public class GenericConnectionPoolTest { public static final String HOSTNAME = "localhost"; public static final int PORT = 3306; @@ -48,13 +48,13 @@ public class JdbcConnectionPoolTest { @Test public void testMultiConnectionPoolFactory() { MockConnectionPoolFactory mockConnectionPoolFactory = new MockConnectionPoolFactory(); - MysqlPooledDataSourceFactory mysqlPooledDataSourceFactory = - new MysqlPooledDataSourceFactory(); + MockedPooledDataSourceFactory genericPooledDataSourceFactory = + new MockedPooledDataSourceFactory(); JdbcConnectionPools mockInstance = JdbcConnectionPools.getInstance(mockConnectionPoolFactory); JdbcConnectionPools mysqlInstance = - JdbcConnectionPools.getInstance(mysqlPooledDataSourceFactory); - MySqlSourceConfig mySqlSourceConfig = + JdbcConnectionPools.getInstance(genericPooledDataSourceFactory); + MockedSourceConfig mySqlSourceConfig = getMockMySqlSourceConfig(HOSTNAME, PORT, USER_NAME, PASSWORD, DATABASE, TABLE); Assert.assertEquals( @@ -63,21 +63,21 @@ public void testMultiConnectionPoolFactory() { mockConnectionPoolFactory.getJdbcUrl(mySqlSourceConfig)); Assert.assertEquals( mysqlInstance.getJdbcUrl( - mySqlSourceConfig, mysqlPooledDataSourceFactory.getClass().getName()), - mysqlPooledDataSourceFactory.getJdbcUrl(mySqlSourceConfig)); + mySqlSourceConfig, genericPooledDataSourceFactory.getClass().getName()), + genericPooledDataSourceFactory.getJdbcUrl(mySqlSourceConfig)); Assert.assertNotEquals( mysqlInstance.getJdbcUrl( - mySqlSourceConfig, mysqlPooledDataSourceFactory.getClass().getName()), + mySqlSourceConfig, genericPooledDataSourceFactory.getClass().getName()), mockConnectionPoolFactory.getJdbcUrl(mySqlSourceConfig)); } @Test public void testNoDataSourcePoolFactoryIdentifier() { - MysqlPooledDataSourceFactory mysqlPooledDataSourceFactory = - new MysqlPooledDataSourceFactory(); + MockedPooledDataSourceFactory mysqlPooledDataSourceFactory = + new MockedPooledDataSourceFactory(); JdbcConnectionPools mysqlInstance = JdbcConnectionPools.getInstance(mysqlPooledDataSourceFactory); - MySqlSourceConfig mySqlSourceConfig = + MockedSourceConfig mySqlSourceConfig = getMockMySqlSourceConfig(HOSTNAME, PORT, USER_NAME, PASSWORD, DATABASE, TABLE); ConnectionPoolId poolId = new ConnectionPoolId( @@ -94,14 +94,14 @@ public void testNoDataSourcePoolFactoryIdentifier() { () -> mysqlInstance.getOrCreateConnectionPool(poolId, mySqlSourceConfig)); } - private static MySqlSourceConfig getMockMySqlSourceConfig( + private static MockedSourceConfig getMockMySqlSourceConfig( String hostname, int port, String username, String password, String database, String table) { - return new MySqlSourceConfig( + return new MockedSourceConfig( StartupOptions.latest(), Arrays.asList(database), Arrays.asList(table), diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/MySqlChangeEventSourceExampleTest.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/MySqlChangeEventSourceExampleTest.java deleted file mode 100644 index 744b5b44941..00000000000 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/MySqlChangeEventSourceExampleTest.java +++ /dev/null @@ -1,286 +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.connectors.base; - -import org.apache.flink.api.common.eventtime.WatermarkStrategy; -import org.apache.flink.cdc.connectors.base.experimental.MySqlSourceBuilder; -import org.apache.flink.cdc.connectors.base.experimental.utils.MySqlConnectionUtils; -import org.apache.flink.cdc.connectors.base.source.IncrementalSource; -import org.apache.flink.cdc.connectors.base.testutils.MySqlContainer; -import org.apache.flink.cdc.connectors.base.testutils.MySqlVersion; -import org.apache.flink.cdc.connectors.base.testutils.UniqueDatabase; -import org.apache.flink.cdc.debezium.JsonDebeziumDeserializationSchema; -import org.apache.flink.cdc.debezium.table.RowDataDebeziumDeserializeSchema; -import org.apache.flink.runtime.minicluster.RpcServiceSharing; -import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.table.api.DataTypes; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.conversion.RowRowConverter; -import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; -import org.apache.flink.table.types.DataType; -import org.apache.flink.table.types.logical.LogicalType; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.table.types.utils.TypeConversions; -import org.apache.flink.test.util.MiniClusterWithClientResource; -import org.apache.flink.util.CloseableIterator; - -import io.debezium.connector.mysql.MySqlConnection; -import io.debezium.jdbc.JdbcConnection; -import org.junit.BeforeClass; -import org.junit.Ignore; -import org.junit.Rule; -import org.junit.Test; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.testcontainers.containers.output.Slf4jLogConsumer; -import org.testcontainers.lifecycle.Startables; - -import java.sql.SQLException; -import java.time.ZoneId; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.stream.Collectors; -import java.util.stream.Stream; - -import static org.junit.Assert.assertArrayEquals; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; - -/** Example Tests for {@link IncrementalSource}. */ -public class MySqlChangeEventSourceExampleTest { - - private static final Logger LOG = - LoggerFactory.getLogger(MySqlChangeEventSourceExampleTest.class); - - private static final int DEFAULT_PARALLELISM = 4; - private static final MySqlContainer MYSQL_CONTAINER = createMySqlContainer(MySqlVersion.V5_7); - - @Rule - public final MiniClusterWithClientResource miniClusterResource = - new MiniClusterWithClientResource( - new MiniClusterResourceConfiguration.Builder() - .setNumberTaskManagers(1) - .setNumberSlotsPerTaskManager(DEFAULT_PARALLELISM) - .setRpcServiceSharing(RpcServiceSharing.DEDICATED) - .withHaLeadershipControl() - .build()); - - @BeforeClass - public static void startContainers() { - LOG.info("Starting containers..."); - Startables.deepStart(Stream.of(MYSQL_CONTAINER)).join(); - LOG.info("Containers are started."); - } - - private final UniqueDatabase inventoryDatabase = - new UniqueDatabase(MYSQL_CONTAINER, "inventory", "mysqluser", "mysqlpw"); - - @Test - @Ignore("Test ignored because it won't stop and is used for manual test") - public void testConsumingScanEvents() throws Exception { - inventoryDatabase.createAndInitialize(); - MySqlSourceBuilder.MySqlIncrementalSource mySqlChangeEventSource = - new MySqlSourceBuilder() - .hostname(MYSQL_CONTAINER.getHost()) - .port(MYSQL_CONTAINER.getDatabasePort()) - .databaseList(inventoryDatabase.getDatabaseName()) - .tableList(inventoryDatabase.getDatabaseName() + ".products") - .username(inventoryDatabase.getUsername()) - .password(inventoryDatabase.getPassword()) - .serverId("5401-5404") - .deserializer(new JsonDebeziumDeserializationSchema()) - .includeSchemaChanges(true) // output the schema changes as well - .build(); - - StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - // enable checkpoint - env.enableCheckpointing(3000); - // set the source parallelism to 4 - env.fromSource( - mySqlChangeEventSource, - WatermarkStrategy.noWatermarks(), - "MySqlParallelSource") - .setParallelism(4) - .print() - .setParallelism(1); - - env.execute("Print MySQL Snapshot + Binlog"); - } - - @Test - @Ignore("Test ignored because it won't stop and is used for manual test") - public void testConsumingAllEvents() throws Exception { - final DataType dataType = - DataTypes.ROW( - DataTypes.FIELD("id", DataTypes.BIGINT()), - DataTypes.FIELD("name", DataTypes.STRING()), - DataTypes.FIELD("description", DataTypes.STRING()), - DataTypes.FIELD("weight", DataTypes.FLOAT())); - - inventoryDatabase.createAndInitialize(); - final String tableId = inventoryDatabase.getDatabaseName() + ".products"; - MySqlSourceBuilder.MySqlIncrementalSource mySqlChangeEventSource = - new MySqlSourceBuilder() - .hostname(MYSQL_CONTAINER.getHost()) - .port(MYSQL_CONTAINER.getDatabasePort()) - .databaseList(inventoryDatabase.getDatabaseName()) - .tableList(tableId) - .username(inventoryDatabase.getUsername()) - .password(inventoryDatabase.getPassword()) - .serverId("5401-5404") - .deserializer(buildRowDataDebeziumDeserializeSchema(dataType)) - .includeSchemaChanges(true) // output the schema changes as well - .splitSize(2) - .build(); - - StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - - // enable checkpoint - env.enableCheckpointing(3000); - // set the source parallelism to 4 - CloseableIterator iterator = - env.fromSource( - mySqlChangeEventSource, - WatermarkStrategy.noWatermarks(), - "MySqlParallelSource") - .setParallelism(4) - .executeAndCollect(); // collect record - - String[] snapshotExpectedRecords = - new String[] { - "+I[101, scooter, Small 2-wheel scooter, 3.14]", - "+I[102, car battery, 12V car battery, 8.1]", - "+I[103, 12-pack drill bits, 12-pack of drill bits with sizes ranging from #40 to #3, 0.8]", - "+I[104, hammer, 12oz carpenter's hammer, 0.75]", - "+I[105, hammer, 14oz carpenter's hammer, 0.875]", - "+I[106, hammer, 16oz carpenter's hammer, 1.0]", - "+I[107, rocks, box of assorted rocks, 5.3]", - "+I[108, jacket, water resistent black wind breaker, 0.1]", - "+I[109, spare tire, 24 inch spare tire, 22.2]" - }; - - // step-1: consume snapshot data - List snapshotRowDataList = new ArrayList<>(); - for (int i = 0; i < snapshotExpectedRecords.length && iterator.hasNext(); i++) { - snapshotRowDataList.add(iterator.next()); - } - - List snapshotActualRecords = formatResult(snapshotRowDataList, dataType); - assertEqualsInAnyOrder(Arrays.asList(snapshotExpectedRecords), snapshotActualRecords); - - // step-2: make 6 change events in one MySQL transaction - makeBinlogEvents(getConnection(), tableId); - - String[] binlogExpectedRecords = - new String[] { - "-U[103, 12-pack drill bits, 12-pack of drill bits with sizes ranging from #40 to #3, 0.8]", - "+U[103, cart, 12-pack of drill bits with sizes ranging from #40 to #3, 0.8]", - "+I[110, spare tire, 28 inch spare tire, 26.2]", - "-D[110, spare tire, 28 inch spare tire, 26.2]", - "-U[103, cart, 12-pack of drill bits with sizes ranging from #40 to #3, 0.8]", - "+U[103, 12-pack drill bits, 12-pack of drill bits with sizes ranging from #40 to #3, 0.8]" - }; - - // step-3: consume binlog change events - List binlogRowDataList = new ArrayList<>(); - for (int i = 0; i < binlogExpectedRecords.length && iterator.hasNext(); i++) { - binlogRowDataList.add(iterator.next()); - } - List binlogActualRecords = formatResult(binlogRowDataList, dataType); - assertEqualsInAnyOrder(Arrays.asList(binlogExpectedRecords), binlogActualRecords); - - // stop the worker - iterator.close(); - } - - private RowDataDebeziumDeserializeSchema buildRowDataDebeziumDeserializeSchema( - DataType dataType) { - LogicalType logicalType = TypeConversions.fromDataToLogicalType(dataType); - InternalTypeInfo typeInfo = InternalTypeInfo.of(logicalType); - return RowDataDebeziumDeserializeSchema.newBuilder() - .setPhysicalRowType((RowType) dataType.getLogicalType()) - .setResultTypeInfo(typeInfo) - .build(); - } - - private List formatResult(List records, DataType dataType) { - RowRowConverter rowRowConverter = RowRowConverter.create(dataType); - rowRowConverter.open(Thread.currentThread().getContextClassLoader()); - return records.stream() - .map(rowRowConverter::toExternal) - .map(Object::toString) - .collect(Collectors.toList()); - } - - private MySqlConnection getConnection() { - Map properties = new HashMap<>(); - properties.put("database.hostname", MYSQL_CONTAINER.getHost()); - properties.put("database.port", String.valueOf(MYSQL_CONTAINER.getDatabasePort())); - properties.put("database.user", inventoryDatabase.getUsername()); - properties.put("database.password", inventoryDatabase.getPassword()); - properties.put("database.serverTimezone", ZoneId.of("UTC").toString()); - io.debezium.config.Configuration configuration = - io.debezium.config.Configuration.from(properties); - return MySqlConnectionUtils.createMySqlConnection(configuration); - } - - private void makeBinlogEvents(JdbcConnection connection, String tableId) throws SQLException { - try { - connection.setAutoCommit(false); - - // make binlog events - connection.execute( - "UPDATE " + tableId + " SET name = 'cart' where id = 103", - "INSERT INTO " - + tableId - + " VALUES(110,'spare tire','28 inch spare tire','26.2')", - "DELETE FROM " + tableId + " where id = 110", - "UPDATE " + tableId + " SET name = '12-pack drill bits' where id = 103"); - connection.commit(); - } finally { - connection.close(); - } - } - - public static void assertEqualsInAnyOrder(List expected, List actual) { - assertTrue(expected != null && actual != null); - assertEqualsInOrder( - expected.stream().sorted().collect(Collectors.toList()), - actual.stream().sorted().collect(Collectors.toList())); - } - - public static void assertEqualsInOrder(List expected, List actual) { - assertTrue(expected != null && actual != null); - assertEquals(expected.size(), actual.size()); - assertArrayEquals(expected.toArray(new String[0]), actual.toArray(new String[0])); - } - - private static MySqlContainer createMySqlContainer(MySqlVersion version) { - return new MySqlContainer(version) - .withConfigurationOverride("docker/server-gtids/my.cnf") - .withSetupSQL("docker/setup.sql") - .withDatabaseName("flink-test") - .withUsername("flinkuser") - .withPassword("flinkpw") - .withLogConsumer(new Slf4jLogConsumer(LOG)); - } -} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/MySqlSourceMetricsTest.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/MySqlSourceMetricsTest.java deleted file mode 100644 index cbcebbc4626..00000000000 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/MySqlSourceMetricsTest.java +++ /dev/null @@ -1,396 +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.connectors.base; - -import org.apache.flink.api.common.eventtime.WatermarkStrategy; -import org.apache.flink.cdc.common.data.RecordData; -import org.apache.flink.cdc.common.event.DataChangeEvent; -import org.apache.flink.cdc.common.event.Event; -import org.apache.flink.cdc.connectors.base.experimental.MySqlSourceBuilder; -import org.apache.flink.cdc.connectors.base.source.MySqlEventDeserializer; -import org.apache.flink.cdc.connectors.base.testutils.MySqlContainer; -import org.apache.flink.cdc.connectors.base.testutils.MySqlVersion; -import org.apache.flink.cdc.connectors.base.testutils.UniqueDatabase; -import org.apache.flink.cdc.debezium.table.DebeziumChangelogMode; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.metrics.Counter; -import org.apache.flink.metrics.Gauge; -import org.apache.flink.metrics.Metric; -import org.apache.flink.metrics.MetricGroup; -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.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.table.api.DataTypes; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.data.conversion.RowRowConverter; -import org.apache.flink.table.types.DataType; -import org.apache.flink.test.util.MiniClusterWithClientResource; -import org.apache.flink.types.RowKind; -import org.apache.flink.util.CloseableIterator; - -import io.debezium.connector.mysql.MySqlConnection; -import io.debezium.jdbc.JdbcConnection; -import org.junit.Assert; -import org.junit.BeforeClass; -import org.junit.Rule; -import org.junit.Test; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.testcontainers.containers.output.Slf4jLogConsumer; -import org.testcontainers.lifecycle.Startables; - -import java.sql.SQLException; -import java.time.ZoneId; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.stream.Collectors; -import java.util.stream.Stream; - -import static org.junit.Assert.assertArrayEquals; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; - -/** MySQL Source Metrics Tests. */ -public class MySqlSourceMetricsTest { - - private static final Logger LOG = LoggerFactory.getLogger(MySqlSourceMetricsTest.class); - - private static final int DEFAULT_PARALLELISM = 4; - private static final MySqlContainer MYSQL_CONTAINER = createMySqlContainer(MySqlVersion.V5_7); - protected InMemoryReporter metricReporter = InMemoryReporter.createWithRetainedMetrics(); - - @Rule - public final MiniClusterWithClientResource miniClusterResource = - new MiniClusterWithClientResource( - new MiniClusterResourceConfiguration.Builder() - .setNumberTaskManagers(1) - .setNumberSlotsPerTaskManager(DEFAULT_PARALLELISM) - .setRpcServiceSharing(RpcServiceSharing.DEDICATED) - .withHaLeadershipControl() - .setConfiguration( - metricReporter.addToConfiguration(new Configuration())) - .build()); - - @BeforeClass - public static void startContainers() { - LOG.info("Starting containers..."); - Startables.deepStart(Stream.of(MYSQL_CONTAINER)).join(); - LOG.info("Containers are started."); - } - - private final UniqueDatabase inventoryDatabase = - new UniqueDatabase(MYSQL_CONTAINER, "metrics", "mysqluser", "mysqlpw"); - - @Test - public void testSourceMetrics() throws Exception { - final DataType dataType = - DataTypes.ROW( - DataTypes.FIELD("id", DataTypes.BIGINT()), - DataTypes.FIELD("name", DataTypes.STRING()), - DataTypes.FIELD("age", DataTypes.INT())); - - inventoryDatabase.createAndInitialize(); - final String tableId = inventoryDatabase.getDatabaseName() + ".users"; - MySqlSourceBuilder.MySqlIncrementalSource mySqlChangeEventSource = - new MySqlSourceBuilder() - .hostname(MYSQL_CONTAINER.getHost()) - .port(MYSQL_CONTAINER.getDatabasePort()) - .databaseList(inventoryDatabase.getDatabaseName()) - .tableList(tableId) - .username(inventoryDatabase.getUsername()) - .password(inventoryDatabase.getPassword()) - .serverId("5401-5404") - .deserializer(buildRowDataDebeziumDeserializeSchema()) - .includeSchemaChanges(true) // output the schema changes as well - .splitSize(2) - .build(); - - StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - - // enable checkpoint - env.enableCheckpointing(3000); - // set the source parallelism to 4 - CloseableIterator iterator = - env.fromSource( - mySqlChangeEventSource, - WatermarkStrategy.noWatermarks(), - "MySqlParallelSource") - .setParallelism(1) - .executeAndCollect(); // collect record - String[] snapshotExpectedRecords = - new String[] { - "+I[101, Tom, 3]", - "+I[102, Jack, 5]", - "+I[103, Allen, 10]", - "+I[104, Andrew, 13]", - "+I[105, Arnold, 15]", - "+I[106, Claud, 19]", - "+I[107, Howard, 37]", - "+I[108, Jacob, 46]", - "+I[109, Lionel, 58]" - }; - - // step-1: consume snapshot data - List snapshotRowDataList = new ArrayList<>(); - for (int i = 0; i < snapshotExpectedRecords.length && iterator.hasNext(); i++) { - snapshotRowDataList.add(iterator.next()); - } - - List snapshotActualRecords = formatResult(snapshotRowDataList, dataType); - assertEqualsInAnyOrder(Arrays.asList(snapshotExpectedRecords), snapshotActualRecords); - - // step-2: make 6 change events in one MySQL transaction - makeBinlogEvents(getConnection(), tableId); - // mock ddl events - makeDdlEvents(getConnection(), tableId); - - String[] binlogExpectedRecords = - new String[] { - "-U[103, Allen, 10]", - "+U[103, Oswald, 10]", - "+I[110, Terence, 78]", - "-D[110, Terence, 78]", - "-U[103, Oswald, 10]", - "+U[103, Marry, 10]" - }; - - // step-3: consume binlog change events - List binlogRowDataList = new ArrayList<>(); - for (int i = 0; i < 4 && iterator.hasNext(); i++) { - binlogRowDataList.add(iterator.next()); - } - List binlogActualRecords = formatResult(binlogRowDataList, dataType); - assertEqualsInAnyOrder(Arrays.asList(binlogExpectedRecords), binlogActualRecords); - - Set metricGroups = metricReporter.findGroups("users"); - for (MetricGroup enumeratorGroup : metricGroups) { - boolean isTableMetric = true; - for (String scopeComponent : enumeratorGroup.getScopeComponents()) { - if (scopeComponent.contains("enumerator")) { - isTableMetric = false; - break; - } - } - if (!isTableMetric) { - break; - } - Map enumeratorMetrics = - metricReporter.getMetricsByGroup(enumeratorGroup); - Assert.assertEquals( - 1, ((Counter) enumeratorMetrics.get("numDeleteDMLRecords")).getCount()); - Assert.assertEquals( - 1, ((Counter) enumeratorMetrics.get("numInsertDMLRecords")).getCount()); - Assert.assertEquals( - 9, ((Counter) enumeratorMetrics.get("numSnapshotRecords")).getCount()); - // ddl eventd - Assert.assertEquals(1, ((Counter) enumeratorMetrics.get("numDDLRecords")).getCount()); - Assert.assertEquals(13, ((Counter) enumeratorMetrics.get("numRecordsIn")).getCount()); - Assert.assertEquals( - 2, ((Counter) enumeratorMetrics.get("numUpdateDMLRecords")).getCount()); - } - Set enumeratorGroups = metricReporter.findGroups("enumerator"); - for (MetricGroup enumeratorGroup : enumeratorGroups) { - boolean isTableMetric = false; - for (String scopeComponent : enumeratorGroup.getScopeComponents()) { - if (scopeComponent.contains("users")) { - isTableMetric = true; - break; - } - } - Map enumeratorMetrics = - metricReporter.getMetricsByGroup(enumeratorGroup); - if (isTableMetric) { - Assert.assertEquals( - 0, - ((Gauge) enumeratorMetrics.get("numSnapshotSplitsRemaining")) - .getValue() - .intValue()); - Assert.assertEquals( - 5, - ((Gauge) enumeratorMetrics.get("numSnapshotSplitsProcessed")) - .getValue() - .intValue()); - Assert.assertEquals( - 5, - ((Gauge) enumeratorMetrics.get("numSnapshotSplitsFinished")) - .getValue() - .intValue()); - Assert.assertTrue( - ((Gauge) enumeratorMetrics.get("snapshotEndTime")) - .getValue() - .longValue() - > 0); - Assert.assertTrue( - ((Gauge) enumeratorMetrics.get("snapshotStartTime")) - .getValue() - .longValue() - > 0); - } else { - Assert.assertEquals( - 0, - ((Gauge) enumeratorMetrics.get("isSnapshotting")) - .getValue() - .intValue()); - Assert.assertEquals( - 1, - ((Gauge) enumeratorMetrics.get("isStreamReading")) - .getValue() - .intValue()); - Assert.assertEquals( - 1, - ((Gauge) enumeratorMetrics.get("numTablesSnapshotted")) - .getValue() - .intValue()); - Assert.assertEquals( - 0, - ((Gauge) enumeratorMetrics.get("numSnapshotSplitsRemaining")) - .getValue() - .intValue()); - Assert.assertEquals( - 5, - ((Gauge) enumeratorMetrics.get("numSnapshotSplitsProcessed")) - .getValue() - .intValue()); - } - } - // stop the worker - iterator.close(); - } - - private MySqlEventDeserializer buildRowDataDebeziumDeserializeSchema() { - MySqlEventDeserializer deserializer = - new MySqlEventDeserializer(DebeziumChangelogMode.ALL, true); - return deserializer; - } - - private List formatResult(List records, DataType dataType) { - RowRowConverter rowRowConverter = RowRowConverter.create(dataType); - rowRowConverter.open(Thread.currentThread().getContextClassLoader()); - return records.stream() - .flatMap( - item -> { - DataChangeEvent changeEvent = ((DataChangeEvent) item); - RecordData before = changeEvent.before(); - RecordData after = changeEvent.after(); - - switch (changeEvent.op()) { - case INSERT: - GenericRowData insertData = new GenericRowData(3); - insertData.setRowKind(RowKind.INSERT); - convertData(changeEvent.after(), insertData); - return Arrays.stream(new GenericRowData[] {insertData}); - case DELETE: - GenericRowData deleteData = null; - deleteData = new GenericRowData(3); - deleteData.setRowKind(RowKind.DELETE); - convertData(before, deleteData); - return Arrays.stream(new GenericRowData[] {deleteData}); - case UPDATE: - case REPLACE: - GenericRowData beforeData = new GenericRowData(3); - beforeData.setRowKind(RowKind.UPDATE_BEFORE); - convertData(before, beforeData); - - GenericRowData afterData = new GenericRowData(3); - afterData.setRowKind(RowKind.UPDATE_AFTER); - convertData(after, afterData); - return Stream.of(beforeData, afterData) - .filter(row -> row != null); - } - return Stream.empty(); - }) - .map(rowRowConverter::toExternal) - .map(Object::toString) - .collect(Collectors.toList()); - } - - private void convertData(RecordData inputData, GenericRowData outputData) { - outputData.setField(0, inputData.getLong(0)); - outputData.setField(1, StringData.fromString(inputData.getString(1).toString())); - outputData.setField(2, inputData.getInt(2)); - } - - private MySqlConnection getConnection() { - Map properties = new HashMap<>(); - properties.put("database.hostname", MYSQL_CONTAINER.getHost()); - properties.put("database.port", String.valueOf(MYSQL_CONTAINER.getDatabasePort())); - properties.put("database.user", inventoryDatabase.getUsername()); - properties.put("database.password", inventoryDatabase.getPassword()); - properties.put("database.serverTimezone", ZoneId.of("UTC").toString()); - // properties.put("transaction.topic", "transaction_topic"); - io.debezium.config.Configuration configuration = - io.debezium.config.Configuration.from(properties); - return new MySqlConnection(new MySqlConnection.MySqlConnectionConfiguration(configuration)); - } - - private void makeBinlogEvents(JdbcConnection connection, String tableId) throws SQLException { - try { - connection.setAutoCommit(false); - - // make binlog events - connection.execute( - "UPDATE " + tableId + " SET name = 'Oswald' where id = 103", - "INSERT INTO " + tableId + " VALUES(110,'Terence',78)", - "DELETE FROM " + tableId + " where id = 110", - "UPDATE " + tableId + " SET name = 'Marry' where id = 103"); - connection.commit(); - } finally { - connection.close(); - } - } - - private void makeDdlEvents(JdbcConnection connection, String tableId) throws SQLException { - try { - connection.setAutoCommit(false); - // make binlog events - connection.execute("alter table " + tableId + " add test_add_col int null"); - connection.commit(); - } finally { - connection.close(); - } - } - - public static void assertEqualsInAnyOrder(List expected, List actual) { - assertTrue(expected != null && actual != null); - assertEqualsInOrder( - expected.stream().sorted().collect(Collectors.toList()), - actual.stream().sorted().collect(Collectors.toList())); - } - - public static void assertEqualsInOrder(List expected, List actual) { - assertTrue(expected != null && actual != null); - assertEquals(expected.size(), actual.size()); - assertArrayEquals(expected.toArray(new String[0]), actual.toArray(new String[0])); - } - - private static MySqlContainer createMySqlContainer(MySqlVersion version) { - return new MySqlContainer(version) - .withConfigurationOverride("docker/server-gtids/my.cnf") - .withSetupSQL("docker/setup.sql") - .withDatabaseName("flink-test") - .withUsername("flinkuser") - .withPassword("flinkpw") - .withLogConsumer(new Slf4jLogConsumer(LOG)); - } -} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/experimental/EmbeddedFlinkDatabaseHistory.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/experimental/EmbeddedFlinkDatabaseHistory.java deleted file mode 100644 index 6338989c246..00000000000 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/experimental/EmbeddedFlinkDatabaseHistory.java +++ /dev/null @@ -1,159 +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.connectors.base.experimental; - -import org.apache.flink.cdc.connectors.base.source.meta.split.SourceSplitState; - -import io.debezium.config.Configuration; -import io.debezium.relational.TableId; -import io.debezium.relational.Tables; -import io.debezium.relational.ddl.DdlParser; -import io.debezium.relational.history.DatabaseHistory; -import io.debezium.relational.history.DatabaseHistoryException; -import io.debezium.relational.history.DatabaseHistoryListener; -import io.debezium.relational.history.HistoryRecord; -import io.debezium.relational.history.HistoryRecordComparator; -import io.debezium.relational.history.TableChanges; -import io.debezium.relational.history.TableChanges.TableChange; - -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.Map; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; - -/** - * A {@link DatabaseHistory} implementation which store the latest table schema in Flink state. - * - *

It stores/recovers history using data offered by {@link SourceSplitState}. - */ -public class EmbeddedFlinkDatabaseHistory implements DatabaseHistory { - - public static final String DATABASE_HISTORY_INSTANCE_NAME = "database.history.instance.name"; - - public static final ConcurrentMap> TABLE_SCHEMAS = - new ConcurrentHashMap<>(); - - private Map tableSchemas; - private DatabaseHistoryListener listener; - private boolean storeOnlyMonitoredTablesDdl; - private boolean skipUnparseableDDL; - - @Override - public void configure( - Configuration config, - HistoryRecordComparator comparator, - DatabaseHistoryListener listener, - boolean useCatalogBeforeSchema) { - this.listener = listener; - this.storeOnlyMonitoredTablesDdl = config.getBoolean(STORE_ONLY_MONITORED_TABLES_DDL); - this.skipUnparseableDDL = config.getBoolean(SKIP_UNPARSEABLE_DDL_STATEMENTS); - - // recover - String instanceName = config.getString(DATABASE_HISTORY_INSTANCE_NAME); - this.tableSchemas = new HashMap<>(); - for (TableChange tableChange : removeHistory(instanceName)) { - tableSchemas.put(tableChange.getId(), tableChange); - } - } - - @Override - public void start() { - listener.started(); - } - - @Override - public void record( - Map source, Map position, String databaseName, String ddl) - throws DatabaseHistoryException { - throw new UnsupportedOperationException("should not call here, error"); - } - - @Override - public void record( - Map source, - Map position, - String databaseName, - String schemaName, - String ddl, - TableChanges changes) - throws DatabaseHistoryException { - final HistoryRecord record = - new HistoryRecord(source, position, databaseName, schemaName, ddl, changes); - listener.onChangeApplied(record); - } - - @Override - public void recover( - Map source, Map position, Tables schema, DdlParser ddlParser) { - listener.recoveryStarted(); - for (TableChange tableChange : tableSchemas.values()) { - schema.overwriteTable(tableChange.getTable()); - } - listener.recoveryStopped(); - } - - @Override - public void recover( - Map, Map> offsets, Tables schema, DdlParser ddlParser) { - offsets.forEach((source, position) -> recover(source, position, schema, ddlParser)); - } - - @Override - public void stop() { - listener.stopped(); - } - - @Override - public boolean exists() { - return tableSchemas != null && !tableSchemas.isEmpty(); - } - - @Override - public boolean storageExists() { - return true; - } - - @Override - public void initializeStorage() { - // do nothing - } - - @Override - public boolean storeOnlyCapturedTables() { - return storeOnlyMonitoredTablesDdl; - } - - @Override - public boolean skipUnparseableDdlStatements() { - return skipUnparseableDDL; - } - - public static void registerHistory(String engineName, Collection engineHistory) { - TABLE_SCHEMAS.put(engineName, engineHistory); - } - - public static Collection removeHistory(String engineName) { - if (engineName == null) { - return Collections.emptyList(); - } - Collection tableChanges = TABLE_SCHEMAS.remove(engineName); - return tableChanges != null ? tableChanges : Collections.emptyList(); - } -} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/experimental/MySqlChunkSplitter.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/experimental/MySqlChunkSplitter.java deleted file mode 100644 index 497e8e657a6..00000000000 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/experimental/MySqlChunkSplitter.java +++ /dev/null @@ -1,64 +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.connectors.base.experimental; - -import org.apache.flink.cdc.common.annotation.Internal; -import org.apache.flink.cdc.connectors.base.config.JdbcSourceConfig; -import org.apache.flink.cdc.connectors.base.dialect.JdbcDataSourceDialect; -import org.apache.flink.cdc.connectors.base.experimental.utils.MySqlTypeUtils; -import org.apache.flink.cdc.connectors.base.experimental.utils.MySqlUtils; -import org.apache.flink.cdc.connectors.base.source.assigner.splitter.JdbcSourceChunkSplitter; -import org.apache.flink.table.types.DataType; - -import io.debezium.jdbc.JdbcConnection; -import io.debezium.relational.Column; -import io.debezium.relational.TableId; - -import java.sql.SQLException; - -/** The {@code ChunkSplitter} used to split table into a set of chunks for JDBC data source. */ -@Internal -public class MySqlChunkSplitter extends JdbcSourceChunkSplitter { - - public MySqlChunkSplitter(JdbcSourceConfig sourceConfig, JdbcDataSourceDialect dialect) { - super(sourceConfig, dialect); - } - - @Override - public Object queryNextChunkMax( - JdbcConnection jdbc, - TableId tableId, - Column splitColumn, - int chunkSize, - Object includedLowerBound) - throws SQLException { - return MySqlUtils.queryNextChunkMax( - jdbc, tableId, splitColumn.name(), chunkSize, includedLowerBound); - } - - @Override - protected Long queryApproximateRowCnt(JdbcConnection jdbc, TableId tableId) - throws SQLException { - return MySqlUtils.queryApproximateRowCnt(jdbc, tableId); - } - - @Override - protected DataType fromDbzColumn(Column splitColumn) { - return MySqlTypeUtils.fromDbzColumn(splitColumn); - } -} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/experimental/MySqlDialect.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/experimental/MySqlDialect.java deleted file mode 100644 index f9f3e7f6a7f..00000000000 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/experimental/MySqlDialect.java +++ /dev/null @@ -1,182 +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.connectors.base.experimental; - -import org.apache.flink.cdc.common.annotation.Experimental; -import org.apache.flink.cdc.connectors.base.config.JdbcSourceConfig; -import org.apache.flink.cdc.connectors.base.dialect.JdbcDataSourceDialect; -import org.apache.flink.cdc.connectors.base.experimental.config.MySqlSourceConfig; -import org.apache.flink.cdc.connectors.base.experimental.fetch.MySqlScanFetchTask; -import org.apache.flink.cdc.connectors.base.experimental.fetch.MySqlSourceFetchTaskContext; -import org.apache.flink.cdc.connectors.base.experimental.fetch.MySqlStreamFetchTask; -import org.apache.flink.cdc.connectors.base.experimental.utils.MySqlConnectionUtils; -import org.apache.flink.cdc.connectors.base.experimental.utils.MySqlSchema; -import org.apache.flink.cdc.connectors.base.experimental.utils.TableDiscoveryUtils; -import org.apache.flink.cdc.connectors.base.relational.connection.JdbcConnectionFactory; -import org.apache.flink.cdc.connectors.base.relational.connection.JdbcConnectionPoolFactory; -import org.apache.flink.cdc.connectors.base.source.assigner.splitter.ChunkSplitter; -import org.apache.flink.cdc.connectors.base.source.meta.offset.Offset; -import org.apache.flink.cdc.connectors.base.source.meta.split.SourceSplitBase; -import org.apache.flink.cdc.connectors.base.source.reader.external.FetchTask; -import org.apache.flink.util.FlinkRuntimeException; - -import com.github.shyiko.mysql.binlog.BinaryLogClient; -import io.debezium.connector.mysql.MySqlConnection; -import io.debezium.jdbc.JdbcConfiguration; -import io.debezium.jdbc.JdbcConnection; -import io.debezium.relational.TableId; -import io.debezium.relational.Tables; -import io.debezium.relational.history.TableChanges; -import io.debezium.relational.history.TableChanges.TableChange; - -import java.sql.SQLException; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -/** The {@link JdbcDataSourceDialect} implementation for MySQL datasource. */ -@Experimental -public class MySqlDialect implements JdbcDataSourceDialect { - - private static final String QUOTED_CHARACTER = "`"; - - private static final long serialVersionUID = 1L; - private final MySqlSourceConfig sourceConfig; - private transient Tables.TableFilter filters; - private transient MySqlSchema mySqlSchema; - - public MySqlDialect(MySqlSourceConfig sourceConfig) { - this.sourceConfig = sourceConfig; - this.filters = sourceConfig.getTableFilters().dataCollectionFilter(); - } - - @Override - public String getName() { - return "MySQL"; - } - - public JdbcConnection openJdbcConnection(JdbcSourceConfig sourceConfig) { - JdbcConnection jdbc = - new JdbcConnection( - JdbcConfiguration.adapt(sourceConfig.getDbzConfiguration()), - new JdbcConnectionFactory(sourceConfig, getPooledDataSourceFactory()), - QUOTED_CHARACTER, - QUOTED_CHARACTER); - try { - jdbc.connect(); - } catch (Exception e) { - throw new FlinkRuntimeException(e); - } - return jdbc; - } - - @Override - public Offset displayCurrentOffset(JdbcSourceConfig sourceConfig) { - try (JdbcConnection jdbcConnection = openJdbcConnection(sourceConfig)) { - return MySqlConnectionUtils.currentBinlogOffset(jdbcConnection); - } catch (Exception e) { - throw new FlinkRuntimeException("Read the binlog offset error", e); - } - } - - @Override - public boolean isDataCollectionIdCaseSensitive(JdbcSourceConfig sourceConfig) { - try (JdbcConnection jdbcConnection = openJdbcConnection(sourceConfig)) { - return MySqlConnectionUtils.isTableIdCaseSensitive(jdbcConnection); - } catch (SQLException e) { - throw new FlinkRuntimeException("Error reading MySQL variables: " + e.getMessage(), e); - } - } - - @Override - public ChunkSplitter createChunkSplitter(JdbcSourceConfig sourceConfig) { - return new MySqlChunkSplitter(sourceConfig, this); - } - - @Override - public JdbcConnectionPoolFactory getPooledDataSourceFactory() { - return new MysqlPooledDataSourceFactory(); - } - - @Override - public List discoverDataCollections(JdbcSourceConfig sourceConfig) { - MySqlSourceConfig mySqlSourceConfig = (MySqlSourceConfig) sourceConfig; - try (JdbcConnection jdbcConnection = openJdbcConnection(sourceConfig)) { - return TableDiscoveryUtils.listTables( - jdbcConnection, mySqlSourceConfig.getTableFilters()); - } catch (SQLException e) { - throw new FlinkRuntimeException("Error to discover tables: " + e.getMessage(), e); - } - } - - @Override - public Map discoverDataCollectionSchemas(JdbcSourceConfig sourceConfig) { - final List capturedTableIds = discoverDataCollections(sourceConfig); - - try (MySqlConnection jdbc = - MySqlConnectionUtils.createMySqlConnection(sourceConfig.getDbzConfiguration())) { - // fetch table schemas - Map tableSchemas = new HashMap<>(); - for (TableId tableId : capturedTableIds) { - TableChanges.TableChange tableSchema = queryTableSchema(jdbc, tableId); - tableSchemas.put(tableId, tableSchema); - } - return tableSchemas; - } catch (Exception e) { - throw new FlinkRuntimeException( - "Error to discover table schemas: " + e.getMessage(), e); - } - } - - @Override - public TableChange queryTableSchema(JdbcConnection jdbc, TableId tableId) { - if (mySqlSchema == null) { - mySqlSchema = - new MySqlSchema(sourceConfig, isDataCollectionIdCaseSensitive(sourceConfig)); - } - return mySqlSchema.getTableSchema(jdbc, tableId); - } - - @Override - public MySqlSourceFetchTaskContext createFetchTaskContext(JdbcSourceConfig taskSourceConfig) { - final MySqlConnection jdbcConnection = - MySqlConnectionUtils.createMySqlConnection(taskSourceConfig.getDbzConfiguration()); - final BinaryLogClient binaryLogClient = - MySqlConnectionUtils.createBinaryClient(taskSourceConfig.getDbzConfiguration()); - return new MySqlSourceFetchTaskContext( - taskSourceConfig, this, jdbcConnection, binaryLogClient); - } - - @Override - public FetchTask createFetchTask(SourceSplitBase sourceSplitBase) { - if (sourceSplitBase.isSnapshotSplit()) { - return new MySqlScanFetchTask(sourceSplitBase.asSnapshotSplit()); - } else { - return new MySqlStreamFetchTask(sourceSplitBase.asStreamSplit()); - } - } - - @Override - public boolean isIncludeDataCollection(JdbcSourceConfig sourceConfig, TableId tableId) { - if (filters == null) { - this.filters = sourceConfig.getTableFilters().dataCollectionFilter(); - } - - return filters.isIncluded(tableId); - } -} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/experimental/MySqlSourceBuilder.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/experimental/MySqlSourceBuilder.java deleted file mode 100644 index 2e7346b27dc..00000000000 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/experimental/MySqlSourceBuilder.java +++ /dev/null @@ -1,261 +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.connectors.base.experimental; - -import org.apache.flink.cdc.common.annotation.Experimental; -import org.apache.flink.cdc.connectors.base.experimental.config.MySqlSourceConfigFactory; -import org.apache.flink.cdc.connectors.base.experimental.offset.BinlogOffsetFactory; -import org.apache.flink.cdc.connectors.base.options.StartupOptions; -import org.apache.flink.cdc.connectors.base.source.IncrementalSource; -import org.apache.flink.cdc.connectors.base.source.jdbc.JdbcIncrementalSource; -import org.apache.flink.cdc.debezium.DebeziumDeserializationSchema; - -import java.time.Duration; -import java.util.Properties; - -import static org.apache.flink.util.Preconditions.checkNotNull; - -/** - * The builder class for {@link MySqlIncrementalSource} to make it easier for the users to construct - * a {@link MySqlIncrementalSource}. - * - *

{@code
- * MySqlIncrementalSource
- *     .builder()
- *     .hostname("localhost")
- *     .port(3306)
- *     .databaseList("mydb")
- *     .tableList("mydb.users")
- *     .username(username)
- *     .password(password)
- *     .serverId(5400)
- *     .deserializer(new JsonDebeziumDeserializationSchema())
- *     .build();
- * }
- * - *

Check the Java docs of each individual method to learn more about the settings to build a - * {@link MySqlIncrementalSource}. - */ -@Experimental -public class MySqlSourceBuilder { - private final MySqlSourceConfigFactory configFactory = new MySqlSourceConfigFactory(); - private BinlogOffsetFactory offsetFactory; - private MySqlDialect dialect; - private DebeziumDeserializationSchema deserializer; - - public MySqlSourceBuilder hostname(String hostname) { - this.configFactory.hostname(hostname); - return this; - } - - /** Integer port number of the MySQL database server. */ - public MySqlSourceBuilder port(int port) { - this.configFactory.port(port); - return this; - } - - /** - * An required list of regular expressions that match database names to be monitored; any - * database name not included in the whitelist will be excluded from monitoring. - */ - public MySqlSourceBuilder databaseList(String... databaseList) { - this.configFactory.databaseList(databaseList); - return this; - } - - /** - * An required list of regular expressions that match fully-qualified table identifiers for - * tables to be monitored; any table not included in the list will be excluded from monitoring. - * Each identifier is of the form {@code .}. - */ - public MySqlSourceBuilder tableList(String... tableList) { - this.configFactory.tableList(tableList); - return this; - } - - /** Name of the MySQL database to use when connecting to the MySQL database server. */ - public MySqlSourceBuilder username(String username) { - this.configFactory.username(username); - return this; - } - - /** Password to use when connecting to the MySQL database server. */ - public MySqlSourceBuilder password(String password) { - this.configFactory.password(password); - return this; - } - - /** - * A numeric ID or a numeric ID range of this database client, The numeric ID syntax is like - * '5400', the numeric ID range syntax is like '5400-5408', The numeric ID range syntax is - * required when 'scan.incremental.snapshot.enabled' enabled. Every ID must be unique across all - * currently-running database processes in the MySQL cluster. This connector joins the MySQL - * cluster as another server (with this unique ID) so it can read the binlog. By default, a - * random number is generated between 5400 and 6400, though we recommend setting an explicit - * value." - */ - public MySqlSourceBuilder serverId(String serverId) { - this.configFactory.serverId(serverId); - return this; - } - - /** - * The session time zone in database server, e.g. "America/Los_Angeles". It controls how the - * TIMESTAMP type in MYSQL converted to STRING. See more - * https://debezium.io/documentation/reference/1.9/connectors/mysql.html#mysql-temporal-types - */ - public MySqlSourceBuilder serverTimeZone(String timeZone) { - this.configFactory.serverTimeZone(timeZone); - return this; - } - - /** - * The split size (number of rows) of table snapshot, captured tables are split into multiple - * splits when read the snapshot of table. - */ - public MySqlSourceBuilder splitSize(int splitSize) { - this.configFactory.splitSize(splitSize); - return this; - } - - /** - * The group size of split meta, if the meta size exceeds the group size, the meta will be - * divided into multiple groups. - */ - public MySqlSourceBuilder splitMetaGroupSize(int splitMetaGroupSize) { - this.configFactory.splitMetaGroupSize(splitMetaGroupSize); - return this; - } - - /** - * The upper bound of split key evenly distribution factor, the factor is used to determine - * whether the table is evenly distribution or not. - */ - public MySqlSourceBuilder distributionFactorUpper(double distributionFactorUpper) { - this.configFactory.distributionFactorUpper(distributionFactorUpper); - return this; - } - - /** - * The lower bound of split key evenly distribution factor, the factor is used to determine - * whether the table is evenly distribution or not. - */ - public MySqlSourceBuilder distributionFactorLower(double distributionFactorLower) { - this.configFactory.distributionFactorLower(distributionFactorLower); - return this; - } - - /** The maximum fetch size for per poll when read table snapshot. */ - public MySqlSourceBuilder fetchSize(int fetchSize) { - this.configFactory.fetchSize(fetchSize); - return this; - } - - /** - * The maximum time that the connector should wait after trying to connect to the MySQL database - * server before timing out. - */ - public MySqlSourceBuilder connectTimeout(Duration connectTimeout) { - this.configFactory.connectTimeout(connectTimeout); - return this; - } - - /** The max retry times to get connection. */ - public MySqlSourceBuilder connectMaxRetries(int connectMaxRetries) { - this.configFactory.connectMaxRetries(connectMaxRetries); - return this; - } - - /** The connection pool size. */ - public MySqlSourceBuilder connectionPoolSize(int connectionPoolSize) { - this.configFactory.connectionPoolSize(connectionPoolSize); - return this; - } - - /** Whether the {@link MySqlIncrementalSource} should output the schema changes or not. */ - public MySqlSourceBuilder includeSchemaChanges(boolean includeSchemaChanges) { - this.configFactory.includeSchemaChanges(includeSchemaChanges); - return this; - } - - /** Whether the {@link MySqlIncrementalSource} should scan the newly added tables or not. */ - public MySqlSourceBuilder scanNewlyAddedTableEnabled(boolean scanNewlyAddedTableEnabled) { - this.configFactory.scanNewlyAddedTableEnabled(scanNewlyAddedTableEnabled); - return this; - } - - /** Specifies the startup options. */ - public MySqlSourceBuilder startupOptions(StartupOptions startupOptions) { - this.configFactory.startupOptions(startupOptions); - return this; - } - - /** The Debezium MySQL connector properties. For example, "snapshot.mode". */ - public MySqlSourceBuilder debeziumProperties(Properties properties) { - this.configFactory.debeziumProperties(properties); - return this; - } - - /** - * Whether to close idle readers at the end of the snapshot phase. This feature depends on - * FLIP-147: Support Checkpoints After Tasks Finished. The flink version is required to be - * greater than or equal to 1.14, and the configuration - * 'execution.checkpointing.checkpoints-after-tasks-finish.enabled' needs to be set to - * true. - * - *

See more - * https://cwiki.apache.org/confluence/display/FLINK/FLIP-147%3A+Support+Checkpoints+After+Tasks+Finished. - */ - public MySqlSourceBuilder closeIdleReaders(boolean closeIdleReaders) { - this.configFactory.closeIdleReaders(closeIdleReaders); - return this; - } - - /** - * The deserializer used to convert from consumed {@link - * org.apache.kafka.connect.source.SourceRecord}. - */ - public MySqlSourceBuilder deserializer(DebeziumDeserializationSchema deserializer) { - this.deserializer = deserializer; - return this; - } - - /** - * Build the {@link MySqlIncrementalSource}. - * - * @return a MySqlParallelSource with the settings made for this builder. - */ - public MySqlIncrementalSource build() { - this.offsetFactory = new BinlogOffsetFactory(); - this.dialect = new MySqlDialect(configFactory.create(0)); - return new MySqlIncrementalSource<>( - configFactory, checkNotNull(deserializer), offsetFactory, dialect); - } - - /** The {@link IncrementalSource} implementation for MySQL. */ - public static class MySqlIncrementalSource extends JdbcIncrementalSource { - - public MySqlIncrementalSource( - MySqlSourceConfigFactory configFactory, - DebeziumDeserializationSchema deserializationSchema, - BinlogOffsetFactory offsetFactory, - MySqlDialect dataSourceDialect) { - super(configFactory, deserializationSchema, offsetFactory, dataSourceDialect); - } - } -} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/experimental/config/MySqlSourceConfigFactory.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/experimental/config/MySqlSourceConfigFactory.java deleted file mode 100644 index cebec367cab..00000000000 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/experimental/config/MySqlSourceConfigFactory.java +++ /dev/null @@ -1,135 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.cdc.connectors.base.experimental.config; - -import org.apache.flink.cdc.connectors.base.config.JdbcSourceConfigFactory; -import org.apache.flink.cdc.connectors.base.experimental.EmbeddedFlinkDatabaseHistory; - -import io.debezium.config.Configuration; -import io.debezium.connector.mysql.MySqlConnectorConfig; - -import java.util.Properties; -import java.util.UUID; - -import static org.apache.flink.cdc.connectors.base.utils.EnvironmentUtils.checkSupportCheckpointsAfterTasksFinished; -import static org.apache.flink.util.Preconditions.checkNotNull; - -/** A factory to initialize {@link MySqlSourceConfig}. */ -public class MySqlSourceConfigFactory extends JdbcSourceConfigFactory { - - private ServerIdRange serverIdRange; - - /** - * A numeric ID or a numeric ID range of this database client, The numeric ID syntax is like - * '5400', the numeric ID range syntax is like '5400-5408', The numeric ID range syntax is - * required when 'scan.incremental.snapshot.enabled' enabled. Every ID must be unique across all - * currently-running database processes in the MySQL cluster. This connector joins the MySQL - * cluster as another server (with this unique ID) so it can read the binlog. By default, a - * random number is generated between 5400 and 6400, though we recommend setting an explicit - * value." - */ - public MySqlSourceConfigFactory serverId(String serverId) { - this.serverIdRange = ServerIdRange.from(serverId); - return this; - } - - /** Creates a new {@link MySqlSourceConfig} for the given subtask {@code subtaskId}. */ - public MySqlSourceConfig create(int subtaskId) { - checkSupportCheckpointsAfterTasksFinished(closeIdleReaders); - Properties props = new Properties(); - // hard code server name, because we don't need to distinguish it, docs: - // Logical name that identifies and provides a namespace for the particular - // MySQL database server/cluster being monitored. The logical name should be - // unique across all other connectors, since it is used as a prefix for all - // Kafka topic names emanating from this connector. - // Only alphanumeric characters and underscores should be used. - props.setProperty("database.server.name", "mysql_binlog_source"); - props.setProperty("database.hostname", checkNotNull(hostname)); - props.setProperty("database.user", checkNotNull(username)); - props.setProperty("database.password", checkNotNull(password)); - props.setProperty("database.port", String.valueOf(port)); - props.setProperty("database.fetchSize", String.valueOf(fetchSize)); - props.setProperty("database.responseBuffering", "adaptive"); - props.setProperty("database.serverTimezone", serverTimeZone); - // database history - props.setProperty( - "database.history", EmbeddedFlinkDatabaseHistory.class.getCanonicalName()); - props.setProperty("database.history.instance.name", UUID.randomUUID() + "_" + subtaskId); - props.setProperty("database.history.skip.unparseable.ddl", String.valueOf(true)); - props.setProperty("database.history.refer.ddl", String.valueOf(true)); - props.setProperty("connect.timeout.ms", String.valueOf(connectTimeout.toMillis())); - // the underlying debezium reader should always capture the schema changes and forward them. - // Note: the includeSchemaChanges parameter is used to control emitting the schema record, - // only DataStream API program need to emit the schema record, the Table API need not - props.setProperty("include.schema.changes", String.valueOf(true)); - // disable the offset flush totally - props.setProperty("offset.flush.interval.ms", String.valueOf(Long.MAX_VALUE)); - // disable tombstones - props.setProperty("tombstones.on.delete", String.valueOf(false)); - // debezium use "long" mode to handle unsigned bigint by default, - // but it'll cause lose of precise when the value is larger than 2^63, - // so use "precise" mode to avoid it. - props.put("bigint.unsigned.handling.mode", "precise"); - - if (serverIdRange != null) { - props.setProperty("database.server.id.range", String.valueOf(serverIdRange)); - int serverId = serverIdRange.getServerId(subtaskId); - props.setProperty("database.server.id", String.valueOf(serverId)); - } - if (databaseList != null) { - props.setProperty("database.include.list", String.join(",", databaseList)); - } - if (tableList != null) { - props.setProperty("table.include.list", String.join(",", tableList)); - } - if (serverTimeZone != null) { - props.setProperty("database.serverTimezone", serverTimeZone); - } - - // override the user-defined debezium properties - if (dbzProperties != null) { - dbzProperties.forEach(props::put); - } - - Configuration dbzConfiguration = Configuration.from(props); - String driverClassName = dbzConfiguration.getString(MySqlConnectorConfig.JDBC_DRIVER); - return new MySqlSourceConfig( - startupOptions, - databaseList, - tableList, - splitSize, - splitMetaGroupSize, - distributionFactorUpper, - distributionFactorLower, - includeSchemaChanges, - closeIdleReaders, - props, - dbzConfiguration, - driverClassName, - hostname, - port, - username, - password, - fetchSize, - serverTimeZone, - connectTimeout, - connectMaxRetries, - connectionPoolSize, - scanNewlyAddedTableEnabled); - } -} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/experimental/config/ServerIdRange.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/experimental/config/ServerIdRange.java deleted file mode 100644 index de7a7e0a94d..00000000000 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/experimental/config/ServerIdRange.java +++ /dev/null @@ -1,114 +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.connectors.base.experimental.config; - -import org.apache.flink.cdc.connectors.base.options.JdbcSourceOptions; - -import javax.annotation.Nullable; - -import java.io.Serializable; - -import static org.apache.flink.util.Preconditions.checkArgument; - -/** - * This class defines a range of server id. The boundaries of the range are inclusive. - * - * @see JdbcSourceOptions#SERVER_ID - */ -public class ServerIdRange implements Serializable { - private static final long serialVersionUID = 1L; - - /** Start of the range (inclusive). */ - private final int startServerId; - - /** End of the range (inclusive). */ - private final int endServerId; - - public ServerIdRange(int startServerId, int endServerId) { - this.startServerId = startServerId; - this.endServerId = endServerId; - } - - public int getStartServerId() { - return startServerId; - } - - public int getEndServerId() { - return endServerId; - } - - public int getServerId(int subTaskId) { - checkArgument(subTaskId >= 0, "Subtask ID %s shouldn't be a negative number.", subTaskId); - if (subTaskId > getNumberOfServerIds()) { - throw new IllegalArgumentException( - String.format( - "Subtask ID %s is out of server id range %s, " - + "please adjust the server id range to " - + "make the number of server id larger than " - + "the source parallelism.", - subTaskId, this)); - } - return startServerId + subTaskId; - } - - public int getNumberOfServerIds() { - return endServerId - startServerId + 1; - } - - @Override - public String toString() { - if (startServerId == endServerId) { - return String.valueOf(startServerId); - } else { - return startServerId + "-" + endServerId; - } - } - - /** - * Returns a {@link ServerIdRange} from a server id range string which likes '5400-5408' or a - * single server id likes '5400'. - */ - public static @Nullable ServerIdRange from(@Nullable String range) { - if (range == null) { - return null; - } - if (range.contains("-")) { - String[] idArray = range.split("-"); - if (idArray.length != 2) { - throw new IllegalArgumentException( - String.format( - "The server id range should be syntax like '5400-5500', but got: %s", - range)); - } - return new ServerIdRange( - parseServerId(idArray[0].trim()), parseServerId(idArray[1].trim())); - } else { - int serverId = parseServerId(range); - return new ServerIdRange(serverId, serverId); - } - } - - private static int parseServerId(String serverIdValue) { - try { - return Integer.parseInt(serverIdValue); - } catch (NumberFormatException e) { - throw new IllegalStateException( - String.format("The server id %s is not a valid numeric.", serverIdValue), e); - } - } -} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/experimental/fetch/MySqlScanFetchTask.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/experimental/fetch/MySqlScanFetchTask.java deleted file mode 100644 index 791415b4163..00000000000 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/experimental/fetch/MySqlScanFetchTask.java +++ /dev/null @@ -1,335 +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.connectors.base.experimental.fetch; - -import org.apache.flink.cdc.connectors.base.experimental.fetch.MySqlStreamFetchTask.MySqlBinlogSplitReadTask; -import org.apache.flink.cdc.connectors.base.experimental.utils.MySqlUtils; -import org.apache.flink.cdc.connectors.base.relational.JdbcSourceEventDispatcher; -import org.apache.flink.cdc.connectors.base.source.meta.split.SnapshotSplit; -import org.apache.flink.cdc.connectors.base.source.meta.split.StreamSplit; -import org.apache.flink.cdc.connectors.base.source.reader.external.AbstractScanFetchTask; - -import io.debezium.DebeziumException; -import io.debezium.config.Configuration; -import io.debezium.connector.mysql.MySqlConnection; -import io.debezium.connector.mysql.MySqlConnectorConfig; -import io.debezium.connector.mysql.MySqlDatabaseSchema; -import io.debezium.connector.mysql.MySqlOffsetContext; -import io.debezium.connector.mysql.MySqlPartition; -import io.debezium.heartbeat.Heartbeat; -import io.debezium.pipeline.EventDispatcher; -import io.debezium.pipeline.source.AbstractSnapshotChangeEventSource; -import io.debezium.pipeline.source.spi.ChangeEventSource; -import io.debezium.pipeline.source.spi.SnapshotProgressListener; -import io.debezium.pipeline.spi.ChangeRecordEmitter; -import io.debezium.pipeline.spi.SnapshotResult; -import io.debezium.relational.RelationalSnapshotChangeEventSource; -import io.debezium.relational.SnapshotChangeRecordEmitter; -import io.debezium.relational.Table; -import io.debezium.relational.TableId; -import io.debezium.util.Clock; -import io.debezium.util.ColumnUtils; -import io.debezium.util.Strings; -import io.debezium.util.Threads; -import org.apache.kafka.connect.errors.ConnectException; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.sql.PreparedStatement; -import java.sql.ResultSet; -import java.sql.SQLException; -import java.time.Duration; - -import static org.apache.flink.cdc.connectors.base.experimental.utils.MySqlConnectionUtils.createMySqlConnection; - -/** The task to work for fetching data of MySQL table snapshot split . */ -public class MySqlScanFetchTask extends AbstractScanFetchTask { - - public MySqlScanFetchTask(SnapshotSplit split) { - super(split); - } - - @Override - protected void executeDataSnapshot(Context context) throws Exception { - MySqlSourceFetchTaskContext sourceFetchContext = (MySqlSourceFetchTaskContext) context; - MySqlSnapshotSplitReadTask snapshotSplitReadTask = - new MySqlSnapshotSplitReadTask( - sourceFetchContext.getDbzConnectorConfig(), - sourceFetchContext.getOffsetContext(), - sourceFetchContext.getSnapshotChangeEventSourceMetrics(), - sourceFetchContext.getDatabaseSchema(), - sourceFetchContext.getConnection(), - sourceFetchContext.getDispatcher(), - snapshotSplit); - MysqlSnapshotSplitChangeEventSourceContext changeEventSourceContext = - new MysqlSnapshotSplitChangeEventSourceContext(); - - SnapshotResult snapshotResult = - snapshotSplitReadTask.execute( - changeEventSourceContext, - sourceFetchContext.getPartition(), - sourceFetchContext.getOffsetContext()); - - if (!snapshotResult.isCompletedOrSkipped()) { - taskRunning = false; - throw new IllegalStateException( - String.format("Read snapshot for mysql split %s fail", snapshotResult)); - } - } - - @Override - protected void executeBackfillTask(Context context, StreamSplit backfillStreamSplit) - throws Exception { - MySqlSourceFetchTaskContext sourceFetchContext = (MySqlSourceFetchTaskContext) context; - MySqlBinlogSplitReadTask backfillBinlogReadTask = - createBackfillBinlogReadTask(backfillStreamSplit, sourceFetchContext); - backfillBinlogReadTask.execute( - new MysqlSnapshotSplitChangeEventSourceContext(), - sourceFetchContext.getPartition(), - sourceFetchContext.getOffsetContext()); - } - - private MySqlBinlogSplitReadTask createBackfillBinlogReadTask( - StreamSplit backfillBinlogSplit, MySqlSourceFetchTaskContext context) { - // we should only capture events for the current table, - // otherwise, we may can't find corresponding schema - Configuration dezConf = - context.getSourceConfig() - .getDbzConfiguration() - .edit() - .with("table.include.list", snapshotSplit.getTableId().toString()) - // Disable heartbeat event in snapshot split fetcher - .with(Heartbeat.HEARTBEAT_INTERVAL, 0) - .build(); - // task to read binlog and backfill for current split - return new MySqlBinlogSplitReadTask( - new MySqlConnectorConfig(dezConf), - createMySqlConnection(context.getSourceConfig().getDbzConfiguration()), - context.getDispatcher(), - context.getErrorHandler(), - context.getTaskContext(), - context.getStreamingChangeEventSourceMetrics(), - backfillBinlogSplit); - } - - /** A wrapped task to fetch snapshot split of table. */ - public static class MySqlSnapshotSplitReadTask - extends AbstractSnapshotChangeEventSource { - - private static final Logger LOG = LoggerFactory.getLogger(MySqlSnapshotSplitReadTask.class); - - /** Interval for showing a log statement with the progress while scanning a single table. */ - private static final Duration LOG_INTERVAL = Duration.ofMillis(10_000); - - private final MySqlConnectorConfig connectorConfig; - private final MySqlDatabaseSchema databaseSchema; - private final MySqlConnection jdbcConnection; - private final JdbcSourceEventDispatcher dispatcher; - private final Clock clock; - private final SnapshotSplit snapshotSplit; - private final MySqlOffsetContext offsetContext; - private final SnapshotProgressListener snapshotProgressListener; - - public MySqlSnapshotSplitReadTask( - MySqlConnectorConfig connectorConfig, - MySqlOffsetContext previousOffset, - SnapshotProgressListener snapshotProgressListener, - MySqlDatabaseSchema databaseSchema, - MySqlConnection jdbcConnection, - JdbcSourceEventDispatcher dispatcher, - SnapshotSplit snapshotSplit) { - super(connectorConfig, snapshotProgressListener); - this.offsetContext = previousOffset; - this.connectorConfig = connectorConfig; - this.databaseSchema = databaseSchema; - this.jdbcConnection = jdbcConnection; - this.dispatcher = dispatcher; - this.clock = Clock.SYSTEM; - this.snapshotSplit = snapshotSplit; - this.snapshotProgressListener = snapshotProgressListener; - } - - @Override - public SnapshotResult execute( - ChangeEventSourceContext context, - MySqlPartition partition, - MySqlOffsetContext previousOffset) - throws InterruptedException { - SnapshottingTask snapshottingTask = getSnapshottingTask(partition, previousOffset); - final MySqlSnapshotContext ctx; - try { - ctx = prepare(partition); - } catch (Exception e) { - LOG.error("Failed to initialize snapshot context.", e); - throw new RuntimeException(e); - } - try { - return doExecute(context, previousOffset, ctx, snapshottingTask); - } catch (InterruptedException e) { - LOG.warn("Snapshot was interrupted before completion"); - throw e; - } catch (Exception t) { - throw new DebeziumException(t); - } - } - - @Override - protected SnapshotResult doExecute( - ChangeEventSourceContext context, - MySqlOffsetContext previousOffset, - SnapshotContext snapshotContext, - SnapshottingTask snapshottingTask) - throws Exception { - final MySqlSnapshotContext ctx = (MySqlSnapshotContext) snapshotContext; - ctx.offset = offsetContext; - createDataEvents(ctx, snapshotSplit.getTableId()); - - return SnapshotResult.completed(ctx.offset); - } - - @Override - protected SnapshottingTask getSnapshottingTask( - MySqlPartition partition, MySqlOffsetContext previousOffset) { - return new SnapshottingTask(false, true); - } - - @Override - protected MySqlSnapshotContext prepare(MySqlPartition partition) throws Exception { - return new MySqlSnapshotContext(partition); - } - - private static class MySqlSnapshotContext - extends RelationalSnapshotChangeEventSource.RelationalSnapshotContext< - MySqlPartition, MySqlOffsetContext> { - - public MySqlSnapshotContext(MySqlPartition partition) throws SQLException { - super(partition, ""); - } - } - - private void createDataEvents(MySqlSnapshotContext snapshotContext, TableId tableId) - throws Exception { - EventDispatcher.SnapshotReceiver snapshotReceiver = - dispatcher.getSnapshotChangeEventReceiver(); - LOG.debug("Snapshotting table {}", tableId); - createDataEventsForTable( - snapshotContext, snapshotReceiver, databaseSchema.tableFor(tableId)); - snapshotReceiver.completeSnapshot(); - } - - /** Dispatches the data change events for the records of a single table. */ - private void createDataEventsForTable( - MySqlSnapshotContext snapshotContext, - EventDispatcher.SnapshotReceiver snapshotReceiver, - Table table) - throws InterruptedException { - - long exportStart = clock.currentTimeInMillis(); - LOG.info( - "Exporting data from split '{}' of table {}", - snapshotSplit.splitId(), - table.id()); - - final String selectSql = - MySqlUtils.buildSplitScanQuery( - snapshotSplit.getTableId(), - snapshotSplit.getSplitKeyType(), - snapshotSplit.getSplitStart() == null, - snapshotSplit.getSplitEnd() == null); - LOG.info( - "For split '{}' of table {} using select statement: '{}'", - snapshotSplit.splitId(), - table.id(), - selectSql); - - try (PreparedStatement selectStatement = - MySqlUtils.readTableSplitDataStatement( - jdbcConnection, - selectSql, - snapshotSplit.getSplitStart() == null, - snapshotSplit.getSplitEnd() == null, - snapshotSplit.getSplitStart(), - snapshotSplit.getSplitEnd(), - snapshotSplit.getSplitKeyType().getFieldCount(), - connectorConfig.getQueryFetchSize()); - ResultSet rs = selectStatement.executeQuery()) { - - ColumnUtils.ColumnArray columnArray = ColumnUtils.toArray(rs, table); - long rows = 0; - Threads.Timer logTimer = getTableScanLogTimer(); - - while (rs.next()) { - rows++; - final Object[] row = - jdbcConnection.rowToArray(table, databaseSchema, rs, columnArray); - if (logTimer.expired()) { - long stop = clock.currentTimeInMillis(); - LOG.info( - "Exported {} records for split '{}' after {}", - rows, - snapshotSplit.splitId(), - Strings.duration(stop - exportStart)); - snapshotProgressListener.rowsScanned( - snapshotContext.partition, table.id(), rows); - logTimer = getTableScanLogTimer(); - } - dispatcher.dispatchSnapshotEvent( - snapshotContext.partition, - table.id(), - getChangeRecordEmitter(snapshotContext, table.id(), row), - snapshotReceiver); - } - LOG.info( - "Finished exporting {} records for split '{}', total duration '{}'", - rows, - snapshotSplit.splitId(), - Strings.duration(clock.currentTimeInMillis() - exportStart)); - } catch (SQLException e) { - throw new ConnectException("Snapshotting of table " + table.id() + " failed", e); - } - } - - protected ChangeRecordEmitter getChangeRecordEmitter( - MySqlSnapshotContext snapshotContext, TableId tableId, Object[] row) { - snapshotContext.offset.event(tableId, clock.currentTime()); - return new SnapshotChangeRecordEmitter<>( - snapshotContext.partition, snapshotContext.offset, row, clock); - } - - private Threads.Timer getTableScanLogTimer() { - return Threads.timer(clock, LOG_INTERVAL); - } - } - - /** - * The {@link ChangeEventSource.ChangeEventSourceContext} implementation for bounded binlog task - * of a snapshot split task. - */ - public class MysqlSnapshotSplitChangeEventSourceContext - implements ChangeEventSource.ChangeEventSourceContext { - - public void finished() { - taskRunning = false; - } - - @Override - public boolean isRunning() { - return taskRunning; - } - } -} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/experimental/fetch/MySqlSourceFetchTaskContext.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/experimental/fetch/MySqlSourceFetchTaskContext.java deleted file mode 100644 index 6422de4339d..00000000000 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/experimental/fetch/MySqlSourceFetchTaskContext.java +++ /dev/null @@ -1,364 +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.connectors.base.experimental.fetch; - -import org.apache.flink.cdc.connectors.base.config.JdbcSourceConfig; -import org.apache.flink.cdc.connectors.base.dialect.JdbcDataSourceDialect; -import org.apache.flink.cdc.connectors.base.experimental.EmbeddedFlinkDatabaseHistory; -import org.apache.flink.cdc.connectors.base.experimental.config.MySqlSourceConfig; -import org.apache.flink.cdc.connectors.base.experimental.handler.MySqlSchemaChangeEventHandler; -import org.apache.flink.cdc.connectors.base.experimental.offset.BinlogOffset; -import org.apache.flink.cdc.connectors.base.experimental.utils.MySqlUtils; -import org.apache.flink.cdc.connectors.base.relational.JdbcSourceEventDispatcher; -import org.apache.flink.cdc.connectors.base.source.meta.offset.Offset; -import org.apache.flink.cdc.connectors.base.source.meta.split.SourceSplitBase; -import org.apache.flink.cdc.connectors.base.source.reader.external.JdbcSourceFetchTaskContext; -import org.apache.flink.table.types.logical.RowType; - -import com.github.shyiko.mysql.binlog.BinaryLogClient; -import io.debezium.connector.AbstractSourceInfo; -import io.debezium.connector.base.ChangeEventQueue; -import io.debezium.connector.mysql.MySqlChangeEventSourceMetricsFactory; -import io.debezium.connector.mysql.MySqlConnection; -import io.debezium.connector.mysql.MySqlConnectorConfig; -import io.debezium.connector.mysql.MySqlDatabaseSchema; -import io.debezium.connector.mysql.MySqlErrorHandler; -import io.debezium.connector.mysql.MySqlOffsetContext; -import io.debezium.connector.mysql.MySqlPartition; -import io.debezium.connector.mysql.MySqlStreamingChangeEventSourceMetrics; -import io.debezium.connector.mysql.MySqlTaskContext; -import io.debezium.connector.mysql.MySqlTopicSelector; -import io.debezium.data.Envelope; -import io.debezium.pipeline.DataChangeEvent; -import io.debezium.pipeline.ErrorHandler; -import io.debezium.pipeline.metrics.SnapshotChangeEventSourceMetrics; -import io.debezium.pipeline.source.spi.EventMetadataProvider; -import io.debezium.pipeline.spi.OffsetContext; -import io.debezium.pipeline.spi.Offsets; -import io.debezium.relational.Table; -import io.debezium.relational.TableId; -import io.debezium.relational.Tables; -import io.debezium.schema.DataCollectionId; -import io.debezium.schema.TopicSelector; -import io.debezium.util.Collect; -import org.apache.kafka.connect.data.Struct; -import org.apache.kafka.connect.source.SourceRecord; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.time.Instant; -import java.util.List; -import java.util.Map; - -/** The context for fetch task that fetching data of snapshot split from MySQL data source. */ -public class MySqlSourceFetchTaskContext extends JdbcSourceFetchTaskContext { - - private static final Logger LOG = LoggerFactory.getLogger(MySqlSourceFetchTaskContext.class); - private final MySqlConnection connection; - private final BinaryLogClient binaryLogClient; - private final MySqlEventMetadataProvider metadataProvider; - - private MySqlDatabaseSchema databaseSchema; - private MySqlTaskContextImpl taskContext; - private MySqlOffsetContext offsetContext; - private SnapshotChangeEventSourceMetrics snapshotChangeEventSourceMetrics; - private MySqlStreamingChangeEventSourceMetrics streamingChangeEventSourceMetrics; - private TopicSelector topicSelector; - private JdbcSourceEventDispatcher dispatcher; - private MySqlPartition mySqlPartition; - private ChangeEventQueue queue; - private MySqlErrorHandler errorHandler; - - public MySqlSourceFetchTaskContext( - JdbcSourceConfig sourceConfig, - JdbcDataSourceDialect dataSourceDialect, - MySqlConnection connection, - BinaryLogClient binaryLogClient) { - super(sourceConfig, dataSourceDialect); - this.connection = connection; - this.binaryLogClient = binaryLogClient; - this.metadataProvider = new MySqlEventMetadataProvider(); - } - - @Override - public void configure(SourceSplitBase sourceSplitBase) { - // initial stateful objects - final MySqlConnectorConfig connectorConfig = getDbzConnectorConfig(); - final boolean tableIdCaseInsensitive = connection.isTableIdCaseSensitive(); - this.topicSelector = MySqlTopicSelector.defaultSelector(connectorConfig); - EmbeddedFlinkDatabaseHistory.registerHistory( - sourceConfig - .getDbzConfiguration() - .getString(EmbeddedFlinkDatabaseHistory.DATABASE_HISTORY_INSTANCE_NAME), - sourceSplitBase.getTableSchemas().values()); - this.databaseSchema = - MySqlUtils.createMySqlDatabaseSchema(connectorConfig, tableIdCaseInsensitive); - this.offsetContext = - loadStartingOffsetState( - new MySqlOffsetContext.Loader(connectorConfig), sourceSplitBase); - this.mySqlPartition = new MySqlPartition(connectorConfig.getLogicalName()); - - validateAndLoadDatabaseHistory(offsetContext, databaseSchema); - - this.taskContext = - new MySqlTaskContextImpl(connectorConfig, databaseSchema, binaryLogClient); - final int queueSize = - sourceSplitBase.isSnapshotSplit() - ? Integer.MAX_VALUE - : getSourceConfig().getDbzConnectorConfig().getMaxQueueSize(); - this.queue = - new ChangeEventQueue.Builder() - .pollInterval(connectorConfig.getPollInterval()) - .maxBatchSize(connectorConfig.getMaxBatchSize()) - .maxQueueSize(queueSize) - .maxQueueSizeInBytes(connectorConfig.getMaxQueueSizeInBytes()) - .loggingContextSupplier( - () -> - taskContext.configureLoggingContext( - "mysql-cdc-connector-task")) - // do not buffer any element, we use signal event - // .buffering() - .build(); - this.dispatcher = - new JdbcSourceEventDispatcher<>( - connectorConfig, - topicSelector, - databaseSchema, - queue, - connectorConfig.getTableFilters().dataCollectionFilter(), - DataChangeEvent::new, - metadataProvider, - schemaNameAdjuster, - new MySqlSchemaChangeEventHandler()); - - final MySqlChangeEventSourceMetricsFactory changeEventSourceMetricsFactory = - new MySqlChangeEventSourceMetricsFactory( - new MySqlStreamingChangeEventSourceMetrics( - taskContext, queue, metadataProvider)); - this.snapshotChangeEventSourceMetrics = - changeEventSourceMetricsFactory.getSnapshotMetrics( - taskContext, queue, metadataProvider); - this.streamingChangeEventSourceMetrics = - (MySqlStreamingChangeEventSourceMetrics) - changeEventSourceMetricsFactory.getStreamingMetrics( - taskContext, queue, metadataProvider); - this.errorHandler = new MySqlErrorHandler(connectorConfig, queue); - } - - @Override - public MySqlSourceConfig getSourceConfig() { - return (MySqlSourceConfig) sourceConfig; - } - - public MySqlConnection getConnection() { - return connection; - } - - public BinaryLogClient getBinaryLogClient() { - return binaryLogClient; - } - - @Override - public MySqlPartition getPartition() { - return mySqlPartition; - } - - public MySqlTaskContextImpl getTaskContext() { - return taskContext; - } - - @Override - public MySqlConnectorConfig getDbzConnectorConfig() { - return (MySqlConnectorConfig) super.getDbzConnectorConfig(); - } - - @Override - public MySqlOffsetContext getOffsetContext() { - return offsetContext; - } - - public SnapshotChangeEventSourceMetrics getSnapshotChangeEventSourceMetrics() { - return snapshotChangeEventSourceMetrics; - } - - public MySqlStreamingChangeEventSourceMetrics getStreamingChangeEventSourceMetrics() { - return streamingChangeEventSourceMetrics; - } - - @Override - public ErrorHandler getErrorHandler() { - return errorHandler; - } - - @Override - public MySqlDatabaseSchema getDatabaseSchema() { - return databaseSchema; - } - - @Override - public RowType getSplitType(Table table) { - return MySqlUtils.getSplitType(table); - } - - @Override - public JdbcSourceEventDispatcher getDispatcher() { - return dispatcher; - } - - @Override - public ChangeEventQueue getQueue() { - return queue; - } - - @Override - public Tables.TableFilter getTableFilter() { - return getDbzConnectorConfig().getTableFilters().dataCollectionFilter(); - } - - @Override - public Offset getStreamOffset(SourceRecord sourceRecord) { - return MySqlUtils.getBinlogPosition(sourceRecord); - } - - @Override - public void close() throws Exception { - connection.close(); - } - - /** Loads the connector's persistent offset (if present) via the given loader. */ - private MySqlOffsetContext loadStartingOffsetState( - OffsetContext.Loader loader, SourceSplitBase mySqlSplit) { - Offset offset = - mySqlSplit.isSnapshotSplit() - ? BinlogOffset.INITIAL_OFFSET - : mySqlSplit.asStreamSplit().getStartingOffset(); - - MySqlOffsetContext mySqlOffsetContext = loader.load(offset.getOffset()); - - if (!isBinlogAvailable(mySqlOffsetContext)) { - throw new IllegalStateException( - "The connector is trying to read binlog starting at " - + mySqlOffsetContext.getSourceInfo() - + ", but this is no longer " - + "available on the server. Reconfigure the connector to use a snapshot when needed."); - } - return mySqlOffsetContext; - } - - private boolean isBinlogAvailable(MySqlOffsetContext offset) { - String binlogFilename = - offset.getSourceInfo().getString(BinlogOffset.BINLOG_FILENAME_OFFSET_KEY); - if (binlogFilename == null) { - return true; // start at current position - } - if (binlogFilename.equals("")) { - return true; // start at beginning - } - - // Accumulate the available binlog filenames ... - List logNames = connection.availableBinlogFiles(); - - // And compare with the one we're supposed to use ... - boolean found = logNames.stream().anyMatch(binlogFilename::equals); - if (!found) { - LOG.info( - "Connector requires binlog file '{}', but MySQL only has {}", - binlogFilename, - String.join(", ", logNames)); - } else { - LOG.info("MySQL has the binlog file '{}' required by the connector", binlogFilename); - } - return found; - } - - private void validateAndLoadDatabaseHistory( - MySqlOffsetContext offset, MySqlDatabaseSchema schema) { - schema.initializeStorage(); - schema.recover(Offsets.of(mySqlPartition, offset)); - } - - /** A subclass implementation of {@link MySqlTaskContext} which reuses one BinaryLogClient. */ - public static class MySqlTaskContextImpl extends MySqlTaskContext { - - private final BinaryLogClient reusedBinaryLogClient; - - public MySqlTaskContextImpl( - MySqlConnectorConfig config, - MySqlDatabaseSchema schema, - BinaryLogClient reusedBinaryLogClient) { - super(config, schema); - this.reusedBinaryLogClient = reusedBinaryLogClient; - } - - @Override - public BinaryLogClient getBinaryLogClient() { - return reusedBinaryLogClient; - } - } - - /** Copied from debezium for accessing here. */ - public static class MySqlEventMetadataProvider implements EventMetadataProvider { - public static final String SERVER_ID_KEY = "server_id"; - - public static final String GTID_KEY = "gtid"; - public static final String BINLOG_FILENAME_OFFSET_KEY = "file"; - public static final String BINLOG_POSITION_OFFSET_KEY = "pos"; - public static final String BINLOG_ROW_IN_EVENT_OFFSET_KEY = "row"; - public static final String THREAD_KEY = "thread"; - public static final String QUERY_KEY = "query"; - - @Override - public Instant getEventTimestamp( - DataCollectionId source, OffsetContext offset, Object key, Struct value) { - if (value == null) { - return null; - } - final Struct sourceInfo = value.getStruct(Envelope.FieldName.SOURCE); - if (source == null) { - return null; - } - final Long timestamp = sourceInfo.getInt64(AbstractSourceInfo.TIMESTAMP_KEY); - return timestamp == null ? null : Instant.ofEpochMilli(timestamp); - } - - @Override - public Map getEventSourcePosition( - DataCollectionId source, OffsetContext offset, Object key, Struct value) { - if (value == null) { - return null; - } - final Struct sourceInfo = value.getStruct(Envelope.FieldName.SOURCE); - if (source == null) { - return null; - } - return Collect.hashMapOf( - BINLOG_FILENAME_OFFSET_KEY, - sourceInfo.getString(BINLOG_FILENAME_OFFSET_KEY), - BINLOG_POSITION_OFFSET_KEY, - Long.toString(sourceInfo.getInt64(BINLOG_POSITION_OFFSET_KEY)), - BINLOG_ROW_IN_EVENT_OFFSET_KEY, - Integer.toString(sourceInfo.getInt32(BINLOG_ROW_IN_EVENT_OFFSET_KEY))); - } - - @Override - public String getTransactionId( - DataCollectionId source, OffsetContext offset, Object key, Struct value) { - return ((MySqlOffsetContext) offset).getTransactionId(); - } - } -} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/experimental/fetch/MySqlStreamFetchTask.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/experimental/fetch/MySqlStreamFetchTask.java deleted file mode 100644 index cdfb02c6947..00000000000 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/experimental/fetch/MySqlStreamFetchTask.java +++ /dev/null @@ -1,188 +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.connectors.base.experimental.fetch; - -import org.apache.flink.cdc.connectors.base.experimental.offset.BinlogOffset; -import org.apache.flink.cdc.connectors.base.relational.JdbcSourceEventDispatcher; -import org.apache.flink.cdc.connectors.base.source.meta.split.SourceSplitBase; -import org.apache.flink.cdc.connectors.base.source.meta.split.StreamSplit; -import org.apache.flink.cdc.connectors.base.source.meta.wartermark.WatermarkKind; -import org.apache.flink.cdc.connectors.base.source.reader.external.FetchTask; - -import com.github.shyiko.mysql.binlog.event.Event; -import io.debezium.DebeziumException; -import io.debezium.connector.mysql.MySqlConnection; -import io.debezium.connector.mysql.MySqlConnectorConfig; -import io.debezium.connector.mysql.MySqlOffsetContext; -import io.debezium.connector.mysql.MySqlPartition; -import io.debezium.connector.mysql.MySqlStreamingChangeEventSource; -import io.debezium.connector.mysql.MySqlStreamingChangeEventSourceMetrics; -import io.debezium.connector.mysql.MySqlTaskContext; -import io.debezium.pipeline.ErrorHandler; -import io.debezium.pipeline.source.spi.ChangeEventSource; -import io.debezium.util.Clock; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.HashMap; -import java.util.Map; - -/** The task to work for fetching data of MySQL table stream split . */ -public class MySqlStreamFetchTask implements FetchTask { - - private final StreamSplit split; - private volatile boolean taskRunning = false; - - public MySqlStreamFetchTask(StreamSplit split) { - this.split = split; - } - - @Override - public void execute(Context context) throws Exception { - MySqlSourceFetchTaskContext sourceFetchContext = (MySqlSourceFetchTaskContext) context; - taskRunning = true; - MySqlBinlogSplitReadTask binlogSplitReadTask = - new MySqlBinlogSplitReadTask( - sourceFetchContext.getDbzConnectorConfig(), - sourceFetchContext.getConnection(), - sourceFetchContext.getDispatcher(), - sourceFetchContext.getErrorHandler(), - sourceFetchContext.getTaskContext(), - sourceFetchContext.getStreamingChangeEventSourceMetrics(), - split); - BinlogSplitChangeEventSourceContext changeEventSourceContext = - new BinlogSplitChangeEventSourceContext(); - binlogSplitReadTask.execute( - changeEventSourceContext, - sourceFetchContext.getPartition(), - sourceFetchContext.getOffsetContext()); - } - - @Override - public boolean isRunning() { - return taskRunning; - } - - @Override - public StreamSplit getSplit() { - return split; - } - - @Override - public void close() { - taskRunning = false; - } - - /** - * A wrapped task to read all binlog for table and also supports read bounded (from lowWatermark - * to highWatermark) binlog. - */ - public static class MySqlBinlogSplitReadTask extends MySqlStreamingChangeEventSource { - - private static final Logger LOG = LoggerFactory.getLogger(MySqlBinlogSplitReadTask.class); - private final StreamSplit binlogSplit; - private final JdbcSourceEventDispatcher dispatcher; - private final ErrorHandler errorHandler; - private ChangeEventSourceContext context; - - public MySqlBinlogSplitReadTask( - MySqlConnectorConfig connectorConfig, - MySqlConnection connection, - JdbcSourceEventDispatcher dispatcher, - ErrorHandler errorHandler, - MySqlTaskContext taskContext, - MySqlStreamingChangeEventSourceMetrics metrics, - StreamSplit binlogSplit) { - super( - connectorConfig, - connection, - dispatcher, - errorHandler, - Clock.SYSTEM, - taskContext, - metrics); - this.binlogSplit = binlogSplit; - this.dispatcher = dispatcher; - this.errorHandler = errorHandler; - } - - @Override - public void execute( - ChangeEventSourceContext context, - MySqlPartition partition, - MySqlOffsetContext offsetContext) - throws InterruptedException { - this.context = context; - super.execute(context, partition, offsetContext); - } - - @Override - protected void handleEvent( - MySqlPartition partition, MySqlOffsetContext offsetContext, Event event) { - super.handleEvent(partition, offsetContext, event); - // check do we need to stop for fetch binlog for snapshot split. - if (isBoundedRead()) { - final BinlogOffset currentBinlogOffset = - getBinlogPosition(offsetContext.getOffset()); - // reach the high watermark, the binlog fetcher should be finished - if (currentBinlogOffset.isAtOrAfter(binlogSplit.getEndingOffset())) { - // send binlog end event - try { - dispatcher.dispatchWatermarkEvent( - partition.getSourcePartition(), - binlogSplit, - currentBinlogOffset, - WatermarkKind.END); - } catch (InterruptedException e) { - LOG.error("Send signal event error.", e); - errorHandler.setProducerThrowable( - new DebeziumException("Error processing binlog signal event", e)); - } - // tell fetcher the binlog task finished - ((MySqlScanFetchTask.MysqlSnapshotSplitChangeEventSourceContext) context) - .finished(); - } - } - } - - private boolean isBoundedRead() { - return !BinlogOffset.NO_STOPPING_OFFSET.equals(binlogSplit.getEndingOffset()); - } - - public static BinlogOffset getBinlogPosition(Map offset) { - Map offsetStrMap = new HashMap<>(); - for (Map.Entry entry : offset.entrySet()) { - offsetStrMap.put( - entry.getKey(), - entry.getValue() == null ? null : entry.getValue().toString()); - } - return new BinlogOffset(offsetStrMap); - } - } - - /** - * The {@link ChangeEventSource.ChangeEventSourceContext} implementation for binlog split task. - */ - private class BinlogSplitChangeEventSourceContext - implements ChangeEventSource.ChangeEventSourceContext { - @Override - public boolean isRunning() { - return taskRunning; - } - } -} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/experimental/handler/MySqlSchemaChangeEventHandler.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/experimental/handler/MySqlSchemaChangeEventHandler.java deleted file mode 100644 index 45858321ae4..00000000000 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/experimental/handler/MySqlSchemaChangeEventHandler.java +++ /dev/null @@ -1,50 +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.connectors.base.experimental.handler; - -import org.apache.flink.cdc.connectors.base.relational.handler.SchemaChangeEventHandler; - -import io.debezium.schema.SchemaChangeEvent; -import org.apache.kafka.connect.data.Struct; - -import java.util.HashMap; -import java.util.Map; - -import static org.apache.flink.cdc.connectors.base.experimental.offset.BinlogOffset.BINLOG_FILENAME_OFFSET_KEY; -import static org.apache.flink.cdc.connectors.base.experimental.offset.BinlogOffset.BINLOG_POSITION_OFFSET_KEY; -import static org.apache.flink.cdc.connectors.base.experimental.offset.BinlogOffset.SERVER_ID_KEY; - -/** - * This MySqlSchemaChangeEventHandler helps to parse the source struct in SchemaChangeEvent and - * generate source info. - */ -public class MySqlSchemaChangeEventHandler implements SchemaChangeEventHandler { - - @Override - public Map parseSource(SchemaChangeEvent event) { - Map source = new HashMap<>(); - Struct sourceInfo = event.getSource(); - String fileName = sourceInfo.getString(BINLOG_FILENAME_OFFSET_KEY); - Long pos = sourceInfo.getInt64(BINLOG_POSITION_OFFSET_KEY); - Long serverId = sourceInfo.getInt64(SERVER_ID_KEY); - source.put(SERVER_ID_KEY, serverId); - source.put(BINLOG_FILENAME_OFFSET_KEY, fileName); - source.put(BINLOG_POSITION_OFFSET_KEY, pos); - return source; - } -} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/experimental/offset/BinlogOffset.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/experimental/offset/BinlogOffset.java deleted file mode 100644 index e4ecdd2a157..00000000000 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/experimental/offset/BinlogOffset.java +++ /dev/null @@ -1,217 +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.connectors.base.experimental.offset; - -import org.apache.flink.cdc.connectors.base.source.meta.offset.Offset; - -import io.debezium.connector.mysql.GtidSet; -import org.apache.commons.lang3.StringUtils; - -import javax.annotation.Nullable; - -import java.util.HashMap; -import java.util.Map; - -/** - * A structure describes a fine grained offset in a binlog event including binlog position and gtid - * set etc. - * - *

This structure can also be used to deal the binlog event in transaction, a transaction may - * contains multiple change events, and each change event may contain multiple rows. When restart - * from a specific {@link BinlogOffset}, we need to skip the processed change events and the - * processed rows. - */ -public class BinlogOffset extends Offset { - - private static final long serialVersionUID = 1L; - - public static final String BINLOG_FILENAME_OFFSET_KEY = "file"; - public static final String BINLOG_POSITION_OFFSET_KEY = "pos"; - public static final String EVENTS_TO_SKIP_OFFSET_KEY = "event"; - public static final String ROWS_TO_SKIP_OFFSET_KEY = "row"; - public static final String GTID_SET_KEY = "gtids"; - public static final String TIMESTAMP_KEY = "ts_sec"; - public static final String SERVER_ID_KEY = "server_id"; - - public static final BinlogOffset INITIAL_OFFSET = new BinlogOffset("", 0); - public static final BinlogOffset NO_STOPPING_OFFSET = new BinlogOffset("", Long.MIN_VALUE); - - public BinlogOffset(Map offset) { - this.offset = offset; - } - - public BinlogOffset(String filename, long position) { - this(filename, position, 0L, 0L, 0L, null, null); - } - - public BinlogOffset(long binlogEpochSecs) { - this(null, 0L, 0L, 0L, binlogEpochSecs, null, null); - } - - public BinlogOffset( - String filename, - long position, - long restartSkipEvents, - long restartSkipRows, - long binlogEpochSecs, - @Nullable String restartGtidSet, - @Nullable Integer serverId) { - Map offsetMap = new HashMap<>(); - offsetMap.put(BINLOG_FILENAME_OFFSET_KEY, filename); - offsetMap.put(BINLOG_POSITION_OFFSET_KEY, String.valueOf(position)); - offsetMap.put(EVENTS_TO_SKIP_OFFSET_KEY, String.valueOf(restartSkipEvents)); - offsetMap.put(ROWS_TO_SKIP_OFFSET_KEY, String.valueOf(restartSkipRows)); - offsetMap.put(TIMESTAMP_KEY, String.valueOf(binlogEpochSecs)); - if (restartGtidSet != null) { - offsetMap.put(GTID_SET_KEY, restartGtidSet); - } - if (serverId != null) { - offsetMap.put(SERVER_ID_KEY, String.valueOf(serverId)); - } - this.offset = offsetMap; - } - - public String getFilename() { - return offset.get(BINLOG_FILENAME_OFFSET_KEY); - } - - public long getPosition() { - return longOffsetValue(offset, BINLOG_POSITION_OFFSET_KEY); - } - - public long getRestartSkipEvents() { - return longOffsetValue(offset, EVENTS_TO_SKIP_OFFSET_KEY); - } - - public long getRestartSkipRows() { - return longOffsetValue(offset, ROWS_TO_SKIP_OFFSET_KEY); - } - - public String getGtidSet() { - return offset.get(GTID_SET_KEY); - } - - public long getTimestamp() { - return longOffsetValue(offset, TIMESTAMP_KEY); - } - - public Long getServerId() { - return longOffsetValue(offset, SERVER_ID_KEY); - } - - /** - * This method is inspired by {@link io.debezium.relational.history.HistoryRecordComparator}. - */ - @Override - public int compareTo(Offset offset) { - BinlogOffset that = (BinlogOffset) offset; - // the NO_STOPPING_OFFSET is the max offset - if (NO_STOPPING_OFFSET.equals(that) && NO_STOPPING_OFFSET.equals(this)) { - return 0; - } - if (NO_STOPPING_OFFSET.equals(this)) { - return 1; - } - if (NO_STOPPING_OFFSET.equals(that)) { - return -1; - } - - String gtidSetStr = this.getGtidSet(); - String targetGtidSetStr = that.getGtidSet(); - if (StringUtils.isNotEmpty(targetGtidSetStr)) { - // The target offset uses GTIDs, so we ideally compare using GTIDs ... - if (StringUtils.isNotEmpty(gtidSetStr)) { - // Both have GTIDs, so base the comparison entirely on the GTID sets. - GtidSet gtidSet = new GtidSet(gtidSetStr); - GtidSet targetGtidSet = new GtidSet(targetGtidSetStr); - if (gtidSet.equals(targetGtidSet)) { - long restartSkipEvents = this.getRestartSkipEvents(); - long targetRestartSkipEvents = that.getRestartSkipEvents(); - return Long.compare(restartSkipEvents, targetRestartSkipEvents); - } - // The GTIDs are not an exact match, so figure out if this is a subset of the target - // offset - // ... - return gtidSet.isContainedWithin(targetGtidSet) ? -1 : 1; - } - // The target offset did use GTIDs while this did not use GTIDs. So, we assume - // that this offset is older since GTIDs are often enabled but rarely disabled. - // And if they are disabled, - // it is likely that this offset would not include GTIDs as we would be trying - // to read the binlog of a - // server that no longer has GTIDs. And if they are enabled, disabled, and re-enabled, - // per - // https://dev.mysql.com/doc/refman/5.7/en/replication-gtids-failover.html all properly - // configured slaves that - // use GTIDs should always have the complete set of GTIDs copied from the master, in - // which case - // again we know that this offset not having GTIDs is before the target offset ... - return -1; - } else if (StringUtils.isNotEmpty(gtidSetStr)) { - // This offset has a GTID but the target offset does not, so per the previous paragraph - // we - // assume that previous - // is not at or before ... - return 1; - } - - // Both offsets are missing GTIDs. Look at the servers ... - long serverId = this.getServerId(); - long targetServerId = that.getServerId(); - - if (serverId != targetServerId) { - // These are from different servers, and their binlog coordinates are not related. So - // the only thing we can do - // is compare timestamps, and we have to assume that the server timestamps can be - // compared ... - long timestamp = this.getTimestamp(); - long targetTimestamp = that.getTimestamp(); - return Long.compare(timestamp, targetTimestamp); - } - - // First compare the MySQL binlog filenames - if (this.getFilename().compareToIgnoreCase(that.getFilename()) != 0) { - return this.getFilename().compareToIgnoreCase(that.getFilename()); - } - - // The filenames are the same, so compare the positions - if (this.getPosition() != that.getPosition()) { - return Long.compare(this.getPosition(), that.getPosition()); - } - - // The positions are the same, so compare the completed events in the transaction ... - if (this.getRestartSkipEvents() != that.getRestartSkipEvents()) { - return Long.compare(this.getRestartSkipEvents(), that.getRestartSkipEvents()); - } - - // The completed events are the same, so compare the row number ... - return Long.compare(this.getRestartSkipRows(), that.getRestartSkipRows()); - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (!(o instanceof BinlogOffset)) { - return false; - } - BinlogOffset that = (BinlogOffset) o; - return offset.equals(that.offset); - } -} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/experimental/offset/BinlogOffsetFactory.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/experimental/offset/BinlogOffsetFactory.java deleted file mode 100644 index 1fe782169cd..00000000000 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/experimental/offset/BinlogOffsetFactory.java +++ /dev/null @@ -1,60 +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.connectors.base.experimental.offset; - -import org.apache.flink.cdc.connectors.base.source.meta.offset.Offset; -import org.apache.flink.cdc.connectors.base.source.meta.offset.OffsetFactory; -import org.apache.flink.util.FlinkRuntimeException; - -import java.util.Map; - -/** An offset factory class create {@link BinlogOffset} instance. */ -public class BinlogOffsetFactory extends OffsetFactory { - - public BinlogOffsetFactory() {} - - @Override - public Offset newOffset(Map offset) { - return new BinlogOffset(offset); - } - - @Override - public Offset newOffset(String filename, Long position) { - return new BinlogOffset(filename, position); - } - - @Override - public Offset newOffset(Long position) { - throw new FlinkRuntimeException("not supported create new Offset by Long position."); - } - - @Override - public Offset createTimestampOffset(long timestampMillis) { - return new BinlogOffset(timestampMillis / 1000); - } - - @Override - public Offset createInitialOffset() { - return BinlogOffset.INITIAL_OFFSET; - } - - @Override - public Offset createNoStoppingOffset() { - return BinlogOffset.NO_STOPPING_OFFSET; - } -} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/experimental/utils/MySqlConnectionUtils.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/experimental/utils/MySqlConnectionUtils.java deleted file mode 100644 index 1a1903baa14..00000000000 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/experimental/utils/MySqlConnectionUtils.java +++ /dev/null @@ -1,164 +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.connectors.base.experimental.utils; - -import org.apache.flink.cdc.connectors.base.experimental.offset.BinlogOffset; -import org.apache.flink.util.FlinkRuntimeException; - -import com.github.shyiko.mysql.binlog.BinaryLogClient; -import io.debezium.config.Configuration; -import io.debezium.connector.mysql.MySqlConnection; -import io.debezium.connector.mysql.MySqlConnectorConfig; -import io.debezium.connector.mysql.MySqlDatabaseSchema; -import io.debezium.connector.mysql.MySqlSystemVariables; -import io.debezium.connector.mysql.MySqlTopicSelector; -import io.debezium.connector.mysql.MySqlValueConverters; -import io.debezium.jdbc.JdbcConnection; -import io.debezium.jdbc.JdbcValueConverters; -import io.debezium.jdbc.TemporalPrecisionMode; -import io.debezium.relational.TableId; -import io.debezium.schema.TopicSelector; -import io.debezium.util.SchemaNameAdjuster; - -import java.sql.SQLException; -import java.util.HashMap; -import java.util.Map; - -/** MySQL connection Utilities. */ -public class MySqlConnectionUtils { - - /** Creates a new {@link MySqlConnection}, but not open the connection. */ - public static MySqlConnection createMySqlConnection(Configuration dbzConfiguration) { - return new MySqlConnection( - new MySqlConnection.MySqlConnectionConfiguration(dbzConfiguration)); - } - - /** Creates a new {@link BinaryLogClient} for consuming mysql binlog. */ - public static BinaryLogClient createBinaryClient(Configuration dbzConfiguration) { - final MySqlConnectorConfig connectorConfig = new MySqlConnectorConfig(dbzConfiguration); - return new BinaryLogClient( - connectorConfig.hostname(), - connectorConfig.port(), - connectorConfig.username(), - connectorConfig.password()); - } - - /** Creates a new {@link MySqlDatabaseSchema} to monitor the latest MySql database schemas. */ - public static MySqlDatabaseSchema createMySqlDatabaseSchema( - MySqlConnectorConfig dbzMySqlConfig, boolean isTableIdCaseSensitive) { - TopicSelector topicSelector = MySqlTopicSelector.defaultSelector(dbzMySqlConfig); - SchemaNameAdjuster schemaNameAdjuster = SchemaNameAdjuster.create(); - MySqlValueConverters valueConverters = getValueConverters(dbzMySqlConfig); - return new MySqlDatabaseSchema( - dbzMySqlConfig, - valueConverters, - topicSelector, - schemaNameAdjuster, - isTableIdCaseSensitive); - } - - /** Fetch current binlog offsets in MySql Server. */ - public static BinlogOffset currentBinlogOffset(JdbcConnection jdbc) { - final String showMasterStmt = "SHOW MASTER STATUS"; - try { - return jdbc.queryAndMap( - showMasterStmt, - rs -> { - if (rs.next()) { - final String binlogFilename = rs.getString(1); - final long binlogPosition = rs.getLong(2); - final String gtidSet = - rs.getMetaData().getColumnCount() > 4 ? rs.getString(5) : null; - return new BinlogOffset( - binlogFilename, binlogPosition, 0L, 0, 0, gtidSet, null); - } else { - throw new FlinkRuntimeException( - "Cannot read the binlog filename and position via '" - + showMasterStmt - + "'. Make sure your server is correctly configured"); - } - }); - } catch (SQLException e) { - throw new FlinkRuntimeException( - "Cannot read the binlog filename and position via '" - + showMasterStmt - + "'. Make sure your server is correctly configured", - e); - } - } - - // -------------------------------------------------------------------------------------------- - - private static MySqlValueConverters getValueConverters(MySqlConnectorConfig dbzMySqlConfig) { - TemporalPrecisionMode timePrecisionMode = dbzMySqlConfig.getTemporalPrecisionMode(); - JdbcValueConverters.DecimalMode decimalMode = dbzMySqlConfig.getDecimalMode(); - String bigIntUnsignedHandlingModeStr = - dbzMySqlConfig - .getConfig() - .getString(MySqlConnectorConfig.BIGINT_UNSIGNED_HANDLING_MODE); - MySqlConnectorConfig.BigIntUnsignedHandlingMode bigIntUnsignedHandlingMode = - MySqlConnectorConfig.BigIntUnsignedHandlingMode.parse( - bigIntUnsignedHandlingModeStr); - JdbcValueConverters.BigIntUnsignedMode bigIntUnsignedMode = - bigIntUnsignedHandlingMode.asBigIntUnsignedMode(); - - boolean timeAdjusterEnabled = - dbzMySqlConfig.getConfig().getBoolean(MySqlConnectorConfig.ENABLE_TIME_ADJUSTER); - return new MySqlValueConverters( - decimalMode, - timePrecisionMode, - bigIntUnsignedMode, - dbzMySqlConfig.binaryHandlingMode(), - timeAdjusterEnabled ? MySqlValueConverters::adjustTemporal : x -> x, - MySqlValueConverters::defaultParsingErrorHandler); - } - - public static boolean isTableIdCaseSensitive(JdbcConnection connection) { - return !"0" - .equals( - readMySqlSystemVariables(connection) - .get(MySqlSystemVariables.LOWER_CASE_TABLE_NAMES)); - } - - public static Map readMySqlSystemVariables(JdbcConnection connection) { - // Read the system variables from the MySQL instance and get the current database name ... - return querySystemVariables(connection, "SHOW VARIABLES"); - } - - private static Map querySystemVariables( - JdbcConnection connection, String statement) { - final Map variables = new HashMap<>(); - try { - connection.query( - statement, - rs -> { - while (rs.next()) { - String varName = rs.getString(1); - String value = rs.getString(2); - if (varName != null && value != null) { - variables.put(varName, value); - } - } - }); - } catch (SQLException e) { - throw new FlinkRuntimeException("Error reading MySQL variables: " + e.getMessage(), e); - } - - return variables; - } -} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/experimental/utils/MySqlSchema.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/experimental/utils/MySqlSchema.java deleted file mode 100644 index 8954007bbae..00000000000 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/experimental/utils/MySqlSchema.java +++ /dev/null @@ -1,110 +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.connectors.base.experimental.utils; - -import org.apache.flink.cdc.connectors.base.experimental.config.MySqlSourceConfig; -import org.apache.flink.util.FlinkRuntimeException; - -import io.debezium.connector.mysql.MySqlConnectorConfig; -import io.debezium.connector.mysql.MySqlDatabaseSchema; -import io.debezium.connector.mysql.MySqlOffsetContext; -import io.debezium.connector.mysql.MySqlPartition; -import io.debezium.jdbc.JdbcConnection; -import io.debezium.relational.TableId; -import io.debezium.relational.history.TableChanges; -import io.debezium.relational.history.TableChanges.TableChange; -import io.debezium.schema.SchemaChangeEvent; - -import java.sql.SQLException; -import java.time.Instant; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -import static org.apache.flink.cdc.connectors.base.experimental.utils.MySqlConnectionUtils.createMySqlDatabaseSchema; - -/** A component used to get schema by table path. */ -public class MySqlSchema { - private static final String SHOW_CREATE_TABLE = "SHOW CREATE TABLE "; - private static final String DESC_TABLE = "DESC "; - - private final MySqlConnectorConfig connectorConfig; - private final MySqlDatabaseSchema databaseSchema; - private final Map schemasByTableId; - - public MySqlSchema(MySqlSourceConfig sourceConfig, boolean isTableIdCaseSensitive) { - this.connectorConfig = sourceConfig.getDbzConnectorConfig(); - this.databaseSchema = createMySqlDatabaseSchema(connectorConfig, isTableIdCaseSensitive); - this.schemasByTableId = new HashMap<>(); - } - - /** - * Gets table schema for the given table path. It will request to MySQL server by running `SHOW - * CREATE TABLE` if cache missed. - */ - public TableChange getTableSchema(JdbcConnection jdbc, TableId tableId) { - // read schema from cache first - TableChange schema = schemasByTableId.get(tableId); - if (schema == null) { - schema = readTableSchema(jdbc, tableId); - schemasByTableId.put(tableId, schema); - } - return schema; - } - - private TableChanges.TableChange readTableSchema(JdbcConnection jdbc, TableId tableId) { - final Map tableChangeMap = new HashMap<>(); - final String sql = "SHOW CREATE TABLE " + MySqlUtils.quote(tableId); - try { - jdbc.query( - sql, - rs -> { - if (rs.next()) { - final String ddl = rs.getString(2); - final MySqlOffsetContext offsetContext = - MySqlOffsetContext.initial(connectorConfig); - final MySqlPartition partition = - new MySqlPartition(connectorConfig.getLogicalName()); - List schemaChangeEvents = - databaseSchema.parseSnapshotDdl( - partition, - ddl, - tableId.catalog(), - offsetContext, - Instant.now()); - for (SchemaChangeEvent schemaChangeEvent : schemaChangeEvents) { - for (TableChanges.TableChange tableChange : - schemaChangeEvent.getTableChanges()) { - tableChangeMap.put(tableId, tableChange); - } - } - } - }); - } catch (SQLException e) { - throw new FlinkRuntimeException( - String.format("Failed to read schema for table %s by running %s", tableId, sql), - e); - } - if (!tableChangeMap.containsKey(tableId)) { - throw new FlinkRuntimeException( - String.format("Can't obtain schema for table %s by running %s", tableId, sql)); - } - - return tableChangeMap.get(tableId); - } -} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/experimental/utils/MySqlTypeUtils.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/experimental/utils/MySqlTypeUtils.java deleted file mode 100644 index ea17b6d691b..00000000000 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/experimental/utils/MySqlTypeUtils.java +++ /dev/null @@ -1,133 +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.connectors.base.experimental.utils; - -import org.apache.flink.table.api.DataTypes; -import org.apache.flink.table.types.DataType; - -import io.debezium.relational.Column; - -/** Utilities for converting from MySQL types to Flink types. */ -public class MySqlTypeUtils { - - // ------ MySQL Type ------ - private static final String BIT = "BIT"; - private static final String TINYINT = "TINYINT"; - private static final String TINYINT_UNSIGNED = "TINYINT UNSIGNED"; - private static final String SMALLINT = "SMALLINT"; - private static final String SMALLINT_UNSIGNED = "SMALLINT UNSIGNED"; - private static final String MEDIUMINT = "MEDIUMINT"; - private static final String MEDIUMINT_UNSIGNED = "MEDIUMINT UNSIGNED"; - private static final String INT = "INT"; - private static final String INT_UNSIGNED = "INT UNSIGNED"; - private static final String BIGINT = "BIGINT"; - private static final String BIGINT_UNSIGNED = "BIGINT UNSIGNED"; - private static final String FLOAT = "FLOAT"; - private static final String FLOAT_UNSIGNED = "FLOAT UNSIGNED"; - private static final String DOUBLE = "DOUBLE"; - private static final String DOUBLE_UNSIGNED = "DOUBLE UNSIGNED"; - private static final String DECIMAL = "DECIMAL"; - private static final String DECIMAL_UNSIGNED = "DECIMAL UNSIGNED"; - private static final String CHAR = "CHAR"; - private static final String VARCHAR = "VARCHAR"; - private static final String TINYTEXT = "TINYTEXT"; - private static final String MEDIUMTEXT = "MEDIUMTEXT"; - private static final String TEXT = "TEXT"; - private static final String LONGTEXT = "LONGTEXT"; - private static final String DATE = "DATE"; - private static final String TIME = "TIME"; - private static final String DATETIME = "DATETIME"; - private static final String TIMESTAMP = "TIMESTAMP"; - private static final String YEAR = "YEAR"; - private static final String BINARY = "BINARY"; - private static final String VARBINARY = "VARBINARY"; - private static final String TINYBLOB = "TINYBLOB"; - private static final String MEDIUMBLOB = "MEDIUMBLOB"; - private static final String BLOB = "BLOB"; - private static final String LONGBLOB = "LONGBLOB"; - private static final String JSON = "JSON"; - private static final String SET = "SET"; - private static final String ENUM = "ENUM"; - private static final String GEOMETRY = "GEOMETRY"; - private static final String UNKNOWN = "UNKNOWN"; - - /** Returns a corresponding Flink data type from a debezium {@link Column}. */ - public static DataType fromDbzColumn(Column column) { - DataType dataType = convertFromColumn(column); - if (column.isOptional()) { - return dataType; - } else { - return dataType.notNull(); - } - } - - /** - * Returns a corresponding Flink data type from a debezium {@link Column} with nullable always - * be true. - */ - private static DataType convertFromColumn(Column column) { - String typeName = column.typeName(); - switch (typeName) { - case TINYINT: - return column.length() == 1 ? DataTypes.BOOLEAN() : DataTypes.TINYINT(); - case TINYINT_UNSIGNED: - case SMALLINT: - return DataTypes.SMALLINT(); - case SMALLINT_UNSIGNED: - case INT: - case MEDIUMINT: - return DataTypes.INT(); - case INT_UNSIGNED: - case MEDIUMINT_UNSIGNED: - case BIGINT: - return DataTypes.BIGINT(); - case BIGINT_UNSIGNED: - return DataTypes.DECIMAL(20, 0); - case FLOAT: - return DataTypes.FLOAT(); - case DOUBLE: - return DataTypes.DOUBLE(); - case DECIMAL: - return DataTypes.DECIMAL(column.length(), column.scale().orElse(0)); - case TIME: - return column.length() >= 0 ? DataTypes.TIME(column.length()) : DataTypes.TIME(); - case DATE: - return DataTypes.DATE(); - case DATETIME: - case TIMESTAMP: - return column.length() >= 0 - ? DataTypes.TIMESTAMP(column.length()) - : DataTypes.TIMESTAMP(); - case CHAR: - return DataTypes.CHAR(column.length()); - case VARCHAR: - return DataTypes.VARCHAR(column.length()); - case TEXT: - return DataTypes.STRING(); - case BINARY: - return DataTypes.BINARY(column.length()); - case VARBINARY: - return DataTypes.VARBINARY(column.length()); - case BLOB: - return DataTypes.BYTES(); - default: - throw new UnsupportedOperationException( - String.format("Don't support MySQL type '%s' yet.", typeName)); - } - } -} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/experimental/utils/MySqlUtils.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/experimental/utils/MySqlUtils.java deleted file mode 100644 index f85a283a7b4..00000000000 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/experimental/utils/MySqlUtils.java +++ /dev/null @@ -1,372 +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.connectors.base.experimental.utils; - -import org.apache.flink.cdc.connectors.base.experimental.offset.BinlogOffset; -import org.apache.flink.table.api.ValidationException; -import org.apache.flink.table.types.logical.RowType; - -import io.debezium.connector.mysql.MySqlConnectorConfig; -import io.debezium.connector.mysql.MySqlDatabaseSchema; -import io.debezium.connector.mysql.MySqlTopicSelector; -import io.debezium.connector.mysql.MySqlValueConverters; -import io.debezium.jdbc.JdbcConnection; -import io.debezium.jdbc.JdbcValueConverters; -import io.debezium.jdbc.TemporalPrecisionMode; -import io.debezium.relational.Column; -import io.debezium.relational.Table; -import io.debezium.relational.TableId; -import io.debezium.schema.TopicSelector; -import io.debezium.util.SchemaNameAdjuster; -import org.apache.kafka.connect.source.SourceRecord; - -import java.sql.Connection; -import java.sql.PreparedStatement; -import java.sql.SQLException; -import java.util.HashMap; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.Optional; -import java.util.stream.Collectors; - -import static org.apache.flink.table.api.DataTypes.FIELD; -import static org.apache.flink.table.api.DataTypes.ROW; - -/** Utils to prepare MySQL SQL statement. */ -public class MySqlUtils { - - private MySqlUtils() {} - - public static long queryApproximateRowCnt(JdbcConnection jdbc, TableId tableId) - throws SQLException { - // The statement used to get approximate row count which is less - // accurate than COUNT(*), but is more efficient for large table. - final String useDatabaseStatement = String.format("USE %s;", quote(tableId.catalog())); - final String rowCountQuery = String.format("SHOW TABLE STATUS LIKE '%s';", tableId.table()); - jdbc.executeWithoutCommitting(useDatabaseStatement); - return jdbc.queryAndMap( - rowCountQuery, - rs -> { - if (!rs.next() || rs.getMetaData().getColumnCount() < 5) { - throw new SQLException( - String.format( - "No result returned after running query [%s]", - rowCountQuery)); - } - return rs.getLong(5); - }); - } - - public static Object queryNextChunkMax( - JdbcConnection jdbc, - TableId tableId, - String splitColumnName, - int chunkSize, - Object includedLowerBound) - throws SQLException { - String quotedColumn = jdbc.quotedColumnIdString(splitColumnName); - String query = - String.format( - "SELECT MAX(%s) FROM (" - + "SELECT %s FROM %s WHERE %s >= ? ORDER BY %s ASC LIMIT %s" - + ") AS T", - quotedColumn, - quotedColumn, - jdbc.quotedTableIdString(tableId), - quotedColumn, - quotedColumn, - chunkSize); - return jdbc.prepareQueryAndMap( - query, - ps -> ps.setObject(1, includedLowerBound), - rs -> { - if (!rs.next()) { - // this should never happen - throw new SQLException( - String.format( - "No result returned after running query [%s]", query)); - } - return rs.getObject(1); - }); - } - - public static String buildSplitScanQuery( - TableId tableId, RowType pkRowType, boolean isFirstSplit, boolean isLastSplit) { - return buildSplitQuery(tableId, pkRowType, isFirstSplit, isLastSplit, -1, true); - } - - private static String buildSplitQuery( - TableId tableId, - RowType pkRowType, - boolean isFirstSplit, - boolean isLastSplit, - int limitSize, - boolean isScanningData) { - final String condition; - - if (isFirstSplit && isLastSplit) { - condition = null; - } else if (isFirstSplit) { - final StringBuilder sql = new StringBuilder(); - addPrimaryKeyColumnsToCondition(pkRowType, sql, " <= ?"); - if (isScanningData) { - sql.append(" AND NOT ("); - addPrimaryKeyColumnsToCondition(pkRowType, sql, " = ?"); - sql.append(")"); - } - condition = sql.toString(); - } else if (isLastSplit) { - final StringBuilder sql = new StringBuilder(); - addPrimaryKeyColumnsToCondition(pkRowType, sql, " >= ?"); - condition = sql.toString(); - } else { - final StringBuilder sql = new StringBuilder(); - addPrimaryKeyColumnsToCondition(pkRowType, sql, " >= ?"); - if (isScanningData) { - sql.append(" AND NOT ("); - addPrimaryKeyColumnsToCondition(pkRowType, sql, " = ?"); - sql.append(")"); - } - sql.append(" AND "); - addPrimaryKeyColumnsToCondition(pkRowType, sql, " <= ?"); - condition = sql.toString(); - } - - if (isScanningData) { - return buildSelectWithRowLimits( - tableId, limitSize, "*", Optional.ofNullable(condition), Optional.empty()); - } else { - final String orderBy = - pkRowType.getFieldNames().stream().collect(Collectors.joining(", ")); - return buildSelectWithBoundaryRowLimits( - tableId, - limitSize, - getPrimaryKeyColumnsProjection(pkRowType), - getMaxPrimaryKeyColumnsProjection(pkRowType), - Optional.ofNullable(condition), - orderBy); - } - } - - public static PreparedStatement readTableSplitDataStatement( - JdbcConnection jdbc, - String sql, - boolean isFirstSplit, - boolean isLastSplit, - Object[] splitStart, - Object[] splitEnd, - int primaryKeyNum, - int fetchSize) { - try { - final PreparedStatement statement = initStatement(jdbc, sql, fetchSize); - if (isFirstSplit && isLastSplit) { - return statement; - } - if (isFirstSplit) { - for (int i = 0; i < primaryKeyNum; i++) { - statement.setObject(i + 1, splitEnd[i]); - statement.setObject(i + 1 + primaryKeyNum, splitEnd[i]); - } - } else if (isLastSplit) { - for (int i = 0; i < primaryKeyNum; i++) { - statement.setObject(i + 1, splitStart[i]); - } - } else { - for (int i = 0; i < primaryKeyNum; i++) { - statement.setObject(i + 1, splitStart[i]); - statement.setObject(i + 1 + primaryKeyNum, splitEnd[i]); - statement.setObject(i + 1 + 2 * primaryKeyNum, splitEnd[i]); - } - } - return statement; - } catch (Exception e) { - throw new RuntimeException("Failed to build the split data read statement.", e); - } - } - - public static RowType getSplitType(Table table) { - List primaryKeys = table.primaryKeyColumns(); - if (primaryKeys.isEmpty()) { - throw new ValidationException( - String.format( - "Incremental snapshot for tables requires primary key," - + " but table %s doesn't have primary key.", - table.id())); - } - - // use first field in primary key as the split key - return getSplitType(primaryKeys.get(0)); - } - - /** Creates a new {@link MySqlDatabaseSchema} to monitor the latest MySql database schemas. */ - public static MySqlDatabaseSchema createMySqlDatabaseSchema( - MySqlConnectorConfig dbzMySqlConfig, boolean isTableIdCaseSensitive) { - TopicSelector topicSelector = MySqlTopicSelector.defaultSelector(dbzMySqlConfig); - SchemaNameAdjuster schemaNameAdjuster = SchemaNameAdjuster.create(); - MySqlValueConverters valueConverters = getValueConverters(dbzMySqlConfig); - return new MySqlDatabaseSchema( - dbzMySqlConfig, - valueConverters, - topicSelector, - schemaNameAdjuster, - isTableIdCaseSensitive); - } - - private static MySqlValueConverters getValueConverters(MySqlConnectorConfig dbzMySqlConfig) { - TemporalPrecisionMode timePrecisionMode = dbzMySqlConfig.getTemporalPrecisionMode(); - JdbcValueConverters.DecimalMode decimalMode = dbzMySqlConfig.getDecimalMode(); - String bigIntUnsignedHandlingModeStr = - dbzMySqlConfig - .getConfig() - .getString(MySqlConnectorConfig.BIGINT_UNSIGNED_HANDLING_MODE); - MySqlConnectorConfig.BigIntUnsignedHandlingMode bigIntUnsignedHandlingMode = - MySqlConnectorConfig.BigIntUnsignedHandlingMode.parse( - bigIntUnsignedHandlingModeStr); - JdbcValueConverters.BigIntUnsignedMode bigIntUnsignedMode = - bigIntUnsignedHandlingMode.asBigIntUnsignedMode(); - - boolean timeAdjusterEnabled = - dbzMySqlConfig.getConfig().getBoolean(MySqlConnectorConfig.ENABLE_TIME_ADJUSTER); - return new MySqlValueConverters( - decimalMode, - timePrecisionMode, - bigIntUnsignedMode, - dbzMySqlConfig.binaryHandlingMode(), - timeAdjusterEnabled ? MySqlValueConverters::adjustTemporal : x -> x, - MySqlValueConverters::defaultParsingErrorHandler); - } - - public static BinlogOffset getBinlogPosition(SourceRecord dataRecord) { - return getBinlogPosition(dataRecord.sourceOffset()); - } - - public static BinlogOffset getBinlogPosition(Map offset) { - Map offsetStrMap = new HashMap<>(); - for (Map.Entry entry : offset.entrySet()) { - offsetStrMap.put( - entry.getKey(), entry.getValue() == null ? null : entry.getValue().toString()); - } - return new BinlogOffset(offsetStrMap); - } - - public static RowType getSplitType(Column splitColumn) { - return (RowType) - ROW(FIELD(splitColumn.name(), MySqlTypeUtils.fromDbzColumn(splitColumn))) - .getLogicalType(); - } - - public static String quote(String dbOrTableName) { - return "`" + dbOrTableName + "`"; - } - - public static String quote(TableId tableId) { - return tableId.toQuotedString('`'); - } - - private static PreparedStatement initStatement(JdbcConnection jdbc, String sql, int fetchSize) - throws SQLException { - final Connection connection = jdbc.connection(); - connection.setAutoCommit(false); - final PreparedStatement statement = connection.prepareStatement(sql); - statement.setFetchSize(fetchSize); - return statement; - } - - private static void addPrimaryKeyColumnsToCondition( - RowType pkRowType, StringBuilder sql, String predicate) { - for (Iterator fieldNamesIt = pkRowType.getFieldNames().iterator(); - fieldNamesIt.hasNext(); ) { - sql.append(fieldNamesIt.next()).append(predicate); - if (fieldNamesIt.hasNext()) { - sql.append(" AND "); - } - } - } - - private static String getPrimaryKeyColumnsProjection(RowType pkRowType) { - StringBuilder sql = new StringBuilder(); - for (Iterator fieldNamesIt = pkRowType.getFieldNames().iterator(); - fieldNamesIt.hasNext(); ) { - sql.append(fieldNamesIt.next()); - if (fieldNamesIt.hasNext()) { - sql.append(" , "); - } - } - return sql.toString(); - } - - private static String getMaxPrimaryKeyColumnsProjection(RowType pkRowType) { - StringBuilder sql = new StringBuilder(); - for (Iterator fieldNamesIt = pkRowType.getFieldNames().iterator(); - fieldNamesIt.hasNext(); ) { - sql.append("MAX(" + fieldNamesIt.next() + ")"); - if (fieldNamesIt.hasNext()) { - sql.append(" , "); - } - } - return sql.toString(); - } - - private static String buildSelectWithRowLimits( - TableId tableId, - int limit, - String projection, - Optional condition, - Optional orderBy) { - final StringBuilder sql = new StringBuilder("SELECT "); - sql.append(projection).append(" FROM "); - sql.append(quotedTableIdString(tableId)); - if (condition.isPresent()) { - sql.append(" WHERE ").append(condition.get()); - } - if (orderBy.isPresent()) { - sql.append(" ORDER BY ").append(orderBy.get()); - } - if (limit > 0) { - sql.append(" LIMIT ").append(limit); - } - return sql.toString(); - } - - private static String buildSelectWithBoundaryRowLimits( - TableId tableId, - int limit, - String projection, - String maxColumnProjection, - Optional condition, - String orderBy) { - final StringBuilder sql = new StringBuilder("SELECT "); - sql.append(maxColumnProjection); - sql.append(" FROM ("); - sql.append("SELECT "); - sql.append(projection); - sql.append(" FROM "); - sql.append(quotedTableIdString(tableId)); - if (condition.isPresent()) { - sql.append(" WHERE ").append(condition.get()); - } - sql.append(" ORDER BY ").append(orderBy).append(" LIMIT ").append(limit); - sql.append(") T"); - return sql.toString(); - } - - private static String quotedTableIdString(TableId tableId) { - return tableId.toQuotedString('`'); - } -} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/experimental/utils/TableDiscoveryUtils.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/experimental/utils/TableDiscoveryUtils.java deleted file mode 100644 index 2b997024b57..00000000000 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/experimental/utils/TableDiscoveryUtils.java +++ /dev/null @@ -1,88 +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.connectors.base.experimental.utils; - -import io.debezium.jdbc.JdbcConnection; -import io.debezium.relational.RelationalTableFilters; -import io.debezium.relational.TableId; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.sql.SQLException; -import java.util.ArrayList; -import java.util.List; - -/** Utilities to discovery matched tables. */ -public class TableDiscoveryUtils { - private static final Logger LOG = LoggerFactory.getLogger(TableDiscoveryUtils.class); - - public static List listTables(JdbcConnection jdbc, RelationalTableFilters tableFilters) - throws SQLException { - final List capturedTableIds = new ArrayList<>(); - // ------------------- - // READ DATABASE NAMES - // ------------------- - // Get the list of databases ... - LOG.info("Read list of available databases"); - final List databaseNames = new ArrayList<>(); - - jdbc.query( - "SHOW DATABASES", - rs -> { - while (rs.next()) { - databaseNames.add(rs.getString(1)); - } - }); - LOG.info("\t list of available databases is: {}", databaseNames); - - // ---------------- - // READ TABLE NAMES - // ---------------- - // Get the list of table IDs for each database. We can't use a prepared statement with - // MySQL, so we have to build the SQL statement each time. Although in other cases this - // might lead to SQL injection, in our case we are reading the database names from the - // database and not taking them from the user ... - LOG.info("Read list of available tables in each database"); - for (String dbName : databaseNames) { - try { - jdbc.query( - "SHOW FULL TABLES IN " - + MySqlUtils.quote(dbName) - + " where Table_Type = 'BASE TABLE'", - rs -> { - while (rs.next()) { - TableId tableId = new TableId(dbName, null, rs.getString(1)); - if (tableFilters.dataCollectionFilter().isIncluded(tableId)) { - capturedTableIds.add(tableId); - LOG.info("\t including '{}' for further processing", tableId); - } else { - LOG.info("\t '{}' is filtered out of capturing", tableId); - } - } - }); - } catch (SQLException e) { - // We were unable to execute the query or process the results, so skip this ... - LOG.warn( - "\t skipping database '{}' due to error reading tables: {}", - dbName, - e.getMessage()); - } - } - return capturedTableIds; - } -} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/experimental/MysqlPooledDataSourceFactory.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/mocked/MockedPooledDataSourceFactory.java similarity index 91% rename from flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/experimental/MysqlPooledDataSourceFactory.java rename to flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/mocked/MockedPooledDataSourceFactory.java index a857a16c979..c728961ccb8 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/experimental/MysqlPooledDataSourceFactory.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/mocked/MockedPooledDataSourceFactory.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.flink.cdc.connectors.base.experimental; +package org.apache.flink.cdc.connectors.base.mocked; import org.apache.flink.cdc.common.annotation.Experimental; import org.apache.flink.cdc.connectors.base.config.JdbcSourceConfig; @@ -23,7 +23,7 @@ /** A MySQL datasource factory. */ @Experimental -public class MysqlPooledDataSourceFactory extends JdbcConnectionPoolFactory { +public class MockedPooledDataSourceFactory extends JdbcConnectionPoolFactory { public static final String JDBC_URL_PATTERN = "jdbc:mysql://%s:%s/?useInformationSchema=true&nullCatalogMeansCurrent=false&useUnicode=true&characterEncoding=UTF-8&characterSetResults=UTF-8&zeroDateTimeBehavior=CONVERT_TO_NULL"; diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/experimental/config/MySqlSourceConfig.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/mocked/MockedSourceConfig.java similarity index 95% rename from flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/experimental/config/MySqlSourceConfig.java rename to flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/mocked/MockedSourceConfig.java index 5d7c201d70d..b503c90c4aa 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/experimental/config/MySqlSourceConfig.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/mocked/MockedSourceConfig.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.flink.cdc.connectors.base.experimental.config; +package org.apache.flink.cdc.connectors.base.mocked; import org.apache.flink.cdc.connectors.base.config.JdbcSourceConfig; import org.apache.flink.cdc.connectors.base.options.StartupOptions; @@ -31,13 +31,13 @@ * Describes the connection information of the Mysql database and the configuration information for * performing snapshotting and streaming reading, such as splitSize. */ -public class MySqlSourceConfig extends JdbcSourceConfig { +public class MockedSourceConfig extends JdbcSourceConfig { private static final long serialVersionUID = 1L; private transient MySqlConnectorConfig dbzMySqlConfig; - public MySqlSourceConfig( + public MockedSourceConfig( StartupOptions startupOptions, List databaseList, List tableList, diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/source/MySqlEventDeserializer.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/source/MySqlEventDeserializer.java deleted file mode 100644 index fa241a46c47..00000000000 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/source/MySqlEventDeserializer.java +++ /dev/null @@ -1,121 +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.connectors.base.source; - -import org.apache.flink.cdc.common.annotation.Internal; -import org.apache.flink.cdc.common.data.binary.BinaryStringData; -import org.apache.flink.cdc.common.event.SchemaChangeEvent; -import org.apache.flink.cdc.common.event.TableId; -import org.apache.flink.cdc.connectors.base.source.parser.CustomMySqlAntlrDdlParser; -import org.apache.flink.cdc.debezium.event.DebeziumEventDeserializationSchema; -import org.apache.flink.cdc.debezium.table.DebeziumChangelogMode; - -import com.fasterxml.jackson.databind.ObjectMapper; -import io.debezium.data.Envelope; -import io.debezium.relational.Tables; -import io.debezium.relational.history.HistoryRecord; -import org.apache.kafka.connect.data.Schema; -import org.apache.kafka.connect.data.Struct; -import org.apache.kafka.connect.source.SourceRecord; - -import java.io.IOException; -import java.util.Collections; -import java.util.List; -import java.util.Map; - -import static org.apache.flink.cdc.connectors.base.utils.SourceRecordUtils.getHistoryRecord; - -/** Event deserializer for {@link MySqlDataSource}. */ -@Internal -public class MySqlEventDeserializer extends DebeziumEventDeserializationSchema { - - private static final long serialVersionUID = 1L; - - public static final String SCHEMA_CHANGE_EVENT_KEY_NAME = - "io.debezium.connector.mysql.SchemaChangeKey"; - - private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); - - private final boolean includeSchemaChanges; - - private transient Tables tables; - private transient CustomMySqlAntlrDdlParser customParser; - - public MySqlEventDeserializer( - DebeziumChangelogMode changelogMode, boolean includeSchemaChanges) { - super(new MySqlSchemaDataTypeInference(), changelogMode); - this.includeSchemaChanges = includeSchemaChanges; - } - - @Override - protected List deserializeSchemaChangeRecord(SourceRecord record) { - if (includeSchemaChanges) { - if (customParser == null) { - customParser = new CustomMySqlAntlrDdlParser(); - tables = new Tables(); - } - - try { - HistoryRecord historyRecord = getHistoryRecord(record); - - String databaseName = - historyRecord.document().getString(HistoryRecord.Fields.DATABASE_NAME); - String ddl = - historyRecord.document().getString(HistoryRecord.Fields.DDL_STATEMENTS); - customParser.setCurrentDatabase(databaseName); - customParser.parse(ddl, tables); - return customParser.getAndClearParsedEvents(); - } catch (IOException e) { - throw new IllegalStateException("Failed to parse the schema change : " + record, e); - } - } - return Collections.emptyList(); - } - - @Override - protected boolean isDataChangeRecord(SourceRecord record) { - Schema valueSchema = record.valueSchema(); - Struct value = (Struct) record.value(); - return value != null - && valueSchema != null - && valueSchema.field(Envelope.FieldName.OPERATION) != null - && value.getString(Envelope.FieldName.OPERATION) != null; - } - - @Override - protected boolean isSchemaChangeRecord(SourceRecord record) { - Schema keySchema = record.keySchema(); - return keySchema != null && SCHEMA_CHANGE_EVENT_KEY_NAME.equalsIgnoreCase(keySchema.name()); - } - - @Override - protected TableId getTableId(SourceRecord record) { - String[] parts = record.topic().split("\\."); - return TableId.tableId(parts[1], parts[2]); - } - - @Override - protected Map getMetadata(SourceRecord record) { - return Collections.emptyMap(); - } - - @Override - protected Object convertToString(Object dbzObj, Schema schema) { - return BinaryStringData.fromString(dbzObj.toString()); - } -} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/source/MySqlSchemaDataTypeInference.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/source/MySqlSchemaDataTypeInference.java deleted file mode 100644 index 3042a61e4ea..00000000000 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/source/MySqlSchemaDataTypeInference.java +++ /dev/null @@ -1,45 +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.connectors.base.source; - -import org.apache.flink.cdc.common.annotation.Internal; -import org.apache.flink.cdc.common.types.DataType; -import org.apache.flink.cdc.common.types.DataTypes; -import org.apache.flink.cdc.debezium.event.DebeziumSchemaDataTypeInference; - -import io.debezium.data.geometry.Geometry; -import io.debezium.data.geometry.Point; -import org.apache.kafka.connect.data.Schema; - -/** {@link DataType} inference for MySQL debezium {@link Schema}. */ -@Internal -public class MySqlSchemaDataTypeInference extends DebeziumSchemaDataTypeInference { - - private static final long serialVersionUID = 1L; - - protected DataType inferStruct(Object value, Schema schema) { - // the Geometry datatype in MySQL will be converted to - // a String with Json format - if (Point.LOGICAL_NAME.equals(schema.name()) - || Geometry.LOGICAL_NAME.equals(schema.name())) { - return DataTypes.STRING(); - } else { - return super.inferStruct(value, schema); - } - } -} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/source/assigner/state/PendingSplitsStateSerializerTest.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/source/assigner/state/PendingSplitsStateSerializerTest.java deleted file mode 100644 index 0ef107a2ec1..00000000000 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/source/assigner/state/PendingSplitsStateSerializerTest.java +++ /dev/null @@ -1,297 +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.connectors.base.source.assigner.state; - -import org.apache.flink.cdc.connectors.base.source.assigner.AssignerStatus; -import org.apache.flink.cdc.connectors.base.source.assigner.state.version5.HybridPendingSplitsStateVersion5; -import org.apache.flink.cdc.connectors.base.source.assigner.state.version5.PendingSplitsStateSerializerVersion5; -import org.apache.flink.cdc.connectors.base.source.assigner.state.version5.SnapshotPendingSplitsStateVersion5; -import org.apache.flink.cdc.connectors.base.source.assigner.state.version6.HybridPendingSplitsStateVersion6; -import org.apache.flink.cdc.connectors.base.source.assigner.state.version6.PendingSplitsStateSerializerVersion6; -import org.apache.flink.cdc.connectors.base.source.assigner.state.version6.SnapshotPendingSplitsStateVersion6; -import org.apache.flink.cdc.connectors.base.source.meta.offset.Offset; -import org.apache.flink.cdc.connectors.base.source.meta.offset.OffsetFactory; -import org.apache.flink.cdc.connectors.base.source.meta.split.SchemalessSnapshotSplit; -import org.apache.flink.cdc.connectors.base.source.meta.split.SourceSplitSerializer; -import org.apache.flink.table.types.logical.BigIntType; -import org.apache.flink.table.types.logical.RowType; - -import io.debezium.relational.Column; -import io.debezium.relational.Table; -import io.debezium.relational.TableEditor; -import io.debezium.relational.TableId; -import io.debezium.relational.history.TableChanges; -import org.junit.Assert; -import org.junit.Test; - -import java.io.IOException; -import java.util.Arrays; -import java.util.Collections; -import java.util.HashMap; -import java.util.Map; - -import static org.apache.flink.cdc.connectors.base.source.meta.split.SnapshotSplit.generateSplitId; - -/** Tests for {@link PendingSplitsStateSerializer}. */ -public class PendingSplitsStateSerializerTest { - - private TableId tableId = TableId.parse("catalog.schema.table1"); - - @Test - public void testPendingSplitsStateSerializerAndDeserialize() throws IOException { - StreamPendingSplitsState streamPendingSplitsStateBefore = - new StreamPendingSplitsState(true); - PendingSplitsStateSerializer pendingSplitsStateSerializer = - new PendingSplitsStateSerializer(constructSourceSplitSerializer()); - PendingSplitsState streamSplitsStateAfter = - pendingSplitsStateSerializer.deserializePendingSplitsState( - 7, pendingSplitsStateSerializer.serialize(streamPendingSplitsStateBefore)); - Assert.assertEquals(streamPendingSplitsStateBefore, streamSplitsStateAfter); - - SnapshotPendingSplitsState snapshotPendingSplitsStateBefore = - constructSnapshotPendingSplitsState(AssignerStatus.NEWLY_ADDED_ASSIGNING); - PendingSplitsState snapshotPendingSplitsStateAfter = - pendingSplitsStateSerializer.deserializePendingSplitsState( - 7, - pendingSplitsStateSerializer.serialize(snapshotPendingSplitsStateBefore)); - Assert.assertEquals(snapshotPendingSplitsStateBefore, snapshotPendingSplitsStateAfter); - - HybridPendingSplitsState hybridPendingSplitsStateBefore = - new HybridPendingSplitsState(snapshotPendingSplitsStateBefore, false); - PendingSplitsState hybridPendingSplitsStateAfter = - pendingSplitsStateSerializer.deserializePendingSplitsState( - 7, pendingSplitsStateSerializer.serialize(hybridPendingSplitsStateBefore)); - Assert.assertEquals(hybridPendingSplitsStateBefore, hybridPendingSplitsStateAfter); - } - - @Test - public void testPendingSplitsStateSerializerCompatibilityVersion5() throws IOException { - StreamPendingSplitsState streamPendingSplitsStateBefore = - new StreamPendingSplitsState(true); - PendingSplitsStateSerializer pendingSplitsStateSerializer = - new PendingSplitsStateSerializer(constructSourceSplitSerializer()); - PendingSplitsState streamSplitsStateAfter = - pendingSplitsStateSerializer.deserializePendingSplitsState( - 5, - PendingSplitsStateSerializerVersion5.serialize( - streamPendingSplitsStateBefore)); - Assert.assertEquals(streamPendingSplitsStateBefore, streamSplitsStateAfter); - - SnapshotPendingSplitsState expectedSnapshotSplitsState = - constructSnapshotPendingSplitsState(AssignerStatus.INITIAL_ASSIGNING); - PendingSplitsState snapshotPendingSplitsStateAfter = - pendingSplitsStateSerializer.deserializePendingSplitsState( - 5, - PendingSplitsStateSerializerVersion5.serialize( - constructSnapshotPendingSplitsStateVersion5(false))); - Assert.assertEquals(expectedSnapshotSplitsState, snapshotPendingSplitsStateAfter); - - HybridPendingSplitsState expectedHybridPendingSplitsState = - new HybridPendingSplitsState( - constructSnapshotPendingSplitsState( - AssignerStatus.INITIAL_ASSIGNING_FINISHED), - false); - PendingSplitsState hybridPendingSplitsStateAfter = - pendingSplitsStateSerializer.deserializePendingSplitsState( - 5, - PendingSplitsStateSerializerVersion5.serialize( - new HybridPendingSplitsStateVersion5( - constructSnapshotPendingSplitsStateVersion5(true), false))); - Assert.assertEquals(expectedHybridPendingSplitsState, hybridPendingSplitsStateAfter); - } - - @Test - public void testPendingSplitsStateSerializerCompatibilityVersion6() throws IOException { - StreamPendingSplitsState streamPendingSplitsStateBefore = - new StreamPendingSplitsState(true); - PendingSplitsStateSerializer pendingSplitsStateSerializer = - new PendingSplitsStateSerializer(constructSourceSplitSerializer()); - PendingSplitsState streamSplitsStateAfter = - pendingSplitsStateSerializer.deserializePendingSplitsState( - 6, - PendingSplitsStateSerializerVersion6.serialize( - streamPendingSplitsStateBefore)); - Assert.assertEquals(streamPendingSplitsStateBefore, streamSplitsStateAfter); - - SnapshotPendingSplitsState expectedSnapshotSplitsState = - constructSnapshotPendingSplitsState(AssignerStatus.INITIAL_ASSIGNING); - PendingSplitsState snapshotPendingSplitsStateAfter = - pendingSplitsStateSerializer.deserializePendingSplitsState( - 6, - PendingSplitsStateSerializerVersion6.serialize( - constructSnapshotPendingSplitsStateVersion6( - AssignerStatus.INITIAL_ASSIGNING))); - Assert.assertEquals(expectedSnapshotSplitsState, snapshotPendingSplitsStateAfter); - - HybridPendingSplitsState expectedHybridPendingSplitsState = - new HybridPendingSplitsState( - constructSnapshotPendingSplitsState( - AssignerStatus.INITIAL_ASSIGNING_FINISHED), - false); - PendingSplitsState hybridPendingSplitsStateAfter = - pendingSplitsStateSerializer.deserializePendingSplitsState( - 6, - PendingSplitsStateSerializerVersion6.serialize( - new HybridPendingSplitsStateVersion6( - constructSnapshotPendingSplitsStateVersion6( - AssignerStatus.INITIAL_ASSIGNING_FINISHED), - false))); - Assert.assertEquals(expectedHybridPendingSplitsState, hybridPendingSplitsStateAfter); - } - - private SourceSplitSerializer constructSourceSplitSerializer() { - return new SourceSplitSerializer() { - @Override - public OffsetFactory getOffsetFactory() { - return new OffsetFactory() { - @Override - public Offset newOffset(Map offset) { - return null; - } - - @Override - public Offset newOffset(String filename, Long position) { - return null; - } - - @Override - public Offset newOffset(Long position) { - return null; - } - - @Override - public Offset createTimestampOffset(long timestampMillis) { - return null; - } - - @Override - public Offset createInitialOffset() { - return null; - } - - @Override - public Offset createNoStoppingOffset() { - return null; - } - }; - } - }; - } - - private SchemalessSnapshotSplit constuctSchemalessSnapshotSplit() { - return new SchemalessSnapshotSplit( - tableId, - generateSplitId(tableId, 0), - new RowType( - Collections.singletonList(new RowType.RowField("id", new BigIntType()))), - null, - null, - null); - } - - private SnapshotPendingSplitsState constructSnapshotPendingSplitsState( - AssignerStatus assignerStatus) { - SchemalessSnapshotSplit schemalessSnapshotSplit = constuctSchemalessSnapshotSplit(); - Map assignedSplits = new HashMap<>(); - assignedSplits.put(tableId.toQuotedString('`'), schemalessSnapshotSplit); - Map tableSchemas = new HashMap<>(); - tableSchemas.put( - tableId, - new TableChanges.TableChange( - TableChanges.TableChangeType.CREATE, createTable(tableId))); - return new SnapshotPendingSplitsState( - Arrays.asList(tableId), - Arrays.asList(schemalessSnapshotSplit), - assignedSplits, - tableSchemas, - new HashMap<>(), - assignerStatus, - Arrays.asList(TableId.parse("catalog2.schema2.table2")), - true, - true, - new HashMap<>()); - } - - private SnapshotPendingSplitsStateVersion5 constructSnapshotPendingSplitsStateVersion5( - boolean isAssignerFinished) { - SchemalessSnapshotSplit schemalessSnapshotSplit = constuctSchemalessSnapshotSplit(); - Map assignedSplits = new HashMap<>(); - assignedSplits.put(tableId.toQuotedString('`'), schemalessSnapshotSplit); - Map tableSchemas = new HashMap<>(); - tableSchemas.put( - tableId, - new TableChanges.TableChange( - TableChanges.TableChangeType.CREATE, createTable(tableId))); - return new SnapshotPendingSplitsStateVersion5( - Arrays.asList(tableId), - Arrays.asList(schemalessSnapshotSplit), - assignedSplits, - tableSchemas, - new HashMap<>(), - isAssignerFinished, - Arrays.asList(TableId.parse("catalog2.schema2.table2")), - true, - true); - } - - private SnapshotPendingSplitsStateVersion6 constructSnapshotPendingSplitsStateVersion6( - AssignerStatus assignerStatus) { - SchemalessSnapshotSplit schemalessSnapshotSplit = constuctSchemalessSnapshotSplit(); - Map assignedSplits = new HashMap<>(); - assignedSplits.put(tableId.toQuotedString('`'), schemalessSnapshotSplit); - Map tableSchemas = new HashMap<>(); - tableSchemas.put( - tableId, - new TableChanges.TableChange( - TableChanges.TableChangeType.CREATE, createTable(tableId))); - return new SnapshotPendingSplitsStateVersion6( - Arrays.asList(tableId), - Arrays.asList(schemalessSnapshotSplit), - assignedSplits, - tableSchemas, - new HashMap<>(), - assignerStatus, - Arrays.asList(TableId.parse("catalog2.schema2.table2")), - true, - true); - } - - private static Table createTable(TableId id) { - TableEditor editor = Table.editor().tableId(id).setDefaultCharsetName("UTF8"); - editor.setComment("comment"); - editor.addColumn( - Column.editor() - .name("id") - .jdbcType(1) - .nativeType(2) - .length(100) - .scale(30) - .create()); - editor.addColumn( - Column.editor() - .name("value") - .jdbcType(2) - .nativeType(3) - .length(50) - .scale(30) - .create()); - - editor.setPrimaryKeyNames(Arrays.asList("id")); - return editor.create(); - } -} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/source/assigner/state/version5/HybridPendingSplitsStateVersion5.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/source/assigner/state/version5/HybridPendingSplitsStateVersion5.java deleted file mode 100644 index 4659c1fc823..00000000000 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/source/assigner/state/version5/HybridPendingSplitsStateVersion5.java +++ /dev/null @@ -1,45 +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.connectors.base.source.assigner.state.version5; - -import org.apache.flink.cdc.connectors.base.source.assigner.state.PendingSplitsState; - -/** - * The 5th version of HybridPendingSplitsState. The modification of the 6th version: Change - * isAssignerFinished(boolean) to assignStatus in SnapshotPendingSplitsState to represent a more - * comprehensive assignment status. - */ -public class HybridPendingSplitsStateVersion5 extends PendingSplitsState { - private final SnapshotPendingSplitsStateVersion5 snapshotPendingSplits; - private final boolean isStreamSplitAssigned; - - public HybridPendingSplitsStateVersion5( - SnapshotPendingSplitsStateVersion5 snapshotPendingSplits, - boolean isStreamSplitAssigned) { - this.snapshotPendingSplits = snapshotPendingSplits; - this.isStreamSplitAssigned = isStreamSplitAssigned; - } - - public SnapshotPendingSplitsStateVersion5 getSnapshotPendingSplits() { - return snapshotPendingSplits; - } - - public boolean isStreamSplitAssigned() { - return isStreamSplitAssigned; - } -} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/source/assigner/state/version5/PendingSplitsStateSerializerVersion5.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/source/assigner/state/version5/PendingSplitsStateSerializerVersion5.java deleted file mode 100644 index 29b644fbe0b..00000000000 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/source/assigner/state/version5/PendingSplitsStateSerializerVersion5.java +++ /dev/null @@ -1,161 +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.connectors.base.source.assigner.state.version5; - -import org.apache.flink.cdc.connectors.base.source.assigner.state.PendingSplitsState; -import org.apache.flink.cdc.connectors.base.source.assigner.state.StreamPendingSplitsState; -import org.apache.flink.cdc.connectors.base.source.meta.offset.Offset; -import org.apache.flink.cdc.connectors.base.source.meta.split.SchemalessSnapshotSplit; -import org.apache.flink.cdc.connectors.base.source.meta.split.version4.LegacySourceSplitSerializierVersion4; -import org.apache.flink.cdc.connectors.base.utils.SerializerUtils; -import org.apache.flink.cdc.debezium.history.FlinkJsonTableChangeSerializer; -import org.apache.flink.core.memory.DataOutputSerializer; - -import io.debezium.document.DocumentWriter; -import io.debezium.relational.TableId; -import io.debezium.relational.history.TableChanges; - -import java.io.IOException; -import java.nio.charset.StandardCharsets; -import java.util.Collection; -import java.util.List; -import java.util.Map; - -/** - * The 5th version of PendingSplitsStateSerializer. The modification of the 6th version: Change - * isAssignerFinished(boolean) to assignStatus in SnapshotPendingSplitsState to represent a more - * comprehensive assignment status. - */ -public class PendingSplitsStateSerializerVersion5 { - - private static final ThreadLocal SERIALIZER_CACHE = - ThreadLocal.withInitial(() -> new DataOutputSerializer(64)); - - private static final int SNAPSHOT_PENDING_SPLITS_STATE_FLAG = 1; - private static final int STREAM_PENDING_SPLITS_STATE_FLAG = 2; - private static final int HYBRID_PENDING_SPLITS_STATE_FLAG = 3; - - public static byte[] serialize(PendingSplitsState state) throws IOException { - final DataOutputSerializer out = SERIALIZER_CACHE.get(); - out.writeInt(5); - - if (state instanceof SnapshotPendingSplitsStateVersion5) { - out.writeInt(SNAPSHOT_PENDING_SPLITS_STATE_FLAG); - serializeSnapshotPendingSplitsState((SnapshotPendingSplitsStateVersion5) state, out); - } else if (state instanceof HybridPendingSplitsStateVersion5) { - out.writeInt(HYBRID_PENDING_SPLITS_STATE_FLAG); - serializeHybridPendingSplitsState((HybridPendingSplitsStateVersion5) state, out); - } else if (state instanceof StreamPendingSplitsState) { - out.writeInt(STREAM_PENDING_SPLITS_STATE_FLAG); - serializeStreamPendingSplitsState((StreamPendingSplitsState) state, out); - } - - final byte[] result = out.getCopyOfBuffer(); - out.clear(); - return result; - } - - private static void serializeHybridPendingSplitsState( - HybridPendingSplitsStateVersion5 state, DataOutputSerializer out) throws IOException { - serializeSnapshotPendingSplitsState(state.getSnapshotPendingSplits(), out); - out.writeBoolean(state.isStreamSplitAssigned()); - } - - private static void serializeSnapshotPendingSplitsState( - SnapshotPendingSplitsStateVersion5 state, DataOutputSerializer out) throws IOException { - writeTableIds(state.getAlreadyProcessedTables(), out); - writeRemainingSplits(state.getRemainingSplits(), out); - writeAssignedSnapshotSplits(state.getAssignedSplits(), out); - writeFinishedOffsets(state.getSplitFinishedOffsets(), out); - out.writeBoolean(state.isAssignerFinished()); - writeTableIds(state.getRemainingTables(), out); - out.writeBoolean(state.isTableIdCaseSensitive()); - writeTableSchemas(state.getTableSchemas(), out); - } - - private static void serializeStreamPendingSplitsState( - StreamPendingSplitsState state, DataOutputSerializer out) throws IOException { - out.writeBoolean(state.isStreamSplitAssigned()); - } - - private static void writeTableIds(Collection tableIds, DataOutputSerializer out) - throws IOException { - final int size = tableIds.size(); - out.writeInt(size); - for (TableId tableId : tableIds) { - boolean useCatalogBeforeSchema = SerializerUtils.shouldUseCatalogBeforeSchema(tableId); - out.writeBoolean(useCatalogBeforeSchema); - out.writeUTF(tableId.toString()); - } - } - - private static void writeRemainingSplits( - List remainingSplits, DataOutputSerializer out) - throws IOException { - final int size = remainingSplits.size(); - out.writeInt(size); - for (SchemalessSnapshotSplit split : remainingSplits) { - byte[] splitBytes = LegacySourceSplitSerializierVersion4.serialize(split); - out.writeInt(splitBytes.length); - out.write(splitBytes); - } - } - - private static void writeAssignedSnapshotSplits( - Map assignedSplits, DataOutputSerializer out) - throws IOException { - final int size = assignedSplits.size(); - out.writeInt(size); - for (Map.Entry entry : assignedSplits.entrySet()) { - out.writeUTF(entry.getKey()); - byte[] splitBytes = LegacySourceSplitSerializierVersion4.serialize(entry.getValue()); - out.writeInt(splitBytes.length); - out.write(splitBytes); - } - } - - private static void writeFinishedOffsets( - Map splitsInfo, DataOutputSerializer out) throws IOException { - final int size = splitsInfo.size(); - out.writeInt(size); - for (Map.Entry splitInfo : splitsInfo.entrySet()) { - out.writeUTF(splitInfo.getKey()); - LegacySourceSplitSerializierVersion4.writeOffsetPosition(splitInfo.getValue(), out); - } - } - - private static void writeTableSchemas( - Map tableSchemas, DataOutputSerializer out) - throws IOException { - FlinkJsonTableChangeSerializer jsonSerializer = new FlinkJsonTableChangeSerializer(); - DocumentWriter documentWriter = DocumentWriter.defaultWriter(); - final int size = tableSchemas.size(); - out.writeInt(size); - for (Map.Entry entry : tableSchemas.entrySet()) { - boolean useCatalogBeforeSchema = - SerializerUtils.shouldUseCatalogBeforeSchema(entry.getKey()); - out.writeBoolean(useCatalogBeforeSchema); - out.writeUTF(entry.getKey().toString()); - final String tableChangeStr = - documentWriter.write(jsonSerializer.toDocument(entry.getValue())); - final byte[] tableChangeBytes = tableChangeStr.getBytes(StandardCharsets.UTF_8); - out.writeInt(tableChangeBytes.length); - out.write(tableChangeBytes); - } - } -} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/source/assigner/state/version5/SnapshotPendingSplitsStateVersion5.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/source/assigner/state/version5/SnapshotPendingSplitsStateVersion5.java deleted file mode 100644 index 177bb242a81..00000000000 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/source/assigner/state/version5/SnapshotPendingSplitsStateVersion5.java +++ /dev/null @@ -1,133 +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.connectors.base.source.assigner.state.version5; - -import org.apache.flink.cdc.connectors.base.source.assigner.state.PendingSplitsState; -import org.apache.flink.cdc.connectors.base.source.enumerator.IncrementalSourceEnumerator; -import org.apache.flink.cdc.connectors.base.source.meta.offset.Offset; -import org.apache.flink.cdc.connectors.base.source.meta.split.SchemalessSnapshotSplit; -import org.apache.flink.cdc.connectors.base.source.reader.IncrementalSourceSplitReader; - -import io.debezium.relational.TableId; -import io.debezium.relational.history.TableChanges; - -import java.util.List; -import java.util.Map; - -/** - * The 5th version of PendingSplitsStateSerializer. The modification of the 6th version: Change - * isAssignerFinished(boolean) to assignStatus in SnapshotPendingSplitsState to represent a more - * comprehensive assignment status. - */ -public class SnapshotPendingSplitsStateVersion5 extends PendingSplitsState { - - /** The tables in the checkpoint. */ - private final List remainingTables; - - /** - * The paths that are no longer in the enumerator checkpoint, but have been processed before and - * should this be ignored. Relevant only for sources in continuous monitoring mode. - */ - private final List alreadyProcessedTables; - - /** The splits in the checkpoint. */ - private final List remainingSplits; - - /** - * The snapshot splits that the {@link IncrementalSourceEnumerator} has assigned to {@link - * IncrementalSourceSplitReader}s. - */ - private final Map assignedSplits; - - /** - * The offsets of finished (snapshot) splits that the {@link IncrementalSourceEnumerator} has - * received from {@link IncrementalSourceSplitReader}s. - */ - private final Map splitFinishedOffsets; - - /** - * Whether the snapshot split assigner is finished, which indicates there is no more splits and - * all records of splits have been completely processed in the pipeline. - */ - private final boolean isAssignerFinished; - - /** Whether the table identifier is case sensitive. */ - private final boolean isTableIdCaseSensitive; - - /** Whether the remaining tables are keep when snapshot state. */ - private final boolean isRemainingTablesCheckpointed; - - private final Map tableSchemas; - - public SnapshotPendingSplitsStateVersion5( - List alreadyProcessedTables, - List remainingSplits, - Map assignedSplits, - Map tableSchemas, - Map splitFinishedOffsets, - boolean isAssignerFinished, - List remainingTables, - boolean isTableIdCaseSensitive, - boolean isRemainingTablesCheckpointed) { - this.alreadyProcessedTables = alreadyProcessedTables; - this.remainingSplits = remainingSplits; - this.assignedSplits = assignedSplits; - this.splitFinishedOffsets = splitFinishedOffsets; - this.isAssignerFinished = isAssignerFinished; - this.remainingTables = remainingTables; - this.isTableIdCaseSensitive = isTableIdCaseSensitive; - this.isRemainingTablesCheckpointed = isRemainingTablesCheckpointed; - this.tableSchemas = tableSchemas; - } - - public List getAlreadyProcessedTables() { - return alreadyProcessedTables; - } - - public List getRemainingSplits() { - return remainingSplits; - } - - public Map getAssignedSplits() { - return assignedSplits; - } - - public Map getTableSchemas() { - return tableSchemas; - } - - public Map getSplitFinishedOffsets() { - return splitFinishedOffsets; - } - - public boolean isAssignerFinished() { - return isAssignerFinished; - } - - public List getRemainingTables() { - return remainingTables; - } - - public boolean isTableIdCaseSensitive() { - return isTableIdCaseSensitive; - } - - public boolean isRemainingTablesCheckpointed() { - return isRemainingTablesCheckpointed; - } -} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/source/assigner/state/version6/HybridPendingSplitsStateVersion6.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/source/assigner/state/version6/HybridPendingSplitsStateVersion6.java deleted file mode 100644 index de39156c4d8..00000000000 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/source/assigner/state/version6/HybridPendingSplitsStateVersion6.java +++ /dev/null @@ -1,41 +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.connectors.base.source.assigner.state.version6; - -import org.apache.flink.cdc.connectors.base.source.assigner.state.PendingSplitsState; - -/** The 6th version of HybridPendingSplitsState. */ -public class HybridPendingSplitsStateVersion6 extends PendingSplitsState { - private final SnapshotPendingSplitsStateVersion6 snapshotPendingSplits; - private final boolean isStreamSplitAssigned; - - public HybridPendingSplitsStateVersion6( - SnapshotPendingSplitsStateVersion6 snapshotPendingSplits, - boolean isStreamSplitAssigned) { - this.snapshotPendingSplits = snapshotPendingSplits; - this.isStreamSplitAssigned = isStreamSplitAssigned; - } - - public SnapshotPendingSplitsStateVersion6 getSnapshotPendingSplits() { - return snapshotPendingSplits; - } - - public boolean isStreamSplitAssigned() { - return isStreamSplitAssigned; - } -} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/source/assigner/state/version6/PendingSplitsStateSerializerVersion6.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/source/assigner/state/version6/PendingSplitsStateSerializerVersion6.java deleted file mode 100644 index 1e75e0543df..00000000000 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/source/assigner/state/version6/PendingSplitsStateSerializerVersion6.java +++ /dev/null @@ -1,157 +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.connectors.base.source.assigner.state.version6; - -import org.apache.flink.cdc.connectors.base.source.assigner.state.PendingSplitsState; -import org.apache.flink.cdc.connectors.base.source.assigner.state.StreamPendingSplitsState; -import org.apache.flink.cdc.connectors.base.source.meta.offset.Offset; -import org.apache.flink.cdc.connectors.base.source.meta.split.SchemalessSnapshotSplit; -import org.apache.flink.cdc.connectors.base.source.meta.split.version4.LegacySourceSplitSerializierVersion4; -import org.apache.flink.cdc.connectors.base.utils.SerializerUtils; -import org.apache.flink.cdc.debezium.history.FlinkJsonTableChangeSerializer; -import org.apache.flink.core.memory.DataOutputSerializer; - -import io.debezium.document.DocumentWriter; -import io.debezium.relational.TableId; -import io.debezium.relational.history.TableChanges; - -import java.io.IOException; -import java.nio.charset.StandardCharsets; -import java.util.Collection; -import java.util.List; -import java.util.Map; - -/** The 6th version of PendingSplitsStateSerializer. */ -public class PendingSplitsStateSerializerVersion6 { - - private static final ThreadLocal SERIALIZER_CACHE = - ThreadLocal.withInitial(() -> new DataOutputSerializer(64)); - - private static final int SNAPSHOT_PENDING_SPLITS_STATE_FLAG = 1; - private static final int STREAM_PENDING_SPLITS_STATE_FLAG = 2; - private static final int HYBRID_PENDING_SPLITS_STATE_FLAG = 3; - - public static byte[] serialize(PendingSplitsState state) throws IOException { - final DataOutputSerializer out = SERIALIZER_CACHE.get(); - out.writeInt(5); - - if (state instanceof SnapshotPendingSplitsStateVersion6) { - out.writeInt(SNAPSHOT_PENDING_SPLITS_STATE_FLAG); - serializeSnapshotPendingSplitsState((SnapshotPendingSplitsStateVersion6) state, out); - } else if (state instanceof HybridPendingSplitsStateVersion6) { - out.writeInt(HYBRID_PENDING_SPLITS_STATE_FLAG); - serializeHybridPendingSplitsState((HybridPendingSplitsStateVersion6) state, out); - } else if (state instanceof StreamPendingSplitsState) { - out.writeInt(STREAM_PENDING_SPLITS_STATE_FLAG); - serializeStreamPendingSplitsState((StreamPendingSplitsState) state, out); - } - - final byte[] result = out.getCopyOfBuffer(); - out.clear(); - return result; - } - - private static void serializeHybridPendingSplitsState( - HybridPendingSplitsStateVersion6 state, DataOutputSerializer out) throws IOException { - serializeSnapshotPendingSplitsState(state.getSnapshotPendingSplits(), out); - out.writeBoolean(state.isStreamSplitAssigned()); - } - - private static void serializeSnapshotPendingSplitsState( - SnapshotPendingSplitsStateVersion6 state, DataOutputSerializer out) throws IOException { - writeTableIds(state.getAlreadyProcessedTables(), out); - writeRemainingSplits(state.getRemainingSplits(), out); - writeAssignedSnapshotSplits(state.getAssignedSplits(), out); - writeFinishedOffsets(state.getSplitFinishedOffsets(), out); - out.writeInt(state.getSnapshotAssignerStatus().getStatusCode()); - writeTableIds(state.getRemainingTables(), out); - out.writeBoolean(state.isTableIdCaseSensitive()); - writeTableSchemas(state.getTableSchemas(), out); - } - - private static void serializeStreamPendingSplitsState( - StreamPendingSplitsState state, DataOutputSerializer out) throws IOException { - out.writeBoolean(state.isStreamSplitAssigned()); - } - - private static void writeTableIds(Collection tableIds, DataOutputSerializer out) - throws IOException { - final int size = tableIds.size(); - out.writeInt(size); - for (TableId tableId : tableIds) { - boolean useCatalogBeforeSchema = SerializerUtils.shouldUseCatalogBeforeSchema(tableId); - out.writeBoolean(useCatalogBeforeSchema); - out.writeUTF(tableId.toString()); - } - } - - private static void writeRemainingSplits( - List remainingSplits, DataOutputSerializer out) - throws IOException { - final int size = remainingSplits.size(); - out.writeInt(size); - for (SchemalessSnapshotSplit split : remainingSplits) { - byte[] splitBytes = LegacySourceSplitSerializierVersion4.serialize(split); - out.writeInt(splitBytes.length); - out.write(splitBytes); - } - } - - private static void writeAssignedSnapshotSplits( - Map assignedSplits, DataOutputSerializer out) - throws IOException { - final int size = assignedSplits.size(); - out.writeInt(size); - for (Map.Entry entry : assignedSplits.entrySet()) { - out.writeUTF(entry.getKey()); - byte[] splitBytes = LegacySourceSplitSerializierVersion4.serialize(entry.getValue()); - out.writeInt(splitBytes.length); - out.write(splitBytes); - } - } - - private static void writeFinishedOffsets( - Map splitsInfo, DataOutputSerializer out) throws IOException { - final int size = splitsInfo.size(); - out.writeInt(size); - for (Map.Entry splitInfo : splitsInfo.entrySet()) { - out.writeUTF(splitInfo.getKey()); - LegacySourceSplitSerializierVersion4.writeOffsetPosition(splitInfo.getValue(), out); - } - } - - private static void writeTableSchemas( - Map tableSchemas, DataOutputSerializer out) - throws IOException { - FlinkJsonTableChangeSerializer jsonSerializer = new FlinkJsonTableChangeSerializer(); - DocumentWriter documentWriter = DocumentWriter.defaultWriter(); - final int size = tableSchemas.size(); - out.writeInt(size); - for (Map.Entry entry : tableSchemas.entrySet()) { - boolean useCatalogBeforeSchema = - SerializerUtils.shouldUseCatalogBeforeSchema(entry.getKey()); - out.writeBoolean(useCatalogBeforeSchema); - out.writeUTF(entry.getKey().toString()); - final String tableChangeStr = - documentWriter.write(jsonSerializer.toDocument(entry.getValue())); - final byte[] tableChangeBytes = tableChangeStr.getBytes(StandardCharsets.UTF_8); - out.writeInt(tableChangeBytes.length); - out.write(tableChangeBytes); - } - } -} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/source/assigner/state/version6/SnapshotPendingSplitsStateVersion6.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/source/assigner/state/version6/SnapshotPendingSplitsStateVersion6.java deleted file mode 100644 index 0ac12e44e29..00000000000 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/source/assigner/state/version6/SnapshotPendingSplitsStateVersion6.java +++ /dev/null @@ -1,130 +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.connectors.base.source.assigner.state.version6; - -import org.apache.flink.cdc.connectors.base.source.assigner.AssignerStatus; -import org.apache.flink.cdc.connectors.base.source.assigner.state.PendingSplitsState; -import org.apache.flink.cdc.connectors.base.source.enumerator.IncrementalSourceEnumerator; -import org.apache.flink.cdc.connectors.base.source.meta.offset.Offset; -import org.apache.flink.cdc.connectors.base.source.meta.split.SchemalessSnapshotSplit; -import org.apache.flink.cdc.connectors.base.source.reader.IncrementalSourceSplitReader; - -import io.debezium.relational.TableId; -import io.debezium.relational.history.TableChanges; - -import java.util.List; -import java.util.Map; - -/** - * The 6th version of PendingSplitsStateSerializer. The modification of the 7th version: Add - * splitFinishedCheckpointIds variables. - */ -public class SnapshotPendingSplitsStateVersion6 extends PendingSplitsState { - - /** The tables in the checkpoint. */ - private final List remainingTables; - - /** - * The paths that are no longer in the enumerator checkpoint, but have been processed before and - * should this be ignored. Relevant only for sources in continuous monitoring mode. - */ - private final List alreadyProcessedTables; - - /** The splits in the checkpoint. */ - private final List remainingSplits; - - /** - * The snapshot splits that the {@link IncrementalSourceEnumerator} has assigned to {@link - * IncrementalSourceSplitReader}s. - */ - private final Map assignedSplits; - - /* The {@link AssignerStatus} that indicates the snapshot assigner status. */ - private final AssignerStatus assignerStatus; - - /** - * The offsets of finished (snapshot) splits that the {@link IncrementalSourceEnumerator} has - * received from {@link IncrementalSourceSplitReader}s. - */ - private final Map splitFinishedOffsets; - - /** Whether the table identifier is case sensitive. */ - private final boolean isTableIdCaseSensitive; - - /** Whether the remaining tables are keep when snapshot state. */ - private final boolean isRemainingTablesCheckpointed; - - private final Map tableSchemas; - - public SnapshotPendingSplitsStateVersion6( - List alreadyProcessedTables, - List remainingSplits, - Map assignedSplits, - Map tableSchemas, - Map splitFinishedOffsets, - AssignerStatus assignerStatus, - List remainingTables, - boolean isTableIdCaseSensitive, - boolean isRemainingTablesCheckpointed) { - this.alreadyProcessedTables = alreadyProcessedTables; - this.remainingSplits = remainingSplits; - this.assignedSplits = assignedSplits; - this.splitFinishedOffsets = splitFinishedOffsets; - this.assignerStatus = assignerStatus; - this.remainingTables = remainingTables; - this.isTableIdCaseSensitive = isTableIdCaseSensitive; - this.isRemainingTablesCheckpointed = isRemainingTablesCheckpointed; - this.tableSchemas = tableSchemas; - } - - public List getAlreadyProcessedTables() { - return alreadyProcessedTables; - } - - public List getRemainingSplits() { - return remainingSplits; - } - - public Map getAssignedSplits() { - return assignedSplits; - } - - public Map getTableSchemas() { - return tableSchemas; - } - - public Map getSplitFinishedOffsets() { - return splitFinishedOffsets; - } - - public List getRemainingTables() { - return remainingTables; - } - - public boolean isTableIdCaseSensitive() { - return isTableIdCaseSensitive; - } - - public boolean isRemainingTablesCheckpointed() { - return isRemainingTablesCheckpointed; - } - - public AssignerStatus getSnapshotAssignerStatus() { - return assignerStatus; - } -} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/source/meta/split/SourceSplitSerializerTest.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/source/meta/split/SourceSplitSerializerTest.java deleted file mode 100644 index f81e1f93b80..00000000000 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/source/meta/split/SourceSplitSerializerTest.java +++ /dev/null @@ -1,154 +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.connectors.base.source.meta.split; - -import org.apache.flink.cdc.connectors.base.source.meta.offset.Offset; -import org.apache.flink.cdc.connectors.base.source.meta.offset.OffsetFactory; -import org.apache.flink.cdc.connectors.base.source.meta.split.version4.LegacySourceSplitSerializierVersion4; -import org.apache.flink.cdc.connectors.base.source.meta.split.version4.StreamSplitVersion4; -import org.apache.flink.table.types.logical.BigIntType; -import org.apache.flink.table.types.logical.RowType; - -import io.debezium.relational.TableId; -import org.junit.Test; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; -import java.util.Map; - -import static org.junit.Assert.assertEquals; - -/** Tests for {@link SourceSplitSerializer}. */ -public class SourceSplitSerializerTest { - - @Test - public void testSourceSplitSerializeAndDeserialize() throws IOException { - SnapshotSplit snapshotSplitBefore = constuctSnapshotSplit(); - SourceSplitSerializer sourceSplitSerializer = constructSourceSplitSerializer(); - SnapshotSplit snapshotSplitAfter = - (SnapshotSplit) - sourceSplitSerializer.deserialize( - sourceSplitSerializer.getVersion(), - sourceSplitSerializer.serialize(snapshotSplitBefore)); - - assertEquals(snapshotSplitBefore, snapshotSplitAfter); - - StreamSplit streamSplitBefore = constuctStreamSplit(true); - StreamSplit streamSplitAfter = - (StreamSplit) - sourceSplitSerializer.deserialize( - sourceSplitSerializer.getVersion(), - sourceSplitSerializer.serialize(streamSplitBefore)); - - assertEquals(streamSplitBefore, streamSplitAfter); - } - - @Test - public void testStreamSplitBackwardCompatibility() throws IOException { - SnapshotSplit snapshotSplitBefore = constuctSnapshotSplit(); - SourceSplitSerializer sourceSplitSerializer = constructSourceSplitSerializer(); - SnapshotSplit snapshotSplitAfter = - (SnapshotSplit) - sourceSplitSerializer.deserialize( - 4, - LegacySourceSplitSerializierVersion4.serialize( - snapshotSplitBefore)); - - assertEquals(snapshotSplitBefore, snapshotSplitAfter); - - StreamSplitVersion4 streamSplitVersion4Before = constuctStreamSplitVersion4(); - StreamSplit expectedStreamSplit = constuctStreamSplit(false); - StreamSplit streamSplitAfter = - (StreamSplit) - sourceSplitSerializer.deserialize( - 4, - LegacySourceSplitSerializierVersion4.serialize( - streamSplitVersion4Before)); - - assertEquals(expectedStreamSplit, streamSplitAfter); - } - - private SnapshotSplit constuctSnapshotSplit() { - return new SnapshotSplit( - new TableId("cata`log\"", "s\"che`ma", "ta\"ble.1`"), - 0, - new RowType( - Collections.singletonList(new RowType.RowField("id", new BigIntType()))), - null, - null, - null, - new HashMap<>()); - } - - private StreamSplit constuctStreamSplit(boolean isSuspend) { - return new StreamSplit( - "database1.schema1.table1", - null, - null, - new ArrayList<>(), - new HashMap<>(), - 0, - isSuspend); - } - - private StreamSplitVersion4 constuctStreamSplitVersion4() { - return new StreamSplitVersion4( - "database1.schema1.table1", null, null, new ArrayList<>(), new HashMap<>(), 0); - } - - private SourceSplitSerializer constructSourceSplitSerializer() { - return new SourceSplitSerializer() { - @Override - public OffsetFactory getOffsetFactory() { - return new OffsetFactory() { - @Override - public Offset newOffset(Map offset) { - return null; - } - - @Override - public Offset newOffset(String filename, Long position) { - return null; - } - - @Override - public Offset newOffset(Long position) { - return null; - } - - @Override - public Offset createTimestampOffset(long timestampMillis) { - return null; - } - - @Override - public Offset createInitialOffset() { - return null; - } - - @Override - public Offset createNoStoppingOffset() { - return null; - } - }; - } - }; - } -} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/source/meta/split/version4/LegacySourceSplitSerializierVersion4.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/source/meta/split/version4/LegacySourceSplitSerializierVersion4.java deleted file mode 100644 index 780e5c4f29f..00000000000 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/source/meta/split/version4/LegacySourceSplitSerializierVersion4.java +++ /dev/null @@ -1,130 +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.connectors.base.source.meta.split.version4; - -import org.apache.flink.cdc.connectors.base.source.meta.offset.Offset; -import org.apache.flink.cdc.connectors.base.source.meta.offset.OffsetDeserializerSerializer; -import org.apache.flink.cdc.connectors.base.source.meta.split.FinishedSnapshotSplitInfo; -import org.apache.flink.cdc.connectors.base.source.meta.split.SnapshotSplit; -import org.apache.flink.cdc.connectors.base.source.meta.split.SourceSplitBase; -import org.apache.flink.cdc.connectors.base.utils.SerializerUtils; -import org.apache.flink.cdc.debezium.history.FlinkJsonTableChangeSerializer; -import org.apache.flink.core.memory.DataOutputSerializer; - -import io.debezium.document.DocumentWriter; -import io.debezium.relational.TableId; -import io.debezium.relational.history.TableChanges; - -import java.io.IOException; -import java.nio.charset.StandardCharsets; -import java.util.List; -import java.util.Map; - -/** The 4th version of legacy SourceSplitSerializier for test. */ -public class LegacySourceSplitSerializierVersion4 { - private static final ThreadLocal SERIALIZER_CACHE = - ThreadLocal.withInitial(() -> new DataOutputSerializer(64)); - private static final int SNAPSHOT_SPLIT_FLAG = 1; - private static final int STREAM_SPLIT_FLAG = 2; - - public static byte[] serialize(SourceSplitBase sourceSplit) throws IOException { - final DataOutputSerializer out = SERIALIZER_CACHE.get(); - if (sourceSplit instanceof SnapshotSplit) { - serializeSnapshotSplit(sourceSplit.asSnapshotSplit(), out); - } else if (sourceSplit instanceof StreamSplitVersion4) { - serializeStreamSplit((StreamSplitVersion4) sourceSplit, out); - } - - final byte[] result = out.getCopyOfBuffer(); - out.clear(); - return result; - } - - private static void serializeSnapshotSplit( - SnapshotSplit snapshotSplit, DataOutputSerializer out) throws IOException { - out.writeInt(SNAPSHOT_SPLIT_FLAG); - boolean useCatalogBeforeSchema = - SerializerUtils.shouldUseCatalogBeforeSchema(snapshotSplit.getTableId()); - out.writeBoolean(useCatalogBeforeSchema); - out.writeUTF(snapshotSplit.getTableId().toDoubleQuotedString()); - out.writeUTF(snapshotSplit.splitId()); - out.writeUTF(snapshotSplit.getSplitKeyType().asSerializableString()); - - final Object[] splitStart = snapshotSplit.getSplitStart(); - final Object[] splitEnd = snapshotSplit.getSplitEnd(); - // rowToSerializedString deals null case - out.writeUTF(SerializerUtils.rowToSerializedString(splitStart)); - out.writeUTF(SerializerUtils.rowToSerializedString(splitEnd)); - writeOffsetPosition(snapshotSplit.getHighWatermark(), out); - writeTableSchemas(snapshotSplit.getTableSchemas(), out); - } - - private static void serializeStreamSplit( - StreamSplitVersion4 streamSplit, DataOutputSerializer out) throws IOException { - - out.writeInt(STREAM_SPLIT_FLAG); - out.writeUTF(streamSplit.splitId()); - out.writeUTF(""); - writeOffsetPosition(streamSplit.getStartingOffset(), out); - writeOffsetPosition(streamSplit.getEndingOffset(), out); - writeFinishedSplitsInfo(streamSplit.getFinishedSnapshotSplitInfos(), out); - writeTableSchemas(streamSplit.getTableSchemas(), out); - out.writeInt(streamSplit.getTotalFinishedSplitSize()); - } - - public static void writeTableSchemas( - Map tableSchemas, DataOutputSerializer out) - throws IOException { - FlinkJsonTableChangeSerializer jsonSerializer = new FlinkJsonTableChangeSerializer(); - DocumentWriter documentWriter = DocumentWriter.defaultWriter(); - final int size = tableSchemas.size(); - out.writeInt(size); - for (Map.Entry entry : tableSchemas.entrySet()) { - boolean useCatalogBeforeSchema = - SerializerUtils.shouldUseCatalogBeforeSchema(entry.getKey()); - out.writeBoolean(useCatalogBeforeSchema); - out.writeUTF(entry.getKey().toString()); - final String tableChangeStr = - documentWriter.write(jsonSerializer.toDocument(entry.getValue())); - final byte[] tableChangeBytes = tableChangeStr.getBytes(StandardCharsets.UTF_8); - out.writeInt(tableChangeBytes.length); - out.write(tableChangeBytes); - } - } - - private static void writeFinishedSplitsInfo( - List finishedSplitsInfo, DataOutputSerializer out) - throws IOException { - final int size = finishedSplitsInfo.size(); - out.writeInt(size); - for (FinishedSnapshotSplitInfo splitInfo : finishedSplitsInfo) { - splitInfo.serialize(out); - } - } - - public static void writeOffsetPosition(Offset offset, DataOutputSerializer out) - throws IOException { - out.writeBoolean(offset != null); - if (offset != null) { - byte[] offsetBytes = - OffsetDeserializerSerializer.OffsetSerializer.INSTANCE.serialize(offset); - out.writeInt(offsetBytes.length); - out.write(offsetBytes); - } - } -} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/source/meta/split/version4/StreamSplitVersion4.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/source/meta/split/version4/StreamSplitVersion4.java deleted file mode 100644 index c47b802b000..00000000000 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/source/meta/split/version4/StreamSplitVersion4.java +++ /dev/null @@ -1,86 +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.connectors.base.source.meta.split.version4; - -import org.apache.flink.cdc.connectors.base.source.meta.offset.Offset; -import org.apache.flink.cdc.connectors.base.source.meta.split.FinishedSnapshotSplitInfo; -import org.apache.flink.cdc.connectors.base.source.meta.split.SourceSplitBase; - -import io.debezium.relational.TableId; -import io.debezium.relational.history.TableChanges; - -import javax.annotation.Nullable; - -import java.util.List; -import java.util.Map; - -/** - * The 5th version of StreamSplit. - * - *

In the 5th version, SourceSplit, and SourceSplitBase and SnapshotSplit are same with 4th. Only - * Add isSuspended(boolean) to StreamSplit, which means whether stream split read is suspended. - */ -public class StreamSplitVersion4 extends SourceSplitBase { - - private final Offset startingOffset; - private final Offset endingOffset; - private final List finishedSnapshotSplitInfos; - private final Map tableSchemas; - private final int totalFinishedSplitSize; - @Nullable transient byte[] serializedFormCache; - - public StreamSplitVersion4( - String splitId, - Offset startingOffset, - Offset endingOffset, - List finishedSnapshotSplitInfos, - Map tableSchemas, - int totalFinishedSplitSize) { - super(splitId); - this.startingOffset = startingOffset; - this.endingOffset = endingOffset; - this.finishedSnapshotSplitInfos = finishedSnapshotSplitInfos; - this.tableSchemas = tableSchemas; - this.totalFinishedSplitSize = totalFinishedSplitSize; - } - - public Offset getStartingOffset() { - return startingOffset; - } - - public Offset getEndingOffset() { - return endingOffset; - } - - public List getFinishedSnapshotSplitInfos() { - return finishedSnapshotSplitInfos; - } - - @Override - public Map getTableSchemas() { - return tableSchemas; - } - - public int getTotalFinishedSplitSize() { - return totalFinishedSplitSize; - } - - public boolean isCompletedSplit() { - return totalFinishedSplitSize == finishedSnapshotSplitInfos.size(); - } -} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/source/parser/CustomAlterTableParserListener.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/source/parser/CustomAlterTableParserListener.java deleted file mode 100644 index ac051724738..00000000000 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/source/parser/CustomAlterTableParserListener.java +++ /dev/null @@ -1,443 +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.connectors.base.source.parser; - -import org.apache.flink.cdc.common.event.AddColumnEvent; -import org.apache.flink.cdc.common.event.AlterColumnTypeEvent; -import org.apache.flink.cdc.common.event.CreateTableEvent; -import org.apache.flink.cdc.common.event.DropColumnEvent; -import org.apache.flink.cdc.common.event.DropTableEvent; -import org.apache.flink.cdc.common.event.RenameColumnEvent; -import org.apache.flink.cdc.common.event.SchemaChangeEvent; -import org.apache.flink.cdc.common.event.TruncateTableEvent; -import org.apache.flink.cdc.common.schema.Schema; -import org.apache.flink.cdc.common.types.DataType; -import org.apache.flink.cdc.common.types.DataTypes; -import org.apache.flink.table.types.logical.LogicalTypeRoot; - -import io.debezium.connector.mysql.antlr.MySqlAntlrDdlParser; -import io.debezium.connector.mysql.antlr.listener.AlterTableParserListener; -import io.debezium.ddl.parser.mysql.generated.MySqlParser; -import io.debezium.ddl.parser.mysql.generated.MySqlParserBaseListener; -import io.debezium.relational.Column; -import io.debezium.relational.ColumnEditor; -import io.debezium.relational.TableEditor; -import io.debezium.relational.TableId; -import org.antlr.v4.runtime.tree.ParseTreeListener; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; -import java.util.stream.Collectors; - -import static org.apache.flink.cdc.connectors.base.experimental.utils.MySqlTypeUtils.fromDbzColumn; - -/** Copied from {@link AlterTableParserListener} in Debezium 1.9.8.Final. */ -public class CustomAlterTableParserListener extends MySqlParserBaseListener { - - private static final int STARTING_INDEX = 1; - - private static final Logger LOG = LoggerFactory.getLogger(CustomAlterTableParserListener.class); - - private final MySqlAntlrDdlParser parser; - private final List listeners; - private final LinkedList changes; - private org.apache.flink.cdc.common.event.TableId currentTable; - private List columnEditors; - private CustomColumnDefinitionParserListener columnDefinitionListener; - private TableEditor tableEditor; - - private int parsingColumnIndex = STARTING_INDEX; - - public CustomAlterTableParserListener( - MySqlAntlrDdlParser parser, - List listeners, - LinkedList changes) { - this.parser = parser; - this.listeners = listeners; - this.changes = changes; - } - - @Override - public void enterColumnCreateTable(MySqlParser.ColumnCreateTableContext ctx) { - TableId tableId = parser.parseQualifiedTableId(ctx.tableName().fullId()); - if (parser.databaseTables().forTable(tableId) == null) { - tableEditor = parser.databaseTables().editOrCreateTable(tableId); - } - super.enterColumnCreateTable(ctx); - } - - @Override - public void exitColumnCreateTable(MySqlParser.ColumnCreateTableContext ctx) { - parser.runIfNotNull( - () -> { - // Make sure that the table's character set has been set ... - if (!tableEditor.hasDefaultCharsetName()) { - tableEditor.setDefaultCharsetName( - parser.charsetForTable(tableEditor.tableId())); - } - listeners.remove(columnDefinitionListener); - columnDefinitionListener = null; - // remove column definition parser listener - final String defaultCharsetName = tableEditor.create().defaultCharsetName(); - tableEditor.setColumns( - tableEditor.columns().stream() - .map( - column -> { - final ColumnEditor columnEditor = column.edit(); - if (columnEditor.charsetNameOfTable() == null) { - columnEditor.charsetNameOfTable( - defaultCharsetName); - } - return columnEditor; - }) - .map(ColumnEditor::create) - .collect(Collectors.toList())); - parser.databaseTables().overwriteTable(tableEditor.create()); - parser.signalCreateTable(tableEditor.tableId(), ctx); - - Schema.Builder builder = Schema.newBuilder(); - tableEditor.columns().forEach(column -> builder.column(toCdcColumn(column))); - if (tableEditor.hasPrimaryKey()) { - builder.primaryKey(tableEditor.primaryKeyColumnNames()); - } - changes.add( - new CreateTableEvent( - toCdcTableId(tableEditor.tableId()), builder.build())); - }, - tableEditor); - super.exitColumnCreateTable(ctx); - } - - @Override - public void enterColumnDeclaration(MySqlParser.ColumnDeclarationContext ctx) { - parser.runIfNotNull( - () -> { - String columnName = parser.parseName(ctx.uid()); - ColumnEditor columnEditor = Column.editor().name(columnName); - if (columnDefinitionListener == null) { - columnDefinitionListener = - new CustomColumnDefinitionParserListener( - tableEditor, columnEditor, parser, listeners); - listeners.add(columnDefinitionListener); - } else { - columnDefinitionListener.setColumnEditor(columnEditor); - } - }, - tableEditor); - super.enterColumnDeclaration(ctx); - } - - @Override - public void exitColumnDeclaration(MySqlParser.ColumnDeclarationContext ctx) { - parser.runIfNotNull( - () -> { - tableEditor.addColumn(columnDefinitionListener.getColumn()); - }, - tableEditor, - columnDefinitionListener); - super.exitColumnDeclaration(ctx); - } - - @Override - public void enterPrimaryKeyTableConstraint(MySqlParser.PrimaryKeyTableConstraintContext ctx) { - parser.runIfNotNull( - () -> { - parser.parsePrimaryIndexColumnNames(ctx.indexColumnNames(), tableEditor); - }, - tableEditor); - super.enterPrimaryKeyTableConstraint(ctx); - } - - @Override - public void enterUniqueKeyTableConstraint(MySqlParser.UniqueKeyTableConstraintContext ctx) { - parser.runIfNotNull( - () -> { - if (!tableEditor.hasPrimaryKey()) { - parser.parsePrimaryIndexColumnNames(ctx.indexColumnNames(), tableEditor); - } - }, - tableEditor); - super.enterUniqueKeyTableConstraint(ctx); - } - - @Override - public void enterAlterTable(MySqlParser.AlterTableContext ctx) { - this.currentTable = toCdcTableId(parser.parseQualifiedTableId(ctx.tableName().fullId())); - super.enterAlterTable(ctx); - } - - @Override - public void exitAlterTable(MySqlParser.AlterTableContext ctx) { - listeners.remove(columnDefinitionListener); - super.exitAlterTable(ctx); - this.currentTable = null; - } - - @Override - public void enterAlterByAddColumn(MySqlParser.AlterByAddColumnContext ctx) { - String columnName = parser.parseName(ctx.uid(0)); - ColumnEditor columnEditor = Column.editor().name(columnName); - columnDefinitionListener = - new CustomColumnDefinitionParserListener( - tableEditor, columnEditor, parser, listeners); - listeners.add(columnDefinitionListener); - super.exitAlterByAddColumn(ctx); - } - - @Override - public void exitAlterByAddColumn(MySqlParser.AlterByAddColumnContext ctx) { - parser.runIfNotNull( - () -> { - Column column = columnDefinitionListener.getColumn(); - if (ctx.FIRST() != null) { - changes.add( - new AddColumnEvent( - currentTable, - Collections.singletonList( - new AddColumnEvent.ColumnWithPosition( - toCdcColumn(column), - AddColumnEvent.ColumnPosition.FIRST, - null)))); - } else if (ctx.AFTER() != null) { - String afterColumn = parser.parseName(ctx.uid(1)); - changes.add( - new AddColumnEvent( - currentTable, - Collections.singletonList( - new AddColumnEvent.ColumnWithPosition( - toCdcColumn(column), - AddColumnEvent.ColumnPosition.AFTER, - afterColumn)))); - } else { - changes.add( - new AddColumnEvent( - currentTable, - Collections.singletonList( - new AddColumnEvent.ColumnWithPosition( - toCdcColumn(column))))); - } - listeners.remove(columnDefinitionListener); - }, - columnDefinitionListener); - super.exitAlterByAddColumn(ctx); - } - - @Override - public void enterAlterByAddColumns(MySqlParser.AlterByAddColumnsContext ctx) { - // multiple columns are added. Initialize a list of column editors for them - columnEditors = new ArrayList<>(ctx.uid().size()); - for (MySqlParser.UidContext uidContext : ctx.uid()) { - String columnName = parser.parseName(uidContext); - columnEditors.add(Column.editor().name(columnName)); - } - columnDefinitionListener = - new CustomColumnDefinitionParserListener( - tableEditor, columnEditors.get(0), parser, listeners); - listeners.add(columnDefinitionListener); - super.enterAlterByAddColumns(ctx); - } - - @Override - public void exitColumnDefinition(MySqlParser.ColumnDefinitionContext ctx) { - parser.runIfNotNull( - () -> { - if (columnEditors != null) { - // column editor list is not null when a multiple columns are parsed in one - // statement - if (columnEditors.size() > parsingColumnIndex) { - // assign next column editor to parse another column definition - columnDefinitionListener.setColumnEditor( - columnEditors.get(parsingColumnIndex++)); - } - } - }, - columnEditors); - super.exitColumnDefinition(ctx); - } - - @Override - public void exitAlterByAddColumns(MySqlParser.AlterByAddColumnsContext ctx) { - parser.runIfNotNull( - () -> { - List addedColumns = new ArrayList<>(); - columnEditors.forEach( - columnEditor -> { - Column column = columnEditor.create(); - addedColumns.add( - new AddColumnEvent.ColumnWithPosition(toCdcColumn(column))); - }); - changes.add(new AddColumnEvent(currentTable, addedColumns)); - listeners.remove(columnDefinitionListener); - columnEditors = null; - parsingColumnIndex = STARTING_INDEX; - }, - columnEditors); - super.exitAlterByAddColumns(ctx); - } - - @Override - public void enterAlterByChangeColumn(MySqlParser.AlterByChangeColumnContext ctx) { - String oldColumnName = parser.parseName(ctx.oldColumn); - ColumnEditor columnEditor = Column.editor().name(oldColumnName); - columnEditor.unsetDefaultValueExpression(); - - columnDefinitionListener = - new CustomColumnDefinitionParserListener( - tableEditor, columnEditor, parser, listeners); - listeners.add(columnDefinitionListener); - super.enterAlterByChangeColumn(ctx); - } - - @Override - public void exitAlterByChangeColumn(MySqlParser.AlterByChangeColumnContext ctx) { - parser.runIfNotNull( - () -> { - Column column = columnDefinitionListener.getColumn(); - String newColumnName = parser.parseName(ctx.newColumn); - - Map typeMapping = new HashMap<>(); - typeMapping.put(column.name(), convertDataType(fromDbzColumn(column))); - changes.add(new AlterColumnTypeEvent(currentTable, typeMapping)); - - if (newColumnName != null && !column.name().equalsIgnoreCase(newColumnName)) { - Map renameMap = new HashMap<>(); - renameMap.put(column.name(), newColumnName); - changes.add(new RenameColumnEvent(currentTable, renameMap)); - } - listeners.remove(columnDefinitionListener); - }, - columnDefinitionListener); - super.exitAlterByChangeColumn(ctx); - } - - private DataType convertDataType(org.apache.flink.table.types.DataType dataType) { - if (dataType.getLogicalType().is(LogicalTypeRoot.INTEGER)) { - return DataTypes.INT(); - } - if (dataType.getLogicalType().is(LogicalTypeRoot.BIGINT)) { - return DataTypes.BIGINT(); - } - if (dataType.getLogicalType().is(LogicalTypeRoot.FLOAT)) { - return DataTypes.FLOAT(); - } - if (dataType.getLogicalType().is(LogicalTypeRoot.DATE)) { - return DataTypes.DATE(); - } - return DataTypes.STRING(); - } - - @Override - public void enterAlterByDropColumn(MySqlParser.AlterByDropColumnContext ctx) { - String removedColName = parser.parseName(ctx.uid()); - changes.add(new DropColumnEvent(currentTable, Collections.singletonList(removedColName))); - super.enterAlterByDropColumn(ctx); - } - - @Override - public void enterAlterByRenameColumn(MySqlParser.AlterByRenameColumnContext ctx) { - String oldColumnName = parser.parseName(ctx.oldColumn); - ColumnEditor columnEditor = Column.editor().name(oldColumnName); - columnDefinitionListener = - new CustomColumnDefinitionParserListener( - tableEditor, columnEditor, parser, listeners); - listeners.add(columnDefinitionListener); - super.enterAlterByRenameColumn(ctx); - } - - @Override - public void enterAlterByModifyColumn(MySqlParser.AlterByModifyColumnContext ctx) { - String oldColumnName = parser.parseName(ctx.uid(0)); - ColumnEditor columnEditor = Column.editor().name(oldColumnName); - columnEditor.unsetDefaultValueExpression(); - - columnDefinitionListener = - new CustomColumnDefinitionParserListener( - tableEditor, columnEditor, parser, listeners); - listeners.add(columnDefinitionListener); - super.enterAlterByModifyColumn(ctx); - } - - @Override - public void exitAlterByModifyColumn(MySqlParser.AlterByModifyColumnContext ctx) { - parser.runIfNotNull( - () -> { - Column column = columnDefinitionListener.getColumn(); - Map typeMapping = new HashMap<>(); - typeMapping.put(column.name(), convertDataType(fromDbzColumn(column))); - changes.add(new AlterColumnTypeEvent(currentTable, typeMapping)); - listeners.remove(columnDefinitionListener); - }, - columnDefinitionListener); - super.exitAlterByModifyColumn(ctx); - } - - @Override - public void exitAlterByRenameColumn(MySqlParser.AlterByRenameColumnContext ctx) { - parser.runIfNotNull( - () -> { - Column column = columnDefinitionListener.getColumn(); - String newColumnName = parser.parseName(ctx.newColumn); - if (newColumnName != null && !column.name().equalsIgnoreCase(newColumnName)) { - Map renameMap = new HashMap<>(); - renameMap.put(column.name(), newColumnName); - changes.add(new RenameColumnEvent(currentTable, renameMap)); - } - listeners.remove(columnDefinitionListener); - }, - columnDefinitionListener); - super.exitAlterByRenameColumn(ctx); - } - - @Override - public void exitTruncateTable(MySqlParser.TruncateTableContext ctx) { - TableId tableId = parser.parseQualifiedTableId(ctx.tableName().fullId()); - changes.add(new TruncateTableEvent(toCdcTableId(tableId))); - super.exitTruncateTable(ctx); - } - - @Override - public void exitDropTable(MySqlParser.DropTableContext ctx) { - ctx.tables() - .tableName() - .forEach( - evt -> { - TableId tableId = parser.parseQualifiedTableId(evt.fullId()); - changes.add(new DropTableEvent(toCdcTableId(tableId))); - }); - super.exitDropTable(ctx); - } - - private org.apache.flink.cdc.common.schema.Column toCdcColumn(Column dbzColumn) { - return org.apache.flink.cdc.common.schema.Column.physicalColumn( - dbzColumn.name(), - convertDataType(fromDbzColumn(dbzColumn)), - dbzColumn.comment(), - dbzColumn.defaultValueExpression().orElse(null)); - } - - private org.apache.flink.cdc.common.event.TableId toCdcTableId(TableId dbzTableId) { - return org.apache.flink.cdc.common.event.TableId.tableId( - dbzTableId.catalog(), dbzTableId.table()); - } -} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/source/parser/CustomColumnDefinitionParserListener.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/source/parser/CustomColumnDefinitionParserListener.java deleted file mode 100644 index 1c7045d2934..00000000000 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/source/parser/CustomColumnDefinitionParserListener.java +++ /dev/null @@ -1,333 +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.connectors.base.source.parser; - -import io.debezium.antlr.AntlrDdlParser; -import io.debezium.antlr.DataTypeResolver; -import io.debezium.connector.mysql.antlr.MySqlAntlrDdlParser; -import io.debezium.connector.mysql.antlr.listener.DefaultValueParserListener; -import io.debezium.ddl.parser.mysql.generated.MySqlParser; -import io.debezium.ddl.parser.mysql.generated.MySqlParserBaseListener; -import io.debezium.relational.Column; -import io.debezium.relational.ColumnEditor; -import io.debezium.relational.TableEditor; -import io.debezium.relational.ddl.DataType; -import io.debezium.util.Strings; -import org.antlr.v4.runtime.tree.ParseTreeListener; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.sql.Types; -import java.util.List; -import java.util.concurrent.atomic.AtomicReference; -import java.util.regex.Pattern; -import java.util.stream.Collectors; - -/** Parser listener that is parsing column definition part of MySQL statements. */ -public class CustomColumnDefinitionParserListener extends MySqlParserBaseListener { - - private static final Logger LOGGER = - LoggerFactory.getLogger(CustomColumnDefinitionParserListener.class); - - private static final Pattern DOT = Pattern.compile("\\."); - private final MySqlAntlrDdlParser parser; - private final DataTypeResolver dataTypeResolver; - private ColumnEditor columnEditor; - private boolean uniqueColumn; - private AtomicReference optionalColumn = new AtomicReference<>(); - private DefaultValueParserListener defaultValueListener; - private final TableEditor tableEditor; - - private final List listeners; - - public CustomColumnDefinitionParserListener( - TableEditor tableEditor, - ColumnEditor columnEditor, - MySqlAntlrDdlParser parser, - List listeners) { - this.tableEditor = tableEditor; - this.columnEditor = columnEditor; - this.parser = parser; - this.dataTypeResolver = parser.dataTypeResolver(); - this.listeners = listeners; - } - - public void setColumnEditor(ColumnEditor columnEditor) { - this.columnEditor = columnEditor; - } - - public ColumnEditor getColumnEditor() { - return columnEditor; - } - - public Column getColumn() { - return columnEditor.create(); - } - - @Override - public void enterColumnDefinition(MySqlParser.ColumnDefinitionContext ctx) { - uniqueColumn = false; - optionalColumn = new AtomicReference<>(); - resolveColumnDataType(ctx.dataType()); - defaultValueListener = new DefaultValueParserListener(columnEditor, optionalColumn); - listeners.add(defaultValueListener); - super.enterColumnDefinition(ctx); - } - - @Override - public void exitColumnDefinition(MySqlParser.ColumnDefinitionContext ctx) { - if (optionalColumn.get() != null) { - columnEditor.optional(optionalColumn.get().booleanValue()); - } - defaultValueListener.exitDefaultValue(false); - listeners.remove(defaultValueListener); - super.exitColumnDefinition(ctx); - } - - @Override - public void enterUniqueKeyColumnConstraint(MySqlParser.UniqueKeyColumnConstraintContext ctx) { - uniqueColumn = true; - super.enterUniqueKeyColumnConstraint(ctx); - } - - @Override - public void enterPrimaryKeyColumnConstraint(MySqlParser.PrimaryKeyColumnConstraintContext ctx) { - // this rule will be parsed only if no primary key is set in a table - // otherwise the statement can't be executed due to multiple primary key error - optionalColumn.set(Boolean.FALSE); - tableEditor.addColumn(columnEditor.create()); - tableEditor.setPrimaryKeyNames(columnEditor.name()); - super.enterPrimaryKeyColumnConstraint(ctx); - } - - @Override - public void enterCommentColumnConstraint(MySqlParser.CommentColumnConstraintContext ctx) { - if (!parser.skipComments()) { - if (ctx.STRING_LITERAL() != null) { - columnEditor.comment(parser.withoutQuotes(ctx.STRING_LITERAL().getText())); - } - } - super.enterCommentColumnConstraint(ctx); - } - - @Override - public void enterNullNotnull(MySqlParser.NullNotnullContext ctx) { - optionalColumn.set(Boolean.valueOf(ctx.NOT() == null)); - super.enterNullNotnull(ctx); - } - - @Override - public void enterAutoIncrementColumnConstraint( - MySqlParser.AutoIncrementColumnConstraintContext ctx) { - columnEditor.autoIncremented(true); - columnEditor.generated(true); - super.enterAutoIncrementColumnConstraint(ctx); - } - - @Override - public void enterSerialDefaultColumnConstraint( - MySqlParser.SerialDefaultColumnConstraintContext ctx) { - serialColumn(); - super.enterSerialDefaultColumnConstraint(ctx); - } - - private void resolveColumnDataType(MySqlParser.DataTypeContext dataTypeContext) { - String charsetName = null; - DataType dataType = dataTypeResolver.resolveDataType(dataTypeContext); - - if (dataTypeContext instanceof MySqlParser.StringDataTypeContext) { - // Same as LongVarcharDataTypeContext but with dimension handling - MySqlParser.StringDataTypeContext stringDataTypeContext = - (MySqlParser.StringDataTypeContext) dataTypeContext; - - if (stringDataTypeContext.lengthOneDimension() != null) { - Integer length = - parseLength( - stringDataTypeContext - .lengthOneDimension() - .decimalLiteral() - .getText()); - columnEditor.length(length); - } - - charsetName = - parser.extractCharset( - stringDataTypeContext.charsetName(), - stringDataTypeContext.collationName()); - } else if (dataTypeContext instanceof MySqlParser.LongVarcharDataTypeContext) { - // Same as StringDataTypeContext but without dimension handling - MySqlParser.LongVarcharDataTypeContext longVarcharTypeContext = - (MySqlParser.LongVarcharDataTypeContext) dataTypeContext; - - charsetName = - parser.extractCharset( - longVarcharTypeContext.charsetName(), - longVarcharTypeContext.collationName()); - } else if (dataTypeContext instanceof MySqlParser.NationalStringDataTypeContext) { - MySqlParser.NationalStringDataTypeContext nationalStringDataTypeContext = - (MySqlParser.NationalStringDataTypeContext) dataTypeContext; - - if (nationalStringDataTypeContext.lengthOneDimension() != null) { - Integer length = - parseLength( - nationalStringDataTypeContext - .lengthOneDimension() - .decimalLiteral() - .getText()); - columnEditor.length(length); - } - } else if (dataTypeContext instanceof MySqlParser.NationalVaryingStringDataTypeContext) { - MySqlParser.NationalVaryingStringDataTypeContext nationalVaryingStringDataTypeContext = - (MySqlParser.NationalVaryingStringDataTypeContext) dataTypeContext; - - if (nationalVaryingStringDataTypeContext.lengthOneDimension() != null) { - Integer length = - parseLength( - nationalVaryingStringDataTypeContext - .lengthOneDimension() - .decimalLiteral() - .getText()); - columnEditor.length(length); - } - } else if (dataTypeContext instanceof MySqlParser.DimensionDataTypeContext) { - MySqlParser.DimensionDataTypeContext dimensionDataTypeContext = - (MySqlParser.DimensionDataTypeContext) dataTypeContext; - - Integer length = null; - Integer scale = null; - if (dimensionDataTypeContext.lengthOneDimension() != null) { - length = - parseLength( - dimensionDataTypeContext - .lengthOneDimension() - .decimalLiteral() - .getText()); - } - - if (dimensionDataTypeContext.lengthTwoDimension() != null) { - List decimalLiterals = - dimensionDataTypeContext.lengthTwoDimension().decimalLiteral(); - length = parseLength(decimalLiterals.get(0).getText()); - scale = Integer.valueOf(decimalLiterals.get(1).getText()); - } - - if (dimensionDataTypeContext.lengthTwoOptionalDimension() != null) { - List decimalLiterals = - dimensionDataTypeContext.lengthTwoOptionalDimension().decimalLiteral(); - if (decimalLiterals.get(0).REAL_LITERAL() != null) { - String[] digits = DOT.split(decimalLiterals.get(0).getText()); - if (Strings.isNullOrEmpty(digits[0]) || Integer.valueOf(digits[0]) == 0) { - // Set default value 10 according mysql engine - length = 10; - } else { - length = parseLength(digits[0]); - } - } else { - length = parseLength(decimalLiterals.get(0).getText()); - } - - if (decimalLiterals.size() > 1) { - scale = Integer.valueOf(decimalLiterals.get(1).getText()); - } - } - if (length != null) { - columnEditor.length(length); - } - if (scale != null) { - columnEditor.scale(scale); - } - } else if (dataTypeContext instanceof MySqlParser.CollectionDataTypeContext) { - MySqlParser.CollectionDataTypeContext collectionDataTypeContext = - (MySqlParser.CollectionDataTypeContext) dataTypeContext; - if (collectionDataTypeContext.charsetName() != null) { - charsetName = collectionDataTypeContext.charsetName().getText(); - } - - if (dataType.name().equalsIgnoreCase("SET")) { - // After DBZ-132, it will always be comma separated - int optionsSize = - collectionDataTypeContext.collectionOptions().collectionOption().size(); - columnEditor.length( - Math.max(0, optionsSize * 2 - 1)); // number of options + number of commas - } else { - columnEditor.length(1); - } - } - - String dataTypeName = dataType.name().toUpperCase(); - - if (dataTypeName.equals("ENUM") || dataTypeName.equals("SET")) { - // type expression has to be set, because the value converter needs to know the enum or - // set options - MySqlParser.CollectionDataTypeContext collectionDataTypeContext = - (MySqlParser.CollectionDataTypeContext) dataTypeContext; - - List collectionOptions = - collectionDataTypeContext.collectionOptions().collectionOption().stream() - .map(AntlrDdlParser::getText) - .collect(Collectors.toList()); - - columnEditor.type(dataTypeName); - columnEditor.enumValues(collectionOptions); - } else if (dataTypeName.equals("SERIAL")) { - // SERIAL is an alias for BIGINT UNSIGNED NOT NULL AUTO_INCREMENT UNIQUE - columnEditor.type("BIGINT UNSIGNED"); - serialColumn(); - } else { - columnEditor.type(dataTypeName); - } - - int jdbcDataType = dataType.jdbcType(); - columnEditor.jdbcType(jdbcDataType); - - if (columnEditor.length() == -1) { - columnEditor.length((int) dataType.length()); - } - if (!columnEditor.scale().isPresent() && dataType.scale() != Column.UNSET_INT_VALUE) { - columnEditor.scale(dataType.scale()); - } - if (Types.NCHAR == jdbcDataType || Types.NVARCHAR == jdbcDataType) { - // NCHAR and NVARCHAR columns always uses utf8 as charset - columnEditor.charsetName("utf8"); - } else { - columnEditor.charsetName(charsetName); - } - } - - private Integer parseLength(String lengthStr) { - Long length = Long.parseLong(lengthStr); - if (length > Integer.MAX_VALUE) { - LOGGER.warn( - "The length '{}' of the column `{}` is too large to be supported, truncating it to '{}'", - length, - columnEditor.name(), - Integer.MAX_VALUE); - length = (long) Integer.MAX_VALUE; - } - return length.intValue(); - } - - private void serialColumn() { - if (optionalColumn.get() == null) { - optionalColumn.set(Boolean.FALSE); - } - uniqueColumn = true; - columnEditor.autoIncremented(true); - columnEditor.generated(true); - } -} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/source/parser/CustomMySqlAntlrDdlParser.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/source/parser/CustomMySqlAntlrDdlParser.java deleted file mode 100644 index bf6564ded28..00000000000 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/source/parser/CustomMySqlAntlrDdlParser.java +++ /dev/null @@ -1,288 +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.connectors.base.source.parser; - -import org.apache.flink.cdc.common.event.SchemaChangeEvent; - -import io.debezium.antlr.AntlrDdlParserListener; -import io.debezium.antlr.DataTypeResolver; -import io.debezium.connector.mysql.antlr.MySqlAntlrDdlParser; -import io.debezium.ddl.parser.mysql.generated.MySqlParser; - -import java.sql.Types; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.LinkedList; -import java.util.List; - -/** A ddl parser that will use custom listener. */ -public class CustomMySqlAntlrDdlParser extends MySqlAntlrDdlParser { - - private final LinkedList parsedEvents; - - public CustomMySqlAntlrDdlParser() { - super(); - this.parsedEvents = new LinkedList<>(); - } - - // Overriding this method because the BIT type requires default length dimension of 1. - // Remove it when debezium fixed this issue. - @Override - protected DataTypeResolver initializeDataTypeResolver() { - DataTypeResolver.Builder dataTypeResolverBuilder = new DataTypeResolver.Builder(); - - dataTypeResolverBuilder.registerDataTypes( - MySqlParser.StringDataTypeContext.class.getCanonicalName(), - Arrays.asList( - new DataTypeResolver.DataTypeEntry(Types.CHAR, MySqlParser.CHAR), - new DataTypeResolver.DataTypeEntry( - Types.VARCHAR, MySqlParser.CHAR, MySqlParser.VARYING), - new DataTypeResolver.DataTypeEntry(Types.VARCHAR, MySqlParser.VARCHAR), - new DataTypeResolver.DataTypeEntry(Types.VARCHAR, MySqlParser.TINYTEXT), - new DataTypeResolver.DataTypeEntry(Types.VARCHAR, MySqlParser.TEXT), - new DataTypeResolver.DataTypeEntry(Types.VARCHAR, MySqlParser.MEDIUMTEXT), - new DataTypeResolver.DataTypeEntry(Types.VARCHAR, MySqlParser.LONGTEXT), - new DataTypeResolver.DataTypeEntry(Types.NCHAR, MySqlParser.NCHAR), - new DataTypeResolver.DataTypeEntry( - Types.NVARCHAR, MySqlParser.NCHAR, MySqlParser.VARYING), - new DataTypeResolver.DataTypeEntry(Types.NVARCHAR, MySqlParser.NVARCHAR), - new DataTypeResolver.DataTypeEntry( - Types.CHAR, MySqlParser.CHAR, MySqlParser.BINARY), - new DataTypeResolver.DataTypeEntry( - Types.VARCHAR, MySqlParser.VARCHAR, MySqlParser.BINARY), - new DataTypeResolver.DataTypeEntry( - Types.VARCHAR, MySqlParser.TINYTEXT, MySqlParser.BINARY), - new DataTypeResolver.DataTypeEntry( - Types.VARCHAR, MySqlParser.TEXT, MySqlParser.BINARY), - new DataTypeResolver.DataTypeEntry( - Types.VARCHAR, MySqlParser.MEDIUMTEXT, MySqlParser.BINARY), - new DataTypeResolver.DataTypeEntry( - Types.VARCHAR, MySqlParser.LONGTEXT, MySqlParser.BINARY), - new DataTypeResolver.DataTypeEntry( - Types.NCHAR, MySqlParser.NCHAR, MySqlParser.BINARY), - new DataTypeResolver.DataTypeEntry( - Types.NVARCHAR, MySqlParser.NVARCHAR, MySqlParser.BINARY), - new DataTypeResolver.DataTypeEntry(Types.CHAR, MySqlParser.CHARACTER), - new DataTypeResolver.DataTypeEntry( - Types.VARCHAR, MySqlParser.CHARACTER, MySqlParser.VARYING))); - dataTypeResolverBuilder.registerDataTypes( - MySqlParser.NationalStringDataTypeContext.class.getCanonicalName(), - Arrays.asList( - new DataTypeResolver.DataTypeEntry( - Types.NVARCHAR, MySqlParser.NATIONAL, MySqlParser.VARCHAR) - .setSuffixTokens(MySqlParser.BINARY), - new DataTypeResolver.DataTypeEntry( - Types.NCHAR, MySqlParser.NATIONAL, MySqlParser.CHARACTER) - .setSuffixTokens(MySqlParser.BINARY), - new DataTypeResolver.DataTypeEntry( - Types.NVARCHAR, MySqlParser.NCHAR, MySqlParser.VARCHAR) - .setSuffixTokens(MySqlParser.BINARY))); - dataTypeResolverBuilder.registerDataTypes( - MySqlParser.NationalVaryingStringDataTypeContext.class.getCanonicalName(), - Arrays.asList( - new DataTypeResolver.DataTypeEntry( - Types.NVARCHAR, - MySqlParser.NATIONAL, - MySqlParser.CHAR, - MySqlParser.VARYING), - new DataTypeResolver.DataTypeEntry( - Types.NVARCHAR, - MySqlParser.NATIONAL, - MySqlParser.CHARACTER, - MySqlParser.VARYING))); - dataTypeResolverBuilder.registerDataTypes( - MySqlParser.DimensionDataTypeContext.class.getCanonicalName(), - Arrays.asList( - new DataTypeResolver.DataTypeEntry(Types.SMALLINT, MySqlParser.TINYINT) - .setSuffixTokens( - MySqlParser.SIGNED, - MySqlParser.UNSIGNED, - MySqlParser.ZEROFILL), - new DataTypeResolver.DataTypeEntry(Types.SMALLINT, MySqlParser.INT1) - .setSuffixTokens( - MySqlParser.SIGNED, - MySqlParser.UNSIGNED, - MySqlParser.ZEROFILL), - new DataTypeResolver.DataTypeEntry(Types.SMALLINT, MySqlParser.SMALLINT) - .setSuffixTokens( - MySqlParser.SIGNED, - MySqlParser.UNSIGNED, - MySqlParser.ZEROFILL), - new DataTypeResolver.DataTypeEntry(Types.SMALLINT, MySqlParser.INT2) - .setSuffixTokens( - MySqlParser.SIGNED, - MySqlParser.UNSIGNED, - MySqlParser.ZEROFILL), - new DataTypeResolver.DataTypeEntry(Types.INTEGER, MySqlParser.MEDIUMINT) - .setSuffixTokens( - MySqlParser.SIGNED, - MySqlParser.UNSIGNED, - MySqlParser.ZEROFILL), - new DataTypeResolver.DataTypeEntry(Types.INTEGER, MySqlParser.INT3) - .setSuffixTokens( - MySqlParser.SIGNED, - MySqlParser.UNSIGNED, - MySqlParser.ZEROFILL), - new DataTypeResolver.DataTypeEntry(Types.INTEGER, MySqlParser.MIDDLEINT) - .setSuffixTokens( - MySqlParser.SIGNED, - MySqlParser.UNSIGNED, - MySqlParser.ZEROFILL), - new DataTypeResolver.DataTypeEntry(Types.INTEGER, MySqlParser.INT) - .setSuffixTokens( - MySqlParser.SIGNED, - MySqlParser.UNSIGNED, - MySqlParser.ZEROFILL), - new DataTypeResolver.DataTypeEntry(Types.INTEGER, MySqlParser.INTEGER) - .setSuffixTokens( - MySqlParser.SIGNED, - MySqlParser.UNSIGNED, - MySqlParser.ZEROFILL), - new DataTypeResolver.DataTypeEntry(Types.INTEGER, MySqlParser.INT4) - .setSuffixTokens( - MySqlParser.SIGNED, - MySqlParser.UNSIGNED, - MySqlParser.ZEROFILL), - new DataTypeResolver.DataTypeEntry(Types.BIGINT, MySqlParser.BIGINT) - .setSuffixTokens( - MySqlParser.SIGNED, - MySqlParser.UNSIGNED, - MySqlParser.ZEROFILL), - new DataTypeResolver.DataTypeEntry(Types.BIGINT, MySqlParser.INT8) - .setSuffixTokens( - MySqlParser.SIGNED, - MySqlParser.UNSIGNED, - MySqlParser.ZEROFILL), - new DataTypeResolver.DataTypeEntry(Types.REAL, MySqlParser.REAL) - .setSuffixTokens( - MySqlParser.SIGNED, - MySqlParser.UNSIGNED, - MySqlParser.ZEROFILL), - new DataTypeResolver.DataTypeEntry(Types.DOUBLE, MySqlParser.DOUBLE) - .setSuffixTokens( - MySqlParser.PRECISION, - MySqlParser.SIGNED, - MySqlParser.UNSIGNED, - MySqlParser.ZEROFILL), - new DataTypeResolver.DataTypeEntry(Types.DOUBLE, MySqlParser.FLOAT8) - .setSuffixTokens( - MySqlParser.PRECISION, - MySqlParser.SIGNED, - MySqlParser.UNSIGNED, - MySqlParser.ZEROFILL), - new DataTypeResolver.DataTypeEntry(Types.FLOAT, MySqlParser.FLOAT) - .setSuffixTokens( - MySqlParser.SIGNED, - MySqlParser.UNSIGNED, - MySqlParser.ZEROFILL), - new DataTypeResolver.DataTypeEntry(Types.FLOAT, MySqlParser.FLOAT4) - .setSuffixTokens( - MySqlParser.SIGNED, - MySqlParser.UNSIGNED, - MySqlParser.ZEROFILL), - new DataTypeResolver.DataTypeEntry(Types.DECIMAL, MySqlParser.DECIMAL) - .setSuffixTokens( - MySqlParser.SIGNED, - MySqlParser.UNSIGNED, - MySqlParser.ZEROFILL) - .setDefaultLengthScaleDimension(10, 0), - new DataTypeResolver.DataTypeEntry(Types.DECIMAL, MySqlParser.DEC) - .setSuffixTokens( - MySqlParser.SIGNED, - MySqlParser.UNSIGNED, - MySqlParser.ZEROFILL) - .setDefaultLengthScaleDimension(10, 0), - new DataTypeResolver.DataTypeEntry(Types.DECIMAL, MySqlParser.FIXED) - .setSuffixTokens( - MySqlParser.SIGNED, - MySqlParser.UNSIGNED, - MySqlParser.ZEROFILL) - .setDefaultLengthScaleDimension(10, 0), - new DataTypeResolver.DataTypeEntry(Types.NUMERIC, MySqlParser.NUMERIC) - .setSuffixTokens( - MySqlParser.SIGNED, - MySqlParser.UNSIGNED, - MySqlParser.ZEROFILL) - .setDefaultLengthScaleDimension(10, 0), - new DataTypeResolver.DataTypeEntry(Types.BIT, MySqlParser.BIT) - .setDefaultLengthDimension(1), - new DataTypeResolver.DataTypeEntry(Types.TIME, MySqlParser.TIME), - new DataTypeResolver.DataTypeEntry( - Types.TIMESTAMP_WITH_TIMEZONE, MySqlParser.TIMESTAMP), - new DataTypeResolver.DataTypeEntry(Types.TIMESTAMP, MySqlParser.DATETIME), - new DataTypeResolver.DataTypeEntry(Types.BINARY, MySqlParser.BINARY), - new DataTypeResolver.DataTypeEntry(Types.VARBINARY, MySqlParser.VARBINARY), - new DataTypeResolver.DataTypeEntry(Types.BLOB, MySqlParser.BLOB), - new DataTypeResolver.DataTypeEntry(Types.INTEGER, MySqlParser.YEAR))); - dataTypeResolverBuilder.registerDataTypes( - MySqlParser.SimpleDataTypeContext.class.getCanonicalName(), - Arrays.asList( - new DataTypeResolver.DataTypeEntry(Types.DATE, MySqlParser.DATE), - new DataTypeResolver.DataTypeEntry(Types.BLOB, MySqlParser.TINYBLOB), - new DataTypeResolver.DataTypeEntry(Types.BLOB, MySqlParser.MEDIUMBLOB), - new DataTypeResolver.DataTypeEntry(Types.BLOB, MySqlParser.LONGBLOB), - new DataTypeResolver.DataTypeEntry(Types.BOOLEAN, MySqlParser.BOOL), - new DataTypeResolver.DataTypeEntry(Types.BOOLEAN, MySqlParser.BOOLEAN), - new DataTypeResolver.DataTypeEntry(Types.BIGINT, MySqlParser.SERIAL))); - dataTypeResolverBuilder.registerDataTypes( - MySqlParser.CollectionDataTypeContext.class.getCanonicalName(), - Arrays.asList( - new DataTypeResolver.DataTypeEntry(Types.CHAR, MySqlParser.ENUM) - .setSuffixTokens(MySqlParser.BINARY), - new DataTypeResolver.DataTypeEntry(Types.CHAR, MySqlParser.SET) - .setSuffixTokens(MySqlParser.BINARY))); - dataTypeResolverBuilder.registerDataTypes( - MySqlParser.SpatialDataTypeContext.class.getCanonicalName(), - Arrays.asList( - new DataTypeResolver.DataTypeEntry( - Types.OTHER, MySqlParser.GEOMETRYCOLLECTION), - new DataTypeResolver.DataTypeEntry(Types.OTHER, MySqlParser.GEOMCOLLECTION), - new DataTypeResolver.DataTypeEntry(Types.OTHER, MySqlParser.LINESTRING), - new DataTypeResolver.DataTypeEntry( - Types.OTHER, MySqlParser.MULTILINESTRING), - new DataTypeResolver.DataTypeEntry(Types.OTHER, MySqlParser.MULTIPOINT), - new DataTypeResolver.DataTypeEntry(Types.OTHER, MySqlParser.MULTIPOLYGON), - new DataTypeResolver.DataTypeEntry(Types.OTHER, MySqlParser.POINT), - new DataTypeResolver.DataTypeEntry(Types.OTHER, MySqlParser.POLYGON), - new DataTypeResolver.DataTypeEntry(Types.OTHER, MySqlParser.JSON), - new DataTypeResolver.DataTypeEntry(Types.OTHER, MySqlParser.GEOMETRY))); - dataTypeResolverBuilder.registerDataTypes( - MySqlParser.LongVarbinaryDataTypeContext.class.getCanonicalName(), - Arrays.asList( - new DataTypeResolver.DataTypeEntry(Types.BLOB, MySqlParser.LONG) - .setSuffixTokens(MySqlParser.VARBINARY))); - dataTypeResolverBuilder.registerDataTypes( - MySqlParser.LongVarcharDataTypeContext.class.getCanonicalName(), - Arrays.asList( - new DataTypeResolver.DataTypeEntry(Types.VARCHAR, MySqlParser.LONG) - .setSuffixTokens(MySqlParser.VARCHAR))); - - return dataTypeResolverBuilder.build(); - } - - @Override - protected AntlrDdlParserListener createParseTreeWalkerListener() { - return new CustomMySqlAntlrDdlParserListener(this, parsedEvents); - } - - public List getAndClearParsedEvents() { - List result = new ArrayList<>(parsedEvents); - parsedEvents.clear(); - return result; - } -} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/source/parser/CustomMySqlAntlrDdlParserListener.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/source/parser/CustomMySqlAntlrDdlParserListener.java deleted file mode 100644 index 076ac06cd76..00000000000 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/source/parser/CustomMySqlAntlrDdlParserListener.java +++ /dev/null @@ -1,143 +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.connectors.base.source.parser; - -import org.apache.flink.cdc.common.event.SchemaChangeEvent; - -import io.debezium.antlr.AntlrDdlParserListener; -import io.debezium.antlr.ProxyParseTreeListenerUtil; -import io.debezium.connector.mysql.antlr.MySqlAntlrDdlParser; -import io.debezium.connector.mysql.antlr.listener.AlterViewParserListener; -import io.debezium.connector.mysql.antlr.listener.CreateAndAlterDatabaseParserListener; -import io.debezium.connector.mysql.antlr.listener.CreateTableParserListener; -import io.debezium.connector.mysql.antlr.listener.CreateUniqueIndexParserListener; -import io.debezium.connector.mysql.antlr.listener.CreateViewParserListener; -import io.debezium.connector.mysql.antlr.listener.DropDatabaseParserListener; -import io.debezium.connector.mysql.antlr.listener.DropTableParserListener; -import io.debezium.connector.mysql.antlr.listener.DropViewParserListener; -import io.debezium.connector.mysql.antlr.listener.MySqlAntlrDdlParserListener; -import io.debezium.connector.mysql.antlr.listener.RenameTableParserListener; -import io.debezium.connector.mysql.antlr.listener.SetStatementParserListener; -import io.debezium.connector.mysql.antlr.listener.TruncateTableParserListener; -import io.debezium.connector.mysql.antlr.listener.UseStatementParserListener; -import io.debezium.ddl.parser.mysql.generated.MySqlParser; -import io.debezium.ddl.parser.mysql.generated.MySqlParserBaseListener; -import io.debezium.text.ParsingException; -import org.antlr.v4.runtime.ParserRuleContext; -import org.antlr.v4.runtime.tree.ErrorNode; -import org.antlr.v4.runtime.tree.ParseTreeListener; -import org.antlr.v4.runtime.tree.TerminalNode; - -import java.util.ArrayList; -import java.util.Collection; -import java.util.LinkedList; -import java.util.List; -import java.util.concurrent.CopyOnWriteArrayList; - -/** - * Copied from {@link MySqlAntlrDdlParserListener} in Debezium 1.9.8.final. - * - *

This listener's constructor will use some modified listener. - */ -public class CustomMySqlAntlrDdlParserListener extends MySqlParserBaseListener - implements AntlrDdlParserListener { - - /** Collection of listeners for delegation of events. */ - private final List listeners = new CopyOnWriteArrayList<>(); - - /** Flag for skipping phase. */ - private boolean skipNodes; - - /** - * Count of skipped nodes. Each enter event during skipping phase will increase the counter and - * each exit event will decrease it. When counter will be decreased to 0, the skipping phase - * will end. - */ - private int skippedNodesCount = 0; - - /** Collection of catched exceptions. */ - private final Collection errors = new ArrayList<>(); - - public CustomMySqlAntlrDdlParserListener( - MySqlAntlrDdlParser parser, LinkedList parsedEvents) { - // initialize listeners - listeners.add(new CreateAndAlterDatabaseParserListener(parser)); - listeners.add(new DropDatabaseParserListener(parser)); - listeners.add(new CreateTableParserListener(parser, listeners)); - listeners.add(new CustomAlterTableParserListener(parser, listeners, parsedEvents)); - listeners.add(new DropTableParserListener(parser)); - listeners.add(new RenameTableParserListener(parser)); - listeners.add(new TruncateTableParserListener(parser)); - listeners.add(new CreateViewParserListener(parser, listeners)); - listeners.add(new AlterViewParserListener(parser, listeners)); - listeners.add(new DropViewParserListener(parser)); - listeners.add(new CreateUniqueIndexParserListener(parser)); - listeners.add(new SetStatementParserListener(parser)); - listeners.add(new UseStatementParserListener(parser)); - } - - /** - * Returns all caught errors during tree walk. - * - * @return list of Parsing exceptions - */ - @Override - public Collection getErrors() { - return errors; - } - - @Override - public void enterEveryRule(ParserRuleContext ctx) { - if (skipNodes) { - skippedNodesCount++; - } else { - ProxyParseTreeListenerUtil.delegateEnterRule(ctx, listeners, errors); - } - } - - @Override - public void exitEveryRule(ParserRuleContext ctx) { - if (skipNodes) { - if (skippedNodesCount == 0) { - // back in the node where skipping started - skipNodes = false; - } else { - // going up in a tree, means decreasing a number of skipped nodes - skippedNodesCount--; - } - } else { - ProxyParseTreeListenerUtil.delegateExitRule(ctx, listeners, errors); - } - } - - @Override - public void visitErrorNode(ErrorNode node) { - ProxyParseTreeListenerUtil.visitErrorNode(node, listeners, errors); - } - - @Override - public void visitTerminal(TerminalNode node) { - ProxyParseTreeListenerUtil.visitTerminal(node, listeners, errors); - } - - @Override - public void enterRoutineBody(MySqlParser.RoutineBodyContext ctx) { - // this is a grammar rule for BEGIN ... END part of statements. Skip it. - skipNodes = true; - } -} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/testutils/MySqlContainer.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/testutils/MySqlContainer.java deleted file mode 100644 index 445ed69e83f..00000000000 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/testutils/MySqlContainer.java +++ /dev/null @@ -1,176 +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.connectors.base.testutils; - -import org.testcontainers.containers.ContainerLaunchException; -import org.testcontainers.containers.JdbcDatabaseContainer; -import org.testcontainers.utility.DockerImageName; - -import java.util.HashSet; -import java.util.Set; - -/** - * Docker container for MySQL. The difference between this class and {@link - * org.testcontainers.containers.MySQLContainer} is that TC MySQLContainer has problems when - * overriding mysql conf file, i.e. my.cnf. - */ -public class MySqlContainer extends JdbcDatabaseContainer { - - public static final String IMAGE = "mysql"; - public static final Integer MYSQL_PORT = 3306; - - private static final String MY_CNF_CONFIG_OVERRIDE_PARAM_NAME = "MY_CNF"; - private static final String SETUP_SQL_PARAM_NAME = "SETUP_SQL"; - private static final String MYSQL_ROOT_USER = "root"; - - private String databaseName = "test"; - private String username = "test"; - private String password = "test"; - - public MySqlContainer() { - this(MySqlVersion.V5_7); - } - - public MySqlContainer(MySqlVersion version) { - super(DockerImageName.parse(IMAGE + ":" + version.getVersion())); - addExposedPort(MYSQL_PORT); - } - - @Override - protected Set getLivenessCheckPorts() { - return new HashSet<>(getMappedPort(MYSQL_PORT)); - } - - @Override - protected void configure() { - optionallyMapResourceParameterAsVolume( - MY_CNF_CONFIG_OVERRIDE_PARAM_NAME, "/etc/mysql/", "mysql-default-conf"); - - if (parameters.containsKey(SETUP_SQL_PARAM_NAME)) { - optionallyMapResourceParameterAsVolume( - SETUP_SQL_PARAM_NAME, "/docker-entrypoint-initdb.d/", "N/A"); - } - - addEnv("MYSQL_DATABASE", databaseName); - addEnv("MYSQL_USER", username); - if (password != null && !password.isEmpty()) { - addEnv("MYSQL_PASSWORD", password); - addEnv("MYSQL_ROOT_PASSWORD", password); - } else if (MYSQL_ROOT_USER.equalsIgnoreCase(username)) { - addEnv("MYSQL_ALLOW_EMPTY_PASSWORD", "yes"); - } else { - throw new ContainerLaunchException( - "Empty password can be used only with the root user"); - } - setStartupAttempts(3); - } - - @Override - public String getDriverClassName() { - try { - Class.forName("com.mysql.cj.jdbc.Driver"); - return "com.mysql.cj.jdbc.Driver"; - } catch (ClassNotFoundException e) { - return "com.mysql.jdbc.Driver"; - } - } - - public String getJdbcUrl(String databaseName) { - String additionalUrlParams = constructUrlParameters("?", "&"); - return "jdbc:mysql://" - + getHost() - + ":" - + getDatabasePort() - + "/" - + databaseName - + additionalUrlParams; - } - - @Override - public String getJdbcUrl() { - return getJdbcUrl(databaseName); - } - - public int getDatabasePort() { - return getMappedPort(MYSQL_PORT); - } - - @Override - protected String constructUrlForConnection(String queryString) { - String url = super.constructUrlForConnection(queryString); - - if (!url.contains("useSSL=")) { - String separator = url.contains("?") ? "&" : "?"; - url = url + separator + "useSSL=false"; - } - - if (!url.contains("allowPublicKeyRetrieval=")) { - url = url + "&allowPublicKeyRetrieval=true"; - } - - return url; - } - - @Override - public String getDatabaseName() { - return databaseName; - } - - @Override - public String getUsername() { - return username; - } - - @Override - public String getPassword() { - return password; - } - - @Override - protected String getTestQueryString() { - return "SELECT 1"; - } - - public MySqlContainer withConfigurationOverride(String s) { - parameters.put(MY_CNF_CONFIG_OVERRIDE_PARAM_NAME, s); - return this; - } - - public MySqlContainer withSetupSQL(String sqlPath) { - parameters.put(SETUP_SQL_PARAM_NAME, sqlPath); - return this; - } - - @Override - public MySqlContainer withDatabaseName(final String databaseName) { - this.databaseName = databaseName; - return this; - } - - @Override - public MySqlContainer withUsername(final String username) { - this.username = username; - return this; - } - - @Override - public MySqlContainer withPassword(final String password) { - this.password = password; - return this; - } -} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/testutils/MySqlVersion.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/testutils/MySqlVersion.java deleted file mode 100644 index 0b1ff47a510..00000000000 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/testutils/MySqlVersion.java +++ /dev/null @@ -1,41 +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.connectors.base.testutils; - -/** MySql version enum. */ -public enum MySqlVersion { - V5_5("5.5"), - V5_6("5.6"), - V5_7("5.7"), - V8_0("8.0"); - - private final String version; - - MySqlVersion(String version) { - this.version = version; - } - - public String getVersion() { - return version; - } - - @Override - public String toString() { - return "MySqlVersion{" + "version='" + version + '\'' + '}'; - } -} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/testutils/UniqueDatabase.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/testutils/UniqueDatabase.java deleted file mode 100644 index 1b44004362c..00000000000 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/test/java/org/apache/flink/cdc/connectors/base/testutils/UniqueDatabase.java +++ /dev/null @@ -1,152 +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.connectors.base.testutils; - -import java.net.URL; -import java.nio.file.Files; -import java.nio.file.Paths; -import java.sql.Connection; -import java.sql.DriverManager; -import java.sql.SQLException; -import java.sql.Statement; -import java.util.Arrays; -import java.util.List; -import java.util.Random; -import java.util.regex.Matcher; -import java.util.regex.Pattern; -import java.util.stream.Collectors; -import java.util.stream.Stream; - -import static org.junit.Assert.assertNotNull; - -/** - * Create and populate a unique instance of a MySQL database for each run of JUnit test. A user of - * class needs to provide a logical name for Debezium and database name. It is expected that there - * is a init file in src/test/resources/ddl/<database_name>.sql. The database - * name is enriched with a unique suffix that guarantees complete isolation between runs - * <database_name>_<suffix> - * - *

This class is inspired from Debezium project. - */ -public class UniqueDatabase { - - private static final String[] CREATE_DATABASE_DDL = - new String[] {"CREATE DATABASE $DBNAME$;", "USE $DBNAME$;"}; - private static final Pattern COMMENT_PATTERN = Pattern.compile("^(.*)--.*$"); - - private final MySqlContainer container; - private final String databaseName; - private final String templateName; - private final String username; - private final String password; - - public UniqueDatabase( - MySqlContainer container, String databaseName, String username, String password) { - this( - container, - databaseName, - Integer.toUnsignedString(new Random().nextInt(), 36), - username, - password); - } - - private UniqueDatabase( - MySqlContainer container, - String databaseName, - final String identifier, - String username, - String password) { - this.container = container; - this.databaseName = databaseName + "_" + identifier; - this.templateName = databaseName; - this.username = username; - this.password = password; - } - - public String getHost() { - return container.getHost(); - } - - public int getDatabasePort() { - return container.getDatabasePort(); - } - - public String getDatabaseName() { - return databaseName; - } - - public String getUsername() { - return username; - } - - public String getPassword() { - return password; - } - - /** @return Fully qualified table name <databaseName>.<tableName> */ - public String qualifiedTableName(final String tableName) { - return String.format("%s.%s", databaseName, tableName); - } - - /** Creates the database and populates it with initialization SQL script. */ - public void createAndInitialize() { - final String ddlFile = String.format("ddl/%s.sql", templateName); - final URL ddlTestFile = UniqueDatabase.class.getClassLoader().getResource(ddlFile); - assertNotNull("Cannot locate " + ddlFile, ddlTestFile); - try { - try (Connection connection = - DriverManager.getConnection( - container.getJdbcUrl(), username, password); - Statement statement = connection.createStatement()) { - final List statements = - Arrays.stream( - Stream.concat( - Arrays.stream(CREATE_DATABASE_DDL), - Files.readAllLines( - Paths.get(ddlTestFile.toURI())) - .stream()) - .map(String::trim) - .filter(x -> !x.startsWith("--") && !x.isEmpty()) - .map( - x -> { - final Matcher m = - COMMENT_PATTERN.matcher(x); - return m.matches() ? m.group(1) : x; - }) - .map(this::convertSQL) - .collect(Collectors.joining("\n")) - .split(";")) - .map(x -> x.replace("$$", ";")) - .collect(Collectors.toList()); - for (String stmt : statements) { - statement.execute(stmt); - } - } - } catch (final Exception e) { - throw new IllegalStateException(e); - } - } - - public Connection getJdbcConnection() throws SQLException { - return DriverManager.getConnection(container.getJdbcUrl(databaseName), username, password); - } - - private String convertSQL(final String sql) { - return sql.replace("$DBNAME$", databaseName); - } -}